mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: add SealedBlock in reth-primitives-traits (#13735)
This commit is contained in:
@ -8,8 +8,8 @@ use reth_engine_primitives::InvalidBlockHook;
|
||||
use reth_evm::{
|
||||
state_change::post_block_balance_increments, system_calls::SystemCaller, ConfigureEvm,
|
||||
};
|
||||
use reth_primitives::{NodePrimitives, SealedBlockWithSenders, SealedHeader};
|
||||
use reth_primitives_traits::SignedTransaction;
|
||||
use reth_primitives::{NodePrimitives, RecoveredBlock, SealedHeader};
|
||||
use reth_primitives_traits::{BlockBody, SignedTransaction};
|
||||
use reth_provider::{BlockExecutionOutput, ChainSpecProvider, StateProviderFactory};
|
||||
use reth_revm::{
|
||||
database::StateProviderDatabase, db::states::bundle_state::BundleRetention, DatabaseCommit,
|
||||
@ -58,7 +58,7 @@ where
|
||||
fn on_invalid_block<N>(
|
||||
&self,
|
||||
parent_header: &SealedHeader<N::BlockHeader>,
|
||||
block: &SealedBlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
output: &BlockExecutionOutput<N::Receipt>,
|
||||
trie_updates: Option<(&TrieUpdates, B256)>,
|
||||
) -> eyre::Result<()>
|
||||
@ -87,7 +87,7 @@ where
|
||||
|
||||
// Re-execute all of the transactions in the block to load all touched accounts into
|
||||
// the cache DB.
|
||||
for tx in block.transactions() {
|
||||
for tx in block.body().transactions() {
|
||||
self.evm_config.fill_tx_env(
|
||||
evm.tx_mut(),
|
||||
tx,
|
||||
@ -101,10 +101,8 @@ where
|
||||
|
||||
// use U256::MAX here for difficulty, because fetching it is annoying
|
||||
// NOTE: This is not mut because we are not doing the DAO irregular state change here
|
||||
let balance_increments = post_block_balance_increments(
|
||||
self.provider.chain_spec().as_ref(),
|
||||
&block.clone().unseal().block,
|
||||
);
|
||||
let balance_increments =
|
||||
post_block_balance_increments(self.provider.chain_spec().as_ref(), block);
|
||||
|
||||
// increment balances
|
||||
db.increment_balances(balance_increments)?;
|
||||
@ -302,7 +300,7 @@ where
|
||||
fn on_invalid_block(
|
||||
&self,
|
||||
parent_header: &SealedHeader<N::BlockHeader>,
|
||||
block: &SealedBlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
output: &BlockExecutionOutput<N::Receipt>,
|
||||
trie_updates: Option<(&TrieUpdates, B256)>,
|
||||
) {
|
||||
|
||||
@ -4,7 +4,7 @@ use crate::ForkchoiceStatus;
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_primitives::B256;
|
||||
use alloy_rpc_types_engine::ForkchoiceState;
|
||||
use reth_primitives::{EthPrimitives, SealedBlockFor};
|
||||
use reth_primitives::{EthPrimitives, SealedBlock};
|
||||
use reth_primitives_traits::{NodePrimitives, SealedHeader};
|
||||
use std::{
|
||||
fmt::{Display, Formatter, Result},
|
||||
@ -18,9 +18,9 @@ pub enum BeaconConsensusEngineEvent<N: NodePrimitives = EthPrimitives> {
|
||||
/// The fork choice state was updated, and the current fork choice status
|
||||
ForkchoiceUpdated(ForkchoiceState, ForkchoiceStatus),
|
||||
/// A block was added to the fork chain.
|
||||
ForkBlockAdded(Arc<SealedBlockFor<N::Block>>, Duration),
|
||||
ForkBlockAdded(Arc<SealedBlock<N::Block>>, Duration),
|
||||
/// A block was added to the canonical chain, and the elapsed time validating the block
|
||||
CanonicalBlockAdded(Arc<SealedBlockFor<N::Block>>, Duration),
|
||||
CanonicalBlockAdded(Arc<SealedBlock<N::Block>>, Duration),
|
||||
/// A canonical chain was committed, and the elapsed time committing the data
|
||||
CanonicalChainCommitted(Box<SealedHeader<N::BlockHeader>>, Duration),
|
||||
/// The consensus engine is involved in live sync, and has specific progress
|
||||
|
||||
@ -1,6 +1,6 @@
|
||||
use alloy_primitives::B256;
|
||||
use reth_execution_types::BlockExecutionOutput;
|
||||
use reth_primitives::{NodePrimitives, SealedBlockWithSenders, SealedHeader};
|
||||
use reth_primitives::{NodePrimitives, RecoveredBlock, SealedHeader};
|
||||
use reth_trie::updates::TrieUpdates;
|
||||
|
||||
/// An invalid block hook.
|
||||
@ -9,7 +9,7 @@ pub trait InvalidBlockHook<N: NodePrimitives>: Send + Sync {
|
||||
fn on_invalid_block(
|
||||
&self,
|
||||
parent_header: &SealedHeader<N::BlockHeader>,
|
||||
block: &SealedBlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
output: &BlockExecutionOutput<N::Receipt>,
|
||||
trie_updates: Option<(&TrieUpdates, B256)>,
|
||||
);
|
||||
@ -20,7 +20,7 @@ where
|
||||
N: NodePrimitives,
|
||||
F: Fn(
|
||||
&SealedHeader<N::BlockHeader>,
|
||||
&SealedBlockWithSenders<N::Block>,
|
||||
&RecoveredBlock<N::Block>,
|
||||
&BlockExecutionOutput<N::Receipt>,
|
||||
Option<(&TrieUpdates, B256)>,
|
||||
) + Send
|
||||
@ -29,7 +29,7 @@ where
|
||||
fn on_invalid_block(
|
||||
&self,
|
||||
parent_header: &SealedHeader<N::BlockHeader>,
|
||||
block: &SealedBlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
output: &BlockExecutionOutput<N::Receipt>,
|
||||
trie_updates: Option<(&TrieUpdates, B256)>,
|
||||
) {
|
||||
|
||||
@ -33,7 +33,7 @@ pub use reth_payload_primitives::{
|
||||
PayloadTypes,
|
||||
};
|
||||
use reth_payload_primitives::{InvalidPayloadAttributesError, PayloadAttributes};
|
||||
use reth_primitives::{NodePrimitives, SealedBlockFor};
|
||||
use reth_primitives::{NodePrimitives, SealedBlock};
|
||||
use reth_primitives_traits::Block;
|
||||
use serde::{de::DeserializeOwned, ser::Serialize};
|
||||
|
||||
@ -86,7 +86,7 @@ pub trait EngineTypes:
|
||||
|
||||
/// Converts a [`BuiltPayload`] into an [`ExecutionPayload`] and [`ExecutionPayloadSidecar`].
|
||||
fn block_to_payload(
|
||||
block: SealedBlockFor<
|
||||
block: SealedBlock<
|
||||
<<Self::BuiltPayload as BuiltPayload>::Primitives as NodePrimitives>::Block,
|
||||
>,
|
||||
) -> (ExecutionPayload, ExecutionPayloadSidecar);
|
||||
@ -109,7 +109,7 @@ pub trait PayloadValidator: fmt::Debug + Send + Sync + Unpin + 'static {
|
||||
&self,
|
||||
payload: ExecutionPayload,
|
||||
sidecar: ExecutionPayloadSidecar,
|
||||
) -> Result<SealedBlockFor<Self::Block>, PayloadError>;
|
||||
) -> Result<SealedBlock<Self::Block>, PayloadError>;
|
||||
}
|
||||
|
||||
/// Type that validates the payloads processed by the engine.
|
||||
|
||||
@ -16,7 +16,7 @@ pub use reth_engine_tree::{
|
||||
};
|
||||
use reth_evm::execute::BlockExecutorProvider;
|
||||
use reth_network_p2p::BlockClient;
|
||||
use reth_node_types::{BlockTy, BodyTy, HeaderTy, NodeTypes, NodeTypesWithEngine};
|
||||
use reth_node_types::{BlockTy, NodeTypes, NodeTypesWithEngine};
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_primitives::EthPrimitives;
|
||||
use reth_provider::{
|
||||
@ -52,10 +52,13 @@ type EngineServiceType<N, Client> = ChainOrchestrator<
|
||||
/// The type that drives the chain forward and communicates progress.
|
||||
#[pin_project]
|
||||
#[allow(missing_debug_implementations)]
|
||||
// TODO(mattsse): remove hidde once fixed : <https://github.com/rust-lang/rust/issues/135363>
|
||||
// otherwise rustdoc fails to resolve the alias
|
||||
#[doc(hidden)]
|
||||
pub struct EngineService<N, Client, E>
|
||||
where
|
||||
N: EngineNodeTypes,
|
||||
Client: BlockClient<Header = HeaderTy<N>, Body = BodyTy<N>> + 'static,
|
||||
Client: BlockClient<Block = BlockTy<N>> + 'static,
|
||||
E: BlockExecutorProvider + 'static,
|
||||
{
|
||||
orchestrator: EngineServiceType<N, Client>,
|
||||
@ -65,7 +68,7 @@ where
|
||||
impl<N, Client, E> EngineService<N, Client, E>
|
||||
where
|
||||
N: EngineNodeTypes,
|
||||
Client: BlockClient<Header = HeaderTy<N>, Body = BodyTy<N>> + 'static,
|
||||
Client: BlockClient<Block = BlockTy<N>> + 'static,
|
||||
E: BlockExecutorProvider<Primitives = N::Primitives> + 'static,
|
||||
{
|
||||
/// Constructor for `EngineService`.
|
||||
@ -133,7 +136,7 @@ where
|
||||
impl<N, Client, E> Stream for EngineService<N, Client, E>
|
||||
where
|
||||
N: EngineNodeTypes,
|
||||
Client: BlockClient<Header = HeaderTy<N>, Body = BodyTy<N>> + 'static,
|
||||
Client: BlockClient<Block = BlockTy<N>> + 'static,
|
||||
E: BlockExecutorProvider + 'static,
|
||||
{
|
||||
type Item = ChainEvent<BeaconConsensusEngineEvent<N::Primitives>>;
|
||||
|
||||
@ -274,7 +274,7 @@ mod tests {
|
||||
gas_limit: ETHEREUM_BLOCK_GAS_LIMIT,
|
||||
..Default::default()
|
||||
};
|
||||
let header = SealedHeader::seal(header);
|
||||
let header = SealedHeader::seal_slow(header);
|
||||
insert_headers_into_client(&client, header, 0..total_blocks);
|
||||
|
||||
let tip = client.highest_block().expect("there should be blocks here").hash();
|
||||
|
||||
@ -9,7 +9,7 @@ use reth_network_p2p::{
|
||||
full_block::{FetchFullBlockFuture, FetchFullBlockRangeFuture, FullBlockClient},
|
||||
BlockClient,
|
||||
};
|
||||
use reth_primitives::{SealedBlockFor, SealedBlockWithSenders};
|
||||
use reth_primitives::{RecoveredBlock, SealedBlock};
|
||||
use reth_primitives_traits::Block;
|
||||
use std::{
|
||||
cmp::{Ordering, Reverse},
|
||||
@ -45,7 +45,7 @@ pub enum DownloadAction {
|
||||
#[derive(Debug)]
|
||||
pub enum DownloadOutcome<B: Block> {
|
||||
/// Downloaded blocks.
|
||||
Blocks(Vec<SealedBlockWithSenders<B>>),
|
||||
Blocks(Vec<RecoveredBlock<B>>),
|
||||
/// New download started.
|
||||
NewDownloadStarted {
|
||||
/// How many blocks are pending in this download.
|
||||
@ -69,7 +69,7 @@ where
|
||||
inflight_block_range_requests: Vec<FetchFullBlockRangeFuture<Client>>,
|
||||
/// Buffered blocks from downloads - this is a min-heap of blocks, using the block number for
|
||||
/// ordering. This means the blocks will be popped from the heap with ascending block numbers.
|
||||
set_buffered_blocks: BinaryHeap<Reverse<OrderedSealedBlockWithSenders<B>>>,
|
||||
set_buffered_blocks: BinaryHeap<Reverse<OrderedRecoveredBlock<B>>>,
|
||||
/// Engine download metrics.
|
||||
metrics: BlockDownloaderMetrics,
|
||||
/// Pending events to be emitted.
|
||||
@ -78,14 +78,11 @@ where
|
||||
|
||||
impl<Client, B> BasicBlockDownloader<Client, B>
|
||||
where
|
||||
Client: BlockClient<Header = B::Header, Body = B::Body> + 'static,
|
||||
Client: BlockClient<Block = B> + 'static,
|
||||
B: Block,
|
||||
{
|
||||
/// Create a new instance
|
||||
pub fn new(
|
||||
client: Client,
|
||||
consensus: Arc<dyn Consensus<Client::Header, Client::Body, Error = ConsensusError>>,
|
||||
) -> Self {
|
||||
pub fn new(client: Client, consensus: Arc<dyn Consensus<B, Error = ConsensusError>>) -> Self {
|
||||
Self {
|
||||
full_block_client: FullBlockClient::new(client, consensus),
|
||||
inflight_full_block_requests: Vec::new(),
|
||||
@ -192,7 +189,7 @@ where
|
||||
|
||||
impl<Client, B> BlockDownloader for BasicBlockDownloader<Client, B>
|
||||
where
|
||||
Client: BlockClient<Header = B::Header, Body = B::Body>,
|
||||
Client: BlockClient<Block = B>,
|
||||
B: Block,
|
||||
{
|
||||
type Block = B;
|
||||
@ -233,9 +230,7 @@ where
|
||||
.into_iter()
|
||||
.map(|b| {
|
||||
let senders = b.senders().unwrap_or_default();
|
||||
OrderedSealedBlockWithSenders(SealedBlockWithSenders::new_unchecked(
|
||||
b, senders,
|
||||
))
|
||||
OrderedRecoveredBlock(RecoveredBlock::new_sealed(b, senders))
|
||||
})
|
||||
.map(Reverse),
|
||||
);
|
||||
@ -252,7 +247,7 @@ where
|
||||
}
|
||||
|
||||
// drain all unique element of the block buffer if there are any
|
||||
let mut downloaded_blocks: Vec<SealedBlockWithSenders<B>> =
|
||||
let mut downloaded_blocks: Vec<RecoveredBlock<B>> =
|
||||
Vec::with_capacity(self.set_buffered_blocks.len());
|
||||
while let Some(block) = self.set_buffered_blocks.pop() {
|
||||
// peek ahead and pop duplicates
|
||||
@ -269,32 +264,32 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
/// A wrapper type around [`SealedBlockWithSenders`] that implements the [Ord]
|
||||
/// A wrapper type around [`RecoveredBlock`] that implements the [Ord]
|
||||
/// trait by block number.
|
||||
#[derive(Debug, Clone, PartialEq, Eq)]
|
||||
struct OrderedSealedBlockWithSenders<B: Block>(SealedBlockWithSenders<B>);
|
||||
struct OrderedRecoveredBlock<B: Block>(RecoveredBlock<B>);
|
||||
|
||||
impl<B: Block> PartialOrd for OrderedSealedBlockWithSenders<B> {
|
||||
impl<B: Block> PartialOrd for OrderedRecoveredBlock<B> {
|
||||
fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
|
||||
Some(self.cmp(other))
|
||||
}
|
||||
}
|
||||
|
||||
impl<B: Block> Ord for OrderedSealedBlockWithSenders<B> {
|
||||
impl<B: Block> Ord for OrderedRecoveredBlock<B> {
|
||||
fn cmp(&self, other: &Self) -> Ordering {
|
||||
self.0.number().cmp(&other.0.number())
|
||||
}
|
||||
}
|
||||
|
||||
impl<B: Block> From<SealedBlockFor<B>> for OrderedSealedBlockWithSenders<B> {
|
||||
fn from(block: SealedBlockFor<B>) -> Self {
|
||||
impl<B: Block> From<SealedBlock<B>> for OrderedRecoveredBlock<B> {
|
||||
fn from(block: SealedBlock<B>) -> Self {
|
||||
let senders = block.senders().unwrap_or_default();
|
||||
Self(SealedBlockWithSenders::new_unchecked(block, senders))
|
||||
Self(RecoveredBlock::new_sealed(block, senders))
|
||||
}
|
||||
}
|
||||
|
||||
impl<B: Block> From<OrderedSealedBlockWithSenders<B>> for SealedBlockWithSenders<B> {
|
||||
fn from(value: OrderedSealedBlockWithSenders<B>) -> Self {
|
||||
impl<B: Block> From<OrderedRecoveredBlock<B>> for RecoveredBlock<B> {
|
||||
fn from(value: OrderedRecoveredBlock<B>) -> Self {
|
||||
value.0
|
||||
}
|
||||
}
|
||||
@ -348,7 +343,7 @@ mod tests {
|
||||
gas_limit: ETHEREUM_BLOCK_GAS_LIMIT,
|
||||
..Default::default()
|
||||
};
|
||||
let header = SealedHeader::seal(header);
|
||||
let header = SealedHeader::seal_slow(header);
|
||||
|
||||
insert_headers_into_client(&client, header, 0..total_blocks);
|
||||
let consensus = Arc::new(EthBeaconConsensus::new(chain_spec));
|
||||
|
||||
@ -9,7 +9,7 @@ use alloy_primitives::B256;
|
||||
use futures::{Stream, StreamExt};
|
||||
use reth_chain_state::ExecutedBlock;
|
||||
use reth_engine_primitives::{BeaconConsensusEngineEvent, BeaconEngineMessage, EngineTypes};
|
||||
use reth_primitives::{NodePrimitives, SealedBlockWithSenders};
|
||||
use reth_primitives::{NodePrimitives, RecoveredBlock};
|
||||
use reth_primitives_traits::Block;
|
||||
use std::{
|
||||
collections::HashSet,
|
||||
@ -306,7 +306,7 @@ pub enum FromEngine<Req, B: Block> {
|
||||
/// Request from the engine.
|
||||
Request(Req),
|
||||
/// Downloaded blocks from the network.
|
||||
DownloadedBlocks(Vec<SealedBlockWithSenders<B>>),
|
||||
DownloadedBlocks(Vec<RecoveredBlock<B>>),
|
||||
}
|
||||
|
||||
impl<Req: Display, B: Block> Display for FromEngine<Req, B> {
|
||||
|
||||
@ -76,7 +76,7 @@ pub fn insert_headers_into_client(
|
||||
header.parent_hash = hash;
|
||||
header.number += 1;
|
||||
header.timestamp += 1;
|
||||
sealed_header = SealedHeader::seal(header);
|
||||
sealed_header = SealedHeader::seal_slow(header);
|
||||
client.insert(sealed_header.clone(), body.clone());
|
||||
}
|
||||
}
|
||||
|
||||
@ -2,7 +2,7 @@ use crate::tree::metrics::BlockBufferMetrics;
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_primitives::{BlockHash, BlockNumber};
|
||||
use reth_network::cache::LruCache;
|
||||
use reth_primitives::SealedBlockWithSenders;
|
||||
use reth_primitives::RecoveredBlock;
|
||||
use reth_primitives_traits::Block;
|
||||
use std::collections::{BTreeMap, HashMap, HashSet};
|
||||
|
||||
@ -20,7 +20,7 @@ use std::collections::{BTreeMap, HashMap, HashSet};
|
||||
#[derive(Debug)]
|
||||
pub(super) struct BlockBuffer<B: Block> {
|
||||
/// All blocks in the buffer stored by their block hash.
|
||||
pub(crate) blocks: HashMap<BlockHash, SealedBlockWithSenders<B>>,
|
||||
pub(crate) blocks: HashMap<BlockHash, RecoveredBlock<B>>,
|
||||
/// Map of any parent block hash (even the ones not currently in the buffer)
|
||||
/// to the buffered children.
|
||||
/// Allows connecting buffered blocks by parent.
|
||||
@ -50,12 +50,12 @@ impl<B: Block> BlockBuffer<B> {
|
||||
}
|
||||
|
||||
/// Return reference to the requested block.
|
||||
pub(super) fn block(&self, hash: &BlockHash) -> Option<&SealedBlockWithSenders<B>> {
|
||||
pub(super) fn block(&self, hash: &BlockHash) -> Option<&RecoveredBlock<B>> {
|
||||
self.blocks.get(hash)
|
||||
}
|
||||
|
||||
/// Return a reference to the lowest ancestor of the given block in the buffer.
|
||||
pub(super) fn lowest_ancestor(&self, hash: &BlockHash) -> Option<&SealedBlockWithSenders<B>> {
|
||||
pub(super) fn lowest_ancestor(&self, hash: &BlockHash) -> Option<&RecoveredBlock<B>> {
|
||||
let mut current_block = self.blocks.get(hash)?;
|
||||
while let Some(parent) = self.blocks.get(¤t_block.parent_hash()) {
|
||||
current_block = parent;
|
||||
@ -64,7 +64,7 @@ impl<B: Block> BlockBuffer<B> {
|
||||
}
|
||||
|
||||
/// Insert a correct block inside the buffer.
|
||||
pub(super) fn insert_block(&mut self, block: SealedBlockWithSenders<B>) {
|
||||
pub(super) fn insert_block(&mut self, block: RecoveredBlock<B>) {
|
||||
let hash = block.hash();
|
||||
|
||||
self.parent_to_child.entry(block.parent_hash()).or_default().insert(hash);
|
||||
@ -90,7 +90,7 @@ impl<B: Block> BlockBuffer<B> {
|
||||
pub(super) fn remove_block_with_children(
|
||||
&mut self,
|
||||
parent_hash: &BlockHash,
|
||||
) -> Vec<SealedBlockWithSenders<B>> {
|
||||
) -> Vec<RecoveredBlock<B>> {
|
||||
let removed = self
|
||||
.remove_block(parent_hash)
|
||||
.into_iter()
|
||||
@ -149,7 +149,7 @@ impl<B: Block> BlockBuffer<B> {
|
||||
/// This method will only remove the block if it's present inside `self.blocks`.
|
||||
/// The block might be missing from other collections, the method will only ensure that it has
|
||||
/// been removed.
|
||||
fn remove_block(&mut self, hash: &BlockHash) -> Option<SealedBlockWithSenders<B>> {
|
||||
fn remove_block(&mut self, hash: &BlockHash) -> Option<RecoveredBlock<B>> {
|
||||
let block = self.blocks.remove(hash)?;
|
||||
self.remove_from_earliest_blocks(block.number(), hash);
|
||||
self.remove_from_parent(block.parent_hash(), hash);
|
||||
@ -158,7 +158,7 @@ impl<B: Block> BlockBuffer<B> {
|
||||
}
|
||||
|
||||
/// Remove all children and their descendants for the given blocks and return them.
|
||||
fn remove_children(&mut self, parent_hashes: Vec<BlockHash>) -> Vec<SealedBlockWithSenders<B>> {
|
||||
fn remove_children(&mut self, parent_hashes: Vec<BlockHash>) -> Vec<RecoveredBlock<B>> {
|
||||
// remove all parent child connection and all the child children blocks that are connected
|
||||
// to the discarded parent blocks.
|
||||
let mut remove_parent_children = parent_hashes;
|
||||
@ -184,15 +184,19 @@ mod tests {
|
||||
use super::*;
|
||||
use alloy_eips::BlockNumHash;
|
||||
use alloy_primitives::BlockHash;
|
||||
use reth_primitives::SealedBlockWithSenders;
|
||||
use reth_primitives::RecoveredBlock;
|
||||
use reth_testing_utils::generators::{self, random_block, BlockParams, Rng};
|
||||
use std::collections::HashMap;
|
||||
|
||||
/// Create random block with specified number and parent hash.
|
||||
fn create_block<R: Rng>(rng: &mut R, number: u64, parent: BlockHash) -> SealedBlockWithSenders {
|
||||
fn create_block<R: Rng>(
|
||||
rng: &mut R,
|
||||
number: u64,
|
||||
parent: BlockHash,
|
||||
) -> RecoveredBlock<reth_primitives::Block> {
|
||||
let block =
|
||||
random_block(rng, number, BlockParams { parent: Some(parent), ..Default::default() });
|
||||
block.seal_with_senders().unwrap()
|
||||
block.try_recover().unwrap()
|
||||
}
|
||||
|
||||
/// Assert that all buffer collections have the same data length.
|
||||
@ -210,7 +214,10 @@ mod tests {
|
||||
}
|
||||
|
||||
/// Assert that the block was removed from all buffer collections.
|
||||
fn assert_block_removal<B: Block>(buffer: &BlockBuffer<B>, block: &SealedBlockWithSenders) {
|
||||
fn assert_block_removal<B: Block>(
|
||||
buffer: &BlockBuffer<B>,
|
||||
block: &RecoveredBlock<reth_primitives::Block>,
|
||||
) {
|
||||
assert!(!buffer.blocks.contains_key(&block.hash()));
|
||||
assert!(buffer
|
||||
.parent_to_child
|
||||
|
||||
@ -4,7 +4,7 @@ use alloy_consensus::BlockHeader;
|
||||
use reth_consensus::ConsensusError;
|
||||
use reth_errors::{BlockExecutionError, BlockValidationError, ProviderError};
|
||||
use reth_evm::execute::InternalBlockExecutionError;
|
||||
use reth_primitives::SealedBlockFor;
|
||||
use reth_primitives::SealedBlock;
|
||||
use reth_primitives_traits::{Block, BlockBody};
|
||||
use tokio::sync::oneshot::error::TryRecvError;
|
||||
|
||||
@ -27,7 +27,7 @@ pub enum AdvancePersistenceError {
|
||||
.block.parent_hash(),
|
||||
.kind)]
|
||||
struct InsertBlockErrorData<B: Block> {
|
||||
block: SealedBlockFor<B>,
|
||||
block: SealedBlock<B>,
|
||||
#[source]
|
||||
kind: InsertBlockErrorKind,
|
||||
}
|
||||
@ -45,11 +45,11 @@ impl<B: Block> std::fmt::Debug for InsertBlockErrorData<B> {
|
||||
}
|
||||
|
||||
impl<B: Block> InsertBlockErrorData<B> {
|
||||
const fn new(block: SealedBlockFor<B>, kind: InsertBlockErrorKind) -> Self {
|
||||
const fn new(block: SealedBlock<B>, kind: InsertBlockErrorKind) -> Self {
|
||||
Self { block, kind }
|
||||
}
|
||||
|
||||
fn boxed(block: SealedBlockFor<B>, kind: InsertBlockErrorKind) -> Box<Self> {
|
||||
fn boxed(block: SealedBlock<B>, kind: InsertBlockErrorKind) -> Box<Self> {
|
||||
Box::new(Self::new(block, kind))
|
||||
}
|
||||
}
|
||||
@ -65,23 +65,23 @@ pub struct InsertBlockError<B: Block> {
|
||||
|
||||
impl<B: Block> InsertBlockError<B> {
|
||||
/// Create a new `InsertInvalidBlockErrorTwo`
|
||||
pub fn new(block: SealedBlockFor<B>, kind: InsertBlockErrorKind) -> Self {
|
||||
pub fn new(block: SealedBlock<B>, kind: InsertBlockErrorKind) -> Self {
|
||||
Self { inner: InsertBlockErrorData::boxed(block, kind) }
|
||||
}
|
||||
|
||||
/// Create a new `InsertInvalidBlockError` from a consensus error
|
||||
pub fn consensus_error(error: ConsensusError, block: SealedBlockFor<B>) -> Self {
|
||||
pub fn consensus_error(error: ConsensusError, block: SealedBlock<B>) -> Self {
|
||||
Self::new(block, InsertBlockErrorKind::Consensus(error))
|
||||
}
|
||||
|
||||
/// Create a new `InsertInvalidBlockError` from a consensus error
|
||||
pub fn sender_recovery_error(block: SealedBlockFor<B>) -> Self {
|
||||
pub fn sender_recovery_error(block: SealedBlock<B>) -> Self {
|
||||
Self::new(block, InsertBlockErrorKind::SenderRecovery)
|
||||
}
|
||||
|
||||
/// Consumes the error and returns the block that resulted in the error
|
||||
#[inline]
|
||||
pub fn into_block(self) -> SealedBlockFor<B> {
|
||||
pub fn into_block(self) -> SealedBlock<B> {
|
||||
self.inner.block
|
||||
}
|
||||
|
||||
@ -93,13 +93,13 @@ impl<B: Block> InsertBlockError<B> {
|
||||
|
||||
/// Returns the block that resulted in the error
|
||||
#[inline]
|
||||
pub const fn block(&self) -> &SealedBlockFor<B> {
|
||||
pub const fn block(&self) -> &SealedBlock<B> {
|
||||
&self.inner.block
|
||||
}
|
||||
|
||||
/// Consumes the type and returns the block and error kind.
|
||||
#[inline]
|
||||
pub fn split(self) -> (SealedBlockFor<B>, InsertBlockErrorKind) {
|
||||
pub fn split(self) -> (SealedBlock<B>, InsertBlockErrorKind) {
|
||||
let inner = *self.inner;
|
||||
(inner.block, inner.kind)
|
||||
}
|
||||
|
||||
@ -1,6 +1,6 @@
|
||||
use alloy_primitives::B256;
|
||||
use reth_engine_primitives::InvalidBlockHook;
|
||||
use reth_primitives::{NodePrimitives, SealedBlockWithSenders, SealedHeader};
|
||||
use reth_primitives::{NodePrimitives, RecoveredBlock, SealedHeader};
|
||||
use reth_provider::BlockExecutionOutput;
|
||||
use reth_trie::updates::TrieUpdates;
|
||||
|
||||
@ -13,7 +13,7 @@ impl<N: NodePrimitives> InvalidBlockHook<N> for NoopInvalidBlockHook {
|
||||
fn on_invalid_block(
|
||||
&self,
|
||||
_parent_header: &SealedHeader<N::BlockHeader>,
|
||||
_block: &SealedBlockWithSenders<N::Block>,
|
||||
_block: &RecoveredBlock<N::Block>,
|
||||
_output: &BlockExecutionOutput<N::Receipt>,
|
||||
_trie_updates: Option<(&TrieUpdates, B256)>,
|
||||
) {
|
||||
@ -33,7 +33,7 @@ impl<N: NodePrimitives> InvalidBlockHook<N> for InvalidBlockHooks<N> {
|
||||
fn on_invalid_block(
|
||||
&self,
|
||||
parent_header: &SealedHeader<N::BlockHeader>,
|
||||
block: &SealedBlockWithSenders<N::Block>,
|
||||
block: &RecoveredBlock<N::Block>,
|
||||
output: &BlockExecutionOutput<N::Receipt>,
|
||||
trie_updates: Option<(&TrieUpdates, B256)>,
|
||||
) {
|
||||
|
||||
@ -111,7 +111,7 @@ mod tests {
|
||||
fn test_hit_eviction() {
|
||||
let mut cache = InvalidHeaderCache::new(10);
|
||||
let header = Header::default();
|
||||
let header = SealedHeader::seal(header);
|
||||
let header = SealedHeader::seal_slow(header);
|
||||
cache.insert(header.block_with_parent());
|
||||
assert_eq!(cache.headers.get(&header.hash()).unwrap().hit_count, 0);
|
||||
|
||||
|
||||
@ -36,8 +36,7 @@ use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_payload_builder_primitives::PayloadBuilder;
|
||||
use reth_payload_primitives::PayloadBuilderAttributes;
|
||||
use reth_primitives::{
|
||||
EthPrimitives, GotExpected, NodePrimitives, SealedBlockFor, SealedBlockWithSenders,
|
||||
SealedHeader,
|
||||
EthPrimitives, GotExpected, NodePrimitives, RecoveredBlock, SealedBlock, SealedHeader,
|
||||
};
|
||||
use reth_primitives_traits::Block;
|
||||
use reth_provider::{
|
||||
@ -147,7 +146,7 @@ impl<N: NodePrimitives> TreeState<N> {
|
||||
}
|
||||
|
||||
/// Returns the block by hash.
|
||||
fn block_by_hash(&self, hash: B256) -> Option<Arc<SealedBlockFor<N::Block>>> {
|
||||
fn block_by_hash(&self, hash: B256) -> Option<Arc<SealedBlock<N::Block>>> {
|
||||
self.blocks_by_hash.get(&hash).map(|b| b.block.clone())
|
||||
}
|
||||
|
||||
@ -737,7 +736,7 @@ where
|
||||
/// block request processing isn't blocked for a long time.
|
||||
fn on_downloaded(
|
||||
&mut self,
|
||||
mut blocks: Vec<SealedBlockWithSenders<N::Block>>,
|
||||
mut blocks: Vec<RecoveredBlock<N::Block>>,
|
||||
) -> Result<Option<TreeEvent>, InsertBlockFatalError> {
|
||||
if blocks.is_empty() {
|
||||
// nothing to execute
|
||||
@ -1603,11 +1602,11 @@ where
|
||||
.provider
|
||||
.sealed_block_with_senders(hash.into(), TransactionVariant::WithHash)?
|
||||
.ok_or_else(|| ProviderError::HeaderNotFound(hash.into()))?
|
||||
.split();
|
||||
.split_sealed();
|
||||
let execution_output = self
|
||||
.provider
|
||||
.get_state(block.number())?
|
||||
.ok_or_else(|| ProviderError::StateForNumberNotFound(block.number()))?;
|
||||
.get_state(block.header().number())?
|
||||
.ok_or_else(|| ProviderError::StateForNumberNotFound(block.header().number()))?;
|
||||
let hashed_state = self.provider.hashed_post_state(execution_output.state());
|
||||
|
||||
Ok(Some(ExecutedBlock {
|
||||
@ -1650,7 +1649,7 @@ where
|
||||
.tree_state
|
||||
.block_by_hash(hash)
|
||||
// TODO: clone for compatibility. should we return an Arc here?
|
||||
.map(|block| block.as_ref().clone().unseal());
|
||||
.map(|block| block.as_ref().clone().into_block());
|
||||
}
|
||||
Ok(block)
|
||||
}
|
||||
@ -1798,11 +1797,10 @@ where
|
||||
|
||||
/// Validate if block is correct and satisfies all the consensus rules that concern the header
|
||||
/// and block body itself.
|
||||
fn validate_block(
|
||||
&self,
|
||||
block: &SealedBlockWithSenders<N::Block>,
|
||||
) -> Result<(), ConsensusError> {
|
||||
if let Err(e) = self.consensus.validate_header_with_total_difficulty(block, U256::MAX) {
|
||||
fn validate_block(&self, block: &RecoveredBlock<N::Block>) -> Result<(), ConsensusError> {
|
||||
if let Err(e) =
|
||||
self.consensus.validate_header_with_total_difficulty(block.header(), U256::MAX)
|
||||
{
|
||||
error!(
|
||||
target: "engine::tree",
|
||||
?block,
|
||||
@ -1817,7 +1815,7 @@ where
|
||||
return Err(e)
|
||||
}
|
||||
|
||||
if let Err(e) = self.consensus.validate_block_pre_execution(block) {
|
||||
if let Err(e) = self.consensus.validate_block_pre_execution(block.sealed_block()) {
|
||||
error!(target: "engine::tree", ?block, "Failed to validate block {}: {e}", block.hash());
|
||||
return Err(e)
|
||||
}
|
||||
@ -1870,21 +1868,21 @@ where
|
||||
/// Returns an error if sender recovery failed or inserting into the buffer failed.
|
||||
fn buffer_block_without_senders(
|
||||
&mut self,
|
||||
block: SealedBlockFor<N::Block>,
|
||||
block: SealedBlock<N::Block>,
|
||||
) -> Result<(), InsertBlockError<N::Block>> {
|
||||
match block.try_seal_with_senders() {
|
||||
match block.try_recover() {
|
||||
Ok(block) => self.buffer_block(block),
|
||||
Err(block) => Err(InsertBlockError::sender_recovery_error(block)),
|
||||
Err(err) => Err(InsertBlockError::sender_recovery_error(err.into_inner())),
|
||||
}
|
||||
}
|
||||
|
||||
/// Pre-validates the block and inserts it into the buffer.
|
||||
fn buffer_block(
|
||||
&mut self,
|
||||
block: SealedBlockWithSenders<N::Block>,
|
||||
block: RecoveredBlock<N::Block>,
|
||||
) -> Result<(), InsertBlockError<N::Block>> {
|
||||
if let Err(err) = self.validate_block(&block) {
|
||||
return Err(InsertBlockError::consensus_error(err, block.block))
|
||||
return Err(InsertBlockError::consensus_error(err, block.into_sealed_block()))
|
||||
}
|
||||
self.state.buffer.insert_block(block);
|
||||
Ok(())
|
||||
@ -2139,7 +2137,7 @@ where
|
||||
#[instrument(level = "trace", skip_all, fields(block_hash = %block.hash(), block_num = %block.number(),), target = "engine::tree")]
|
||||
fn on_downloaded_block(
|
||||
&mut self,
|
||||
block: SealedBlockWithSenders<N::Block>,
|
||||
block: RecoveredBlock<N::Block>,
|
||||
) -> Result<Option<TreeEvent>, InsertBlockFatalError> {
|
||||
let block_num_hash = block.num_hash();
|
||||
let lowest_buffered_ancestor = self.lowest_buffered_ancestor_or(block_num_hash.hash);
|
||||
@ -2194,25 +2192,25 @@ where
|
||||
|
||||
fn insert_block_without_senders(
|
||||
&mut self,
|
||||
block: SealedBlockFor<N::Block>,
|
||||
block: SealedBlock<N::Block>,
|
||||
) -> Result<InsertPayloadOk, InsertBlockError<N::Block>> {
|
||||
match block.try_seal_with_senders() {
|
||||
match block.try_recover() {
|
||||
Ok(block) => self.insert_block(block),
|
||||
Err(block) => Err(InsertBlockError::sender_recovery_error(block)),
|
||||
Err(err) => Err(InsertBlockError::sender_recovery_error(err.into_inner())),
|
||||
}
|
||||
}
|
||||
|
||||
fn insert_block(
|
||||
&mut self,
|
||||
block: SealedBlockWithSenders<N::Block>,
|
||||
block: RecoveredBlock<N::Block>,
|
||||
) -> Result<InsertPayloadOk, InsertBlockError<N::Block>> {
|
||||
self.insert_block_inner(block.clone())
|
||||
.map_err(|kind| InsertBlockError::new(block.block, kind))
|
||||
.map_err(|kind| InsertBlockError::new(block.into_sealed_block(), kind))
|
||||
}
|
||||
|
||||
fn insert_block_inner(
|
||||
&mut self,
|
||||
block: SealedBlockWithSenders<N::Block>,
|
||||
block: RecoveredBlock<N::Block>,
|
||||
) -> Result<InsertPayloadOk, InsertBlockErrorKind> {
|
||||
debug!(target: "engine::tree", block=?block.num_hash(), parent = ?block.parent_hash(), state_root = ?block.state_root(), "Inserting new block into tree");
|
||||
|
||||
@ -2270,8 +2268,7 @@ where
|
||||
|
||||
let block_number = block.number();
|
||||
let block_hash = block.hash();
|
||||
let sealed_block = Arc::new(block.block.clone());
|
||||
let block = block.unseal();
|
||||
let sealed_block = Arc::new(block.clone_sealed_block());
|
||||
|
||||
let persistence_not_in_progress = !self.persistence_state.in_progress();
|
||||
|
||||
@ -2341,12 +2338,7 @@ where
|
||||
PostExecutionInput::new(&output.receipts, &output.requests),
|
||||
) {
|
||||
// call post-block hook
|
||||
self.invalid_block_hook.on_invalid_block(
|
||||
&parent_block,
|
||||
&block.clone().seal_slow(),
|
||||
&output,
|
||||
None,
|
||||
);
|
||||
self.invalid_block_hook.on_invalid_block(&parent_block, &block, &output, None);
|
||||
return Err(err.into())
|
||||
}
|
||||
|
||||
@ -2458,7 +2450,7 @@ where
|
||||
// call post-block hook
|
||||
self.invalid_block_hook.on_invalid_block(
|
||||
&parent_block,
|
||||
&block.clone().seal_slow(),
|
||||
&block,
|
||||
&output,
|
||||
Some((&trie_output, state_root)),
|
||||
);
|
||||
@ -2841,7 +2833,8 @@ mod tests {
|
||||
use reth_ethereum_consensus::EthBeaconConsensus;
|
||||
use reth_ethereum_engine_primitives::{EthEngineTypes, EthereumEngineValidator};
|
||||
use reth_evm::test_utils::MockExecutorProvider;
|
||||
use reth_primitives::{Block, BlockExt, EthPrimitives};
|
||||
use reth_primitives::{Block, EthPrimitives};
|
||||
use reth_primitives_traits::Block as _;
|
||||
use reth_provider::test_utils::MockEthProvider;
|
||||
use reth_rpc_types_compat::engine::{block_to_payload_v1, payload::block_to_payload_v3};
|
||||
use reth_trie::{updates::TrieUpdates, HashedPostState};
|
||||
@ -2951,7 +2944,7 @@ mod tests {
|
||||
let (from_tree_tx, from_tree_rx) = unbounded_channel();
|
||||
|
||||
let header = chain_spec.genesis_header().clone();
|
||||
let header = SealedHeader::seal(header);
|
||||
let header = SealedHeader::seal_slow(header);
|
||||
let engine_api_tree_state = EngineApiTreeState::new(10, 10, header.num_hash());
|
||||
let canonical_in_memory_state = CanonicalInMemoryState::with_head(header, None, None);
|
||||
|
||||
@ -3023,12 +3016,7 @@ mod tests {
|
||||
self.persist_blocks(
|
||||
blocks
|
||||
.into_iter()
|
||||
.map(|b| {
|
||||
SealedBlockWithSenders::new_unchecked(
|
||||
(*b.block).clone(),
|
||||
b.senders().clone(),
|
||||
)
|
||||
})
|
||||
.map(|b| RecoveredBlock::new_sealed(b.block().clone(), b.senders().clone()))
|
||||
.collect(),
|
||||
);
|
||||
|
||||
@ -3049,7 +3037,7 @@ mod tests {
|
||||
|
||||
fn insert_block(
|
||||
&mut self,
|
||||
block: SealedBlockWithSenders,
|
||||
block: RecoveredBlock<reth_primitives::Block>,
|
||||
) -> Result<InsertPayloadOk, InsertBlockError<Block>> {
|
||||
let execution_outcome = self.block_builder.get_execution_outcome(block.clone());
|
||||
self.extend_execution_outcome([execution_outcome]);
|
||||
@ -3114,8 +3102,8 @@ mod tests {
|
||||
}
|
||||
}
|
||||
|
||||
async fn send_new_payload(&mut self, block: SealedBlockWithSenders) {
|
||||
let payload = block_to_payload_v3(block.block.clone());
|
||||
async fn send_new_payload(&mut self, block: RecoveredBlock<reth_primitives::Block>) {
|
||||
let payload = block_to_payload_v3(block.clone_sealed_block());
|
||||
self.tree
|
||||
.on_new_payload(
|
||||
payload.into(),
|
||||
@ -3129,7 +3117,7 @@ mod tests {
|
||||
|
||||
async fn insert_chain(
|
||||
&mut self,
|
||||
chain: impl IntoIterator<Item = SealedBlockWithSenders> + Clone,
|
||||
chain: impl IntoIterator<Item = RecoveredBlock<reth_primitives::Block>> + Clone,
|
||||
) {
|
||||
for block in chain.clone() {
|
||||
self.insert_block(block.clone()).unwrap();
|
||||
@ -3151,16 +3139,16 @@ mod tests {
|
||||
|
||||
async fn check_fork_chain_insertion(
|
||||
&mut self,
|
||||
chain: impl IntoIterator<Item = SealedBlockWithSenders> + Clone,
|
||||
chain: impl IntoIterator<Item = RecoveredBlock<reth_primitives::Block>> + Clone,
|
||||
) {
|
||||
for block in chain {
|
||||
self.check_fork_block_added(block.block.hash()).await;
|
||||
self.check_fork_block_added(block.hash()).await;
|
||||
}
|
||||
}
|
||||
|
||||
async fn check_canon_chain_insertion(
|
||||
&mut self,
|
||||
chain: impl IntoIterator<Item = SealedBlockWithSenders> + Clone,
|
||||
chain: impl IntoIterator<Item = RecoveredBlock<reth_primitives::Block>> + Clone,
|
||||
) {
|
||||
for block in chain.clone() {
|
||||
self.check_canon_block_added(block.hash()).await;
|
||||
@ -3186,33 +3174,35 @@ mod tests {
|
||||
block,
|
||||
_,
|
||||
)) => {
|
||||
assert!(block.hash() == expected_hash);
|
||||
assert_eq!(block.hash(), expected_hash);
|
||||
}
|
||||
_ => panic!("Unexpected event: {:#?}", event),
|
||||
}
|
||||
}
|
||||
|
||||
fn persist_blocks(&self, blocks: Vec<SealedBlockWithSenders>) {
|
||||
fn persist_blocks(&self, blocks: Vec<RecoveredBlock<reth_primitives::Block>>) {
|
||||
let mut block_data: Vec<(B256, Block)> = Vec::with_capacity(blocks.len());
|
||||
let mut headers_data: Vec<(B256, Header)> = Vec::with_capacity(blocks.len());
|
||||
|
||||
for block in &blocks {
|
||||
let unsealed_block = block.clone().unseal();
|
||||
block_data.push((block.hash(), unsealed_block.clone().block));
|
||||
headers_data.push((block.hash(), unsealed_block.header.clone()));
|
||||
block_data.push((block.hash(), block.clone_block()));
|
||||
headers_data.push((block.hash(), block.header().clone()));
|
||||
}
|
||||
|
||||
self.provider.extend_blocks(block_data);
|
||||
self.provider.extend_headers(headers_data);
|
||||
}
|
||||
|
||||
fn setup_range_insertion_for_valid_chain(&mut self, chain: Vec<SealedBlockWithSenders>) {
|
||||
fn setup_range_insertion_for_valid_chain(
|
||||
&mut self,
|
||||
chain: Vec<RecoveredBlock<reth_primitives::Block>>,
|
||||
) {
|
||||
self.setup_range_insertion_for_chain(chain, None)
|
||||
}
|
||||
|
||||
fn setup_range_insertion_for_invalid_chain(
|
||||
&mut self,
|
||||
chain: Vec<SealedBlockWithSenders>,
|
||||
chain: Vec<RecoveredBlock<reth_primitives::Block>>,
|
||||
index: usize,
|
||||
) {
|
||||
self.setup_range_insertion_for_chain(chain, Some(index))
|
||||
@ -3220,7 +3210,7 @@ mod tests {
|
||||
|
||||
fn setup_range_insertion_for_chain(
|
||||
&mut self,
|
||||
chain: Vec<SealedBlockWithSenders>,
|
||||
chain: Vec<RecoveredBlock<reth_primitives::Block>>,
|
||||
invalid_index: Option<usize>,
|
||||
) {
|
||||
// setting up execution outcomes for the chain, the blocks will be
|
||||
@ -3392,7 +3382,7 @@ mod tests {
|
||||
|
||||
// ensure block is buffered
|
||||
let buffered = test_harness.tree.state.buffer.block(&hash).unwrap();
|
||||
assert_eq!(buffered.block, sealed);
|
||||
assert_eq!(buffered.clone_sealed_block(), sealed);
|
||||
}
|
||||
|
||||
#[test]
|
||||
@ -3732,7 +3722,7 @@ mod tests {
|
||||
|
||||
for block in &chain_a {
|
||||
test_harness.tree.state.tree_state.insert_executed(ExecutedBlock {
|
||||
block: Arc::new(block.block.clone()),
|
||||
block: Arc::new(block.clone_sealed_block()),
|
||||
senders: Arc::new(block.senders().to_vec()),
|
||||
execution_output: Arc::new(ExecutionOutcome::default()),
|
||||
hashed_state: Arc::new(HashedPostState::default()),
|
||||
@ -3743,7 +3733,7 @@ mod tests {
|
||||
|
||||
for block in &chain_b {
|
||||
test_harness.tree.state.tree_state.insert_executed(ExecutedBlock {
|
||||
block: Arc::new(block.block.clone()),
|
||||
block: Arc::new(block.clone_sealed_block()),
|
||||
senders: Arc::new(block.senders().to_vec()),
|
||||
execution_output: Arc::new(ExecutionOutcome::default()),
|
||||
hashed_state: Arc::new(HashedPostState::default()),
|
||||
@ -3755,19 +3745,19 @@ mod tests {
|
||||
let mut expected_new = Vec::new();
|
||||
for block in &chain_b {
|
||||
// reorg to chain from block b
|
||||
let result = test_harness.tree.on_new_head(block.block.hash()).unwrap();
|
||||
let result = test_harness.tree.on_new_head(block.hash()).unwrap();
|
||||
assert_matches!(result, Some(NewCanonicalChain::Reorg { .. }));
|
||||
|
||||
expected_new.push(block);
|
||||
if let Some(NewCanonicalChain::Reorg { new, old }) = result {
|
||||
assert_eq!(new.len(), expected_new.len());
|
||||
for (index, block) in expected_new.iter().enumerate() {
|
||||
assert_eq!(new[index].block.hash(), block.block.hash());
|
||||
assert_eq!(new[index].block.hash(), block.hash());
|
||||
}
|
||||
|
||||
assert_eq!(old.len(), chain_a.len());
|
||||
for (index, block) in chain_a.iter().enumerate() {
|
||||
assert_eq!(old[index].block.hash(), block.block.hash());
|
||||
assert_eq!(old[index].block.hash(), block.hash());
|
||||
}
|
||||
}
|
||||
|
||||
@ -3936,9 +3926,7 @@ mod tests {
|
||||
|
||||
let backfill_tip_block = main_chain[(backfill_finished_block_number - 1) as usize].clone();
|
||||
// add block to mock provider to enable persistence clean up.
|
||||
test_harness
|
||||
.provider
|
||||
.add_block(backfill_tip_block.hash(), backfill_tip_block.block.unseal());
|
||||
test_harness.provider.add_block(backfill_tip_block.hash(), backfill_tip_block.into_block());
|
||||
test_harness.tree.on_engine_message(FromEngine::Event(backfill_finished)).unwrap();
|
||||
|
||||
let event = test_harness.from_tree_rx.recv().await.unwrap();
|
||||
@ -4154,7 +4142,7 @@ mod tests {
|
||||
|
||||
// extend base chain
|
||||
let extension_chain = test_harness.block_builder.create_fork(old_head, 5);
|
||||
let fork_block = extension_chain.last().unwrap().block.clone();
|
||||
let fork_block = extension_chain.last().unwrap().clone_sealed_block();
|
||||
|
||||
test_harness.setup_range_insertion_for_valid_chain(extension_chain.clone());
|
||||
test_harness.insert_chain(extension_chain).await;
|
||||
@ -4252,7 +4240,7 @@ mod tests {
|
||||
|
||||
// extend base chain
|
||||
let extension_chain = test_harness.block_builder.create_fork(old_head, 5);
|
||||
let fork_block = extension_chain.last().unwrap().block.clone();
|
||||
let fork_block = extension_chain.last().unwrap().clone_sealed_block();
|
||||
test_harness.insert_chain(extension_chain).await;
|
||||
|
||||
// fcu to old_head
|
||||
@ -4322,7 +4310,7 @@ mod tests {
|
||||
test_harness.send_new_payload(block.clone()).await;
|
||||
|
||||
if index < side_chain.len() - invalid_index - 1 {
|
||||
test_harness.send_fcu(block.block.hash(), ForkchoiceStatus::Valid).await;
|
||||
test_harness.send_fcu(block.hash(), ForkchoiceStatus::Valid).await;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -13,6 +13,7 @@ workspace = true
|
||||
[dependencies]
|
||||
# reth
|
||||
reth-primitives.workspace = true
|
||||
reth-primitives-traits.workspace = true
|
||||
reth-errors.workspace = true
|
||||
reth-consensus-common.workspace = true
|
||||
reth-fs-util.workspace = true
|
||||
|
||||
@ -19,9 +19,9 @@ use reth_evm::{
|
||||
};
|
||||
use reth_payload_validator::ExecutionPayloadValidator;
|
||||
use reth_primitives::{
|
||||
proofs, transaction::SignedTransactionIntoRecoveredExt, Block, BlockBody, BlockExt, Receipt,
|
||||
Receipts,
|
||||
proofs, transaction::SignedTransactionIntoRecoveredExt, Block, BlockBody, Receipt, Receipts,
|
||||
};
|
||||
use reth_primitives_traits::block::Block as _;
|
||||
use reth_provider::{BlockReader, ExecutionOutcome, ProviderError, StateProviderFactory};
|
||||
use reth_revm::{
|
||||
database::StateProviderDatabase,
|
||||
|
||||
Reference in New Issue
Block a user