mirror of
https://github.com/paradigmxyz/reth.git
synced 2026-01-09 15:28:01 -05:00
feat: add SealedBlock in reth-primitives-traits (#13735)
This commit is contained in:
461
Cargo.lock
generated
461
Cargo.lock
generated
File diff suppressed because it is too large
Load Diff
@@ -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};
|
||||
@@ -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());
|
||||
@@ -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()),
|
||||
},
|
||||
)
|
||||
}
|
||||
|
||||
@@ -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,
|
||||
@@ -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};
|
||||
@@ -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();
|
||||
|
||||
@@ -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},
|
||||
@@ -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,
|
||||
@@ -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 =
|
||||
@@ -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");
|
||||
@@ -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,
|
||||
|
||||
@@ -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;
|
||||
@@ -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>>,
|
||||
@@ -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
|
||||
|
||||
@@ -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;
|
||||
@@ -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,
|
||||
@@ -135,25 +137,19 @@ 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(
|
||||
@@ -161,7 +157,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
|
||||
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(())
|
||||
}
|
||||
@@ -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,
|
||||
|
||||
@@ -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;
|
||||
@@ -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);
|
||||
|
||||
@@ -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(
|
||||
|
||||
@@ -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;
|
||||
@@ -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(),
|
||||
);
|
||||
}
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -12,10 +12,9 @@ use reth_chainspec::ChainInfo;
|
||||
use reth_execution_types::{Chain, ExecutionOutcome};
|
||||
use reth_metrics::{metrics::Gauge, Metrics};
|
||||
use reth_primitives::{
|
||||
BlockWithSenders, EthPrimitives, NodePrimitives, Receipts, SealedBlock, SealedBlockFor,
|
||||
SealedBlockWithSenders, SealedHeader,
|
||||
EthPrimitives, NodePrimitives, Receipts, RecoveredBlock, SealedBlock, SealedHeader,
|
||||
};
|
||||
use reth_primitives_traits::{Block, BlockBody as _, SignedTransaction};
|
||||
use reth_primitives_traits::{BlockBody as _, SignedTransaction};
|
||||
use reth_storage_api::StateProviderBox;
|
||||
use reth_trie::{updates::TrieUpdates, HashedPostState};
|
||||
use std::{collections::BTreeMap, sync::Arc, time::Instant};
|
||||
@@ -160,7 +159,7 @@ impl<N: NodePrimitives> CanonicalInMemoryStateInner<N> {
|
||||
}
|
||||
|
||||
type PendingBlockAndReceipts<N> =
|
||||
(SealedBlockFor<<N as NodePrimitives>::Block>, Vec<reth_primitives_traits::ReceiptTy<N>>);
|
||||
(SealedBlock<<N as NodePrimitives>::Block>, Vec<reth_primitives_traits::ReceiptTy<N>>);
|
||||
|
||||
/// This type is responsible for providing the blocks, receipts, and state for
|
||||
/// all canonical blocks not on disk yet and keeps track of the block range that
|
||||
@@ -471,17 +470,17 @@ impl<N: NodePrimitives> CanonicalInMemoryState<N> {
|
||||
}
|
||||
|
||||
/// Returns the `SealedBlock` corresponding to the pending state.
|
||||
pub fn pending_block(&self) -> Option<SealedBlock<N::BlockHeader, N::BlockBody>> {
|
||||
pub fn pending_block(&self) -> Option<SealedBlock<N::Block>> {
|
||||
self.pending_state().map(|block_state| block_state.block_ref().block().clone())
|
||||
}
|
||||
|
||||
/// Returns the `SealedBlockWithSenders` corresponding to the pending state.
|
||||
pub fn pending_block_with_senders(&self) -> Option<SealedBlockWithSenders<N::Block>>
|
||||
/// Returns the `RecoveredBlock` corresponding to the pending state.
|
||||
pub fn pending_recovered_block(&self) -> Option<RecoveredBlock<N::Block>>
|
||||
where
|
||||
N::SignedTx: SignedTransaction,
|
||||
{
|
||||
self.pending_state()
|
||||
.and_then(|block_state| block_state.block_ref().block().clone().seal_with_senders())
|
||||
.and_then(|block_state| block_state.block_ref().block().clone().try_recover().ok())
|
||||
}
|
||||
|
||||
/// Returns a tuple with the `SealedBlock` corresponding to the pending
|
||||
@@ -636,19 +635,11 @@ impl<N: NodePrimitives> BlockState<N> {
|
||||
&self.block
|
||||
}
|
||||
|
||||
/// Returns the block with senders for the state.
|
||||
pub fn block_with_senders(&self) -> BlockWithSenders<N::Block> {
|
||||
/// Returns a clone of the block with recovered senders for the state.
|
||||
pub fn clone_recovered_block(&self) -> RecoveredBlock<N::Block> {
|
||||
let block = self.block.block().clone();
|
||||
let senders = self.block.senders().clone();
|
||||
let (header, body) = block.split();
|
||||
BlockWithSenders::new_unchecked(N::Block::new(header.unseal(), body), senders)
|
||||
}
|
||||
|
||||
/// Returns the sealed block with senders for the state.
|
||||
pub fn sealed_block_with_senders(&self) -> SealedBlockWithSenders<N::Block> {
|
||||
let block = self.block.block().clone();
|
||||
let senders = self.block.senders().clone();
|
||||
SealedBlockWithSenders::new_unchecked(block, senders)
|
||||
RecoveredBlock::new_sealed(block, senders)
|
||||
}
|
||||
|
||||
/// Returns the hash of executed block that determines the state.
|
||||
@@ -803,7 +794,7 @@ impl<N: NodePrimitives> BlockState<N> {
|
||||
#[derive(Clone, Debug, PartialEq, Eq, Default)]
|
||||
pub struct ExecutedBlock<N: NodePrimitives = EthPrimitives> {
|
||||
/// Sealed block the rest of fields refer to.
|
||||
pub block: Arc<SealedBlockFor<N::Block>>,
|
||||
pub block: Arc<SealedBlock<N::Block>>,
|
||||
/// Block's senders.
|
||||
pub senders: Arc<Vec<Address>>,
|
||||
/// Block's execution outcome.
|
||||
@@ -817,7 +808,7 @@ pub struct ExecutedBlock<N: NodePrimitives = EthPrimitives> {
|
||||
impl<N: NodePrimitives> ExecutedBlock<N> {
|
||||
/// [`ExecutedBlock`] constructor.
|
||||
pub const fn new(
|
||||
block: Arc<SealedBlockFor<N::Block>>,
|
||||
block: Arc<SealedBlock<N::Block>>,
|
||||
senders: Arc<Vec<Address>>,
|
||||
execution_output: Arc<ExecutionOutcome<N::Receipt>>,
|
||||
hashed_state: Arc<HashedPostState>,
|
||||
@@ -827,7 +818,7 @@ impl<N: NodePrimitives> ExecutedBlock<N> {
|
||||
}
|
||||
|
||||
/// Returns a reference to the executed block.
|
||||
pub fn block(&self) -> &SealedBlockFor<N::Block> {
|
||||
pub fn block(&self) -> &SealedBlock<N::Block> {
|
||||
&self.block
|
||||
}
|
||||
|
||||
@@ -836,11 +827,11 @@ impl<N: NodePrimitives> ExecutedBlock<N> {
|
||||
&self.senders
|
||||
}
|
||||
|
||||
/// Returns a [`SealedBlockWithSenders`]
|
||||
/// Returns a [`RecoveredBlock`]
|
||||
///
|
||||
/// Note: this clones the block and senders.
|
||||
pub fn sealed_block_with_senders(&self) -> SealedBlockWithSenders<N::Block> {
|
||||
SealedBlockWithSenders::new_unchecked((*self.block).clone(), (*self.senders).clone())
|
||||
pub fn clone_recovered_block(&self) -> RecoveredBlock<N::Block> {
|
||||
RecoveredBlock::new_sealed((*self.block).clone(), (*self.senders).clone())
|
||||
}
|
||||
|
||||
/// Returns a reference to the block's execution outcome
|
||||
@@ -899,7 +890,7 @@ impl<N: NodePrimitives<SignedTx: SignedTransaction>> NewCanonicalChain<N> {
|
||||
Self::Commit { new } => {
|
||||
let new = Arc::new(new.iter().fold(Chain::default(), |mut chain, exec| {
|
||||
chain.append_block(
|
||||
exec.sealed_block_with_senders(),
|
||||
exec.clone_recovered_block(),
|
||||
exec.execution_outcome().clone(),
|
||||
);
|
||||
chain
|
||||
@@ -909,14 +900,14 @@ impl<N: NodePrimitives<SignedTx: SignedTransaction>> NewCanonicalChain<N> {
|
||||
Self::Reorg { new, old } => {
|
||||
let new = Arc::new(new.iter().fold(Chain::default(), |mut chain, exec| {
|
||||
chain.append_block(
|
||||
exec.sealed_block_with_senders(),
|
||||
exec.clone_recovered_block(),
|
||||
exec.execution_outcome().clone(),
|
||||
);
|
||||
chain
|
||||
}));
|
||||
let old = Arc::new(old.iter().fold(Chain::default(), |mut chain, exec| {
|
||||
chain.append_block(
|
||||
exec.sealed_block_with_senders(),
|
||||
exec.clone_recovered_block(),
|
||||
exec.execution_outcome().clone(),
|
||||
);
|
||||
chain
|
||||
@@ -930,7 +921,7 @@ impl<N: NodePrimitives<SignedTx: SignedTransaction>> NewCanonicalChain<N> {
|
||||
///
|
||||
/// Returns the new tip for [`Self::Reorg`] and [`Self::Commit`] variants which commit at least
|
||||
/// 1 new block.
|
||||
pub fn tip(&self) -> &SealedBlockFor<N::Block> {
|
||||
pub fn tip(&self) -> &SealedBlock<N::Block> {
|
||||
match self {
|
||||
Self::Commit { new } | Self::Reorg { new, .. } => {
|
||||
new.last().expect("non empty blocks").block()
|
||||
@@ -1325,8 +1316,8 @@ mod tests {
|
||||
|
||||
// Check the pending block with senders
|
||||
assert_eq!(
|
||||
state.pending_block_with_senders().unwrap(),
|
||||
block2.block().clone().seal_with_senders().unwrap()
|
||||
state.pending_recovered_block().unwrap(),
|
||||
block2.block().clone().try_recover().unwrap()
|
||||
);
|
||||
|
||||
// Check the pending block and receipts
|
||||
@@ -1529,7 +1520,7 @@ mod tests {
|
||||
chain_commit.to_chain_notification(),
|
||||
CanonStateNotification::Commit {
|
||||
new: Arc::new(Chain::new(
|
||||
vec![block0.sealed_block_with_senders(), block1.sealed_block_with_senders()],
|
||||
vec![block0.clone_recovered_block(), block1.clone_recovered_block()],
|
||||
sample_execution_outcome.clone(),
|
||||
None
|
||||
))
|
||||
@@ -1546,12 +1537,12 @@ mod tests {
|
||||
chain_reorg.to_chain_notification(),
|
||||
CanonStateNotification::Reorg {
|
||||
old: Arc::new(Chain::new(
|
||||
vec![block1.sealed_block_with_senders(), block2.sealed_block_with_senders()],
|
||||
vec![block1.clone_recovered_block(), block2.clone_recovered_block()],
|
||||
sample_execution_outcome.clone(),
|
||||
None
|
||||
)),
|
||||
new: Arc::new(Chain::new(
|
||||
vec![block1a.sealed_block_with_senders(), block2a.sealed_block_with_senders()],
|
||||
vec![block1a.clone_recovered_block(), block2a.clone_recovered_block()],
|
||||
sample_execution_outcome,
|
||||
None
|
||||
))
|
||||
|
||||
@@ -3,7 +3,7 @@
|
||||
use alloy_eips::eip2718::Encodable2718;
|
||||
use derive_more::{Deref, DerefMut};
|
||||
use reth_execution_types::{BlockReceipts, Chain};
|
||||
use reth_primitives::{NodePrimitives, SealedBlockWithSenders, SealedHeader};
|
||||
use reth_primitives::{NodePrimitives, RecoveredBlock, SealedHeader};
|
||||
use reth_storage_api::NodePrimitivesProvider;
|
||||
use std::{
|
||||
pin::Pin,
|
||||
@@ -123,7 +123,7 @@ impl<N: NodePrimitives> CanonStateNotification<N> {
|
||||
///
|
||||
/// Returns the new tip for [`Self::Reorg`] and [`Self::Commit`] variants which commit at least
|
||||
/// 1 new block.
|
||||
pub fn tip(&self) -> &SealedBlockWithSenders<N::Block> {
|
||||
pub fn tip(&self) -> &RecoveredBlock<N::Block> {
|
||||
match self {
|
||||
Self::Commit { new } | Self::Reorg { new, .. } => new.tip(),
|
||||
}
|
||||
@@ -221,7 +221,7 @@ mod tests {
|
||||
|
||||
#[test]
|
||||
fn test_commit_notification() {
|
||||
let block: SealedBlockWithSenders = Default::default();
|
||||
let block: RecoveredBlock<reth_primitives::Block> = Default::default();
|
||||
let block1_hash = B256::new([0x01; 32]);
|
||||
let block2_hash = B256::new([0x02; 32]);
|
||||
|
||||
@@ -254,7 +254,7 @@ mod tests {
|
||||
|
||||
#[test]
|
||||
fn test_reorg_notification() {
|
||||
let block: SealedBlockWithSenders = Default::default();
|
||||
let block: RecoveredBlock<reth_primitives::Block> = Default::default();
|
||||
let block1_hash = B256::new([0x01; 32]);
|
||||
let block2_hash = B256::new([0x02; 32]);
|
||||
let block3_hash = B256::new([0x03; 32]);
|
||||
@@ -306,10 +306,12 @@ mod tests {
|
||||
let tx = TransactionSigned::default();
|
||||
body.transactions.push(tx);
|
||||
|
||||
let block: SealedBlockWithSenders =
|
||||
SealedBlock::new(SealedHeader::seal(alloy_consensus::Header::default()), body)
|
||||
.seal_with_senders()
|
||||
.unwrap();
|
||||
let block = SealedBlock::<alloy_consensus::Block<TransactionSigned>>::from_sealed_parts(
|
||||
SealedHeader::seal_slow(alloy_consensus::Header::default()),
|
||||
body,
|
||||
)
|
||||
.try_recover()
|
||||
.unwrap();
|
||||
|
||||
// Create a clone of the default block and customize it to act as block1.
|
||||
let mut block1 = block.clone();
|
||||
@@ -372,10 +374,13 @@ mod tests {
|
||||
// Define block1 for the old chain segment, which will be reverted.
|
||||
let mut body = BlockBody::<TransactionSigned>::default();
|
||||
body.transactions.push(TransactionSigned::default());
|
||||
let mut old_block1: SealedBlockWithSenders =
|
||||
SealedBlock::new(SealedHeader::seal(alloy_consensus::Header::default()), body)
|
||||
.seal_with_senders()
|
||||
.unwrap();
|
||||
let mut old_block1 =
|
||||
SealedBlock::<alloy_consensus::Block<TransactionSigned>>::from_sealed_parts(
|
||||
SealedHeader::seal_slow(alloy_consensus::Header::default()),
|
||||
body,
|
||||
)
|
||||
.try_recover()
|
||||
.unwrap();
|
||||
old_block1.set_block_number(1);
|
||||
old_block1.set_hash(B256::new([0x01; 32]));
|
||||
|
||||
@@ -400,10 +405,13 @@ mod tests {
|
||||
// Define block2 for the new chain segment, which will be committed.
|
||||
let mut body = BlockBody::<TransactionSigned>::default();
|
||||
body.transactions.push(TransactionSigned::default());
|
||||
let mut new_block1: SealedBlockWithSenders =
|
||||
SealedBlock::new(SealedHeader::seal(alloy_consensus::Header::default()), body)
|
||||
.seal_with_senders()
|
||||
.unwrap();
|
||||
let mut new_block1 =
|
||||
SealedBlock::<alloy_consensus::Block<TransactionSigned>>::from_sealed_parts(
|
||||
SealedHeader::seal_slow(alloy_consensus::Header::default()),
|
||||
body,
|
||||
)
|
||||
.try_recover()
|
||||
.unwrap();
|
||||
new_block1.set_block_number(2);
|
||||
new_block1.set_hash(B256::new([0x02; 32]));
|
||||
|
||||
|
||||
@@ -17,8 +17,8 @@ use reth_chainspec::{ChainSpec, EthereumHardfork, MIN_TRANSACTION_GAS};
|
||||
use reth_execution_types::{Chain, ExecutionOutcome};
|
||||
use reth_primitives::{
|
||||
proofs::{calculate_receipt_root, calculate_transaction_root, calculate_withdrawals_root},
|
||||
BlockBody, EthPrimitives, NodePrimitives, Receipt, Receipts, RecoveredTx, SealedBlock,
|
||||
SealedBlockWithSenders, SealedHeader, Transaction, TransactionSigned,
|
||||
BlockBody, EthPrimitives, NodePrimitives, Receipt, Receipts, RecoveredBlock, RecoveredTx,
|
||||
SealedBlock, SealedHeader, Transaction, TransactionSigned,
|
||||
};
|
||||
use reth_primitives_traits::Account;
|
||||
use reth_storage_api::NodePrimitivesProvider;
|
||||
@@ -86,12 +86,12 @@ impl<N: NodePrimitives> TestBlockBuilder<N> {
|
||||
U256::from(INITIAL_BASE_FEE * MIN_TRANSACTION_GAS)
|
||||
}
|
||||
|
||||
/// Generates a random [`SealedBlockWithSenders`].
|
||||
/// Generates a random [`RecoveredBlock`].
|
||||
pub fn generate_random_block(
|
||||
&mut self,
|
||||
number: BlockNumber,
|
||||
parent_hash: B256,
|
||||
) -> SealedBlockWithSenders {
|
||||
) -> RecoveredBlock<reth_primitives::Block> {
|
||||
let mut rng = thread_rng();
|
||||
|
||||
let mock_tx = |nonce: u64| -> RecoveredTx<_> {
|
||||
@@ -168,8 +168,8 @@ impl<N: NodePrimitives> TestBlockBuilder<N> {
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let block = SealedBlock::new(
|
||||
SealedHeader::seal(header),
|
||||
let block = SealedBlock::from_sealed_parts(
|
||||
SealedHeader::seal_slow(header),
|
||||
BlockBody {
|
||||
transactions: transactions.into_iter().map(|tx| tx.into_tx()).collect(),
|
||||
ommers: Vec::new(),
|
||||
@@ -177,7 +177,8 @@ impl<N: NodePrimitives> TestBlockBuilder<N> {
|
||||
},
|
||||
);
|
||||
|
||||
SealedBlockWithSenders::new(block, vec![self.signer; num_txs as usize]).unwrap()
|
||||
RecoveredBlock::try_recover_sealed_with_senders(block, vec![self.signer; num_txs as usize])
|
||||
.unwrap()
|
||||
}
|
||||
|
||||
/// Creates a fork chain with the given base block.
|
||||
@@ -185,13 +186,13 @@ impl<N: NodePrimitives> TestBlockBuilder<N> {
|
||||
&mut self,
|
||||
base_block: &SealedBlock,
|
||||
length: u64,
|
||||
) -> Vec<SealedBlockWithSenders> {
|
||||
) -> Vec<RecoveredBlock<reth_primitives::Block>> {
|
||||
let mut fork = Vec::with_capacity(length as usize);
|
||||
let mut parent = base_block.clone();
|
||||
|
||||
for _ in 0..length {
|
||||
let block = self.generate_random_block(parent.number + 1, parent.hash());
|
||||
parent = block.block.clone();
|
||||
parent = block.clone_sealed_block();
|
||||
fork.push(block);
|
||||
}
|
||||
|
||||
@@ -207,7 +208,7 @@ impl<N: NodePrimitives> TestBlockBuilder<N> {
|
||||
) -> ExecutedBlock {
|
||||
let block_with_senders = self.generate_random_block(block_number, parent_hash);
|
||||
|
||||
let (block, senders) = block_with_senders.split();
|
||||
let (block, senders) = block_with_senders.split_sealed();
|
||||
ExecutedBlock::new(
|
||||
Arc::new(block),
|
||||
Arc::new(senders),
|
||||
@@ -258,7 +259,10 @@ impl<N: NodePrimitives> TestBlockBuilder<N> {
|
||||
/// Returns the execution outcome for a block created with this builder.
|
||||
/// In order to properly include the bundle state, the signer balance is
|
||||
/// updated.
|
||||
pub fn get_execution_outcome(&mut self, block: SealedBlockWithSenders) -> ExecutionOutcome {
|
||||
pub fn get_execution_outcome(
|
||||
&mut self,
|
||||
block: RecoveredBlock<reth_primitives::Block>,
|
||||
) -> ExecutionOutcome {
|
||||
let receipts = block
|
||||
.body()
|
||||
.transactions
|
||||
|
||||
@@ -20,7 +20,7 @@ use reth_network_p2p::{
|
||||
bodies::downloader::BodyDownloader,
|
||||
headers::downloader::{HeaderDownloader, SyncTarget},
|
||||
};
|
||||
use reth_node_api::{BlockTy, BodyTy, HeaderTy};
|
||||
use reth_node_api::BlockTy;
|
||||
use reth_node_core::version::SHORT_VERSION;
|
||||
use reth_node_events::node::NodeEvent;
|
||||
use reth_provider::{
|
||||
@@ -169,7 +169,7 @@ pub fn build_import_pipeline<N, C, E>(
|
||||
) -> eyre::Result<(Pipeline<N>, impl Stream<Item = NodeEvent<N::Primitives>>)>
|
||||
where
|
||||
N: ProviderNodeTypes + CliNodeTypes,
|
||||
C: Consensus<HeaderTy<N>, BodyTy<N>, Error = ConsensusError> + 'static,
|
||||
C: Consensus<BlockTy<N>, Error = ConsensusError> + 'static,
|
||||
E: BlockExecutorProvider<Primitives = N::Primitives>,
|
||||
{
|
||||
if !file_client.has_canonical_blocks() {
|
||||
|
||||
@@ -1,16 +1,14 @@
|
||||
use alloy_consensus::{BlockHeader, Header};
|
||||
use alloy_primitives::{BlockNumber, B256, U256};
|
||||
use alloy_rlp::Decodable;
|
||||
|
||||
use alloy_consensus::{BlockHeader, Header};
|
||||
use reth_codecs::Compact;
|
||||
use reth_node_builder::NodePrimitives;
|
||||
use reth_primitives::{SealedBlock, SealedBlockWithSenders, SealedHeader, StaticFileSegment};
|
||||
use reth_primitives::{SealedBlock, SealedHeader, StaticFileSegment};
|
||||
use reth_provider::{
|
||||
providers::StaticFileProvider, BlockWriter, StageCheckpointWriter, StaticFileProviderFactory,
|
||||
StaticFileWriter, StorageLocation,
|
||||
};
|
||||
use reth_stages::{StageCheckpoint, StageId};
|
||||
|
||||
use std::{fs::File, io::Read, path::PathBuf};
|
||||
use tracing::info;
|
||||
|
||||
@@ -69,8 +67,12 @@ where
|
||||
+ StaticFileProviderFactory<Primitives: NodePrimitives<BlockHeader: Compact>>,
|
||||
{
|
||||
provider_rw.insert_block(
|
||||
SealedBlockWithSenders::new(SealedBlock::new(header.clone(), Default::default()), vec![])
|
||||
.expect("no senders or txes"),
|
||||
SealedBlock::<<Provider::Primitives as NodePrimitives>::Block>::from_sealed_parts(
|
||||
header.clone(),
|
||||
Default::default(),
|
||||
)
|
||||
.try_recover()
|
||||
.expect("no senders or txes"),
|
||||
StorageLocation::Database,
|
||||
)?;
|
||||
|
||||
|
||||
@@ -14,7 +14,6 @@ workspace = true
|
||||
# reth
|
||||
reth-chainspec.workspace = true
|
||||
reth-consensus.workspace = true
|
||||
reth-primitives.workspace = true
|
||||
|
||||
# ethereum
|
||||
alloy-primitives.workspace = true
|
||||
@@ -23,6 +22,7 @@ alloy-consensus.workspace = true
|
||||
alloy-eips.workspace = true
|
||||
|
||||
[dev-dependencies]
|
||||
reth-primitives.workspace = true
|
||||
alloy-consensus.workspace = true
|
||||
rand.workspace = true
|
||||
|
||||
@@ -34,6 +34,6 @@ std = [
|
||||
"alloy-primitives/std",
|
||||
"reth-chainspec/std",
|
||||
"reth-consensus/std",
|
||||
"reth-primitives/std",
|
||||
"reth-primitives-traits/std"
|
||||
"reth-primitives-traits/std",
|
||||
"reth-primitives/std"
|
||||
]
|
||||
|
||||
@@ -1,11 +1,14 @@
|
||||
//! Collection of methods for block validation.
|
||||
|
||||
use alloy_consensus::{constants::MAXIMUM_EXTRA_DATA_SIZE, BlockHeader, EMPTY_OMMER_ROOT_HASH};
|
||||
use alloy_consensus::{
|
||||
constants::MAXIMUM_EXTRA_DATA_SIZE, BlockHeader as _, EMPTY_OMMER_ROOT_HASH,
|
||||
};
|
||||
use alloy_eips::{calc_next_block_base_fee, eip4844::DATA_GAS_PER_BLOB, eip7840::BlobParams};
|
||||
use reth_chainspec::{EthChainSpec, EthereumHardfork, EthereumHardforks};
|
||||
use reth_consensus::ConsensusError;
|
||||
use reth_primitives::SealedBlock;
|
||||
use reth_primitives_traits::{BlockBody, GotExpected, SealedHeader};
|
||||
use reth_primitives_traits::{
|
||||
Block, BlockBody, BlockHeader, GotExpected, SealedBlock, SealedHeader,
|
||||
};
|
||||
|
||||
/// Gas used needs to be less than gas limit. Gas used is going to be checked after execution.
|
||||
#[inline]
|
||||
@@ -38,8 +41,8 @@ pub fn validate_header_base_fee<H: BlockHeader, ChainSpec: EthereumHardforks>(
|
||||
///
|
||||
/// [EIP-4895]: https://eips.ethereum.org/EIPS/eip-4895
|
||||
#[inline]
|
||||
pub fn validate_shanghai_withdrawals<H: BlockHeader, B: BlockBody>(
|
||||
block: &SealedBlock<H, B>,
|
||||
pub fn validate_shanghai_withdrawals<B: Block>(
|
||||
block: &SealedBlock<B>,
|
||||
) -> Result<(), ConsensusError> {
|
||||
let withdrawals = block.body().withdrawals().ok_or(ConsensusError::BodyWithdrawalsMissing)?;
|
||||
let withdrawals_root = alloy_consensus::proofs::calculate_withdrawals_root(withdrawals);
|
||||
@@ -59,9 +62,7 @@ pub fn validate_shanghai_withdrawals<H: BlockHeader, B: BlockBody>(
|
||||
///
|
||||
/// [EIP-4844]: https://eips.ethereum.org/EIPS/eip-4844
|
||||
#[inline]
|
||||
pub fn validate_cancun_gas<H: BlockHeader, B: BlockBody>(
|
||||
block: &SealedBlock<H, B>,
|
||||
) -> Result<(), ConsensusError> {
|
||||
pub fn validate_cancun_gas<B: Block>(block: &SealedBlock<B>) -> Result<(), ConsensusError> {
|
||||
// Check that the blob gas used in the header matches the sum of the blob gas used by each
|
||||
// blob tx
|
||||
let header_blob_gas_used = block.blob_gas_used().ok_or(ConsensusError::BlobGasUsedMissing)?;
|
||||
@@ -127,13 +128,12 @@ where
|
||||
/// - Compares the transactions root in the block header to the block body
|
||||
/// - Pre-execution transaction validation
|
||||
/// - (Optionally) Compares the receipts root in the block header to the block body
|
||||
pub fn validate_block_pre_execution<H, B, ChainSpec>(
|
||||
block: &SealedBlock<H, B>,
|
||||
pub fn validate_block_pre_execution<B, ChainSpec>(
|
||||
block: &SealedBlock<B>,
|
||||
chain_spec: &ChainSpec,
|
||||
) -> Result<(), ConsensusError>
|
||||
where
|
||||
H: BlockHeader,
|
||||
B: BlockBody,
|
||||
B: Block,
|
||||
ChainSpec: EthereumHardforks,
|
||||
{
|
||||
// Check ommers hash
|
||||
@@ -377,15 +377,13 @@ mod tests {
|
||||
transactions_root: proofs::calculate_transaction_root(&[transaction.clone()]),
|
||||
..Default::default()
|
||||
};
|
||||
let header = SealedHeader::seal(header);
|
||||
|
||||
let body = BlockBody {
|
||||
transactions: vec![transaction],
|
||||
ommers: vec![],
|
||||
withdrawals: Some(Withdrawals::default()),
|
||||
};
|
||||
|
||||
let block = SealedBlock::new(header, body);
|
||||
let block = SealedBlock::seal_slow(alloy_consensus::Block { header, body });
|
||||
|
||||
// 10 blobs times the blob gas per blob.
|
||||
let expected_blob_gas_used = 10 * DATA_GAS_PER_BLOB;
|
||||
|
||||
@@ -16,10 +16,10 @@ use alloy_consensus::Header;
|
||||
use alloy_eips::eip7685::Requests;
|
||||
use alloy_primitives::{BlockHash, BlockNumber, Bloom, B256, U256};
|
||||
use reth_primitives::{
|
||||
BlockBody, BlockWithSenders, EthPrimitives, GotExpected, GotExpectedBoxed,
|
||||
InvalidTransactionError, NodePrimitives, Receipt, SealedBlock, SealedHeader,
|
||||
EthPrimitives, GotExpected, GotExpectedBoxed, InvalidTransactionError, NodePrimitives, Receipt,
|
||||
RecoveredBlock, SealedBlock, SealedHeader,
|
||||
};
|
||||
use reth_primitives_traits::constants::MINIMUM_GAS_LIMIT;
|
||||
use reth_primitives_traits::{constants::MINIMUM_GAS_LIMIT, Block};
|
||||
|
||||
/// A consensus implementation that does nothing.
|
||||
pub mod noop;
|
||||
@@ -47,9 +47,7 @@ impl<'a, R> PostExecutionInput<'a, R> {
|
||||
/// [`Consensus`] implementation which knows full node primitives and is able to validation block's
|
||||
/// execution outcome.
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait FullConsensus<N: NodePrimitives = EthPrimitives>:
|
||||
AsConsensus<N::BlockHeader, N::BlockBody>
|
||||
{
|
||||
pub trait FullConsensus<N: NodePrimitives = EthPrimitives>: AsConsensus<N::Block> {
|
||||
/// Validate a block considering world state, i.e. things that can not be checked before
|
||||
/// execution.
|
||||
///
|
||||
@@ -58,22 +56,22 @@ pub trait FullConsensus<N: NodePrimitives = EthPrimitives>:
|
||||
/// Note: validating blocks does not include other validations of the Consensus
|
||||
fn validate_block_post_execution(
|
||||
&self,
|
||||
block: &BlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
input: PostExecutionInput<'_, N::Receipt>,
|
||||
) -> Result<(), ConsensusError>;
|
||||
}
|
||||
|
||||
/// Consensus is a protocol that chooses canonical chain.
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait Consensus<H = Header, B = BlockBody>: AsHeaderValidator<H> {
|
||||
pub trait Consensus<B: Block>: AsHeaderValidator<B::Header> {
|
||||
/// The error type related to consensus.
|
||||
type Error;
|
||||
|
||||
/// Ensures that body field values match the header.
|
||||
fn validate_body_against_header(
|
||||
&self,
|
||||
body: &B,
|
||||
header: &SealedHeader<H>,
|
||||
body: &B::Body,
|
||||
header: &SealedHeader<B::Header>,
|
||||
) -> Result<(), Self::Error>;
|
||||
|
||||
/// Validate a block disregarding world state, i.e. things that can be checked before sender
|
||||
@@ -85,7 +83,7 @@ pub trait Consensus<H = Header, B = BlockBody>: AsHeaderValidator<H> {
|
||||
/// **This should not be called for the genesis block**.
|
||||
///
|
||||
/// Note: validating blocks does not include other validations of the Consensus
|
||||
fn validate_block_pre_execution(&self, block: &SealedBlock<H, B>) -> Result<(), Self::Error>;
|
||||
fn validate_block_pre_execution(&self, block: &SealedBlock<B>) -> Result<(), Self::Error>;
|
||||
}
|
||||
|
||||
/// HeaderValidator is a protocol that validates headers and their relationships.
|
||||
@@ -170,15 +168,15 @@ impl<T: HeaderValidator<H>, H> AsHeaderValidator<H> for T {
|
||||
}
|
||||
|
||||
/// Helper trait to cast `Arc<dyn FullConsensus>` to `Arc<dyn Consensus>`
|
||||
pub trait AsConsensus<H, B>: Consensus<H, B> {
|
||||
pub trait AsConsensus<B: Block>: Consensus<B> {
|
||||
/// Converts the [`Arc`] of self to [`Arc`] of [`HeaderValidator`]
|
||||
fn as_consensus<'a>(self: Arc<Self>) -> Arc<dyn Consensus<H, B, Error = Self::Error> + 'a>
|
||||
fn as_consensus<'a>(self: Arc<Self>) -> Arc<dyn Consensus<B, Error = Self::Error> + 'a>
|
||||
where
|
||||
Self: 'a;
|
||||
}
|
||||
|
||||
impl<T: Consensus<H, B>, H, B> AsConsensus<H, B> for T {
|
||||
fn as_consensus<'a>(self: Arc<Self>) -> Arc<dyn Consensus<H, B, Error = Self::Error> + 'a>
|
||||
impl<T: Consensus<B>, B: Block> AsConsensus<B> for T {
|
||||
fn as_consensus<'a>(self: Arc<Self>) -> Arc<dyn Consensus<B, Error = Self::Error> + 'a>
|
||||
where
|
||||
Self: 'a,
|
||||
{
|
||||
|
||||
@@ -1,6 +1,7 @@
|
||||
use crate::{Consensus, ConsensusError, FullConsensus, HeaderValidator, PostExecutionInput};
|
||||
use alloy_primitives::U256;
|
||||
use reth_primitives::{BlockWithSenders, NodePrimitives, SealedBlock, SealedHeader};
|
||||
use reth_primitives::{NodePrimitives, RecoveredBlock, SealedBlock, SealedHeader};
|
||||
use reth_primitives_traits::Block;
|
||||
|
||||
/// A Consensus implementation that does nothing.
|
||||
#[derive(Debug, Copy, Clone, Default)]
|
||||
@@ -29,18 +30,18 @@ impl<H> HeaderValidator<H> for NoopConsensus {
|
||||
}
|
||||
}
|
||||
|
||||
impl<H, B> Consensus<H, B> for NoopConsensus {
|
||||
impl<B: Block> Consensus<B> for NoopConsensus {
|
||||
type Error = ConsensusError;
|
||||
|
||||
fn validate_body_against_header(
|
||||
&self,
|
||||
_body: &B,
|
||||
_header: &SealedHeader<H>,
|
||||
_body: &B::Body,
|
||||
_header: &SealedHeader<B::Header>,
|
||||
) -> Result<(), Self::Error> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn validate_block_pre_execution(&self, _block: &SealedBlock<H, B>) -> Result<(), Self::Error> {
|
||||
fn validate_block_pre_execution(&self, _block: &SealedBlock<B>) -> Result<(), Self::Error> {
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
@@ -48,7 +49,7 @@ impl<H, B> Consensus<H, B> for NoopConsensus {
|
||||
impl<N: NodePrimitives> FullConsensus<N> for NoopConsensus {
|
||||
fn validate_block_post_execution(
|
||||
&self,
|
||||
_block: &BlockWithSenders<N::Block>,
|
||||
_block: &RecoveredBlock<N::Block>,
|
||||
_input: PostExecutionInput<'_, N::Receipt>,
|
||||
) -> Result<(), Self::Error> {
|
||||
Ok(())
|
||||
|
||||
@@ -1,7 +1,8 @@
|
||||
use crate::{Consensus, ConsensusError, FullConsensus, HeaderValidator, PostExecutionInput};
|
||||
use alloy_primitives::U256;
|
||||
use core::sync::atomic::{AtomicBool, Ordering};
|
||||
use reth_primitives::{BlockWithSenders, NodePrimitives, SealedBlock, SealedHeader};
|
||||
use reth_primitives::{NodePrimitives, RecoveredBlock, SealedBlock, SealedHeader};
|
||||
use reth_primitives_traits::Block;
|
||||
|
||||
/// Consensus engine implementation for testing
|
||||
#[derive(Debug)]
|
||||
@@ -49,7 +50,7 @@ impl TestConsensus {
|
||||
impl<N: NodePrimitives> FullConsensus<N> for TestConsensus {
|
||||
fn validate_block_post_execution(
|
||||
&self,
|
||||
_block: &BlockWithSenders<N::Block>,
|
||||
_block: &RecoveredBlock<N::Block>,
|
||||
_input: PostExecutionInput<'_, N::Receipt>,
|
||||
) -> Result<(), ConsensusError> {
|
||||
if self.fail_validation() {
|
||||
@@ -60,13 +61,13 @@ impl<N: NodePrimitives> FullConsensus<N> for TestConsensus {
|
||||
}
|
||||
}
|
||||
|
||||
impl<H, B> Consensus<H, B> for TestConsensus {
|
||||
impl<B: Block> Consensus<B> for TestConsensus {
|
||||
type Error = ConsensusError;
|
||||
|
||||
fn validate_body_against_header(
|
||||
&self,
|
||||
_body: &B,
|
||||
_header: &SealedHeader<H>,
|
||||
_body: &B::Body,
|
||||
_header: &SealedHeader<B::Header>,
|
||||
) -> Result<(), Self::Error> {
|
||||
if self.fail_body_against_header() {
|
||||
Err(ConsensusError::BaseFeeMissing)
|
||||
@@ -75,7 +76,7 @@ impl<H, B> Consensus<H, B> for TestConsensus {
|
||||
}
|
||||
}
|
||||
|
||||
fn validate_block_pre_execution(&self, _block: &SealedBlock<H, B>) -> Result<(), Self::Error> {
|
||||
fn validate_block_pre_execution(&self, _block: &SealedBlock<B>) -> Result<(), Self::Error> {
|
||||
if self.fail_validation() {
|
||||
Err(ConsensusError::BaseFeeMissing)
|
||||
} else {
|
||||
|
||||
@@ -11,7 +11,7 @@ use eyre::Ok;
|
||||
use futures_util::Future;
|
||||
use reth_chainspec::EthereumHardforks;
|
||||
use reth_network_api::test_utils::PeersHandleProvider;
|
||||
use reth_node_api::{Block, BlockTy, EngineTypes, FullNodeComponents};
|
||||
use reth_node_api::{Block, BlockBody, BlockTy, EngineTypes, FullNodeComponents};
|
||||
use reth_node_builder::{rpc::RethRpcAddOns, FullNode, NodeTypes, NodeTypesWithEngine};
|
||||
use reth_node_core::primitives::SignedTransaction;
|
||||
use reth_payload_primitives::{BuiltPayload, PayloadBuilderAttributes};
|
||||
@@ -232,7 +232,7 @@ where
|
||||
// get head block from notifications stream and verify the tx has been pushed to the
|
||||
// pool is actually present in the canonical block
|
||||
let head = self.engine_api.canonical_stream.next().await.unwrap();
|
||||
let tx = head.tip().transactions().first();
|
||||
let tx = head.tip().body().transactions().first();
|
||||
assert_eq!(tx.unwrap().tx_hash().as_slice(), tip_tx_hash.as_slice());
|
||||
|
||||
loop {
|
||||
|
||||
@@ -8,8 +8,8 @@ use reth_engine_primitives::InvalidBlockHook;
|
||||
use reth_evm::{
|
||||
state_change::post_block_balance_increments, system_calls::SystemCaller, ConfigureEvm,
|
||||
};
|
||||
use reth_primitives::{NodePrimitives, SealedBlockWithSenders, SealedHeader};
|
||||
use reth_primitives_traits::SignedTransaction;
|
||||
use reth_primitives::{NodePrimitives, RecoveredBlock, SealedHeader};
|
||||
use reth_primitives_traits::{BlockBody, SignedTransaction};
|
||||
use reth_provider::{BlockExecutionOutput, ChainSpecProvider, StateProviderFactory};
|
||||
use reth_revm::{
|
||||
database::StateProviderDatabase, db::states::bundle_state::BundleRetention, DatabaseCommit,
|
||||
@@ -58,7 +58,7 @@ where
|
||||
fn on_invalid_block<N>(
|
||||
&self,
|
||||
parent_header: &SealedHeader<N::BlockHeader>,
|
||||
block: &SealedBlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
output: &BlockExecutionOutput<N::Receipt>,
|
||||
trie_updates: Option<(&TrieUpdates, B256)>,
|
||||
) -> eyre::Result<()>
|
||||
@@ -87,7 +87,7 @@ where
|
||||
|
||||
// Re-execute all of the transactions in the block to load all touched accounts into
|
||||
// the cache DB.
|
||||
for tx in block.transactions() {
|
||||
for tx in block.body().transactions() {
|
||||
self.evm_config.fill_tx_env(
|
||||
evm.tx_mut(),
|
||||
tx,
|
||||
@@ -101,10 +101,8 @@ where
|
||||
|
||||
// use U256::MAX here for difficulty, because fetching it is annoying
|
||||
// NOTE: This is not mut because we are not doing the DAO irregular state change here
|
||||
let balance_increments = post_block_balance_increments(
|
||||
self.provider.chain_spec().as_ref(),
|
||||
&block.clone().unseal().block,
|
||||
);
|
||||
let balance_increments =
|
||||
post_block_balance_increments(self.provider.chain_spec().as_ref(), block);
|
||||
|
||||
// increment balances
|
||||
db.increment_balances(balance_increments)?;
|
||||
@@ -302,7 +300,7 @@ where
|
||||
fn on_invalid_block(
|
||||
&self,
|
||||
parent_header: &SealedHeader<N::BlockHeader>,
|
||||
block: &SealedBlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
output: &BlockExecutionOutput<N::Receipt>,
|
||||
trie_updates: Option<(&TrieUpdates, B256)>,
|
||||
) {
|
||||
|
||||
@@ -4,7 +4,7 @@ use crate::ForkchoiceStatus;
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_primitives::B256;
|
||||
use alloy_rpc_types_engine::ForkchoiceState;
|
||||
use reth_primitives::{EthPrimitives, SealedBlockFor};
|
||||
use reth_primitives::{EthPrimitives, SealedBlock};
|
||||
use reth_primitives_traits::{NodePrimitives, SealedHeader};
|
||||
use std::{
|
||||
fmt::{Display, Formatter, Result},
|
||||
@@ -18,9 +18,9 @@ pub enum BeaconConsensusEngineEvent<N: NodePrimitives = EthPrimitives> {
|
||||
/// The fork choice state was updated, and the current fork choice status
|
||||
ForkchoiceUpdated(ForkchoiceState, ForkchoiceStatus),
|
||||
/// A block was added to the fork chain.
|
||||
ForkBlockAdded(Arc<SealedBlockFor<N::Block>>, Duration),
|
||||
ForkBlockAdded(Arc<SealedBlock<N::Block>>, Duration),
|
||||
/// A block was added to the canonical chain, and the elapsed time validating the block
|
||||
CanonicalBlockAdded(Arc<SealedBlockFor<N::Block>>, Duration),
|
||||
CanonicalBlockAdded(Arc<SealedBlock<N::Block>>, Duration),
|
||||
/// A canonical chain was committed, and the elapsed time committing the data
|
||||
CanonicalChainCommitted(Box<SealedHeader<N::BlockHeader>>, Duration),
|
||||
/// The consensus engine is involved in live sync, and has specific progress
|
||||
|
||||
@@ -1,6 +1,6 @@
|
||||
use alloy_primitives::B256;
|
||||
use reth_execution_types::BlockExecutionOutput;
|
||||
use reth_primitives::{NodePrimitives, SealedBlockWithSenders, SealedHeader};
|
||||
use reth_primitives::{NodePrimitives, RecoveredBlock, SealedHeader};
|
||||
use reth_trie::updates::TrieUpdates;
|
||||
|
||||
/// An invalid block hook.
|
||||
@@ -9,7 +9,7 @@ pub trait InvalidBlockHook<N: NodePrimitives>: Send + Sync {
|
||||
fn on_invalid_block(
|
||||
&self,
|
||||
parent_header: &SealedHeader<N::BlockHeader>,
|
||||
block: &SealedBlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
output: &BlockExecutionOutput<N::Receipt>,
|
||||
trie_updates: Option<(&TrieUpdates, B256)>,
|
||||
);
|
||||
@@ -20,7 +20,7 @@ where
|
||||
N: NodePrimitives,
|
||||
F: Fn(
|
||||
&SealedHeader<N::BlockHeader>,
|
||||
&SealedBlockWithSenders<N::Block>,
|
||||
&RecoveredBlock<N::Block>,
|
||||
&BlockExecutionOutput<N::Receipt>,
|
||||
Option<(&TrieUpdates, B256)>,
|
||||
) + Send
|
||||
@@ -29,7 +29,7 @@ where
|
||||
fn on_invalid_block(
|
||||
&self,
|
||||
parent_header: &SealedHeader<N::BlockHeader>,
|
||||
block: &SealedBlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
output: &BlockExecutionOutput<N::Receipt>,
|
||||
trie_updates: Option<(&TrieUpdates, B256)>,
|
||||
) {
|
||||
|
||||
@@ -33,7 +33,7 @@ pub use reth_payload_primitives::{
|
||||
PayloadTypes,
|
||||
};
|
||||
use reth_payload_primitives::{InvalidPayloadAttributesError, PayloadAttributes};
|
||||
use reth_primitives::{NodePrimitives, SealedBlockFor};
|
||||
use reth_primitives::{NodePrimitives, SealedBlock};
|
||||
use reth_primitives_traits::Block;
|
||||
use serde::{de::DeserializeOwned, ser::Serialize};
|
||||
|
||||
@@ -86,7 +86,7 @@ pub trait EngineTypes:
|
||||
|
||||
/// Converts a [`BuiltPayload`] into an [`ExecutionPayload`] and [`ExecutionPayloadSidecar`].
|
||||
fn block_to_payload(
|
||||
block: SealedBlockFor<
|
||||
block: SealedBlock<
|
||||
<<Self::BuiltPayload as BuiltPayload>::Primitives as NodePrimitives>::Block,
|
||||
>,
|
||||
) -> (ExecutionPayload, ExecutionPayloadSidecar);
|
||||
@@ -109,7 +109,7 @@ pub trait PayloadValidator: fmt::Debug + Send + Sync + Unpin + 'static {
|
||||
&self,
|
||||
payload: ExecutionPayload,
|
||||
sidecar: ExecutionPayloadSidecar,
|
||||
) -> Result<SealedBlockFor<Self::Block>, PayloadError>;
|
||||
) -> Result<SealedBlock<Self::Block>, PayloadError>;
|
||||
}
|
||||
|
||||
/// Type that validates the payloads processed by the engine.
|
||||
|
||||
@@ -16,7 +16,7 @@ pub use reth_engine_tree::{
|
||||
};
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_network_p2p::BlockClient;
|
||||
use reth_node_types::{BlockTy, BodyTy, HeaderTy, NodeTypes, NodeTypesWithEngine};
|
||||
use reth_node_types::{BlockTy, NodeTypes, NodeTypesWithEngine};
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_primitives::EthPrimitives;
|
||||
use reth_provider::{
|
||||
@@ -52,10 +52,13 @@ type EngineServiceType<N, Client> = ChainOrchestrator<
|
||||
/// The type that drives the chain forward and communicates progress.
|
||||
#[pin_project]
|
||||
#[allow(missing_debug_implementations)]
|
||||
// TODO(mattsse): remove hidde once fixed : <https://github.com/rust-lang/rust/issues/135363>
|
||||
// otherwise rustdoc fails to resolve the alias
|
||||
#[doc(hidden)]
|
||||
pub struct EngineService<N, Client, E>
|
||||
where
|
||||
N: EngineNodeTypes,
|
||||
Client: BlockClient<Header = HeaderTy<N>, Body = BodyTy<N>> + 'static,
|
||||
Client: BlockClient<Block = BlockTy<N>> + 'static,
|
||||
E: BlockExecutorProvider + 'static,
|
||||
{
|
||||
orchestrator: EngineServiceType<N, Client>,
|
||||
@@ -65,7 +68,7 @@ where
|
||||
impl<N, Client, E> EngineService<N, Client, E>
|
||||
where
|
||||
N: EngineNodeTypes,
|
||||
Client: BlockClient<Header = HeaderTy<N>, Body = BodyTy<N>> + 'static,
|
||||
Client: BlockClient<Block = BlockTy<N>> + 'static,
|
||||
E: BlockExecutorProvider<Primitives = N::Primitives> + 'static,
|
||||
{
|
||||
/// Constructor for `EngineService`.
|
||||
@@ -133,7 +136,7 @@ where
|
||||
impl<N, Client, E> Stream for EngineService<N, Client, E>
|
||||
where
|
||||
N: EngineNodeTypes,
|
||||
Client: BlockClient<Header = HeaderTy<N>, Body = BodyTy<N>> + 'static,
|
||||
Client: BlockClient<Block = BlockTy<N>> + 'static,
|
||||
E: BlockExecutorProvider + 'static,
|
||||
{
|
||||
type Item = ChainEvent<BeaconConsensusEngineEvent<N::Primitives>>;
|
||||
|
||||
@@ -274,7 +274,7 @@ mod tests {
|
||||
gas_limit: ETHEREUM_BLOCK_GAS_LIMIT,
|
||||
..Default::default()
|
||||
};
|
||||
let header = SealedHeader::seal(header);
|
||||
let header = SealedHeader::seal_slow(header);
|
||||
insert_headers_into_client(&client, header, 0..total_blocks);
|
||||
|
||||
let tip = client.highest_block().expect("there should be blocks here").hash();
|
||||
|
||||
@@ -9,7 +9,7 @@ use reth_network_p2p::{
|
||||
full_block::{FetchFullBlockFuture, FetchFullBlockRangeFuture, FullBlockClient},
|
||||
BlockClient,
|
||||
};
|
||||
use reth_primitives::{SealedBlockFor, SealedBlockWithSenders};
|
||||
use reth_primitives::{RecoveredBlock, SealedBlock};
|
||||
use reth_primitives_traits::Block;
|
||||
use std::{
|
||||
cmp::{Ordering, Reverse},
|
||||
@@ -45,7 +45,7 @@ pub enum DownloadAction {
|
||||
#[derive(Debug)]
|
||||
pub enum DownloadOutcome<B: Block> {
|
||||
/// Downloaded blocks.
|
||||
Blocks(Vec<SealedBlockWithSenders<B>>),
|
||||
Blocks(Vec<RecoveredBlock<B>>),
|
||||
/// New download started.
|
||||
NewDownloadStarted {
|
||||
/// How many blocks are pending in this download.
|
||||
@@ -69,7 +69,7 @@ where
|
||||
inflight_block_range_requests: Vec<FetchFullBlockRangeFuture<Client>>,
|
||||
/// Buffered blocks from downloads - this is a min-heap of blocks, using the block number for
|
||||
/// ordering. This means the blocks will be popped from the heap with ascending block numbers.
|
||||
set_buffered_blocks: BinaryHeap<Reverse<OrderedSealedBlockWithSenders<B>>>,
|
||||
set_buffered_blocks: BinaryHeap<Reverse<OrderedRecoveredBlock<B>>>,
|
||||
/// Engine download metrics.
|
||||
metrics: BlockDownloaderMetrics,
|
||||
/// Pending events to be emitted.
|
||||
@@ -78,14 +78,11 @@ where
|
||||
|
||||
impl<Client, B> BasicBlockDownloader<Client, B>
|
||||
where
|
||||
Client: BlockClient<Header = B::Header, Body = B::Body> + 'static,
|
||||
Client: BlockClient<Block = B> + 'static,
|
||||
B: Block,
|
||||
{
|
||||
/// Create a new instance
|
||||
pub fn new(
|
||||
client: Client,
|
||||
consensus: Arc<dyn Consensus<Client::Header, Client::Body, Error = ConsensusError>>,
|
||||
) -> Self {
|
||||
pub fn new(client: Client, consensus: Arc<dyn Consensus<B, Error = ConsensusError>>) -> Self {
|
||||
Self {
|
||||
full_block_client: FullBlockClient::new(client, consensus),
|
||||
inflight_full_block_requests: Vec::new(),
|
||||
@@ -192,7 +189,7 @@ where
|
||||
|
||||
impl<Client, B> BlockDownloader for BasicBlockDownloader<Client, B>
|
||||
where
|
||||
Client: BlockClient<Header = B::Header, Body = B::Body>,
|
||||
Client: BlockClient<Block = B>,
|
||||
B: Block,
|
||||
{
|
||||
type Block = B;
|
||||
@@ -233,9 +230,7 @@ where
|
||||
.into_iter()
|
||||
.map(|b| {
|
||||
let senders = b.senders().unwrap_or_default();
|
||||
OrderedSealedBlockWithSenders(SealedBlockWithSenders::new_unchecked(
|
||||
b, senders,
|
||||
))
|
||||
OrderedRecoveredBlock(RecoveredBlock::new_sealed(b, senders))
|
||||
})
|
||||
.map(Reverse),
|
||||
);
|
||||
@@ -252,7 +247,7 @@ where
|
||||
}
|
||||
|
||||
// drain all unique element of the block buffer if there are any
|
||||
let mut downloaded_blocks: Vec<SealedBlockWithSenders<B>> =
|
||||
let mut downloaded_blocks: Vec<RecoveredBlock<B>> =
|
||||
Vec::with_capacity(self.set_buffered_blocks.len());
|
||||
while let Some(block) = self.set_buffered_blocks.pop() {
|
||||
// peek ahead and pop duplicates
|
||||
@@ -269,32 +264,32 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
/// A wrapper type around [`SealedBlockWithSenders`] that implements the [Ord]
|
||||
/// A wrapper type around [`RecoveredBlock`] that implements the [Ord]
|
||||
/// trait by block number.
|
||||
#[derive(Debug, Clone, PartialEq, Eq)]
|
||||
struct OrderedSealedBlockWithSenders<B: Block>(SealedBlockWithSenders<B>);
|
||||
struct OrderedRecoveredBlock<B: Block>(RecoveredBlock<B>);
|
||||
|
||||
impl<B: Block> PartialOrd for OrderedSealedBlockWithSenders<B> {
|
||||
impl<B: Block> PartialOrd for OrderedRecoveredBlock<B> {
|
||||
fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
|
||||
Some(self.cmp(other))
|
||||
}
|
||||
}
|
||||
|
||||
impl<B: Block> Ord for OrderedSealedBlockWithSenders<B> {
|
||||
impl<B: Block> Ord for OrderedRecoveredBlock<B> {
|
||||
fn cmp(&self, other: &Self) -> Ordering {
|
||||
self.0.number().cmp(&other.0.number())
|
||||
}
|
||||
}
|
||||
|
||||
impl<B: Block> From<SealedBlockFor<B>> for OrderedSealedBlockWithSenders<B> {
|
||||
fn from(block: SealedBlockFor<B>) -> Self {
|
||||
impl<B: Block> From<SealedBlock<B>> for OrderedRecoveredBlock<B> {
|
||||
fn from(block: SealedBlock<B>) -> Self {
|
||||
let senders = block.senders().unwrap_or_default();
|
||||
Self(SealedBlockWithSenders::new_unchecked(block, senders))
|
||||
Self(RecoveredBlock::new_sealed(block, senders))
|
||||
}
|
||||
}
|
||||
|
||||
impl<B: Block> From<OrderedSealedBlockWithSenders<B>> for SealedBlockWithSenders<B> {
|
||||
fn from(value: OrderedSealedBlockWithSenders<B>) -> Self {
|
||||
impl<B: Block> From<OrderedRecoveredBlock<B>> for RecoveredBlock<B> {
|
||||
fn from(value: OrderedRecoveredBlock<B>) -> Self {
|
||||
value.0
|
||||
}
|
||||
}
|
||||
@@ -348,7 +343,7 @@ mod tests {
|
||||
gas_limit: ETHEREUM_BLOCK_GAS_LIMIT,
|
||||
..Default::default()
|
||||
};
|
||||
let header = SealedHeader::seal(header);
|
||||
let header = SealedHeader::seal_slow(header);
|
||||
|
||||
insert_headers_into_client(&client, header, 0..total_blocks);
|
||||
let consensus = Arc::new(EthBeaconConsensus::new(chain_spec));
|
||||
|
||||
@@ -9,7 +9,7 @@ use alloy_primitives::B256;
|
||||
use futures::{Stream, StreamExt};
|
||||
use reth_chain_state::ExecutedBlock;
|
||||
use reth_engine_primitives::{BeaconConsensusEngineEvent, BeaconEngineMessage, EngineTypes};
|
||||
use reth_primitives::{NodePrimitives, SealedBlockWithSenders};
|
||||
use reth_primitives::{NodePrimitives, RecoveredBlock};
|
||||
use reth_primitives_traits::Block;
|
||||
use std::{
|
||||
collections::HashSet,
|
||||
@@ -306,7 +306,7 @@ pub enum FromEngine<Req, B: Block> {
|
||||
/// Request from the engine.
|
||||
Request(Req),
|
||||
/// Downloaded blocks from the network.
|
||||
DownloadedBlocks(Vec<SealedBlockWithSenders<B>>),
|
||||
DownloadedBlocks(Vec<RecoveredBlock<B>>),
|
||||
}
|
||||
|
||||
impl<Req: Display, B: Block> Display for FromEngine<Req, B> {
|
||||
|
||||
@@ -76,7 +76,7 @@ pub fn insert_headers_into_client(
|
||||
header.parent_hash = hash;
|
||||
header.number += 1;
|
||||
header.timestamp += 1;
|
||||
sealed_header = SealedHeader::seal(header);
|
||||
sealed_header = SealedHeader::seal_slow(header);
|
||||
client.insert(sealed_header.clone(), body.clone());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -2,7 +2,7 @@ use crate::tree::metrics::BlockBufferMetrics;
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_primitives::{BlockHash, BlockNumber};
|
||||
use reth_network::cache::LruCache;
|
||||
use reth_primitives::SealedBlockWithSenders;
|
||||
use reth_primitives::RecoveredBlock;
|
||||
use reth_primitives_traits::Block;
|
||||
use std::collections::{BTreeMap, HashMap, HashSet};
|
||||
|
||||
@@ -20,7 +20,7 @@ use std::collections::{BTreeMap, HashMap, HashSet};
|
||||
#[derive(Debug)]
|
||||
pub(super) struct BlockBuffer<B: Block> {
|
||||
/// All blocks in the buffer stored by their block hash.
|
||||
pub(crate) blocks: HashMap<BlockHash, SealedBlockWithSenders<B>>,
|
||||
pub(crate) blocks: HashMap<BlockHash, RecoveredBlock<B>>,
|
||||
/// Map of any parent block hash (even the ones not currently in the buffer)
|
||||
/// to the buffered children.
|
||||
/// Allows connecting buffered blocks by parent.
|
||||
@@ -50,12 +50,12 @@ impl<B: Block> BlockBuffer<B> {
|
||||
}
|
||||
|
||||
/// Return reference to the requested block.
|
||||
pub(super) fn block(&self, hash: &BlockHash) -> Option<&SealedBlockWithSenders<B>> {
|
||||
pub(super) fn block(&self, hash: &BlockHash) -> Option<&RecoveredBlock<B>> {
|
||||
self.blocks.get(hash)
|
||||
}
|
||||
|
||||
/// Return a reference to the lowest ancestor of the given block in the buffer.
|
||||
pub(super) fn lowest_ancestor(&self, hash: &BlockHash) -> Option<&SealedBlockWithSenders<B>> {
|
||||
pub(super) fn lowest_ancestor(&self, hash: &BlockHash) -> Option<&RecoveredBlock<B>> {
|
||||
let mut current_block = self.blocks.get(hash)?;
|
||||
while let Some(parent) = self.blocks.get(¤t_block.parent_hash()) {
|
||||
current_block = parent;
|
||||
@@ -64,7 +64,7 @@ impl<B: Block> BlockBuffer<B> {
|
||||
}
|
||||
|
||||
/// Insert a correct block inside the buffer.
|
||||
pub(super) fn insert_block(&mut self, block: SealedBlockWithSenders<B>) {
|
||||
pub(super) fn insert_block(&mut self, block: RecoveredBlock<B>) {
|
||||
let hash = block.hash();
|
||||
|
||||
self.parent_to_child.entry(block.parent_hash()).or_default().insert(hash);
|
||||
@@ -90,7 +90,7 @@ impl<B: Block> BlockBuffer<B> {
|
||||
pub(super) fn remove_block_with_children(
|
||||
&mut self,
|
||||
parent_hash: &BlockHash,
|
||||
) -> Vec<SealedBlockWithSenders<B>> {
|
||||
) -> Vec<RecoveredBlock<B>> {
|
||||
let removed = self
|
||||
.remove_block(parent_hash)
|
||||
.into_iter()
|
||||
@@ -149,7 +149,7 @@ impl<B: Block> BlockBuffer<B> {
|
||||
/// This method will only remove the block if it's present inside `self.blocks`.
|
||||
/// The block might be missing from other collections, the method will only ensure that it has
|
||||
/// been removed.
|
||||
fn remove_block(&mut self, hash: &BlockHash) -> Option<SealedBlockWithSenders<B>> {
|
||||
fn remove_block(&mut self, hash: &BlockHash) -> Option<RecoveredBlock<B>> {
|
||||
let block = self.blocks.remove(hash)?;
|
||||
self.remove_from_earliest_blocks(block.number(), hash);
|
||||
self.remove_from_parent(block.parent_hash(), hash);
|
||||
@@ -158,7 +158,7 @@ impl<B: Block> BlockBuffer<B> {
|
||||
}
|
||||
|
||||
/// Remove all children and their descendants for the given blocks and return them.
|
||||
fn remove_children(&mut self, parent_hashes: Vec<BlockHash>) -> Vec<SealedBlockWithSenders<B>> {
|
||||
fn remove_children(&mut self, parent_hashes: Vec<BlockHash>) -> Vec<RecoveredBlock<B>> {
|
||||
// remove all parent child connection and all the child children blocks that are connected
|
||||
// to the discarded parent blocks.
|
||||
let mut remove_parent_children = parent_hashes;
|
||||
@@ -184,15 +184,19 @@ mod tests {
|
||||
use super::*;
|
||||
use alloy_eips::BlockNumHash;
|
||||
use alloy_primitives::BlockHash;
|
||||
use reth_primitives::SealedBlockWithSenders;
|
||||
use reth_primitives::RecoveredBlock;
|
||||
use reth_testing_utils::generators::{self, random_block, BlockParams, Rng};
|
||||
use std::collections::HashMap;
|
||||
|
||||
/// Create random block with specified number and parent hash.
|
||||
fn create_block<R: Rng>(rng: &mut R, number: u64, parent: BlockHash) -> SealedBlockWithSenders {
|
||||
fn create_block<R: Rng>(
|
||||
rng: &mut R,
|
||||
number: u64,
|
||||
parent: BlockHash,
|
||||
) -> RecoveredBlock<reth_primitives::Block> {
|
||||
let block =
|
||||
random_block(rng, number, BlockParams { parent: Some(parent), ..Default::default() });
|
||||
block.seal_with_senders().unwrap()
|
||||
block.try_recover().unwrap()
|
||||
}
|
||||
|
||||
/// Assert that all buffer collections have the same data length.
|
||||
@@ -210,7 +214,10 @@ mod tests {
|
||||
}
|
||||
|
||||
/// Assert that the block was removed from all buffer collections.
|
||||
fn assert_block_removal<B: Block>(buffer: &BlockBuffer<B>, block: &SealedBlockWithSenders) {
|
||||
fn assert_block_removal<B: Block>(
|
||||
buffer: &BlockBuffer<B>,
|
||||
block: &RecoveredBlock<reth_primitives::Block>,
|
||||
) {
|
||||
assert!(!buffer.blocks.contains_key(&block.hash()));
|
||||
assert!(buffer
|
||||
.parent_to_child
|
||||
|
||||
@@ -4,7 +4,7 @@ use alloy_consensus::BlockHeader;
|
||||
use reth_consensus::ConsensusError;
|
||||
use reth_errors::{BlockExecutionError, BlockValidationError, ProviderError};
|
||||
use reth_evm::execute::InternalBlockExecutionError;
|
||||
use reth_primitives::SealedBlockFor;
|
||||
use reth_primitives::SealedBlock;
|
||||
use reth_primitives_traits::{Block, BlockBody};
|
||||
use tokio::sync::oneshot::error::TryRecvError;
|
||||
|
||||
@@ -27,7 +27,7 @@ pub enum AdvancePersistenceError {
|
||||
.block.parent_hash(),
|
||||
.kind)]
|
||||
struct InsertBlockErrorData<B: Block> {
|
||||
block: SealedBlockFor<B>,
|
||||
block: SealedBlock<B>,
|
||||
#[source]
|
||||
kind: InsertBlockErrorKind,
|
||||
}
|
||||
@@ -45,11 +45,11 @@ impl<B: Block> std::fmt::Debug for InsertBlockErrorData<B> {
|
||||
}
|
||||
|
||||
impl<B: Block> InsertBlockErrorData<B> {
|
||||
const fn new(block: SealedBlockFor<B>, kind: InsertBlockErrorKind) -> Self {
|
||||
const fn new(block: SealedBlock<B>, kind: InsertBlockErrorKind) -> Self {
|
||||
Self { block, kind }
|
||||
}
|
||||
|
||||
fn boxed(block: SealedBlockFor<B>, kind: InsertBlockErrorKind) -> Box<Self> {
|
||||
fn boxed(block: SealedBlock<B>, kind: InsertBlockErrorKind) -> Box<Self> {
|
||||
Box::new(Self::new(block, kind))
|
||||
}
|
||||
}
|
||||
@@ -65,23 +65,23 @@ pub struct InsertBlockError<B: Block> {
|
||||
|
||||
impl<B: Block> InsertBlockError<B> {
|
||||
/// Create a new `InsertInvalidBlockErrorTwo`
|
||||
pub fn new(block: SealedBlockFor<B>, kind: InsertBlockErrorKind) -> Self {
|
||||
pub fn new(block: SealedBlock<B>, kind: InsertBlockErrorKind) -> Self {
|
||||
Self { inner: InsertBlockErrorData::boxed(block, kind) }
|
||||
}
|
||||
|
||||
/// Create a new `InsertInvalidBlockError` from a consensus error
|
||||
pub fn consensus_error(error: ConsensusError, block: SealedBlockFor<B>) -> Self {
|
||||
pub fn consensus_error(error: ConsensusError, block: SealedBlock<B>) -> Self {
|
||||
Self::new(block, InsertBlockErrorKind::Consensus(error))
|
||||
}
|
||||
|
||||
/// Create a new `InsertInvalidBlockError` from a consensus error
|
||||
pub fn sender_recovery_error(block: SealedBlockFor<B>) -> Self {
|
||||
pub fn sender_recovery_error(block: SealedBlock<B>) -> Self {
|
||||
Self::new(block, InsertBlockErrorKind::SenderRecovery)
|
||||
}
|
||||
|
||||
/// Consumes the error and returns the block that resulted in the error
|
||||
#[inline]
|
||||
pub fn into_block(self) -> SealedBlockFor<B> {
|
||||
pub fn into_block(self) -> SealedBlock<B> {
|
||||
self.inner.block
|
||||
}
|
||||
|
||||
@@ -93,13 +93,13 @@ impl<B: Block> InsertBlockError<B> {
|
||||
|
||||
/// Returns the block that resulted in the error
|
||||
#[inline]
|
||||
pub const fn block(&self) -> &SealedBlockFor<B> {
|
||||
pub const fn block(&self) -> &SealedBlock<B> {
|
||||
&self.inner.block
|
||||
}
|
||||
|
||||
/// Consumes the type and returns the block and error kind.
|
||||
#[inline]
|
||||
pub fn split(self) -> (SealedBlockFor<B>, InsertBlockErrorKind) {
|
||||
pub fn split(self) -> (SealedBlock<B>, InsertBlockErrorKind) {
|
||||
let inner = *self.inner;
|
||||
(inner.block, inner.kind)
|
||||
}
|
||||
|
||||
@@ -1,6 +1,6 @@
|
||||
use alloy_primitives::B256;
|
||||
use reth_engine_primitives::InvalidBlockHook;
|
||||
use reth_primitives::{NodePrimitives, SealedBlockWithSenders, SealedHeader};
|
||||
use reth_primitives::{NodePrimitives, RecoveredBlock, SealedHeader};
|
||||
use reth_provider::BlockExecutionOutput;
|
||||
use reth_trie::updates::TrieUpdates;
|
||||
|
||||
@@ -13,7 +13,7 @@ impl<N: NodePrimitives> InvalidBlockHook<N> for NoopInvalidBlockHook {
|
||||
fn on_invalid_block(
|
||||
&self,
|
||||
_parent_header: &SealedHeader<N::BlockHeader>,
|
||||
_block: &SealedBlockWithSenders<N::Block>,
|
||||
_block: &RecoveredBlock<N::Block>,
|
||||
_output: &BlockExecutionOutput<N::Receipt>,
|
||||
_trie_updates: Option<(&TrieUpdates, B256)>,
|
||||
) {
|
||||
@@ -33,7 +33,7 @@ impl<N: NodePrimitives> InvalidBlockHook<N> for InvalidBlockHooks<N> {
|
||||
fn on_invalid_block(
|
||||
&self,
|
||||
parent_header: &SealedHeader<N::BlockHeader>,
|
||||
block: &SealedBlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
output: &BlockExecutionOutput<N::Receipt>,
|
||||
trie_updates: Option<(&TrieUpdates, B256)>,
|
||||
) {
|
||||
|
||||
@@ -111,7 +111,7 @@ mod tests {
|
||||
fn test_hit_eviction() {
|
||||
let mut cache = InvalidHeaderCache::new(10);
|
||||
let header = Header::default();
|
||||
let header = SealedHeader::seal(header);
|
||||
let header = SealedHeader::seal_slow(header);
|
||||
cache.insert(header.block_with_parent());
|
||||
assert_eq!(cache.headers.get(&header.hash()).unwrap().hit_count, 0);
|
||||
|
||||
|
||||
@@ -36,8 +36,7 @@ use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_payload_builder_primitives::PayloadBuilder;
|
||||
use reth_payload_primitives::PayloadBuilderAttributes;
|
||||
use reth_primitives::{
|
||||
EthPrimitives, GotExpected, NodePrimitives, SealedBlockFor, SealedBlockWithSenders,
|
||||
SealedHeader,
|
||||
EthPrimitives, GotExpected, NodePrimitives, RecoveredBlock, SealedBlock, SealedHeader,
|
||||
};
|
||||
use reth_primitives_traits::Block;
|
||||
use reth_provider::{
|
||||
@@ -147,7 +146,7 @@ impl<N: NodePrimitives> TreeState<N> {
|
||||
}
|
||||
|
||||
/// Returns the block by hash.
|
||||
fn block_by_hash(&self, hash: B256) -> Option<Arc<SealedBlockFor<N::Block>>> {
|
||||
fn block_by_hash(&self, hash: B256) -> Option<Arc<SealedBlock<N::Block>>> {
|
||||
self.blocks_by_hash.get(&hash).map(|b| b.block.clone())
|
||||
}
|
||||
|
||||
@@ -737,7 +736,7 @@ where
|
||||
/// block request processing isn't blocked for a long time.
|
||||
fn on_downloaded(
|
||||
&mut self,
|
||||
mut blocks: Vec<SealedBlockWithSenders<N::Block>>,
|
||||
mut blocks: Vec<RecoveredBlock<N::Block>>,
|
||||
) -> Result<Option<TreeEvent>, InsertBlockFatalError> {
|
||||
if blocks.is_empty() {
|
||||
// nothing to execute
|
||||
@@ -1603,11 +1602,11 @@ where
|
||||
.provider
|
||||
.sealed_block_with_senders(hash.into(), TransactionVariant::WithHash)?
|
||||
.ok_or_else(|| ProviderError::HeaderNotFound(hash.into()))?
|
||||
.split();
|
||||
.split_sealed();
|
||||
let execution_output = self
|
||||
.provider
|
||||
.get_state(block.number())?
|
||||
.ok_or_else(|| ProviderError::StateForNumberNotFound(block.number()))?;
|
||||
.get_state(block.header().number())?
|
||||
.ok_or_else(|| ProviderError::StateForNumberNotFound(block.header().number()))?;
|
||||
let hashed_state = self.provider.hashed_post_state(execution_output.state());
|
||||
|
||||
Ok(Some(ExecutedBlock {
|
||||
@@ -1650,7 +1649,7 @@ where
|
||||
.tree_state
|
||||
.block_by_hash(hash)
|
||||
// TODO: clone for compatibility. should we return an Arc here?
|
||||
.map(|block| block.as_ref().clone().unseal());
|
||||
.map(|block| block.as_ref().clone().into_block());
|
||||
}
|
||||
Ok(block)
|
||||
}
|
||||
@@ -1798,11 +1797,10 @@ where
|
||||
|
||||
/// Validate if block is correct and satisfies all the consensus rules that concern the header
|
||||
/// and block body itself.
|
||||
fn validate_block(
|
||||
&self,
|
||||
block: &SealedBlockWithSenders<N::Block>,
|
||||
) -> Result<(), ConsensusError> {
|
||||
if let Err(e) = self.consensus.validate_header_with_total_difficulty(block, U256::MAX) {
|
||||
fn validate_block(&self, block: &RecoveredBlock<N::Block>) -> Result<(), ConsensusError> {
|
||||
if let Err(e) =
|
||||
self.consensus.validate_header_with_total_difficulty(block.header(), U256::MAX)
|
||||
{
|
||||
error!(
|
||||
target: "engine::tree",
|
||||
?block,
|
||||
@@ -1817,7 +1815,7 @@ where
|
||||
return Err(e)
|
||||
}
|
||||
|
||||
if let Err(e) = self.consensus.validate_block_pre_execution(block) {
|
||||
if let Err(e) = self.consensus.validate_block_pre_execution(block.sealed_block()) {
|
||||
error!(target: "engine::tree", ?block, "Failed to validate block {}: {e}", block.hash());
|
||||
return Err(e)
|
||||
}
|
||||
@@ -1870,21 +1868,21 @@ where
|
||||
/// Returns an error if sender recovery failed or inserting into the buffer failed.
|
||||
fn buffer_block_without_senders(
|
||||
&mut self,
|
||||
block: SealedBlockFor<N::Block>,
|
||||
block: SealedBlock<N::Block>,
|
||||
) -> Result<(), InsertBlockError<N::Block>> {
|
||||
match block.try_seal_with_senders() {
|
||||
match block.try_recover() {
|
||||
Ok(block) => self.buffer_block(block),
|
||||
Err(block) => Err(InsertBlockError::sender_recovery_error(block)),
|
||||
Err(err) => Err(InsertBlockError::sender_recovery_error(err.into_inner())),
|
||||
}
|
||||
}
|
||||
|
||||
/// Pre-validates the block and inserts it into the buffer.
|
||||
fn buffer_block(
|
||||
&mut self,
|
||||
block: SealedBlockWithSenders<N::Block>,
|
||||
block: RecoveredBlock<N::Block>,
|
||||
) -> Result<(), InsertBlockError<N::Block>> {
|
||||
if let Err(err) = self.validate_block(&block) {
|
||||
return Err(InsertBlockError::consensus_error(err, block.block))
|
||||
return Err(InsertBlockError::consensus_error(err, block.into_sealed_block()))
|
||||
}
|
||||
self.state.buffer.insert_block(block);
|
||||
Ok(())
|
||||
@@ -2139,7 +2137,7 @@ where
|
||||
#[instrument(level = "trace", skip_all, fields(block_hash = %block.hash(), block_num = %block.number(),), target = "engine::tree")]
|
||||
fn on_downloaded_block(
|
||||
&mut self,
|
||||
block: SealedBlockWithSenders<N::Block>,
|
||||
block: RecoveredBlock<N::Block>,
|
||||
) -> Result<Option<TreeEvent>, InsertBlockFatalError> {
|
||||
let block_num_hash = block.num_hash();
|
||||
let lowest_buffered_ancestor = self.lowest_buffered_ancestor_or(block_num_hash.hash);
|
||||
@@ -2194,25 +2192,25 @@ where
|
||||
|
||||
fn insert_block_without_senders(
|
||||
&mut self,
|
||||
block: SealedBlockFor<N::Block>,
|
||||
block: SealedBlock<N::Block>,
|
||||
) -> Result<InsertPayloadOk, InsertBlockError<N::Block>> {
|
||||
match block.try_seal_with_senders() {
|
||||
match block.try_recover() {
|
||||
Ok(block) => self.insert_block(block),
|
||||
Err(block) => Err(InsertBlockError::sender_recovery_error(block)),
|
||||
Err(err) => Err(InsertBlockError::sender_recovery_error(err.into_inner())),
|
||||
}
|
||||
}
|
||||
|
||||
fn insert_block(
|
||||
&mut self,
|
||||
block: SealedBlockWithSenders<N::Block>,
|
||||
block: RecoveredBlock<N::Block>,
|
||||
) -> Result<InsertPayloadOk, InsertBlockError<N::Block>> {
|
||||
self.insert_block_inner(block.clone())
|
||||
.map_err(|kind| InsertBlockError::new(block.block, kind))
|
||||
.map_err(|kind| InsertBlockError::new(block.into_sealed_block(), kind))
|
||||
}
|
||||
|
||||
fn insert_block_inner(
|
||||
&mut self,
|
||||
block: SealedBlockWithSenders<N::Block>,
|
||||
block: RecoveredBlock<N::Block>,
|
||||
) -> Result<InsertPayloadOk, InsertBlockErrorKind> {
|
||||
debug!(target: "engine::tree", block=?block.num_hash(), parent = ?block.parent_hash(), state_root = ?block.state_root(), "Inserting new block into tree");
|
||||
|
||||
@@ -2270,8 +2268,7 @@ where
|
||||
|
||||
let block_number = block.number();
|
||||
let block_hash = block.hash();
|
||||
let sealed_block = Arc::new(block.block.clone());
|
||||
let block = block.unseal();
|
||||
let sealed_block = Arc::new(block.clone_sealed_block());
|
||||
|
||||
let persistence_not_in_progress = !self.persistence_state.in_progress();
|
||||
|
||||
@@ -2341,12 +2338,7 @@ where
|
||||
PostExecutionInput::new(&output.receipts, &output.requests),
|
||||
) {
|
||||
// call post-block hook
|
||||
self.invalid_block_hook.on_invalid_block(
|
||||
&parent_block,
|
||||
&block.clone().seal_slow(),
|
||||
&output,
|
||||
None,
|
||||
);
|
||||
self.invalid_block_hook.on_invalid_block(&parent_block, &block, &output, None);
|
||||
return Err(err.into())
|
||||
}
|
||||
|
||||
@@ -2458,7 +2450,7 @@ where
|
||||
// call post-block hook
|
||||
self.invalid_block_hook.on_invalid_block(
|
||||
&parent_block,
|
||||
&block.clone().seal_slow(),
|
||||
&block,
|
||||
&output,
|
||||
Some((&trie_output, state_root)),
|
||||
);
|
||||
@@ -2841,7 +2833,8 @@ mod tests {
|
||||
use reth_ethereum_consensus::EthBeaconConsensus;
|
||||
use reth_ethereum_engine_primitives::{EthEngineTypes, EthereumEngineValidator};
|
||||
use reth_evm::test_utils::MockExecutorProvider;
|
||||
use reth_primitives::{Block, BlockExt, EthPrimitives};
|
||||
use reth_primitives::{Block, EthPrimitives};
|
||||
use reth_primitives_traits::Block as _;
|
||||
use reth_provider::test_utils::MockEthProvider;
|
||||
use reth_rpc_types_compat::engine::{block_to_payload_v1, payload::block_to_payload_v3};
|
||||
use reth_trie::{updates::TrieUpdates, HashedPostState};
|
||||
@@ -2951,7 +2944,7 @@ mod tests {
|
||||
let (from_tree_tx, from_tree_rx) = unbounded_channel();
|
||||
|
||||
let header = chain_spec.genesis_header().clone();
|
||||
let header = SealedHeader::seal(header);
|
||||
let header = SealedHeader::seal_slow(header);
|
||||
let engine_api_tree_state = EngineApiTreeState::new(10, 10, header.num_hash());
|
||||
let canonical_in_memory_state = CanonicalInMemoryState::with_head(header, None, None);
|
||||
|
||||
@@ -3023,12 +3016,7 @@ mod tests {
|
||||
self.persist_blocks(
|
||||
blocks
|
||||
.into_iter()
|
||||
.map(|b| {
|
||||
SealedBlockWithSenders::new_unchecked(
|
||||
(*b.block).clone(),
|
||||
b.senders().clone(),
|
||||
)
|
||||
})
|
||||
.map(|b| RecoveredBlock::new_sealed(b.block().clone(), b.senders().clone()))
|
||||
.collect(),
|
||||
);
|
||||
|
||||
@@ -3049,7 +3037,7 @@ mod tests {
|
||||
|
||||
fn insert_block(
|
||||
&mut self,
|
||||
block: SealedBlockWithSenders,
|
||||
block: RecoveredBlock<reth_primitives::Block>,
|
||||
) -> Result<InsertPayloadOk, InsertBlockError<Block>> {
|
||||
let execution_outcome = self.block_builder.get_execution_outcome(block.clone());
|
||||
self.extend_execution_outcome([execution_outcome]);
|
||||
@@ -3114,8 +3102,8 @@ mod tests {
|
||||
}
|
||||
}
|
||||
|
||||
async fn send_new_payload(&mut self, block: SealedBlockWithSenders) {
|
||||
let payload = block_to_payload_v3(block.block.clone());
|
||||
async fn send_new_payload(&mut self, block: RecoveredBlock<reth_primitives::Block>) {
|
||||
let payload = block_to_payload_v3(block.clone_sealed_block());
|
||||
self.tree
|
||||
.on_new_payload(
|
||||
payload.into(),
|
||||
@@ -3129,7 +3117,7 @@ mod tests {
|
||||
|
||||
async fn insert_chain(
|
||||
&mut self,
|
||||
chain: impl IntoIterator<Item = SealedBlockWithSenders> + Clone,
|
||||
chain: impl IntoIterator<Item = RecoveredBlock<reth_primitives::Block>> + Clone,
|
||||
) {
|
||||
for block in chain.clone() {
|
||||
self.insert_block(block.clone()).unwrap();
|
||||
@@ -3151,16 +3139,16 @@ mod tests {
|
||||
|
||||
async fn check_fork_chain_insertion(
|
||||
&mut self,
|
||||
chain: impl IntoIterator<Item = SealedBlockWithSenders> + Clone,
|
||||
chain: impl IntoIterator<Item = RecoveredBlock<reth_primitives::Block>> + Clone,
|
||||
) {
|
||||
for block in chain {
|
||||
self.check_fork_block_added(block.block.hash()).await;
|
||||
self.check_fork_block_added(block.hash()).await;
|
||||
}
|
||||
}
|
||||
|
||||
async fn check_canon_chain_insertion(
|
||||
&mut self,
|
||||
chain: impl IntoIterator<Item = SealedBlockWithSenders> + Clone,
|
||||
chain: impl IntoIterator<Item = RecoveredBlock<reth_primitives::Block>> + Clone,
|
||||
) {
|
||||
for block in chain.clone() {
|
||||
self.check_canon_block_added(block.hash()).await;
|
||||
@@ -3186,33 +3174,35 @@ mod tests {
|
||||
block,
|
||||
_,
|
||||
)) => {
|
||||
assert!(block.hash() == expected_hash);
|
||||
assert_eq!(block.hash(), expected_hash);
|
||||
}
|
||||
_ => panic!("Unexpected event: {:#?}", event),
|
||||
}
|
||||
}
|
||||
|
||||
fn persist_blocks(&self, blocks: Vec<SealedBlockWithSenders>) {
|
||||
fn persist_blocks(&self, blocks: Vec<RecoveredBlock<reth_primitives::Block>>) {
|
||||
let mut block_data: Vec<(B256, Block)> = Vec::with_capacity(blocks.len());
|
||||
let mut headers_data: Vec<(B256, Header)> = Vec::with_capacity(blocks.len());
|
||||
|
||||
for block in &blocks {
|
||||
let unsealed_block = block.clone().unseal();
|
||||
block_data.push((block.hash(), unsealed_block.clone().block));
|
||||
headers_data.push((block.hash(), unsealed_block.header.clone()));
|
||||
block_data.push((block.hash(), block.clone_block()));
|
||||
headers_data.push((block.hash(), block.header().clone()));
|
||||
}
|
||||
|
||||
self.provider.extend_blocks(block_data);
|
||||
self.provider.extend_headers(headers_data);
|
||||
}
|
||||
|
||||
fn setup_range_insertion_for_valid_chain(&mut self, chain: Vec<SealedBlockWithSenders>) {
|
||||
fn setup_range_insertion_for_valid_chain(
|
||||
&mut self,
|
||||
chain: Vec<RecoveredBlock<reth_primitives::Block>>,
|
||||
) {
|
||||
self.setup_range_insertion_for_chain(chain, None)
|
||||
}
|
||||
|
||||
fn setup_range_insertion_for_invalid_chain(
|
||||
&mut self,
|
||||
chain: Vec<SealedBlockWithSenders>,
|
||||
chain: Vec<RecoveredBlock<reth_primitives::Block>>,
|
||||
index: usize,
|
||||
) {
|
||||
self.setup_range_insertion_for_chain(chain, Some(index))
|
||||
@@ -3220,7 +3210,7 @@ mod tests {
|
||||
|
||||
fn setup_range_insertion_for_chain(
|
||||
&mut self,
|
||||
chain: Vec<SealedBlockWithSenders>,
|
||||
chain: Vec<RecoveredBlock<reth_primitives::Block>>,
|
||||
invalid_index: Option<usize>,
|
||||
) {
|
||||
// setting up execution outcomes for the chain, the blocks will be
|
||||
@@ -3392,7 +3382,7 @@ mod tests {
|
||||
|
||||
// ensure block is buffered
|
||||
let buffered = test_harness.tree.state.buffer.block(&hash).unwrap();
|
||||
assert_eq!(buffered.block, sealed);
|
||||
assert_eq!(buffered.clone_sealed_block(), sealed);
|
||||
}
|
||||
|
||||
#[test]
|
||||
@@ -3732,7 +3722,7 @@ mod tests {
|
||||
|
||||
for block in &chain_a {
|
||||
test_harness.tree.state.tree_state.insert_executed(ExecutedBlock {
|
||||
block: Arc::new(block.block.clone()),
|
||||
block: Arc::new(block.clone_sealed_block()),
|
||||
senders: Arc::new(block.senders().to_vec()),
|
||||
execution_output: Arc::new(ExecutionOutcome::default()),
|
||||
hashed_state: Arc::new(HashedPostState::default()),
|
||||
@@ -3743,7 +3733,7 @@ mod tests {
|
||||
|
||||
for block in &chain_b {
|
||||
test_harness.tree.state.tree_state.insert_executed(ExecutedBlock {
|
||||
block: Arc::new(block.block.clone()),
|
||||
block: Arc::new(block.clone_sealed_block()),
|
||||
senders: Arc::new(block.senders().to_vec()),
|
||||
execution_output: Arc::new(ExecutionOutcome::default()),
|
||||
hashed_state: Arc::new(HashedPostState::default()),
|
||||
@@ -3755,19 +3745,19 @@ mod tests {
|
||||
let mut expected_new = Vec::new();
|
||||
for block in &chain_b {
|
||||
// reorg to chain from block b
|
||||
let result = test_harness.tree.on_new_head(block.block.hash()).unwrap();
|
||||
let result = test_harness.tree.on_new_head(block.hash()).unwrap();
|
||||
assert_matches!(result, Some(NewCanonicalChain::Reorg { .. }));
|
||||
|
||||
expected_new.push(block);
|
||||
if let Some(NewCanonicalChain::Reorg { new, old }) = result {
|
||||
assert_eq!(new.len(), expected_new.len());
|
||||
for (index, block) in expected_new.iter().enumerate() {
|
||||
assert_eq!(new[index].block.hash(), block.block.hash());
|
||||
assert_eq!(new[index].block.hash(), block.hash());
|
||||
}
|
||||
|
||||
assert_eq!(old.len(), chain_a.len());
|
||||
for (index, block) in chain_a.iter().enumerate() {
|
||||
assert_eq!(old[index].block.hash(), block.block.hash());
|
||||
assert_eq!(old[index].block.hash(), block.hash());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -3936,9 +3926,7 @@ mod tests {
|
||||
|
||||
let backfill_tip_block = main_chain[(backfill_finished_block_number - 1) as usize].clone();
|
||||
// add block to mock provider to enable persistence clean up.
|
||||
test_harness
|
||||
.provider
|
||||
.add_block(backfill_tip_block.hash(), backfill_tip_block.block.unseal());
|
||||
test_harness.provider.add_block(backfill_tip_block.hash(), backfill_tip_block.into_block());
|
||||
test_harness.tree.on_engine_message(FromEngine::Event(backfill_finished)).unwrap();
|
||||
|
||||
let event = test_harness.from_tree_rx.recv().await.unwrap();
|
||||
@@ -4154,7 +4142,7 @@ mod tests {
|
||||
|
||||
// extend base chain
|
||||
let extension_chain = test_harness.block_builder.create_fork(old_head, 5);
|
||||
let fork_block = extension_chain.last().unwrap().block.clone();
|
||||
let fork_block = extension_chain.last().unwrap().clone_sealed_block();
|
||||
|
||||
test_harness.setup_range_insertion_for_valid_chain(extension_chain.clone());
|
||||
test_harness.insert_chain(extension_chain).await;
|
||||
@@ -4252,7 +4240,7 @@ mod tests {
|
||||
|
||||
// extend base chain
|
||||
let extension_chain = test_harness.block_builder.create_fork(old_head, 5);
|
||||
let fork_block = extension_chain.last().unwrap().block.clone();
|
||||
let fork_block = extension_chain.last().unwrap().clone_sealed_block();
|
||||
test_harness.insert_chain(extension_chain).await;
|
||||
|
||||
// fcu to old_head
|
||||
@@ -4322,7 +4310,7 @@ mod tests {
|
||||
test_harness.send_new_payload(block.clone()).await;
|
||||
|
||||
if index < side_chain.len() - invalid_index - 1 {
|
||||
test_harness.send_fcu(block.block.hash(), ForkchoiceStatus::Valid).await;
|
||||
test_harness.send_fcu(block.hash(), ForkchoiceStatus::Valid).await;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -13,6 +13,7 @@ workspace = true
|
||||
[dependencies]
|
||||
# reth
|
||||
reth-primitives.workspace = true
|
||||
reth-primitives-traits.workspace = true
|
||||
reth-errors.workspace = true
|
||||
reth-consensus-common.workspace = true
|
||||
reth-fs-util.workspace = true
|
||||
|
||||
@@ -19,9 +19,9 @@ use reth_evm::{
|
||||
};
|
||||
use reth_payload_validator::ExecutionPayloadValidator;
|
||||
use reth_primitives::{
|
||||
proofs, transaction::SignedTransactionIntoRecoveredExt, Block, BlockBody, BlockExt, Receipt,
|
||||
Receipts,
|
||||
proofs, transaction::SignedTransactionIntoRecoveredExt, Block, BlockBody, Receipt, Receipts,
|
||||
};
|
||||
use reth_primitives_traits::block::Block as _;
|
||||
use reth_provider::{BlockReader, ExecutionOutcome, ProviderError, StateProviderFactory};
|
||||
use reth_revm::{
|
||||
database::StateProviderDatabase,
|
||||
|
||||
@@ -8,7 +8,7 @@
|
||||
#![cfg_attr(not(test), warn(unused_crate_dependencies))]
|
||||
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
|
||||
|
||||
use alloy_consensus::{BlockHeader, EMPTY_OMMER_ROOT_HASH};
|
||||
use alloy_consensus::EMPTY_OMMER_ROOT_HASH;
|
||||
use alloy_eips::{eip7840::BlobParams, merge::ALLOWED_FUTURE_BLOCK_TIME_SECONDS};
|
||||
use alloy_primitives::U256;
|
||||
use reth_chainspec::{EthChainSpec, EthereumHardforks};
|
||||
@@ -21,10 +21,10 @@ use reth_consensus_common::validation::{
|
||||
validate_against_parent_timestamp, validate_block_pre_execution, validate_body_against_header,
|
||||
validate_header_base_fee, validate_header_extra_data, validate_header_gas,
|
||||
};
|
||||
use reth_primitives::{BlockWithSenders, NodePrimitives, Receipt, SealedBlock, SealedHeader};
|
||||
use reth_primitives::{NodePrimitives, Receipt, RecoveredBlock, SealedBlock, SealedHeader};
|
||||
use reth_primitives_traits::{
|
||||
constants::{GAS_LIMIT_BOUND_DIVISOR, MINIMUM_GAS_LIMIT},
|
||||
BlockBody,
|
||||
Block, BlockHeader,
|
||||
};
|
||||
use std::{fmt::Debug, sync::Arc, time::SystemTime};
|
||||
|
||||
@@ -103,30 +103,29 @@ where
|
||||
{
|
||||
fn validate_block_post_execution(
|
||||
&self,
|
||||
block: &BlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
input: PostExecutionInput<'_>,
|
||||
) -> Result<(), ConsensusError> {
|
||||
validate_block_post_execution(block, &self.chain_spec, input.receipts, input.requests)
|
||||
}
|
||||
}
|
||||
|
||||
impl<H, B, ChainSpec: Send + Sync + EthChainSpec + EthereumHardforks + Debug> Consensus<H, B>
|
||||
impl<B, ChainSpec: Send + Sync + EthChainSpec + EthereumHardforks + Debug> Consensus<B>
|
||||
for EthBeaconConsensus<ChainSpec>
|
||||
where
|
||||
H: BlockHeader,
|
||||
B: BlockBody,
|
||||
B: Block,
|
||||
{
|
||||
type Error = ConsensusError;
|
||||
|
||||
fn validate_body_against_header(
|
||||
&self,
|
||||
body: &B,
|
||||
header: &SealedHeader<H>,
|
||||
body: &B::Body,
|
||||
header: &SealedHeader<B::Header>,
|
||||
) -> Result<(), Self::Error> {
|
||||
validate_body_against_header(body, header.header())
|
||||
}
|
||||
|
||||
fn validate_block_pre_execution(&self, block: &SealedBlock<H, B>) -> Result<(), Self::Error> {
|
||||
fn validate_block_pre_execution(&self, block: &SealedBlock<B>) -> Result<(), Self::Error> {
|
||||
validate_block_pre_execution(block, &self.chain_spec)
|
||||
}
|
||||
}
|
||||
@@ -361,7 +360,7 @@ mod tests {
|
||||
};
|
||||
|
||||
assert_eq!(
|
||||
EthBeaconConsensus::new(chain_spec).validate_header(&SealedHeader::seal(header,)),
|
||||
EthBeaconConsensus::new(chain_spec).validate_header(&SealedHeader::seal_slow(header,)),
|
||||
Ok(())
|
||||
);
|
||||
}
|
||||
|
||||
@@ -3,7 +3,7 @@ use alloy_eips::eip7685::Requests;
|
||||
use alloy_primitives::{Bloom, B256};
|
||||
use reth_chainspec::EthereumHardforks;
|
||||
use reth_consensus::ConsensusError;
|
||||
use reth_primitives::{gas_spent_by_transactions, BlockWithSenders, GotExpected, Receipt};
|
||||
use reth_primitives::{gas_spent_by_transactions, GotExpected, Receipt, RecoveredBlock};
|
||||
use reth_primitives_traits::Block;
|
||||
|
||||
/// Validate a block with regard to execution results:
|
||||
@@ -11,7 +11,7 @@ use reth_primitives_traits::Block;
|
||||
/// - Compares the receipts root in the block header to the block body
|
||||
/// - Compares the gas used in the block header to the actual gas usage after execution
|
||||
pub fn validate_block_post_execution<B, ChainSpec>(
|
||||
block: &BlockWithSenders<B>,
|
||||
block: &RecoveredBlock<B>,
|
||||
chain_spec: &ChainSpec,
|
||||
receipts: &[Receipt],
|
||||
requests: &Requests,
|
||||
|
||||
@@ -22,7 +22,7 @@ use reth_payload_primitives::{
|
||||
PayloadOrAttributes, PayloadTypes,
|
||||
};
|
||||
use reth_payload_validator::ExecutionPayloadValidator;
|
||||
use reth_primitives::{Block, NodePrimitives, SealedBlock, SealedBlockFor};
|
||||
use reth_primitives::{Block, NodePrimitives, SealedBlock};
|
||||
use reth_rpc_types_compat::engine::payload::block_to_payload;
|
||||
use std::sync::Arc;
|
||||
|
||||
@@ -54,7 +54,7 @@ where
|
||||
type ExecutionPayloadEnvelopeV4 = ExecutionPayloadEnvelopeV4;
|
||||
|
||||
fn block_to_payload(
|
||||
block: SealedBlockFor<
|
||||
block: SealedBlock<
|
||||
<<Self::BuiltPayload as BuiltPayload>::Primitives as NodePrimitives>::Block,
|
||||
>,
|
||||
) -> (ExecutionPayload, ExecutionPayloadSidecar) {
|
||||
|
||||
@@ -5,7 +5,7 @@ use crate::{
|
||||
EthEvmConfig,
|
||||
};
|
||||
use alloc::{boxed::Box, sync::Arc, vec::Vec};
|
||||
use alloy_consensus::Transaction;
|
||||
use alloy_consensus::{BlockHeader, Transaction};
|
||||
use alloy_eips::{eip6110, eip7685::Requests};
|
||||
use core::fmt::Display;
|
||||
use reth_chainspec::{ChainSpec, EthereumHardfork, EthereumHardforks, MAINNET};
|
||||
@@ -21,7 +21,8 @@ use reth_evm::{
|
||||
system_calls::{OnStateHook, SystemCaller},
|
||||
ConfigureEvm, TxEnvOverrides,
|
||||
};
|
||||
use reth_primitives::{BlockWithSenders, EthPrimitives, Receipt};
|
||||
use reth_primitives::{EthPrimitives, Receipt, RecoveredBlock};
|
||||
use reth_primitives_traits::BlockBody;
|
||||
use reth_revm::db::State;
|
||||
use revm_primitives::{
|
||||
db::{Database, DatabaseCommit},
|
||||
@@ -129,31 +130,34 @@ where
|
||||
self.tx_env_overrides = Some(tx_env_overrides);
|
||||
}
|
||||
|
||||
fn apply_pre_execution_changes(&mut self, block: &BlockWithSenders) -> Result<(), Self::Error> {
|
||||
fn apply_pre_execution_changes(
|
||||
&mut self,
|
||||
block: &RecoveredBlock<reth_primitives::Block>,
|
||||
) -> Result<(), Self::Error> {
|
||||
// Set state clear flag if the block is after the Spurious Dragon hardfork.
|
||||
let state_clear_flag =
|
||||
(*self.chain_spec).is_spurious_dragon_active_at_block(block.header.number);
|
||||
(*self.chain_spec).is_spurious_dragon_active_at_block(block.number());
|
||||
self.state.set_state_clear_flag(state_clear_flag);
|
||||
|
||||
let mut evm = self.evm_config.evm_for_block(&mut self.state, &block.header);
|
||||
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
|
||||
|
||||
self.system_caller.apply_pre_execution_changes(&block.header, &mut evm)?;
|
||||
self.system_caller.apply_pre_execution_changes(block.header(), &mut evm)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn execute_transactions(
|
||||
&mut self,
|
||||
block: &BlockWithSenders,
|
||||
block: &RecoveredBlock<reth_primitives::Block>,
|
||||
) -> Result<ExecuteOutput<Receipt>, Self::Error> {
|
||||
let mut evm = self.evm_config.evm_for_block(&mut self.state, &block.header);
|
||||
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
|
||||
|
||||
let mut cumulative_gas_used = 0;
|
||||
let mut receipts = Vec::with_capacity(block.body.transactions.len());
|
||||
let mut receipts = Vec::with_capacity(block.body().transaction_count());
|
||||
for (sender, transaction) in block.transactions_with_sender() {
|
||||
// The sum of the transaction’s gas limit, Tg, and the gas utilized in this block prior,
|
||||
// must be no greater than the block’s gasLimit.
|
||||
let block_available_gas = block.header.gas_limit - cumulative_gas_used;
|
||||
let block_available_gas = block.gas_limit() - cumulative_gas_used;
|
||||
if transaction.gas_limit() > block_available_gas {
|
||||
return Err(BlockValidationError::TransactionGasLimitMoreThanAvailableBlockGas {
|
||||
transaction_gas_limit: transaction.gas_limit(),
|
||||
@@ -204,10 +208,10 @@ where
|
||||
|
||||
fn apply_post_execution_changes(
|
||||
&mut self,
|
||||
block: &BlockWithSenders,
|
||||
block: &RecoveredBlock<reth_primitives::Block>,
|
||||
receipts: &[Receipt],
|
||||
) -> Result<Requests, Self::Error> {
|
||||
let mut evm = self.evm_config.evm_for_block(&mut self.state, &block.header);
|
||||
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
|
||||
|
||||
let requests = if self.chain_spec.is_prague_active_at_timestamp(block.timestamp) {
|
||||
// Collect all EIP-6110 deposits
|
||||
@@ -227,10 +231,10 @@ where
|
||||
};
|
||||
drop(evm);
|
||||
|
||||
let mut balance_increments = post_block_balance_increments(&self.chain_spec, &block.block);
|
||||
let mut balance_increments = post_block_balance_increments(&self.chain_spec, block);
|
||||
|
||||
// Irregular state change at Ethereum DAO hardfork
|
||||
if self.chain_spec.fork(EthereumHardfork::Dao).transitions_at_block(block.number) {
|
||||
if self.chain_spec.fork(EthereumHardfork::Dao).transitions_at_block(block.number()) {
|
||||
// drain balances from hardcoded addresses.
|
||||
let drained_balance: u128 = self
|
||||
.state
|
||||
@@ -267,7 +271,7 @@ where
|
||||
|
||||
fn validate_block_post_execution(
|
||||
&self,
|
||||
block: &BlockWithSenders,
|
||||
block: &RecoveredBlock<reth_primitives::Block>,
|
||||
receipts: &[Receipt],
|
||||
requests: &Requests,
|
||||
) -> Result<(), ConsensusError> {
|
||||
@@ -311,8 +315,8 @@ mod tests {
|
||||
BasicBlockExecutorProvider, BatchExecutor, BlockExecutorProvider, Executor,
|
||||
};
|
||||
use reth_execution_types::BlockExecutionOutput;
|
||||
use reth_primitives::{Account, Block, BlockBody, BlockExt, Transaction};
|
||||
use reth_primitives_traits::crypto::secp256k1::public_key_to_address;
|
||||
use reth_primitives::{Account, Block, BlockBody, Transaction};
|
||||
use reth_primitives_traits::{crypto::secp256k1::public_key_to_address, Block as _};
|
||||
use reth_revm::{
|
||||
database::StateProviderDatabase, test_utils::StateProviderTest, TransitionState,
|
||||
};
|
||||
@@ -388,7 +392,7 @@ mod tests {
|
||||
|
||||
// attempt to execute a block without parent beacon block root, expect err
|
||||
let err = executor
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block {
|
||||
header: header.clone(),
|
||||
body: BlockBody { transactions: vec![], ommers: vec![], withdrawals: None },
|
||||
@@ -409,7 +413,7 @@ mod tests {
|
||||
|
||||
// Now execute a block with the fixed header, ensure that it does not fail
|
||||
executor
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block {
|
||||
header: header.clone(),
|
||||
body: BlockBody { transactions: vec![], ommers: vec![], withdrawals: None },
|
||||
@@ -469,7 +473,7 @@ mod tests {
|
||||
// attempt to execute an empty block with parent beacon block root, this should not fail
|
||||
provider
|
||||
.batch_executor(StateProviderDatabase::new(&db))
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block {
|
||||
header,
|
||||
body: BlockBody { transactions: vec![], ommers: vec![], withdrawals: None },
|
||||
@@ -513,7 +517,7 @@ mod tests {
|
||||
|
||||
// attempt to execute an empty block with parent beacon block root, this should not fail
|
||||
executor
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block {
|
||||
header,
|
||||
body: BlockBody { transactions: vec![], ommers: vec![], withdrawals: None },
|
||||
@@ -549,7 +553,7 @@ mod tests {
|
||||
// attempt to execute the genesis block with non-zero parent beacon block root, expect err
|
||||
header.parent_beacon_block_root = Some(B256::with_last_byte(0x69));
|
||||
let _err = executor
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block { header: header.clone(), body: Default::default() },
|
||||
vec![],
|
||||
))
|
||||
@@ -564,7 +568,7 @@ mod tests {
|
||||
// now try to process the genesis block again, this time ensuring that a system contract
|
||||
// call does not occur
|
||||
executor
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block { header, body: Default::default() },
|
||||
vec![],
|
||||
))
|
||||
@@ -612,7 +616,7 @@ mod tests {
|
||||
|
||||
// Now execute a block with the fixed header, ensure that it does not fail
|
||||
executor
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block { header: header.clone(), body: Default::default() },
|
||||
vec![],
|
||||
))
|
||||
@@ -682,7 +686,7 @@ mod tests {
|
||||
|
||||
// attempt to execute an empty block, this should not fail
|
||||
executor
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block { header, body: Default::default() },
|
||||
vec![],
|
||||
))
|
||||
@@ -719,7 +723,7 @@ mod tests {
|
||||
|
||||
// attempt to execute genesis block, this should not fail
|
||||
executor
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block { header, body: Default::default() },
|
||||
vec![],
|
||||
))
|
||||
@@ -763,7 +767,7 @@ mod tests {
|
||||
|
||||
// attempt to execute the fork activation block, this should not fail
|
||||
executor
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block { header, body: Default::default() },
|
||||
vec![],
|
||||
))
|
||||
@@ -814,7 +818,7 @@ mod tests {
|
||||
|
||||
// attempt to execute the fork activation block, this should not fail
|
||||
executor
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block { header, body: Default::default() },
|
||||
vec![],
|
||||
))
|
||||
@@ -847,7 +851,7 @@ mod tests {
|
||||
|
||||
// attempt to execute the genesis block, this should not fail
|
||||
executor
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block { header, body: Default::default() },
|
||||
vec![],
|
||||
))
|
||||
@@ -876,7 +880,7 @@ mod tests {
|
||||
let header_hash = header.hash_slow();
|
||||
|
||||
executor
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block { header, body: Default::default() },
|
||||
vec![],
|
||||
))
|
||||
@@ -908,7 +912,7 @@ mod tests {
|
||||
};
|
||||
|
||||
executor
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block { header, body: Default::default() },
|
||||
vec![],
|
||||
))
|
||||
@@ -1111,7 +1115,7 @@ mod tests {
|
||||
|
||||
let header = Header { timestamp: 1, number: 1, ..Header::default() };
|
||||
|
||||
let block = &BlockWithSenders::new_unchecked(
|
||||
let block = &RecoveredBlock::new_unhashed(
|
||||
Block {
|
||||
header,
|
||||
body: BlockBody {
|
||||
|
||||
@@ -3,7 +3,7 @@ use alloy_genesis::Genesis;
|
||||
use alloy_primitives::{b256, hex};
|
||||
use futures::StreamExt;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_node_api::{FullNodeComponents, FullNodePrimitives, NodeTypes};
|
||||
use reth_node_api::{BlockBody, FullNodeComponents, FullNodePrimitives, NodeTypes};
|
||||
use reth_node_builder::{
|
||||
rpc::RethRpcAddOns, EngineNodeLauncher, FullNode, NodeBuilder, NodeConfig, NodeHandle,
|
||||
};
|
||||
@@ -65,7 +65,7 @@ where
|
||||
|
||||
let head = notifications.next().await.unwrap();
|
||||
|
||||
let tx = &head.tip().transactions()[0];
|
||||
let tx = &head.tip().body().transactions()[0];
|
||||
assert_eq!(tx.trie_hash(), hash);
|
||||
println!("mined transaction: {hash}");
|
||||
}
|
||||
|
||||
@@ -30,10 +30,11 @@ use reth_payload_builder_primitives::PayloadBuilderError;
|
||||
use reth_payload_primitives::PayloadBuilderAttributes;
|
||||
use reth_primitives::{
|
||||
proofs::{self},
|
||||
Block, BlockBody, BlockExt, EthereumHardforks, InvalidTransactionError, Receipt,
|
||||
TransactionSigned,
|
||||
Block, BlockBody, EthereumHardforks, InvalidTransactionError, Receipt, TransactionSigned,
|
||||
};
|
||||
use reth_primitives_traits::Block as _;
|
||||
use reth_revm::database::StateProviderDatabase;
|
||||
use reth_storage_api::StateProviderFactory;
|
||||
use reth_transaction_pool::{
|
||||
error::InvalidPoolTransactionError, noop::NoopTransactionPool, BestTransactions,
|
||||
BestTransactionsAttributes, PoolTransaction, TransactionPool, ValidPoolTransaction,
|
||||
@@ -51,7 +52,6 @@ use tracing::{debug, trace, warn};
|
||||
|
||||
mod config;
|
||||
pub use config::*;
|
||||
use reth_storage_api::StateProviderFactory;
|
||||
|
||||
type BestTransactionsIter<Pool> = Box<
|
||||
dyn BestTransactions<Item = Arc<ValidPoolTransaction<<Pool as TransactionPool>::Transaction>>>,
|
||||
|
||||
@@ -8,8 +8,7 @@ use alloy_primitives::{Address, BlockHash, BlockNumber, TxHash};
|
||||
use core::{fmt, ops::RangeInclusive};
|
||||
use reth_execution_errors::{BlockExecutionError, InternalBlockExecutionError};
|
||||
use reth_primitives::{
|
||||
transaction::SignedTransactionIntoRecoveredExt, RecoveredTx, SealedBlockFor,
|
||||
SealedBlockWithSenders, SealedHeader,
|
||||
transaction::SignedTransactionIntoRecoveredExt, RecoveredBlock, RecoveredTx, SealedHeader,
|
||||
};
|
||||
use reth_primitives_traits::{Block, BlockBody, NodePrimitives, SignedTransaction};
|
||||
use reth_trie::updates::TrieUpdates;
|
||||
@@ -29,7 +28,7 @@ use revm::db::BundleState;
|
||||
#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))]
|
||||
pub struct Chain<N: NodePrimitives = reth_primitives::EthPrimitives> {
|
||||
/// All blocks in this chain.
|
||||
blocks: BTreeMap<BlockNumber, SealedBlockWithSenders<N::Block>>,
|
||||
blocks: BTreeMap<BlockNumber, RecoveredBlock<N::Block>>,
|
||||
/// The outcome of block execution for this chain.
|
||||
///
|
||||
/// This field contains the state of all accounts after the execution of all blocks in this
|
||||
@@ -60,11 +59,12 @@ impl<N: NodePrimitives> Chain<N> {
|
||||
///
|
||||
/// A chain of blocks should not be empty.
|
||||
pub fn new(
|
||||
blocks: impl IntoIterator<Item = SealedBlockWithSenders<N::Block>>,
|
||||
blocks: impl IntoIterator<Item = RecoveredBlock<N::Block>>,
|
||||
execution_outcome: ExecutionOutcome<N::Receipt>,
|
||||
trie_updates: Option<TrieUpdates>,
|
||||
) -> Self {
|
||||
let blocks = blocks.into_iter().map(|b| (b.number(), b)).collect::<BTreeMap<_, _>>();
|
||||
let blocks =
|
||||
blocks.into_iter().map(|b| (b.header().number(), b)).collect::<BTreeMap<_, _>>();
|
||||
debug_assert!(!blocks.is_empty(), "Chain should have at least one block");
|
||||
|
||||
Self { blocks, execution_outcome, trie_updates }
|
||||
@@ -72,7 +72,7 @@ impl<N: NodePrimitives> Chain<N> {
|
||||
|
||||
/// Create new Chain from a single block and its state.
|
||||
pub fn from_block(
|
||||
block: SealedBlockWithSenders<N::Block>,
|
||||
block: RecoveredBlock<N::Block>,
|
||||
execution_outcome: ExecutionOutcome<N::Receipt>,
|
||||
trie_updates: Option<TrieUpdates>,
|
||||
) -> Self {
|
||||
@@ -80,12 +80,12 @@ impl<N: NodePrimitives> Chain<N> {
|
||||
}
|
||||
|
||||
/// Get the blocks in this chain.
|
||||
pub const fn blocks(&self) -> &BTreeMap<BlockNumber, SealedBlockWithSenders<N::Block>> {
|
||||
pub const fn blocks(&self) -> &BTreeMap<BlockNumber, RecoveredBlock<N::Block>> {
|
||||
&self.blocks
|
||||
}
|
||||
|
||||
/// Consumes the type and only returns the blocks in this chain.
|
||||
pub fn into_blocks(self) -> BTreeMap<BlockNumber, SealedBlockWithSenders<N::Block>> {
|
||||
pub fn into_blocks(self) -> BTreeMap<BlockNumber, RecoveredBlock<N::Block>> {
|
||||
self.blocks
|
||||
}
|
||||
|
||||
@@ -131,15 +131,7 @@ impl<N: NodePrimitives> Chain<N> {
|
||||
}
|
||||
|
||||
/// Returns the block with matching hash.
|
||||
pub fn block(&self, block_hash: BlockHash) -> Option<&SealedBlockFor<N::Block>> {
|
||||
self.block_with_senders(block_hash).map(|block| &block.block)
|
||||
}
|
||||
|
||||
/// Returns the block with matching hash.
|
||||
pub fn block_with_senders(
|
||||
&self,
|
||||
block_hash: BlockHash,
|
||||
) -> Option<&SealedBlockWithSenders<N::Block>> {
|
||||
pub fn block_with_senders(&self, block_hash: BlockHash) -> Option<&RecoveredBlock<N::Block>> {
|
||||
self.blocks.iter().find_map(|(_num, block)| (block.hash() == block_hash).then_some(block))
|
||||
}
|
||||
|
||||
@@ -183,15 +175,14 @@ impl<N: NodePrimitives> Chain<N> {
|
||||
}
|
||||
|
||||
/// Returns an iterator over all blocks in the chain with increasing block number.
|
||||
pub fn blocks_iter(&self) -> impl Iterator<Item = &SealedBlockWithSenders<N::Block>> + '_ {
|
||||
pub fn blocks_iter(&self) -> impl Iterator<Item = &RecoveredBlock<N::Block>> + '_ {
|
||||
self.blocks().iter().map(|block| block.1)
|
||||
}
|
||||
|
||||
/// Returns an iterator over all blocks and their receipts in the chain.
|
||||
pub fn blocks_and_receipts(
|
||||
&self,
|
||||
) -> impl Iterator<Item = (&SealedBlockWithSenders<N::Block>, &Vec<Option<N::Receipt>>)> + '_
|
||||
{
|
||||
) -> impl Iterator<Item = (&RecoveredBlock<N::Block>, &Vec<Option<N::Receipt>>)> + '_ {
|
||||
self.blocks_iter().zip(self.block_receipts_iter())
|
||||
}
|
||||
|
||||
@@ -199,7 +190,10 @@ impl<N: NodePrimitives> Chain<N> {
|
||||
#[track_caller]
|
||||
pub fn fork_block(&self) -> ForkBlock {
|
||||
let first = self.first();
|
||||
ForkBlock { number: first.number().saturating_sub(1), hash: first.parent_hash() }
|
||||
ForkBlock {
|
||||
number: first.header().number().saturating_sub(1),
|
||||
hash: first.header().parent_hash(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Get the first block in this chain.
|
||||
@@ -208,7 +202,7 @@ impl<N: NodePrimitives> Chain<N> {
|
||||
///
|
||||
/// If chain doesn't have any blocks.
|
||||
#[track_caller]
|
||||
pub fn first(&self) -> &SealedBlockWithSenders<N::Block> {
|
||||
pub fn first(&self) -> &RecoveredBlock<N::Block> {
|
||||
self.blocks.first_key_value().expect("Chain should have at least one block").1
|
||||
}
|
||||
|
||||
@@ -218,7 +212,7 @@ impl<N: NodePrimitives> Chain<N> {
|
||||
///
|
||||
/// If chain doesn't have any blocks.
|
||||
#[track_caller]
|
||||
pub fn tip(&self) -> &SealedBlockWithSenders<N::Block> {
|
||||
pub fn tip(&self) -> &RecoveredBlock<N::Block> {
|
||||
self.blocks.last_key_value().expect("Chain should have at least one block").1
|
||||
}
|
||||
|
||||
@@ -233,7 +227,7 @@ impl<N: NodePrimitives> Chain<N> {
|
||||
///
|
||||
/// If chain doesn't have any blocks.
|
||||
pub fn range(&self) -> RangeInclusive<BlockNumber> {
|
||||
self.first().number()..=self.tip().number()
|
||||
self.first().header().number()..=self.tip().header().number()
|
||||
}
|
||||
|
||||
/// Get all receipts for the given block.
|
||||
@@ -270,10 +264,10 @@ impl<N: NodePrimitives> Chain<N> {
|
||||
/// This method assumes that blocks attachment to the chain has already been validated.
|
||||
pub fn append_block(
|
||||
&mut self,
|
||||
block: SealedBlockWithSenders<N::Block>,
|
||||
block: RecoveredBlock<N::Block>,
|
||||
execution_outcome: ExecutionOutcome<N::Receipt>,
|
||||
) {
|
||||
self.blocks.insert(block.number(), block);
|
||||
self.blocks.insert(block.header().number(), block);
|
||||
self.execution_outcome.extend(execution_outcome);
|
||||
self.trie_updates.take(); // reset
|
||||
}
|
||||
@@ -375,7 +369,7 @@ impl<N: NodePrimitives> Chain<N> {
|
||||
/// Wrapper type for `blocks` display in `Chain`
|
||||
#[derive(Debug)]
|
||||
pub struct DisplayBlocksChain<'a, B: reth_primitives_traits::Block>(
|
||||
pub &'a BTreeMap<BlockNumber, SealedBlockWithSenders<B>>,
|
||||
pub &'a BTreeMap<BlockNumber, RecoveredBlock<B>>,
|
||||
);
|
||||
|
||||
impl<B: reth_primitives_traits::Block> fmt::Display for DisplayBlocksChain<'_, B> {
|
||||
@@ -396,7 +390,7 @@ impl<B: reth_primitives_traits::Block> fmt::Display for DisplayBlocksChain<'_, B
|
||||
/// All blocks in the chain
|
||||
#[derive(Clone, Debug, Default, PartialEq, Eq)]
|
||||
pub struct ChainBlocks<'a, B: Block> {
|
||||
blocks: Cow<'a, BTreeMap<BlockNumber, SealedBlockWithSenders<B>>>,
|
||||
blocks: Cow<'a, BTreeMap<BlockNumber, RecoveredBlock<B>>>,
|
||||
}
|
||||
|
||||
impl<B: Block<Body: BlockBody<Transaction: SignedTransaction>>> ChainBlocks<'_, B> {
|
||||
@@ -404,13 +398,13 @@ impl<B: Block<Body: BlockBody<Transaction: SignedTransaction>>> ChainBlocks<'_,
|
||||
///
|
||||
/// Note: this always yields at least one block.
|
||||
#[inline]
|
||||
pub fn into_blocks(self) -> impl Iterator<Item = SealedBlockWithSenders<B>> {
|
||||
pub fn into_blocks(self) -> impl Iterator<Item = RecoveredBlock<B>> {
|
||||
self.blocks.into_owned().into_values()
|
||||
}
|
||||
|
||||
/// Creates an iterator over all blocks in the chain with increasing block number.
|
||||
#[inline]
|
||||
pub fn iter(&self) -> impl Iterator<Item = (&BlockNumber, &SealedBlockWithSenders<B>)> {
|
||||
pub fn iter(&self) -> impl Iterator<Item = (&BlockNumber, &RecoveredBlock<B>)> {
|
||||
self.blocks.iter()
|
||||
}
|
||||
|
||||
@@ -420,7 +414,7 @@ impl<B: Block<Body: BlockBody<Transaction: SignedTransaction>>> ChainBlocks<'_,
|
||||
///
|
||||
/// Chains always have at least one block.
|
||||
#[inline]
|
||||
pub fn tip(&self) -> &SealedBlockWithSenders<B> {
|
||||
pub fn tip(&self) -> &RecoveredBlock<B> {
|
||||
self.blocks.last_key_value().expect("Chain should have at least one block").1
|
||||
}
|
||||
|
||||
@@ -430,7 +424,7 @@ impl<B: Block<Body: BlockBody<Transaction: SignedTransaction>>> ChainBlocks<'_,
|
||||
///
|
||||
/// Chains always have at least one block.
|
||||
#[inline]
|
||||
pub fn first(&self) -> &SealedBlockWithSenders<B> {
|
||||
pub fn first(&self) -> &RecoveredBlock<B> {
|
||||
self.blocks.first_key_value().expect("Chain should have at least one block").1
|
||||
}
|
||||
|
||||
@@ -461,13 +455,15 @@ impl<B: Block<Body: BlockBody<Transaction: SignedTransaction>>> ChainBlocks<'_,
|
||||
/// Returns an iterator over all transaction hashes in the block
|
||||
#[inline]
|
||||
pub fn transaction_hashes(&self) -> impl Iterator<Item = TxHash> + '_ {
|
||||
self.blocks.values().flat_map(|block| block.transactions().iter().map(|tx| tx.trie_hash()))
|
||||
self.blocks
|
||||
.values()
|
||||
.flat_map(|block| block.body().transactions().iter().map(|tx| tx.trie_hash()))
|
||||
}
|
||||
}
|
||||
|
||||
impl<B: Block> IntoIterator for ChainBlocks<'_, B> {
|
||||
type Item = (BlockNumber, SealedBlockWithSenders<B>);
|
||||
type IntoIter = std::collections::btree_map::IntoIter<BlockNumber, SealedBlockWithSenders<B>>;
|
||||
type Item = (BlockNumber, RecoveredBlock<B>);
|
||||
type IntoIter = std::collections::btree_map::IntoIter<BlockNumber, RecoveredBlock<B>>;
|
||||
|
||||
fn into_iter(self) -> Self::IntoIter {
|
||||
#[allow(clippy::unnecessary_to_owned)]
|
||||
@@ -536,9 +532,7 @@ pub(super) mod serde_bincode_compat {
|
||||
use crate::ExecutionOutcome;
|
||||
use alloc::borrow::Cow;
|
||||
use alloy_primitives::BlockNumber;
|
||||
use reth_primitives::{
|
||||
serde_bincode_compat::SealedBlockWithSenders, EthPrimitives, NodePrimitives,
|
||||
};
|
||||
use reth_primitives::{serde_bincode_compat::RecoveredBlock, EthPrimitives, NodePrimitives};
|
||||
use reth_primitives_traits::{serde_bincode_compat::SerdeBincodeCompat, Block};
|
||||
use reth_trie_common::serde_bincode_compat::updates::TrieUpdates;
|
||||
use serde::{ser::SerializeMap, Deserialize, Deserializer, Serialize, Serializer};
|
||||
@@ -563,21 +557,25 @@ pub(super) mod serde_bincode_compat {
|
||||
#[derive(Debug, Serialize, Deserialize)]
|
||||
pub struct Chain<'a, N = EthPrimitives>
|
||||
where
|
||||
N: NodePrimitives,
|
||||
N: NodePrimitives<
|
||||
Block: Block<Header: SerdeBincodeCompat, Body: SerdeBincodeCompat> + 'static,
|
||||
>,
|
||||
{
|
||||
blocks: SealedBlocksWithSenders<'a, N::Block>,
|
||||
blocks: RecoveredBlocks<'a, N::Block>,
|
||||
execution_outcome: Cow<'a, ExecutionOutcome<N::Receipt>>,
|
||||
trie_updates: Option<TrieUpdates<'a>>,
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
struct SealedBlocksWithSenders<'a, B: reth_primitives_traits::Block>(
|
||||
Cow<'a, BTreeMap<BlockNumber, reth_primitives::SealedBlockWithSenders<B>>>,
|
||||
);
|
||||
struct RecoveredBlocks<
|
||||
'a,
|
||||
B: reth_primitives_traits::Block<Header: SerdeBincodeCompat, Body: SerdeBincodeCompat>
|
||||
+ 'static,
|
||||
>(Cow<'a, BTreeMap<BlockNumber, reth_primitives::RecoveredBlock<B>>>);
|
||||
|
||||
impl<B> Serialize for SealedBlocksWithSenders<'_, B>
|
||||
impl<B> Serialize for RecoveredBlocks<'_, B>
|
||||
where
|
||||
B: Block<Header: SerdeBincodeCompat, Body: SerdeBincodeCompat>,
|
||||
B: Block<Header: SerdeBincodeCompat, Body: SerdeBincodeCompat> + 'static,
|
||||
{
|
||||
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
|
||||
where
|
||||
@@ -586,23 +584,23 @@ pub(super) mod serde_bincode_compat {
|
||||
let mut state = serializer.serialize_map(Some(self.0.len()))?;
|
||||
|
||||
for (block_number, block) in self.0.iter() {
|
||||
state.serialize_entry(block_number, &SealedBlockWithSenders::<'_>::from(block))?;
|
||||
state.serialize_entry(block_number, &RecoveredBlock::<'_, B>::from(block))?;
|
||||
}
|
||||
|
||||
state.end()
|
||||
}
|
||||
}
|
||||
|
||||
impl<'de, B> Deserialize<'de> for SealedBlocksWithSenders<'_, B>
|
||||
impl<'de, B> Deserialize<'de> for RecoveredBlocks<'_, B>
|
||||
where
|
||||
B: Block<Header: SerdeBincodeCompat, Body: SerdeBincodeCompat>,
|
||||
B: Block<Header: SerdeBincodeCompat, Body: SerdeBincodeCompat> + 'static,
|
||||
{
|
||||
fn deserialize<D>(deserializer: D) -> Result<Self, D::Error>
|
||||
where
|
||||
D: Deserializer<'de>,
|
||||
{
|
||||
Ok(Self(Cow::Owned(
|
||||
BTreeMap::<BlockNumber, SealedBlockWithSenders<'_, B>>::deserialize(deserializer)
|
||||
BTreeMap::<BlockNumber, RecoveredBlock<'_, B>>::deserialize(deserializer)
|
||||
.map(|blocks| blocks.into_iter().map(|(n, b)| (n, b.into())).collect())?,
|
||||
)))
|
||||
}
|
||||
@@ -610,11 +608,13 @@ pub(super) mod serde_bincode_compat {
|
||||
|
||||
impl<'a, N> From<&'a super::Chain<N>> for Chain<'a, N>
|
||||
where
|
||||
N: NodePrimitives,
|
||||
N: NodePrimitives<
|
||||
Block: Block<Header: SerdeBincodeCompat, Body: SerdeBincodeCompat> + 'static,
|
||||
>,
|
||||
{
|
||||
fn from(value: &'a super::Chain<N>) -> Self {
|
||||
Self {
|
||||
blocks: SealedBlocksWithSenders(Cow::Borrowed(&value.blocks)),
|
||||
blocks: RecoveredBlocks(Cow::Borrowed(&value.blocks)),
|
||||
execution_outcome: Cow::Borrowed(&value.execution_outcome),
|
||||
trie_updates: value.trie_updates.as_ref().map(Into::into),
|
||||
}
|
||||
@@ -623,7 +623,9 @@ pub(super) mod serde_bincode_compat {
|
||||
|
||||
impl<'a, N> From<Chain<'a, N>> for super::Chain<N>
|
||||
where
|
||||
N: NodePrimitives,
|
||||
N: NodePrimitives<
|
||||
Block: Block<Header: SerdeBincodeCompat, Body: SerdeBincodeCompat> + 'static,
|
||||
>,
|
||||
{
|
||||
fn from(value: Chain<'a, N>) -> Self {
|
||||
Self {
|
||||
@@ -634,8 +636,13 @@ pub(super) mod serde_bincode_compat {
|
||||
}
|
||||
}
|
||||
|
||||
impl SerializeAs<super::Chain> for Chain<'_> {
|
||||
fn serialize_as<S>(source: &super::Chain, serializer: S) -> Result<S::Ok, S::Error>
|
||||
impl<N> SerializeAs<super::Chain<N>> for Chain<'_, N>
|
||||
where
|
||||
N: NodePrimitives<
|
||||
Block: Block<Header: SerdeBincodeCompat, Body: SerdeBincodeCompat> + 'static,
|
||||
>,
|
||||
{
|
||||
fn serialize_as<S>(source: &super::Chain<N>, serializer: S) -> Result<S::Ok, S::Error>
|
||||
where
|
||||
S: Serializer,
|
||||
{
|
||||
@@ -643,8 +650,13 @@ pub(super) mod serde_bincode_compat {
|
||||
}
|
||||
}
|
||||
|
||||
impl<'de> DeserializeAs<'de, super::Chain> for Chain<'de> {
|
||||
fn deserialize_as<D>(deserializer: D) -> Result<super::Chain, D::Error>
|
||||
impl<'de, N> DeserializeAs<'de, super::Chain<N>> for Chain<'de, N>
|
||||
where
|
||||
N: NodePrimitives<
|
||||
Block: Block<Header: SerdeBincodeCompat, Body: SerdeBincodeCompat> + 'static,
|
||||
>,
|
||||
{
|
||||
fn deserialize_as<D>(deserializer: D) -> Result<super::Chain<N>, D::Error>
|
||||
where
|
||||
D: Deserializer<'de>,
|
||||
{
|
||||
@@ -654,14 +666,13 @@ pub(super) mod serde_bincode_compat {
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::super::{serde_bincode_compat, Chain};
|
||||
use arbitrary::Arbitrary;
|
||||
use rand::Rng;
|
||||
use reth_primitives::SealedBlockWithSenders;
|
||||
use reth_primitives::RecoveredBlock;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use serde_with::serde_as;
|
||||
|
||||
use super::super::{serde_bincode_compat, Chain};
|
||||
|
||||
#[test]
|
||||
fn test_chain_bincode_roundtrip() {
|
||||
#[serde_as]
|
||||
@@ -675,10 +686,8 @@ pub(super) mod serde_bincode_compat {
|
||||
rand::thread_rng().fill(bytes.as_mut_slice());
|
||||
let data = Data {
|
||||
chain: Chain::new(
|
||||
vec![SealedBlockWithSenders::arbitrary(&mut arbitrary::Unstructured::new(
|
||||
&bytes,
|
||||
))
|
||||
.unwrap()],
|
||||
vec![RecoveredBlock::arbitrary(&mut arbitrary::Unstructured::new(&bytes))
|
||||
.unwrap()],
|
||||
Default::default(),
|
||||
None,
|
||||
),
|
||||
@@ -716,7 +725,7 @@ mod tests {
|
||||
|
||||
#[test]
|
||||
fn chain_append() {
|
||||
let block: SealedBlockWithSenders = Default::default();
|
||||
let block: RecoveredBlock<reth_primitives::Block> = Default::default();
|
||||
let block1_hash = B256::new([0x01; 32]);
|
||||
let block2_hash = B256::new([0x02; 32]);
|
||||
let block3_hash = B256::new([0x03; 32]);
|
||||
@@ -727,10 +736,10 @@ mod tests {
|
||||
let mut block3 = block.clone();
|
||||
let mut block4 = block;
|
||||
|
||||
block1.block.set_hash(block1_hash);
|
||||
block2.block.set_hash(block2_hash);
|
||||
block3.block.set_hash(block3_hash);
|
||||
block4.block.set_hash(block4_hash);
|
||||
block1.set_hash(block1_hash);
|
||||
block2.set_hash(block2_hash);
|
||||
block3.set_hash(block3_hash);
|
||||
block4.set_hash(block4_hash);
|
||||
|
||||
block3.set_parent_hash(block2_hash);
|
||||
|
||||
@@ -780,13 +789,13 @@ mod tests {
|
||||
vec![],
|
||||
);
|
||||
|
||||
let mut block1: SealedBlockWithSenders = Default::default();
|
||||
let mut block1: RecoveredBlock<reth_primitives::Block> = Default::default();
|
||||
let block1_hash = B256::new([15; 32]);
|
||||
block1.set_block_number(1);
|
||||
block1.set_hash(block1_hash);
|
||||
block1.push_sender(Address::new([4; 20]));
|
||||
|
||||
let mut block2: SealedBlockWithSenders = Default::default();
|
||||
let mut block2: RecoveredBlock<reth_primitives::Block> = Default::default();
|
||||
let block2_hash = B256::new([16; 32]);
|
||||
block2.set_block_number(2);
|
||||
block2.set_hash(block2_hash);
|
||||
@@ -846,8 +855,8 @@ mod tests {
|
||||
|
||||
#[test]
|
||||
fn receipts_by_block_hash() {
|
||||
// Create a default SealedBlockWithSenders object
|
||||
let block: SealedBlockWithSenders = Default::default();
|
||||
// Create a default RecoveredBlock object
|
||||
let block: RecoveredBlock<reth_primitives::Block> = Default::default();
|
||||
|
||||
// Define block hashes for block1 and block2
|
||||
let block1_hash = B256::new([0x01; 32]);
|
||||
@@ -858,8 +867,8 @@ mod tests {
|
||||
let mut block2 = block;
|
||||
|
||||
// Set the hashes of block1 and block2
|
||||
block1.block.set_hash(block1_hash);
|
||||
block2.block.set_hash(block2_hash);
|
||||
block1.set_hash(block1_hash);
|
||||
block2.set_hash(block2_hash);
|
||||
|
||||
// Create a random receipt object, receipt1
|
||||
let receipt1 = Receipt {
|
||||
|
||||
@@ -17,8 +17,7 @@ use alloy_primitives::{
|
||||
};
|
||||
use core::fmt::Display;
|
||||
use reth_consensus::ConsensusError;
|
||||
use reth_primitives::{BlockWithSenders, NodePrimitives, Receipt};
|
||||
use reth_primitives_traits::Block;
|
||||
use reth_primitives::{NodePrimitives, Receipt, RecoveredBlock};
|
||||
use reth_prune_types::PruneModes;
|
||||
use reth_revm::batch::BlockBatchRecord;
|
||||
use revm::{
|
||||
@@ -151,7 +150,7 @@ pub trait BlockExecutorProvider: Send + Sync + Clone + Unpin + 'static {
|
||||
/// the returned state.
|
||||
type Executor<DB: Database<Error: Into<ProviderError> + Display>>: for<'a> Executor<
|
||||
DB,
|
||||
Input<'a> = &'a BlockWithSenders<<Self::Primitives as NodePrimitives>::Block>,
|
||||
Input<'a> = &'a RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
Output = BlockExecutionOutput<<Self::Primitives as NodePrimitives>::Receipt>,
|
||||
Error = BlockExecutionError,
|
||||
>;
|
||||
@@ -159,7 +158,7 @@ pub trait BlockExecutorProvider: Send + Sync + Clone + Unpin + 'static {
|
||||
/// An executor that can execute a batch of blocks given a database.
|
||||
type BatchExecutor<DB: Database<Error: Into<ProviderError> + Display>>: for<'a> BatchExecutor<
|
||||
DB,
|
||||
Input<'a> = &'a BlockWithSenders<<Self::Primitives as NodePrimitives>::Block>,
|
||||
Input<'a> = &'a RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
Output = ExecutionOutcome<<Self::Primitives as NodePrimitives>::Receipt>,
|
||||
Error = BlockExecutionError,
|
||||
>;
|
||||
@@ -206,19 +205,19 @@ pub trait BlockExecutionStrategy {
|
||||
/// Applies any necessary changes before executing the block's transactions.
|
||||
fn apply_pre_execution_changes(
|
||||
&mut self,
|
||||
block: &BlockWithSenders<<Self::Primitives as NodePrimitives>::Block>,
|
||||
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
) -> Result<(), Self::Error>;
|
||||
|
||||
/// Executes all transactions in the block.
|
||||
fn execute_transactions(
|
||||
&mut self,
|
||||
block: &BlockWithSenders<<Self::Primitives as NodePrimitives>::Block>,
|
||||
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
) -> Result<ExecuteOutput<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>;
|
||||
|
||||
/// Applies any necessary changes after executing the block's transactions.
|
||||
fn apply_post_execution_changes(
|
||||
&mut self,
|
||||
block: &BlockWithSenders<<Self::Primitives as NodePrimitives>::Block>,
|
||||
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
receipts: &[<Self::Primitives as NodePrimitives>::Receipt],
|
||||
) -> Result<Requests, Self::Error>;
|
||||
|
||||
@@ -240,7 +239,7 @@ pub trait BlockExecutionStrategy {
|
||||
/// Validate a block with regard to execution results.
|
||||
fn validate_block_post_execution(
|
||||
&self,
|
||||
_block: &BlockWithSenders<<Self::Primitives as NodePrimitives>::Block>,
|
||||
_block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
|
||||
_receipts: &[<Self::Primitives as NodePrimitives>::Receipt],
|
||||
_requests: &Requests,
|
||||
) -> Result<(), ConsensusError> {
|
||||
@@ -338,7 +337,7 @@ where
|
||||
S: BlockExecutionStrategy<DB = DB>,
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
{
|
||||
type Input<'a> = &'a BlockWithSenders<<S::Primitives as NodePrimitives>::Block>;
|
||||
type Input<'a> = &'a RecoveredBlock<<S::Primitives as NodePrimitives>::Block>;
|
||||
type Output = BlockExecutionOutput<<S::Primitives as NodePrimitives>::Receipt>;
|
||||
type Error = S::Error;
|
||||
|
||||
@@ -425,7 +424,7 @@ where
|
||||
S: BlockExecutionStrategy<DB = DB, Error = BlockExecutionError>,
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
{
|
||||
type Input<'a> = &'a BlockWithSenders<<S::Primitives as NodePrimitives>::Block>;
|
||||
type Input<'a> = &'a RecoveredBlock<<S::Primitives as NodePrimitives>::Block>;
|
||||
type Output = ExecutionOutcome<<S::Primitives as NodePrimitives>::Receipt>;
|
||||
type Error = BlockExecutionError;
|
||||
|
||||
@@ -548,7 +547,7 @@ mod tests {
|
||||
struct TestExecutor<DB>(PhantomData<DB>);
|
||||
|
||||
impl<DB> Executor<DB> for TestExecutor<DB> {
|
||||
type Input<'a> = &'a BlockWithSenders;
|
||||
type Input<'a> = &'a RecoveredBlock<reth_primitives::Block>;
|
||||
type Output = BlockExecutionOutput<Receipt>;
|
||||
type Error = BlockExecutionError;
|
||||
|
||||
@@ -580,7 +579,7 @@ mod tests {
|
||||
}
|
||||
|
||||
impl<DB> BatchExecutor<DB> for TestExecutor<DB> {
|
||||
type Input<'a> = &'a BlockWithSenders;
|
||||
type Input<'a> = &'a RecoveredBlock<reth_primitives::Block>;
|
||||
type Output = ExecutionOutcome;
|
||||
type Error = BlockExecutionError;
|
||||
|
||||
@@ -661,21 +660,21 @@ mod tests {
|
||||
|
||||
fn apply_pre_execution_changes(
|
||||
&mut self,
|
||||
_block: &BlockWithSenders,
|
||||
_block: &RecoveredBlock<reth_primitives::Block>,
|
||||
) -> Result<(), Self::Error> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn execute_transactions(
|
||||
&mut self,
|
||||
_block: &BlockWithSenders,
|
||||
_block: &RecoveredBlock<reth_primitives::Block>,
|
||||
) -> Result<ExecuteOutput<Receipt>, Self::Error> {
|
||||
Ok(self.execute_transactions_result.clone())
|
||||
}
|
||||
|
||||
fn apply_post_execution_changes(
|
||||
&mut self,
|
||||
_block: &BlockWithSenders,
|
||||
_block: &RecoveredBlock<reth_primitives::Block>,
|
||||
_receipts: &[Receipt],
|
||||
) -> Result<Requests, Self::Error> {
|
||||
Ok(self.apply_post_execution_changes_result.clone())
|
||||
@@ -697,7 +696,7 @@ mod tests {
|
||||
|
||||
fn validate_block_post_execution(
|
||||
&self,
|
||||
_block: &BlockWithSenders,
|
||||
_block: &RecoveredBlock<reth_primitives::Block>,
|
||||
_receipts: &[Receipt],
|
||||
_requests: &Requests,
|
||||
) -> Result<(), ConsensusError> {
|
||||
|
||||
@@ -7,7 +7,7 @@ use alloy_consensus::BlockHeader;
|
||||
use metrics::{Counter, Gauge, Histogram};
|
||||
use reth_execution_types::BlockExecutionOutput;
|
||||
use reth_metrics::Metrics;
|
||||
use reth_primitives::BlockWithSenders;
|
||||
use reth_primitives::RecoveredBlock;
|
||||
use revm_primitives::EvmState;
|
||||
use std::time::Instant;
|
||||
|
||||
@@ -68,7 +68,7 @@ pub struct ExecutorMetrics {
|
||||
}
|
||||
|
||||
impl ExecutorMetrics {
|
||||
fn metered<F, R, B>(&self, block: &BlockWithSenders<B>, f: F) -> R
|
||||
fn metered<F, R, B>(&self, block: &RecoveredBlock<B>, f: F) -> R
|
||||
where
|
||||
F: FnOnce() -> R,
|
||||
B: reth_primitives_traits::Block,
|
||||
@@ -97,13 +97,13 @@ impl ExecutorMetrics {
|
||||
pub fn execute_metered<'a, E, DB, O, Error, B>(
|
||||
&self,
|
||||
executor: E,
|
||||
input: &'a BlockWithSenders<B>,
|
||||
input: &'a RecoveredBlock<B>,
|
||||
state_hook: Box<dyn OnStateHook>,
|
||||
) -> Result<BlockExecutionOutput<O>, Error>
|
||||
where
|
||||
E: Executor<
|
||||
DB,
|
||||
Input<'a> = &'a BlockWithSenders<B>,
|
||||
Input<'a> = &'a RecoveredBlock<B>,
|
||||
Output = BlockExecutionOutput<O>,
|
||||
Error = Error,
|
||||
>,
|
||||
@@ -131,9 +131,9 @@ impl ExecutorMetrics {
|
||||
}
|
||||
|
||||
/// Execute the given block and update metrics for the execution.
|
||||
pub fn metered_one<F, R, B>(&self, input: &BlockWithSenders<B>, f: F) -> R
|
||||
pub fn metered_one<F, R, B>(&self, input: &RecoveredBlock<B>, f: F) -> R
|
||||
where
|
||||
F: FnOnce(&BlockWithSenders<B>) -> R,
|
||||
F: FnOnce(&RecoveredBlock<B>) -> R,
|
||||
B: reth_primitives_traits::Block,
|
||||
{
|
||||
self.metered(input, || f(input))
|
||||
@@ -158,7 +158,7 @@ mod tests {
|
||||
|
||||
impl Executor<()> for MockExecutor {
|
||||
type Input<'a>
|
||||
= &'a BlockWithSenders
|
||||
= &'a RecoveredBlock<reth_primitives::Block>
|
||||
where
|
||||
Self: 'a;
|
||||
type Output = BlockExecutionOutput<()>;
|
||||
@@ -229,7 +229,7 @@ mod tests {
|
||||
fn test_executor_metrics_hook_metrics_recorded() {
|
||||
let snapshotter = setup_test_recorder();
|
||||
let metrics = ExecutorMetrics::default();
|
||||
let input = BlockWithSenders::default();
|
||||
let input = RecoveredBlock::default();
|
||||
|
||||
let (tx, _rx) = mpsc::channel();
|
||||
let expected_output = 42;
|
||||
@@ -278,7 +278,7 @@ mod tests {
|
||||
#[test]
|
||||
fn test_executor_metrics_hook_called() {
|
||||
let metrics = ExecutorMetrics::default();
|
||||
let input = BlockWithSenders::default();
|
||||
let input = RecoveredBlock::default();
|
||||
|
||||
let (tx, rx) = mpsc::channel();
|
||||
let expected_output = 42;
|
||||
|
||||
@@ -4,7 +4,7 @@ use alloy_primitives::BlockNumber;
|
||||
use core::fmt::Display;
|
||||
use reth_execution_errors::BlockExecutionError;
|
||||
use reth_execution_types::{BlockExecutionOutput, ExecutionOutcome};
|
||||
use reth_primitives::{BlockWithSenders, NodePrimitives};
|
||||
use reth_primitives::{NodePrimitives, RecoveredBlock};
|
||||
use reth_prune_types::PruneModes;
|
||||
use reth_storage_errors::provider::ProviderError;
|
||||
use revm::State;
|
||||
@@ -45,7 +45,7 @@ impl<P: NodePrimitives> BlockExecutorProvider for NoopBlockExecutorProvider<P> {
|
||||
}
|
||||
|
||||
impl<DB, P: NodePrimitives> Executor<DB> for NoopBlockExecutorProvider<P> {
|
||||
type Input<'a> = &'a BlockWithSenders<P::Block>;
|
||||
type Input<'a> = &'a RecoveredBlock<P::Block>;
|
||||
type Output = BlockExecutionOutput<P::Receipt>;
|
||||
type Error = BlockExecutionError;
|
||||
|
||||
@@ -77,7 +77,7 @@ impl<DB, P: NodePrimitives> Executor<DB> for NoopBlockExecutorProvider<P> {
|
||||
}
|
||||
|
||||
impl<DB, P: NodePrimitives> BatchExecutor<DB> for NoopBlockExecutorProvider<P> {
|
||||
type Input<'a> = &'a BlockWithSenders<P::Block>;
|
||||
type Input<'a> = &'a RecoveredBlock<P::Block>;
|
||||
type Output = ExecutionOutcome<P::Receipt>;
|
||||
type Error = BlockExecutionError;
|
||||
|
||||
|
||||
@@ -5,6 +5,7 @@ use alloy_eips::eip4895::Withdrawal;
|
||||
use alloy_primitives::{map::HashMap, Address};
|
||||
use reth_chainspec::EthereumHardforks;
|
||||
use reth_consensus_common::calc;
|
||||
use reth_primitives::SealedBlock;
|
||||
use reth_primitives_traits::BlockBody;
|
||||
|
||||
/// Collect all balance changes at the end of the block.
|
||||
@@ -14,7 +15,7 @@ use reth_primitives_traits::BlockBody;
|
||||
#[inline]
|
||||
pub fn post_block_balance_increments<ChainSpec, Block>(
|
||||
chain_spec: &ChainSpec,
|
||||
block: &Block,
|
||||
block: &SealedBlock<Block>,
|
||||
) -> HashMap<Address, u128>
|
||||
where
|
||||
ChainSpec: EthereumHardforks,
|
||||
|
||||
@@ -12,7 +12,7 @@ use alloy_primitives::BlockNumber;
|
||||
use parking_lot::Mutex;
|
||||
use reth_execution_errors::BlockExecutionError;
|
||||
use reth_execution_types::ExecutionOutcome;
|
||||
use reth_primitives::{BlockWithSenders, EthPrimitives, NodePrimitives, Receipt, Receipts};
|
||||
use reth_primitives::{EthPrimitives, NodePrimitives, Receipt, Receipts, RecoveredBlock};
|
||||
use reth_prune_types::PruneModes;
|
||||
use reth_storage_errors::provider::ProviderError;
|
||||
use revm::State;
|
||||
@@ -55,7 +55,7 @@ impl BlockExecutorProvider for MockExecutorProvider {
|
||||
}
|
||||
|
||||
impl<DB> Executor<DB> for MockExecutorProvider {
|
||||
type Input<'a> = &'a BlockWithSenders;
|
||||
type Input<'a> = &'a RecoveredBlock<reth_primitives::Block>;
|
||||
type Output = BlockExecutionOutput<Receipt>;
|
||||
type Error = BlockExecutionError;
|
||||
|
||||
@@ -97,7 +97,7 @@ impl<DB> Executor<DB> for MockExecutorProvider {
|
||||
}
|
||||
|
||||
impl<DB> BatchExecutor<DB> for MockExecutorProvider {
|
||||
type Input<'a> = &'a BlockWithSenders;
|
||||
type Input<'a> = &'a RecoveredBlock<reth_primitives::Block>;
|
||||
type Output = ExecutionOutcome;
|
||||
type Error = BlockExecutionError;
|
||||
|
||||
|
||||
@@ -10,7 +10,7 @@ use reth_evm::execute::{
|
||||
BatchExecutor, BlockExecutionError, BlockExecutionOutput, BlockExecutorProvider, Executor,
|
||||
};
|
||||
use reth_node_api::{Block as _, BlockBody as _, NodePrimitives};
|
||||
use reth_primitives::{BlockExt, BlockWithSenders, Receipt};
|
||||
use reth_primitives::{Receipt, RecoveredBlock};
|
||||
use reth_primitives_traits::{format_gas_throughput, SignedTransaction};
|
||||
use reth_provider::{
|
||||
BlockReader, Chain, HeaderProvider, ProviderError, StateProviderFactory, TransactionVariant,
|
||||
@@ -107,10 +107,9 @@ where
|
||||
let execute_start = Instant::now();
|
||||
|
||||
// Unseal the block for execution
|
||||
let (block, senders) = block.split();
|
||||
let (header, body) = block.split();
|
||||
let (unsealed_header, hash) = header.split();
|
||||
let block = P::Block::new(unsealed_header, body).with_senders_unchecked(senders);
|
||||
let (block, senders) = block.split_sealed();
|
||||
let (header, body) = block.split_sealed_header_body();
|
||||
let block = P::Block::new_sealed(header, body).with_senders(senders);
|
||||
|
||||
executor.execute_and_verify_one(&block)?;
|
||||
execution_duration += execute_start.elapsed();
|
||||
@@ -118,7 +117,7 @@ where
|
||||
// TODO(alexey): report gas metrics using `block.header.gas_used`
|
||||
|
||||
// Seal the block back and save it
|
||||
blocks.push(block.seal_unchecked(hash));
|
||||
blocks.push(block);
|
||||
|
||||
// Check if we should commit now
|
||||
let bundle_size_hint = executor.size_hint().unwrap_or_default() as u64;
|
||||
@@ -151,7 +150,7 @@ where
|
||||
/// Single block Backfill job started for a specific range.
|
||||
///
|
||||
/// It implements [`Iterator`] which executes a block each time the
|
||||
/// iterator is advanced and yields ([`BlockWithSenders`], [`BlockExecutionOutput`])
|
||||
/// iterator is advanced and yields ([`RecoveredBlock`], [`BlockExecutionOutput`])
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct SingleBlockBackfillJob<E, P> {
|
||||
pub(crate) executor: E,
|
||||
@@ -166,7 +165,7 @@ where
|
||||
P: HeaderProvider + BlockReader + StateProviderFactory,
|
||||
{
|
||||
type Item = BackfillJobResult<(
|
||||
BlockWithSenders<P::Block>,
|
||||
RecoveredBlock<P::Block>,
|
||||
BlockExecutionOutput<<E::Primitives as NodePrimitives>::Receipt>,
|
||||
)>;
|
||||
|
||||
@@ -183,7 +182,11 @@ where
|
||||
/// Converts the single block backfill job into a stream.
|
||||
pub fn into_stream(
|
||||
self,
|
||||
) -> StreamBackfillJob<E, P, (BlockWithSenders, BlockExecutionOutput<Receipt>)> {
|
||||
) -> StreamBackfillJob<
|
||||
E,
|
||||
P,
|
||||
(RecoveredBlock<reth_primitives::Block>, BlockExecutionOutput<Receipt>),
|
||||
> {
|
||||
self.into()
|
||||
}
|
||||
|
||||
@@ -192,7 +195,7 @@ where
|
||||
&self,
|
||||
block_number: u64,
|
||||
) -> BackfillJobResult<(
|
||||
BlockWithSenders<P::Block>,
|
||||
RecoveredBlock<P::Block>,
|
||||
BlockExecutionOutput<<E::Primitives as NodePrimitives>::Receipt>,
|
||||
)> {
|
||||
// Fetch the block with senders for execution.
|
||||
@@ -206,7 +209,7 @@ where
|
||||
self.provider.history_by_block_number(block_number.saturating_sub(1))?,
|
||||
));
|
||||
|
||||
trace!(target: "exex::backfill", number = block_number, txs = block_with_senders.block.body().transactions().len(), "Executing block");
|
||||
trace!(target: "exex::backfill", number = block_number, txs = block_with_senders.body().transaction_count(), "Executing block");
|
||||
|
||||
let block_execution_output = executor.execute(&block_with_senders)?;
|
||||
|
||||
@@ -310,8 +313,7 @@ mod tests {
|
||||
let (block, mut execution_output) = res?;
|
||||
execution_output.state.reverts.sort();
|
||||
|
||||
let sealed_block_with_senders = blocks_and_execution_outcomes[i].0.clone();
|
||||
let expected_block = sealed_block_with_senders.unseal();
|
||||
let expected_block = blocks_and_execution_outcomes[i].0.clone();
|
||||
let expected_output = &blocks_and_execution_outcomes[i].1;
|
||||
|
||||
assert_eq!(block, expected_block);
|
||||
|
||||
@@ -7,7 +7,7 @@ use futures::{
|
||||
};
|
||||
use reth_evm::execute::{BlockExecutionError, BlockExecutionOutput, BlockExecutorProvider};
|
||||
use reth_node_api::NodePrimitives;
|
||||
use reth_primitives::{BlockWithSenders, EthPrimitives};
|
||||
use reth_primitives::{EthPrimitives, RecoveredBlock};
|
||||
use reth_provider::{BlockReader, Chain, StateProviderFactory};
|
||||
use reth_prune_types::PruneModes;
|
||||
use reth_stages_api::ExecutionStageThresholds;
|
||||
@@ -38,7 +38,7 @@ struct BackfillTaskOutput<T> {
|
||||
type BackfillTasks<T> = FuturesOrdered<JoinHandle<BackfillTaskOutput<T>>>;
|
||||
|
||||
type SingleBlockStreamItem<N = EthPrimitives> = (
|
||||
BlockWithSenders<<N as NodePrimitives>::Block>,
|
||||
RecoveredBlock<<N as NodePrimitives>::Block>,
|
||||
BlockExecutionOutput<<N as NodePrimitives>::Receipt>,
|
||||
);
|
||||
type BatchBlockStreamItem<N = EthPrimitives> = Chain<N>;
|
||||
@@ -278,8 +278,7 @@ mod tests {
|
||||
// execute first block
|
||||
let (block, mut execution_output) = backfill_stream.next().await.unwrap().unwrap();
|
||||
execution_output.state.reverts.sort();
|
||||
let sealed_block_with_senders = blocks_and_execution_outcomes[0].0.clone();
|
||||
let expected_block = sealed_block_with_senders.unseal();
|
||||
let expected_block = blocks_and_execution_outcomes[0].0.clone();
|
||||
let expected_output = &blocks_and_execution_outcomes[0].1;
|
||||
assert_eq!(block, expected_block);
|
||||
assert_eq!(&execution_output, expected_output);
|
||||
|
||||
@@ -1,6 +1,6 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use alloy_consensus::{constants::ETH_TO_WEI, Header, TxEip2930};
|
||||
use alloy_consensus::{constants::ETH_TO_WEI, BlockHeader, Header, TxEip2930};
|
||||
use alloy_genesis::{Genesis, GenesisAccount};
|
||||
use alloy_primitives::{b256, Address, TxKind, U256};
|
||||
use eyre::OptionExt;
|
||||
@@ -8,9 +8,8 @@ use reth_chainspec::{ChainSpec, ChainSpecBuilder, EthereumHardfork, MAINNET, MIN
|
||||
use reth_evm::execute::{BatchExecutor, BlockExecutionOutput, BlockExecutorProvider, Executor};
|
||||
use reth_evm_ethereum::execute::EthExecutorProvider;
|
||||
use reth_node_api::FullNodePrimitives;
|
||||
use reth_primitives::{
|
||||
Block, BlockBody, BlockExt, BlockWithSenders, Receipt, SealedBlockWithSenders, Transaction,
|
||||
};
|
||||
use reth_primitives::{Block, BlockBody, Receipt, RecoveredBlock, Transaction};
|
||||
use reth_primitives_traits::Block as _;
|
||||
use reth_provider::{
|
||||
providers::ProviderNodeTypes, BlockWriter as _, ExecutionOutcome, LatestStateProviderRef,
|
||||
ProviderFactory,
|
||||
@@ -53,7 +52,7 @@ pub(crate) fn chain_spec(address: Address) -> Arc<ChainSpec> {
|
||||
pub(crate) fn execute_block_and_commit_to_database<N>(
|
||||
provider_factory: &ProviderFactory<N>,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
block: &BlockWithSenders,
|
||||
block: &RecoveredBlock<reth_primitives::Block>,
|
||||
) -> eyre::Result<BlockExecutionOutput<Receipt>>
|
||||
where
|
||||
N: ProviderNodeTypes<
|
||||
@@ -73,13 +72,12 @@ where
|
||||
block_execution_output.state.reverts.sort();
|
||||
|
||||
// Convert the block execution output to an execution outcome for committing to the database
|
||||
let execution_outcome = to_execution_outcome(block.number, &block_execution_output);
|
||||
let execution_outcome = to_execution_outcome(block.number(), &block_execution_output);
|
||||
|
||||
// Commit the block's execution outcome to the database
|
||||
let provider_rw = provider_factory.provider_rw()?;
|
||||
let block = block.clone().seal_slow();
|
||||
provider_rw.append_blocks_with_state(
|
||||
vec![block],
|
||||
vec![block.clone()],
|
||||
&execution_outcome,
|
||||
Default::default(),
|
||||
Default::default(),
|
||||
@@ -92,7 +90,8 @@ where
|
||||
fn blocks(
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
key_pair: Keypair,
|
||||
) -> eyre::Result<(BlockWithSenders, BlockWithSenders)> {
|
||||
) -> eyre::Result<(RecoveredBlock<reth_primitives::Block>, RecoveredBlock<reth_primitives::Block>)>
|
||||
{
|
||||
// First block has a transaction that transfers some ETH to zero address
|
||||
let block1 = Block {
|
||||
header: Header {
|
||||
@@ -128,7 +127,7 @@ fn blocks(
|
||||
// Second block resends the same transaction with increased nonce
|
||||
let block2 = Block {
|
||||
header: Header {
|
||||
parent_hash: block1.header.hash_slow(),
|
||||
parent_hash: block1.hash(),
|
||||
receipts_root: b256!(
|
||||
"d3a6acf9a244d78b33831df95d472c4128ea85bf079a1d41e32ed0b7d2244c9e"
|
||||
),
|
||||
@@ -164,7 +163,7 @@ pub(crate) fn blocks_and_execution_outputs<N>(
|
||||
provider_factory: ProviderFactory<N>,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
key_pair: Keypair,
|
||||
) -> eyre::Result<Vec<(SealedBlockWithSenders, BlockExecutionOutput<Receipt>)>>
|
||||
) -> eyre::Result<Vec<(RecoveredBlock<reth_primitives::Block>, BlockExecutionOutput<Receipt>)>>
|
||||
where
|
||||
N: ProviderNodeTypes<
|
||||
Primitives: FullNodePrimitives<
|
||||
@@ -181,9 +180,6 @@ where
|
||||
let block_output2 =
|
||||
execute_block_and_commit_to_database(&provider_factory, chain_spec, &block2)?;
|
||||
|
||||
let block1 = block1.seal_slow();
|
||||
let block2 = block2.seal_slow();
|
||||
|
||||
Ok(vec![(block1, block_output1), (block2, block_output2)])
|
||||
}
|
||||
|
||||
@@ -191,7 +187,7 @@ pub(crate) fn blocks_and_execution_outcome<N>(
|
||||
provider_factory: ProviderFactory<N>,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
key_pair: Keypair,
|
||||
) -> eyre::Result<(Vec<SealedBlockWithSenders>, ExecutionOutcome)>
|
||||
) -> eyre::Result<(Vec<RecoveredBlock<reth_primitives::Block>>, ExecutionOutcome)>
|
||||
where
|
||||
N: ProviderNodeTypes,
|
||||
N::Primitives:
|
||||
@@ -207,9 +203,6 @@ where
|
||||
let mut execution_outcome = executor.execute_and_verify_batch(vec![&block1, &block2])?;
|
||||
execution_outcome.state_mut().reverts.sort();
|
||||
|
||||
let block1 = block1.seal_slow();
|
||||
let block2 = block2.seal_slow();
|
||||
|
||||
// Commit the block's execution outcome to the database
|
||||
let provider_rw = provider_factory.provider_rw()?;
|
||||
provider_rw.append_blocks_with_state(
|
||||
|
||||
@@ -664,7 +664,7 @@ mod tests {
|
||||
use reth_db_common::init::init_genesis;
|
||||
use reth_evm::test_utils::MockExecutorProvider;
|
||||
use reth_evm_ethereum::execute::EthExecutorProvider;
|
||||
use reth_primitives::SealedBlockWithSenders;
|
||||
use reth_primitives::RecoveredBlock;
|
||||
use reth_provider::{
|
||||
providers::BlockchainProvider, test_utils::create_test_provider_factory, BlockReader,
|
||||
BlockWriter, Chain, DatabaseProviderFactory, StorageLocation, TransactionVariant,
|
||||
@@ -766,9 +766,9 @@ mod tests {
|
||||
ExExManager::new((), vec![exex_handle], 10, wal, empty_finalized_header_stream());
|
||||
|
||||
// Define the notification for testing
|
||||
let mut block1: SealedBlockWithSenders = Default::default();
|
||||
block1.block.set_hash(B256::new([0x01; 32]));
|
||||
block1.block.set_block_number(10);
|
||||
let mut block1: RecoveredBlock<reth_primitives::Block> = Default::default();
|
||||
block1.set_hash(B256::new([0x01; 32]));
|
||||
block1.set_block_number(10);
|
||||
|
||||
let notification1 = ExExNotification::ChainCommitted {
|
||||
new: Arc::new(Chain::new(vec![block1.clone()], Default::default(), Default::default())),
|
||||
@@ -784,9 +784,9 @@ mod tests {
|
||||
assert_eq!(exex_manager.next_id, 1);
|
||||
|
||||
// Push another notification
|
||||
let mut block2: SealedBlockWithSenders = Default::default();
|
||||
block2.block.set_hash(B256::new([0x02; 32]));
|
||||
block2.block.set_block_number(20);
|
||||
let mut block2: RecoveredBlock<reth_primitives::Block> = Default::default();
|
||||
block2.set_hash(B256::new([0x02; 32]));
|
||||
block2.set_block_number(20);
|
||||
|
||||
let notification2 = ExExNotification::ChainCommitted {
|
||||
new: Arc::new(Chain::new(vec![block2.clone()], Default::default(), Default::default())),
|
||||
@@ -827,7 +827,7 @@ mod tests {
|
||||
);
|
||||
|
||||
// Push some notifications to fill part of the buffer
|
||||
let mut block1: SealedBlockWithSenders = Default::default();
|
||||
let mut block1: RecoveredBlock<reth_primitives::Block> = Default::default();
|
||||
block1.set_hash(B256::new([0x01; 32]));
|
||||
block1.set_block_number(10);
|
||||
|
||||
@@ -1116,13 +1116,13 @@ mod tests {
|
||||
assert_eq!(exex_handle.next_notification_id, 0);
|
||||
|
||||
// Setup two blocks for the chain commit notification
|
||||
let mut block1: SealedBlockWithSenders = Default::default();
|
||||
block1.block.set_hash(B256::new([0x01; 32]));
|
||||
block1.block.set_block_number(10);
|
||||
let mut block1: RecoveredBlock<reth_primitives::Block> = Default::default();
|
||||
block1.set_hash(B256::new([0x01; 32]));
|
||||
block1.set_block_number(10);
|
||||
|
||||
let mut block2: SealedBlockWithSenders = Default::default();
|
||||
block2.block.set_hash(B256::new([0x02; 32]));
|
||||
block2.block.set_block_number(11);
|
||||
let mut block2: RecoveredBlock<reth_primitives::Block> = Default::default();
|
||||
block2.set_hash(B256::new([0x02; 32]));
|
||||
block2.set_block_number(11);
|
||||
|
||||
// Setup a notification
|
||||
let notification = ExExNotification::ChainCommitted {
|
||||
@@ -1169,9 +1169,9 @@ mod tests {
|
||||
// Set finished_height to a value higher than the block tip
|
||||
exex_handle.finished_height = Some(BlockNumHash::new(15, B256::random()));
|
||||
|
||||
let mut block1: SealedBlockWithSenders = Default::default();
|
||||
block1.block.set_hash(B256::new([0x01; 32]));
|
||||
block1.block.set_block_number(10);
|
||||
let mut block1: RecoveredBlock<reth_primitives::Block> = Default::default();
|
||||
block1.set_hash(B256::new([0x01; 32]));
|
||||
block1.set_block_number(10);
|
||||
|
||||
let notification = ExExNotification::ChainCommitted {
|
||||
new: Arc::new(Chain::new(vec![block1.clone()], Default::default(), Default::default())),
|
||||
@@ -1300,7 +1300,7 @@ mod tests {
|
||||
genesis_block.number + 1,
|
||||
BlockParams { parent: Some(genesis_hash), ..Default::default() },
|
||||
)
|
||||
.seal_with_senders::<reth_primitives::Block>()
|
||||
.try_recover()
|
||||
.unwrap();
|
||||
let provider_rw = provider_factory.database_provider_rw().unwrap();
|
||||
provider_rw.insert_block(block.clone(), StorageLocation::Database).unwrap();
|
||||
|
||||
@@ -435,16 +435,16 @@ where
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use crate::Wal;
|
||||
|
||||
use super::*;
|
||||
use crate::Wal;
|
||||
use alloy_consensus::Header;
|
||||
use alloy_eips::BlockNumHash;
|
||||
use eyre::OptionExt;
|
||||
use futures::StreamExt;
|
||||
use reth_db_common::init::init_genesis;
|
||||
use reth_evm_ethereum::execute::EthExecutorProvider;
|
||||
use reth_primitives::{Block, BlockExt};
|
||||
use reth_primitives::Block;
|
||||
use reth_primitives_traits::Block as _;
|
||||
use reth_provider::{
|
||||
providers::BlockchainProvider, test_utils::create_test_provider_factory, BlockWriter,
|
||||
Chain, DatabaseProviderFactory, StorageLocation,
|
||||
@@ -473,10 +473,8 @@ mod tests {
|
||||
BlockParams { parent: Some(genesis_hash), tx_count: Some(0), ..Default::default() },
|
||||
);
|
||||
let provider_rw = provider_factory.provider_rw()?;
|
||||
provider_rw.insert_block(
|
||||
node_head_block.clone().seal_with_senders().ok_or_eyre("failed to recover senders")?,
|
||||
StorageLocation::Database,
|
||||
)?;
|
||||
provider_rw
|
||||
.insert_block(node_head_block.clone().try_recover()?, StorageLocation::Database)?;
|
||||
provider_rw.commit()?;
|
||||
|
||||
let node_head = Head {
|
||||
@@ -494,8 +492,7 @@ mod tests {
|
||||
node_head.number + 1,
|
||||
BlockParams { parent: Some(node_head.hash), ..Default::default() },
|
||||
)
|
||||
.seal_with_senders()
|
||||
.ok_or_eyre("failed to recover senders")?],
|
||||
.try_recover()?],
|
||||
Default::default(),
|
||||
None,
|
||||
)),
|
||||
@@ -565,8 +562,7 @@ mod tests {
|
||||
..Default::default()
|
||||
}
|
||||
.seal_slow()
|
||||
.seal_with_senders()
|
||||
.ok_or_eyre("failed to recover senders")?],
|
||||
.try_recover()?],
|
||||
Default::default(),
|
||||
None,
|
||||
)),
|
||||
@@ -611,8 +607,7 @@ mod tests {
|
||||
genesis_block.number + 1,
|
||||
BlockParams { parent: Some(genesis_hash), tx_count: Some(0), ..Default::default() },
|
||||
)
|
||||
.seal_with_senders::<reth_primitives::Block>()
|
||||
.ok_or_eyre("failed to recover senders")?;
|
||||
.try_recover()?;
|
||||
let node_head = Head {
|
||||
number: node_head_block.number,
|
||||
hash: node_head_block.hash(),
|
||||
@@ -638,10 +633,7 @@ mod tests {
|
||||
let exex_head = ExExHead { block: exex_head_block.num_hash() };
|
||||
let exex_head_notification = ExExNotification::ChainCommitted {
|
||||
new: Arc::new(Chain::new(
|
||||
vec![exex_head_block
|
||||
.clone()
|
||||
.seal_with_senders()
|
||||
.ok_or_eyre("failed to recover senders")?],
|
||||
vec![exex_head_block.clone().try_recover()?],
|
||||
Default::default(),
|
||||
None,
|
||||
)),
|
||||
@@ -655,8 +647,7 @@ mod tests {
|
||||
node_head.number + 1,
|
||||
BlockParams { parent: Some(node_head.hash), ..Default::default() },
|
||||
)
|
||||
.seal_with_senders()
|
||||
.ok_or_eyre("failed to recover senders")?],
|
||||
.try_recover()?],
|
||||
Default::default(),
|
||||
None,
|
||||
)),
|
||||
@@ -713,10 +704,7 @@ mod tests {
|
||||
);
|
||||
let exex_head_notification = ExExNotification::ChainCommitted {
|
||||
new: Arc::new(Chain::new(
|
||||
vec![exex_head_block
|
||||
.clone()
|
||||
.seal_with_senders()
|
||||
.ok_or_eyre("failed to recover senders")?],
|
||||
vec![exex_head_block.clone().try_recover()?],
|
||||
Default::default(),
|
||||
None,
|
||||
)),
|
||||
@@ -736,8 +724,7 @@ mod tests {
|
||||
genesis_block.number + 1,
|
||||
BlockParams { parent: Some(genesis_hash), ..Default::default() },
|
||||
)
|
||||
.seal_with_senders()
|
||||
.ok_or_eyre("failed to recover senders")?],
|
||||
.try_recover()?],
|
||||
Default::default(),
|
||||
None,
|
||||
)),
|
||||
|
||||
@@ -233,18 +233,15 @@ where
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::sync::Arc;
|
||||
|
||||
use crate::wal::{cache::CachedBlock, Wal};
|
||||
use alloy_primitives::B256;
|
||||
use eyre::OptionExt;
|
||||
use itertools::Itertools;
|
||||
use reth_exex_types::ExExNotification;
|
||||
use reth_provider::Chain;
|
||||
use reth_testing_utils::generators::{
|
||||
self, random_block, random_block_range, BlockParams, BlockRangeParams,
|
||||
};
|
||||
|
||||
use crate::wal::{cache::CachedBlock, Wal};
|
||||
use std::sync::Arc;
|
||||
|
||||
fn read_notifications(wal: &Wal) -> eyre::Result<Vec<ExExNotification>> {
|
||||
wal.inner.storage.files_range()?.map_or(Ok(Vec::new()), |range| {
|
||||
@@ -279,26 +276,20 @@ mod tests {
|
||||
// Create 4 canonical blocks and one reorged block with number 2
|
||||
let blocks = random_block_range(&mut rng, 0..=3, BlockRangeParams::default())
|
||||
.into_iter()
|
||||
.map(|block| {
|
||||
block
|
||||
.seal_with_senders::<reth_primitives::Block>()
|
||||
.ok_or_eyre("failed to recover senders")
|
||||
})
|
||||
.collect::<eyre::Result<Vec<_>>>()?;
|
||||
.map(|block| block.try_recover())
|
||||
.collect::<Result<Vec<_>, _>>()?;
|
||||
let block_1_reorged = random_block(
|
||||
&mut rng,
|
||||
1,
|
||||
BlockParams { parent: Some(blocks[0].hash()), ..Default::default() },
|
||||
)
|
||||
.seal_with_senders::<reth_primitives::Block>()
|
||||
.ok_or_eyre("failed to recover senders")?;
|
||||
.try_recover()?;
|
||||
let block_2_reorged = random_block(
|
||||
&mut rng,
|
||||
2,
|
||||
BlockParams { parent: Some(blocks[1].hash()), ..Default::default() },
|
||||
)
|
||||
.seal_with_senders::<reth_primitives::Block>()
|
||||
.ok_or_eyre("failed to recover senders")?;
|
||||
.try_recover()?;
|
||||
|
||||
// Create notifications for the above blocks.
|
||||
// 1. Committed notification for blocks with number 0 and 1
|
||||
|
||||
@@ -178,14 +178,11 @@ where
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::{fs::File, sync::Arc};
|
||||
|
||||
use eyre::OptionExt;
|
||||
use super::Storage;
|
||||
use reth_exex_types::ExExNotification;
|
||||
use reth_provider::Chain;
|
||||
use reth_testing_utils::generators::{self, random_block};
|
||||
|
||||
use super::Storage;
|
||||
use std::{fs::File, sync::Arc};
|
||||
|
||||
#[test]
|
||||
fn test_roundtrip() -> eyre::Result<()> {
|
||||
@@ -194,12 +191,8 @@ mod tests {
|
||||
let temp_dir = tempfile::tempdir()?;
|
||||
let storage: Storage = Storage::new(&temp_dir)?;
|
||||
|
||||
let old_block = random_block(&mut rng, 0, Default::default())
|
||||
.seal_with_senders()
|
||||
.ok_or_eyre("failed to recover senders")?;
|
||||
let new_block = random_block(&mut rng, 0, Default::default())
|
||||
.seal_with_senders()
|
||||
.ok_or_eyre("failed to recover senders")?;
|
||||
let old_block = random_block(&mut rng, 0, Default::default()).try_recover()?;
|
||||
let new_block = random_block(&mut rng, 0, Default::default()).try_recover()?;
|
||||
|
||||
let notification = ExExNotification::ChainReorged {
|
||||
new: Arc::new(Chain::new(vec![new_block], Default::default(), None)),
|
||||
|
||||
@@ -27,6 +27,7 @@ reth-node-builder = { workspace = true, features = ["test-utils"] }
|
||||
reth-node-ethereum.workspace = true
|
||||
reth-payload-builder.workspace = true
|
||||
reth-primitives.workspace = true
|
||||
reth-primitives-traits.workspace = true
|
||||
reth-provider = { workspace = true, features = ["test-utils"] }
|
||||
reth-tasks.workspace = true
|
||||
reth-transaction-pool = { workspace = true, features = ["test-utils"] }
|
||||
|
||||
@@ -45,12 +45,14 @@ use reth_node_ethereum::{
|
||||
EthEngineTypes, EthEvmConfig,
|
||||
};
|
||||
use reth_payload_builder::noop::NoopPayloadBuilderService;
|
||||
use reth_primitives::{BlockExt, EthPrimitives, Head, SealedBlockWithSenders, TransactionSigned};
|
||||
use reth_provider::{providers::StaticFileProvider, BlockReader, EthStorage, ProviderFactory};
|
||||
use reth_primitives::{EthPrimitives, Head, RecoveredBlock, TransactionSigned};
|
||||
use reth_primitives_traits::Block as _;
|
||||
use reth_provider::{
|
||||
providers::{BlockchainProvider, StaticFileProvider},
|
||||
BlockReader, EthStorage, ProviderFactory,
|
||||
};
|
||||
use reth_tasks::TaskManager;
|
||||
use reth_transaction_pool::test_utils::{testing_pool, TestPool};
|
||||
|
||||
use reth_provider::providers::BlockchainProvider;
|
||||
use tempfile::TempDir;
|
||||
use thiserror::Error;
|
||||
use tokio::sync::mpsc::{Sender, UnboundedReceiver};
|
||||
@@ -185,7 +187,7 @@ pub type TestExExContext = ExExContext<Adapter>;
|
||||
#[derive(Debug)]
|
||||
pub struct TestExExHandle {
|
||||
/// Genesis block that was inserted into the storage
|
||||
pub genesis: SealedBlockWithSenders,
|
||||
pub genesis: RecoveredBlock<reth_primitives::Block>,
|
||||
/// Provider Factory for accessing the emphemeral storage of the host node
|
||||
pub provider_factory: ProviderFactory<NodeTypesWithDBAdapter<TestNode, TmpDB>>,
|
||||
/// Channel for receiving events from the Execution Extension
|
||||
@@ -304,8 +306,7 @@ pub async fn test_exex_context_with_chain_spec(
|
||||
.block_by_hash(genesis_hash)?
|
||||
.ok_or_else(|| eyre::eyre!("genesis block not found"))?
|
||||
.seal_slow()
|
||||
.seal_with_senders::<reth_primitives::Block>()
|
||||
.ok_or_else(|| eyre::eyre!("failed to recover senders"))?;
|
||||
.try_recover()?;
|
||||
|
||||
let head = Head {
|
||||
number: genesis.number,
|
||||
|
||||
@@ -174,7 +174,7 @@ pub(super) mod serde_bincode_compat {
|
||||
use arbitrary::Arbitrary;
|
||||
use rand::Rng;
|
||||
use reth_execution_types::Chain;
|
||||
use reth_primitives::SealedBlockWithSenders;
|
||||
use reth_primitives::RecoveredBlock;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use serde_with::serde_as;
|
||||
use std::sync::Arc;
|
||||
@@ -193,18 +193,14 @@ pub(super) mod serde_bincode_compat {
|
||||
let data = Data {
|
||||
notification: ExExNotification::ChainReorged {
|
||||
old: Arc::new(Chain::new(
|
||||
vec![SealedBlockWithSenders::arbitrary(&mut arbitrary::Unstructured::new(
|
||||
&bytes,
|
||||
))
|
||||
.unwrap()],
|
||||
vec![RecoveredBlock::arbitrary(&mut arbitrary::Unstructured::new(&bytes))
|
||||
.unwrap()],
|
||||
Default::default(),
|
||||
None,
|
||||
)),
|
||||
new: Arc::new(Chain::new(
|
||||
vec![SealedBlockWithSenders::arbitrary(&mut arbitrary::Unstructured::new(
|
||||
&bytes,
|
||||
))
|
||||
.unwrap()],
|
||||
vec![RecoveredBlock::arbitrary(&mut arbitrary::Unstructured::new(&bytes))
|
||||
.unwrap()],
|
||||
Default::default(),
|
||||
None,
|
||||
)),
|
||||
|
||||
@@ -15,7 +15,7 @@ use reth_network_p2p::{
|
||||
error::{DownloadError, DownloadResult},
|
||||
};
|
||||
use reth_primitives::SealedHeader;
|
||||
use reth_primitives_traits::size::InMemorySize;
|
||||
use reth_primitives_traits::{size::InMemorySize, Block};
|
||||
use reth_storage_api::HeaderProvider;
|
||||
use reth_tasks::{TaskSpawner, TokioTaskExecutor};
|
||||
use std::{
|
||||
@@ -35,11 +35,15 @@ use tracing::info;
|
||||
/// All blocks in a batch are fetched at the same time.
|
||||
#[must_use = "Stream does nothing unless polled"]
|
||||
#[derive(Debug)]
|
||||
pub struct BodiesDownloader<B: BodiesClient, Provider: HeaderProvider> {
|
||||
pub struct BodiesDownloader<
|
||||
B: Block,
|
||||
C: BodiesClient<Body = B::Body>,
|
||||
Provider: HeaderProvider<Header = B::Header>,
|
||||
> {
|
||||
/// The bodies client
|
||||
client: Arc<B>,
|
||||
client: Arc<C>,
|
||||
/// The consensus client
|
||||
consensus: Arc<dyn Consensus<Provider::Header, B::Body, Error = ConsensusError>>,
|
||||
consensus: Arc<dyn Consensus<B, Error = ConsensusError>>,
|
||||
/// The database handle
|
||||
provider: Provider,
|
||||
/// The maximum number of non-empty blocks per one request
|
||||
@@ -57,19 +61,20 @@ pub struct BodiesDownloader<B: BodiesClient, Provider: HeaderProvider> {
|
||||
/// The latest block number returned.
|
||||
latest_queued_block_number: Option<BlockNumber>,
|
||||
/// Requests in progress
|
||||
in_progress_queue: BodiesRequestQueue<Provider::Header, B>,
|
||||
in_progress_queue: BodiesRequestQueue<B, C>,
|
||||
/// Buffered responses
|
||||
buffered_responses: BinaryHeap<OrderedBodiesResponse<Provider::Header, B::Body>>,
|
||||
buffered_responses: BinaryHeap<OrderedBodiesResponse<B>>,
|
||||
/// Queued body responses that can be returned for insertion into the database.
|
||||
queued_bodies: Vec<BlockResponse<Provider::Header, B::Body>>,
|
||||
queued_bodies: Vec<BlockResponse<B>>,
|
||||
/// The bodies downloader metrics.
|
||||
metrics: BodyDownloaderMetrics,
|
||||
}
|
||||
|
||||
impl<B, Provider> BodiesDownloader<B, Provider>
|
||||
impl<B, C, Provider> BodiesDownloader<B, C, Provider>
|
||||
where
|
||||
B: BodiesClient<Body: InMemorySize> + 'static,
|
||||
Provider: HeaderProvider<Header: BlockHeader> + Unpin + 'static,
|
||||
B: Block,
|
||||
C: BodiesClient<Body = B::Body> + 'static,
|
||||
Provider: HeaderProvider<Header = B::Header> + Unpin + 'static,
|
||||
{
|
||||
/// Returns the next contiguous request.
|
||||
fn next_headers_request(&self) -> DownloadResult<Option<Vec<SealedHeader<Provider::Header>>>> {
|
||||
@@ -97,7 +102,7 @@ where
|
||||
&self,
|
||||
range: RangeInclusive<BlockNumber>,
|
||||
max_non_empty: u64,
|
||||
) -> DownloadResult<Option<Vec<SealedHeader<Provider::Header>>>> {
|
||||
) -> DownloadResult<Option<Vec<SealedHeader<B::Header>>>> {
|
||||
if range.is_empty() || max_non_empty == 0 {
|
||||
return Ok(None)
|
||||
}
|
||||
@@ -193,16 +198,14 @@ where
|
||||
}
|
||||
|
||||
/// Queues bodies and sets the latest queued block number
|
||||
fn queue_bodies(&mut self, bodies: Vec<BlockResponse<Provider::Header, B::Body>>) {
|
||||
fn queue_bodies(&mut self, bodies: Vec<BlockResponse<B>>) {
|
||||
self.latest_queued_block_number = Some(bodies.last().expect("is not empty").block_number());
|
||||
self.queued_bodies.extend(bodies);
|
||||
self.metrics.queued_blocks.set(self.queued_bodies.len() as f64);
|
||||
}
|
||||
|
||||
/// Removes the next response from the buffer.
|
||||
fn pop_buffered_response(
|
||||
&mut self,
|
||||
) -> Option<OrderedBodiesResponse<Provider::Header, B::Body>> {
|
||||
fn pop_buffered_response(&mut self) -> Option<OrderedBodiesResponse<B>> {
|
||||
let resp = self.buffered_responses.pop()?;
|
||||
self.metrics.buffered_responses.decrement(1.);
|
||||
self.buffered_blocks_size_bytes -= resp.size();
|
||||
@@ -212,10 +215,10 @@ where
|
||||
}
|
||||
|
||||
/// Adds a new response to the internal buffer
|
||||
fn buffer_bodies_response(&mut self, response: Vec<BlockResponse<Provider::Header, B::Body>>) {
|
||||
fn buffer_bodies_response(&mut self, response: Vec<BlockResponse<B>>) {
|
||||
// take into account capacity
|
||||
let size = response.iter().map(BlockResponse::size).sum::<usize>() +
|
||||
response.capacity() * mem::size_of::<BlockResponse<Provider::Header, B::Body>>();
|
||||
response.capacity() * mem::size_of::<BlockResponse<B>>();
|
||||
|
||||
let response = OrderedBodiesResponse { resp: response, size };
|
||||
let response_len = response.len();
|
||||
@@ -229,7 +232,7 @@ where
|
||||
}
|
||||
|
||||
/// Returns a response if it's first block number matches the next expected.
|
||||
fn try_next_buffered(&mut self) -> Option<Vec<BlockResponse<Provider::Header, B::Body>>> {
|
||||
fn try_next_buffered(&mut self) -> Option<Vec<BlockResponse<B>>> {
|
||||
if let Some(next) = self.buffered_responses.peek() {
|
||||
let expected = self.next_expected_block_number();
|
||||
let next_block_range = next.block_range();
|
||||
@@ -255,7 +258,7 @@ where
|
||||
|
||||
/// Returns the next batch of block bodies that can be returned if we have enough buffered
|
||||
/// bodies
|
||||
fn try_split_next_batch(&mut self) -> Option<Vec<BlockResponse<Provider::Header, B::Body>>> {
|
||||
fn try_split_next_batch(&mut self) -> Option<Vec<BlockResponse<B>>> {
|
||||
if self.queued_bodies.len() >= self.stream_batch_size {
|
||||
let next_batch = self.queued_bodies.drain(..self.stream_batch_size).collect::<Vec<_>>();
|
||||
self.queued_bodies.shrink_to_fit();
|
||||
@@ -280,24 +283,19 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<B, Provider> BodiesDownloader<B, Provider>
|
||||
impl<B, C, Provider> BodiesDownloader<B, C, Provider>
|
||||
where
|
||||
B: BodiesClient + 'static,
|
||||
Provider: HeaderProvider + Unpin + 'static,
|
||||
Self: BodyDownloader + 'static,
|
||||
B: Block + 'static,
|
||||
C: BodiesClient<Body = B::Body> + 'static,
|
||||
Provider: HeaderProvider<Header = B::Header> + Unpin + 'static,
|
||||
{
|
||||
/// Spawns the downloader task via [`tokio::task::spawn`]
|
||||
pub fn into_task(
|
||||
self,
|
||||
) -> TaskDownloader<<Self as BodyDownloader>::Header, <Self as BodyDownloader>::Body> {
|
||||
pub fn into_task(self) -> TaskDownloader<B> {
|
||||
self.into_task_with(&TokioTaskExecutor::default())
|
||||
}
|
||||
|
||||
/// Convert the downloader into a [`TaskDownloader`] by spawning it via the given spawner.
|
||||
pub fn into_task_with<S>(
|
||||
self,
|
||||
spawner: &S,
|
||||
) -> TaskDownloader<<Self as BodyDownloader>::Header, <Self as BodyDownloader>::Body>
|
||||
pub fn into_task_with<S>(self, spawner: &S) -> TaskDownloader<B>
|
||||
where
|
||||
S: TaskSpawner,
|
||||
{
|
||||
@@ -305,13 +303,13 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<B, Provider> BodyDownloader for BodiesDownloader<B, Provider>
|
||||
impl<B, C, Provider> BodyDownloader for BodiesDownloader<B, C, Provider>
|
||||
where
|
||||
B: BodiesClient<Body: Debug + InMemorySize> + 'static,
|
||||
Provider: HeaderProvider + Unpin + 'static,
|
||||
B: Block + 'static,
|
||||
C: BodiesClient<Body = B::Body> + 'static,
|
||||
Provider: HeaderProvider<Header = B::Header> + Unpin + 'static,
|
||||
{
|
||||
type Header = Provider::Header;
|
||||
type Body = B::Body;
|
||||
type Block = B;
|
||||
|
||||
/// Set a new download range (exclusive).
|
||||
///
|
||||
@@ -356,12 +354,13 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<B, Provider> Stream for BodiesDownloader<B, Provider>
|
||||
impl<B, C, Provider> Stream for BodiesDownloader<B, C, Provider>
|
||||
where
|
||||
B: BodiesClient<Body: InMemorySize> + 'static,
|
||||
Provider: HeaderProvider + Unpin + 'static,
|
||||
B: Block + 'static,
|
||||
C: BodiesClient<Body = B::Body> + 'static,
|
||||
Provider: HeaderProvider<Header = B::Header> + Unpin + 'static,
|
||||
{
|
||||
type Item = BodyDownloaderResult<Provider::Header, B::Body>;
|
||||
type Item = BodyDownloaderResult<B>;
|
||||
|
||||
fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
let this = self.get_mut();
|
||||
@@ -443,13 +442,13 @@ where
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
struct OrderedBodiesResponse<H, B> {
|
||||
resp: Vec<BlockResponse<H, B>>,
|
||||
struct OrderedBodiesResponse<B: Block> {
|
||||
resp: Vec<BlockResponse<B>>,
|
||||
/// The total size of the response in bytes
|
||||
size: usize,
|
||||
}
|
||||
|
||||
impl<H, B> OrderedBodiesResponse<H, B> {
|
||||
impl<B: Block> OrderedBodiesResponse<B> {
|
||||
#[inline]
|
||||
fn len(&self) -> usize {
|
||||
self.resp.len()
|
||||
@@ -464,10 +463,7 @@ impl<H, B> OrderedBodiesResponse<H, B> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<H, B> OrderedBodiesResponse<H, B>
|
||||
where
|
||||
H: BlockHeader,
|
||||
{
|
||||
impl<B: Block> OrderedBodiesResponse<B> {
|
||||
/// Returns the block number of the first element
|
||||
///
|
||||
/// # Panics
|
||||
@@ -485,21 +481,21 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<H: BlockHeader, B> PartialEq for OrderedBodiesResponse<H, B> {
|
||||
impl<B: Block> PartialEq for OrderedBodiesResponse<B> {
|
||||
fn eq(&self, other: &Self) -> bool {
|
||||
self.first_block_number() == other.first_block_number()
|
||||
}
|
||||
}
|
||||
|
||||
impl<H: BlockHeader, B> Eq for OrderedBodiesResponse<H, B> {}
|
||||
impl<B: Block> Eq for OrderedBodiesResponse<B> {}
|
||||
|
||||
impl<H: BlockHeader, B> PartialOrd for OrderedBodiesResponse<H, B> {
|
||||
impl<B: Block> PartialOrd for OrderedBodiesResponse<B> {
|
||||
fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
|
||||
Some(self.cmp(other))
|
||||
}
|
||||
}
|
||||
|
||||
impl<H: BlockHeader, B> Ord for OrderedBodiesResponse<H, B> {
|
||||
impl<B: Block> Ord for OrderedBodiesResponse<B> {
|
||||
fn cmp(&self, other: &Self) -> Ordering {
|
||||
self.first_block_number().cmp(&other.first_block_number()).reverse()
|
||||
}
|
||||
@@ -576,15 +572,16 @@ impl BodiesDownloaderBuilder {
|
||||
}
|
||||
|
||||
/// Consume self and return the concurrent downloader.
|
||||
pub fn build<B, Provider>(
|
||||
pub fn build<B, C, Provider>(
|
||||
self,
|
||||
client: B,
|
||||
consensus: Arc<dyn Consensus<Provider::Header, B::Body, Error = ConsensusError>>,
|
||||
client: C,
|
||||
consensus: Arc<dyn Consensus<B, Error = ConsensusError>>,
|
||||
provider: Provider,
|
||||
) -> BodiesDownloader<B, Provider>
|
||||
) -> BodiesDownloader<B, C, Provider>
|
||||
where
|
||||
B: BodiesClient + 'static,
|
||||
Provider: HeaderProvider,
|
||||
B: Block,
|
||||
C: BodiesClient<Body = B::Body> + 'static,
|
||||
Provider: HeaderProvider<Header = B::Header>,
|
||||
{
|
||||
let Self {
|
||||
request_limit,
|
||||
@@ -646,15 +643,16 @@ mod tests {
|
||||
);
|
||||
let (_static_dir, static_dir_path) = create_test_static_files_dir();
|
||||
|
||||
let mut downloader = BodiesDownloaderBuilder::default().build(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
db,
|
||||
MAINNET.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
),
|
||||
);
|
||||
let mut downloader = BodiesDownloaderBuilder::default()
|
||||
.build::<reth_primitives::Block, _, _>(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
db,
|
||||
MAINNET.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
),
|
||||
);
|
||||
downloader.set_download_range(0..=19).expect("failed to set download range");
|
||||
|
||||
assert_matches!(
|
||||
@@ -689,16 +687,17 @@ mod tests {
|
||||
let client = Arc::new(TestBodiesClient::default().with_bodies(bodies.clone()));
|
||||
let (_static_dir, static_dir_path) = create_test_static_files_dir();
|
||||
|
||||
let mut downloader =
|
||||
BodiesDownloaderBuilder::default().with_request_limit(request_limit).build(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
db,
|
||||
MAINNET.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
),
|
||||
);
|
||||
let mut downloader = BodiesDownloaderBuilder::default()
|
||||
.with_request_limit(request_limit)
|
||||
.build::<reth_primitives::Block, _, _>(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
db,
|
||||
MAINNET.clone(),
|
||||
StaticFileProvider::read_write(static_dir_path).unwrap(),
|
||||
),
|
||||
);
|
||||
downloader.set_download_range(0..=199).expect("failed to set download range");
|
||||
|
||||
let _ = downloader.collect::<Vec<_>>().await;
|
||||
@@ -724,7 +723,7 @@ mod tests {
|
||||
let mut downloader = BodiesDownloaderBuilder::default()
|
||||
.with_stream_batch_size(stream_batch_size)
|
||||
.with_request_limit(request_limit)
|
||||
.build(
|
||||
.build::<reth_primitives::Block, _, _>(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
@@ -760,7 +759,9 @@ mod tests {
|
||||
let client = Arc::new(TestBodiesClient::default().with_bodies(bodies.clone()));
|
||||
let (_static_dir, static_dir_path) = create_test_static_files_dir();
|
||||
|
||||
let mut downloader = BodiesDownloaderBuilder::default().with_stream_batch_size(100).build(
|
||||
let mut downloader = BodiesDownloaderBuilder::default()
|
||||
.with_stream_batch_size(100)
|
||||
.build::<reth_primitives::Block, _, _>(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
@@ -806,7 +807,7 @@ mod tests {
|
||||
.with_stream_batch_size(10)
|
||||
.with_request_limit(1)
|
||||
.with_max_buffered_blocks_size_bytes(1)
|
||||
.build(
|
||||
.build::<reth_primitives::Block, _, _>(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
@@ -843,7 +844,7 @@ mod tests {
|
||||
let mut downloader = BodiesDownloaderBuilder::default()
|
||||
.with_request_limit(3)
|
||||
.with_stream_batch_size(100)
|
||||
.build(
|
||||
.build::<reth_primitives::Block, _, _>(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
ProviderFactory::<MockNodeTypesWithDB>::new(
|
||||
|
||||
@@ -4,29 +4,26 @@ use reth_network_p2p::{
|
||||
bodies::{downloader::BodyDownloader, response::BlockResponse},
|
||||
error::{DownloadError, DownloadResult},
|
||||
};
|
||||
use reth_primitives_traits::Block;
|
||||
use std::{fmt::Debug, ops::RangeInclusive};
|
||||
|
||||
/// A [`BodyDownloader`] implementation that does nothing.
|
||||
#[derive(Debug, Default)]
|
||||
#[non_exhaustive]
|
||||
pub struct NoopBodiesDownloader<H, B> {
|
||||
_header: std::marker::PhantomData<H>,
|
||||
_body: std::marker::PhantomData<B>,
|
||||
pub struct NoopBodiesDownloader<B> {
|
||||
_block: std::marker::PhantomData<B>,
|
||||
}
|
||||
|
||||
impl<H: Debug + Send + Sync + Unpin + 'static, B: Debug + Send + Sync + Unpin + 'static>
|
||||
BodyDownloader for NoopBodiesDownloader<H, B>
|
||||
{
|
||||
type Body = B;
|
||||
type Header = H;
|
||||
impl<B: Block + 'static> BodyDownloader for NoopBodiesDownloader<B> {
|
||||
type Block = B;
|
||||
|
||||
fn set_download_range(&mut self, _: RangeInclusive<BlockNumber>) -> DownloadResult<()> {
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl<H, B> Stream for NoopBodiesDownloader<H, B> {
|
||||
type Item = Result<Vec<BlockResponse<H, B>>, DownloadError>;
|
||||
impl<B: Block + 'static> Stream for NoopBodiesDownloader<B> {
|
||||
type Item = Result<Vec<BlockResponse<B>>, DownloadError>;
|
||||
|
||||
fn poll_next(
|
||||
self: std::pin::Pin<&mut Self>,
|
||||
|
||||
@@ -10,7 +10,7 @@ use reth_network_p2p::{
|
||||
error::DownloadResult,
|
||||
};
|
||||
use reth_primitives::SealedHeader;
|
||||
use reth_primitives_traits::InMemorySize;
|
||||
use reth_primitives_traits::Block;
|
||||
use std::{
|
||||
pin::Pin,
|
||||
sync::Arc,
|
||||
@@ -20,19 +20,19 @@ use std::{
|
||||
/// The wrapper around [`FuturesUnordered`] that keeps information
|
||||
/// about the blocks currently being requested.
|
||||
#[derive(Debug)]
|
||||
pub(crate) struct BodiesRequestQueue<H, B: BodiesClient> {
|
||||
pub(crate) struct BodiesRequestQueue<B: Block, C: BodiesClient<Body = B::Body>> {
|
||||
/// Inner body request queue.
|
||||
inner: FuturesUnordered<BodiesRequestFuture<H, B>>,
|
||||
inner: FuturesUnordered<BodiesRequestFuture<B, C>>,
|
||||
/// The downloader metrics.
|
||||
metrics: BodyDownloaderMetrics,
|
||||
/// Last requested block number.
|
||||
pub(crate) last_requested_block_number: Option<BlockNumber>,
|
||||
}
|
||||
|
||||
impl<H, B> BodiesRequestQueue<H, B>
|
||||
impl<B, C> BodiesRequestQueue<B, C>
|
||||
where
|
||||
B: BodiesClient + 'static,
|
||||
H: BlockHeader,
|
||||
B: Block,
|
||||
C: BodiesClient<Body = B::Body> + 'static,
|
||||
{
|
||||
/// Create new instance of request queue.
|
||||
pub(crate) fn new(metrics: BodyDownloaderMetrics) -> Self {
|
||||
@@ -58,9 +58,9 @@ where
|
||||
/// Expects a sorted list of headers.
|
||||
pub(crate) fn push_new_request(
|
||||
&mut self,
|
||||
client: Arc<B>,
|
||||
consensus: Arc<dyn Consensus<H, B::Body, Error = ConsensusError>>,
|
||||
request: Vec<SealedHeader<H>>,
|
||||
client: Arc<C>,
|
||||
consensus: Arc<dyn Consensus<B, Error = ConsensusError>>,
|
||||
request: Vec<SealedHeader<B::Header>>,
|
||||
) {
|
||||
// Set last max requested block number
|
||||
self.last_requested_block_number = request
|
||||
@@ -78,12 +78,12 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<H, B> Stream for BodiesRequestQueue<H, B>
|
||||
impl<B, C> Stream for BodiesRequestQueue<B, C>
|
||||
where
|
||||
H: BlockHeader + Send + Sync + Unpin + 'static,
|
||||
B: BodiesClient<Body: InMemorySize> + 'static,
|
||||
B: Block + 'static,
|
||||
C: BodiesClient<Body = B::Body> + 'static,
|
||||
{
|
||||
type Item = DownloadResult<Vec<BlockResponse<H, B::Body>>>;
|
||||
type Item = DownloadResult<Vec<BlockResponse<B>>>;
|
||||
|
||||
fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
self.get_mut().inner.poll_next_unpin(cx)
|
||||
|
||||
@@ -10,7 +10,7 @@ use reth_network_p2p::{
|
||||
};
|
||||
use reth_network_peers::{PeerId, WithPeerId};
|
||||
use reth_primitives::{BlockBody, GotExpected, SealedBlock, SealedHeader};
|
||||
use reth_primitives_traits::InMemorySize;
|
||||
use reth_primitives_traits::{Block, InMemorySize};
|
||||
use std::{
|
||||
collections::VecDeque,
|
||||
mem,
|
||||
@@ -38,31 +38,31 @@ use std::{
|
||||
/// All errors regarding the response cause the peer to get penalized, meaning that adversaries
|
||||
/// that try to give us bodies that do not match the requested order are going to be penalized
|
||||
/// and eventually disconnected.
|
||||
pub(crate) struct BodiesRequestFuture<H, B: BodiesClient> {
|
||||
client: Arc<B>,
|
||||
consensus: Arc<dyn Consensus<H, B::Body, Error = ConsensusError>>,
|
||||
pub(crate) struct BodiesRequestFuture<B: Block, C: BodiesClient<Body = B::Body>> {
|
||||
client: Arc<C>,
|
||||
consensus: Arc<dyn Consensus<B, Error = ConsensusError>>,
|
||||
metrics: BodyDownloaderMetrics,
|
||||
/// Metrics for individual responses. This can be used to observe how the size (in bytes) of
|
||||
/// responses change while bodies are being downloaded.
|
||||
response_metrics: ResponseMetrics,
|
||||
// Headers to download. The collection is shrunk as responses are buffered.
|
||||
pending_headers: VecDeque<SealedHeader<H>>,
|
||||
pending_headers: VecDeque<SealedHeader<B::Header>>,
|
||||
/// Internal buffer for all blocks
|
||||
buffer: Vec<BlockResponse<H, B::Body>>,
|
||||
fut: Option<B::Output>,
|
||||
buffer: Vec<BlockResponse<B>>,
|
||||
fut: Option<C::Output>,
|
||||
/// Tracks how many bodies we requested in the last request.
|
||||
last_request_len: Option<usize>,
|
||||
}
|
||||
|
||||
impl<H, B> BodiesRequestFuture<H, B>
|
||||
impl<B, C> BodiesRequestFuture<B, C>
|
||||
where
|
||||
H: BlockHeader,
|
||||
B: BodiesClient + 'static,
|
||||
B: Block,
|
||||
C: BodiesClient<Body = B::Body> + 'static,
|
||||
{
|
||||
/// Returns an empty future. Use [`BodiesRequestFuture::with_headers`] to set the request.
|
||||
pub(crate) fn new(
|
||||
client: Arc<B>,
|
||||
consensus: Arc<dyn Consensus<H, B::Body, Error = ConsensusError>>,
|
||||
client: Arc<C>,
|
||||
consensus: Arc<dyn Consensus<B, Error = ConsensusError>>,
|
||||
metrics: BodyDownloaderMetrics,
|
||||
) -> Self {
|
||||
Self {
|
||||
@@ -77,7 +77,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn with_headers(mut self, headers: Vec<SealedHeader<H>>) -> Self {
|
||||
pub(crate) fn with_headers(mut self, headers: Vec<SealedHeader<B::Header>>) -> Self {
|
||||
self.buffer.reserve_exact(headers.len());
|
||||
self.pending_headers = VecDeque::from(headers);
|
||||
// Submit the request only if there are any headers to download.
|
||||
@@ -163,9 +163,9 @@ where
|
||||
///
|
||||
/// This method removes headers from the internal collection.
|
||||
/// If the response fails validation, then the header will be put back.
|
||||
fn try_buffer_blocks(&mut self, bodies: Vec<B::Body>) -> DownloadResult<()>
|
||||
fn try_buffer_blocks(&mut self, bodies: Vec<C::Body>) -> DownloadResult<()>
|
||||
where
|
||||
B::Body: InMemorySize,
|
||||
C::Body: InMemorySize,
|
||||
{
|
||||
let bodies_capacity = bodies.capacity();
|
||||
let bodies_len = bodies.len();
|
||||
@@ -180,7 +180,7 @@ where
|
||||
|
||||
if next_header.is_empty() {
|
||||
// increment empty block body metric
|
||||
total_size += mem::size_of::<BlockBody>();
|
||||
total_size += mem::size_of::<C::Body>();
|
||||
self.buffer.push(BlockResponse::Empty(next_header));
|
||||
} else {
|
||||
let next_body = bodies.next().unwrap();
|
||||
@@ -188,7 +188,7 @@ where
|
||||
// increment full block body metric
|
||||
total_size += next_body.size();
|
||||
|
||||
let block = SealedBlock::new(next_header, next_body);
|
||||
let block = SealedBlock::from_sealed_parts(next_header, next_body);
|
||||
|
||||
if let Err(error) = self.consensus.validate_block_pre_execution(&block) {
|
||||
// Body is invalid, put the header back and return an error
|
||||
@@ -214,12 +214,12 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<H, B> Future for BodiesRequestFuture<H, B>
|
||||
impl<B, C> Future for BodiesRequestFuture<B, C>
|
||||
where
|
||||
H: BlockHeader + Unpin + Send + Sync + 'static,
|
||||
B: BodiesClient<Body: InMemorySize> + 'static,
|
||||
B: Block + 'static,
|
||||
C: BodiesClient<Body = B::Body> + 'static,
|
||||
{
|
||||
type Output = DownloadResult<Vec<BlockResponse<H, B::Body>>>;
|
||||
type Output = DownloadResult<Vec<BlockResponse<B>>>;
|
||||
|
||||
fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
|
||||
let this = self.get_mut();
|
||||
@@ -275,7 +275,7 @@ mod tests {
|
||||
let headers = random_header_range(&mut rng, 0..20, B256::ZERO);
|
||||
|
||||
let client = Arc::new(TestBodiesClient::default());
|
||||
let fut = BodiesRequestFuture::new(
|
||||
let fut = BodiesRequestFuture::<reth_primitives::Block, _>::new(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
BodyDownloaderMetrics::default(),
|
||||
@@ -299,7 +299,7 @@ mod tests {
|
||||
let client = Arc::new(
|
||||
TestBodiesClient::default().with_bodies(bodies.clone()).with_max_batch_size(batch_size),
|
||||
);
|
||||
let fut = BodiesRequestFuture::new(
|
||||
let fut = BodiesRequestFuture::<reth_primitives::Block, _>::new(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
BodyDownloaderMetrics::default(),
|
||||
|
||||
@@ -6,6 +6,7 @@ use reth_network_p2p::{
|
||||
bodies::downloader::{BodyDownloader, BodyDownloaderResult},
|
||||
error::DownloadResult,
|
||||
};
|
||||
use reth_primitives_traits::Block;
|
||||
use reth_tasks::{TaskSpawner, TokioTaskExecutor};
|
||||
use std::{
|
||||
fmt::Debug,
|
||||
@@ -24,15 +25,13 @@ pub const BODIES_TASK_BUFFER_SIZE: usize = 4;
|
||||
/// A [BodyDownloader] that drives a spawned [BodyDownloader] on a spawned task.
|
||||
#[derive(Debug)]
|
||||
#[pin_project]
|
||||
pub struct TaskDownloader<H, B> {
|
||||
pub struct TaskDownloader<B: Block> {
|
||||
#[pin]
|
||||
from_downloader: ReceiverStream<BodyDownloaderResult<H, B>>,
|
||||
from_downloader: ReceiverStream<BodyDownloaderResult<B>>,
|
||||
to_downloader: UnboundedSender<RangeInclusive<BlockNumber>>,
|
||||
}
|
||||
|
||||
// === impl TaskDownloader ===
|
||||
|
||||
impl<H: Send + Sync + Unpin + 'static, B: Send + Sync + Unpin + 'static> TaskDownloader<H, B> {
|
||||
impl<B: Block + 'static> TaskDownloader<B> {
|
||||
/// Spawns the given `downloader` via [`tokio::task::spawn`] returns a [`TaskDownloader`] that's
|
||||
/// connected to that task.
|
||||
///
|
||||
@@ -46,25 +45,27 @@ impl<H: Send + Sync + Unpin + 'static, B: Send + Sync + Unpin + 'static> TaskDow
|
||||
/// use reth_consensus::{Consensus, ConsensusError};
|
||||
/// use reth_downloaders::bodies::{bodies::BodiesDownloaderBuilder, task::TaskDownloader};
|
||||
/// use reth_network_p2p::bodies::client::BodiesClient;
|
||||
/// use reth_primitives_traits::InMemorySize;
|
||||
/// use reth_primitives_traits::{Block, InMemorySize};
|
||||
/// use reth_storage_api::HeaderProvider;
|
||||
/// use std::{fmt::Debug, sync::Arc};
|
||||
///
|
||||
/// fn t<
|
||||
/// B: BodiesClient<Body: Debug + InMemorySize> + 'static,
|
||||
/// Provider: HeaderProvider<Header = alloy_consensus::Header> + Unpin + 'static,
|
||||
/// B: Block + 'static,
|
||||
/// C: BodiesClient<Body = B::Body> + 'static,
|
||||
/// Provider: HeaderProvider<Header = B::Header> + Unpin + 'static,
|
||||
/// >(
|
||||
/// client: Arc<B>,
|
||||
/// consensus: Arc<dyn Consensus<Provider::Header, B::Body, Error = ConsensusError>>,
|
||||
/// client: Arc<C>,
|
||||
/// consensus: Arc<dyn Consensus<B, Error = ConsensusError>>,
|
||||
/// provider: Provider,
|
||||
/// ) {
|
||||
/// let downloader = BodiesDownloaderBuilder::default().build(client, consensus, provider);
|
||||
/// let downloader =
|
||||
/// BodiesDownloaderBuilder::default().build::<B, _, _>(client, consensus, provider);
|
||||
/// let downloader = TaskDownloader::spawn(downloader);
|
||||
/// }
|
||||
/// ```
|
||||
pub fn spawn<T>(downloader: T) -> Self
|
||||
where
|
||||
T: BodyDownloader<Header = H, Body = B> + 'static,
|
||||
T: BodyDownloader<Block = B> + 'static,
|
||||
{
|
||||
Self::spawn_with(downloader, &TokioTaskExecutor::default())
|
||||
}
|
||||
@@ -73,7 +74,7 @@ impl<H: Send + Sync + Unpin + 'static, B: Send + Sync + Unpin + 'static> TaskDow
|
||||
/// that's connected to that task.
|
||||
pub fn spawn_with<T, S>(downloader: T, spawner: &S) -> Self
|
||||
where
|
||||
T: BodyDownloader<Header = H, Body = B> + 'static,
|
||||
T: BodyDownloader<Block = B> + 'static,
|
||||
S: TaskSpawner,
|
||||
{
|
||||
let (bodies_tx, bodies_rx) = mpsc::channel(BODIES_TASK_BUFFER_SIZE);
|
||||
@@ -91,11 +92,8 @@ impl<H: Send + Sync + Unpin + 'static, B: Send + Sync + Unpin + 'static> TaskDow
|
||||
}
|
||||
}
|
||||
|
||||
impl<H: Debug + Send + Sync + Unpin + 'static, B: Debug + Send + Sync + Unpin + 'static>
|
||||
BodyDownloader for TaskDownloader<H, B>
|
||||
{
|
||||
type Header = H;
|
||||
type Body = B;
|
||||
impl<B: Block + 'static> BodyDownloader for TaskDownloader<B> {
|
||||
type Block = B;
|
||||
|
||||
fn set_download_range(&mut self, range: RangeInclusive<BlockNumber>) -> DownloadResult<()> {
|
||||
let _ = self.to_downloader.send(range);
|
||||
@@ -103,8 +101,8 @@ impl<H: Debug + Send + Sync + Unpin + 'static, B: Debug + Send + Sync + Unpin +
|
||||
}
|
||||
}
|
||||
|
||||
impl<H, B> Stream for TaskDownloader<H, B> {
|
||||
type Item = BodyDownloaderResult<H, B>;
|
||||
impl<B: Block + 'static> Stream for TaskDownloader<B> {
|
||||
type Item = BodyDownloaderResult<B>;
|
||||
|
||||
fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
self.project().from_downloader.poll_next(cx)
|
||||
@@ -114,7 +112,7 @@ impl<H, B> Stream for TaskDownloader<H, B> {
|
||||
/// A [`BodyDownloader`] that runs on its own task
|
||||
struct SpawnedDownloader<T: BodyDownloader> {
|
||||
updates: UnboundedReceiverStream<RangeInclusive<BlockNumber>>,
|
||||
bodies_tx: PollSender<BodyDownloaderResult<T::Header, T::Body>>,
|
||||
bodies_tx: PollSender<BodyDownloaderResult<T::Block>>,
|
||||
downloader: T,
|
||||
}
|
||||
|
||||
@@ -197,7 +195,7 @@ mod tests {
|
||||
let client = Arc::new(
|
||||
TestBodiesClient::default().with_bodies(bodies.clone()).with_should_delay(true),
|
||||
);
|
||||
let downloader = BodiesDownloaderBuilder::default().build(
|
||||
let downloader = BodiesDownloaderBuilder::default().build::<reth_primitives::Block, _, _>(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
factory,
|
||||
@@ -219,7 +217,7 @@ mod tests {
|
||||
reth_tracing::init_test_tracing();
|
||||
let factory = create_test_provider_factory();
|
||||
|
||||
let downloader = BodiesDownloaderBuilder::default().build(
|
||||
let downloader = BodiesDownloaderBuilder::default().build::<reth_primitives::Block, _, _>(
|
||||
Arc::new(TestBodiesClient::default()),
|
||||
Arc::new(TestConsensus::default()),
|
||||
factory,
|
||||
|
||||
@@ -7,13 +7,14 @@ use alloy_primitives::B256;
|
||||
use reth_db::{tables, DatabaseEnv};
|
||||
use reth_db_api::{database::Database, transaction::DbTxMut};
|
||||
use reth_network_p2p::bodies::response::BlockResponse;
|
||||
use reth_primitives::{Block, BlockBody, SealedBlock, SealedHeader};
|
||||
use reth_primitives::{BlockBody, SealedBlock, SealedHeader};
|
||||
use reth_primitives_traits::Block;
|
||||
use std::collections::HashMap;
|
||||
|
||||
pub(crate) fn zip_blocks<'a, H: Clone + BlockHeader + 'a, B>(
|
||||
headers: impl Iterator<Item = &'a SealedHeader<H>>,
|
||||
bodies: &mut HashMap<B256, B>,
|
||||
) -> Vec<BlockResponse<H, B>> {
|
||||
pub(crate) fn zip_blocks<'a, B: Block>(
|
||||
headers: impl Iterator<Item = &'a SealedHeader<B::Header>>,
|
||||
bodies: &mut HashMap<B256, B::Body>,
|
||||
) -> Vec<BlockResponse<B>> {
|
||||
headers
|
||||
.into_iter()
|
||||
.map(|header| {
|
||||
@@ -21,7 +22,7 @@ pub(crate) fn zip_blocks<'a, H: Clone + BlockHeader + 'a, B>(
|
||||
if header.is_empty() {
|
||||
BlockResponse::Empty(header.clone())
|
||||
} else {
|
||||
BlockResponse::Full(SealedBlock::new(header.clone(), body))
|
||||
BlockResponse::Full(SealedBlock::from_sealed_parts(header.clone(), body))
|
||||
}
|
||||
})
|
||||
.collect()
|
||||
@@ -30,7 +31,7 @@ pub(crate) fn zip_blocks<'a, H: Clone + BlockHeader + 'a, B>(
|
||||
pub(crate) fn create_raw_bodies(
|
||||
headers: impl IntoIterator<Item = SealedHeader>,
|
||||
bodies: &mut HashMap<B256, BlockBody>,
|
||||
) -> Vec<Block> {
|
||||
) -> Vec<reth_primitives::Block> {
|
||||
headers
|
||||
.into_iter()
|
||||
.map(|header| {
|
||||
@@ -45,7 +46,7 @@ pub(crate) fn insert_headers(db: &DatabaseEnv, headers: &[SealedHeader]) {
|
||||
db.update(|tx| {
|
||||
for header in headers {
|
||||
tx.put::<tables::CanonicalHeaders>(header.number, header.hash()).unwrap();
|
||||
tx.put::<tables::Headers>(header.number, header.clone().unseal()).unwrap();
|
||||
tx.put::<tables::Headers>(header.number, header.clone_header()).unwrap();
|
||||
}
|
||||
})
|
||||
.expect("failed to commit")
|
||||
|
||||
@@ -11,6 +11,7 @@ use reth_network_p2p::{
|
||||
error::RequestError,
|
||||
headers::client::{HeadersClient, HeadersDirection, HeadersFut, HeadersRequest},
|
||||
priority::Priority,
|
||||
BlockClient,
|
||||
};
|
||||
use reth_network_peers::PeerId;
|
||||
use reth_primitives::SealedHeader;
|
||||
@@ -40,7 +41,7 @@ pub const DEFAULT_BYTE_LEN_CHUNK_CHAIN_FILE: u64 = 1_000_000_000;
|
||||
/// transactions in memory for use in the bodies stage.
|
||||
///
|
||||
/// This reads the entire file into memory, so it is not suitable for large files.
|
||||
#[derive(Debug)]
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct FileClient<B: Block = reth_primitives::Block> {
|
||||
/// The buffered headers retrieved when fetching new bodies.
|
||||
headers: HashMap<BlockNumber, B::Header>,
|
||||
@@ -116,7 +117,7 @@ impl<B: FullBlock> FileClient<B> {
|
||||
/// Clones and returns the highest header of this client has or `None` if empty. Seals header
|
||||
/// before returning.
|
||||
pub fn tip_header(&self) -> Option<SealedHeader<B::Header>> {
|
||||
self.headers.get(&self.max_block()?).map(|h| SealedHeader::seal(h.clone()))
|
||||
self.headers.get(&self.max_block()?).map(|h| SealedHeader::seal_slow(h.clone()))
|
||||
}
|
||||
|
||||
/// Returns true if all blocks are canonical (no gaps)
|
||||
@@ -350,6 +351,10 @@ impl<B: FullBlock> DownloadClient for FileClient<B> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<B: FullBlock> BlockClient for FileClient<B> {
|
||||
type Block = B;
|
||||
}
|
||||
|
||||
/// Chunks file into several [`FileClient`]s.
|
||||
#[derive(Debug)]
|
||||
pub struct ChunkedFileReader {
|
||||
@@ -546,11 +551,12 @@ mod tests {
|
||||
|
||||
let client: Arc<FileClient> =
|
||||
Arc::new(FileClient::from_file(file.into()).await.unwrap().with_bodies(bodies.clone()));
|
||||
let mut downloader = BodiesDownloaderBuilder::default().build(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
factory,
|
||||
);
|
||||
let mut downloader = BodiesDownloaderBuilder::default()
|
||||
.build::<reth_primitives::Block, _, _>(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
factory,
|
||||
);
|
||||
downloader.set_download_range(0..=19).expect("failed to set download range");
|
||||
|
||||
assert_matches!(
|
||||
@@ -571,10 +577,10 @@ mod tests {
|
||||
let file = tempfile::tempfile().unwrap();
|
||||
let client: Arc<FileClient> = Arc::new(
|
||||
FileClient::from_file(file.into()).await.unwrap().with_headers(HashMap::from([
|
||||
(0u64, p0.clone().unseal()),
|
||||
(1, p1.clone().unseal()),
|
||||
(2, p2.clone().unseal()),
|
||||
(3, p3.clone().unseal()),
|
||||
(0u64, p0.clone_header()),
|
||||
(1, p1.clone_header()),
|
||||
(2, p2.clone_header()),
|
||||
(3, p3.clone_header()),
|
||||
])),
|
||||
);
|
||||
|
||||
@@ -628,11 +634,12 @@ mod tests {
|
||||
// insert headers in db for the bodies downloader
|
||||
insert_headers(factory.db_ref().db(), &headers);
|
||||
|
||||
let mut downloader = BodiesDownloaderBuilder::default().build(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
factory,
|
||||
);
|
||||
let mut downloader = BodiesDownloaderBuilder::default()
|
||||
.build::<reth_primitives::Block, _, _>(
|
||||
client.clone(),
|
||||
Arc::new(TestConsensus::default()),
|
||||
factory,
|
||||
);
|
||||
downloader.set_download_range(0..=19).expect("failed to set download range");
|
||||
|
||||
assert_matches!(
|
||||
|
||||
@@ -1,3 +1,4 @@
|
||||
use alloy_primitives::Sealable;
|
||||
use futures::Stream;
|
||||
use reth_network_p2p::headers::{
|
||||
downloader::{HeaderDownloader, SyncTarget},
|
||||
@@ -11,7 +12,9 @@ use std::fmt::Debug;
|
||||
#[non_exhaustive]
|
||||
pub struct NoopHeaderDownloader<H>(std::marker::PhantomData<H>);
|
||||
|
||||
impl<H: Debug + Send + Sync + Unpin + 'static> HeaderDownloader for NoopHeaderDownloader<H> {
|
||||
impl<H: Sealable + Debug + Send + Sync + Unpin + 'static> HeaderDownloader
|
||||
for NoopHeaderDownloader<H>
|
||||
{
|
||||
type Header = H;
|
||||
|
||||
fn update_local_head(&mut self, _: SealedHeader<H>) {}
|
||||
@@ -21,7 +24,7 @@ impl<H: Debug + Send + Sync + Unpin + 'static> HeaderDownloader for NoopHeaderDo
|
||||
fn set_batch_size(&mut self, _: usize) {}
|
||||
}
|
||||
|
||||
impl<H> Stream for NoopHeaderDownloader<H> {
|
||||
impl<H: Sealable> Stream for NoopHeaderDownloader<H> {
|
||||
type Item = Result<Vec<SealedHeader<H>>, HeadersDownloaderError<H>>;
|
||||
|
||||
fn poll_next(
|
||||
|
||||
@@ -4,7 +4,7 @@ use super::task::TaskDownloader;
|
||||
use crate::metrics::HeaderDownloaderMetrics;
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_eips::BlockHashOrNumber;
|
||||
use alloy_primitives::{BlockNumber, B256};
|
||||
use alloy_primitives::{BlockNumber, Sealable, B256};
|
||||
use futures::{stream::Stream, FutureExt};
|
||||
use futures_util::{stream::FuturesUnordered, StreamExt};
|
||||
use rayon::prelude::*;
|
||||
@@ -40,14 +40,14 @@ const REQUESTS_PER_PEER_MULTIPLIER: usize = 5;
|
||||
|
||||
/// Wrapper for internal downloader errors.
|
||||
#[derive(Error, Debug)]
|
||||
enum ReverseHeadersDownloaderError<H> {
|
||||
enum ReverseHeadersDownloaderError<H: Sealable> {
|
||||
#[error(transparent)]
|
||||
Downloader(#[from] HeadersDownloaderError<H>),
|
||||
#[error(transparent)]
|
||||
Response(#[from] Box<HeadersResponseError>),
|
||||
}
|
||||
|
||||
impl<H> From<HeadersResponseError> for ReverseHeadersDownloaderError<H> {
|
||||
impl<H: Sealable> From<HeadersResponseError> for ReverseHeadersDownloaderError<H> {
|
||||
fn from(value: HeadersResponseError) -> Self {
|
||||
Self::Response(Box::new(value))
|
||||
}
|
||||
@@ -251,7 +251,8 @@ where
|
||||
) -> Result<(), ReverseHeadersDownloaderError<H::Header>> {
|
||||
let mut validated = Vec::with_capacity(headers.len());
|
||||
|
||||
let sealed_headers = headers.into_par_iter().map(SealedHeader::seal).collect::<Vec<_>>();
|
||||
let sealed_headers =
|
||||
headers.into_par_iter().map(SealedHeader::seal_slow).collect::<Vec<_>>();
|
||||
for parent in sealed_headers {
|
||||
// Validate that the header is the parent header of the last validated header.
|
||||
if let Some(validated_header) =
|
||||
@@ -378,7 +379,7 @@ where
|
||||
}
|
||||
|
||||
let header = headers.swap_remove(0);
|
||||
let target = SealedHeader::seal(header);
|
||||
let target = SealedHeader::seal_slow(header);
|
||||
|
||||
match sync_target {
|
||||
SyncTargetBlock::Hash(hash) | SyncTargetBlock::HashAndNumber { hash, .. } => {
|
||||
|
||||
@@ -1,3 +1,4 @@
|
||||
use alloy_primitives::Sealable;
|
||||
use futures::{FutureExt, Stream};
|
||||
use futures_util::StreamExt;
|
||||
use pin_project::pin_project;
|
||||
@@ -23,7 +24,7 @@ pub const HEADERS_TASK_BUFFER_SIZE: usize = 8;
|
||||
/// A [HeaderDownloader] that drives a spawned [HeaderDownloader] on a spawned task.
|
||||
#[derive(Debug)]
|
||||
#[pin_project]
|
||||
pub struct TaskDownloader<H> {
|
||||
pub struct TaskDownloader<H: Sealable> {
|
||||
#[pin]
|
||||
from_downloader: ReceiverStream<HeadersDownloaderResult<Vec<SealedHeader<H>>, H>>,
|
||||
to_downloader: UnboundedSender<DownloaderUpdates<H>>,
|
||||
@@ -31,7 +32,7 @@ pub struct TaskDownloader<H> {
|
||||
|
||||
// === impl TaskDownloader ===
|
||||
|
||||
impl<H: Send + Sync + Unpin + 'static> TaskDownloader<H> {
|
||||
impl<H: Sealable + Send + Sync + Unpin + 'static> TaskDownloader<H> {
|
||||
/// Spawns the given `downloader` via [`tokio::task::spawn`] and returns a [`TaskDownloader`]
|
||||
/// that's connected to that task.
|
||||
///
|
||||
@@ -83,7 +84,7 @@ impl<H: Send + Sync + Unpin + 'static> TaskDownloader<H> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<H: Debug + Send + Sync + Unpin + 'static> HeaderDownloader for TaskDownloader<H> {
|
||||
impl<H: Sealable + Debug + Send + Sync + Unpin + 'static> HeaderDownloader for TaskDownloader<H> {
|
||||
type Header = H;
|
||||
|
||||
fn update_sync_gap(&mut self, head: SealedHeader<H>, target: SyncTarget) {
|
||||
@@ -103,7 +104,7 @@ impl<H: Debug + Send + Sync + Unpin + 'static> HeaderDownloader for TaskDownload
|
||||
}
|
||||
}
|
||||
|
||||
impl<H> Stream for TaskDownloader<H> {
|
||||
impl<H: Sealable> Stream for TaskDownloader<H> {
|
||||
type Item = HeadersDownloaderResult<Vec<SealedHeader<H>>, H>;
|
||||
|
||||
fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
|
||||
@@ -9,5 +9,5 @@ pub(crate) fn child_header(parent: &SealedHeader) -> SealedHeader {
|
||||
let mut child = parent.as_ref().clone();
|
||||
child.number += 1;
|
||||
child.parent_hash = parent.hash_slow();
|
||||
SealedHeader::seal(child)
|
||||
SealedHeader::seal_slow(child)
|
||||
}
|
||||
|
||||
@@ -11,6 +11,7 @@ use reth_network_p2p::{
|
||||
error::{PeerRequestResult, RequestError},
|
||||
headers::client::{HeadersClient, HeadersRequest},
|
||||
priority::Priority,
|
||||
BlockClient,
|
||||
};
|
||||
use reth_network_peers::PeerId;
|
||||
use reth_network_types::ReputationChangeKind;
|
||||
@@ -96,3 +97,7 @@ impl<N: NetworkPrimitives> BodiesClient for FetchClient<N> {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<N: NetworkPrimitives> BlockClient for FetchClient<N> {
|
||||
type Block = N::Block;
|
||||
}
|
||||
|
||||
@@ -2,10 +2,11 @@ use super::response::BlockResponse;
|
||||
use crate::error::DownloadResult;
|
||||
use alloy_primitives::BlockNumber;
|
||||
use futures::Stream;
|
||||
use std::{fmt::Debug, ops::RangeInclusive};
|
||||
use reth_primitives_traits::Block;
|
||||
use std::ops::RangeInclusive;
|
||||
|
||||
/// Body downloader return type.
|
||||
pub type BodyDownloaderResult<H, B> = DownloadResult<Vec<BlockResponse<H, B>>>;
|
||||
pub type BodyDownloaderResult<B> = DownloadResult<Vec<BlockResponse<B>>>;
|
||||
|
||||
/// A downloader capable of fetching and yielding block bodies from block headers.
|
||||
///
|
||||
@@ -13,13 +14,10 @@ pub type BodyDownloaderResult<H, B> = DownloadResult<Vec<BlockResponse<H, B>>>;
|
||||
/// while a [`BodiesClient`][crate::bodies::client::BodiesClient] represents a client capable of
|
||||
/// fulfilling these requests.
|
||||
pub trait BodyDownloader:
|
||||
Send + Sync + Stream<Item = BodyDownloaderResult<Self::Header, Self::Body>> + Unpin
|
||||
Send + Sync + Stream<Item = BodyDownloaderResult<Self::Block>> + Unpin
|
||||
{
|
||||
/// The type of header that is being used
|
||||
type Header: Debug + Send + Sync + Unpin + 'static;
|
||||
|
||||
/// The type of the body that is being downloaded.
|
||||
type Body: Debug + Send + Sync + Unpin + 'static;
|
||||
/// The Block type this downloader supports
|
||||
type Block: Block + 'static;
|
||||
|
||||
/// Method for setting the download range.
|
||||
fn set_download_range(&mut self, range: RangeInclusive<BlockNumber>) -> DownloadResult<()>;
|
||||
|
||||
@@ -1,31 +1,26 @@
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_primitives::{BlockNumber, U256};
|
||||
use reth_primitives::{BlockBody, SealedBlock, SealedHeader};
|
||||
use reth_primitives_traits::InMemorySize;
|
||||
use reth_primitives::{SealedBlock, SealedHeader};
|
||||
use reth_primitives_traits::{Block, InMemorySize};
|
||||
/// The block response
|
||||
#[derive(PartialEq, Eq, Debug, Clone)]
|
||||
pub enum BlockResponse<H, B = BlockBody> {
|
||||
pub enum BlockResponse<B: Block> {
|
||||
/// Full block response (with transactions or ommers)
|
||||
Full(SealedBlock<H, B>),
|
||||
Full(SealedBlock<B>),
|
||||
/// The empty block response
|
||||
Empty(SealedHeader<H>),
|
||||
Empty(SealedHeader<B::Header>),
|
||||
}
|
||||
|
||||
impl<H, B> BlockResponse<H, B>
|
||||
impl<B> BlockResponse<B>
|
||||
where
|
||||
H: BlockHeader,
|
||||
B: Block,
|
||||
{
|
||||
/// Return the reference to the response header
|
||||
pub const fn header(&self) -> &SealedHeader<H> {
|
||||
match self {
|
||||
Self::Full(block) => block.sealed_header(),
|
||||
Self::Empty(header) => header,
|
||||
}
|
||||
}
|
||||
|
||||
/// Return the block number
|
||||
pub fn block_number(&self) -> BlockNumber {
|
||||
self.header().number()
|
||||
match self {
|
||||
Self::Full(block) => block.number(),
|
||||
Self::Empty(header) => header.number(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Return the reference to the response header
|
||||
@@ -37,7 +32,7 @@ where
|
||||
}
|
||||
|
||||
/// Return the reference to the response body
|
||||
pub fn into_body(self) -> Option<B> {
|
||||
pub fn into_body(self) -> Option<B::Body> {
|
||||
match self {
|
||||
Self::Full(block) => Some(block.into_body()),
|
||||
Self::Empty(_) => None,
|
||||
@@ -45,7 +40,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<H: InMemorySize, B: InMemorySize> InMemorySize for BlockResponse<H, B> {
|
||||
impl<B: Block> InMemorySize for BlockResponse<B> {
|
||||
#[inline]
|
||||
fn size(&self) -> usize {
|
||||
match self {
|
||||
|
||||
@@ -30,7 +30,7 @@ where
|
||||
Client: BlockClient,
|
||||
{
|
||||
client: Client,
|
||||
consensus: Arc<dyn Consensus<Client::Header, Client::Body, Error = ConsensusError>>,
|
||||
consensus: Arc<dyn Consensus<Client::Block, Error = ConsensusError>>,
|
||||
}
|
||||
|
||||
impl<Client> FullBlockClient<Client>
|
||||
@@ -40,7 +40,7 @@ where
|
||||
/// Creates a new instance of `FullBlockClient`.
|
||||
pub fn new(
|
||||
client: Client,
|
||||
consensus: Arc<dyn Consensus<Client::Header, Client::Body, Error = ConsensusError>>,
|
||||
consensus: Arc<dyn Consensus<Client::Block, Error = ConsensusError>>,
|
||||
) -> Self {
|
||||
Self { client, consensus }
|
||||
}
|
||||
@@ -118,7 +118,7 @@ where
|
||||
Client: BlockClient,
|
||||
{
|
||||
client: Client,
|
||||
consensus: Arc<dyn Consensus<Client::Header, Client::Body, Error = ConsensusError>>,
|
||||
consensus: Arc<dyn Consensus<Client::Block, Error = ConsensusError>>,
|
||||
hash: B256,
|
||||
request: FullBlockRequest<Client>,
|
||||
header: Option<SealedHeader<Client::Header>>,
|
||||
@@ -140,7 +140,7 @@ where
|
||||
}
|
||||
|
||||
/// Returns the [`SealedBlock`] if the request is complete and valid.
|
||||
fn take_block(&mut self) -> Option<SealedBlock<Client::Header, Client::Body>> {
|
||||
fn take_block(&mut self) -> Option<SealedBlock<Client::Block>> {
|
||||
if self.header.is_none() || self.body.is_none() {
|
||||
return None
|
||||
}
|
||||
@@ -148,7 +148,7 @@ where
|
||||
let header = self.header.take().unwrap();
|
||||
let resp = self.body.take().unwrap();
|
||||
match resp {
|
||||
BodyResponse::Validated(body) => Some(SealedBlock::new(header, body)),
|
||||
BodyResponse::Validated(body) => Some(SealedBlock::from_sealed_parts(header, body)),
|
||||
BodyResponse::PendingValidation(resp) => {
|
||||
// ensure the block is valid, else retry
|
||||
if let Err(err) = self.consensus.validate_body_against_header(resp.data(), &header)
|
||||
@@ -159,7 +159,7 @@ where
|
||||
self.request.body = Some(self.client.get_block_body(self.hash));
|
||||
return None
|
||||
}
|
||||
Some(SealedBlock::new(header, resp.into_data()))
|
||||
Some(SealedBlock::from_sealed_parts(header, resp.into_data()))
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -182,7 +182,7 @@ impl<Client> Future for FetchFullBlockFuture<Client>
|
||||
where
|
||||
Client: BlockClient<Header: BlockHeader + Sealable> + 'static,
|
||||
{
|
||||
type Output = SealedBlock<Client::Header, Client::Body>;
|
||||
type Output = SealedBlock<Client::Block>;
|
||||
|
||||
fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
|
||||
let this = self.get_mut();
|
||||
@@ -196,7 +196,7 @@ where
|
||||
match res {
|
||||
Ok(maybe_header) => {
|
||||
let (peer, maybe_header) =
|
||||
maybe_header.map(|h| h.map(SealedHeader::seal)).split();
|
||||
maybe_header.map(|h| h.map(SealedHeader::seal_slow)).split();
|
||||
if let Some(header) = maybe_header {
|
||||
if header.hash() == this.hash {
|
||||
this.header = Some(header);
|
||||
@@ -330,7 +330,7 @@ where
|
||||
/// The client used to fetch headers and bodies.
|
||||
client: Client,
|
||||
/// The consensus instance used to validate the blocks.
|
||||
consensus: Arc<dyn Consensus<Client::Header, Client::Body, Error = ConsensusError>>,
|
||||
consensus: Arc<dyn Consensus<Client::Block, Error = ConsensusError>>,
|
||||
/// The block hash to start fetching from (inclusive).
|
||||
start_hash: B256,
|
||||
/// How many blocks to fetch: `len([start_hash, ..]) == count`
|
||||
@@ -388,7 +388,7 @@ where
|
||||
///
|
||||
/// These are returned in falling order starting with the requested `hash`, i.e. with
|
||||
/// descending block numbers.
|
||||
fn take_blocks(&mut self) -> Option<Vec<SealedBlock<Client::Header, Client::Body>>> {
|
||||
fn take_blocks(&mut self) -> Option<Vec<SealedBlock<Client::Block>>> {
|
||||
if !self.is_bodies_complete() {
|
||||
// not done with bodies yet
|
||||
return None
|
||||
@@ -421,7 +421,8 @@ where
|
||||
}
|
||||
};
|
||||
|
||||
valid_responses.push(SealedBlock::new(header.clone(), body));
|
||||
valid_responses
|
||||
.push(SealedBlock::<Client::Block>::from_sealed_parts(header.clone(), body));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -429,7 +430,7 @@ where
|
||||
// put response hashes back into bodies map since we aren't returning them as a
|
||||
// response
|
||||
for block in valid_responses {
|
||||
let (header, body) = block.split();
|
||||
let (header, body) = block.split_sealed_header_body();
|
||||
self.bodies.insert(header, BodyResponse::Validated(body));
|
||||
}
|
||||
|
||||
@@ -447,7 +448,7 @@ where
|
||||
|
||||
fn on_headers_response(&mut self, headers: WithPeerId<Vec<Client::Header>>) {
|
||||
let (peer, mut headers_falling) =
|
||||
headers.map(|h| h.into_iter().map(SealedHeader::seal).collect::<Vec<_>>()).split();
|
||||
headers.map(|h| h.into_iter().map(SealedHeader::seal_slow).collect::<Vec<_>>()).split();
|
||||
|
||||
// fill in the response if it's the correct length
|
||||
if headers_falling.len() == self.count as usize {
|
||||
@@ -505,7 +506,7 @@ impl<Client> Future for FetchFullBlockRangeFuture<Client>
|
||||
where
|
||||
Client: BlockClient<Header: Debug + BlockHeader + Sealable + Clone + Hash + Eq> + 'static,
|
||||
{
|
||||
type Output = Vec<SealedBlock<Client::Header, Client::Body>>;
|
||||
type Output = Vec<SealedBlock<Client::Block>>;
|
||||
|
||||
fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
|
||||
let this = self.get_mut();
|
||||
@@ -658,7 +659,7 @@ mod tests {
|
||||
let client = FullBlockClient::test_client(client);
|
||||
|
||||
let received = client.get_full_block(header.hash()).await;
|
||||
assert_eq!(received, SealedBlock::new(header, body));
|
||||
assert_eq!(received, SealedBlock::from_sealed_parts(header, body));
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -671,7 +672,7 @@ mod tests {
|
||||
|
||||
let received = client.get_full_block_range(header.hash(), 1).await;
|
||||
let received = received.first().expect("response should include a block");
|
||||
assert_eq!(*received, SealedBlock::new(header, body));
|
||||
assert_eq!(*received, SealedBlock::from_sealed_parts(header, body));
|
||||
}
|
||||
|
||||
/// Inserts headers and returns the last header and block body.
|
||||
@@ -687,7 +688,7 @@ mod tests {
|
||||
header.parent_hash = hash;
|
||||
header.number += 1;
|
||||
|
||||
sealed_header = SealedHeader::seal(header);
|
||||
sealed_header = SealedHeader::seal_slow(header);
|
||||
|
||||
client.insert(sealed_header.clone(), body.clone());
|
||||
}
|
||||
@@ -703,7 +704,7 @@ mod tests {
|
||||
|
||||
let received = client.get_full_block_range(header.hash(), 1).await;
|
||||
let received = received.first().expect("response should include a block");
|
||||
assert_eq!(*received, SealedBlock::new(header.clone(), body));
|
||||
assert_eq!(*received, SealedBlock::from_sealed_parts(header.clone(), body));
|
||||
|
||||
let received = client.get_full_block_range(header.hash(), 10).await;
|
||||
assert_eq!(received.len(), 10);
|
||||
@@ -722,7 +723,7 @@ mod tests {
|
||||
|
||||
let received = client.get_full_block_range(header.hash(), 1).await;
|
||||
let received = received.first().expect("response should include a block");
|
||||
assert_eq!(*received, SealedBlock::new(header.clone(), body));
|
||||
assert_eq!(*received, SealedBlock::from_sealed_parts(header.clone(), body));
|
||||
|
||||
let received = client.get_full_block_range(header.hash(), 50).await;
|
||||
assert_eq!(received.len(), 50);
|
||||
|
||||
@@ -1,11 +1,10 @@
|
||||
use super::error::HeadersDownloaderResult;
|
||||
use crate::error::{DownloadError, DownloadResult};
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_eips::{eip1898::BlockWithParent, BlockHashOrNumber};
|
||||
use alloy_primitives::B256;
|
||||
use alloy_primitives::{Sealable, B256};
|
||||
use futures::Stream;
|
||||
use reth_consensus::HeaderValidator;
|
||||
use reth_primitives::SealedHeader;
|
||||
use reth_primitives_traits::{BlockHeader, SealedHeader};
|
||||
use std::fmt::Debug;
|
||||
|
||||
/// A downloader capable of fetching and yielding block headers.
|
||||
@@ -22,7 +21,7 @@ pub trait HeaderDownloader:
|
||||
+ Unpin
|
||||
{
|
||||
/// The header type being downloaded.
|
||||
type Header: Debug + Send + Sync + Unpin + 'static;
|
||||
type Header: Sealable + Debug + Send + Sync + Unpin + 'static;
|
||||
|
||||
/// Updates the gap to sync which ranges from local head to the sync target
|
||||
///
|
||||
|
||||
@@ -1,3 +1,4 @@
|
||||
use alloy_primitives::Sealable;
|
||||
use derive_more::{Display, Error};
|
||||
use reth_consensus::ConsensusError;
|
||||
use reth_primitives::SealedHeader;
|
||||
@@ -7,7 +8,7 @@ pub type HeadersDownloaderResult<T, H> = Result<T, HeadersDownloaderError<H>>;
|
||||
|
||||
/// Error variants that can happen when sending requests to a session.
|
||||
#[derive(Debug, Clone, Eq, PartialEq, Display, Error)]
|
||||
pub enum HeadersDownloaderError<H> {
|
||||
pub enum HeadersDownloaderError<H: Sealable> {
|
||||
/// The downloaded header cannot be attached to the local head,
|
||||
/// but is valid otherwise.
|
||||
#[display("valid downloaded header cannot be attached to the local head: {error}")]
|
||||
|
||||
@@ -47,19 +47,20 @@ pub mod test_utils;
|
||||
|
||||
pub use bodies::client::BodiesClient;
|
||||
pub use headers::client::HeadersClient;
|
||||
use reth_primitives_traits::Block;
|
||||
|
||||
/// Helper trait that unifies network behaviour needed for fetching blocks.
|
||||
pub trait BlockClient: HeadersClient + BodiesClient + Unpin + Clone {}
|
||||
|
||||
impl<T> BlockClient for T where T: HeadersClient + BodiesClient + Unpin + Clone {}
|
||||
/// Helper trait that unifies network behaviour needed for fetching entire blocks.
|
||||
pub trait BlockClient:
|
||||
HeadersClient<Header = <Self::Block as Block>::Header>
|
||||
+ BodiesClient<Body = <Self::Block as Block>::Body>
|
||||
+ Unpin
|
||||
+ Clone
|
||||
{
|
||||
/// The Block type that this client fetches.
|
||||
type Block: Block;
|
||||
}
|
||||
|
||||
/// The [`BlockClient`] providing Ethereum block parts.
|
||||
pub trait EthBlockClient:
|
||||
BlockClient<Header = alloy_consensus::Header, Body = reth_primitives::BlockBody>
|
||||
{
|
||||
}
|
||||
pub trait EthBlockClient: BlockClient<Block = reth_primitives::Block> {}
|
||||
|
||||
impl<T> EthBlockClient for T where
|
||||
T: BlockClient<Header = alloy_consensus::Header, Body = reth_primitives::BlockBody>
|
||||
{
|
||||
}
|
||||
impl<T> EthBlockClient for T where T: BlockClient<Block = reth_primitives::Block> {}
|
||||
|
||||
@@ -4,6 +4,7 @@ use crate::{
|
||||
error::PeerRequestResult,
|
||||
headers::client::{HeadersClient, HeadersRequest},
|
||||
priority::Priority,
|
||||
BlockClient,
|
||||
};
|
||||
use alloy_consensus::Header;
|
||||
use alloy_eips::{BlockHashOrNumber, BlockNumHash};
|
||||
@@ -134,7 +135,7 @@ impl TestFullBlockClient {
|
||||
self.headers.lock().iter().max_by_key(|(_, header)| header.number).and_then(
|
||||
|(hash, header)| {
|
||||
self.bodies.lock().get(hash).map(|body| {
|
||||
SealedBlock::new(SealedHeader::new(header.clone(), *hash), body.clone())
|
||||
SealedBlock::from_parts_unchecked(header.clone(), body.clone(), *hash)
|
||||
})
|
||||
},
|
||||
)
|
||||
@@ -243,3 +244,7 @@ impl BodiesClient for TestFullBlockClient {
|
||||
)))
|
||||
}
|
||||
}
|
||||
|
||||
impl BlockClient for TestFullBlockClient {
|
||||
type Block = reth_primitives::Block;
|
||||
}
|
||||
|
||||
@@ -12,7 +12,7 @@ use crate::{
|
||||
};
|
||||
use alloy_consensus::Header;
|
||||
use futures::{Future, FutureExt, Stream, StreamExt};
|
||||
use reth_consensus::{test_utils::TestConsensus, Consensus, ConsensusError};
|
||||
use reth_consensus::{test_utils::TestConsensus, HeaderValidator};
|
||||
use reth_eth_wire_types::HeadersDirection;
|
||||
use reth_network_peers::{PeerId, WithPeerId};
|
||||
use reth_primitives::SealedHeader;
|
||||
@@ -146,13 +146,7 @@ impl Stream for TestDownload {
|
||||
}
|
||||
|
||||
let empty: SealedHeader = SealedHeader::default();
|
||||
if let Err(error) =
|
||||
<dyn Consensus<_, Error = ConsensusError>>::validate_header_against_parent(
|
||||
&this.consensus,
|
||||
&empty,
|
||||
&empty,
|
||||
)
|
||||
{
|
||||
if let Err(error) = this.consensus.validate_header_against_parent(&empty, &empty) {
|
||||
this.done = true;
|
||||
return Poll::Ready(Some(Err(DownloadError::HeaderValidation {
|
||||
hash: empty.hash(),
|
||||
@@ -165,7 +159,7 @@ impl Stream for TestDownload {
|
||||
Ok(resp) => {
|
||||
// Skip head and seal headers
|
||||
let mut headers =
|
||||
resp.1.into_iter().skip(1).map(SealedHeader::seal).collect::<Vec<_>>();
|
||||
resp.1.into_iter().skip(1).map(SealedHeader::seal_slow).collect::<Vec<_>>();
|
||||
headers.sort_unstable_by_key(|h| h.number);
|
||||
headers.into_iter().for_each(|h| this.buffer.push(h));
|
||||
this.done = true;
|
||||
|
||||
@@ -10,7 +10,7 @@ use crate::{
|
||||
use reth_consensus::{ConsensusError, FullConsensus};
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_network::NetworkPrimitives;
|
||||
use reth_node_api::{BodyTy, HeaderTy, NodeTypes, NodeTypesWithEngine, TxTy};
|
||||
use reth_node_api::{BlockTy, BodyTy, HeaderTy, NodeTypes, NodeTypesWithEngine, TxTy};
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_transaction_pool::{PoolTransaction, TransactionPool};
|
||||
use std::{future::Future, marker::PhantomData};
|
||||
@@ -310,6 +310,7 @@ where
|
||||
Primitives: NetworkPrimitives<
|
||||
BlockHeader = HeaderTy<Node::Types>,
|
||||
BlockBody = BodyTy<Node::Types>,
|
||||
Block = BlockTy<Node::Types>,
|
||||
>,
|
||||
>,
|
||||
PayloadB: PayloadServiceBuilder<Node, PoolB::Pool>,
|
||||
@@ -393,7 +394,11 @@ pub trait NodeComponentsBuilder<Node: FullNodeTypes>: Send {
|
||||
|
||||
impl<Node, N, F, Fut, Pool, EVM, Executor, Cons> NodeComponentsBuilder<Node> for F
|
||||
where
|
||||
N: NetworkPrimitives<BlockHeader = HeaderTy<Node::Types>, BlockBody = BodyTy<Node::Types>>,
|
||||
N: NetworkPrimitives<
|
||||
BlockHeader = HeaderTy<Node::Types>,
|
||||
BlockBody = BodyTy<Node::Types>,
|
||||
Block = BlockTy<Node::Types>,
|
||||
>,
|
||||
Node: FullNodeTypes,
|
||||
F: FnOnce(&BuilderContext<Node>) -> Fut + Send,
|
||||
Fut: Future<Output = eyre::Result<Components<Node, N, Pool, EVM, Executor, Cons>>> + Send,
|
||||
|
||||
@@ -27,7 +27,9 @@ use reth_consensus::{ConsensusError, FullConsensus};
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_network::{NetworkHandle, NetworkPrimitives};
|
||||
use reth_network_api::FullNetwork;
|
||||
use reth_node_api::{BodyTy, HeaderTy, NodeTypes, NodeTypesWithEngine, PayloadBuilder, TxTy};
|
||||
use reth_node_api::{
|
||||
BlockTy, BodyTy, HeaderTy, NodeTypes, NodeTypesWithEngine, PayloadBuilder, TxTy,
|
||||
};
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_transaction_pool::{PoolTransaction, TransactionPool};
|
||||
|
||||
@@ -53,9 +55,7 @@ pub trait NodeComponents<T: FullNodeTypes>: Clone + Unpin + Send + Sync + 'stati
|
||||
+ 'static;
|
||||
|
||||
/// Network API.
|
||||
type Network: FullNetwork<
|
||||
Client: BlockClient<Header = HeaderTy<T::Types>, Body = BodyTy<T::Types>>,
|
||||
>;
|
||||
type Network: FullNetwork<Client: BlockClient<Block = BlockTy<T::Types>>>;
|
||||
|
||||
/// Builds new blocks.
|
||||
type PayloadBuilder: PayloadBuilder<PayloadType = <T::Types as NodeTypesWithEngine>::Engine>
|
||||
@@ -102,7 +102,11 @@ pub struct Components<Node: FullNodeTypes, N: NetworkPrimitives, Pool, EVM, Exec
|
||||
impl<Node, Pool, EVM, Executor, Cons, N> NodeComponents<Node>
|
||||
for Components<Node, N, Pool, EVM, Executor, Cons>
|
||||
where
|
||||
N: NetworkPrimitives<BlockHeader = HeaderTy<Node::Types>, BlockBody = BodyTy<Node::Types>>,
|
||||
N: NetworkPrimitives<
|
||||
BlockHeader = HeaderTy<Node::Types>,
|
||||
BlockBody = BodyTy<Node::Types>,
|
||||
Block = BlockTy<Node::Types>,
|
||||
>,
|
||||
Node: FullNodeTypes,
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
|
||||
+ Unpin
|
||||
|
||||
@@ -2,6 +2,7 @@
|
||||
|
||||
use std::sync::Arc;
|
||||
|
||||
use crate::BlockTy;
|
||||
use alloy_primitives::{BlockNumber, B256};
|
||||
use reth_config::{config::StageConfig, PruneConfig};
|
||||
use reth_consensus::{Consensus, ConsensusError};
|
||||
@@ -14,7 +15,7 @@ use reth_exex::ExExManagerHandle;
|
||||
use reth_network_p2p::{
|
||||
bodies::downloader::BodyDownloader, headers::downloader::HeaderDownloader, BlockClient,
|
||||
};
|
||||
use reth_node_api::{BodyTy, HeaderTy};
|
||||
use reth_node_api::HeaderTy;
|
||||
use reth_provider::{providers::ProviderNodeTypes, ProviderFactory};
|
||||
use reth_stages::{prelude::DefaultStages, stages::ExecutionStage, Pipeline, StageSet};
|
||||
use reth_static_file::StaticFileProducer;
|
||||
@@ -27,7 +28,7 @@ use tokio::sync::watch;
|
||||
pub fn build_networked_pipeline<N, Client, Executor>(
|
||||
config: &StageConfig,
|
||||
client: Client,
|
||||
consensus: Arc<dyn Consensus<Client::Header, Client::Body, Error = ConsensusError>>,
|
||||
consensus: Arc<dyn Consensus<BlockTy<N>, Error = ConsensusError>>,
|
||||
provider_factory: ProviderFactory<N>,
|
||||
task_executor: &TaskExecutor,
|
||||
metrics_tx: reth_stages::MetricEventsSender,
|
||||
@@ -39,7 +40,7 @@ pub fn build_networked_pipeline<N, Client, Executor>(
|
||||
) -> eyre::Result<Pipeline<N>>
|
||||
where
|
||||
N: ProviderNodeTypes,
|
||||
Client: BlockClient<Header = HeaderTy<N>, Body = BodyTy<N>> + 'static,
|
||||
Client: BlockClient<Block = BlockTy<N>> + 'static,
|
||||
Executor: BlockExecutorProvider<Primitives = N::Primitives>,
|
||||
{
|
||||
// building network downloaders using the fetch client
|
||||
@@ -75,7 +76,7 @@ pub fn build_pipeline<N, H, B, Executor>(
|
||||
stage_config: &StageConfig,
|
||||
header_downloader: H,
|
||||
body_downloader: B,
|
||||
consensus: Arc<dyn Consensus<H::Header, B::Body, Error = ConsensusError>>,
|
||||
consensus: Arc<dyn Consensus<BlockTy<N>, Error = ConsensusError>>,
|
||||
max_block: Option<u64>,
|
||||
metrics_tx: reth_stages::MetricEventsSender,
|
||||
prune_config: Option<PruneConfig>,
|
||||
@@ -86,7 +87,7 @@ pub fn build_pipeline<N, H, B, Executor>(
|
||||
where
|
||||
N: ProviderNodeTypes,
|
||||
H: HeaderDownloader<Header = HeaderTy<N>> + 'static,
|
||||
B: BodyDownloader<Header = HeaderTy<N>, Body = BodyTy<N>> + 'static,
|
||||
B: BodyDownloader<Block = BlockTy<N>> + 'static,
|
||||
Executor: BlockExecutorProvider<Primitives = N::Primitives>,
|
||||
{
|
||||
let mut builder = Pipeline::<N>::builder();
|
||||
|
||||
@@ -10,7 +10,7 @@ use reth_network_p2p::{
|
||||
bodies::client::BodiesClient, headers::client::HeadersClient, priority::Priority,
|
||||
};
|
||||
use reth_primitives::SealedBlock;
|
||||
use reth_primitives_traits::SealedHeader;
|
||||
use reth_primitives_traits::{Block, SealedHeader};
|
||||
use std::{
|
||||
env::VarError,
|
||||
path::{Path, PathBuf},
|
||||
@@ -49,7 +49,7 @@ where
|
||||
eyre::bail!("Invalid number of headers received. Expected: 1. Received: 0")
|
||||
};
|
||||
|
||||
let header = SealedHeader::seal(header);
|
||||
let header = SealedHeader::seal_slow(header);
|
||||
|
||||
let valid = match id {
|
||||
BlockHashOrNumber::Hash(hash) => header.hash() == hash,
|
||||
@@ -69,13 +69,14 @@ where
|
||||
}
|
||||
|
||||
/// Get a body from network based on header
|
||||
pub async fn get_single_body<H, Client>(
|
||||
pub async fn get_single_body<B, Client>(
|
||||
client: Client,
|
||||
header: SealedHeader<H>,
|
||||
consensus: impl Consensus<H, Client::Body, Error = ConsensusError>,
|
||||
) -> Result<SealedBlock<H, Client::Body>>
|
||||
header: SealedHeader<B::Header>,
|
||||
consensus: impl Consensus<B, Error = ConsensusError>,
|
||||
) -> Result<SealedBlock<B>>
|
||||
where
|
||||
Client: BodiesClient,
|
||||
B: Block,
|
||||
Client: BodiesClient<Body = B::Body>,
|
||||
{
|
||||
let (peer_id, response) = client.get_block_body(header.hash()).await?.split();
|
||||
|
||||
@@ -84,7 +85,7 @@ where
|
||||
eyre::bail!("Invalid number of bodies received. Expected: 1. Received: 0")
|
||||
};
|
||||
|
||||
let block = SealedBlock::new(header, body);
|
||||
let block = SealedBlock::from_sealed_parts(header, body);
|
||||
consensus.validate_block_pre_execution(&block)?;
|
||||
|
||||
Ok(block)
|
||||
|
||||
@@ -19,7 +19,7 @@ mod op_sepolia;
|
||||
|
||||
use alloc::{boxed::Box, vec, vec::Vec};
|
||||
use alloy_chains::Chain;
|
||||
use alloy_consensus::Header;
|
||||
use alloy_consensus::{BlockHeader, Header};
|
||||
use alloy_genesis::Genesis;
|
||||
use alloy_primitives::{B256, U256};
|
||||
pub use base::BASE_MAINNET;
|
||||
@@ -200,12 +200,12 @@ impl OpChainSpec {
|
||||
/// Caution: Caller must ensure that holocene is active in the parent header.
|
||||
///
|
||||
/// See also [Base fee computation](https://github.com/ethereum-optimism/specs/blob/main/specs/protocol/holocene/exec-engine.md#base-fee-computation)
|
||||
pub fn decode_holocene_base_fee(
|
||||
pub fn decode_holocene_base_fee<H: BlockHeader>(
|
||||
&self,
|
||||
parent: &Header,
|
||||
parent: &H,
|
||||
timestamp: u64,
|
||||
) -> Result<u64, EIP1559ParamError> {
|
||||
let (elasticity, denominator) = decode_holocene_extra_data(&parent.extra_data)?;
|
||||
let (elasticity, denominator) = decode_holocene_extra_data(parent.extra_data())?;
|
||||
let base_fee = if elasticity == 0 && denominator == 0 {
|
||||
parent
|
||||
.next_block_base_fee(self.base_fee_params_at_timestamp(timestamp))
|
||||
@@ -220,15 +220,15 @@ impl OpChainSpec {
|
||||
/// Read from parent to determine the base fee for the next block
|
||||
///
|
||||
/// See also [Base fee computation](https://github.com/ethereum-optimism/specs/blob/main/specs/protocol/holocene/exec-engine.md#base-fee-computation)
|
||||
pub fn next_block_base_fee(
|
||||
pub fn next_block_base_fee<H: BlockHeader>(
|
||||
&self,
|
||||
parent: &Header,
|
||||
parent: &H,
|
||||
timestamp: u64,
|
||||
) -> Result<U256, EIP1559ParamError> {
|
||||
// > if Holocene is active in parent_header.timestamp, then the parameters from
|
||||
// > parent_header.extraData are used.
|
||||
let is_holocene_activated =
|
||||
self.inner.is_fork_active_at_timestamp(OpHardfork::Holocene, parent.timestamp);
|
||||
self.inner.is_fork_active_at_timestamp(OpHardfork::Holocene, parent.timestamp());
|
||||
|
||||
// If we are in the Holocene, we need to use the base fee params
|
||||
// from the parent block's extra data.
|
||||
|
||||
@@ -41,18 +41,18 @@ reth-optimism-chainspec.workspace = true
|
||||
[features]
|
||||
default = ["std"]
|
||||
std = [
|
||||
"reth-chainspec/std",
|
||||
"reth-consensus/std",
|
||||
"reth-consensus-common/std",
|
||||
"reth-primitives/std",
|
||||
"reth-optimism-forks/std",
|
||||
"reth-optimism-chainspec/std",
|
||||
"reth-optimism-primitives/std",
|
||||
"alloy-eips/std",
|
||||
"alloy-primitives/std",
|
||||
"alloy-consensus/std",
|
||||
"alloy-trie/std",
|
||||
"op-alloy-consensus/std",
|
||||
"reth-primitives-traits/std"
|
||||
"reth-chainspec/std",
|
||||
"reth-consensus/std",
|
||||
"reth-consensus-common/std",
|
||||
"reth-primitives/std",
|
||||
"reth-primitives-traits/std",
|
||||
"reth-optimism-forks/std",
|
||||
"reth-optimism-chainspec/std",
|
||||
"reth-optimism-primitives/std",
|
||||
"alloy-eips/std",
|
||||
"alloy-primitives/std",
|
||||
"alloy-consensus/std",
|
||||
"alloy-trie/std",
|
||||
"op-alloy-consensus/std",
|
||||
]
|
||||
optimism = ["reth-primitives/optimism", "reth-optimism-primitives/optimism"]
|
||||
|
||||
@@ -13,7 +13,7 @@
|
||||
extern crate alloc;
|
||||
|
||||
use alloc::sync::Arc;
|
||||
use alloy_consensus::{BlockHeader, Header, EMPTY_OMMER_ROOT_HASH};
|
||||
use alloy_consensus::{BlockHeader as _, EMPTY_OMMER_ROOT_HASH};
|
||||
use alloy_eips::eip7840::BlobParams;
|
||||
use alloy_primitives::{B64, U256};
|
||||
use reth_chainspec::EthereumHardforks;
|
||||
@@ -28,11 +28,12 @@ use reth_consensus_common::validation::{
|
||||
};
|
||||
use reth_optimism_chainspec::OpChainSpec;
|
||||
use reth_optimism_forks::OpHardforks;
|
||||
use reth_optimism_primitives::{OpBlock, OpBlockBody, OpPrimitives, OpReceipt};
|
||||
use reth_primitives::{BlockWithSenders, GotExpected, SealedBlockFor, SealedHeader};
|
||||
use reth_optimism_primitives::{OpBlock, OpPrimitives, OpReceipt};
|
||||
use reth_primitives::{GotExpected, RecoveredBlock, SealedHeader};
|
||||
|
||||
mod proof;
|
||||
pub use proof::calculate_receipt_root_no_memo_optimism;
|
||||
use reth_primitives_traits::{Block, BlockBody, BlockHeader, SealedBlock};
|
||||
|
||||
mod validation;
|
||||
pub use validation::validate_block_post_execution;
|
||||
@@ -56,33 +57,34 @@ impl OpBeaconConsensus {
|
||||
impl FullConsensus<OpPrimitives> for OpBeaconConsensus {
|
||||
fn validate_block_post_execution(
|
||||
&self,
|
||||
block: &BlockWithSenders<OpBlock>,
|
||||
block: &RecoveredBlock<OpBlock>,
|
||||
input: PostExecutionInput<'_, OpReceipt>,
|
||||
) -> Result<(), ConsensusError> {
|
||||
validate_block_post_execution(&block.header, &self.chain_spec, input.receipts)
|
||||
validate_block_post_execution(block.header(), &self.chain_spec, input.receipts)
|
||||
}
|
||||
}
|
||||
|
||||
impl Consensus<Header, OpBlockBody> for OpBeaconConsensus {
|
||||
impl<B: Block> Consensus<B> for OpBeaconConsensus {
|
||||
type Error = ConsensusError;
|
||||
|
||||
fn validate_body_against_header(
|
||||
&self,
|
||||
body: &OpBlockBody,
|
||||
header: &SealedHeader,
|
||||
body: &B::Body,
|
||||
header: &SealedHeader<B::Header>,
|
||||
) -> Result<(), ConsensusError> {
|
||||
validate_body_against_header(body, header.header())
|
||||
}
|
||||
|
||||
fn validate_block_pre_execution(
|
||||
&self,
|
||||
block: &SealedBlockFor<OpBlock>,
|
||||
) -> Result<(), ConsensusError> {
|
||||
fn validate_block_pre_execution(&self, block: &SealedBlock<B>) -> Result<(), ConsensusError> {
|
||||
// Check ommers hash
|
||||
let ommers_hash = block.body().calculate_ommers_root();
|
||||
if block.ommers_hash != ommers_hash {
|
||||
if Some(block.ommers_hash()) != ommers_hash {
|
||||
return Err(ConsensusError::BodyOmmersHashDiff(
|
||||
GotExpected { got: ommers_hash, expected: block.ommers_hash }.into(),
|
||||
GotExpected {
|
||||
got: ommers_hash.unwrap_or(EMPTY_OMMER_ROOT_HASH),
|
||||
expected: block.ommers_hash(),
|
||||
}
|
||||
.into(),
|
||||
))
|
||||
}
|
||||
|
||||
@@ -92,11 +94,11 @@ impl Consensus<Header, OpBlockBody> for OpBeaconConsensus {
|
||||
}
|
||||
|
||||
// EIP-4895: Beacon chain push withdrawals as operations
|
||||
if self.chain_spec.is_shanghai_active_at_timestamp(block.timestamp) {
|
||||
if self.chain_spec.is_shanghai_active_at_timestamp(block.timestamp()) {
|
||||
validate_shanghai_withdrawals(block)?;
|
||||
}
|
||||
|
||||
if self.chain_spec.is_cancun_active_at_timestamp(block.timestamp) {
|
||||
if self.chain_spec.is_cancun_active_at_timestamp(block.timestamp()) {
|
||||
validate_cancun_gas(block)?;
|
||||
}
|
||||
|
||||
@@ -104,20 +106,20 @@ impl Consensus<Header, OpBlockBody> for OpBeaconConsensus {
|
||||
}
|
||||
}
|
||||
|
||||
impl HeaderValidator for OpBeaconConsensus {
|
||||
fn validate_header(&self, header: &SealedHeader) -> Result<(), ConsensusError> {
|
||||
impl<H: BlockHeader> HeaderValidator<H> for OpBeaconConsensus {
|
||||
fn validate_header(&self, header: &SealedHeader<H>) -> Result<(), ConsensusError> {
|
||||
validate_header_gas(header.header())?;
|
||||
validate_header_base_fee(header.header(), &self.chain_spec)
|
||||
}
|
||||
|
||||
fn validate_header_against_parent(
|
||||
&self,
|
||||
header: &SealedHeader,
|
||||
parent: &SealedHeader,
|
||||
header: &SealedHeader<H>,
|
||||
parent: &SealedHeader<H>,
|
||||
) -> Result<(), ConsensusError> {
|
||||
validate_against_parent_hash_number(header.header(), parent)?;
|
||||
|
||||
if self.chain_spec.is_bedrock_active_at_block(header.number) {
|
||||
if self.chain_spec.is_bedrock_active_at_block(header.number()) {
|
||||
validate_against_parent_timestamp(header.header(), parent.header())?;
|
||||
}
|
||||
|
||||
@@ -125,12 +127,12 @@ impl HeaderValidator for OpBeaconConsensus {
|
||||
// <https://github.com/ethereum-optimism/specs/blob/main/specs/protocol/holocene/exec-engine.md#base-fee-computation>
|
||||
// > if Holocene is active in parent_header.timestamp, then the parameters from
|
||||
// > parent_header.extraData are used.
|
||||
if self.chain_spec.is_holocene_active_at_timestamp(parent.timestamp) {
|
||||
if self.chain_spec.is_holocene_active_at_timestamp(parent.timestamp()) {
|
||||
let header_base_fee =
|
||||
header.base_fee_per_gas().ok_or(ConsensusError::BaseFeeMissing)?;
|
||||
let expected_base_fee = self
|
||||
.chain_spec
|
||||
.decode_holocene_base_fee(parent, header.timestamp)
|
||||
.decode_holocene_base_fee(parent.header(), header.timestamp())
|
||||
.map_err(|_| ConsensusError::BaseFeeMissing)?;
|
||||
if expected_base_fee != header_base_fee {
|
||||
return Err(ConsensusError::BaseFeeDiff(GotExpected {
|
||||
@@ -147,7 +149,7 @@ impl HeaderValidator for OpBeaconConsensus {
|
||||
}
|
||||
|
||||
// ensure that the blob gas fields for this block
|
||||
if self.chain_spec.is_cancun_active_at_timestamp(header.timestamp) {
|
||||
if self.chain_spec.is_cancun_active_at_timestamp(header.timestamp()) {
|
||||
validate_against_parent_4844(header.header(), parent.header(), BlobParams::cancun())?;
|
||||
}
|
||||
|
||||
@@ -156,20 +158,20 @@ impl HeaderValidator for OpBeaconConsensus {
|
||||
|
||||
fn validate_header_with_total_difficulty(
|
||||
&self,
|
||||
header: &Header,
|
||||
header: &H,
|
||||
_total_difficulty: U256,
|
||||
) -> Result<(), ConsensusError> {
|
||||
// with OP-stack Bedrock activation number determines when TTD (eth Merge) has been reached.
|
||||
debug_assert!(
|
||||
self.chain_spec.is_bedrock_active_at_block(header.number),
|
||||
self.chain_spec.is_bedrock_active_at_block(header.number()),
|
||||
"manually import OVM blocks"
|
||||
);
|
||||
|
||||
if header.nonce != B64::ZERO {
|
||||
if header.nonce() != Some(B64::ZERO) {
|
||||
return Err(ConsensusError::TheMergeNonceIsNotZero)
|
||||
}
|
||||
|
||||
if header.ommers_hash != EMPTY_OMMER_ROOT_HASH {
|
||||
if header.ommers_hash() != EMPTY_OMMER_ROOT_HASH {
|
||||
return Err(ConsensusError::TheMergeOmmerRootIsNotEmpty)
|
||||
}
|
||||
|
||||
|
||||
@@ -5,7 +5,7 @@ use crate::{
|
||||
OpReceiptBuilder, ReceiptBuilderCtx,
|
||||
};
|
||||
use alloc::{boxed::Box, sync::Arc, vec::Vec};
|
||||
use alloy_consensus::{Eip658Value, Receipt, Transaction as _};
|
||||
use alloy_consensus::{BlockHeader, Eip658Value, Receipt, Transaction as _};
|
||||
use alloy_eips::eip7685::Requests;
|
||||
use core::fmt::Display;
|
||||
use op_alloy_consensus::{DepositTransaction, OpDepositReceipt};
|
||||
@@ -25,8 +25,8 @@ use reth_optimism_chainspec::OpChainSpec;
|
||||
use reth_optimism_consensus::validate_block_post_execution;
|
||||
use reth_optimism_forks::OpHardfork;
|
||||
use reth_optimism_primitives::{DepositReceipt, OpPrimitives, OpReceipt};
|
||||
use reth_primitives::{BlockWithSenders, NodePrimitives};
|
||||
use reth_primitives_traits::{Block, BlockBody, SignedTransaction};
|
||||
use reth_primitives::{NodePrimitives, RecoveredBlock};
|
||||
use reth_primitives_traits::{BlockBody, SignedTransaction};
|
||||
use reth_revm::{Database, State};
|
||||
use revm_primitives::{db::DatabaseCommit, ResultAndState};
|
||||
use tracing::trace;
|
||||
@@ -161,11 +161,11 @@ where
|
||||
|
||||
fn apply_pre_execution_changes(
|
||||
&mut self,
|
||||
block: &BlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
) -> Result<(), Self::Error> {
|
||||
// Set state clear flag if the block is after the Spurious Dragon hardfork.
|
||||
let state_clear_flag =
|
||||
(*self.chain_spec).is_spurious_dragon_active_at_block(block.header().number);
|
||||
(*self.chain_spec).is_spurious_dragon_active_at_block(block.number());
|
||||
self.state.set_state_clear_flag(state_clear_flag);
|
||||
|
||||
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
|
||||
@@ -189,21 +189,19 @@ where
|
||||
|
||||
fn execute_transactions(
|
||||
&mut self,
|
||||
block: &BlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
) -> Result<ExecuteOutput<N::Receipt>, Self::Error> {
|
||||
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
|
||||
|
||||
let is_regolith = self
|
||||
.chain_spec
|
||||
.fork(OpHardfork::Regolith)
|
||||
.active_at_timestamp(block.header().timestamp);
|
||||
let is_regolith =
|
||||
self.chain_spec.fork(OpHardfork::Regolith).active_at_timestamp(block.timestamp());
|
||||
|
||||
let mut cumulative_gas_used = 0;
|
||||
let mut receipts = Vec::with_capacity(block.body().transactions().len());
|
||||
let mut receipts = Vec::with_capacity(block.body().transaction_count());
|
||||
for (sender, transaction) in block.transactions_with_sender() {
|
||||
// The sum of the transaction’s gas limit, Tg, and the gas utilized in this block prior,
|
||||
// must be no greater than the block’s gasLimit.
|
||||
let block_available_gas = block.header().gas_limit - cumulative_gas_used;
|
||||
let block_available_gas = block.gas_limit() - cumulative_gas_used;
|
||||
if transaction.gas_limit() > block_available_gas &&
|
||||
(is_regolith || !transaction.is_deposit())
|
||||
{
|
||||
@@ -298,11 +296,10 @@ where
|
||||
|
||||
fn apply_post_execution_changes(
|
||||
&mut self,
|
||||
block: &BlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
_receipts: &[N::Receipt],
|
||||
) -> Result<Requests, Self::Error> {
|
||||
let balance_increments =
|
||||
post_block_balance_increments(&self.chain_spec.clone(), &block.block);
|
||||
let balance_increments = post_block_balance_increments(&self.chain_spec.clone(), block);
|
||||
// increment balances
|
||||
self.state
|
||||
.increment_balances(balance_increments.clone())
|
||||
@@ -328,7 +325,7 @@ where
|
||||
|
||||
fn validate_block_post_execution(
|
||||
&self,
|
||||
block: &BlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
receipts: &[N::Receipt],
|
||||
_requests: &Requests,
|
||||
) -> Result<(), ConsensusError> {
|
||||
@@ -455,7 +452,7 @@ mod tests {
|
||||
|
||||
// Attempt to execute a block with one deposit and one non-deposit transaction
|
||||
executor
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block {
|
||||
header,
|
||||
body: BlockBody { transactions: vec![tx, tx_deposit], ..Default::default() },
|
||||
@@ -531,7 +528,7 @@ mod tests {
|
||||
|
||||
// attempt to execute an empty block with parent beacon block root, this should not fail
|
||||
executor
|
||||
.execute_and_verify_one(&BlockWithSenders::new_unchecked(
|
||||
.execute_and_verify_one(&RecoveredBlock::new_unhashed(
|
||||
Block {
|
||||
header,
|
||||
body: BlockBody { transactions: vec![tx, tx_deposit], ..Default::default() },
|
||||
|
||||
@@ -204,7 +204,7 @@ mod tests {
|
||||
};
|
||||
use reth_optimism_chainspec::BASE_MAINNET;
|
||||
use reth_optimism_primitives::{OpBlock, OpPrimitives, OpReceipt};
|
||||
use reth_primitives::{Account, Log, Receipts, SealedBlockWithSenders};
|
||||
use reth_primitives::{Account, Log, Receipts, RecoveredBlock};
|
||||
use reth_revm::{
|
||||
db::{BundleState, CacheDB, EmptyDBTyped},
|
||||
inspectors::NoOpInspector,
|
||||
@@ -530,8 +530,8 @@ mod tests {
|
||||
|
||||
#[test]
|
||||
fn receipts_by_block_hash() {
|
||||
// Create a default SealedBlockWithSenders object
|
||||
let block: SealedBlockWithSenders<OpBlock> = Default::default();
|
||||
// Create a default recovered block
|
||||
let block: RecoveredBlock<OpBlock> = Default::default();
|
||||
|
||||
// Define block hashes for block1 and block2
|
||||
let block1_hash = B256::new([0x01; 32]);
|
||||
@@ -542,11 +542,11 @@ mod tests {
|
||||
let mut block2 = block;
|
||||
|
||||
// Set the hashes of block1 and block2
|
||||
block1.block.set_block_number(10);
|
||||
block1.block.set_hash(block1_hash);
|
||||
block1.set_block_number(10);
|
||||
block1.set_hash(block1_hash);
|
||||
|
||||
block2.block.set_block_number(11);
|
||||
block2.block.set_hash(block2_hash);
|
||||
block2.set_block_number(11);
|
||||
block2.set_hash(block2_hash);
|
||||
|
||||
// Create a random receipt object, receipt1
|
||||
let receipt1 = OpReceipt::Legacy(Receipt {
|
||||
|
||||
@@ -20,7 +20,7 @@ use reth_optimism_forks::{OpHardfork, OpHardforks};
|
||||
use reth_optimism_payload_builder::{OpBuiltPayload, OpPayloadBuilderAttributes};
|
||||
use reth_optimism_primitives::OpBlock;
|
||||
use reth_payload_validator::ExecutionPayloadValidator;
|
||||
use reth_primitives::SealedBlockFor;
|
||||
use reth_primitives::SealedBlock;
|
||||
use reth_rpc_types_compat::engine::payload::block_to_payload;
|
||||
use std::sync::Arc;
|
||||
|
||||
@@ -51,7 +51,7 @@ where
|
||||
type ExecutionPayloadEnvelopeV4 = OpExecutionPayloadEnvelopeV4;
|
||||
|
||||
fn block_to_payload(
|
||||
block: SealedBlockFor<
|
||||
block: SealedBlock<
|
||||
<<Self::BuiltPayload as BuiltPayload>::Primitives as NodePrimitives>::Block,
|
||||
>,
|
||||
) -> (ExecutionPayload, ExecutionPayloadSidecar) {
|
||||
@@ -96,7 +96,7 @@ impl PayloadValidator for OpEngineValidator {
|
||||
&self,
|
||||
payload: ExecutionPayload,
|
||||
sidecar: ExecutionPayloadSidecar,
|
||||
) -> Result<SealedBlockFor<Self::Block>, PayloadError> {
|
||||
) -> Result<SealedBlock<Self::Block>, PayloadError> {
|
||||
self.inner.ensure_well_formed_payload(payload, sidecar)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -400,10 +400,9 @@ where
|
||||
self.validate_all(transactions)
|
||||
}
|
||||
|
||||
fn on_new_head_block<H, B>(&self, new_tip_block: &SealedBlock<H, B>)
|
||||
fn on_new_head_block<B>(&self, new_tip_block: &SealedBlock<B>)
|
||||
where
|
||||
H: reth_primitives_traits::BlockHeader,
|
||||
B: BlockBody,
|
||||
B: Block,
|
||||
{
|
||||
self.inner.on_new_head_block(new_tip_block);
|
||||
self.update_l1_block_info(
|
||||
|
||||
@@ -15,6 +15,7 @@ workspace = true
|
||||
# reth
|
||||
reth-chainspec.workspace = true
|
||||
reth-primitives.workspace = true
|
||||
reth-primitives-traits.workspace = true
|
||||
reth-revm = { workspace = true, features = ["witness"] }
|
||||
reth-transaction-pool.workspace = true
|
||||
reth-provider.workspace = true
|
||||
|
||||
@@ -25,9 +25,9 @@ use reth_payload_builder_primitives::PayloadBuilderError;
|
||||
use reth_payload_primitives::PayloadBuilderAttributes;
|
||||
use reth_payload_util::{NoopPayloadTransactions, PayloadTransactions};
|
||||
use reth_primitives::{
|
||||
proofs, transaction::SignedTransactionIntoRecoveredExt, Block, BlockBody, BlockExt,
|
||||
SealedHeader, TxType,
|
||||
proofs, transaction::SignedTransactionIntoRecoveredExt, Block, BlockBody, SealedHeader, TxType,
|
||||
};
|
||||
use reth_primitives_traits::block::Block as _;
|
||||
use reth_provider::{
|
||||
HashedPostStateProvider, ProviderError, StateProofProvider, StateProviderFactory,
|
||||
StateRootProvider,
|
||||
|
||||
@@ -17,7 +17,7 @@ use reth_optimism_chainspec::OpChainSpec;
|
||||
use reth_optimism_primitives::{OpBlock, OpPrimitives, OpTransactionSigned};
|
||||
use reth_payload_builder::EthPayloadBuilderAttributes;
|
||||
use reth_payload_primitives::{BuiltPayload, PayloadBuilderAttributes};
|
||||
use reth_primitives::{transaction::WithEncoded, SealedBlockFor};
|
||||
use reth_primitives::{transaction::WithEncoded, SealedBlock};
|
||||
use reth_rpc_types_compat::engine::payload::{
|
||||
block_to_payload_v1, block_to_payload_v3, convert_block_to_payload_field_v2,
|
||||
};
|
||||
@@ -135,7 +135,7 @@ pub struct OpBuiltPayload {
|
||||
/// Identifier of the payload
|
||||
pub(crate) id: PayloadId,
|
||||
/// The built block
|
||||
pub(crate) block: Arc<SealedBlockFor<OpBlock>>,
|
||||
pub(crate) block: Arc<SealedBlock<OpBlock>>,
|
||||
/// Block execution data for the payload, if any.
|
||||
pub(crate) executed_block: Option<ExecutedBlock<OpPrimitives>>,
|
||||
/// The fees of the block
|
||||
@@ -155,7 +155,7 @@ impl OpBuiltPayload {
|
||||
/// Initializes the payload with the given initial block.
|
||||
pub const fn new(
|
||||
id: PayloadId,
|
||||
block: Arc<SealedBlockFor<OpBlock>>,
|
||||
block: Arc<SealedBlock<OpBlock>>,
|
||||
fees: U256,
|
||||
chain_spec: Arc<OpChainSpec>,
|
||||
attributes: OpPayloadBuilderAttributes,
|
||||
@@ -170,7 +170,7 @@ impl OpBuiltPayload {
|
||||
}
|
||||
|
||||
/// Returns the built block(sealed)
|
||||
pub fn block(&self) -> &SealedBlockFor<OpBlock> {
|
||||
pub fn block(&self) -> &SealedBlock<OpBlock> {
|
||||
&self.block
|
||||
}
|
||||
|
||||
@@ -188,7 +188,7 @@ impl OpBuiltPayload {
|
||||
impl BuiltPayload for OpBuiltPayload {
|
||||
type Primitives = OpPrimitives;
|
||||
|
||||
fn block(&self) -> &SealedBlockFor<OpBlock> {
|
||||
fn block(&self) -> &SealedBlock<OpBlock> {
|
||||
&self.block
|
||||
}
|
||||
|
||||
@@ -208,7 +208,7 @@ impl BuiltPayload for OpBuiltPayload {
|
||||
impl BuiltPayload for &OpBuiltPayload {
|
||||
type Primitives = OpPrimitives;
|
||||
|
||||
fn block(&self) -> &SealedBlockFor<OpBlock> {
|
||||
fn block(&self) -> &SealedBlock<OpBlock> {
|
||||
(**self).block()
|
||||
}
|
||||
|
||||
|
||||
@@ -14,7 +14,7 @@ use reth_evm::ConfigureEvm;
|
||||
use reth_optimism_consensus::calculate_receipt_root_no_memo_optimism;
|
||||
use reth_optimism_forks::OpHardforks;
|
||||
use reth_optimism_primitives::{OpBlock, OpReceipt, OpTransactionSigned};
|
||||
use reth_primitives::{logs_bloom, BlockBody, SealedBlockWithSenders};
|
||||
use reth_primitives::{logs_bloom, BlockBody, RecoveredBlock};
|
||||
use reth_provider::{
|
||||
BlockReader, BlockReaderIdExt, ChainSpecProvider, ProviderBlock, ProviderHeader,
|
||||
ProviderReceipt, ProviderTx, ReceiptProvider, StateProviderFactory,
|
||||
@@ -64,7 +64,7 @@ where
|
||||
&self,
|
||||
) -> Result<
|
||||
Option<(
|
||||
SealedBlockWithSenders<ProviderBlock<Self::Provider>>,
|
||||
RecoveredBlock<ProviderBlock<Self::Provider>>,
|
||||
Vec<ProviderReceipt<Self::Provider>>,
|
||||
)>,
|
||||
Self::Error,
|
||||
@@ -80,8 +80,7 @@ where
|
||||
.provider()
|
||||
.block_with_senders(block_id, Default::default())
|
||||
.map_err(Self::Error::from_eth_err)?
|
||||
.ok_or(EthApiError::HeaderNotFound(block_id.into()))?
|
||||
.seal_unchecked(latest.hash());
|
||||
.ok_or(EthApiError::HeaderNotFound(block_id.into()))?;
|
||||
|
||||
let receipts = self
|
||||
.provider()
|
||||
|
||||
@@ -7,7 +7,7 @@ use alloy_eips::eip4895::Withdrawals;
|
||||
use alloy_primitives::{Address, B256, U256};
|
||||
use reth_payload_builder::PayloadId;
|
||||
use reth_payload_primitives::BuiltPayload;
|
||||
use reth_primitives::{NodePrimitives, SealedBlockFor};
|
||||
use reth_primitives::{NodePrimitives, SealedBlock};
|
||||
|
||||
use alloy_eips::eip7685::Requests;
|
||||
use std::{error::Error, fmt};
|
||||
@@ -155,7 +155,7 @@ where
|
||||
{
|
||||
type Primitives = L::Primitives;
|
||||
|
||||
fn block(&self) -> &SealedBlockFor<<L::Primitives as NodePrimitives>::Block> {
|
||||
fn block(&self) -> &SealedBlock<<L::Primitives as NodePrimitives>::Block> {
|
||||
match self {
|
||||
Self::Left(l) => l.block(),
|
||||
Self::Right(r) => r.block(),
|
||||
|
||||
@@ -31,7 +31,7 @@
|
||||
//! use alloy_consensus::Header;
|
||||
//! use alloy_primitives::U256;
|
||||
//! use reth_payload_builder::{EthBuiltPayload, PayloadBuilderError, KeepPayloadJobAlive, EthPayloadBuilderAttributes, PayloadJob, PayloadJobGenerator, PayloadKind};
|
||||
//! use reth_primitives::{Block, BlockExt};
|
||||
//! use reth_primitives::{Block, SealedBlock};
|
||||
//!
|
||||
//! /// The generator type that creates new jobs that builds empty blocks.
|
||||
//! pub struct EmptyBlockPayloadJobGenerator;
|
||||
@@ -67,7 +67,7 @@
|
||||
//! },
|
||||
//! ..Default::default()
|
||||
//! };
|
||||
//! let payload = EthBuiltPayload::new(self.attributes.id, Arc::new(block.seal_slow()), U256::ZERO, None, None);
|
||||
//! let payload = EthBuiltPayload::new(self.attributes.id, Arc::new(SealedBlock::seal_slow(block)), U256::ZERO, None, None);
|
||||
//! Ok(payload)
|
||||
//! }
|
||||
//!
|
||||
|
||||
@@ -9,7 +9,8 @@ use alloy_primitives::U256;
|
||||
use reth_chain_state::{CanonStateNotification, ExecutedBlock};
|
||||
use reth_payload_builder_primitives::PayloadBuilderError;
|
||||
use reth_payload_primitives::{PayloadKind, PayloadTypes};
|
||||
use reth_primitives::{Block, BlockExt};
|
||||
use reth_primitives::Block;
|
||||
use reth_primitives_traits::Block as _;
|
||||
use std::{
|
||||
future::Future,
|
||||
pin::Pin,
|
||||
|
||||
@@ -5,7 +5,7 @@ use alloy_eips::{
|
||||
use alloy_primitives::{Address, B256, U256};
|
||||
use alloy_rpc_types_engine::{PayloadAttributes as EthPayloadAttributes, PayloadId};
|
||||
use reth_chain_state::ExecutedBlock;
|
||||
use reth_primitives::{NodePrimitives, SealedBlockFor};
|
||||
use reth_primitives::{NodePrimitives, SealedBlock};
|
||||
|
||||
/// Represents a built payload type that contains a built `SealedBlock` and can be converted into
|
||||
/// engine API execution payloads.
|
||||
@@ -14,7 +14,7 @@ pub trait BuiltPayload: Send + Sync + std::fmt::Debug {
|
||||
type Primitives: NodePrimitives;
|
||||
|
||||
/// Returns the built block (sealed)
|
||||
fn block(&self) -> &SealedBlockFor<<Self::Primitives as NodePrimitives>::Block>;
|
||||
fn block(&self) -> &SealedBlock<<Self::Primitives as NodePrimitives>::Block>;
|
||||
|
||||
/// Returns the fees collected for the built block
|
||||
fn fees(&self) -> U256;
|
||||
|
||||
@@ -12,8 +12,8 @@ use alloy_rpc_types::engine::{
|
||||
ExecutionPayload, ExecutionPayloadSidecar, MaybeCancunPayloadFields, PayloadError,
|
||||
};
|
||||
use reth_chainspec::EthereumHardforks;
|
||||
use reth_primitives::{BlockBody, BlockExt, Header, SealedBlock};
|
||||
use reth_primitives_traits::SignedTransaction;
|
||||
use reth_primitives::SealedBlock;
|
||||
use reth_primitives_traits::{Block, SignedTransaction};
|
||||
use std::sync::Arc;
|
||||
|
||||
/// Execution payload validator.
|
||||
@@ -59,9 +59,9 @@ impl<ChainSpec: EthereumHardforks> ExecutionPayloadValidator<ChainSpec> {
|
||||
///
|
||||
/// Ensures that the number of blob versioned hashes matches the number hashes included in the
|
||||
/// _separate_ `block_versioned_hashes` of the cancun payload fields.
|
||||
fn ensure_matching_blob_versioned_hashes<T: SignedTransaction>(
|
||||
fn ensure_matching_blob_versioned_hashes<B: Block>(
|
||||
&self,
|
||||
sealed_block: &SealedBlock<Header, BlockBody<T>>,
|
||||
sealed_block: &SealedBlock<B>,
|
||||
cancun_fields: &MaybeCancunPayloadFields,
|
||||
) -> Result<(), PayloadError> {
|
||||
let num_blob_versioned_hashes = sealed_block.blob_versioned_hashes_iter().count();
|
||||
@@ -116,7 +116,7 @@ impl<ChainSpec: EthereumHardforks> ExecutionPayloadValidator<ChainSpec> {
|
||||
&self,
|
||||
payload: ExecutionPayload,
|
||||
sidecar: ExecutionPayloadSidecar,
|
||||
) -> Result<SealedBlock<Header, BlockBody<T>>, PayloadError> {
|
||||
) -> Result<SealedBlock<reth_primitives::Block<T>>, PayloadError> {
|
||||
let expected_hash = payload.block_hash();
|
||||
|
||||
// First parse the block
|
||||
|
||||
@@ -1,10 +1,11 @@
|
||||
//! Block body abstraction.
|
||||
|
||||
use crate::{
|
||||
BlockHeader, FullSignedTx, InMemorySize, MaybeSerde, MaybeSerdeBincodeCompat, SignedTransaction,
|
||||
transaction::signed::RecoveryError, BlockHeader, FullSignedTx, InMemorySize, MaybeSerde,
|
||||
MaybeSerdeBincodeCompat, SignedTransaction,
|
||||
};
|
||||
use alloc::{fmt, vec::Vec};
|
||||
use alloy_consensus::{Header, Transaction};
|
||||
use alloy_consensus::{Header, Transaction, Typed2718};
|
||||
use alloy_eips::{eip2718::Encodable2718, eip4895::Withdrawals};
|
||||
use alloy_primitives::{Address, Bytes, B256};
|
||||
|
||||
@@ -14,6 +15,9 @@ pub trait FullBlockBody: BlockBody<Transaction: FullSignedTx> + MaybeSerdeBincod
|
||||
impl<T> FullBlockBody for T where T: BlockBody<Transaction: FullSignedTx> + MaybeSerdeBincodeCompat {}
|
||||
|
||||
/// Abstraction for block's body.
|
||||
///
|
||||
/// This type is a container for everything that is included in a block except the header.
|
||||
/// For ethereum this includes transactions, ommers, and withdrawals.
|
||||
pub trait BlockBody:
|
||||
Send
|
||||
+ Sync
|
||||
@@ -47,9 +51,15 @@ pub trait BlockBody:
|
||||
fn transaction_count(&self) -> usize {
|
||||
self.transactions().len()
|
||||
}
|
||||
|
||||
/// Consume the block body and return a [`Vec`] of transactions.
|
||||
fn into_transactions(self) -> Vec<Self::Transaction>;
|
||||
|
||||
/// Returns `true` if the block body contains a transaction of the given type.
|
||||
fn contains_transaction_type(&self, tx_type: u8) -> bool {
|
||||
self.transactions().iter().any(|tx| tx.is_type(tx_type))
|
||||
}
|
||||
|
||||
/// Calculate the transaction root for the block body.
|
||||
fn calculate_tx_root(&self) -> B256 {
|
||||
alloy_consensus::proofs::calculate_transaction_root(self.transactions())
|
||||
@@ -115,6 +125,16 @@ pub trait BlockBody:
|
||||
crate::transaction::recover::recover_signers(self.transactions())
|
||||
}
|
||||
|
||||
/// Recover signer addresses for all transactions in the block body.
|
||||
///
|
||||
/// Returns an error if some transaction's signature is invalid.
|
||||
fn try_recover_signers(&self) -> Result<Vec<Address>, RecoveryError>
|
||||
where
|
||||
Self::Transaction: SignedTransaction,
|
||||
{
|
||||
self.recover_signers().ok_or(RecoveryError)
|
||||
}
|
||||
|
||||
/// Recover signer addresses for all transactions in the block body _without ensuring that the
|
||||
/// signature has a low `s` value_.
|
||||
///
|
||||
@@ -125,6 +145,17 @@ pub trait BlockBody:
|
||||
{
|
||||
crate::transaction::recover::recover_signers_unchecked(self.transactions())
|
||||
}
|
||||
|
||||
/// Recover signer addresses for all transactions in the block body _without ensuring that the
|
||||
/// signature has a low `s` value_.
|
||||
///
|
||||
/// Returns an error if some transaction's signature is invalid.
|
||||
fn try_recover_signers_unchecked(&self) -> Result<Vec<Address>, RecoveryError>
|
||||
where
|
||||
Self::Transaction: SignedTransaction,
|
||||
{
|
||||
self.recover_signers_unchecked().ok_or(RecoveryError)
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> BlockBody for alloy_consensus::BlockBody<T>
|
||||
|
||||
33
crates/primitives-traits/src/block/error.rs
Normal file
33
crates/primitives-traits/src/block/error.rs
Normal file
@@ -0,0 +1,33 @@
|
||||
//! Error types for the `block` module.
|
||||
|
||||
use crate::transaction::signed::RecoveryError;
|
||||
|
||||
/// Type alias for [`BlockRecoveryError`] with a [`SealedBlock`](crate::SealedBlock) value.
|
||||
pub type SealedBlockRecoveryError<B> = BlockRecoveryError<crate::SealedBlock<B>>;
|
||||
|
||||
/// Error when recovering a block from [`SealedBlock`](crate::SealedBlock) to
|
||||
/// [`RecoveredBlock`](crate::RecoveredBlock).
|
||||
///
|
||||
/// This error is returned when the block recovery fails and contains the erroneous block, because
|
||||
/// recovering a block takes ownership of the block.
|
||||
#[derive(Debug, Clone, thiserror::Error)]
|
||||
#[error("Failed to recover the block")]
|
||||
pub struct BlockRecoveryError<T>(pub T);
|
||||
|
||||
impl<T> BlockRecoveryError<T> {
|
||||
/// Create a new error.
|
||||
pub const fn new(inner: T) -> Self {
|
||||
Self(inner)
|
||||
}
|
||||
|
||||
/// Unwrap the error and return the original value.
|
||||
pub fn into_inner(self) -> T {
|
||||
self.0
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> From<BlockRecoveryError<T>> for RecoveryError {
|
||||
fn from(_: BlockRecoveryError<T>) -> Self {
|
||||
Self
|
||||
}
|
||||
}
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user