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