feat: add SealedBlock in reth-primitives-traits (#13735)

This commit is contained in:
Matthias Seitz
2025-01-15 02:12:43 +01:00
committed by GitHub
parent 1267718c7e
commit 83b2fb9b41
171 changed files with 3231 additions and 2866 deletions

461
Cargo.lock generated

File diff suppressed because it is too large Load Diff

View File

@@ -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()),
},
)
}

View File

@@ -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();

View File

@@ -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,

View File

@@ -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

View File

@@ -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,

View File

@@ -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(

View File

@@ -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

View File

@@ -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

View File

@@ -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
))

View File

@@ -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]));

View File

@@ -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

View File

@@ -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() {

View File

@@ -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,
)?;

View File

@@ -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"
]

View File

@@ -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;

View File

@@ -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,
{

View File

@@ -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(())

View File

@@ -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 {

View File

@@ -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 {

View File

@@ -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)>,
) {

View File

@@ -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

View File

@@ -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)>,
) {

View File

@@ -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.

View File

@@ -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>>;

View File

@@ -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();

View File

@@ -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));

View File

@@ -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> {

View File

@@ -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());
}
}

View File

@@ -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(&current_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

View File

@@ -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)
}

View File

@@ -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)>,
) {

View File

@@ -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);

View File

@@ -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;
}
}

View File

@@ -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

View File

@@ -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,

View File

@@ -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(())
);
}

View File

@@ -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,

View File

@@ -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) {

View File

@@ -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 transactions gas limit, Tg, and the gas utilized in this block prior,
// must be no greater than the blocks gasLimit.
let block_available_gas = block.header.gas_limit - cumulative_gas_used;
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 {

View File

@@ -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}");
}

View File

@@ -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>>>,

View File

@@ -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 {

View File

@@ -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> {

View File

@@ -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;

View File

@@ -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;

View File

@@ -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,

View File

@@ -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;

View File

@@ -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);

View File

@@ -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);

View File

@@ -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(

View File

@@ -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();

View File

@@ -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,
)),

View File

@@ -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

View File

@@ -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)),

View File

@@ -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"] }

View File

@@ -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,

View File

@@ -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,
)),

View File

@@ -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(

View File

@@ -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>,

View File

@@ -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)

View File

@@ -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(),

View File

@@ -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,

View File

@@ -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")

View File

@@ -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!(

View File

@@ -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(

View File

@@ -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, .. } => {

View File

@@ -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>> {

View File

@@ -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)
}

View File

@@ -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;
}

View File

@@ -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<()>;

View File

@@ -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 {

View File

@@ -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);

View File

@@ -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
///

View File

@@ -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}")]

View File

@@ -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> {}

View File

@@ -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;
}

View File

@@ -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;

View File

@@ -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,

View File

@@ -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

View File

@@ -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();

View File

@@ -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)

View File

@@ -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.

View File

@@ -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"]

View File

@@ -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)
}

View File

@@ -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 transactions gas limit, Tg, and the gas utilized in this block prior,
// must be no greater than the blocks gasLimit.
let block_available_gas = block.header().gas_limit - cumulative_gas_used;
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() },

View File

@@ -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 {

View File

@@ -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)
}
}

View File

@@ -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(

View File

@@ -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

View File

@@ -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,

View File

@@ -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()
}

View File

@@ -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()

View File

@@ -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(),

View File

@@ -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)
//! }
//!

View File

@@ -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,

View File

@@ -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;

View File

@@ -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

View File

@@ -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>

View 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