feat: relax provider bounds (#12857)

This commit is contained in:
Arsenii Kulikov
2024-11-26 20:26:36 +04:00
committed by GitHub
parent d51b347c81
commit 2ce741709f
37 changed files with 365 additions and 236 deletions

2
Cargo.lock generated
View File

@ -7270,7 +7270,6 @@ dependencies = [
"reth-payload-primitives",
"reth-payload-validator",
"reth-primitives",
"reth-primitives-traits",
"reth-provider",
"reth-prune",
"reth-prune-types",
@ -7565,6 +7564,7 @@ dependencies = [
name = "reth-execution-types"
version = "1.1.2"
dependencies = [
"alloy-consensus",
"alloy-eips",
"alloy-primitives",
"arbitrary",

View File

@ -25,7 +25,7 @@ use reth_fs_util as fs;
use reth_node_api::{BlockTy, EngineApiMessageVersion, PayloadBuilderAttributes};
use reth_node_ethereum::{EthEvmConfig, EthExecutorProvider};
use reth_primitives::{
BlobTransaction, BlockExt, PooledTransactionsElement, SealedBlock, SealedBlockWithSenders,
BlobTransaction, BlockExt, PooledTransactionsElement, SealedBlockFor, SealedBlockWithSenders,
SealedHeader, Transaction, TransactionSigned,
};
use reth_provider::{
@ -90,7 +90,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
fn lookup_best_block<N: ProviderNodeTypes<ChainSpec = C::ChainSpec>>(
&self,
factory: ProviderFactory<N>,
) -> RethResult<Arc<SealedBlock>> {
) -> RethResult<Arc<SealedBlockFor<BlockTy<N>>>> {
let provider = factory.provider()?;
let best_number =

View File

@ -19,6 +19,7 @@ use reth_evm::execute::{BlockExecutorProvider, Executor};
use reth_execution_types::ExecutionOutcome;
use reth_network::{BlockDownloaderProvider, NetworkHandle};
use reth_network_api::NetworkInfo;
use reth_node_api::{BlockTy, NodePrimitives};
use reth_node_ethereum::EthExecutorProvider;
use reth_primitives::BlockExt;
use reth_provider::{
@ -56,7 +57,12 @@ pub struct Command<C: ChainSpecParser> {
}
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
async fn build_network<N: ProviderNodeTypes<ChainSpec = C::ChainSpec>>(
async fn build_network<
N: ProviderNodeTypes<
ChainSpec = C::ChainSpec,
Primitives: NodePrimitives<Block = reth_primitives::Block>,
>,
>(
&self,
config: &Config,
task_executor: TaskExecutor,
@ -143,7 +149,7 @@ impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
(
&block
.clone()
.unseal()
.unseal::<BlockTy<N>>()
.with_recovered_senders()
.ok_or(BlockValidationError::SenderRecoveryError)?,
merkle_block_td + block.difficulty,

View File

@ -17,7 +17,7 @@ use reth_evm::execute::{BatchExecutor, BlockExecutorProvider};
use reth_network::{BlockDownloaderProvider, NetworkHandle};
use reth_network_api::NetworkInfo;
use reth_network_p2p::full_block::FullBlockClient;
use reth_node_api::BlockTy;
use reth_node_api::{BlockTy, NodePrimitives};
use reth_node_ethereum::EthExecutorProvider;
use reth_provider::{
providers::ProviderNodeTypes, BlockNumReader, BlockWriter, ChainSpecProvider,
@ -56,7 +56,12 @@ pub struct Command<C: ChainSpecParser> {
}
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
async fn build_network<N: ProviderNodeTypes<ChainSpec = C::ChainSpec>>(
async fn build_network<
N: ProviderNodeTypes<
ChainSpec = C::ChainSpec,
Primitives: NodePrimitives<Block = reth_primitives::Block>,
>,
>(
&self,
config: &Config,
task_executor: TaskExecutor,

View File

@ -18,7 +18,7 @@ use reth_engine_util::engine_store::{EngineMessageStore, StoredEngineApiMessage}
use reth_fs_util as fs;
use reth_network::{BlockDownloaderProvider, NetworkHandle};
use reth_network_api::NetworkInfo;
use reth_node_api::{EngineApiMessageVersion, NodeTypesWithDBAdapter};
use reth_node_api::{EngineApiMessageVersion, NodePrimitives, NodeTypesWithDBAdapter};
use reth_node_ethereum::{EthEngineTypes, EthEvmConfig, EthExecutorProvider};
use reth_payload_builder::{PayloadBuilderHandle, PayloadBuilderService};
use reth_provider::{
@ -55,7 +55,12 @@ pub struct Command<C: ChainSpecParser> {
}
impl<C: ChainSpecParser<ChainSpec = ChainSpec>> Command<C> {
async fn build_network<N: ProviderNodeTypes<ChainSpec = C::ChainSpec>>(
async fn build_network<
N: ProviderNodeTypes<
ChainSpec = C::ChainSpec,
Primitives: NodePrimitives<Block = reth_primitives::Block>,
>,
>(
&self,
config: &Config,
task_executor: TaskExecutor,

View File

@ -5,23 +5,15 @@ use reth_consensus::Consensus;
use reth_db::{static_file::BlockHashMask, tables};
use reth_db_api::{cursor::DbCursorRO, transaction::DbTx};
use reth_node_types::NodeTypesWithDB;
use reth_primitives::{EthPrimitives, StaticFileSegment};
use reth_primitives::StaticFileSegment;
use reth_provider::{
providers::{NodeTypesForProvider, ProviderNodeTypes},
ChainStateBlockReader, ChainStateBlockWriter, ProviderFactory, StaticFileProviderFactory,
StatsReader,
providers::ProviderNodeTypes, ChainStateBlockReader, ChainStateBlockWriter, ProviderFactory,
StaticFileProviderFactory, StatsReader,
};
use reth_storage_errors::provider::ProviderResult;
use std::{collections::BTreeMap, sync::Arc};
/// A helper trait with requirements for [`ProviderNodeTypes`] to be used within [`TreeExternals`].
pub trait NodeTypesForTree: NodeTypesForProvider<Primitives = EthPrimitives> {}
impl<T> NodeTypesForTree for T where T: NodeTypesForProvider<Primitives = EthPrimitives> {}
/// A helper trait with requirements for [`ProviderNodeTypes`] to be used within [`TreeExternals`].
pub trait TreeNodeTypes: ProviderNodeTypes + NodeTypesForTree {}
impl<T> TreeNodeTypes for T where T: ProviderNodeTypes + NodeTypesForTree {}
pub use reth_provider::providers::{NodeTypesForTree, TreeNodeTypes};
/// A container for external components.
///

View File

@ -4,18 +4,18 @@ use crate::{
CanonStateNotification, CanonStateNotificationSender, CanonStateNotifications,
ChainInfoTracker, MemoryOverlayStateProvider,
};
use alloy_consensus::Header;
use alloy_eips::{BlockHashOrNumber, BlockNumHash};
use alloy_consensus::BlockHeader;
use alloy_eips::{eip2718::Encodable2718, BlockHashOrNumber, BlockNumHash};
use alloy_primitives::{map::HashMap, Address, TxHash, B256};
use parking_lot::RwLock;
use reth_chainspec::ChainInfo;
use reth_execution_types::{Chain, ExecutionOutcome};
use reth_metrics::{metrics::Gauge, Metrics};
use reth_primitives::{
BlockWithSenders, NodePrimitives, Receipts, SealedBlock, SealedBlockWithSenders, SealedHeader,
TransactionMeta, TransactionSigned,
BlockWithSenders, HeaderExt, NodePrimitives, Receipts, SealedBlock, SealedBlockFor,
SealedBlockWithSenders, SealedHeader, TransactionMeta,
};
use reth_primitives_traits::BlockBody as _;
use reth_primitives_traits::{Block, BlockBody as _, SignedTransaction};
use reth_storage_api::StateProviderBox;
use reth_trie::{updates::TrieUpdates, HashedPostState};
use std::{collections::BTreeMap, sync::Arc, time::Instant};
@ -159,10 +159,8 @@ impl<N: NodePrimitives> CanonicalInMemoryStateInner<N> {
}
}
type PendingBlockAndReceipts<N> = (
SealedBlock<reth_primitives_traits::HeaderTy<N>, reth_primitives_traits::BodyTy<N>>,
Vec<reth_primitives_traits::ReceiptTy<N>>,
);
type PendingBlockAndReceipts<N> =
(SealedBlockFor<<N as NodePrimitives>::Block>, Vec<reth_primitives_traits::ReceiptTy<N>>);
/// This type is responsible for providing the blocks, receipts, and state for
/// all canonical blocks not on disk yet and keeps track of the block range that
@ -172,13 +170,7 @@ pub struct CanonicalInMemoryState<N: NodePrimitives = reth_primitives::EthPrimit
pub(crate) inner: Arc<CanonicalInMemoryStateInner<N>>,
}
impl<N> CanonicalInMemoryState<N>
where
N: NodePrimitives<
BlockHeader = alloy_consensus::Header,
BlockBody = reth_primitives::BlockBody,
>,
{
impl<N: NodePrimitives> CanonicalInMemoryState<N> {
/// Create a new in-memory state with the given blocks, numbers, pending state, and optional
/// finalized header.
pub fn new(
@ -250,7 +242,7 @@ where
/// Note: This assumes that the parent block of the pending block is canonical.
pub fn set_pending_block(&self, pending: ExecutedBlock<N>) {
// fetch the state of the pending block's parent block
let parent = self.state_by_hash(pending.block().parent_hash);
let parent = self.state_by_hash(pending.block().parent_hash());
let pending = BlockState::with_parent(pending, parent);
self.inner.in_memory_state.pending.send_modify(|p| {
p.replace(pending);
@ -274,14 +266,14 @@ where
// we first remove the blocks from the reorged chain
for block in reorged {
let hash = block.block().hash();
let number = block.block().number;
let number = block.block().number();
blocks.remove(&hash);
numbers.remove(&number);
}
// insert the new blocks
for block in new_blocks {
let parent = blocks.get(&block.block().parent_hash).cloned();
let parent = blocks.get(&block.block().parent_hash()).cloned();
let block_state = BlockState::with_parent(block, parent);
let hash = block_state.hash();
let number = block_state.number();
@ -341,16 +333,16 @@ where
// height)
let mut old_blocks = blocks
.drain()
.filter(|(_, b)| b.block_ref().block().number > persisted_height)
.filter(|(_, b)| b.block_ref().block().number() > persisted_height)
.map(|(_, b)| b.block.clone())
.collect::<Vec<_>>();
// sort the blocks by number so we can insert them back in natural order (low -> high)
old_blocks.sort_unstable_by_key(|block| block.block().number);
old_blocks.sort_unstable_by_key(|block| block.block().number());
// re-insert the blocks in natural order and connect them to their parent blocks
for block in old_blocks {
let parent = blocks.get(&block.block().parent_hash).cloned();
let parent = blocks.get(&block.block().parent_hash()).cloned();
let block_state = BlockState::with_parent(block, parent);
let hash = block_state.hash();
let number = block_state.number();
@ -363,7 +355,7 @@ where
// also shift the pending state if it exists
self.inner.in_memory_state.pending.send_modify(|p| {
if let Some(p) = p.as_mut() {
p.parent = blocks.get(&p.block_ref().block.parent_hash).cloned();
p.parent = blocks.get(&p.block_ref().block.parent_hash()).cloned();
}
});
}
@ -474,7 +466,7 @@ where
}
/// Returns the `Header` corresponding to the pending state.
pub fn pending_header(&self) -> Option<Header> {
pub fn pending_header(&self) -> Option<N::BlockHeader> {
self.pending_sealed_header().map(|sealed_header| sealed_header.unseal())
}
@ -484,7 +476,10 @@ where
}
/// Returns the `SealedBlockWithSenders` corresponding to the pending state.
pub fn pending_block_with_senders(&self) -> Option<SealedBlockWithSenders> {
pub fn pending_block_with_senders(&self) -> Option<SealedBlockWithSenders<N::Block>>
where
N::SignedTx: SignedTransaction,
{
self.pending_state()
.and_then(|block_state| block_state.block_ref().block().clone().seal_with_senders())
}
@ -546,7 +541,10 @@ where
}
/// Returns a `TransactionSigned` for the given `TxHash` if found.
pub fn transaction_by_hash(&self, hash: TxHash) -> Option<TransactionSigned> {
pub fn transaction_by_hash(&self, hash: TxHash) -> Option<N::SignedTx>
where
N::SignedTx: Encodable2718,
{
for block_state in self.canonical_chain() {
if let Some(tx) = block_state
.block_ref()
@ -554,7 +552,7 @@ where
.body
.transactions()
.iter()
.find(|tx| tx.hash() == hash)
.find(|tx| tx.trie_hash() == hash)
{
return Some(tx.clone())
}
@ -567,7 +565,10 @@ where
pub fn transaction_by_hash_with_meta(
&self,
tx_hash: TxHash,
) -> Option<(TransactionSigned, TransactionMeta)> {
) -> Option<(N::SignedTx, TransactionMeta)>
where
N::SignedTx: Encodable2718,
{
for block_state in self.canonical_chain() {
if let Some((index, tx)) = block_state
.block_ref()
@ -576,16 +577,16 @@ where
.transactions()
.iter()
.enumerate()
.find(|(_, tx)| tx.hash() == tx_hash)
.find(|(_, tx)| tx.trie_hash() == tx_hash)
{
let meta = TransactionMeta {
tx_hash,
index: index as u64,
block_hash: block_state.hash(),
block_number: block_state.block_ref().block.number,
base_fee: block_state.block_ref().block.header.base_fee_per_gas,
timestamp: block_state.block_ref().block.timestamp,
excess_blob_gas: block_state.block_ref().block.excess_blob_gas,
block_number: block_state.block_ref().block.number(),
base_fee: block_state.block_ref().block.header.base_fee_per_gas(),
timestamp: block_state.block_ref().block.timestamp(),
excess_blob_gas: block_state.block_ref().block.excess_blob_gas(),
};
return Some((tx.clone(), meta))
}
@ -636,14 +637,15 @@ impl<N: NodePrimitives> BlockState<N> {
}
/// Returns the block with senders for the state.
pub fn block_with_senders(&self) -> BlockWithSenders {
pub fn block_with_senders(&self) -> BlockWithSenders<N::Block> {
let block = self.block.block().clone();
let senders = self.block.senders().clone();
BlockWithSenders::new_unchecked(block.unseal(), senders)
let (header, body) = block.split();
BlockWithSenders::new_unchecked(N::Block::new(header.unseal(), body), senders)
}
/// Returns the sealed block with senders for the state.
pub fn sealed_block_with_senders(&self) -> SealedBlockWithSenders {
pub fn sealed_block_with_senders(&self) -> SealedBlockWithSenders<N::Block> {
let block = self.block.block().clone();
let senders = self.block.senders().clone();
SealedBlockWithSenders { block, senders }
@ -656,13 +658,13 @@ impl<N: NodePrimitives> BlockState<N> {
/// Returns the block number of executed block that determines the state.
pub fn number(&self) -> u64 {
self.block.block().number
self.block.block().number()
}
/// Returns the state root after applying the executed block that determines
/// the state.
pub fn state_root(&self) -> B256 {
self.block.block().header.state_root
self.block.block().header.state_root()
}
/// Returns the `Receipts` of executed block that determines the state.
@ -748,7 +750,10 @@ impl<N: NodePrimitives> BlockState<N> {
}
/// Tries to find a transaction by [`TxHash`] in the chain ending at this block.
pub fn transaction_on_chain(&self, hash: TxHash) -> Option<TransactionSigned> {
pub fn transaction_on_chain(&self, hash: TxHash) -> Option<N::SignedTx>
where
N::SignedTx: Encodable2718,
{
self.chain().find_map(|block_state| {
block_state
.block_ref()
@ -756,7 +761,7 @@ impl<N: NodePrimitives> BlockState<N> {
.body
.transactions()
.iter()
.find(|tx| tx.hash() == hash)
.find(|tx| tx.trie_hash() == hash)
.cloned()
})
}
@ -765,7 +770,10 @@ impl<N: NodePrimitives> BlockState<N> {
pub fn transaction_meta_on_chain(
&self,
tx_hash: TxHash,
) -> Option<(TransactionSigned, TransactionMeta)> {
) -> Option<(N::SignedTx, TransactionMeta)>
where
N::SignedTx: Encodable2718,
{
self.chain().find_map(|block_state| {
block_state
.block_ref()
@ -774,16 +782,16 @@ impl<N: NodePrimitives> BlockState<N> {
.transactions()
.iter()
.enumerate()
.find(|(_, tx)| tx.hash() == tx_hash)
.find(|(_, tx)| tx.trie_hash() == tx_hash)
.map(|(index, tx)| {
let meta = TransactionMeta {
tx_hash,
index: index as u64,
block_hash: block_state.hash(),
block_number: block_state.block_ref().block.number,
base_fee: block_state.block_ref().block.header.base_fee_per_gas,
timestamp: block_state.block_ref().block.timestamp,
excess_blob_gas: block_state.block_ref().block.excess_blob_gas,
block_number: block_state.block_ref().block.number(),
base_fee: block_state.block_ref().block.header.base_fee_per_gas(),
timestamp: block_state.block_ref().block.timestamp(),
excess_blob_gas: block_state.block_ref().block.excess_blob_gas(),
};
(tx.clone(), meta)
})
@ -795,7 +803,7 @@ impl<N: NodePrimitives> BlockState<N> {
#[derive(Clone, Debug, PartialEq, Eq, Default)]
pub struct ExecutedBlock<N: NodePrimitives = reth_primitives::EthPrimitives> {
/// Sealed block the rest of fields refer to.
pub block: Arc<SealedBlock>,
pub block: Arc<SealedBlockFor<N::Block>>,
/// Block's senders.
pub senders: Arc<Vec<Address>>,
/// Block's execution outcome.
@ -809,7 +817,7 @@ pub struct ExecutedBlock<N: NodePrimitives = reth_primitives::EthPrimitives> {
impl<N: NodePrimitives> ExecutedBlock<N> {
/// [`ExecutedBlock`] constructor.
pub const fn new(
block: Arc<SealedBlock>,
block: Arc<SealedBlockFor<N::Block>>,
senders: Arc<Vec<Address>>,
execution_output: Arc<ExecutionOutcome<N::Receipt>>,
hashed_state: Arc<HashedPostState>,
@ -819,7 +827,7 @@ impl<N: NodePrimitives> ExecutedBlock<N> {
}
/// Returns a reference to the executed block.
pub fn block(&self) -> &SealedBlock {
pub fn block(&self) -> &SealedBlockFor<N::Block> {
&self.block
}
@ -831,7 +839,7 @@ impl<N: NodePrimitives> ExecutedBlock<N> {
/// Returns a [`SealedBlockWithSenders`]
///
/// Note: this clones the block and senders.
pub fn sealed_block_with_senders(&self) -> SealedBlockWithSenders {
pub fn sealed_block_with_senders(&self) -> SealedBlockWithSenders<N::Block> {
SealedBlockWithSenders { block: (*self.block).clone(), senders: (*self.senders).clone() }
}
@ -869,7 +877,7 @@ pub enum NewCanonicalChain<N: NodePrimitives = reth_primitives::EthPrimitives> {
},
}
impl<N: NodePrimitives> NewCanonicalChain<N> {
impl<N: NodePrimitives<SignedTx: SignedTransaction>> NewCanonicalChain<N> {
/// Returns the length of the new chain.
pub fn new_block_count(&self) -> usize {
match self {
@ -922,7 +930,7 @@ impl<N: NodePrimitives> NewCanonicalChain<N> {
///
/// Returns the new tip for [`Self::Reorg`] and [`Self::Commit`] variants which commit at least
/// 1 new block.
pub fn tip(&self) -> &SealedBlock {
pub fn tip(&self) -> &SealedBlockFor<N::Block> {
match self {
Self::Commit { new } | Self::Reorg { new, .. } => {
new.last().expect("non empty blocks").block()

View File

@ -1,4 +1,5 @@
use super::ExecutedBlock;
use alloy_consensus::BlockHeader;
use alloy_primitives::{
keccak256,
map::{HashMap, HashSet},
@ -75,7 +76,7 @@ macro_rules! impl_state_provider {
impl $($tokens)* BlockHashReader for $type {
fn block_hash(&self, number: BlockNumber) -> ProviderResult<Option<B256>> {
for block in &self.in_memory {
if block.block.number == number {
if block.block.number() == number {
return Ok(Some(block.block.hash()))
}
}
@ -92,9 +93,9 @@ macro_rules! impl_state_provider {
let mut earliest_block_number = None;
let mut in_memory_hashes = Vec::new();
for block in &self.in_memory {
if range.contains(&block.block.number) {
if range.contains(&block.block.number()) {
in_memory_hashes.insert(0, block.block.hash());
earliest_block_number = Some(block.block.number);
earliest_block_number = Some(block.block.number());
}
}

View File

@ -1,5 +1,6 @@
//! Canonical chain state notification trait and types.
use alloy_eips::eip2718::Encodable2718;
use derive_more::{Deref, DerefMut};
use reth_execution_types::{BlockReceipts, Chain};
use reth_primitives::{NodePrimitives, SealedBlockWithSenders, SealedHeader};
@ -122,7 +123,7 @@ impl<N: NodePrimitives> CanonStateNotification<N> {
///
/// Returns the new tip for [`Self::Reorg`] and [`Self::Commit`] variants which commit at least
/// 1 new block.
pub fn tip(&self) -> &SealedBlockWithSenders {
pub fn tip(&self) -> &SealedBlockWithSenders<N::Block> {
match self {
Self::Commit { new } | Self::Reorg { new, .. } => new.tip(),
}
@ -133,7 +134,10 @@ impl<N: NodePrimitives> CanonStateNotification<N> {
///
/// The boolean in the tuple (2nd element) denotes whether the receipt was from the reverted
/// chain segment.
pub fn block_receipts(&self) -> Vec<(BlockReceipts<N::Receipt>, bool)> {
pub fn block_receipts(&self) -> Vec<(BlockReceipts<N::Receipt>, bool)>
where
N::SignedTx: Encodable2718,
{
let mut receipts = Vec::new();
// get old receipts

View File

@ -7,6 +7,7 @@ use reth_db_api::{
};
use reth_db_common::DbTool;
use reth_evm::{execute::BlockExecutorProvider, noop::NoopBlockExecutorProvider};
use reth_node_api::NodePrimitives;
use reth_node_builder::NodeTypesWithDB;
use reth_node_core::dirs::{ChainPath, DataDirPath};
use reth_provider::{
@ -25,7 +26,10 @@ pub(crate) async fn dump_execution_stage<N, E>(
executor: E,
) -> eyre::Result<()>
where
N: ProviderNodeTypes<DB = Arc<DatabaseEnv>>,
N: ProviderNodeTypes<
DB = Arc<DatabaseEnv>,
Primitives: NodePrimitives<Block = reth_primitives::Block>,
>,
E: BlockExecutorProvider,
{
let (output_db, tip_block_number) = setup(from, to, &output_datadir.db(), db_tool)?;
@ -131,7 +135,9 @@ fn import_tables_with_range<N: NodeTypesWithDB>(
/// Dry-run an unwind to FROM block, so we can get the `PlainStorageState` and
/// `PlainAccountState` safely. There might be some state dependency from an address
/// which hasn't been changed in the given range.
fn unwind_and_copy<N: ProviderNodeTypes>(
fn unwind_and_copy<
N: ProviderNodeTypes<Primitives: NodePrimitives<Block = reth_primitives::Block>>,
>(
db_tool: &DbTool<N>,
from: u64,
tip_block_number: u64,
@ -168,7 +174,7 @@ fn dry_run<N, E>(
executor: E,
) -> eyre::Result<()>
where
N: ProviderNodeTypes,
N: ProviderNodeTypes<Primitives: NodePrimitives<Block = reth_primitives::Block>>,
E: BlockExecutorProvider,
{
info!(target: "reth::cli", "Executing stage. [dry-run]");

View File

@ -9,6 +9,7 @@ use reth_db_api::{database::Database, table::TableImporter};
use reth_db_common::DbTool;
use reth_evm::noop::NoopBlockExecutorProvider;
use reth_exex::ExExManagerHandle;
use reth_node_api::NodePrimitives;
use reth_node_core::dirs::{ChainPath, DataDirPath};
use reth_provider::{
providers::{ProviderNodeTypes, StaticFileProvider},
@ -24,7 +25,12 @@ use reth_stages::{
};
use tracing::info;
pub(crate) async fn dump_merkle_stage<N: ProviderNodeTypes<DB = Arc<DatabaseEnv>>>(
pub(crate) async fn dump_merkle_stage<
N: ProviderNodeTypes<
DB = Arc<DatabaseEnv>,
Primitives: NodePrimitives<Block = reth_primitives::Block>,
>,
>(
db_tool: &DbTool<N>,
from: BlockNumber,
to: BlockNumber,
@ -67,7 +73,9 @@ pub(crate) async fn dump_merkle_stage<N: ProviderNodeTypes<DB = Arc<DatabaseEnv>
}
/// Dry-run an unwind to FROM block and copy the necessary table data to the new database.
fn unwind_and_copy<N: ProviderNodeTypes>(
fn unwind_and_copy<
N: ProviderNodeTypes<Primitives: NodePrimitives<Block = reth_primitives::Block>>,
>(
db_tool: &DbTool<N>,
range: (u64, u64),
tip_block_number: u64,

View File

@ -26,7 +26,7 @@ use reth_payload_builder::PayloadBuilderHandle;
use reth_payload_builder_primitives::PayloadBuilder;
use reth_payload_primitives::{PayloadAttributes, PayloadBuilderAttributes};
use reth_payload_validator::ExecutionPayloadValidator;
use reth_primitives::{Head, SealedBlock, SealedHeader};
use reth_primitives::{EthPrimitives, Head, SealedBlock, SealedHeader};
use reth_provider::{
providers::ProviderNodeTypes, BlockIdReader, BlockReader, BlockSource, CanonChainTracker,
ChainSpecProvider, ProviderError, StageCheckpointReader,
@ -84,9 +84,15 @@ const MAX_INVALID_HEADERS: u32 = 512u32;
pub const MIN_BLOCKS_FOR_PIPELINE_RUN: u64 = EPOCH_SLOTS;
/// Helper trait expressing requirements for node types to be used in engine.
pub trait EngineNodeTypes: ProviderNodeTypes + NodeTypesWithEngine {}
pub trait EngineNodeTypes:
ProviderNodeTypes<Primitives = EthPrimitives> + NodeTypesWithEngine
{
}
impl<T> EngineNodeTypes for T where T: ProviderNodeTypes + NodeTypesWithEngine {}
impl<T> EngineNodeTypes for T where
T: ProviderNodeTypes<Primitives = EthPrimitives> + NodeTypesWithEngine
{
}
/// Represents a pending forkchoice update.
///
@ -2879,7 +2885,7 @@ mod tests {
block1.header.set_difficulty(
MAINNET.fork(EthereumHardfork::Paris).ttd().unwrap() - U256::from(1),
);
block1 = block1.unseal().seal_slow();
block1 = block1.unseal::<reth_primitives::Block>().seal_slow();
let (block2, exec_result2) = data.blocks[1].clone();
let mut block2 = block2.unseal().block;
block2.body.withdrawals = None;

View File

@ -26,7 +26,6 @@ reth-payload-builder-primitives.workspace = true
reth-payload-builder.workspace = true
reth-payload-primitives.workspace = true
reth-payload-validator.workspace = true
reth-primitives-traits.workspace = true
reth-primitives.workspace = true
reth-provider.workspace = true
reth-prune.workspace = true
@ -101,7 +100,6 @@ test-utils = [
"reth-evm/test-utils",
"reth-network-p2p/test-utils",
"reth-payload-builder/test-utils",
"reth-primitives-traits/test-utils",
"reth-primitives/test-utils",
"reth-provider/test-utils",
"reth-prune-types",

View File

@ -2,8 +2,7 @@ use crate::metrics::PersistenceMetrics;
use alloy_eips::BlockNumHash;
use reth_chain_state::ExecutedBlock;
use reth_errors::ProviderError;
use reth_primitives::BlockBody;
use reth_primitives_traits::FullNodePrimitives;
use reth_primitives::EthPrimitives;
use reth_provider::{
providers::ProviderNodeTypes, writer::UnifiedStorageWriter, BlockHashReader,
ChainStateBlockWriter, DatabaseProviderFactory, ProviderFactory, StaticFileProviderFactory,
@ -20,18 +19,9 @@ use tracing::{debug, error};
/// A helper trait with requirements for [`ProviderNodeTypes`] to be used within
/// [`PersistenceService`].
pub trait PersistenceNodeTypes:
ProviderNodeTypes<
Primitives: FullNodePrimitives<Block = reth_primitives::Block, BlockBody = BlockBody>,
>
{
}
impl<T> PersistenceNodeTypes for T where
T: ProviderNodeTypes<
Primitives: FullNodePrimitives<Block = reth_primitives::Block, BlockBody = BlockBody>,
>
{
}
pub trait PersistenceNodeTypes: ProviderNodeTypes<Primitives = EthPrimitives> {}
impl<T> PersistenceNodeTypes for T where T: ProviderNodeTypes<Primitives = EthPrimitives> {}
/// Writes parts of reth's in memory tree state to the database and static files.
///
/// This is meant to be a spawned service that listens for various incoming persistence operations,

View File

@ -2281,7 +2281,7 @@ where
self.metrics.block_validation.record_state_root(&trie_output, root_elapsed.as_secs_f64());
debug!(target: "engine::tree", ?root_elapsed, block=?sealed_block.num_hash(), "Calculated state root");
let executed = ExecutedBlock {
let executed: ExecutedBlock = ExecutedBlock {
block: sealed_block.clone(),
senders: Arc::new(block.senders),
execution_output: Arc::new(ExecutionOutcome::from((output, block_number))),

View File

@ -1,11 +1,12 @@
use std::sync::Arc;
use alloy_eips::eip2718::Encodable2718;
use alloy_genesis::Genesis;
use alloy_primitives::{b256, hex};
use futures::StreamExt;
use reth::{args::DevArgs, rpc::api::eth::helpers::EthTransactions};
use reth_chainspec::ChainSpec;
use reth_node_api::FullNodeComponents;
use reth_node_api::{FullNodeComponents, FullNodePrimitives, NodeTypes};
use reth_node_builder::{
rpc::RethRpcAddOns, EngineNodeLauncher, FullNode, NodeBuilder, NodeConfig, NodeHandle,
};
@ -46,6 +47,7 @@ async fn assert_chain_advances<N, AddOns>(node: FullNode<N, AddOns>)
where
N: FullNodeComponents<Provider: CanonStateSubscriptions>,
AddOns: RethRpcAddOns<N, EthApi: EthTransactions>,
N::Types: NodeTypes<Primitives: FullNodePrimitives>,
{
let mut notifications = node.provider.canonical_state_stream();
@ -64,7 +66,7 @@ where
let head = notifications.next().await.unwrap();
let tx = &head.tip().transactions()[0];
assert_eq!(tx.hash(), hash);
assert_eq!(tx.trie_hash(), hash);
println!("mined transaction: {hash}");
}

View File

@ -20,6 +20,7 @@ reth-trie.workspace = true
revm.workspace = true
# alloy
alloy-consensus.workspace = true
alloy-primitives.workspace = true
alloy-eips.workspace = true
@ -42,6 +43,7 @@ serde = [
"alloy-eips/serde",
"alloy-primitives/serde",
"reth-primitives-traits/serde",
"alloy-consensus/serde",
"reth-trie-common/serde",
"reth-trie/serde",
]
@ -51,6 +53,7 @@ serde-bincode-compat = [
"reth-trie-common/serde-bincode-compat",
"serde_with",
"alloy-eips/serde-bincode-compat",
"alloy-consensus/serde-bincode-compat",
]
std = [
"reth-primitives/std",
@ -59,4 +62,5 @@ std = [
"revm/std",
"serde?/std",
"reth-primitives-traits/std",
"alloy-consensus/std",
]

View File

@ -2,16 +2,17 @@
use crate::ExecutionOutcome;
use alloc::{borrow::Cow, collections::BTreeMap};
use alloy_eips::{eip1898::ForkBlock, BlockNumHash};
use alloy_consensus::BlockHeader;
use alloy_eips::{eip1898::ForkBlock, eip2718::Encodable2718, BlockNumHash};
use alloy_primitives::{Address, BlockHash, BlockNumber, TxHash};
use core::{fmt, ops::RangeInclusive};
use reth_execution_errors::{BlockExecutionError, InternalBlockExecutionError};
use reth_primitives::{
SealedBlock, SealedBlockWithSenders, SealedHeader, TransactionSigned,
TransactionSignedEcRecovered,
transaction::SignedTransactionIntoRecoveredExt, SealedBlockFor, SealedBlockWithSenders,
SealedHeader, TransactionSignedEcRecovered,
};
use reth_primitives_traits::NodePrimitives;
use reth_trie_common::updates::TrieUpdates;
use reth_primitives_traits::{Block, BlockBody, NodePrimitives, SignedTransaction};
use reth_trie::updates::TrieUpdates;
use revm::db::BundleState;
/// A chain of blocks and their final state.
@ -28,7 +29,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>,
blocks: BTreeMap<BlockNumber, SealedBlockWithSenders<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
@ -49,11 +50,11 @@ impl<N: NodePrimitives> Chain<N> {
///
/// A chain of blocks should not be empty.
pub fn new(
blocks: impl IntoIterator<Item = SealedBlockWithSenders>,
blocks: impl IntoIterator<Item = SealedBlockWithSenders<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.number(), b)).collect::<BTreeMap<_, _>>();
debug_assert!(!blocks.is_empty(), "Chain should have at least one block");
Self { blocks, execution_outcome, trie_updates }
@ -61,7 +62,7 @@ impl<N: NodePrimitives> Chain<N> {
/// Create new Chain from a single block and its state.
pub fn from_block(
block: SealedBlockWithSenders,
block: SealedBlockWithSenders<N::Block>,
execution_outcome: ExecutionOutcome<N::Receipt>,
trie_updates: Option<TrieUpdates>,
) -> Self {
@ -69,17 +70,17 @@ impl<N: NodePrimitives> Chain<N> {
}
/// Get the blocks in this chain.
pub const fn blocks(&self) -> &BTreeMap<BlockNumber, SealedBlockWithSenders> {
pub const fn blocks(&self) -> &BTreeMap<BlockNumber, SealedBlockWithSenders<N::Block>> {
&self.blocks
}
/// Consumes the type and only returns the blocks in this chain.
pub fn into_blocks(self) -> BTreeMap<BlockNumber, SealedBlockWithSenders> {
pub fn into_blocks(self) -> BTreeMap<BlockNumber, SealedBlockWithSenders<N::Block>> {
self.blocks
}
/// Returns an iterator over all headers in the block with increasing block numbers.
pub fn headers(&self) -> impl Iterator<Item = SealedHeader> + '_ {
pub fn headers(&self) -> impl Iterator<Item = SealedHeader<N::BlockHeader>> + '_ {
self.blocks.values().map(|block| block.header.clone())
}
@ -120,12 +121,15 @@ impl<N: NodePrimitives> Chain<N> {
}
/// Returns the block with matching hash.
pub fn block(&self, block_hash: BlockHash) -> Option<&SealedBlock> {
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> {
pub fn block_with_senders(
&self,
block_hash: BlockHash,
) -> Option<&SealedBlockWithSenders<N::Block>> {
self.blocks.iter().find_map(|(_num, block)| (block.hash() == block_hash).then_some(block))
}
@ -134,7 +138,7 @@ impl<N: NodePrimitives> Chain<N> {
&self,
block_number: BlockNumber,
) -> Option<ExecutionOutcome<N::Receipt>> {
if self.tip().number == block_number {
if self.tip().number() == block_number {
return Some(self.execution_outcome.clone())
}
@ -152,14 +156,14 @@ impl<N: NodePrimitives> Chain<N> {
/// 3. The optional trie updates.
pub fn into_inner(
self,
) -> (ChainBlocks<'static>, ExecutionOutcome<N::Receipt>, Option<TrieUpdates>) {
) -> (ChainBlocks<'static, N::Block>, ExecutionOutcome<N::Receipt>, Option<TrieUpdates>) {
(ChainBlocks { blocks: Cow::Owned(self.blocks) }, self.execution_outcome, self.trie_updates)
}
/// Destructure the chain into its inner components:
/// 1. A reference to the blocks contained in the chain.
/// 2. A reference to the execution outcome representing the final state.
pub const fn inner(&self) -> (ChainBlocks<'_>, &ExecutionOutcome<N::Receipt>) {
pub const fn inner(&self) -> (ChainBlocks<'_, N::Block>, &ExecutionOutcome<N::Receipt>) {
(ChainBlocks { blocks: Cow::Borrowed(&self.blocks) }, &self.execution_outcome)
}
@ -169,14 +173,15 @@ 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> + '_ {
pub fn blocks_iter(&self) -> impl Iterator<Item = &SealedBlockWithSenders<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, &Vec<Option<N::Receipt>>)> + '_ {
) -> impl Iterator<Item = (&SealedBlockWithSenders<N::Block>, &Vec<Option<N::Receipt>>)> + '_
{
self.blocks_iter().zip(self.block_receipts_iter())
}
@ -184,7 +189,7 @@ 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.number().saturating_sub(1), hash: first.parent_hash() }
}
/// Get the first block in this chain.
@ -193,7 +198,7 @@ impl<N: NodePrimitives> Chain<N> {
///
/// If chain doesn't have any blocks.
#[track_caller]
pub fn first(&self) -> &SealedBlockWithSenders {
pub fn first(&self) -> &SealedBlockWithSenders<N::Block> {
self.blocks.first_key_value().expect("Chain should have at least one block").1
}
@ -203,7 +208,7 @@ impl<N: NodePrimitives> Chain<N> {
///
/// If chain doesn't have any blocks.
#[track_caller]
pub fn tip(&self) -> &SealedBlockWithSenders {
pub fn tip(&self) -> &SealedBlockWithSenders<N::Block> {
self.blocks.last_key_value().expect("Chain should have at least one block").1
}
@ -218,7 +223,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().number()..=self.tip().number()
}
/// Get all receipts for the given block.
@ -230,15 +235,18 @@ impl<N: NodePrimitives> Chain<N> {
/// Get all receipts with attachment.
///
/// Attachment includes block number, block hash, transaction hash and transaction index.
pub fn receipts_with_attachment(&self) -> Vec<BlockReceipts<N::Receipt>> {
pub fn receipts_with_attachment(&self) -> Vec<BlockReceipts<N::Receipt>>
where
N::SignedTx: Encodable2718,
{
let mut receipt_attach = Vec::with_capacity(self.blocks().len());
for ((block_num, block), receipts) in
self.blocks().iter().zip(self.execution_outcome.receipts().iter())
{
let mut tx_receipts = Vec::with_capacity(receipts.len());
for (tx, receipt) in block.body.transactions.iter().zip(receipts.iter()) {
for (tx, receipt) in block.body.transactions().iter().zip(receipts.iter()) {
tx_receipts.push((
tx.hash(),
tx.trie_hash(),
receipt.as_ref().expect("receipts have not been pruned").clone(),
));
}
@ -252,10 +260,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,
block: SealedBlockWithSenders<N::Block>,
execution_outcome: ExecutionOutcome<N::Receipt>,
) {
self.blocks.insert(block.number, block);
self.blocks.insert(block.number(), block);
self.execution_outcome.extend(execution_outcome);
self.trie_updates.take(); // reset
}
@ -375,22 +383,22 @@ impl fmt::Display for DisplayBlocksChain<'_> {
/// All blocks in the chain
#[derive(Clone, Debug, Default, PartialEq, Eq)]
pub struct ChainBlocks<'a> {
blocks: Cow<'a, BTreeMap<BlockNumber, SealedBlockWithSenders>>,
pub struct ChainBlocks<'a, B: Block> {
blocks: Cow<'a, BTreeMap<BlockNumber, SealedBlockWithSenders<B>>>,
}
impl ChainBlocks<'_> {
impl<B: Block<Body: BlockBody<Transaction: SignedTransaction>>> ChainBlocks<'_, B> {
/// Creates a consuming iterator over all blocks in the chain with increasing block number.
///
/// Note: this always yields at least one block.
#[inline]
pub fn into_blocks(self) -> impl Iterator<Item = SealedBlockWithSenders> {
pub fn into_blocks(self) -> impl Iterator<Item = SealedBlockWithSenders<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)> {
pub fn iter(&self) -> impl Iterator<Item = (&BlockNumber, &SealedBlockWithSenders<B>)> {
self.blocks.iter()
}
@ -400,7 +408,7 @@ impl ChainBlocks<'_> {
///
/// Chains always have at least one block.
#[inline]
pub fn tip(&self) -> &SealedBlockWithSenders {
pub fn tip(&self) -> &SealedBlockWithSenders<B> {
self.blocks.last_key_value().expect("Chain should have at least one block").1
}
@ -410,21 +418,21 @@ impl ChainBlocks<'_> {
///
/// Chains always have at least one block.
#[inline]
pub fn first(&self) -> &SealedBlockWithSenders {
pub fn first(&self) -> &SealedBlockWithSenders<B> {
self.blocks.first_key_value().expect("Chain should have at least one block").1
}
/// Returns an iterator over all transactions in the chain.
#[inline]
pub fn transactions(&self) -> impl Iterator<Item = &TransactionSigned> + '_ {
self.blocks.values().flat_map(|block| block.body.transactions.iter())
pub fn transactions(&self) -> impl Iterator<Item = &<B::Body as BlockBody>::Transaction> + '_ {
self.blocks.values().flat_map(|block| block.body.transactions().iter())
}
/// Returns an iterator over all transactions and their senders.
#[inline]
pub fn transactions_with_sender(
&self,
) -> impl Iterator<Item = (&Address, &TransactionSigned)> + '_ {
) -> impl Iterator<Item = (&Address, &<B::Body as BlockBody>::Transaction)> + '_ {
self.blocks.values().flat_map(|block| block.transactions_with_sender())
}
@ -434,20 +442,21 @@ impl ChainBlocks<'_> {
#[inline]
pub fn transactions_ecrecovered(
&self,
) -> impl Iterator<Item = TransactionSignedEcRecovered> + '_ {
) -> impl Iterator<Item = TransactionSignedEcRecovered<<B::Body as BlockBody>::Transaction>> + '_
{
self.transactions_with_sender().map(|(signer, tx)| tx.clone().with_signer(*signer))
}
/// 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.hash()))
self.blocks.values().flat_map(|block| block.transactions().iter().map(|tx| tx.trie_hash()))
}
}
impl IntoIterator for ChainBlocks<'_> {
type Item = (BlockNumber, SealedBlockWithSenders);
type IntoIter = std::collections::btree_map::IntoIter<BlockNumber, SealedBlockWithSenders>;
impl<B: Block> IntoIterator for ChainBlocks<'_, B> {
type Item = (BlockNumber, SealedBlockWithSenders<B>);
type IntoIter = std::collections::btree_map::IntoIter<BlockNumber, SealedBlockWithSenders<B>>;
fn into_iter(self) -> Self::IntoIter {
#[allow(clippy::unnecessary_to_owned)]

View File

@ -244,7 +244,7 @@ where
/// Configures the types of the node.
pub fn with_types<T>(self) -> NodeBuilderWithTypes<RethFullAdapter<DB, T>>
where
T: NodeTypesWithEngine<ChainSpec = ChainSpec> + NodeTypesForProvider,
T: NodeTypesWithEngine<ChainSpec = ChainSpec> + NodeTypesForTree,
{
self.with_types_and_provider()
}
@ -268,7 +268,7 @@ where
node: N,
) -> NodeBuilderWithComponents<RethFullAdapter<DB, N>, N::ComponentsBuilder, N::AddOns>
where
N: Node<RethFullAdapter<DB, N>, ChainSpec = ChainSpec> + NodeTypesForProvider,
N: Node<RethFullAdapter<DB, N>, ChainSpec = ChainSpec> + NodeTypesForTree,
{
self.with_types().with_components(node.components_builder()).with_add_ons(node.add_ons())
}
@ -305,7 +305,7 @@ where
/// Configures the types of the node.
pub fn with_types<T>(self) -> WithLaunchContext<NodeBuilderWithTypes<RethFullAdapter<DB, T>>>
where
T: NodeTypesWithEngine<ChainSpec = ChainSpec> + NodeTypesForProvider,
T: NodeTypesWithEngine<ChainSpec = ChainSpec> + NodeTypesForTree,
{
WithLaunchContext { builder: self.builder.with_types(), task_executor: self.task_executor }
}
@ -336,7 +336,7 @@ where
NodeBuilderWithComponents<RethFullAdapter<DB, N>, N::ComponentsBuilder, N::AddOns>,
>
where
N: Node<RethFullAdapter<DB, N>, ChainSpec = ChainSpec> + NodeTypesForProvider,
N: Node<RethFullAdapter<DB, N>, ChainSpec = ChainSpec> + NodeTypesForTree,
{
self.with_types().with_components(node.components_builder()).with_add_ons(node.add_ons())
}

View File

@ -382,7 +382,10 @@ where
pub async fn create_provider_factory<N>(&self) -> eyre::Result<ProviderFactory<N>>
where
N: ProviderNodeTypes<DB = DB, ChainSpec = ChainSpec>,
N::Primitives: FullNodePrimitives<BlockBody = reth_primitives::BlockBody>,
N::Primitives: FullNodePrimitives<
Block = reth_primitives::Block,
BlockBody = reth_primitives::BlockBody,
>,
{
let factory = ProviderFactory::new(
self.right().clone(),
@ -449,7 +452,10 @@ where
) -> eyre::Result<LaunchContextWith<Attached<WithConfigs<ChainSpec>, ProviderFactory<N>>>>
where
N: ProviderNodeTypes<DB = DB, ChainSpec = ChainSpec>,
N::Primitives: FullNodePrimitives<BlockBody = reth_primitives::BlockBody>,
N::Primitives: FullNodePrimitives<
Block = reth_primitives::Block,
BlockBody = reth_primitives::BlockBody,
>,
{
let factory = self.create_provider_factory().await?;
let ctx = LaunchContextWith {

View File

@ -41,7 +41,8 @@ where
N: ProviderNodeTypes,
Client: EthBlockClient + 'static,
Executor: BlockExecutorProvider,
N::Primitives: FullNodePrimitives<BlockBody = reth_primitives::BlockBody>,
N::Primitives:
FullNodePrimitives<Block = reth_primitives::Block, BlockBody = reth_primitives::BlockBody>,
{
// building network downloaders using the fetch client
let header_downloader = ReverseHeadersDownloaderBuilder::new(config.headers)
@ -89,7 +90,8 @@ where
H: HeaderDownloader<Header = alloy_consensus::Header> + 'static,
B: BodyDownloader<Body = BodyTy<N>> + 'static,
Executor: BlockExecutorProvider,
N::Primitives: FullNodePrimitives<BlockBody = reth_primitives::BlockBody>,
N::Primitives:
FullNodePrimitives<Block = reth_primitives::Block, BlockBody = reth_primitives::BlockBody>,
{
let mut builder = Pipeline::<N>::builder();

View File

@ -35,7 +35,6 @@ where
let block_hash = block.hash();
let excess_blob_gas = block.excess_blob_gas;
let timestamp = block.timestamp;
let block = block.unseal();
let l1_block_info =
reth_optimism_evm::extract_l1_info(&block.body).map_err(OpEthApiError::from)?;

View File

@ -3,6 +3,7 @@
use alloc::{fmt, vec::Vec};
use alloy_consensus::Transaction;
use alloy_eips::eip4895::Withdrawals;
use crate::{FullSignedTx, InMemorySize, MaybeArbitrary, MaybeSerde};
@ -30,9 +31,18 @@ pub trait BlockBody:
/// Ordered list of signed transactions as committed in block.
type Transaction: Transaction;
/// Ommer header type.
type OmmerHeader;
/// Returns reference to transactions in block.
fn transactions(&self) -> &[Self::Transaction];
/// Consume the block body and return a [`Vec`] of transactions.
fn into_transactions(self) -> Vec<Self::Transaction>;
/// Returns block withdrawals if any.
fn withdrawals(&self) -> Option<&Withdrawals>;
/// Returns block ommers if any.
fn ommers(&self) -> Option<&[Self::OmmerHeader]>;
}

View File

@ -44,7 +44,7 @@ pub trait Block:
type Header: BlockHeader + 'static;
/// The block's body contains the transactions in the block.
type Body: BlockBody + Send + Sync + Unpin + 'static;
type Body: BlockBody<OmmerHeader = Self::Header> + Send + Sync + Unpin + 'static;
/// Create new block instance.
fn new(header: Self::Header, body: Self::Body) -> Self;

View File

@ -14,7 +14,7 @@ pub trait NodePrimitives:
/// Block header primitive.
type BlockHeader: BlockHeader;
/// Block body primitive.
type BlockBody: BlockBody<Transaction = Self::SignedTx>;
type BlockBody: BlockBody<Transaction = Self::SignedTx, OmmerHeader = Self::BlockHeader>;
/// Signed version of the transaction type.
type SignedTx: Send + Sync + Unpin + Clone + fmt::Debug + PartialEq + Eq + MaybeSerde + 'static;
/// Transaction envelope type ID.

View File

@ -267,11 +267,6 @@ impl<H, B> SealedBlock<H, B> {
}
impl SealedBlock {
/// Unseal the block
pub fn unseal(self) -> Block {
Block { header: self.header.unseal(), body: self.body }
}
/// Returns an iterator over all blob transactions of the block
#[inline]
pub fn blob_transactions_iter(&self) -> impl Iterator<Item = &TransactionSigned> + '_ {
@ -391,6 +386,14 @@ where
Ok(SealedBlockWithSenders { block: self, senders })
}
/// Unseal the block
pub fn unseal<Block>(self) -> Block
where
Block: reth_primitives_traits::Block<Header = H, Body = B>,
{
Block::new(self.header.unseal(), self.body)
}
/// Ensures that the transaction root in the block header is valid.
///
/// The transaction root is the Keccak 256-bit hash of the root node of the trie structure
@ -455,7 +458,7 @@ where
impl<H, B> reth_primitives_traits::Block for SealedBlock<H, B>
where
H: reth_primitives_traits::BlockHeader + 'static,
B: reth_primitives_traits::BlockBody + 'static,
B: reth_primitives_traits::BlockBody<OmmerHeader = H> + 'static,
Self: Serialize + for<'a> Deserialize<'a>,
{
type Header = H;
@ -684,6 +687,7 @@ impl InMemorySize for BlockBody {
impl reth_primitives_traits::BlockBody for BlockBody {
type Transaction = TransactionSigned;
type OmmerHeader = Header;
fn transactions(&self) -> &[Self::Transaction] {
&self.transactions
@ -692,6 +696,14 @@ impl reth_primitives_traits::BlockBody for BlockBody {
fn into_transactions(self) -> Vec<Self::Transaction> {
self.transactions
}
fn withdrawals(&self) -> Option<&Withdrawals> {
self.withdrawals.as_ref()
}
fn ommers(&self) -> Option<&[Self::OmmerHeader]> {
Some(&self.ommers)
}
}
impl From<Block> for BlockBody {
@ -1168,7 +1180,7 @@ mod tests {
fn test_default_seal() {
let block: SealedBlock = SealedBlock::default();
let sealed = block.hash();
let block = block.unseal();
let block: Block = block.unseal();
let block = block.seal_slow();
assert_eq!(sealed, block.hash());
}

View File

@ -3,8 +3,8 @@ use crate::{
BlockWithSenders, SealedBlock,
};
use alloc::vec::Vec;
use alloy_eips::eip2718::Encodable2718;
use reth_primitives_traits::{Block, BlockBody, SealedHeader, SignedTransaction};
use alloy_eips::{eip2718::Encodable2718, BlockNumHash};
use reth_primitives_traits::{Block, BlockBody, BlockHeader, SealedHeader, SignedTransaction};
use revm_primitives::{Address, B256};
/// Extension trait for [`reth_primitives_traits::Block`] implementations
@ -121,3 +121,17 @@ pub trait BlockBodyTxExt: BlockBody {
}
impl<T: BlockBody> BlockBodyTxExt for T {}
/// Extension trait for [`BlockHeader`] adding useful helper methods.
pub trait HeaderExt: BlockHeader {
/// TODO: remove once <https://github.com/alloy-rs/alloy/pull/1687> is released
///
/// Returns the parent block's number and hash
///
/// Note: for the genesis block the parent number is 0 and the parent hash is the zero hash.
fn parent_num_hash(&self) -> BlockNumHash {
BlockNumHash::new(self.number().saturating_sub(1), self.parent_hash())
}
}
impl<T: BlockHeader> HeaderExt for T {}

View File

@ -42,8 +42,12 @@ where
where
Provider: ChainSpecProvider + 'static,
Tasks: TaskSpawner,
Events:
CanonStateSubscriptions<Primitives: NodePrimitives<Receipt = reth_primitives::Receipt>>,
Events: CanonStateSubscriptions<
Primitives: NodePrimitives<
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
>,
>,
{
let fee_history_cache =
FeeHistoryCache::new(self.cache.clone(), self.config.fee_history_cache);

View File

@ -212,7 +212,7 @@ pub async fn fee_history_cache_new_blocks_task<St, Provider, N>(
) where
St: Stream<Item = CanonStateNotification<N>> + Unpin + 'static,
Provider: BlockReaderIdExt + ChainSpecProvider + 'static,
N: NodePrimitives<Receipt = reth_primitives::Receipt>,
N: NodePrimitives<Block = reth_primitives::Block, Receipt = reth_primitives::Receipt>,
{
// We're listening for new blocks emitted when the node is in live sync.
// If the node transitions to stage sync, we need to fetch the missing blocks
@ -249,7 +249,7 @@ pub async fn fee_history_cache_new_blocks_task<St, Provider, N>(
break;
};
let committed = event .committed();
let committed = event.committed();
let (blocks, receipts): (Vec<_>, Vec<_>) = committed
.blocks_and_receipts()
.map(|(block, receipts)| {

View File

@ -103,8 +103,12 @@ where
) -> Self
where
Tasks: TaskSpawner + Clone + 'static,
Events:
CanonStateSubscriptions<Primitives: NodePrimitives<Receipt = reth_primitives::Receipt>>,
Events: CanonStateSubscriptions<
Primitives: NodePrimitives<
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
>,
>,
{
let blocking_task_pool =
BlockingTaskPool::build().expect("failed to build blocking task pool");

View File

@ -32,7 +32,6 @@ where
let block_hash = block.hash();
let excess_blob_gas = block.excess_blob_gas;
let timestamp = block.timestamp;
let block = block.unseal();
return block
.body

View File

@ -2,6 +2,7 @@
use std::sync::Arc;
use alloy_eips::eip2718::Encodable2718;
use alloy_primitives::TxHash;
use alloy_rpc_types_eth::{
pubsub::{
@ -85,8 +86,13 @@ impl<Provider, Pool, Events, Network, Eth> EthPubSubApiServer<Eth::Transaction>
where
Provider: BlockReader + EvmEnvProvider + Clone + 'static,
Pool: TransactionPool + 'static,
Events: CanonStateSubscriptions<Primitives: NodePrimitives<Receipt = reth_primitives::Receipt>>
+ Clone
Events: CanonStateSubscriptions<
Primitives: NodePrimitives<
SignedTx: Encodable2718,
BlockHeader = reth_primitives::Header,
Receipt = reth_primitives::Receipt,
>,
> + Clone
+ 'static,
Network: NetworkInfo + Clone + 'static,
Eth: TransactionCompat + 'static,
@ -120,8 +126,13 @@ async fn handle_accepted<Provider, Pool, Events, Network, Eth>(
where
Provider: BlockReader + EvmEnvProvider + Clone + 'static,
Pool: TransactionPool + 'static,
Events: CanonStateSubscriptions<Primitives: NodePrimitives<Receipt = reth_primitives::Receipt>>
+ Clone
Events: CanonStateSubscriptions<
Primitives: NodePrimitives<
SignedTx: Encodable2718,
BlockHeader = reth_primitives::Header,
Receipt = reth_primitives::Receipt,
>,
> + Clone
+ 'static,
Network: NetworkInfo + Clone + 'static,
Eth: TransactionCompat,
@ -338,8 +349,13 @@ where
impl<Provider, Pool, Events, Network> EthPubSubInner<Provider, Pool, Events, Network>
where
Provider: BlockReader + EvmEnvProvider + 'static,
Events: CanonStateSubscriptions<Primitives: NodePrimitives<Receipt = reth_primitives::Receipt>>
+ 'static,
Events: CanonStateSubscriptions<
Primitives: NodePrimitives<
SignedTx: Encodable2718,
BlockHeader = reth_primitives::Header,
Receipt = reth_primitives::Receipt,
>,
> + 'static,
Network: NetworkInfo + 'static,
Pool: 'static,
{

View File

@ -57,7 +57,7 @@ pub struct BlockchainProvider2<N: NodeTypesWithDB> {
pub(crate) database: ProviderFactory<N>,
/// Tracks the chain info wrt forkchoice updates and in memory canonical
/// state.
pub(crate) canonical_in_memory_state: CanonicalInMemoryState,
pub(crate) canonical_in_memory_state: CanonicalInMemoryState<N::Primitives>,
}
impl<N: NodeTypesWithDB> Clone for BlockchainProvider2<N> {
@ -117,7 +117,7 @@ impl<N: ProviderNodeTypes> BlockchainProvider2<N> {
}
/// Gets a clone of `canonical_in_memory_state`.
pub fn canonical_in_memory_state(&self) -> CanonicalInMemoryState {
pub fn canonical_in_memory_state(&self) -> CanonicalInMemoryState<N::Primitives> {
self.canonical_in_memory_state.clone()
}
@ -132,8 +132,8 @@ impl<N: ProviderNodeTypes> BlockchainProvider2<N> {
/// This uses a given [`BlockState`] to initialize a state provider for that block.
fn block_state_provider(
&self,
state: &BlockState,
) -> ProviderResult<MemoryOverlayStateProvider> {
state: &BlockState<N::Primitives>,
) -> ProviderResult<MemoryOverlayStateProvider<N::Primitives>> {
let anchor_hash = state.anchor().hash;
let latest_historical = self.database.history_by_block_hash(anchor_hash)?;
Ok(state.state_provider(latest_historical))
@ -647,7 +647,7 @@ impl<N: ProviderNodeTypes> StateProviderFactory for BlockchainProvider2<N> {
}
}
impl<N: NodeTypesWithDB> CanonChainTracker for BlockchainProvider2<N>
impl<N: ProviderNodeTypes> CanonChainTracker for BlockchainProvider2<N>
where
Self: BlockReader,
{
@ -721,7 +721,7 @@ impl<N: NodeTypesWithDB<Primitives = EthPrimitives>> CanonStateSubscriptions
}
}
impl<N: NodeTypesWithDB> ForkChoiceSubscriptions for BlockchainProvider2<N> {
impl<N: ProviderNodeTypes> ForkChoiceSubscriptions for BlockchainProvider2<N> {
fn subscribe_safe_block(&self) -> ForkChoiceNotifications {
let receiver = self.canonical_in_memory_state.subscribe_safe_block();
ForkChoiceNotifications(receiver)
@ -2347,7 +2347,7 @@ mod tests {
(block_range, |block: &SealedBlock| block.clone().unseal()),
(block_with_senders_range, |block: &SealedBlock| block
.clone()
.unseal()
.unseal::<reth_primitives::Block>()
.with_senders_unchecked(vec![])),
(sealed_block_with_senders_range, |block: &SealedBlock| block
.clone()
@ -2542,7 +2542,7 @@ mod tests {
block_with_senders,
|block: &SealedBlock, _: TxNumber, _: B256, _: &Vec<Vec<Receipt>>| (
(BlockHashOrNumber::Number(block.number), TransactionVariant::WithHash),
block.clone().unseal().with_recovered_senders()
block.clone().unseal::<reth_primitives::Block>().with_recovered_senders()
),
(BlockHashOrNumber::Number(u64::MAX), TransactionVariant::WithHash)
),
@ -2551,7 +2551,7 @@ mod tests {
block_with_senders,
|block: &SealedBlock, _: TxNumber, _: B256, _: &Vec<Vec<Receipt>>| (
(BlockHashOrNumber::Hash(block.hash()), TransactionVariant::WithHash),
block.clone().unseal().with_recovered_senders()
block.clone().unseal::<reth_primitives::Block>().with_recovered_senders()
),
(BlockHashOrNumber::Hash(B256::random()), TransactionVariant::WithHash)
),
@ -2561,7 +2561,12 @@ mod tests {
|block: &SealedBlock, _: TxNumber, _: B256, _: &Vec<Vec<Receipt>>| (
(BlockHashOrNumber::Number(block.number), TransactionVariant::WithHash),
Some(
block.clone().unseal().with_recovered_senders().unwrap().seal(block.hash())
block
.clone()
.unseal::<reth_primitives::Block>()
.with_recovered_senders()
.unwrap()
.seal(block.hash())
)
),
(BlockHashOrNumber::Number(u64::MAX), TransactionVariant::WithHash)
@ -2572,7 +2577,12 @@ mod tests {
|block: &SealedBlock, _: TxNumber, _: B256, _: &Vec<Vec<Receipt>>| (
(BlockHashOrNumber::Hash(block.hash()), TransactionVariant::WithHash),
Some(
block.clone().unseal().with_recovered_senders().unwrap().seal(block.hash())
block
.clone()
.unseal::<reth_primitives::Block>()
.with_recovered_senders()
.unwrap()
.seal(block.hash())
)
),
(BlockHashOrNumber::Hash(B256::random()), TransactionVariant::WithHash)

View File

@ -8,6 +8,7 @@ use crate::{
};
use alloy_consensus::Header;
use alloy_eips::{
eip2718::Encodable2718,
eip4895::{Withdrawal, Withdrawals},
BlockHashOrNumber, BlockId, BlockNumHash, BlockNumberOrTag, HashOrNumber,
};
@ -48,13 +49,14 @@ use tracing::trace;
/// CAUTION: Avoid holding this provider for too long or the inner database transaction will
/// time-out.
#[derive(Debug)]
#[doc(hidden)] // triggers ICE for `cargo docs`
pub struct ConsistentProvider<N: ProviderNodeTypes> {
/// Storage provider.
storage_provider: <ProviderFactory<N> as DatabaseProviderFactory>::Provider,
/// Head block at time of [`Self`] creation
head_block: Option<Arc<BlockState>>,
head_block: Option<Arc<BlockState<N::Primitives>>>,
/// In-memory canonical state. This is not a snapshot, and can change! Use with caution.
canonical_in_memory_state: CanonicalInMemoryState,
canonical_in_memory_state: CanonicalInMemoryState<N::Primitives>,
}
impl<N: ProviderNodeTypes> ConsistentProvider<N> {
@ -65,7 +67,7 @@ impl<N: ProviderNodeTypes> ConsistentProvider<N> {
/// view of memory and database.
pub fn new(
storage_provider_factory: ProviderFactory<N>,
state: CanonicalInMemoryState,
state: CanonicalInMemoryState<N::Primitives>,
) -> ProviderResult<Self> {
// Each one provides a snapshot at the time of instantiation, but its order matters.
//
@ -307,7 +309,7 @@ impl<N: ProviderNodeTypes> ConsistentProvider<N> {
RangeInclusive<BlockNumber>,
&mut P,
) -> ProviderResult<Vec<T>>,
G: Fn(&BlockState, &mut P) -> Option<T>,
G: Fn(&BlockState<N::Primitives>, &mut P) -> Option<T>,
P: FnMut(&T) -> bool,
{
// Each one provides a snapshot at the time of instantiation, but its order matters.
@ -399,8 +401,8 @@ impl<N: ProviderNodeTypes> ConsistentProvider<N> {
/// This uses a given [`BlockState`] to initialize a state provider for that block.
fn block_state_provider_ref(
&self,
state: &BlockState,
) -> ProviderResult<MemoryOverlayStateProviderRef<'_>> {
state: &BlockState<N::Primitives>,
) -> ProviderResult<MemoryOverlayStateProviderRef<'_, N::Primitives>> {
let anchor_hash = state.anchor().hash;
let latest_historical = self.history_by_block_hash_ref(anchor_hash)?;
let in_memory = state.chain().map(|block_state| block_state.block()).collect();
@ -423,7 +425,7 @@ impl<N: ProviderNodeTypes> ConsistentProvider<N> {
&DatabaseProviderRO<N::DB, N>,
RangeInclusive<TxNumber>,
) -> ProviderResult<Vec<R>>,
M: Fn(RangeInclusive<usize>, &BlockState) -> ProviderResult<Vec<R>>,
M: Fn(RangeInclusive<usize>, &BlockState<N::Primitives>) -> ProviderResult<Vec<R>>,
{
let in_mem_chain = self.head_block.iter().flat_map(|b| b.chain()).collect::<Vec<_>>();
let provider = &self.storage_provider;
@ -445,7 +447,7 @@ impl<N: ProviderNodeTypes> ConsistentProvider<N> {
let (start, end) = self.convert_range_bounds(range, || {
in_mem_chain
.iter()
.map(|b| b.block_ref().block().body.transactions.len() as u64)
.map(|b| b.block_ref().block().body.transactions().len() as u64)
.sum::<u64>() +
last_block_body_index.last_tx_num()
});
@ -477,7 +479,7 @@ impl<N: ProviderNodeTypes> ConsistentProvider<N> {
// Iterate from the lowest block to the highest in-memory chain
for block_state in in_mem_chain.iter().rev() {
let block_tx_count = block_state.block_ref().block().body.transactions.len();
let block_tx_count = block_state.block_ref().block().body.transactions().len();
let remaining = (tx_range.end() - tx_range.start() + 1) as usize;
// If the transaction range start is equal or higher than the next block first
@ -519,7 +521,7 @@ impl<N: ProviderNodeTypes> ConsistentProvider<N> {
) -> ProviderResult<Option<R>>
where
S: FnOnce(&DatabaseProviderRO<N::DB, N>) -> ProviderResult<Option<R>>,
M: Fn(usize, TxNumber, &BlockState) -> ProviderResult<Option<R>>,
M: Fn(usize, TxNumber, &BlockState<N::Primitives>) -> ProviderResult<Option<R>>,
{
let in_mem_chain = self.head_block.iter().flat_map(|b| b.chain()).collect::<Vec<_>>();
let provider = &self.storage_provider;
@ -551,10 +553,10 @@ impl<N: ProviderNodeTypes> ConsistentProvider<N> {
let executed_block = block_state.block_ref();
let block = executed_block.block();
for tx_index in 0..block.body.transactions.len() {
for tx_index in 0..block.body.transactions().len() {
match id {
HashOrNumber::Hash(tx_hash) => {
if tx_hash == block.body.transactions[tx_index].hash() {
if tx_hash == block.body.transactions()[tx_index].trie_hash() {
return fetch_from_block_state(tx_index, in_memory_tx_num, block_state)
}
}
@ -586,7 +588,7 @@ impl<N: ProviderNodeTypes> ConsistentProvider<N> {
) -> ProviderResult<R>
where
S: FnOnce(&DatabaseProviderRO<N::DB, N>) -> ProviderResult<R>,
M: Fn(&BlockState) -> ProviderResult<R>,
M: Fn(&BlockState<N::Primitives>) -> ProviderResult<R>,
{
if let Some(Some(block_state)) = self.head_block.as_ref().map(|b| b.block_on_chain(id)) {
return fetch_from_block_state(block_state)
@ -839,7 +841,7 @@ impl<N: ProviderNodeTypes> BlockReader for ConsistentProvider<N> {
return Ok(Some(Vec::new()))
}
Ok(Some(block_state.block_ref().block().body.ommers.clone()))
Ok(block_state.block_ref().block().body.ommers().map(|o| o.to_vec()))
},
)
}
@ -865,7 +867,7 @@ impl<N: ProviderNodeTypes> BlockReader for ConsistentProvider<N> {
// Iterate from the lowest block in memory until our target block
for state in block_state.chain().collect::<Vec<_>>().into_iter().rev() {
let block_tx_count = state.block_ref().block.body.transactions.len() as u64;
let block_tx_count = state.block_ref().block.body.transactions().len() as u64;
if state.block_ref().block().number == number {
stored_indices.tx_count = block_tx_count;
} else {
@ -962,7 +964,7 @@ impl<N: ProviderNodeTypes> TransactionsProvider for ConsistentProvider<N> {
.block_ref()
.block()
.body
.transactions
.transactions()
.get(tx_index)
.cloned()
.map(Into::into))
@ -982,7 +984,7 @@ impl<N: ProviderNodeTypes> TransactionsProvider for ConsistentProvider<N> {
.block_ref()
.block()
.body
.transactions
.transactions()
.get(tx_index)
.cloned()
.map(Into::into))
@ -1050,11 +1052,7 @@ impl<N: ProviderNodeTypes> TransactionsProvider for ConsistentProvider<N> {
range,
|db_provider, db_range| db_provider.transactions_by_tx_range(db_range),
|index_range, block_state| {
Ok(block_state.block_ref().block().body.transactions[index_range]
.iter()
.cloned()
.map(Into::into)
.collect())
Ok(block_state.block_ref().block().body.transactions()[index_range].to_vec())
},
)
}
@ -1098,12 +1096,13 @@ impl<N: ProviderNodeTypes> ReceiptProvider for ConsistentProvider<N> {
// assuming 1:1 correspondence between transactions and receipts
debug_assert_eq!(
block.body.transactions.len(),
block.body.transactions().len(),
receipts.len(),
"Mismatch between transaction and receipt count"
);
if let Some(tx_index) = block.body.transactions.iter().position(|tx| tx.hash() == hash)
if let Some(tx_index) =
block.body.transactions().iter().position(|tx| tx.trie_hash() == hash)
{
// safe to use tx_index for receipts due to 1:1 correspondence
return Ok(receipts.get(tx_index).cloned());
@ -1181,7 +1180,7 @@ impl<N: ProviderNodeTypes> WithdrawalsProvider for ConsistentProvider<N> {
self.get_in_memory_or_storage_by_block(
id,
|db_provider| db_provider.withdrawals_by_block(id, timestamp),
|block_state| Ok(block_state.block_ref().block().body.withdrawals.clone()),
|block_state| Ok(block_state.block_ref().block().body.withdrawals().cloned()),
)
}
@ -1196,8 +1195,8 @@ impl<N: ProviderNodeTypes> WithdrawalsProvider for ConsistentProvider<N> {
.block_ref()
.block()
.body
.withdrawals
.clone()
.withdrawals()
.cloned()
.and_then(|mut w| w.pop()))
},
)

View File

@ -47,7 +47,7 @@ use reth_execution_types::{Chain, ExecutionOutcome};
use reth_network_p2p::headers::downloader::SyncTarget;
use reth_node_types::{BlockTy, BodyTy, NodeTypes, TxTy};
use reth_primitives::{
Account, Block, BlockExt, BlockWithSenders, Bytecode, GotExpected, Receipt, SealedBlock,
Account, BlockExt, BlockWithSenders, Bytecode, GotExpected, Receipt, SealedBlock,
SealedBlockFor, SealedBlockWithSenders, SealedHeader, StaticFileSegment, StorageEntry,
TransactionMeta, TransactionSignedNoHash,
};
@ -1229,7 +1229,7 @@ impl<TX: DbTx + 'static, N: NodeTypesForProvider> BlockReader for DatabaseProvid
.pop()
.ok_or(ProviderError::InvalidStorageOutput)?;
return Ok(Some(Block { header, body }))
return Ok(Some(Self::Block::new(header, body)))
}
}

View File

@ -21,12 +21,13 @@ use reth_blockchain_tree_api::{
};
use reth_chain_state::{ChainInfoTracker, ForkChoiceNotifications, ForkChoiceSubscriptions};
use reth_chainspec::{ChainInfo, EthereumHardforks};
use reth_db::table::Value;
use reth_db_api::models::{AccountBeforeTx, StoredBlockBodyIndices};
use reth_evm::ConfigureEvmEnv;
use reth_node_types::{BlockTy, FullNodePrimitives, NodeTypes, NodeTypesWithDB, TxTy};
use reth_primitives::{
Account, BlockWithSenders, Receipt, SealedBlock, SealedBlockFor, SealedBlockWithSenders,
SealedHeader, TransactionMeta, TransactionSigned,
Account, BlockWithSenders, EthPrimitives, Receipt, SealedBlock, SealedBlockFor,
SealedBlockWithSenders, SealedHeader, TransactionMeta,
};
use reth_prune_types::{PruneCheckpoint, PruneSegment};
use reth_stages_types::{StageCheckpoint, StageId};
@ -77,10 +78,8 @@ where
ChainSpec: EthereumHardforks,
Storage: ChainStorage<Self::Primitives>,
Primitives: FullNodePrimitives<
SignedTx = TransactionSigned,
SignedTx: Value,
BlockHeader = alloy_consensus::Header,
BlockBody = reth_primitives::BlockBody,
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
>,
>,
@ -92,10 +91,8 @@ impl<T> NodeTypesForProvider for T where
ChainSpec: EthereumHardforks,
Storage: ChainStorage<T::Primitives>,
Primitives: FullNodePrimitives<
SignedTx = TransactionSigned,
SignedTx: Value,
BlockHeader = alloy_consensus::Header,
BlockBody = reth_primitives::BlockBody,
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
>,
>
@ -108,9 +105,18 @@ where
Self: NodeTypesForProvider + NodeTypesWithDB,
{
}
impl<T> ProviderNodeTypes for T where T: NodeTypesForProvider + NodeTypesWithDB {}
/// A helper trait with requirements for [`NodeTypesForProvider`] to be used within legacy
/// blockchain tree.
pub trait NodeTypesForTree: NodeTypesForProvider<Primitives = EthPrimitives> {}
impl<T> NodeTypesForTree for T where T: NodeTypesForProvider<Primitives = EthPrimitives> {}
/// Helper trait with requirements for [`ProviderNodeTypes`] to be used within legacy blockchain
/// tree.
pub trait TreeNodeTypes: ProviderNodeTypes + NodeTypesForTree {}
impl<T> TreeNodeTypes for T where T: ProviderNodeTypes + NodeTypesForTree {}
/// The main type for interacting with the blockchain.
///
/// This type serves as the main entry point for interacting with the blockchain and provides data
@ -342,7 +348,7 @@ impl<N: ProviderNodeTypes> BlockIdReader for BlockchainProvider<N> {
}
}
impl<N: ProviderNodeTypes> BlockReader for BlockchainProvider<N> {
impl<N: TreeNodeTypes> BlockReader for BlockchainProvider<N> {
type Block = BlockTy<N>;
fn find_block_by_hash(

View File

@ -1,8 +1,9 @@
//! Support for maintaining the blob pool.
use alloy_eips::eip2718::Encodable2718;
use alloy_primitives::{BlockNumber, B256};
use reth_execution_types::ChainBlocks;
use reth_primitives_traits::BlockBody as _;
use reth_primitives_traits::{Block, BlockBody, SignedTransaction, TxType};
use std::collections::BTreeMap;
/// The type that is used to track canonical blob transactions.
@ -38,14 +39,17 @@ impl BlobStoreCanonTracker {
///
/// Note: In case this is a chain that's part of a reorg, this replaces previously tracked
/// blocks.
pub fn add_new_chain_blocks(&mut self, blocks: &ChainBlocks<'_>) {
pub fn add_new_chain_blocks<B>(&mut self, blocks: &ChainBlocks<'_, B>)
where
B: Block<Body: BlockBody<Transaction: SignedTransaction>>,
{
let blob_txs = blocks.iter().map(|(num, block)| {
let iter = block
.body
.transactions()
.iter()
.filter(|tx| tx.transaction.is_eip4844())
.map(|tx| tx.hash());
.filter(|tx| tx.tx_type().is_eip4844())
.map(|tx| tx.trie_hash());
(*num, iter)
});
self.add_blocks(blob_txs);