Skip to content

Commit

Permalink
feat: add SealedBlock in reth-primitives-traits (#13735)
Browse files Browse the repository at this point in the history
  • Loading branch information
mattsse authored Jan 15, 2025
1 parent 1267718 commit 83b2fb9
Show file tree
Hide file tree
Showing 171 changed files with 3,231 additions and 2,866 deletions.
461 changes: 227 additions & 234 deletions Cargo.lock

Large diffs are not rendered by default.

21 changes: 19 additions & 2 deletions bin/reth-bench/src/bench/new_payload_fcu.rs
Original file line number Diff line number Diff line change
Expand Up @@ -12,13 +12,14 @@ use crate::{
valid_payload::{call_forkchoice_updated, call_new_payload},
};
use alloy_primitives::B256;
use alloy_provider::Provider;
use alloy_provider::{network::AnyRpcBlock, Provider};
use alloy_rpc_types_engine::ForkchoiceState;
use clap::Parser;
use csv::Writer;
use reth_cli_runner::CliContext;
use reth_node_core::args::BenchmarkArgs;
use reth_primitives::SealedBlock;
use reth_primitives_traits::SealedHeader;
use reth_rpc_types_compat::engine::payload::block_to_payload;
use std::time::Instant;
use tracing::{debug, info};
Expand Down Expand Up @@ -46,7 +47,7 @@ impl Command {
let block_res =
block_provider.get_block_by_number(next_block.into(), true.into()).await;
let block = block_res.unwrap().unwrap();
let block: SealedBlock = block.try_into().unwrap();
let block = from_any_rpc_block(block);
let head_block_hash = block.hash();
let safe_block_hash = block_provider
.get_block_by_number(block.number.saturating_sub(32).into(), false.into());
Expand Down Expand Up @@ -161,3 +162,19 @@ impl Command {
Ok(())
}
}

// TODO(mattsse): integrate in alloy
pub(crate) fn from_any_rpc_block(block: AnyRpcBlock) -> SealedBlock {
let block = block.inner;
let block_hash = block.header.hash;
let block = block.try_map_transactions(|tx| tx.try_into()).unwrap();

SealedBlock::from_sealed_parts(
SealedHeader::new(block.header.inner.into_header_with_defaults(), block_hash),
reth_primitives::BlockBody {
transactions: block.transactions.into_transactions().collect(),
ommers: Default::default(),
withdrawals: block.withdrawals.map(|w| w.into_inner().into()),
},
)
}
4 changes: 2 additions & 2 deletions bin/reth-bench/src/bench/new_payload_only.rs
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@
use crate::{
bench::{
context::BenchContext,
new_payload_fcu::from_any_rpc_block,
output::{
NewPayloadResult, TotalGasOutput, TotalGasRow, GAS_OUTPUT_SUFFIX,
NEW_PAYLOAD_OUTPUT_SUFFIX,
Expand All @@ -16,7 +17,6 @@ use clap::Parser;
use csv::Writer;
use reth_cli_runner::CliContext;
use reth_node_core::args::BenchmarkArgs;
use reth_primitives::SealedBlock;
use reth_rpc_types_compat::engine::payload::block_to_payload;
use std::time::Instant;
use tracing::{debug, info};
Expand Down Expand Up @@ -46,7 +46,7 @@ impl Command {
let block_res =
block_provider.get_block_by_number(next_block.into(), true.into()).await;
let block = block_res.unwrap().unwrap();
let block: SealedBlock = block.try_into().unwrap();
let block = from_any_rpc_block(block);

next_block += 1;
sender.send(block).await.unwrap();
Expand Down
19 changes: 7 additions & 12 deletions bin/reth/src/commands/debug_cmd/build_block.rs
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
//! Command for debugging block building.
use alloy_consensus::TxEip4844;
use alloy_consensus::{BlockHeader, TxEip4844};
use alloy_eips::{
eip2718::Encodable2718,
eip4844::{env_settings::EnvKzgSettings, BlobTransactionSidecar},
Expand All @@ -24,10 +24,8 @@ use reth_execution_types::ExecutionOutcome;
use reth_fs_util as fs;
use reth_node_api::{BlockTy, EngineApiMessageVersion, PayloadBuilderAttributes};
use reth_node_ethereum::{consensus::EthBeaconConsensus, EthEvmConfig, EthExecutorProvider};
use reth_primitives::{
BlockExt, EthPrimitives, SealedBlockFor, SealedBlockWithSenders, SealedHeader, Transaction,
TransactionSigned,
};
use reth_primitives::{EthPrimitives, SealedBlock, SealedHeader, Transaction, TransactionSigned};
use reth_primitives_traits::Block as _;
use reth_provider::{
providers::{BlockchainProvider, ProviderNodeTypes},
BlockHashReader, BlockReader, BlockWriter, ChainSpecProvider, ProviderFactory,
Expand Down Expand Up @@ -86,7 +84,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
fn lookup_best_block<N: ProviderNodeTypes<ChainSpec = C::ChainSpec>>(
&self,
factory: ProviderFactory<N>,
) -> RethResult<Arc<SealedBlockFor<BlockTy<N>>>> {
) -> RethResult<Arc<SealedBlock<BlockTy<N>>>> {
let provider = factory.provider()?;

let best_number =
Expand Down Expand Up @@ -241,17 +239,14 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
consensus.validate_header(block.sealed_header())?;
consensus.validate_block_pre_execution(block)?;

let senders = block.senders().expect("sender recovery failed");
let block_with_senders =
SealedBlockWithSenders::<BlockTy<N>>::new(block.clone(), senders).unwrap();
let block_with_senders = block.clone().try_recover().unwrap();

let state_provider = blockchain_db.latest()?;
let db = StateProviderDatabase::new(&state_provider);
let executor =
EthExecutorProvider::ethereum(provider_factory.chain_spec()).executor(db);

let block_execution_output =
executor.execute(&block_with_senders.clone().unseal())?;
let block_execution_output = executor.execute(&block_with_senders)?;
let execution_outcome =
ExecutionOutcome::from((block_execution_output, block.number));
debug!(target: "reth::cli", ?execution_outcome, "Executed block");
Expand All @@ -262,7 +257,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
hashed_post_state.clone(),
)?;

if state_root != block_with_senders.state_root {
if state_root != block_with_senders.state_root() {
eyre::bail!(
"state root mismatch. expected: {}. got: {}",
block_with_senders.state_root,
Expand Down
6 changes: 3 additions & 3 deletions bin/reth/src/commands/debug_cmd/execution.rs
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
//! Command for debugging execution.
use crate::{args::NetworkArgs, utils::get_single_header};
use crate::{api::BlockTy, args::NetworkArgs, utils::get_single_header};
use alloy_eips::BlockHashOrNumber;
use alloy_primitives::{BlockNumber, B256};
use clap::Parser;
Expand Down Expand Up @@ -64,7 +64,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
&self,
config: &Config,
client: Client,
consensus: Arc<dyn Consensus<Error = ConsensusError>>,
consensus: Arc<dyn Consensus<BlockTy<N>, Error = ConsensusError>>,
provider_factory: ProviderFactory<N>,
task_executor: &TaskExecutor,
static_file_producer: StaticFileProducer<ProviderFactory<N>>,
Expand Down Expand Up @@ -172,7 +172,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
let Environment { provider_factory, config, data_dir } =
self.env.init::<N>(AccessRights::RW)?;

let consensus: Arc<dyn Consensus<Error = ConsensusError>> =
let consensus: Arc<dyn Consensus<BlockTy<N>, Error = ConsensusError>> =
Arc::new(EthBeaconConsensus::new(provider_factory.chain_spec()));

// Configure and build network
Expand Down
45 changes: 20 additions & 25 deletions bin/reth/src/commands/debug_cmd/in_memory_merkle.rs
Original file line number Diff line number Diff line change
@@ -1,9 +1,11 @@
//! Command for debugging in-memory merkle trie calculation.
use crate::{
api::BlockTy,
args::NetworkArgs,
utils::{get_single_body, get_single_header},
};
use alloy_consensus::BlockHeader;
use alloy_eips::BlockHashOrNumber;
use backon::{ConstantBuilder, Retryable};
use clap::Parser;
Expand All @@ -18,9 +20,9 @@ use reth_evm::execute::{BlockExecutorProvider, Executor};
use reth_execution_types::ExecutionOutcome;
use reth_network::{BlockDownloaderProvider, NetworkHandle};
use reth_network_api::NetworkInfo;
use reth_node_api::{BlockTy, NodePrimitives};
use reth_node_api::NodePrimitives;
use reth_node_ethereum::{consensus::EthBeaconConsensus, EthExecutorProvider};
use reth_primitives::{BlockExt, EthPrimitives};
use reth_primitives::{EthPrimitives, SealedBlock};
use reth_provider::{
providers::ProviderNodeTypes, AccountExtReader, ChainSpecProvider, DatabaseProviderFactory,
HashedPostStateProvider, HashingWriter, LatestStateProviderRef, OriginalValuesKnown,
Expand Down Expand Up @@ -135,33 +137,27 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
let client = fetch_client.clone();
let chain = provider_factory.chain_spec();
let consensus = Arc::new(EthBeaconConsensus::new(chain.clone()));
let block = (move || get_single_body(client.clone(), header.clone(), consensus.clone()))
.retry(backoff)
.notify(
|err, _| warn!(target: "reth::cli", "Error requesting body: {err}. Retrying..."),
)
.await?;
let block: SealedBlock<BlockTy<N>> = (move || {
get_single_body(client.clone(), header.clone(), consensus.clone())
})
.retry(backoff)
.notify(|err, _| warn!(target: "reth::cli", "Error requesting body: {err}. Retrying..."))
.await?;

let state_provider = LatestStateProviderRef::new(&provider);
let db = StateProviderDatabase::new(&state_provider);

let executor = EthExecutorProvider::ethereum(provider_factory.chain_spec()).executor(db);
let block_execution_output = executor.execute(
&block
.clone()
.unseal::<BlockTy<N>>()
.with_recovered_senders()
.ok_or(BlockValidationError::SenderRecoveryError)?,
)?;
let execution_outcome = ExecutionOutcome::from((block_execution_output, block.number));
let block_execution_output = executor.execute(&block.clone().try_recover()?)?;
let execution_outcome = ExecutionOutcome::from((block_execution_output, block.number()));

// Unpacked `BundleState::state_root_slow` function
let (in_memory_state_root, in_memory_updates) = StateRoot::overlay_root_with_updates(
provider.tx_ref(),
state_provider.hashed_post_state(execution_outcome.state()),
)?;

if in_memory_state_root == block.state_root {
if in_memory_state_root == block.state_root() {
info!(target: "reth::cli", state_root = ?in_memory_state_root, "Computed in-memory state root matches");
return Ok(())
}
Expand All @@ -170,28 +166,27 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {

// Insert block, state and hashes
provider_rw.insert_historical_block(
block
.clone()
.try_seal_with_senders()
.map_err(|_| BlockValidationError::SenderRecoveryError)?,
block.clone().try_recover().map_err(|_| BlockValidationError::SenderRecoveryError)?,
)?;
provider_rw.write_state(
&execution_outcome,
OriginalValuesKnown::No,
StorageLocation::Database,
)?;
let storage_lists = provider_rw.changed_storages_with_range(block.number..=block.number)?;
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)?;
let account_lists = provider_rw.changed_accounts_with_range(block.number..=block.number)?;
let account_lists =
provider_rw.changed_accounts_with_range(block.number..=block.number())?;
let accounts = provider_rw.basic_accounts(account_lists)?;
provider_rw.insert_account_for_hashing(accounts)?;

let (state_root, incremental_trie_updates) = StateRoot::incremental_root_with_updates(
provider_rw.tx_ref(),
block.number..=block.number,
block.number..=block.number(),
)?;
if state_root != block.state_root {
if state_root != block.state_root() {
eyre::bail!(
"Computed incremental state root mismatch. Expected: {:?}. Got: {:?}",
block.state_root,
Expand Down
12 changes: 6 additions & 6 deletions bin/reth/src/commands/debug_cmd/merkle.rs
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
//! Command for debugging merkle tree calculation.
use crate::{args::NetworkArgs, utils::get_single_header};
use alloy_consensus::BlockHeader;
use alloy_eips::BlockHashOrNumber;
use backon::{ConstantBuilder, Retryable};
use clap::Parser;
Expand Down Expand Up @@ -128,7 +129,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
info!(target: "reth::cli", target_block_number=self.to, "Finished downloading tip of block range");

// build the full block client
let consensus: Arc<dyn Consensus<Error = ConsensusError>> =
let consensus: Arc<dyn Consensus<BlockTy<N>, Error = ConsensusError>> =
Arc::new(EthBeaconConsensus::new(provider_factory.chain_spec()));
let block_range_client = FullBlockClient::new(fetch_client, consensus);

Expand All @@ -153,18 +154,17 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {

for block in blocks.into_iter().rev() {
let block_number = block.number;
let sealed_block = block
.try_seal_with_senders::<BlockTy<N>>()
.map_err(|block| eyre::eyre!("Error sealing block with senders: {block:?}"))?;
let sealed_block =
block.try_recover().map_err(|_| eyre::eyre!("Error sealing block with senders"))?;
trace!(target: "reth::cli", block_number, "Executing block");

provider_rw.insert_block(sealed_block.clone(), StorageLocation::Database)?;

td += sealed_block.difficulty;
td += sealed_block.difficulty();
let mut executor = executor_provider.batch_executor(StateProviderDatabase::new(
LatestStateProviderRef::new(&provider_rw),
));
executor.execute_and_verify_one(&sealed_block.clone().unseal())?;
executor.execute_and_verify_one(&sealed_block)?;
let execution_outcome = executor.finalize();

provider_rw.write_state(
Expand Down
10 changes: 7 additions & 3 deletions book/sources/exex/tracking-state/src/bin/2.rs
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,11 @@ use std::{

use alloy_primitives::BlockNumber;
use futures_util::{FutureExt, TryStreamExt};
use reth::{api::FullNodeComponents, builder::NodeTypes, primitives::EthPrimitives};
use reth::{
api::{BlockBody, FullNodeComponents},
builder::NodeTypes,
primitives::EthPrimitives,
};
use reth_exex::{ExExContext, ExExEvent};
use reth_node_ethereum::EthereumNode;
use reth_tracing::tracing::info;
Expand Down Expand Up @@ -36,7 +40,7 @@ impl<Node: FullNodeComponents<Types: NodeTypes<Primitives = EthPrimitives>>> Fut
while let Some(notification) = ready!(this.ctx.notifications.try_next().poll_unpin(cx))? {
if let Some(reverted_chain) = notification.reverted_chain() {
this.transactions = this.transactions.saturating_sub(
reverted_chain.blocks_iter().map(|b| b.body().transactions.len() as u64).sum(),
reverted_chain.blocks_iter().map(|b| b.body().transaction_count() as u64).sum(),
);
}

Expand All @@ -45,7 +49,7 @@ impl<Node: FullNodeComponents<Types: NodeTypes<Primitives = EthPrimitives>>> Fut

this.transactions += committed_chain
.blocks_iter()
.map(|b| b.body().transactions.len() as u64)
.map(|b| b.body().transaction_count() as u64)
.sum::<u64>();

this.ctx
Expand Down
1 change: 1 addition & 0 deletions crates/chain-state/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ alloy-signer-local = { workspace = true, optional = true }
rand = { workspace = true, optional = true }

[dev-dependencies]
reth-primitives-traits = { workspace = true, features = ["test-utils"] }
reth-testing-utils.workspace = true
alloy-signer.workspace = true
alloy-signer-local.workspace = true
Expand Down
Loading

0 comments on commit 83b2fb9

Please sign in to comment.