refactor: rename BundleStateWithReceipts to BlockExecutionOutcome (#8730)

This commit is contained in:
Thomas Coratger
2024-06-11 18:06:49 +02:00
committed by GitHub
parent 95719da049
commit a5d825edb3
43 changed files with 443 additions and 409 deletions

View File

@ -29,9 +29,8 @@ use reth_primitives::{
SealedBlock, SealedBlockWithSenders, Transaction, TransactionSigned, TxEip4844, B256, U256,
};
use reth_provider::{
providers::BlockchainProvider, BlockHashReader, BlockReader, BlockWriter,
BundleStateWithReceipts, ChainSpecProvider, ProviderFactory, StageCheckpointReader,
StateProviderFactory,
providers::BlockchainProvider, BlockHashReader, BlockReader, BlockWriter, ChainSpecProvider,
ExecutionOutcome, ProviderFactory, StageCheckpointReader, StateProviderFactory,
};
use reth_revm::database::StateProviderDatabase;
use reth_rpc_types::engine::{BlobsBundleV1, PayloadAttributes};
@ -273,17 +272,17 @@ impl Command {
let BlockExecutionOutput { state, receipts, requests, .. } =
executor.execute((&block_with_senders.clone().unseal(), U256::MAX).into())?;
let state = BundleStateWithReceipts::new(
let execution_outcome = ExecutionOutcome::new(
state,
receipts.into(),
block.number,
vec![requests.into()],
);
debug!(target: "reth::cli", ?state, "Executed block");
debug!(target: "reth::cli", ?execution_outcome, "Executed block");
let hashed_state = state.hash_state_slow();
let (state_root, trie_updates) = state
let hashed_post_state = execution_outcome.hash_state_slow();
let (state_root, trie_updates) = execution_outcome
.hash_state_slow()
.state_root_with_updates(provider_factory.provider()?.tx_ref())?;
@ -299,8 +298,8 @@ impl Command {
let provider_rw = provider_factory.provider_rw()?;
provider_rw.append_blocks_with_state(
Vec::from([block_with_senders]),
state,
hashed_state,
execution_outcome,
hashed_post_state,
trie_updates,
None,
)?;

View File

@ -17,7 +17,7 @@ use reth_network::NetworkHandle;
use reth_network_api::NetworkInfo;
use reth_primitives::BlockHashOrNumber;
use reth_provider::{
AccountExtReader, BundleStateWithReceipts, ChainSpecProvider, HashingWriter, HeaderProvider,
AccountExtReader, ChainSpecProvider, ExecutionOutcome, HashingWriter, HeaderProvider,
LatestStateProviderRef, OriginalValuesKnown, ProviderFactory, StageCheckpointReader,
StateWriter, StaticFileProviderFactory, StorageReader,
};
@ -147,16 +147,12 @@ impl Command {
)
.into(),
)?;
let block_state = BundleStateWithReceipts::new(
state,
receipts.into(),
block.number,
vec![requests.into()],
);
let execution_outcome =
ExecutionOutcome::new(state, receipts.into(), block.number, vec![requests.into()]);
// Unpacked `BundleState::state_root_slow` function
let (in_memory_state_root, in_memory_updates) =
block_state.hash_state_slow().state_root_with_updates(provider.tx_ref())?;
execution_outcome.hash_state_slow().state_root_with_updates(provider.tx_ref())?;
if in_memory_state_root == block.state_root {
info!(target: "reth::cli", state_root = ?in_memory_state_root, "Computed in-memory state root matches");
@ -173,7 +169,7 @@ impl Command {
.map_err(|_| BlockValidationError::SenderRecoveryError)?,
None,
)?;
block_state.write_to_storage(provider_rw.tx_ref(), None, OriginalValuesKnown::No)?;
execution_outcome.write_to_storage(provider_rw.tx_ref(), None, OriginalValuesKnown::No)?;
let storage_lists = provider_rw.changed_storages_with_range(block.number..=block.number)?;
let storages = provider_rw.plain_state_storages(storage_lists)?;
provider_rw.insert_storage_for_hashing(storages)?;

View File

@ -13,8 +13,8 @@ use reth_node_core::version::SHORT_VERSION;
use reth_optimism_primitives::bedrock_import::is_dup_tx;
use reth_primitives::{Receipts, StaticFileSegment};
use reth_provider::{
BundleStateWithReceipts, OriginalValuesKnown, ProviderFactory, StageCheckpointReader,
StateWriter, StaticFileProviderFactory, StaticFileWriter, StatsReader,
ExecutionOutcome, OriginalValuesKnown, ProviderFactory, StageCheckpointReader, StateWriter,
StaticFileProviderFactory, StaticFileWriter, StatsReader,
};
use reth_stages::StageId;
use std::path::{Path, PathBuf};
@ -131,20 +131,16 @@ where
);
// We're reusing receipt writing code internal to
// `BundleStateWithReceipts::write_to_storage`, so we just use a default empty
// `ExecutionOutcome::write_to_storage`, so we just use a default empty
// `BundleState`.
let bundled_state = BundleStateWithReceipts::new(
Default::default(),
receipts,
first_block,
Default::default(),
);
let execution_outcome =
ExecutionOutcome::new(Default::default(), receipts, first_block, Default::default());
let static_file_producer =
static_file_provider.get_writer(first_block, StaticFileSegment::Receipts)?;
// finally, write the receipts
bundled_state.write_to_storage::<DB::TXMut>(
execution_outcome.write_to_storage::<DB::TXMut>(
&tx,
Some(static_file_producer),
OriginalValuesKnown::Yes,

View File

@ -3,7 +3,7 @@
use crate::{
metrics::{MakeCanonicalAction, MakeCanonicalDurationsRecorder, TreeMetrics},
state::{BlockchainId, TreeState},
AppendableChain, BlockIndices, BlockchainTreeConfig, BundleStateData, TreeExternals,
AppendableChain, BlockIndices, BlockchainTreeConfig, ExecutionData, TreeExternals,
};
use reth_blockchain_tree_api::{
error::{BlockchainTreeError, CanonicalError, InsertBlockError, InsertBlockErrorKind},
@ -18,10 +18,10 @@ use reth_primitives::{
SealedBlockWithSenders, SealedHeader, StaticFileSegment, B256, U256,
};
use reth_provider::{
BlockExecutionWriter, BlockNumReader, BlockWriter, BundleStateWithReceipts,
CanonStateNotification, CanonStateNotificationSender, CanonStateNotifications, Chain,
ChainSpecProvider, ChainSplit, ChainSplitTarget, DisplayBlocksChain, HeaderProvider,
ProviderError, StaticFileProviderFactory,
BlockExecutionWriter, BlockNumReader, BlockWriter, CanonStateNotification,
CanonStateNotificationSender, CanonStateNotifications, Chain, ChainSpecProvider, ChainSplit,
ChainSplitTarget, DisplayBlocksChain, ExecutionOutcome, HeaderProvider, ProviderError,
StaticFileProviderFactory,
};
use reth_prune_types::PruneModes;
use reth_stages_api::{MetricEvent, MetricEventsSender};
@ -264,7 +264,7 @@ where
/// * block unknown.
/// * `chain_id` not present in state.
/// * there are no parent hashes stored.
pub fn post_state_data(&self, block_hash: BlockHash) -> Option<BundleStateData> {
pub fn post_state_data(&self, block_hash: BlockHash) -> Option<ExecutionData> {
trace!(target: "blockchain_tree", ?block_hash, "Searching for post state data");
let canonical_chain = self.state.block_indices.canonical_chain();
@ -278,7 +278,7 @@ where
return None;
};
let block_number = chain.block_number(block_hash)?;
let state = chain.state_at_block(block_number)?;
let execution_outcome = chain.execution_outcome_at_block(block_number)?;
// get parent hashes
let mut parent_block_hashes = self.all_chain_hashes(chain_id);
@ -295,15 +295,15 @@ where
// get canonical fork.
let canonical_fork = self.canonical_fork(chain_id)?;
return Some(BundleStateData { state, parent_block_hashes, canonical_fork })
return Some(ExecutionData { execution_outcome, parent_block_hashes, canonical_fork })
}
// check if there is canonical block
if let Some(canonical_number) = canonical_chain.canonical_number(&block_hash) {
trace!(target: "blockchain_tree", %block_hash, "Constructing post state data based on canonical chain");
return Some(BundleStateData {
return Some(ExecutionData {
canonical_fork: ForkBlock { number: canonical_number, hash: block_hash },
state: BundleStateWithReceipts::default(),
execution_outcome: ExecutionOutcome::default(),
parent_block_hashes: canonical_chain.inner().clone(),
})
}
@ -629,8 +629,8 @@ where
let chains_to_bump = self.find_all_dependent_chains(&hash);
if !chains_to_bump.is_empty() {
// if there is such chain, revert state to this block.
let mut cloned_state = chain.state().clone();
cloned_state.revert_to(*number);
let mut cloned_execution_outcome = chain.execution_outcome().clone();
cloned_execution_outcome.revert_to(*number);
// prepend state to all chains that fork from this block.
for chain_id in chains_to_bump {
@ -645,7 +645,7 @@ where
chain_tip = ?chain.tip().num_hash(),
"Prepend unwound block state to blockchain tree chain");
chain.prepend_state(cloned_state.state().clone())
chain.prepend_state(cloned_execution_outcome.state().clone())
}
}
}
@ -1398,7 +1398,7 @@ mod tests {
use std::collections::HashMap;
fn setup_externals(
exec_res: Vec<BundleStateWithReceipts>,
exec_res: Vec<ExecutionOutcome>,
) -> TreeExternals<Arc<TempDatabase<DatabaseEnv>>, MockExecutorProvider> {
let chain_spec = Arc::new(
ChainSpecBuilder::default()
@ -1961,12 +1961,12 @@ mod tests {
]))
.assert(&tree);
// chain 0 has two blocks so receipts and reverts len is 2
let chain0 = tree.state.chains.get(&0.into()).unwrap().state();
let chain0 = tree.state.chains.get(&0.into()).unwrap().execution_outcome();
assert_eq!(chain0.receipts().len(), 2);
assert_eq!(chain0.state().reverts.len(), 2);
assert_eq!(chain0.first_block(), block1.number);
// chain 1 has one block so receipts and reverts len is 1
let chain1 = tree.state.chains.get(&1.into()).unwrap().state();
let chain1 = tree.state.chains.get(&1.into()).unwrap().execution_outcome();
assert_eq!(chain1.receipts().len(), 1);
assert_eq!(chain1.state().reverts.len(), 1);
assert_eq!(chain1.first_block(), block2.number);

View File

@ -1,14 +1,14 @@
//! [`BundleStateDataProvider`] implementations used by the tree.
//! [`ExecutionDataProvider`] implementations used by the tree.
use reth_primitives::{BlockHash, BlockNumber, ForkBlock};
use reth_provider::{BundleStateDataProvider, BundleStateForkProvider, BundleStateWithReceipts};
use reth_provider::{BlockExecutionForkProvider, ExecutionDataProvider, ExecutionOutcome};
use std::collections::BTreeMap;
/// Structure that combines references of required data to be a [`BundleStateDataProvider`].
/// Structure that combines references of required data to be a [`ExecutionDataProvider`].
#[derive(Clone, Debug)]
pub struct BundleStateDataRef<'a> {
/// The wrapped state after execution of one or more transactions and/or blocks.
pub state: &'a BundleStateWithReceipts,
/// The execution outcome after execution of one or more transactions and/or blocks.
pub execution_outcome: &'a ExecutionOutcome,
/// The blocks in the sidechain.
pub sidechain_block_hashes: &'a BTreeMap<BlockNumber, BlockHash>,
/// The blocks in the canonical chain.
@ -17,9 +17,9 @@ pub struct BundleStateDataRef<'a> {
pub canonical_fork: ForkBlock,
}
impl<'a> BundleStateDataProvider for BundleStateDataRef<'a> {
fn state(&self) -> &BundleStateWithReceipts {
self.state
impl<'a> ExecutionDataProvider for BundleStateDataRef<'a> {
fn execution_outcome(&self) -> &ExecutionOutcome {
self.execution_outcome
}
fn block_hash(&self, block_number: BlockNumber) -> Option<BlockHash> {
@ -32,17 +32,17 @@ impl<'a> BundleStateDataProvider for BundleStateDataRef<'a> {
}
}
impl<'a> BundleStateForkProvider for BundleStateDataRef<'a> {
impl<'a> BlockExecutionForkProvider for BundleStateDataRef<'a> {
fn canonical_fork(&self) -> ForkBlock {
self.canonical_fork
}
}
/// Structure that owns the relevant data needs to be a [`BundleStateDataProvider`]
/// Structure that owns the relevant data needs to be a [`ExecutionDataProvider`]
#[derive(Clone, Debug)]
pub struct BundleStateData {
/// Post state with changes
pub state: BundleStateWithReceipts,
pub struct ExecutionData {
/// Execution outcome.
pub execution_outcome: ExecutionOutcome,
/// Parent block hashes needs for evm BLOCKHASH opcode.
/// NOTE: it does not mean that all hashes are there but all until finalized are there.
/// Other hashes can be obtained from provider
@ -51,9 +51,9 @@ pub struct BundleStateData {
pub canonical_fork: ForkBlock,
}
impl BundleStateDataProvider for BundleStateData {
fn state(&self) -> &BundleStateWithReceipts {
&self.state
impl ExecutionDataProvider for ExecutionData {
fn execution_outcome(&self) -> &ExecutionOutcome {
&self.execution_outcome
}
fn block_hash(&self, block_number: BlockNumber) -> Option<BlockHash> {
@ -61,7 +61,7 @@ impl BundleStateDataProvider for BundleStateData {
}
}
impl BundleStateForkProvider for BundleStateData {
impl BlockExecutionForkProvider for ExecutionData {
fn canonical_fork(&self) -> ForkBlock {
self.canonical_fork
}

View File

@ -18,7 +18,7 @@ use reth_primitives::{
};
use reth_provider::{
providers::{BundleStateProvider, ConsistentDbView},
BundleStateWithReceipts, Chain, FullBundleStateDataProvider, ProviderError, StateRootProvider,
Chain, ExecutionOutcome, FullExecutionDataProvider, ProviderError, StateRootProvider,
};
use reth_revm::database::StateProviderDatabase;
use reth_trie::updates::TrieUpdates;
@ -78,11 +78,11 @@ impl AppendableChain {
DB: Database + Clone,
E: BlockExecutorProvider,
{
let state = BundleStateWithReceipts::default();
let execution_outcome = ExecutionOutcome::default();
let empty = BTreeMap::new();
let state_provider = BundleStateDataRef {
state: &state,
execution_outcome: &execution_outcome,
sidechain_block_hashes: &empty,
canonical_block_hashes,
canonical_fork,
@ -122,14 +122,14 @@ impl AppendableChain {
BlockchainTreeError::BlockNumberNotFoundInChain { block_number: parent_number },
)?;
let mut state = self.state().clone();
let mut execution_outcome = self.execution_outcome().clone();
// Revert state to the state after execution of the parent block
state.revert_to(parent.number);
execution_outcome.revert_to(parent.number);
// Revert changesets to get the state of the parent that we need to apply the change.
let bundle_state_data = BundleStateDataRef {
state: &state,
execution_outcome: &execution_outcome,
sidechain_block_hashes: &side_chain_block_hashes,
canonical_block_hashes,
canonical_fork,
@ -144,17 +144,17 @@ impl AppendableChain {
)?;
// extending will also optimize few things, mostly related to selfdestruct and wiping of
// storage.
state.extend(block_state);
execution_outcome.extend(block_state);
// remove all receipts and reverts (except the last one), as they belong to the chain we
// forked from and not the new chain we are creating.
let size = state.receipts().len();
state.receipts_mut().drain(0..size - 1);
state.state_mut().take_n_reverts(size - 1);
state.set_first_block(block.number);
let size = execution_outcome.receipts().len();
execution_outcome.receipts_mut().drain(0..size - 1);
execution_outcome.state_mut().take_n_reverts(size - 1);
execution_outcome.set_first_block(block.number);
// If all is okay, return new chain back. Present chain is not modified.
Ok(Self { chain: Chain::from_block(block, state, None) })
Ok(Self { chain: Chain::from_block(block, execution_outcome, None) })
}
/// Validate and execute the given block that _extends the canonical chain_, validating its
@ -166,16 +166,16 @@ 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, E>(
fn validate_and_execute<EDP, DB, E>(
block: SealedBlockWithSenders,
parent_block: &SealedHeader,
bundle_state_data_provider: BSDP,
bundle_state_data_provider: EDP,
externals: &TreeExternals<DB, E>,
block_attachment: BlockAttachment,
block_validation_kind: BlockValidationKind,
) -> Result<(BundleStateWithReceipts, Option<TrieUpdates>), BlockExecutionError>
) -> Result<(ExecutionOutcome, Option<TrieUpdates>), BlockExecutionError>
where
BSDP: FullBundleStateDataProvider,
EDP: FullExecutionDataProvider,
DB: Database + Clone,
E: BlockExecutorProvider,
{
@ -214,12 +214,8 @@ impl AppendableChain {
.consensus
.validate_block_post_execution(&block, PostExecutionInput::new(&receipts, &requests))?;
let bundle_state = BundleStateWithReceipts::new(
state,
receipts.into(),
block.number,
vec![requests.into()],
);
let initial_execution_outcome =
ExecutionOutcome::new(state, receipts.into(), block.number, vec![requests.into()]);
// check state root if the block extends the canonical chain __and__ if state root
// validation was requested.
@ -227,15 +223,16 @@ impl AppendableChain {
// calculate and check state root
let start = Instant::now();
let (state_root, trie_updates) = if block_attachment.is_canonical() {
let mut state = provider.bundle_state_data_provider.state().clone();
state.extend(bundle_state.clone());
let hashed_state = state.hash_state_slow();
let mut execution_outcome =
provider.block_execution_data_provider.execution_outcome().clone();
execution_outcome.extend(initial_execution_outcome.clone());
let hashed_state = execution_outcome.hash_state_slow();
ParallelStateRoot::new(consistent_view, hashed_state)
.incremental_root_with_updates()
.map(|(root, updates)| (root, Some(updates)))
.map_err(ProviderError::from)?
} else {
(provider.state_root(bundle_state.state())?, None)
(provider.state_root(initial_execution_outcome.state())?, None)
};
if block.state_root != state_root {
return Err(ConsensusError::BodyStateRootDiff(
@ -252,9 +249,9 @@ impl AppendableChain {
"Validated state root"
);
Ok((bundle_state, trie_updates))
Ok((initial_execution_outcome, trie_updates))
} else {
Ok((bundle_state, None))
Ok((initial_execution_outcome, None))
}
}
@ -288,7 +285,7 @@ impl AppendableChain {
let parent_block = self.chain.tip();
let bundle_state_data = BundleStateDataRef {
state: self.state(),
execution_outcome: self.execution_outcome(),
sidechain_block_hashes: &side_chain_block_hashes,
canonical_block_hashes,
canonical_fork,

View File

@ -40,7 +40,7 @@ pub mod shareable;
pub use shareable::ShareableBlockchainTree;
mod bundle;
pub use bundle::{BundleStateData, BundleStateDataRef};
pub use bundle::{BundleStateDataRef, ExecutionData};
/// Buffer of not executed blocks.
pub mod block_buffer;

View File

@ -10,7 +10,7 @@ use reth_primitives::{
};
use reth_provider::{
BlockchainTreePendingStateProvider, CanonStateNotificationSender, CanonStateNotifications,
CanonStateSubscriptions, FullBundleStateDataProvider,
CanonStateSubscriptions, FullExecutionDataProvider,
};
use reth_storage_errors::provider::ProviderResult;
use std::collections::BTreeMap;
@ -122,7 +122,7 @@ impl BlockchainTreePendingStateProvider for NoopBlockchainTree {
fn find_pending_state_provider(
&self,
_block_hash: BlockHash,
) -> Option<Box<dyn FullBundleStateDataProvider>> {
) -> Option<Box<dyn FullExecutionDataProvider>> {
None
}
}

View File

@ -14,7 +14,7 @@ use reth_primitives::{
SealedHeader,
};
use reth_provider::{
BlockchainTreePendingStateProvider, CanonStateSubscriptions, FullBundleStateDataProvider,
BlockchainTreePendingStateProvider, CanonStateSubscriptions, FullExecutionDataProvider,
ProviderError,
};
use reth_storage_errors::provider::ProviderResult;
@ -177,7 +177,7 @@ where
fn find_pending_state_provider(
&self,
block_hash: BlockHash,
) -> Option<Box<dyn FullBundleStateDataProvider>> {
) -> Option<Box<dyn FullExecutionDataProvider>> {
trace!(target: "blockchain_tree", ?block_hash, "Finding pending state provider");
let provider = self.tree.read().post_state_data(block_hash)?;
Some(Box::new(provider))

View File

@ -26,9 +26,7 @@ use reth_primitives::{
ChainSpec, Header, Requests, SealedBlock, SealedHeader, TransactionSigned, Withdrawals, B256,
U256,
};
use reth_provider::{
BlockReaderIdExt, BundleStateWithReceipts, StateProviderFactory, StateRootProvider,
};
use reth_provider::{BlockReaderIdExt, ExecutionOutcome, StateProviderFactory, StateRootProvider};
use reth_revm::database::StateProviderDatabase;
use reth_transaction_pool::TransactionPool;
use std::{
@ -349,7 +347,7 @@ impl StorageInner {
provider: &Provider,
chain_spec: Arc<ChainSpec>,
executor: &Executor,
) -> Result<(SealedHeader, BundleStateWithReceipts), BlockExecutionError>
) -> Result<(SealedHeader, ExecutionOutcome), BlockExecutionError>
where
Executor: BlockExecutorProvider,
Provider: StateProviderFactory,
@ -391,7 +389,7 @@ impl StorageInner {
// execute the block
let BlockExecutionOutput { state, receipts, requests: block_execution_requests, .. } =
executor.executor(&mut db).execute((&block, U256::ZERO).into())?;
let bundle_state = BundleStateWithReceipts::new(
let execution_outcome = ExecutionOutcome::new(
state,
receipts.into(),
block.number,
@ -405,10 +403,10 @@ impl StorageInner {
let Block { mut header, body, .. } = block.block;
let body = BlockBody { transactions: body, ommers, withdrawals, requests };
trace!(target: "consensus::auto", ?bundle_state, ?header, ?body, "executed block, calculating state root and completing header");
trace!(target: "consensus::auto", ?execution_outcome, ?header, ?body, "executed block, calculating state root and completing header");
// calculate the state root
header.state_root = db.state_root(bundle_state.state())?;
header.state_root = db.state_root(execution_outcome.state())?;
trace!(target: "consensus::auto", root=?header.state_root, ?body, "calculated root");
// finally insert into storage
@ -417,6 +415,6 @@ impl StorageInner {
// set new header with hash that should have been updated by insert_new_block
let new_header = header.seal(self.best_hash);
Ok((new_header, bundle_state))
Ok((new_header, execution_outcome))
}
}

View File

@ -435,7 +435,7 @@ mod tests {
MAINNET,
};
use reth_provider::{
test_utils::create_test_provider_factory_with_chain_spec, BundleStateWithReceipts,
test_utils::create_test_provider_factory_with_chain_spec, ExecutionOutcome,
};
use reth_prune_types::PruneModes;
use reth_stages::{test_utils::TestStages, ExecOutput, StageError};
@ -447,7 +447,7 @@ mod tests {
struct TestPipelineBuilder {
pipeline_exec_outputs: VecDeque<Result<ExecOutput, StageError>>,
executor_results: Vec<BundleStateWithReceipts>,
executor_results: Vec<ExecutionOutcome>,
max_block: Option<BlockNumber>,
}
@ -472,7 +472,7 @@ mod tests {
/// Set the executor results to use for the test consensus engine.
#[allow(dead_code)]
fn with_executor_results(mut self, executor_results: Vec<BundleStateWithReceipts>) -> Self {
fn with_executor_results(mut self, executor_results: Vec<ExecutionOutcome>) -> Self {
self.executor_results = executor_results;
self
}

View File

@ -25,7 +25,7 @@ use reth_payload_builder::test_utils::spawn_test_payload_service;
use reth_primitives::{BlockNumber, ChainSpec, B256};
use reth_provider::{
providers::BlockchainProvider, test_utils::create_test_provider_factory_with_chain_spec,
BundleStateWithReceipts, HeaderSyncMode,
ExecutionOutcome, HeaderSyncMode,
};
use reth_prune::Pruner;
use reth_prune_types::PruneModes;
@ -142,7 +142,7 @@ impl Default for TestPipelineConfig {
#[derive(Debug)]
enum TestExecutorConfig {
/// Test executor results.
Test(Vec<BundleStateWithReceipts>),
Test(Vec<ExecutionOutcome>),
/// Real executor configuration.
Real,
}
@ -188,7 +188,7 @@ impl TestConsensusEngineBuilder {
}
/// Set the executor results to use for the test consensus engine.
pub fn with_executor_results(mut self, executor_results: Vec<BundleStateWithReceipts>) -> Self {
pub fn with_executor_results(mut self, executor_results: Vec<ExecutionOutcome>) -> Self {
self.executor_config = TestExecutorConfig::Test(executor_results);
self
}
@ -272,7 +272,7 @@ where
/// Set the executor results to use for the test consensus engine.
#[allow(dead_code)]
pub fn with_executor_results(mut self, executor_results: Vec<BundleStateWithReceipts>) -> Self {
pub fn with_executor_results(mut self, executor_results: Vec<ExecutionOutcome>) -> Self {
self.base_config.executor_config = TestExecutorConfig::Test(executor_results);
self
}

View File

@ -12,7 +12,7 @@ use reth_evm::{
},
ConfigureEvm,
};
use reth_execution_types::BundleStateWithReceipts;
use reth_execution_types::ExecutionOutcome;
use reth_primitives::{
BlockNumber, BlockWithSenders, ChainSpec, Hardfork, Header, Receipt, Request, Withdrawals,
MAINNET, U256,
@ -406,7 +406,7 @@ where
DB: Database<Error = ProviderError>,
{
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BundleStateWithReceipts;
type Output = ExecutionOutcome;
type Error = BlockExecutionError;
fn execute_and_verify_one(&mut self, input: Self::Input<'_>) -> Result<(), Self::Error> {
@ -436,7 +436,7 @@ where
fn finalize(mut self) -> Self::Output {
self.stats.log_debug();
BundleStateWithReceipts::new(
ExecutionOutcome::new(
self.executor.state.take_bundle(),
self.batch_record.take_receipts(),
self.batch_record.first_block().unwrap_or_default(),

View File

@ -29,7 +29,7 @@ use reth_primitives::{
revm::env::tx_env_with_recovered,
Block, Header, IntoRecoveredTransaction, Receipt, EMPTY_OMMER_ROOT_HASH, U256,
};
use reth_provider::{BundleStateWithReceipts, StateProviderFactory};
use reth_provider::{ExecutionOutcome, StateProviderFactory};
use reth_revm::{database::StateProviderDatabase, state_change::apply_blockhashes_update};
use reth_transaction_pool::{BestTransactionsAttributes, TransactionPool};
use revm::{
@ -444,19 +444,20 @@ where
// and 4788 contract call
db.merge_transitions(BundleRetention::PlainState);
let bundle = BundleStateWithReceipts::new(
let execution_outcome = ExecutionOutcome::new(
db.take_bundle(),
vec![receipts].into(),
block_number,
vec![requests.clone().unwrap_or_default()],
);
let receipts_root = bundle.receipts_root_slow(block_number).expect("Number is in range");
let logs_bloom = bundle.block_logs_bloom(block_number).expect("Number is in range");
let receipts_root =
execution_outcome.receipts_root_slow(block_number).expect("Number is in range");
let logs_bloom = execution_outcome.block_logs_bloom(block_number).expect("Number is in range");
// calculate the state root
let state_root = {
let state_provider = db.database.0.inner.borrow_mut();
state_provider.db.state_root(bundle.state())?
state_provider.db.state_root(execution_outcome.state())?
};
// create the block header

View File

@ -11,11 +11,12 @@ use revm::{
};
use std::collections::HashMap;
/// Bundle state of post execution changes and reverts.
/// Represents the outcome of block execution, including post-execution changes and reverts.
///
/// Aggregates the changes over an arbitrary number of blocks.
/// The `ExecutionOutcome` structure aggregates the state changes over an arbitrary number of
/// blocks, capturing the resulting state, receipts, and requests following the execution.
#[derive(Default, Debug, Clone, PartialEq, Eq)]
pub struct BundleStateWithReceipts {
pub struct ExecutionOutcome {
/// Bundle state with reverts.
pub bundle: BundleState,
/// The collection of receipts.
@ -45,8 +46,11 @@ pub type AccountRevertInit = (Option<Option<Account>>, Vec<StorageEntry>);
/// Type used to initialize revms reverts.
pub type RevertsInit = HashMap<BlockNumber, HashMap<Address, AccountRevertInit>>;
impl BundleStateWithReceipts {
/// Create Bundle State.
impl ExecutionOutcome {
/// Creates a new `ExecutionOutcome`.
///
/// This constructor initializes a new `ExecutionOutcome` instance with the provided
/// bundle state, receipts, first block number, and EIP-7685 requests.
pub const fn new(
bundle: BundleState,
receipts: Receipts,
@ -56,7 +60,10 @@ impl BundleStateWithReceipts {
Self { bundle, receipts, first_block, requests }
}
/// Create new bundle state with receipts.
/// Creates a new `ExecutionOutcome` from initialization parameters.
///
/// This constructor initializes a new `ExecutionOutcome` instance using detailed
/// initialization parameters.
pub fn new_init(
state_init: BundleStateInit,
revert_init: RevertsInit,
@ -137,7 +144,7 @@ impl BundleStateWithReceipts {
self.bundle.bytecode(code_hash).map(Bytecode)
}
/// Returns [`HashedPostState`] for this bundle state.
/// Returns [`HashedPostState`] for this execution outcome.
/// See [`HashedPostState::from_bundle_state`] for more info.
pub fn hash_state_slow(&self) -> HashedPostState {
HashedPostState::from_bundle_state(&self.bundle.state)
@ -298,6 +305,18 @@ impl BundleStateWithReceipts {
// swap bundles
std::mem::swap(&mut self.bundle, &mut other)
}
/// Create a new instance with updated receipts.
pub fn with_receipts(mut self, receipts: Receipts) -> Self {
self.receipts = receipts;
self
}
/// Create a new instance with updated requests.
pub fn with_requests(mut self, requests: Vec<Requests>) -> Self {
self.requests = requests;
self
}
}
#[cfg(test)]
@ -358,18 +377,18 @@ mod tests {
// Define the first block number
let first_block = 123;
// Create a BundleStateWithReceipts object with the created bundle, receipts, requests, and
// Create a ExecutionOutcome object with the created bundle, receipts, requests, and
// first_block
let exec_res = BundleStateWithReceipts {
let exec_res = ExecutionOutcome {
bundle: bundle.clone(),
receipts: receipts.clone(),
requests: requests.clone(),
first_block,
};
// Assert that creating a new BundleStateWithReceipts using the constructor matches exec_res
// Assert that creating a new ExecutionOutcome using the constructor matches exec_res
assert_eq!(
BundleStateWithReceipts::new(bundle, receipts.clone(), first_block, requests.clone()),
ExecutionOutcome::new(bundle, receipts.clone(), first_block, requests.clone()),
exec_res
);
@ -386,10 +405,10 @@ mod tests {
let mut revert_init: RevertsInit = HashMap::new();
revert_init.insert(123, revert_inner);
// Assert that creating a new BundleStateWithReceipts using the new_init method matches
// Assert that creating a new ExecutionOutcome using the new_init method matches
// exec_res
assert_eq!(
BundleStateWithReceipts::new_init(
ExecutionOutcome::new_init(
state_init,
revert_init,
vec![],
@ -420,9 +439,9 @@ mod tests {
// Define the first block number
let first_block = 123;
// Create a BundleStateWithReceipts object with the created bundle, receipts, requests, and
// Create a ExecutionOutcome object with the created bundle, receipts, requests, and
// first_block
let exec_res = BundleStateWithReceipts {
let exec_res = ExecutionOutcome {
bundle: Default::default(),
receipts,
requests: vec![],
@ -458,9 +477,9 @@ mod tests {
// Define the first block number
let first_block = 123;
// Create a BundleStateWithReceipts object with the created bundle, receipts, requests, and
// Create a ExecutionOutcome object with the created bundle, receipts, requests, and
// first_block
let exec_res = BundleStateWithReceipts {
let exec_res = ExecutionOutcome {
bundle: Default::default(),
receipts,
requests: vec![],
@ -493,9 +512,9 @@ mod tests {
// Define the first block number
let first_block = 123;
// Create a BundleStateWithReceipts object with the created bundle, receipts, requests, and
// Create a ExecutionOutcome object with the created bundle, receipts, requests, and
// first_block
let exec_res = BundleStateWithReceipts {
let exec_res = ExecutionOutcome {
bundle: Default::default(), // Default value for bundle
receipts, // Include the created receipts
requests: vec![], // Empty vector for requests
@ -543,9 +562,9 @@ mod tests {
// Define the first block number
let first_block = 123;
// Create a BundleStateWithReceipts object with the created bundle, receipts, requests, and
// Create a ExecutionOutcome object with the created bundle, receipts, requests, and
// first_block
let exec_res = BundleStateWithReceipts {
let exec_res = ExecutionOutcome {
bundle: Default::default(), // Default value for bundle
receipts, // Include the created receipts
requests: vec![], // Empty vector for requests
@ -558,8 +577,8 @@ mod tests {
// Assert that exec_res is not empty
assert!(!exec_res.is_empty());
// Create a BundleStateWithReceipts object with an empty Receipts object
let exec_res_empty_receipts = BundleStateWithReceipts {
// Create a ExecutionOutcome object with an empty Receipts object
let exec_res_empty_receipts = ExecutionOutcome {
bundle: Default::default(), // Default value for bundle
receipts: receipts_empty, // Include the empty receipts
requests: vec![], // Empty vector for requests

View File

@ -1,6 +1,6 @@
//! Contains [Chain], a chain of blocks and their final state.
use crate::BundleStateWithReceipts;
use crate::ExecutionOutcome;
use reth_execution_errors::BlockExecutionError;
use reth_primitives::{
Address, BlockHash, BlockNumHash, BlockNumber, ForkBlock, Receipt, SealedBlock,
@ -24,11 +24,13 @@ use std::{borrow::Cow, collections::BTreeMap, fmt, ops::RangeInclusive};
pub struct Chain {
/// All blocks in this chain.
blocks: BTreeMap<BlockNumber, SealedBlockWithSenders>,
/// The state of all accounts after execution of the _all_ blocks in this chain's range from
/// [`Chain::first`] to [`Chain::tip`], inclusive.
/// The outcome of block execution for this chain.
///
/// This state also contains the individual changes that lead to the current state.
state: BundleStateWithReceipts,
/// This field contains the state of all accounts after the execution of all blocks in this
/// chain, ranging from the [`Chain::first`] block to the [`Chain::tip`] block, inclusive.
///
/// Additionally, it includes the individual state changes that led to the current state.
execution_outcome: ExecutionOutcome,
/// State trie updates after block is added to the chain.
/// NOTE: Currently, trie updates are present only for
/// single-block chains that extend the canonical chain.
@ -43,22 +45,22 @@ impl Chain {
/// A chain of blocks should not be empty.
pub fn new(
blocks: impl IntoIterator<Item = SealedBlockWithSenders>,
state: BundleStateWithReceipts,
execution_outcome: ExecutionOutcome,
trie_updates: Option<TrieUpdates>,
) -> Self {
let blocks = BTreeMap::from_iter(blocks.into_iter().map(|b| (b.number, b)));
debug_assert!(!blocks.is_empty(), "Chain should have at least one block");
Self { blocks, state, trie_updates }
Self { blocks, execution_outcome, trie_updates }
}
/// Create new Chain from a single block and its state.
pub fn from_block(
block: SealedBlockWithSenders,
state: BundleStateWithReceipts,
execution_outcome: ExecutionOutcome,
trie_updates: Option<TrieUpdates>,
) -> Self {
Self::new([block], state, trie_updates)
Self::new([block], execution_outcome, trie_updates)
}
/// Get the blocks in this chain.
@ -86,14 +88,14 @@ impl Chain {
self.trie_updates.take();
}
/// Get post state of this chain
pub const fn state(&self) -> &BundleStateWithReceipts {
&self.state
/// Get execution outcome of this chain
pub const fn execution_outcome(&self) -> &ExecutionOutcome {
&self.execution_outcome
}
/// Prepends the given state to the current state.
pub fn prepend_state(&mut self, state: BundleState) {
self.state.prepend_state(state);
self.execution_outcome.prepend_state(state);
self.trie_updates.take(); // invalidate cached trie updates
}
@ -117,37 +119,41 @@ impl Chain {
self.blocks.iter().find_map(|(_num, block)| (block.hash() == block_hash).then_some(block))
}
/// Return post state of the block at the `block_number` or None if block is not known
pub fn state_at_block(&self, block_number: BlockNumber) -> Option<BundleStateWithReceipts> {
/// Return execution outcome at the `block_number` or None if block is not known
pub fn execution_outcome_at_block(
&self,
block_number: BlockNumber,
) -> Option<ExecutionOutcome> {
if self.tip().number == block_number {
return Some(self.state.clone())
return Some(self.execution_outcome.clone())
}
if self.blocks.contains_key(&block_number) {
let mut state = self.state.clone();
state.revert_to(block_number);
return Some(state)
let mut execution_outcome = self.execution_outcome.clone();
execution_outcome.revert_to(block_number);
return Some(execution_outcome)
}
None
}
/// Destructure the chain into its inner components, the blocks and the state at the tip of the
/// chain.
pub fn into_inner(
self,
) -> (ChainBlocks<'static>, BundleStateWithReceipts, Option<TrieUpdates>) {
(ChainBlocks { blocks: Cow::Owned(self.blocks) }, self.state, self.trie_updates)
/// Destructure the chain into its inner components:
/// 1. The blocks contained in the chain.
/// 2. The execution outcome representing the final state.
/// 3. The optional trie updates.
pub fn into_inner(self) -> (ChainBlocks<'static>, ExecutionOutcome, Option<TrieUpdates>) {
(ChainBlocks { blocks: Cow::Owned(self.blocks) }, self.execution_outcome, self.trie_updates)
}
/// Destructure the chain into its inner components, the blocks and the state at the tip of the
/// chain.
pub const fn inner(&self) -> (ChainBlocks<'_>, &BundleStateWithReceipts) {
(ChainBlocks { blocks: Cow::Borrowed(&self.blocks) }, &self.state)
/// Destructure the chain into its inner components:
/// 1. A reference to the blocks contained in the chain.
/// 2. A reference to the execution outcome representing the final state.
pub const fn inner(&self) -> (ChainBlocks<'_>, &ExecutionOutcome) {
(ChainBlocks { blocks: Cow::Borrowed(&self.blocks) }, &self.execution_outcome)
}
/// Returns an iterator over all the receipts of the blocks in the chain.
pub fn block_receipts_iter(&self) -> impl Iterator<Item = &Vec<Option<Receipt>>> + '_ {
self.state.receipts().iter()
self.execution_outcome.receipts().iter()
}
/// Returns an iterator over all blocks in the chain with increasing block number.
@ -206,7 +212,7 @@ impl Chain {
/// Get all receipts for the given block.
pub fn receipts_by_block_hash(&self, block_hash: BlockHash) -> Option<Vec<&Receipt>> {
let num = self.block_number(block_hash)?;
self.state.receipts_by_block(num).iter().map(Option::as_ref).collect()
self.execution_outcome.receipts_by_block(num).iter().map(Option::as_ref).collect()
}
/// Get all receipts with attachment.
@ -214,7 +220,8 @@ impl Chain {
/// Attachment includes block number, block hash, transaction hash and transaction index.
pub fn receipts_with_attachment(&self) -> Vec<BlockReceipts> {
let mut receipt_attach = Vec::new();
for ((block_num, block), receipts) in self.blocks().iter().zip(self.state.receipts().iter())
for ((block_num, block), receipts) in
self.blocks().iter().zip(self.execution_outcome.receipts().iter())
{
let mut tx_receipts = Vec::new();
for (tx, receipt) in block.body.iter().zip(receipts.iter()) {
@ -231,9 +238,13 @@ impl Chain {
/// Append a single block with state to the chain.
/// This method assumes that blocks attachment to the chain has already been validated.
pub fn append_block(&mut self, block: SealedBlockWithSenders, state: BundleStateWithReceipts) {
pub fn append_block(
&mut self,
block: SealedBlockWithSenders,
execution_outcome: ExecutionOutcome,
) {
self.blocks.insert(block.number, block);
self.state.extend(state);
self.execution_outcome.extend(execution_outcome);
self.trie_updates.take(); // reset
}
@ -252,7 +263,7 @@ impl Chain {
// Insert blocks from other chain
self.blocks.extend(other.blocks);
self.state.extend(other.state);
self.execution_outcome.extend(other.execution_outcome);
self.trie_updates.take(); // reset
Ok(())
@ -309,19 +320,20 @@ impl Chain {
let split_at = block_number + 1;
let higher_number_blocks = self.blocks.split_off(&split_at);
let state = std::mem::take(&mut self.state);
let (canonical_state, pending_state) = state.split_at(split_at);
let execution_outcome = std::mem::take(&mut self.execution_outcome);
let (canonical_block_exec_outcome, pending_block_exec_outcome) =
execution_outcome.split_at(split_at);
// TODO: Currently, trie updates are reset on chain split.
// Add tests ensuring that it is valid to leave updates in the pending chain.
ChainSplit::Split {
canonical: Self {
state: canonical_state.expect("split in range"),
execution_outcome: canonical_block_exec_outcome.expect("split in range"),
blocks: self.blocks,
trie_updates: None,
},
pending: Self {
state: pending_state,
execution_outcome: pending_block_exec_outcome,
blocks: higher_number_blocks,
trie_updates: None,
},
@ -513,7 +525,7 @@ mod tests {
#[test]
fn test_number_split() {
let block_state1 = BundleStateWithReceipts::new(
let execution_outcome1 = ExecutionOutcome::new(
BundleState::new(
vec![(
Address::new([2; 20]),
@ -529,7 +541,7 @@ mod tests {
vec![],
);
let block_state2 = BundleStateWithReceipts::new(
let execution_outcome2 = ExecutionOutcome::new(
BundleState::new(
vec![(
Address::new([3; 20]),
@ -557,30 +569,37 @@ mod tests {
block2.set_hash(block2_hash);
block2.senders.push(Address::new([4; 20]));
let mut block_state_extended = block_state1;
block_state_extended.extend(block_state2);
let mut block_state_extended = execution_outcome1;
block_state_extended.extend(execution_outcome2);
let chain = Chain::new(vec![block1.clone(), block2.clone()], block_state_extended, None);
let (split1_state, split2_state) = chain.state.clone().split_at(2);
let (split1_execution_outcome, split2_execution_outcome) =
chain.execution_outcome.clone().split_at(2);
let chain_split1 = Chain {
state: split1_state.unwrap(),
execution_outcome: split1_execution_outcome.unwrap(),
blocks: BTreeMap::from([(1, block1.clone())]),
trie_updates: None,
};
let chain_split2 = Chain {
state: split2_state,
execution_outcome: split2_execution_outcome,
blocks: BTreeMap::from([(2, block2.clone())]),
trie_updates: None,
};
// return tip state
assert_eq!(chain.state_at_block(block2.number), Some(chain.state.clone()));
assert_eq!(chain.state_at_block(block1.number), Some(chain_split1.state.clone()));
assert_eq!(
chain.execution_outcome_at_block(block2.number),
Some(chain.execution_outcome.clone())
);
assert_eq!(
chain.execution_outcome_at_block(block1.number),
Some(chain_split1.execution_outcome.clone())
);
// state at unknown block
assert_eq!(chain.state_at_block(100), None);
assert_eq!(chain.execution_outcome_at_block(100), None);
// split in two
assert_eq!(

View File

@ -4,7 +4,7 @@ use crate::execute::{
BatchExecutor, BlockExecutionInput, BlockExecutionOutput, BlockExecutorProvider, Executor,
};
use reth_execution_errors::BlockExecutionError;
use reth_execution_types::BundleStateWithReceipts;
use reth_execution_types::ExecutionOutcome;
use reth_primitives::{BlockNumber, BlockWithSenders, Receipt};
use reth_prune_types::PruneModes;
use reth_storage_errors::provider::ProviderError;
@ -76,19 +76,19 @@ where
A: for<'a> BatchExecutor<
DB,
Input<'a> = BlockExecutionInput<'a, BlockWithSenders>,
Output = BundleStateWithReceipts,
Output = ExecutionOutcome,
Error = BlockExecutionError,
>,
B: for<'a> BatchExecutor<
DB,
Input<'a> = BlockExecutionInput<'a, BlockWithSenders>,
Output = BundleStateWithReceipts,
Output = ExecutionOutcome,
Error = BlockExecutionError,
>,
DB: Database<Error = ProviderError>,
{
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BundleStateWithReceipts;
type Output = ExecutionOutcome;
type Error = BlockExecutionError;
fn execute_and_verify_one(&mut self, input: Self::Input<'_>) -> Result<(), Self::Error> {

View File

@ -1,6 +1,6 @@
//! Traits for execution.
use reth_execution_types::BundleStateWithReceipts;
use reth_execution_types::ExecutionOutcome;
use reth_primitives::{BlockNumber, BlockWithSenders, Receipt, Request, U256};
use reth_prune_types::PruneModes;
use revm::db::BundleState;
@ -91,7 +91,7 @@ pub trait BatchExecutor<DB> {
///
/// Contains the state changes, transaction receipts, and total gas used in the block.
///
/// TODO(mattsse): combine with `BundleStateWithReceipts`
/// TODO(mattsse): combine with `ExecutionOutcome`
#[derive(Debug)]
pub struct BlockExecutionOutput<T> {
/// The changed state of the block after execution.
@ -150,7 +150,7 @@ pub trait BlockExecutorProvider: Send + Sync + Clone + Unpin + 'static {
type BatchExecutor<DB: Database<Error = ProviderError>>: for<'a> BatchExecutor<
DB,
Input<'a> = BlockExecutionInput<'a, BlockWithSenders>,
Output = BundleStateWithReceipts,
Output = ExecutionOutcome,
Error = BlockExecutionError,
>;
@ -216,7 +216,7 @@ mod tests {
impl<DB> BatchExecutor<DB> for TestExecutor<DB> {
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BundleStateWithReceipts;
type Output = ExecutionOutcome;
type Error = BlockExecutionError;
fn execute_and_verify_one(&mut self, _input: Self::Input<'_>) -> Result<(), Self::Error> {

View File

@ -1,7 +1,7 @@
//! A no operation block executor implementation.
use reth_execution_errors::BlockExecutionError;
use reth_execution_types::BundleStateWithReceipts;
use reth_execution_types::ExecutionOutcome;
use reth_primitives::{BlockNumber, BlockWithSenders, Receipt};
use reth_prune_types::PruneModes;
use reth_storage_errors::provider::ProviderError;
@ -50,7 +50,7 @@ impl<DB> Executor<DB> for NoopBlockExecutorProvider {
impl<DB> BatchExecutor<DB> for NoopBlockExecutorProvider {
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BundleStateWithReceipts;
type Output = ExecutionOutcome;
type Error = BlockExecutionError;
fn execute_and_verify_one(&mut self, _: Self::Input<'_>) -> Result<(), Self::Error> {

View File

@ -5,7 +5,7 @@ use crate::execute::{
};
use parking_lot::Mutex;
use reth_execution_errors::BlockExecutionError;
use reth_execution_types::BundleStateWithReceipts;
use reth_execution_types::ExecutionOutcome;
use reth_primitives::{BlockNumber, BlockWithSenders, Receipt};
use reth_prune_types::PruneModes;
use reth_storage_errors::provider::ProviderError;
@ -15,12 +15,12 @@ use std::sync::Arc;
/// A [`BlockExecutorProvider`] that returns mocked execution results.
#[derive(Clone, Debug, Default)]
pub struct MockExecutorProvider {
exec_results: Arc<Mutex<Vec<BundleStateWithReceipts>>>,
exec_results: Arc<Mutex<Vec<ExecutionOutcome>>>,
}
impl MockExecutorProvider {
/// Extend the mocked execution results
pub fn extend(&self, results: impl IntoIterator<Item = impl Into<BundleStateWithReceipts>>) {
pub fn extend(&self, results: impl IntoIterator<Item = impl Into<ExecutionOutcome>>) {
self.exec_results.lock().extend(results.into_iter().map(Into::into));
}
}
@ -51,7 +51,7 @@ impl<DB> Executor<DB> for MockExecutorProvider {
type Error = BlockExecutionError;
fn execute(self, _: Self::Input<'_>) -> Result<Self::Output, Self::Error> {
let BundleStateWithReceipts { bundle, receipts, requests, first_block: _ } =
let ExecutionOutcome { bundle, receipts, requests, first_block: _ } =
self.exec_results.lock().pop().unwrap();
Ok(BlockExecutionOutput {
state: bundle,
@ -64,7 +64,7 @@ impl<DB> Executor<DB> for MockExecutorProvider {
impl<DB> BatchExecutor<DB> for MockExecutorProvider {
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BundleStateWithReceipts;
type Output = ExecutionOutcome;
type Error = BlockExecutionError;
fn execute_and_verify_one(&mut self, _: Self::Input<'_>) -> Result<(), Self::Error> {

View File

@ -13,7 +13,7 @@ use reth_primitives::{
BlockNumber, BlockWithSenders, ChainSpec, Hardfork, Header, Receipt, Receipts, TxType,
Withdrawals, U256,
};
use reth_provider::BundleStateWithReceipts;
use reth_provider::ExecutionOutcome;
use reth_prune_types::PruneModes;
use reth_revm::{
batch::{BlockBatchRecord, BlockExecutorStats},
@ -397,7 +397,7 @@ where
DB: Database<Error = ProviderError>,
{
type Input<'a> = BlockExecutionInput<'a, BlockWithSenders>;
type Output = BundleStateWithReceipts;
type Output = ExecutionOutcome;
type Error = BlockExecutionError;
fn execute_and_verify_one(&mut self, input: Self::Input<'_>) -> Result<(), Self::Error> {
@ -424,7 +424,7 @@ where
fn finalize(mut self) -> Self::Output {
self.stats.log_debug();
BundleStateWithReceipts::new(
ExecutionOutcome::new(
self.executor.state.take_bundle(),
self.batch_record.take_receipts(),
self.batch_record.first_block().unwrap_or_default(),

View File

@ -15,7 +15,7 @@ use reth_primitives::{
Block, ChainSpec, Hardfork, Header, IntoRecoveredTransaction, Receipt, TxType,
EMPTY_OMMER_ROOT_HASH, U256,
};
use reth_provider::{BundleStateWithReceipts, StateProviderFactory};
use reth_provider::{ExecutionOutcome, StateProviderFactory};
use reth_revm::database::StateProviderDatabase;
use reth_transaction_pool::{BestTransactionsAttributes, TransactionPool};
use revm::{
@ -506,25 +506,21 @@ where
// and 4788 contract call
db.merge_transitions(BundleRetention::PlainState);
let bundle = BundleStateWithReceipts::new(
db.take_bundle(),
vec![receipts].into(),
block_number,
Vec::new(),
);
let receipts_root = bundle
let execution_outcome =
ExecutionOutcome::new(db.take_bundle(), vec![receipts].into(), block_number, Vec::new());
let receipts_root = execution_outcome
.optimism_receipts_root_slow(
block_number,
chain_spec.as_ref(),
attributes.payload_attributes.timestamp,
)
.expect("Number is in range");
let logs_bloom = bundle.block_logs_bloom(block_number).expect("Number is in range");
let logs_bloom = execution_outcome.block_logs_bloom(block_number).expect("Number is in range");
// calculate the state root
let state_root = {
let state_provider = db.database.0.inner.borrow_mut();
state_provider.db.state_root(bundle.state())?
state_provider.db.state_root(execution_outcome.state())?
};
// create the block header

View File

@ -206,8 +206,8 @@ where
// extract the state from the notification and put it into the cache
let committed = new_state.committed();
let new_state = committed.state();
for (addr, acc) in new_state.bundle_accounts_iter() {
let new_execution_outcome = committed.execution_outcome();
for (addr, acc) in new_execution_outcome.bundle_accounts_iter() {
if let Some(info) = acc.info.clone() {
// we want pre cache existing accounts and their storage
// this only includes changed accounts and storage but is better than nothing

View File

@ -12,7 +12,7 @@ use reth_primitives::{
Block, BlockId, BlockNumberOrTag, ChainSpec, Header, IntoRecoveredTransaction, Receipt,
Requests, SealedBlockWithSenders, SealedHeader, B256, EMPTY_OMMER_ROOT_HASH, U256,
};
use reth_provider::{BundleStateWithReceipts, ChainSpecProvider, StateProviderFactory};
use reth_provider::{ChainSpecProvider, ExecutionOutcome, StateProviderFactory};
use reth_revm::{
database::StateProviderDatabase,
state_change::{
@ -220,7 +220,7 @@ impl PendingBlockEnv {
// merge all transitions into bundle state.
db.merge_transitions(BundleRetention::PlainState);
let bundle = BundleStateWithReceipts::new(
let execution_outcome = ExecutionOutcome::new(
db.take_bundle(),
vec![receipts].into(),
block_number,
@ -228,7 +228,7 @@ impl PendingBlockEnv {
);
#[cfg(feature = "optimism")]
let receipts_root = bundle
let receipts_root = execution_outcome
.optimism_receipts_root_slow(
block_number,
chain_spec.as_ref(),
@ -237,13 +237,15 @@ impl PendingBlockEnv {
.expect("Block is present");
#[cfg(not(feature = "optimism"))]
let receipts_root = bundle.receipts_root_slow(block_number).expect("Block is present");
let receipts_root =
execution_outcome.receipts_root_slow(block_number).expect("Block is present");
let logs_bloom = bundle.block_logs_bloom(block_number).expect("Block is present");
let logs_bloom =
execution_outcome.block_logs_bloom(block_number).expect("Block is present");
// calculate the state root
let state_provider = &db.database;
let state_root = state_provider.state_root(bundle.state())?;
let state_root = state_provider.state_root(execution_outcome.state())?;
// create the block header
let transactions_root = proofs::calculate_transaction_root(&executed_txs);

View File

@ -8,7 +8,7 @@ use reth_exex::{ExExManagerHandle, ExExNotification};
use reth_primitives::{BlockNumber, Header, StaticFileSegment};
use reth_provider::{
providers::{StaticFileProvider, StaticFileProviderRWRefMut, StaticFileWriter},
BlockReader, BundleStateWithReceipts, Chain, DatabaseProviderRW, HeaderProvider,
BlockReader, Chain, DatabaseProviderRW, ExecutionOutcome, HeaderProvider,
LatestStateProviderRef, OriginalValuesKnown, ProviderError, StateWriter, StatsReader,
TransactionVariant,
};
@ -297,9 +297,8 @@ where
// prepare execution output for writing
let time = Instant::now();
let BundleStateWithReceipts { bundle, receipts, requests, first_block } =
executor.finalize();
let state = BundleStateWithReceipts::new(bundle, receipts, first_block, requests);
let ExecutionOutcome { bundle, receipts, requests, first_block } = executor.finalize();
let state = ExecutionOutcome::new(bundle, receipts, first_block, requests);
let write_preparation_duration = time.elapsed();
// log the gas per second for the range we just executed

View File

@ -378,7 +378,7 @@ mod tests {
use assert_matches::assert_matches;
use reth_primitives::{BlockBody, SealedBlock, SealedBlockWithSenders, B256};
use reth_provider::{
BlockWriter, BundleStateWithReceipts, ProviderFactory, StaticFileProviderFactory,
BlockWriter, ExecutionOutcome, ProviderFactory, StaticFileProviderFactory,
};
use reth_stages_api::StageUnitCheckpoint;
use reth_testing_utils::generators::{self, random_header, random_header_range};
@ -629,7 +629,7 @@ mod tests {
provider
.append_blocks_with_state(
sealed_blocks,
BundleStateWithReceipts::default(),
ExecutionOutcome::default(),
HashedPostState::default(),
TrieUpdates::default(),
None,

View File

@ -13,9 +13,9 @@ use reth_provider::{
bundle_state::{BundleStateInit, RevertsInit},
errors::provider::ProviderResult,
providers::{StaticFileProvider, StaticFileWriter},
BlockHashReader, BlockNumReader, BundleStateWithReceipts, ChainSpecProvider,
DatabaseProviderRW, HashingWriter, HistoryWriter, OriginalValuesKnown, ProviderError,
ProviderFactory, StageCheckpointWriter, StateWriter, StaticFileProviderFactory,
BlockHashReader, BlockNumReader, ChainSpecProvider, DatabaseProviderRW, ExecutionOutcome,
HashingWriter, HistoryWriter, OriginalValuesKnown, ProviderError, ProviderFactory,
StageCheckpointWriter, StateWriter, StaticFileProviderFactory,
};
use reth_stages_types::{StageCheckpoint, StageId};
use reth_trie::{IntermediateStateRootState, StateRoot as StateRootComputer, StateRootProgress};
@ -193,7 +193,7 @@ pub fn insert_state<'a, 'b, DB: Database>(
}
let all_reverts_init: RevertsInit = HashMap::from([(block, reverts_init)]);
let bundle = BundleStateWithReceipts::new_init(
let execution_outcome = ExecutionOutcome::new_init(
state_init,
all_reverts_init,
contracts.into_iter().collect(),
@ -202,7 +202,7 @@ pub fn insert_state<'a, 'b, DB: Database>(
Vec::new(),
);
bundle.write_to_storage(tx, None, OriginalValuesKnown::Yes)?;
execution_outcome.write_to_storage(tx, None, OriginalValuesKnown::Yes)?;
trace!(target: "reth::cli", "Inserted state");

View File

@ -9,7 +9,7 @@ use reth_primitives::StaticFileSegment;
use reth_storage_errors::provider::{ProviderError, ProviderResult};
pub use revm::db::states::OriginalValuesKnown;
impl StateWriter for BundleStateWithReceipts {
impl StateWriter for ExecutionOutcome {
fn write_to_storage<TX>(
self,
tx: &TX,
@ -288,7 +288,7 @@ mod tests {
state.merge_transitions(BundleRetention::Reverts);
BundleStateWithReceipts::new(state.take_bundle(), Receipts::default(), 1, Vec::new())
ExecutionOutcome::new(state.take_bundle(), Receipts::default(), 1, Vec::new())
.write_to_storage(provider.tx_ref(), None, OriginalValuesKnown::Yes)
.expect("Could not write bundle state to DB");
@ -386,7 +386,7 @@ mod tests {
)]));
state.merge_transitions(BundleRetention::Reverts);
BundleStateWithReceipts::new(state.take_bundle(), Receipts::default(), 2, Vec::new())
ExecutionOutcome::new(state.take_bundle(), Receipts::default(), 2, Vec::new())
.write_to_storage(provider.tx_ref(), None, OriginalValuesKnown::Yes)
.expect("Could not write bundle state to DB");
@ -450,7 +450,7 @@ mod tests {
},
)]));
init_state.merge_transitions(BundleRetention::Reverts);
BundleStateWithReceipts::new(init_state.take_bundle(), Receipts::default(), 0, Vec::new())
ExecutionOutcome::new(init_state.take_bundle(), Receipts::default(), 0, Vec::new())
.write_to_storage(provider.tx_ref(), None, OriginalValuesKnown::Yes)
.expect("Could not write init bundle state to DB");
@ -592,7 +592,7 @@ mod tests {
let bundle = state.take_bundle();
BundleStateWithReceipts::new(bundle, Receipts::default(), 1, Vec::new())
ExecutionOutcome::new(bundle, Receipts::default(), 1, Vec::new())
.write_to_storage(provider.tx_ref(), None, OriginalValuesKnown::Yes)
.expect("Could not write bundle state to DB");
@ -755,7 +755,7 @@ mod tests {
},
)]));
init_state.merge_transitions(BundleRetention::Reverts);
BundleStateWithReceipts::new(init_state.take_bundle(), Receipts::default(), 0, Vec::new())
ExecutionOutcome::new(init_state.take_bundle(), Receipts::default(), 0, Vec::new())
.write_to_storage(provider.tx_ref(), None, OriginalValuesKnown::Yes)
.expect("Could not write init bundle state to DB");
@ -800,7 +800,7 @@ mod tests {
// Commit block #1 changes to the database.
state.merge_transitions(BundleRetention::Reverts);
BundleStateWithReceipts::new(state.take_bundle(), Receipts::default(), 1, Vec::new())
ExecutionOutcome::new(state.take_bundle(), Receipts::default(), 1, Vec::new())
.write_to_storage(provider.tx_ref(), None, OriginalValuesKnown::Yes)
.expect("Could not write bundle state to DB");
@ -830,7 +830,7 @@ mod tests {
#[test]
fn revert_to_indices() {
let base = BundleStateWithReceipts {
let base = ExecutionOutcome {
bundle: BundleState::default(),
receipts: vec![vec![Some(Receipt::default()); 2]; 7].into(),
first_block: 10,
@ -896,7 +896,7 @@ mod tests {
let assert_state_root = |state: &State<EmptyDB>, expected: &PreState, msg| {
assert_eq!(
BundleStateWithReceipts::new(
ExecutionOutcome::new(
state.bundle_state.clone(),
Receipts::default(),
0,
@ -1047,7 +1047,7 @@ mod tests {
.build();
assert_eq!(previous_state.reverts.len(), 1);
let mut test = BundleStateWithReceipts {
let mut test = ExecutionOutcome {
bundle: present_state,
receipts: vec![vec![Some(Receipt::default()); 2]; 1].into(),
first_block: 2,

View File

@ -6,7 +6,7 @@ mod state_changes;
mod state_reverts;
pub use bundle_state_with_receipts::{
AccountRevertInit, BundleStateInit, BundleStateWithReceipts, OriginalValuesKnown, RevertsInit,
AccountRevertInit, BundleStateInit, ExecutionOutcome, OriginalValuesKnown, RevertsInit,
};
pub use hashed_state_changes::HashedStateChanges;
pub use state_changes::StateChanges;

View File

@ -1,35 +1,39 @@
use crate::{
AccountReader, BlockHashReader, BundleStateDataProvider, StateProvider, StateRootProvider,
AccountReader, BlockHashReader, ExecutionDataProvider, StateProvider, StateRootProvider,
};
use reth_primitives::{proofs::AccountProof, Account, Address, BlockNumber, Bytecode, B256};
use reth_storage_errors::provider::{ProviderError, ProviderResult};
use reth_trie::updates::TrieUpdates;
use revm::db::BundleState;
/// A state provider that either resolves to data in a wrapped [`crate::BundleStateWithReceipts`],
/// A state provider that resolves to data from either a wrapped [`crate::ExecutionOutcome`]
/// or an underlying state provider.
///
/// This struct combines two sources of state data: the execution outcome and an underlying
/// state provider. It can provide state information by leveraging both the post-block execution
/// changes and the pre-existing state data.
#[derive(Debug)]
pub struct BundleStateProvider<SP: StateProvider, BSDP: BundleStateDataProvider> {
pub struct BundleStateProvider<SP: StateProvider, EDP: ExecutionDataProvider> {
/// The inner state provider.
pub state_provider: SP,
/// Bundle state data.
pub bundle_state_data_provider: BSDP,
/// Block execution data.
pub block_execution_data_provider: EDP,
}
impl<SP: StateProvider, BSDP: BundleStateDataProvider> BundleStateProvider<SP, BSDP> {
impl<SP: StateProvider, EDP: ExecutionDataProvider> BundleStateProvider<SP, EDP> {
/// Create new bundle state provider
pub const fn new(state_provider: SP, bundle_state_data_provider: BSDP) -> Self {
Self { state_provider, bundle_state_data_provider }
pub const fn new(state_provider: SP, block_execution_data_provider: EDP) -> Self {
Self { state_provider, block_execution_data_provider }
}
}
/* Implement StateProvider traits */
impl<SP: StateProvider, BSDP: BundleStateDataProvider> BlockHashReader
for BundleStateProvider<SP, BSDP>
impl<SP: StateProvider, EDP: ExecutionDataProvider> BlockHashReader
for BundleStateProvider<SP, EDP>
{
fn block_hash(&self, block_number: BlockNumber) -> ProviderResult<Option<B256>> {
let block_hash = self.bundle_state_data_provider.block_hash(block_number);
let block_hash = self.block_execution_data_provider.block_hash(block_number);
if block_hash.is_some() {
return Ok(block_hash)
}
@ -45,11 +49,11 @@ impl<SP: StateProvider, BSDP: BundleStateDataProvider> BlockHashReader
}
}
impl<SP: StateProvider, BSDP: BundleStateDataProvider> AccountReader
for BundleStateProvider<SP, BSDP>
{
impl<SP: StateProvider, EDP: ExecutionDataProvider> AccountReader for BundleStateProvider<SP, EDP> {
fn basic_account(&self, address: Address) -> ProviderResult<Option<Account>> {
if let Some(account) = self.bundle_state_data_provider.state().account(&address) {
if let Some(account) =
self.block_execution_data_provider.execution_outcome().account(&address)
{
Ok(account)
} else {
self.state_provider.basic_account(address)
@ -57,11 +61,11 @@ impl<SP: StateProvider, BSDP: BundleStateDataProvider> AccountReader
}
}
impl<SP: StateProvider, BSDP: BundleStateDataProvider> StateRootProvider
for BundleStateProvider<SP, BSDP>
impl<SP: StateProvider, EDP: ExecutionDataProvider> StateRootProvider
for BundleStateProvider<SP, EDP>
{
fn state_root(&self, bundle_state: &BundleState) -> ProviderResult<B256> {
let mut state = self.bundle_state_data_provider.state().state().clone();
let mut state = self.block_execution_data_provider.execution_outcome().state().clone();
state.extend(bundle_state.clone());
self.state_provider.state_root(&state)
}
@ -70,23 +74,23 @@ impl<SP: StateProvider, BSDP: BundleStateDataProvider> StateRootProvider
&self,
bundle_state: &BundleState,
) -> ProviderResult<(B256, TrieUpdates)> {
let mut state = self.bundle_state_data_provider.state().state().clone();
let mut state = self.block_execution_data_provider.execution_outcome().state().clone();
state.extend(bundle_state.clone());
self.state_provider.state_root_with_updates(&state)
}
}
impl<SP: StateProvider, BSDP: BundleStateDataProvider> StateProvider
for BundleStateProvider<SP, BSDP>
{
impl<SP: StateProvider, EDP: ExecutionDataProvider> StateProvider for BundleStateProvider<SP, EDP> {
fn storage(
&self,
account: Address,
storage_key: reth_primitives::StorageKey,
) -> ProviderResult<Option<reth_primitives::StorageValue>> {
let u256_storage_key = storage_key.into();
if let Some(value) =
self.bundle_state_data_provider.state().storage(&account, u256_storage_key)
if let Some(value) = self
.block_execution_data_provider
.execution_outcome()
.storage(&account, u256_storage_key)
{
return Ok(Some(value))
}
@ -95,7 +99,9 @@ impl<SP: StateProvider, BSDP: BundleStateDataProvider> StateProvider
}
fn bytecode_by_hash(&self, code_hash: B256) -> ProviderResult<Option<Bytecode>> {
if let Some(bytecode) = self.bundle_state_data_provider.state().bytecode(&code_hash) {
if let Some(bytecode) =
self.block_execution_data_provider.execution_outcome().bytecode(&code_hash)
{
return Ok(Some(bytecode))
}

View File

@ -1,5 +1,5 @@
use crate::{
bundle_state::{BundleStateInit, BundleStateWithReceipts, HashedStateChanges, RevertsInit},
bundle_state::{BundleStateInit, ExecutionOutcome, HashedStateChanges, RevertsInit},
providers::{database::metrics, static_file::StaticFileWriter, StaticFileProvider},
to_range,
traits::{
@ -366,7 +366,7 @@ impl<TX: DbTxMut + DbTx> DatabaseProvider<TX> {
}
// TODO(joshie) TEMPORARY should be moved to trait providers
/// Unwind or peek at last N blocks of state recreating the [`BundleStateWithReceipts`].
/// Unwind or peek at last N blocks of state recreating the [`ExecutionOutcome`].
///
/// If UNWIND it set to true tip and latest state will be unwind
/// and returned back with all the blocks
@ -393,9 +393,9 @@ impl<TX: DbTxMut + DbTx> DatabaseProvider<TX> {
pub fn unwind_or_peek_state<const TAKE: bool>(
&self,
range: RangeInclusive<BlockNumber>,
) -> ProviderResult<BundleStateWithReceipts> {
) -> ProviderResult<ExecutionOutcome> {
if range.is_empty() {
return Ok(BundleStateWithReceipts::default())
return Ok(ExecutionOutcome::default())
}
let start_block_number = *range.start();
@ -533,7 +533,7 @@ impl<TX: DbTxMut + DbTx> DatabaseProvider<TX> {
receipts.push(block_receipts);
}
Ok(BundleStateWithReceipts::new_init(
Ok(ExecutionOutcome::new_init(
state,
reverts,
Vec::new(),
@ -2673,7 +2673,7 @@ impl<TX: DbTxMut + DbTx> BlockWriter for DatabaseProvider<TX> {
fn append_blocks_with_state(
&self,
blocks: Vec<SealedBlockWithSenders>,
state: BundleStateWithReceipts,
execution_outcome: ExecutionOutcome,
hashed_state: HashedPostState,
trie_updates: TrieUpdates,
prune_modes: Option<&PruneModes>,
@ -2698,7 +2698,7 @@ impl<TX: DbTxMut + DbTx> BlockWriter for DatabaseProvider<TX> {
// Write state and changesets to the database.
// Must be written after blocks because of the receipt lookup.
state.write_to_storage(self.tx_ref(), None, OriginalValuesKnown::No)?;
execution_outcome.write_to_storage(self.tx_ref(), None, OriginalValuesKnown::No)?;
durations_recorder.record_relative(metrics::Action::InsertState);
// insert hashes and intermediate merkle nodes

View File

@ -2,7 +2,7 @@ use crate::{
AccountReader, BlockHashReader, BlockIdReader, BlockNumReader, BlockReader, BlockReaderIdExt,
BlockSource, BlockchainTreePendingStateProvider, CanonChainTracker, CanonStateNotifications,
CanonStateSubscriptions, ChainSpecProvider, ChangeSetReader, DatabaseProviderFactory,
EvmEnvProvider, FullBundleStateDataProvider, HeaderProvider, ProviderError,
EvmEnvProvider, FullExecutionDataProvider, HeaderProvider, ProviderError,
PruneCheckpointReader, ReceiptProvider, ReceiptProviderIdExt, RequestsProvider,
StageCheckpointReader, StateProviderBox, StateProviderFactory, StaticFileProviderFactory,
TransactionVariant, TransactionsProvider, TreeViewer, WithdrawalsProvider,
@ -657,7 +657,7 @@ where
fn pending_with_provider(
&self,
bundle_state_data: Box<dyn FullBundleStateDataProvider>,
bundle_state_data: Box<dyn FullExecutionDataProvider>,
) -> ProviderResult<StateProviderBox> {
let canonical_fork = bundle_state_data.canonical_fork();
trace!(target: "providers::blockchain", ?canonical_fork, "Returning post state provider");
@ -876,7 +876,7 @@ where
fn find_pending_state_provider(
&self,
block_hash: BlockHash,
) -> Option<Box<dyn FullBundleStateDataProvider>> {
) -> Option<Box<dyn FullExecutionDataProvider>> {
self.tree.find_pending_state_provider(block_hash)
}
}

View File

@ -1,5 +1,5 @@
//! Dummy blocks and data for tests
use crate::{BundleStateWithReceipts, DatabaseProviderRW};
use crate::{DatabaseProviderRW, ExecutionOutcome};
use alloy_primitives::Log;
use alloy_rlp::Decodable;
use reth_db::tables;
@ -68,21 +68,21 @@ pub struct BlockchainTestData {
/// Genesis
pub genesis: SealedBlock,
/// Blocks with its execution result
pub blocks: Vec<(SealedBlockWithSenders, BundleStateWithReceipts)>,
pub blocks: Vec<(SealedBlockWithSenders, ExecutionOutcome)>,
}
impl BlockchainTestData {
/// Create test data with two blocks that are connected, specifying their block numbers.
pub fn default_from_number(first: BlockNumber) -> Self {
let one = block1(first);
let mut extended_state = one.1.clone();
let two = block2(first + 1, one.0.hash(), &extended_state);
extended_state.extend(two.1.clone());
let three = block3(first + 2, two.0.hash(), &extended_state);
extended_state.extend(three.1.clone());
let four = block4(first + 3, three.0.hash(), &extended_state);
extended_state.extend(four.1.clone());
let five = block5(first + 4, four.0.hash(), &extended_state);
let mut extended_execution_outcome = one.1.clone();
let two = block2(first + 1, one.0.hash(), &extended_execution_outcome);
extended_execution_outcome.extend(two.1.clone());
let three = block3(first + 2, two.0.hash(), &extended_execution_outcome);
extended_execution_outcome.extend(three.1.clone());
let four = block4(first + 3, three.0.hash(), &extended_execution_outcome);
extended_execution_outcome.extend(four.1.clone());
let five = block5(first + 4, four.0.hash(), &extended_execution_outcome);
Self { genesis: genesis(), blocks: vec![one, two, three, four, five] }
}
}
@ -90,14 +90,14 @@ impl BlockchainTestData {
impl Default for BlockchainTestData {
fn default() -> Self {
let one = block1(1);
let mut extended_state = one.1.clone();
let two = block2(2, one.0.hash(), &extended_state);
extended_state.extend(two.1.clone());
let three = block3(3, two.0.hash(), &extended_state);
extended_state.extend(three.1.clone());
let four = block4(4, three.0.hash(), &extended_state);
extended_state.extend(four.1.clone());
let five = block5(5, four.0.hash(), &extended_state);
let mut extended_execution_outcome = one.1.clone();
let two = block2(2, one.0.hash(), &extended_execution_outcome);
extended_execution_outcome.extend(two.1.clone());
let three = block3(3, two.0.hash(), &extended_execution_outcome);
extended_execution_outcome.extend(three.1.clone());
let four = block4(4, three.0.hash(), &extended_execution_outcome);
extended_execution_outcome.extend(four.1.clone());
let five = block5(5, four.0.hash(), &extended_execution_outcome);
Self { genesis: genesis(), blocks: vec![one, two, three, four, five] }
}
}
@ -114,35 +114,37 @@ pub fn genesis() -> SealedBlock {
}
}
fn bundle_state_root(state: &BundleStateWithReceipts) -> B256 {
state_root_unhashed(state.bundle_accounts_iter().filter_map(|(address, account)| {
account.info.as_ref().map(|info| {
(
address,
fn bundle_state_root(execution_outcome: &ExecutionOutcome) -> B256 {
state_root_unhashed(execution_outcome.bundle_accounts_iter().filter_map(
|(address, account)| {
account.info.as_ref().map(|info| {
(
into_reth_acc(info.clone()),
storage_root_unhashed(
account
.storage
.iter()
.filter(|(_, value)| !value.present_value.is_zero())
.map(|(slot, value)| ((*slot).into(), value.present_value)),
address,
(
into_reth_acc(info.clone()),
storage_root_unhashed(
account
.storage
.iter()
.filter(|(_, value)| !value.present_value.is_zero())
.map(|(slot, value)| ((*slot).into(), value.present_value)),
),
),
),
)
})
}))
)
})
},
))
}
/// Block one that points to genesis
fn block1(number: BlockNumber) -> (SealedBlockWithSenders, BundleStateWithReceipts) {
fn block1(number: BlockNumber) -> (SealedBlockWithSenders, ExecutionOutcome) {
// block changes
let account1: Address = [0x60; 20].into();
let account2: Address = [0x61; 20].into();
let slot = U256::from(5);
let info = AccountInfo { nonce: 1, balance: U256::from(10), ..Default::default() };
let bundle = BundleStateWithReceipts::new(
let execution_outcome = ExecutionOutcome::new(
BundleState::builder(number..=number)
.state_present_account_info(account1, info.clone())
.revert_account_info(number, account1, Some(None))
@ -169,7 +171,7 @@ fn block1(number: BlockNumber) -> (SealedBlockWithSenders, BundleStateWithReceip
Vec::new(),
);
let state_root = bundle_state_root(&bundle);
let state_root = bundle_state_root(&execution_outcome);
assert_eq!(
state_root,
b256!("5d035ccb3e75a9057452ff060b773b213ec1fc353426174068edfc3971a0b6bd")
@ -183,20 +185,20 @@ fn block1(number: BlockNumber) -> (SealedBlockWithSenders, BundleStateWithReceip
header.parent_hash = B256::ZERO;
block.header = header.seal_slow();
(SealedBlockWithSenders { block, senders: vec![Address::new([0x30; 20])] }, bundle)
(SealedBlockWithSenders { block, senders: vec![Address::new([0x30; 20])] }, execution_outcome)
}
/// Block two that points to block 1
fn block2(
number: BlockNumber,
parent_hash: B256,
prev_state: &BundleStateWithReceipts,
) -> (SealedBlockWithSenders, BundleStateWithReceipts) {
prev_execution_outcome: &ExecutionOutcome,
) -> (SealedBlockWithSenders, ExecutionOutcome) {
// block changes
let account: Address = [0x60; 20].into();
let slot = U256::from(5);
let bundle = BundleStateWithReceipts::new(
let execution_outcome = ExecutionOutcome::new(
BundleState::builder(number..=number)
.state_present_account_info(
account,
@ -229,8 +231,8 @@ fn block2(
Vec::new(),
);
let mut extended = prev_state.clone();
extended.extend(bundle.clone());
let mut extended = prev_execution_outcome.clone();
extended.extend(execution_outcome.clone());
let state_root = bundle_state_root(&extended);
assert_eq!(
state_root,
@ -247,15 +249,15 @@ fn block2(
header.parent_hash = parent_hash;
block.header = header.seal_slow();
(SealedBlockWithSenders { block, senders: vec![Address::new([0x31; 20])] }, bundle)
(SealedBlockWithSenders { block, senders: vec![Address::new([0x31; 20])] }, execution_outcome)
}
/// Block three that points to block 2
fn block3(
number: BlockNumber,
parent_hash: B256,
prev_state: &BundleStateWithReceipts,
) -> (SealedBlockWithSenders, BundleStateWithReceipts) {
prev_execution_outcome: &ExecutionOutcome,
) -> (SealedBlockWithSenders, ExecutionOutcome) {
let address_range = 1..=20;
let slot_range = 1..=100;
@ -278,7 +280,7 @@ fn block3(
.revert_account_info(number, address, Some(None))
.revert_storage(number, address, Vec::new());
}
let bundle = BundleStateWithReceipts::new(
let execution_outcome = ExecutionOutcome::new(
bundle_state_builder.build(),
vec![vec![Some(Receipt {
tx_type: TxType::Eip1559,
@ -299,8 +301,8 @@ fn block3(
Vec::new(),
);
let mut extended = prev_state.clone();
extended.extend(bundle.clone());
let mut extended = prev_execution_outcome.clone();
extended.extend(execution_outcome.clone());
let state_root = bundle_state_root(&extended);
let mut block = SealedBlock::decode(&mut BLOCK_RLP.as_slice()).unwrap();
@ -312,15 +314,15 @@ fn block3(
header.parent_hash = parent_hash;
block.header = header.seal_slow();
(SealedBlockWithSenders { block, senders: vec![Address::new([0x31; 20])] }, bundle)
(SealedBlockWithSenders { block, senders: vec![Address::new([0x31; 20])] }, execution_outcome)
}
/// Block four that points to block 3
fn block4(
number: BlockNumber,
parent_hash: B256,
prev_state: &BundleStateWithReceipts,
) -> (SealedBlockWithSenders, BundleStateWithReceipts) {
prev_execution_outcome: &ExecutionOutcome,
) -> (SealedBlockWithSenders, ExecutionOutcome) {
let address_range = 1..=20;
let slot_range = 1..=100;
@ -369,7 +371,7 @@ fn block4(
Vec::from_iter(slot_range.clone().map(|slot| (U256::from(slot), U256::from(slot)))),
);
}
let bundle = BundleStateWithReceipts::new(
let execution_outcome = ExecutionOutcome::new(
bundle_state_builder.build(),
vec![vec![Some(Receipt {
tx_type: TxType::Eip1559,
@ -390,8 +392,8 @@ fn block4(
Vec::new(),
);
let mut extended = prev_state.clone();
extended.extend(bundle.clone());
let mut extended = prev_execution_outcome.clone();
extended.extend(execution_outcome.clone());
let state_root = bundle_state_root(&extended);
let mut block = SealedBlock::decode(&mut BLOCK_RLP.as_slice()).unwrap();
@ -403,15 +405,15 @@ fn block4(
header.parent_hash = parent_hash;
block.header = header.seal_slow();
(SealedBlockWithSenders { block, senders: vec![Address::new([0x31; 20])] }, bundle)
(SealedBlockWithSenders { block, senders: vec![Address::new([0x31; 20])] }, execution_outcome)
}
/// Block five that points to block 4
fn block5(
number: BlockNumber,
parent_hash: B256,
prev_state: &BundleStateWithReceipts,
) -> (SealedBlockWithSenders, BundleStateWithReceipts) {
prev_execution_outcome: &ExecutionOutcome,
) -> (SealedBlockWithSenders, ExecutionOutcome) {
let address_range = 1..=20;
let slot_range = 1..=100;
@ -455,7 +457,7 @@ fn block5(
bundle_state_builder.revert_address(number, address)
};
}
let bundle = BundleStateWithReceipts::new(
let execution_outcome = ExecutionOutcome::new(
bundle_state_builder.build(),
vec![vec![Some(Receipt {
tx_type: TxType::Eip1559,
@ -476,8 +478,8 @@ fn block5(
Vec::new(),
);
let mut extended = prev_state.clone();
extended.extend(bundle.clone());
let mut extended = prev_execution_outcome.clone();
extended.extend(execution_outcome.clone());
let state_root = bundle_state_root(&extended);
let mut block = SealedBlock::decode(&mut BLOCK_RLP.as_slice()).unwrap();
@ -489,5 +491,5 @@ fn block5(
header.parent_hash = parent_hash;
block.header = header.seal_slow();
(SealedBlockWithSenders { block, senders: vec![Address::new([0x31; 20])] }, bundle)
(SealedBlockWithSenders { block, senders: vec![Address::new([0x31; 20])] }, execution_outcome)
}

View File

@ -1,10 +1,9 @@
use crate::{
traits::{BlockSource, ReceiptProvider},
AccountReader, BlockHashReader, BlockIdReader, BlockNumReader, BlockReader, BlockReaderIdExt,
ChainSpecProvider, ChangeSetReader, EvmEnvProvider, FullBundleStateDataProvider,
HeaderProvider, ReceiptProviderIdExt, RequestsProvider, StateProvider, StateProviderBox,
StateProviderFactory, StateRootProvider, TransactionVariant, TransactionsProvider,
WithdrawalsProvider,
ChainSpecProvider, ChangeSetReader, EvmEnvProvider, FullExecutionDataProvider, HeaderProvider,
ReceiptProviderIdExt, RequestsProvider, StateProvider, StateProviderBox, StateProviderFactory,
StateRootProvider, TransactionVariant, TransactionsProvider, WithdrawalsProvider,
};
use parking_lot::Mutex;
use reth_db_api::models::{AccountBeforeTx, StoredBlockBodyIndices};
@ -661,7 +660,7 @@ impl StateProviderFactory for MockEthProvider {
fn pending_with_provider<'a>(
&'a self,
_bundle_state_data: Box<dyn FullBundleStateDataProvider + 'a>,
_bundle_state_data: Box<dyn FullExecutionDataProvider + 'a>,
) -> ProviderResult<StateProviderBox> {
Ok(Box::new(self.clone()))
}

View File

@ -419,7 +419,7 @@ impl StateProviderFactory for NoopProvider {
fn pending_with_provider<'a>(
&'a self,
_bundle_state_data: Box<dyn crate::FullBundleStateDataProvider + 'a>,
_bundle_state_data: Box<dyn crate::FullExecutionDataProvider + 'a>,
) -> ProviderResult<StateProviderBox> {
Ok(Box::new(*self))
}

View File

@ -1,4 +1,4 @@
use crate::{BundleStateWithReceipts, Chain};
use crate::{Chain, ExecutionOutcome};
use reth_db_api::models::StoredBlockBodyIndices;
use reth_primitives::{BlockNumber, SealedBlockWithSenders};
use reth_prune_types::PruneModes;
@ -65,7 +65,7 @@ pub trait BlockWriter: Send + Sync {
fn append_blocks_with_state(
&self,
blocks: Vec<SealedBlockWithSenders>,
state: BundleStateWithReceipts,
execution_outcome: ExecutionOutcome,
hashed_state: HashedPostState,
trie_updates: TrieUpdates,
prune_modes: Option<&PruneModes>,

View File

@ -60,7 +60,7 @@ impl Stream for CanonStateNotificationStream {
}
/// Chain action that is triggered when a new block is imported or old block is reverted.
/// and will return all [`crate::BundleStateWithReceipts`] and
/// and will return all [`crate::ExecutionOutcome`] and
/// [`reth_primitives::SealedBlockWithSenders`] of both reverted and committed blocks.
#[derive(Clone, Debug)]
pub enum CanonStateNotification {

View File

@ -3,7 +3,7 @@ use reth_db_api::transaction::{DbTx, DbTxMut};
use reth_storage_errors::provider::ProviderResult;
use revm::db::OriginalValuesKnown;
/// A helper trait for [`BundleStateWithReceipts`](reth_execution_types::BundleStateWithReceipts) to
/// A helper trait for [`ExecutionOutcome`](reth_execution_types::ExecutionOutcome) to
/// write state and receipts to storage.
pub trait StateWriter {
/// Write the data and receipts to the database or static files if `static_file_producer` is

View File

@ -1,6 +1,6 @@
use super::{AccountReader, BlockHashReader, BlockIdReader, StateRootProvider};
use auto_impl::auto_impl;
use reth_execution_types::BundleStateWithReceipts;
use reth_execution_types::ExecutionOutcome;
use reth_primitives::{
proofs::AccountProof, Address, BlockHash, BlockId, BlockNumHash, BlockNumber, BlockNumberOrTag,
Bytecode, StorageKey, StorageValue, B256, KECCAK_EMPTY, U256,
@ -178,7 +178,7 @@ pub trait StateProviderFactory: BlockIdReader + Send + Sync {
/// Used to inspect or execute transaction on the pending state.
fn pending_with_provider(
&self,
bundle_state_data: Box<dyn FullBundleStateDataProvider>,
bundle_state_data: Box<dyn FullExecutionDataProvider>,
) -> ProviderResult<StateProviderBox>;
}
@ -192,7 +192,7 @@ pub trait BlockchainTreePendingStateProvider: Send + Sync {
fn pending_state_provider(
&self,
block_hash: BlockHash,
) -> ProviderResult<Box<dyn FullBundleStateDataProvider>> {
) -> ProviderResult<Box<dyn FullExecutionDataProvider>> {
self.find_pending_state_provider(block_hash)
.ok_or(ProviderError::StateForHashNotFound(block_hash))
}
@ -201,24 +201,28 @@ pub trait BlockchainTreePendingStateProvider: Send + Sync {
fn find_pending_state_provider(
&self,
block_hash: BlockHash,
) -> Option<Box<dyn FullBundleStateDataProvider>>;
) -> Option<Box<dyn FullExecutionDataProvider>>;
}
/// Post state data needed for execution on it.
/// Provides data required for post-block execution.
///
/// State contains:
/// * [`BundleStateWithReceipts`] contains all changed of accounts and storage of pending chain
/// * block hashes of pending chain and canonical blocks.
/// This trait offers methods to access essential post-execution data, including the state changes
/// in accounts and storage, as well as block hashes for both the pending and canonical chains.
///
/// The trait includes:
/// * [`ExecutionOutcome`] - Captures all account and storage changes in the pending chain.
/// * Block hashes - Provides access to the block hashes of both the pending chain and canonical
/// blocks.
#[auto_impl(&, Box)]
pub trait BundleStateDataProvider: Send + Sync {
/// Return post state
fn state(&self) -> &BundleStateWithReceipts;
pub trait ExecutionDataProvider: Send + Sync {
/// Return the execution outcome.
fn execution_outcome(&self) -> &ExecutionOutcome;
/// Return block hash by block number of pending or canonical chain.
fn block_hash(&self, block_number: BlockNumber) -> Option<BlockHash>;
}
impl BundleStateDataProvider for BundleStateWithReceipts {
fn state(&self) -> &BundleStateWithReceipts {
impl ExecutionDataProvider for ExecutionOutcome {
fn execution_outcome(&self) -> &ExecutionOutcome {
self
}
@ -232,22 +236,22 @@ impl BundleStateDataProvider for BundleStateWithReceipts {
///
/// It contains a canonical fork, the block on what pending chain was forked from.
#[auto_impl(&, Box)]
pub trait BundleStateForkProvider {
pub trait BlockExecutionForkProvider {
/// Return canonical fork, the block on what post state was forked from.
///
/// Needed to create state provider.
fn canonical_fork(&self) -> BlockNumHash;
}
/// Full post state data needed for execution on it.
/// This trait is used to create a state provider over pending state.
/// Provides comprehensive post-execution state data required for further execution.
///
/// This trait is a combination of [`BundleStateDataProvider`] and [`BundleStateForkProvider`].
/// This trait is used to create a state provider over the pending state and is a combination of
/// [`ExecutionDataProvider`] and [`BlockExecutionForkProvider`].
///
/// Pending state contains:
/// * [`BundleStateWithReceipts`] contains all changed of accounts and storage of pending chain
/// * block hashes of pending chain and canonical blocks.
/// * canonical fork, the block on what pending chain was forked from.
pub trait FullBundleStateDataProvider: BundleStateDataProvider + BundleStateForkProvider {}
/// The pending state includes:
/// * `ExecutionOutcome`: Contains all changes to accounts and storage within the pending chain.
/// * Block hashes: Represents hashes of both the pending chain and canonical blocks.
/// * Canonical fork: Denotes the block from which the pending chain forked.
pub trait FullExecutionDataProvider: ExecutionDataProvider + BlockExecutionForkProvider {}
impl<T> FullBundleStateDataProvider for T where T: BundleStateDataProvider + BundleStateForkProvider {}
impl<T> FullExecutionDataProvider for T where T: ExecutionDataProvider + BlockExecutionForkProvider {}

View File

@ -18,8 +18,8 @@ use reth_primitives::{
TryFromRecoveredTransaction,
};
use reth_provider::{
BlockReaderIdExt, BundleStateWithReceipts, CanonStateNotification, ChainSpecProvider,
ProviderError, StateProviderFactory,
BlockReaderIdExt, CanonStateNotification, ChainSpecProvider, ExecutionOutcome, ProviderError,
StateProviderFactory,
};
use reth_tasks::TaskSpawner;
use std::{
@ -556,9 +556,9 @@ where
/// Extracts all changed accounts from the `BundleState`
fn changed_accounts_iter(
state: &BundleStateWithReceipts,
execution_outcome: &ExecutionOutcome,
) -> impl Iterator<Item = ChangedAccount> + '_ {
state
execution_outcome
.accounts_iter()
.filter_map(|(addr, acc)| acc.map(|acc| (addr, acc)))
.map(|(address, acc)| ChangedAccount { address, nonce: acc.nonce, balance: acc.balance })

View File

@ -1,6 +1,6 @@
#![warn(unused_crate_dependencies)]
use reth::providers::BundleStateWithReceipts;
use reth::providers::ExecutionOutcome;
use reth_exex::{ExExContext, ExExEvent, ExExNotification};
use reth_node_api::FullNodeComponents;
use reth_node_ethereum::EthereumNode;
@ -15,14 +15,14 @@ use std::{
struct InMemoryStateExEx<Node: FullNodeComponents> {
/// The context of the ExEx
ctx: ExExContext<Node>,
/// Entire plain state of the chain
state: BundleStateWithReceipts,
/// Execution outcome of the chain
execution_outcome: ExecutionOutcome,
}
impl<Node: FullNodeComponents> InMemoryStateExEx<Node> {
/// Create a new instance of the ExEx
fn new(ctx: ExExContext<Node>) -> Self {
Self { ctx, state: BundleStateWithReceipts::default() }
Self { ctx, execution_outcome: ExecutionOutcome::default() }
}
}
@ -39,18 +39,18 @@ impl<Node: FullNodeComponents + Unpin> Future for InMemoryStateExEx<Node> {
}
ExExNotification::ChainReorged { old, new } => {
// revert to block before the reorg
this.state.revert_to(new.first().number - 1);
this.execution_outcome.revert_to(new.first().number - 1);
info!(from_chain = ?old.range(), to_chain = ?new.range(), "Received reorg");
}
ExExNotification::ChainReverted { old } => {
this.state.revert_to(old.first().number - 1);
this.execution_outcome.revert_to(old.first().number - 1);
info!(reverted_chain = ?old.range(), "Received revert");
}
};
if let Some(committed_chain) = notification.committed_chain() {
// extend the state with the new chain
this.state.extend(committed_chain.state().clone());
this.execution_outcome.extend(committed_chain.execution_outcome().clone());
this.ctx.events.send(ExExEvent::FinishedHeight(committed_chain.tip().number))?;
}
}
@ -76,7 +76,7 @@ mod tests {
use std::pin::pin;
use reth::{
providers::{BundleStateWithReceipts, Chain},
providers::{Chain, ExecutionOutcome},
revm::db::BundleState,
};
use reth_exex_test_utils::{test_exex_context, PollOnce};
@ -89,60 +89,62 @@ mod tests {
let (ctx, handle) = test_exex_context().await?;
let mut exex = pin!(super::InMemoryStateExEx::new(ctx));
let mut expected_state = BundleStateWithReceipts::default();
let mut expected_state = ExecutionOutcome::default();
// Generate first block and its state
let block_1 = random_block(&mut rng, 0, None, Some(1), None)
.seal_with_senders()
.ok_or(eyre::eyre!("failed to recover senders"))?;
let block_number_1 = block_1.number;
let state_1 = BundleStateWithReceipts::new(
let execution_outcome1 = ExecutionOutcome::new(
BundleState::default(),
vec![random_receipt(&mut rng, &block_1.body[0], None)].into(),
block_1.number,
vec![],
);
// Extend the expected state with the first block
expected_state.extend(state_1.clone());
expected_state.extend(execution_outcome1.clone());
// Send a notification to the Execution Extension that the chain with the first block has
// been committed
handle.send_notification_chain_committed(Chain::new(vec![block_1], state_1, None)).await?;
handle
.send_notification_chain_committed(Chain::new(vec![block_1], execution_outcome1, None))
.await?;
exex.poll_once().await?;
// Assert that the state of the first block has been added to the total state
assert_eq!(exex.as_mut().state, expected_state);
assert_eq!(exex.as_mut().execution_outcome, expected_state);
// Generate second block and its state
let block_2 = random_block(&mut rng, 1, None, Some(2), None)
.seal_with_senders()
.ok_or(eyre::eyre!("failed to recover senders"))?;
let state_2 = BundleStateWithReceipts::new(
let execution_outcome2 = ExecutionOutcome::new(
BundleState::default(),
vec![random_receipt(&mut rng, &block_2.body[0], None)].into(),
block_2.number,
vec![],
);
// Extend the expected state with the second block
expected_state.extend(state_2.clone());
// Extend the expected execution outcome with the second block
expected_state.extend(execution_outcome2.clone());
// Send a notification to the Execution Extension that the chain with the second block has
// been committed
let chain_2 = Chain::new(vec![block_2], state_2, None);
let chain_2 = Chain::new(vec![block_2], execution_outcome2, None);
handle.send_notification_chain_committed(chain_2.clone()).await?;
exex.poll_once().await?;
// Assert that the state of the second block has been added to the total state
assert_eq!(exex.as_mut().state, expected_state);
// Assert that the execution outcome of the second block has been added to the total state
assert_eq!(exex.as_mut().execution_outcome, expected_state);
// Send a notification to the Execution Extension that the chain with the second block has
// been reverted
handle.send_notification_chain_reverted(chain_2).await?;
exex.poll_once().await?;
// Assert that the state of the second block has been reverted
// Assert that the execution outcome of the second block has been reverted
expected_state.revert_to(block_number_1);
assert_eq!(exex.as_mut().state, expected_state);
assert_eq!(exex.as_mut().execution_outcome, expected_state);
Ok(())
}

View File

@ -54,10 +54,9 @@ fn main() -> eyre::Result<()> {
#[cfg(test)]
mod tests {
use std::pin::pin;
use reth::providers::{BundleStateWithReceipts, Chain};
use reth::providers::{Chain, ExecutionOutcome};
use reth_exex_test_utils::{test_exex_context, PollOnce};
use std::pin::pin;
#[tokio::test]
async fn test_exex() -> eyre::Result<()> {
@ -71,7 +70,7 @@ mod tests {
handle
.send_notification_chain_committed(Chain::from_block(
handle.genesis.clone(),
BundleStateWithReceipts::default(),
ExecutionOutcome::default(),
None,
))
.await?;

View File

@ -264,7 +264,7 @@ mod tests {
Address, Block, Header, Log, Receipt, Transaction, TransactionSigned, TxKind, TxLegacy,
TxType, U256,
};
use reth_provider::{BundleStateWithReceipts, Chain};
use reth_provider::{Chain, ExecutionOutcome};
use reth_testing_utils::generators::sign_tx_with_random_key_pair;
use rusqlite::Connection;
@ -341,7 +341,7 @@ mod tests {
// Construct a chain
let chain = Chain::new(
vec![block.clone()],
BundleStateWithReceipts::new(
ExecutionOutcome::new(
BundleState::default(),
vec![deposit_tx_receipt, withdrawal_tx_receipt].into(),
block.number,