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

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,