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

@ -10,7 +10,7 @@ use reth_evm::execute::{
BatchExecutor, BlockExecutionError, BlockExecutionOutput, BlockExecutorProvider, Executor,
};
use reth_node_api::{Block as _, BlockBody as _, NodePrimitives};
use reth_primitives::{BlockExt, BlockWithSenders, Receipt};
use reth_primitives::{Receipt, RecoveredBlock};
use reth_primitives_traits::{format_gas_throughput, SignedTransaction};
use reth_provider::{
BlockReader, Chain, HeaderProvider, ProviderError, StateProviderFactory, TransactionVariant,
@ -107,10 +107,9 @@ where
let execute_start = Instant::now();
// Unseal the block for execution
let (block, senders) = block.split();
let (header, body) = block.split();
let (unsealed_header, hash) = header.split();
let block = P::Block::new(unsealed_header, body).with_senders_unchecked(senders);
let (block, senders) = block.split_sealed();
let (header, body) = block.split_sealed_header_body();
let block = P::Block::new_sealed(header, body).with_senders(senders);
executor.execute_and_verify_one(&block)?;
execution_duration += execute_start.elapsed();
@ -118,7 +117,7 @@ where
// TODO(alexey): report gas metrics using `block.header.gas_used`
// Seal the block back and save it
blocks.push(block.seal_unchecked(hash));
blocks.push(block);
// Check if we should commit now
let bundle_size_hint = executor.size_hint().unwrap_or_default() as u64;
@ -151,7 +150,7 @@ where
/// Single block Backfill job started for a specific range.
///
/// It implements [`Iterator`] which executes a block each time the
/// iterator is advanced and yields ([`BlockWithSenders`], [`BlockExecutionOutput`])
/// iterator is advanced and yields ([`RecoveredBlock`], [`BlockExecutionOutput`])
#[derive(Debug, Clone)]
pub struct SingleBlockBackfillJob<E, P> {
pub(crate) executor: E,
@ -166,7 +165,7 @@ where
P: HeaderProvider + BlockReader + StateProviderFactory,
{
type Item = BackfillJobResult<(
BlockWithSenders<P::Block>,
RecoveredBlock<P::Block>,
BlockExecutionOutput<<E::Primitives as NodePrimitives>::Receipt>,
)>;
@ -183,7 +182,11 @@ where
/// Converts the single block backfill job into a stream.
pub fn into_stream(
self,
) -> StreamBackfillJob<E, P, (BlockWithSenders, BlockExecutionOutput<Receipt>)> {
) -> StreamBackfillJob<
E,
P,
(RecoveredBlock<reth_primitives::Block>, BlockExecutionOutput<Receipt>),
> {
self.into()
}
@ -192,7 +195,7 @@ where
&self,
block_number: u64,
) -> BackfillJobResult<(
BlockWithSenders<P::Block>,
RecoveredBlock<P::Block>,
BlockExecutionOutput<<E::Primitives as NodePrimitives>::Receipt>,
)> {
// Fetch the block with senders for execution.
@ -206,7 +209,7 @@ where
self.provider.history_by_block_number(block_number.saturating_sub(1))?,
));
trace!(target: "exex::backfill", number = block_number, txs = block_with_senders.block.body().transactions().len(), "Executing block");
trace!(target: "exex::backfill", number = block_number, txs = block_with_senders.body().transaction_count(), "Executing block");
let block_execution_output = executor.execute(&block_with_senders)?;
@ -310,8 +313,7 @@ mod tests {
let (block, mut execution_output) = res?;
execution_output.state.reverts.sort();
let sealed_block_with_senders = blocks_and_execution_outcomes[i].0.clone();
let expected_block = sealed_block_with_senders.unseal();
let expected_block = blocks_and_execution_outcomes[i].0.clone();
let expected_output = &blocks_and_execution_outcomes[i].1;
assert_eq!(block, expected_block);

View File

@ -7,7 +7,7 @@ use futures::{
};
use reth_evm::execute::{BlockExecutionError, BlockExecutionOutput, BlockExecutorProvider};
use reth_node_api::NodePrimitives;
use reth_primitives::{BlockWithSenders, EthPrimitives};
use reth_primitives::{EthPrimitives, RecoveredBlock};
use reth_provider::{BlockReader, Chain, StateProviderFactory};
use reth_prune_types::PruneModes;
use reth_stages_api::ExecutionStageThresholds;
@ -38,7 +38,7 @@ struct BackfillTaskOutput<T> {
type BackfillTasks<T> = FuturesOrdered<JoinHandle<BackfillTaskOutput<T>>>;
type SingleBlockStreamItem<N = EthPrimitives> = (
BlockWithSenders<<N as NodePrimitives>::Block>,
RecoveredBlock<<N as NodePrimitives>::Block>,
BlockExecutionOutput<<N as NodePrimitives>::Receipt>,
);
type BatchBlockStreamItem<N = EthPrimitives> = Chain<N>;
@ -278,8 +278,7 @@ mod tests {
// execute first block
let (block, mut execution_output) = backfill_stream.next().await.unwrap().unwrap();
execution_output.state.reverts.sort();
let sealed_block_with_senders = blocks_and_execution_outcomes[0].0.clone();
let expected_block = sealed_block_with_senders.unseal();
let expected_block = blocks_and_execution_outcomes[0].0.clone();
let expected_output = &blocks_and_execution_outcomes[0].1;
assert_eq!(block, expected_block);
assert_eq!(&execution_output, expected_output);

View File

@ -1,6 +1,6 @@
use std::sync::Arc;
use alloy_consensus::{constants::ETH_TO_WEI, Header, TxEip2930};
use alloy_consensus::{constants::ETH_TO_WEI, BlockHeader, Header, TxEip2930};
use alloy_genesis::{Genesis, GenesisAccount};
use alloy_primitives::{b256, Address, TxKind, U256};
use eyre::OptionExt;
@ -8,9 +8,8 @@ use reth_chainspec::{ChainSpec, ChainSpecBuilder, EthereumHardfork, MAINNET, MIN
use reth_evm::execute::{BatchExecutor, BlockExecutionOutput, BlockExecutorProvider, Executor};
use reth_evm_ethereum::execute::EthExecutorProvider;
use reth_node_api::FullNodePrimitives;
use reth_primitives::{
Block, BlockBody, BlockExt, BlockWithSenders, Receipt, SealedBlockWithSenders, Transaction,
};
use reth_primitives::{Block, BlockBody, Receipt, RecoveredBlock, Transaction};
use reth_primitives_traits::Block as _;
use reth_provider::{
providers::ProviderNodeTypes, BlockWriter as _, ExecutionOutcome, LatestStateProviderRef,
ProviderFactory,
@ -53,7 +52,7 @@ pub(crate) fn chain_spec(address: Address) -> Arc<ChainSpec> {
pub(crate) fn execute_block_and_commit_to_database<N>(
provider_factory: &ProviderFactory<N>,
chain_spec: Arc<ChainSpec>,
block: &BlockWithSenders,
block: &RecoveredBlock<reth_primitives::Block>,
) -> eyre::Result<BlockExecutionOutput<Receipt>>
where
N: ProviderNodeTypes<
@ -73,13 +72,12 @@ where
block_execution_output.state.reverts.sort();
// Convert the block execution output to an execution outcome for committing to the database
let execution_outcome = to_execution_outcome(block.number, &block_execution_output);
let execution_outcome = to_execution_outcome(block.number(), &block_execution_output);
// Commit the block's execution outcome to the database
let provider_rw = provider_factory.provider_rw()?;
let block = block.clone().seal_slow();
provider_rw.append_blocks_with_state(
vec![block],
vec![block.clone()],
&execution_outcome,
Default::default(),
Default::default(),
@ -92,7 +90,8 @@ where
fn blocks(
chain_spec: Arc<ChainSpec>,
key_pair: Keypair,
) -> eyre::Result<(BlockWithSenders, BlockWithSenders)> {
) -> eyre::Result<(RecoveredBlock<reth_primitives::Block>, RecoveredBlock<reth_primitives::Block>)>
{
// First block has a transaction that transfers some ETH to zero address
let block1 = Block {
header: Header {
@ -128,7 +127,7 @@ fn blocks(
// Second block resends the same transaction with increased nonce
let block2 = Block {
header: Header {
parent_hash: block1.header.hash_slow(),
parent_hash: block1.hash(),
receipts_root: b256!(
"d3a6acf9a244d78b33831df95d472c4128ea85bf079a1d41e32ed0b7d2244c9e"
),
@ -164,7 +163,7 @@ pub(crate) fn blocks_and_execution_outputs<N>(
provider_factory: ProviderFactory<N>,
chain_spec: Arc<ChainSpec>,
key_pair: Keypair,
) -> eyre::Result<Vec<(SealedBlockWithSenders, BlockExecutionOutput<Receipt>)>>
) -> eyre::Result<Vec<(RecoveredBlock<reth_primitives::Block>, BlockExecutionOutput<Receipt>)>>
where
N: ProviderNodeTypes<
Primitives: FullNodePrimitives<
@ -181,9 +180,6 @@ where
let block_output2 =
execute_block_and_commit_to_database(&provider_factory, chain_spec, &block2)?;
let block1 = block1.seal_slow();
let block2 = block2.seal_slow();
Ok(vec![(block1, block_output1), (block2, block_output2)])
}
@ -191,7 +187,7 @@ pub(crate) fn blocks_and_execution_outcome<N>(
provider_factory: ProviderFactory<N>,
chain_spec: Arc<ChainSpec>,
key_pair: Keypair,
) -> eyre::Result<(Vec<SealedBlockWithSenders>, ExecutionOutcome)>
) -> eyre::Result<(Vec<RecoveredBlock<reth_primitives::Block>>, ExecutionOutcome)>
where
N: ProviderNodeTypes,
N::Primitives:
@ -207,9 +203,6 @@ where
let mut execution_outcome = executor.execute_and_verify_batch(vec![&block1, &block2])?;
execution_outcome.state_mut().reverts.sort();
let block1 = block1.seal_slow();
let block2 = block2.seal_slow();
// Commit the block's execution outcome to the database
let provider_rw = provider_factory.provider_rw()?;
provider_rw.append_blocks_with_state(

View File

@ -664,7 +664,7 @@ mod tests {
use reth_db_common::init::init_genesis;
use reth_evm::test_utils::MockExecutorProvider;
use reth_evm_ethereum::execute::EthExecutorProvider;
use reth_primitives::SealedBlockWithSenders;
use reth_primitives::RecoveredBlock;
use reth_provider::{
providers::BlockchainProvider, test_utils::create_test_provider_factory, BlockReader,
BlockWriter, Chain, DatabaseProviderFactory, StorageLocation, TransactionVariant,
@ -766,9 +766,9 @@ mod tests {
ExExManager::new((), vec![exex_handle], 10, wal, empty_finalized_header_stream());
// Define the notification for testing
let mut block1: SealedBlockWithSenders = Default::default();
block1.block.set_hash(B256::new([0x01; 32]));
block1.block.set_block_number(10);
let mut block1: RecoveredBlock<reth_primitives::Block> = Default::default();
block1.set_hash(B256::new([0x01; 32]));
block1.set_block_number(10);
let notification1 = ExExNotification::ChainCommitted {
new: Arc::new(Chain::new(vec![block1.clone()], Default::default(), Default::default())),
@ -784,9 +784,9 @@ mod tests {
assert_eq!(exex_manager.next_id, 1);
// Push another notification
let mut block2: SealedBlockWithSenders = Default::default();
block2.block.set_hash(B256::new([0x02; 32]));
block2.block.set_block_number(20);
let mut block2: RecoveredBlock<reth_primitives::Block> = Default::default();
block2.set_hash(B256::new([0x02; 32]));
block2.set_block_number(20);
let notification2 = ExExNotification::ChainCommitted {
new: Arc::new(Chain::new(vec![block2.clone()], Default::default(), Default::default())),
@ -827,7 +827,7 @@ mod tests {
);
// Push some notifications to fill part of the buffer
let mut block1: SealedBlockWithSenders = Default::default();
let mut block1: RecoveredBlock<reth_primitives::Block> = Default::default();
block1.set_hash(B256::new([0x01; 32]));
block1.set_block_number(10);
@ -1116,13 +1116,13 @@ mod tests {
assert_eq!(exex_handle.next_notification_id, 0);
// Setup two blocks for the chain commit notification
let mut block1: SealedBlockWithSenders = Default::default();
block1.block.set_hash(B256::new([0x01; 32]));
block1.block.set_block_number(10);
let mut block1: RecoveredBlock<reth_primitives::Block> = Default::default();
block1.set_hash(B256::new([0x01; 32]));
block1.set_block_number(10);
let mut block2: SealedBlockWithSenders = Default::default();
block2.block.set_hash(B256::new([0x02; 32]));
block2.block.set_block_number(11);
let mut block2: RecoveredBlock<reth_primitives::Block> = Default::default();
block2.set_hash(B256::new([0x02; 32]));
block2.set_block_number(11);
// Setup a notification
let notification = ExExNotification::ChainCommitted {
@ -1169,9 +1169,9 @@ mod tests {
// Set finished_height to a value higher than the block tip
exex_handle.finished_height = Some(BlockNumHash::new(15, B256::random()));
let mut block1: SealedBlockWithSenders = Default::default();
block1.block.set_hash(B256::new([0x01; 32]));
block1.block.set_block_number(10);
let mut block1: RecoveredBlock<reth_primitives::Block> = Default::default();
block1.set_hash(B256::new([0x01; 32]));
block1.set_block_number(10);
let notification = ExExNotification::ChainCommitted {
new: Arc::new(Chain::new(vec![block1.clone()], Default::default(), Default::default())),
@ -1300,7 +1300,7 @@ mod tests {
genesis_block.number + 1,
BlockParams { parent: Some(genesis_hash), ..Default::default() },
)
.seal_with_senders::<reth_primitives::Block>()
.try_recover()
.unwrap();
let provider_rw = provider_factory.database_provider_rw().unwrap();
provider_rw.insert_block(block.clone(), StorageLocation::Database).unwrap();

View File

@ -435,16 +435,16 @@ where
#[cfg(test)]
mod tests {
use crate::Wal;
use super::*;
use crate::Wal;
use alloy_consensus::Header;
use alloy_eips::BlockNumHash;
use eyre::OptionExt;
use futures::StreamExt;
use reth_db_common::init::init_genesis;
use reth_evm_ethereum::execute::EthExecutorProvider;
use reth_primitives::{Block, BlockExt};
use reth_primitives::Block;
use reth_primitives_traits::Block as _;
use reth_provider::{
providers::BlockchainProvider, test_utils::create_test_provider_factory, BlockWriter,
Chain, DatabaseProviderFactory, StorageLocation,
@ -473,10 +473,8 @@ mod tests {
BlockParams { parent: Some(genesis_hash), tx_count: Some(0), ..Default::default() },
);
let provider_rw = provider_factory.provider_rw()?;
provider_rw.insert_block(
node_head_block.clone().seal_with_senders().ok_or_eyre("failed to recover senders")?,
StorageLocation::Database,
)?;
provider_rw
.insert_block(node_head_block.clone().try_recover()?, StorageLocation::Database)?;
provider_rw.commit()?;
let node_head = Head {
@ -494,8 +492,7 @@ mod tests {
node_head.number + 1,
BlockParams { parent: Some(node_head.hash), ..Default::default() },
)
.seal_with_senders()
.ok_or_eyre("failed to recover senders")?],
.try_recover()?],
Default::default(),
None,
)),
@ -565,8 +562,7 @@ mod tests {
..Default::default()
}
.seal_slow()
.seal_with_senders()
.ok_or_eyre("failed to recover senders")?],
.try_recover()?],
Default::default(),
None,
)),
@ -611,8 +607,7 @@ mod tests {
genesis_block.number + 1,
BlockParams { parent: Some(genesis_hash), tx_count: Some(0), ..Default::default() },
)
.seal_with_senders::<reth_primitives::Block>()
.ok_or_eyre("failed to recover senders")?;
.try_recover()?;
let node_head = Head {
number: node_head_block.number,
hash: node_head_block.hash(),
@ -638,10 +633,7 @@ mod tests {
let exex_head = ExExHead { block: exex_head_block.num_hash() };
let exex_head_notification = ExExNotification::ChainCommitted {
new: Arc::new(Chain::new(
vec![exex_head_block
.clone()
.seal_with_senders()
.ok_or_eyre("failed to recover senders")?],
vec![exex_head_block.clone().try_recover()?],
Default::default(),
None,
)),
@ -655,8 +647,7 @@ mod tests {
node_head.number + 1,
BlockParams { parent: Some(node_head.hash), ..Default::default() },
)
.seal_with_senders()
.ok_or_eyre("failed to recover senders")?],
.try_recover()?],
Default::default(),
None,
)),
@ -713,10 +704,7 @@ mod tests {
);
let exex_head_notification = ExExNotification::ChainCommitted {
new: Arc::new(Chain::new(
vec![exex_head_block
.clone()
.seal_with_senders()
.ok_or_eyre("failed to recover senders")?],
vec![exex_head_block.clone().try_recover()?],
Default::default(),
None,
)),
@ -736,8 +724,7 @@ mod tests {
genesis_block.number + 1,
BlockParams { parent: Some(genesis_hash), ..Default::default() },
)
.seal_with_senders()
.ok_or_eyre("failed to recover senders")?],
.try_recover()?],
Default::default(),
None,
)),

View File

@ -233,18 +233,15 @@ where
#[cfg(test)]
mod tests {
use std::sync::Arc;
use crate::wal::{cache::CachedBlock, Wal};
use alloy_primitives::B256;
use eyre::OptionExt;
use itertools::Itertools;
use reth_exex_types::ExExNotification;
use reth_provider::Chain;
use reth_testing_utils::generators::{
self, random_block, random_block_range, BlockParams, BlockRangeParams,
};
use crate::wal::{cache::CachedBlock, Wal};
use std::sync::Arc;
fn read_notifications(wal: &Wal) -> eyre::Result<Vec<ExExNotification>> {
wal.inner.storage.files_range()?.map_or(Ok(Vec::new()), |range| {
@ -279,26 +276,20 @@ mod tests {
// Create 4 canonical blocks and one reorged block with number 2
let blocks = random_block_range(&mut rng, 0..=3, BlockRangeParams::default())
.into_iter()
.map(|block| {
block
.seal_with_senders::<reth_primitives::Block>()
.ok_or_eyre("failed to recover senders")
})
.collect::<eyre::Result<Vec<_>>>()?;
.map(|block| block.try_recover())
.collect::<Result<Vec<_>, _>>()?;
let block_1_reorged = random_block(
&mut rng,
1,
BlockParams { parent: Some(blocks[0].hash()), ..Default::default() },
)
.seal_with_senders::<reth_primitives::Block>()
.ok_or_eyre("failed to recover senders")?;
.try_recover()?;
let block_2_reorged = random_block(
&mut rng,
2,
BlockParams { parent: Some(blocks[1].hash()), ..Default::default() },
)
.seal_with_senders::<reth_primitives::Block>()
.ok_or_eyre("failed to recover senders")?;
.try_recover()?;
// Create notifications for the above blocks.
// 1. Committed notification for blocks with number 0 and 1

View File

@ -178,14 +178,11 @@ where
#[cfg(test)]
mod tests {
use std::{fs::File, sync::Arc};
use eyre::OptionExt;
use super::Storage;
use reth_exex_types::ExExNotification;
use reth_provider::Chain;
use reth_testing_utils::generators::{self, random_block};
use super::Storage;
use std::{fs::File, sync::Arc};
#[test]
fn test_roundtrip() -> eyre::Result<()> {
@ -194,12 +191,8 @@ mod tests {
let temp_dir = tempfile::tempdir()?;
let storage: Storage = Storage::new(&temp_dir)?;
let old_block = random_block(&mut rng, 0, Default::default())
.seal_with_senders()
.ok_or_eyre("failed to recover senders")?;
let new_block = random_block(&mut rng, 0, Default::default())
.seal_with_senders()
.ok_or_eyre("failed to recover senders")?;
let old_block = random_block(&mut rng, 0, Default::default()).try_recover()?;
let new_block = random_block(&mut rng, 0, Default::default()).try_recover()?;
let notification = ExExNotification::ChainReorged {
new: Arc::new(Chain::new(vec![new_block], Default::default(), None)),

View File

@ -27,6 +27,7 @@ reth-node-builder = { workspace = true, features = ["test-utils"] }
reth-node-ethereum.workspace = true
reth-payload-builder.workspace = true
reth-primitives.workspace = true
reth-primitives-traits.workspace = true
reth-provider = { workspace = true, features = ["test-utils"] }
reth-tasks.workspace = true
reth-transaction-pool = { workspace = true, features = ["test-utils"] }

View File

@ -45,12 +45,14 @@ use reth_node_ethereum::{
EthEngineTypes, EthEvmConfig,
};
use reth_payload_builder::noop::NoopPayloadBuilderService;
use reth_primitives::{BlockExt, EthPrimitives, Head, SealedBlockWithSenders, TransactionSigned};
use reth_provider::{providers::StaticFileProvider, BlockReader, EthStorage, ProviderFactory};
use reth_primitives::{EthPrimitives, Head, RecoveredBlock, TransactionSigned};
use reth_primitives_traits::Block as _;
use reth_provider::{
providers::{BlockchainProvider, StaticFileProvider},
BlockReader, EthStorage, ProviderFactory,
};
use reth_tasks::TaskManager;
use reth_transaction_pool::test_utils::{testing_pool, TestPool};
use reth_provider::providers::BlockchainProvider;
use tempfile::TempDir;
use thiserror::Error;
use tokio::sync::mpsc::{Sender, UnboundedReceiver};
@ -185,7 +187,7 @@ pub type TestExExContext = ExExContext<Adapter>;
#[derive(Debug)]
pub struct TestExExHandle {
/// Genesis block that was inserted into the storage
pub genesis: SealedBlockWithSenders,
pub genesis: RecoveredBlock<reth_primitives::Block>,
/// Provider Factory for accessing the emphemeral storage of the host node
pub provider_factory: ProviderFactory<NodeTypesWithDBAdapter<TestNode, TmpDB>>,
/// Channel for receiving events from the Execution Extension
@ -304,8 +306,7 @@ pub async fn test_exex_context_with_chain_spec(
.block_by_hash(genesis_hash)?
.ok_or_else(|| eyre::eyre!("genesis block not found"))?
.seal_slow()
.seal_with_senders::<reth_primitives::Block>()
.ok_or_else(|| eyre::eyre!("failed to recover senders"))?;
.try_recover()?;
let head = Head {
number: genesis.number,

View File

@ -174,7 +174,7 @@ pub(super) mod serde_bincode_compat {
use arbitrary::Arbitrary;
use rand::Rng;
use reth_execution_types::Chain;
use reth_primitives::SealedBlockWithSenders;
use reth_primitives::RecoveredBlock;
use serde::{Deserialize, Serialize};
use serde_with::serde_as;
use std::sync::Arc;
@ -193,18 +193,14 @@ pub(super) mod serde_bincode_compat {
let data = Data {
notification: ExExNotification::ChainReorged {
old: Arc::new(Chain::new(
vec![SealedBlockWithSenders::arbitrary(&mut arbitrary::Unstructured::new(
&bytes,
))
.unwrap()],
vec![RecoveredBlock::arbitrary(&mut arbitrary::Unstructured::new(&bytes))
.unwrap()],
Default::default(),
None,
)),
new: Arc::new(Chain::new(
vec![SealedBlockWithSenders::arbitrary(&mut arbitrary::Unstructured::new(
&bytes,
))
.unwrap()],
vec![RecoveredBlock::arbitrary(&mut arbitrary::Unstructured::new(&bytes))
.unwrap()],
Default::default(),
None,
)),