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

@ -50,6 +50,7 @@ tokio-util = { workspace = true }
tokio = { workspace = true, features = ["rt", "rt-multi-thread"] }
[dev-dependencies]
reth-primitives-traits.workspace = true
reth-chainspec.workspace = true
reth-network-api.workspace = true
reth-network-peers.workspace = true

View File

@ -8,10 +8,12 @@ use alloy_rpc_types_engine::{
};
use jsonrpsee::core::client::{ClientT, SubscriptionClientT};
use reth_ethereum_engine_primitives::EthEngineTypes;
use reth_primitives::{Block, BlockExt, TransactionSigned};
use reth_primitives::{Block, TransactionSigned};
use reth_primitives_traits::block::Block as _;
use reth_rpc_api::clients::EngineApiClient;
use reth_rpc_layer::JwtSecret;
use reth_rpc_types_compat::engine::payload::block_to_payload_v1;
#[allow(unused_must_use)]
async fn test_basic_engine_calls<C>(client: &C)
where
@ -23,7 +25,7 @@ where
client,
ExecutionPayloadInputV2 {
execution_payload: ExecutionPayloadV1::from_block_slow::<TransactionSigned>(
&block.unseal(),
&block.into_block(),
),
withdrawals: None,
},

View File

@ -1150,12 +1150,14 @@ mod tests {
start..=start + count - 1,
BlockRangeParams { tx_count: 0..2, ..Default::default() },
);
handle.provider.extend_blocks(blocks.iter().cloned().map(|b| (b.hash(), b.unseal())));
handle
.provider
.extend_blocks(blocks.iter().cloned().map(|b| (b.hash(), b.into_block())));
let expected = blocks
.iter()
.cloned()
.map(|b| Some(ExecutionPayloadBodyV1::from_block(b.unseal::<Block>())))
.map(|b| Some(ExecutionPayloadBodyV1::from_block(b.into_block())))
.collect::<Vec<_>>();
let res = api.get_payload_bodies_by_range_v1(start, count).await.unwrap();
@ -1184,7 +1186,7 @@ mod tests {
!first_missing_range.contains(&b.number) &&
!second_missing_range.contains(&b.number)
})
.map(|b| (b.hash(), b.clone().unseal())),
.map(|b| (b.hash(), b.clone().into_block())),
);
let expected = blocks
@ -1197,7 +1199,7 @@ mod tests {
if first_missing_range.contains(&b.number) {
None
} else {
Some(ExecutionPayloadBodyV1::from_block(b.unseal::<Block>()))
Some(ExecutionPayloadBodyV1::from_block(b.into_block()))
}
})
.collect::<Vec<_>>();
@ -1216,7 +1218,7 @@ mod tests {
{
None
} else {
Some(ExecutionPayloadBodyV1::from_block(b.unseal::<Block>()))
Some(ExecutionPayloadBodyV1::from_block(b.into_block()))
}
})
.collect::<Vec<_>>();
@ -1290,7 +1292,7 @@ mod tests {
// Add block and to provider local store and test for mismatch
handle.provider.add_block(
execution_terminal_block.hash(),
execution_terminal_block.clone().unseal(),
execution_terminal_block.clone().into_block(),
);
let res = api.exchange_transition_configuration(transition_config);
@ -1320,7 +1322,7 @@ mod tests {
terminal_block_number,
};
handle.provider.add_block(terminal_block.hash(), terminal_block.unseal());
handle.provider.add_block(terminal_block.hash(), terminal_block.into_block());
let config = api.exchange_transition_configuration(transition_config).unwrap();
assert_eq!(config, transition_config);

View File

@ -15,13 +15,17 @@ use reth_testing_utils::generators::{
};
fn transform_block<F: FnOnce(Block) -> Block>(src: SealedBlock, f: F) -> ExecutionPayload {
let unsealed = src.unseal();
let unsealed = src.into_block();
let mut transformed: Block = f(unsealed);
// Recalculate roots
transformed.header.transactions_root =
proofs::calculate_transaction_root(&transformed.body.transactions);
transformed.header.ommers_hash = proofs::calculate_ommers_root(&transformed.body.ommers);
block_to_payload(SealedBlock::new(SealedHeader::seal(transformed.header), transformed.body)).0
block_to_payload(SealedBlock::from_sealed_parts(
SealedHeader::seal_slow(transformed.header),
transformed.body,
))
.0
}
#[test]
@ -33,7 +37,7 @@ fn payload_body_roundtrip() {
BlockRangeParams { tx_count: 0..2, ..Default::default() },
) {
let payload_body: ExecutionPayloadBodyV1 =
ExecutionPayloadBodyV1::from_block(block.clone().unseal::<Block>());
ExecutionPayloadBodyV1::from_block(block.clone().into_block());
assert_eq!(
Ok(block.body().transactions.clone()),

View File

@ -1,33 +1,30 @@
//! Database access for `eth_` block RPC methods. Loads block and receipt data w.r.t. network.
use std::sync::Arc;
use super::{LoadPendingBlock, LoadReceipt, SpawnBlocking};
use crate::{
node::RpcNodeCoreExt, EthApiTypes, FromEthApiError, FullEthApiTypes, RpcBlock, RpcNodeCore,
RpcReceipt,
};
use alloy_eips::BlockId;
use alloy_primitives::Sealable;
use alloy_rlp::Encodable;
use alloy_rpc_types_eth::{Block, BlockTransactions, Header, Index};
use futures::Future;
use reth_node_api::BlockBody;
use reth_primitives::{SealedBlockFor, SealedBlockWithSenders};
use reth_primitives::{RecoveredBlock, SealedBlock};
use reth_provider::{
BlockIdReader, BlockReader, BlockReaderIdExt, ProviderHeader, ProviderReceipt,
};
use reth_rpc_types_compat::block::from_block;
use revm_primitives::U256;
use crate::{
node::RpcNodeCoreExt, EthApiTypes, FromEthApiError, FullEthApiTypes, RpcBlock, RpcNodeCore,
RpcReceipt,
};
use super::{LoadPendingBlock, LoadReceipt, SpawnBlocking};
use std::sync::Arc;
/// Result type of the fetched block receipts.
pub type BlockReceiptsResult<N, E> = Result<Option<Vec<RpcReceipt<N>>>, E>;
/// Result type of the fetched block and its receipts.
pub type BlockAndReceiptsResult<Eth> = Result<
Option<(
SealedBlockFor<<<Eth as RpcNodeCore>::Provider as BlockReader>::Block>,
SealedBlock<<<Eth as RpcNodeCore>::Provider as BlockReader>::Block>,
Arc<Vec<ProviderReceipt<<Eth as RpcNodeCore>::Provider>>>,
)>,
<Eth as EthApiTypes>::Error,
@ -62,14 +59,8 @@ pub trait EthBlocks: LoadBlock {
{
async move {
let Some(block) = self.block_with_senders(block_id).await? else { return Ok(None) };
let block_hash = block.hash();
let block = from_block(
(*block).clone().unseal(),
full.into(),
Some(block_hash),
self.tx_resp_builder(),
)?;
let block = from_block((*block).clone(), full.into(), self.tx_resp_builder())?;
Ok(Some(block))
}
}
@ -105,7 +96,7 @@ pub trait EthBlocks: LoadBlock {
.get_sealed_block_with_senders(block_hash)
.await
.map_err(Self::Error::from_eth_err)?
.map(|b| b.body().transactions().len()))
.map(|b| b.body().transaction_count()))
}
}
@ -143,7 +134,7 @@ pub trait EthBlocks: LoadBlock {
// If no pending block from provider, build the pending block locally.
if let Some((block, receipts)) = self.local_pending_block().await? {
return Ok(Some((block.block, Arc::new(receipts))));
return Ok(Some((block.into_sealed_block(), Arc::new(receipts))));
}
}
@ -155,7 +146,7 @@ pub trait EthBlocks: LoadBlock {
.get_block_and_receipts(block_hash)
.await
.map_err(Self::Error::from_eth_err)
.map(|b| b.map(|(b, r)| (b.block.clone(), r)))
.map(|b| b.map(|(b, r)| (b.clone_sealed_block(), r)))
}
Ok(None)
@ -219,7 +210,7 @@ pub trait LoadBlock: LoadPendingBlock + SpawnBlocking + RpcNodeCoreExt {
block_id: BlockId,
) -> impl Future<
Output = Result<
Option<Arc<SealedBlockWithSenders<<Self::Provider as BlockReader>::Block>>>,
Option<Arc<RecoveredBlock<<Self::Provider as BlockReader>::Block>>>,
Self::Error,
>,
> + Send {

View File

@ -1,5 +1,7 @@
//! Loads fee history from database. Helper trait for `eth_` fee and transaction RPC methods.
use super::LoadBlock;
use crate::FromEthApiError;
use alloy_consensus::BlockHeader;
use alloy_eips::eip7840::BlobParams;
use alloy_primitives::U256;
@ -14,10 +16,6 @@ use reth_rpc_eth_types::{
};
use tracing::debug;
use crate::FromEthApiError;
use super::LoadBlock;
/// Fee related functions for the [`EthApiServer`](crate::EthApiServer) trait in the
/// `eth_` namespace.
pub trait EthFees: LoadFee {

View File

@ -15,7 +15,7 @@ use reth_evm::{
env::EvmEnv, state_change::post_block_withdrawals_balance_increments,
system_calls::SystemCaller, ConfigureEvm, ConfigureEvmEnv, NextBlockEnvAttributes,
};
use reth_primitives::{BlockExt, InvalidTransactionError, SealedBlockWithSenders};
use reth_primitives::{InvalidTransactionError, RecoveredBlock};
use reth_primitives_traits::Receipt;
use reth_provider::{
BlockReader, BlockReaderIdExt, ChainSpecProvider, ProviderBlock, ProviderError, ProviderHeader,
@ -133,7 +133,7 @@ pub trait LoadPendingBlock:
) -> impl Future<
Output = Result<
Option<(
SealedBlockWithSenders<<Self::Provider as BlockReader>::Block>,
RecoveredBlock<<Self::Provider as BlockReader>::Block>,
Vec<ProviderReceipt<Self::Provider>>,
)>,
Self::Error,
@ -247,10 +247,7 @@ pub trait LoadPendingBlock:
block_env: BlockEnv,
parent_hash: B256,
) -> Result<
(
SealedBlockWithSenders<ProviderBlock<Self::Provider>>,
Vec<ProviderReceipt<Self::Provider>>,
),
(RecoveredBlock<ProviderBlock<Self::Provider>>, Vec<ProviderReceipt<Self::Provider>>),
Self::Error,
>
where
@ -426,6 +423,6 @@ pub trait LoadPendingBlock:
results,
);
Ok((SealedBlockWithSenders::new_unchecked(block.seal_slow(), senders), receipts))
Ok((RecoveredBlock::new_unhashed(block, senders), receipts))
}
}

View File

@ -1,7 +1,6 @@
//! Loads a pending block from database. Helper trait for `eth_` call and trace RPC methods.
use std::{fmt::Display, sync::Arc};
use super::{Call, LoadBlock, LoadPendingBlock, LoadState, LoadTransaction};
use crate::{FromEvmError, RpcNodeCore};
use alloy_consensus::BlockHeader;
use alloy_primitives::B256;
@ -9,7 +8,7 @@ use alloy_rpc_types_eth::{BlockId, TransactionInfo};
use futures::Future;
use reth_chainspec::ChainSpecProvider;
use reth_evm::{env::EvmEnv, system_calls::SystemCaller, ConfigureEvm, ConfigureEvmEnv};
use reth_primitives::SealedBlockWithSenders;
use reth_primitives::RecoveredBlock;
use reth_primitives_traits::{BlockBody, SignedTransaction};
use reth_provider::{BlockReader, ProviderBlock, ProviderHeader, ProviderTx};
use reth_revm::database::StateProviderDatabase;
@ -22,8 +21,7 @@ use revm_inspectors::tracing::{TracingInspector, TracingInspectorConfig};
use revm_primitives::{
BlockEnv, CfgEnvWithHandlerCfg, EnvWithHandlerCfg, EvmState, ExecutionResult, ResultAndState,
};
use super::{Call, LoadBlock, LoadPendingBlock, LoadState, LoadTransaction};
use std::{fmt::Display, sync::Arc};
/// Executes CPU heavy tasks.
pub trait Trace:
@ -246,7 +244,7 @@ pub trait Trace:
fn trace_block_until<F, R>(
&self,
block_id: BlockId,
block: Option<Arc<SealedBlockWithSenders<ProviderBlock<Self::Provider>>>>,
block: Option<Arc<RecoveredBlock<ProviderBlock<Self::Provider>>>>,
highest_index: Option<u64>,
config: TracingInspectorConfig,
f: F,
@ -286,7 +284,7 @@ pub trait Trace:
fn trace_block_until_with_inspector<Setup, Insp, F, R>(
&self,
block_id: BlockId,
block: Option<Arc<SealedBlockWithSenders<ProviderBlock<Self::Provider>>>>,
block: Option<Arc<RecoveredBlock<ProviderBlock<Self::Provider>>>>,
highest_index: Option<u64>,
mut inspector_setup: Setup,
f: F,
@ -350,7 +348,7 @@ pub trait Trace:
// prepare transactions, we do everything upfront to reduce time spent with open
// state
let max_transactions =
highest_index.map_or(block.body().transactions().len(), |highest| {
highest_index.map_or(block.body().transaction_count(), |highest| {
// we need + 1 because the index is 0-based
highest as usize + 1
});
@ -413,7 +411,7 @@ pub trait Trace:
fn trace_block_with<F, R>(
&self,
block_id: BlockId,
block: Option<Arc<SealedBlockWithSenders<ProviderBlock<Self::Provider>>>>,
block: Option<Arc<RecoveredBlock<ProviderBlock<Self::Provider>>>>,
config: TracingInspectorConfig,
f: F,
) -> impl Future<Output = Result<Option<Vec<R>>, Self::Error>> + Send
@ -452,7 +450,7 @@ pub trait Trace:
fn trace_block_inspector<Setup, Insp, F, R>(
&self,
block_id: BlockId,
block: Option<Arc<SealedBlockWithSenders<ProviderBlock<Self::Provider>>>>,
block: Option<Arc<RecoveredBlock<ProviderBlock<Self::Provider>>>>,
insp_setup: Setup,
f: F,
) -> impl Future<Output = Result<Option<Vec<R>>, Self::Error>> + Send
@ -483,7 +481,7 @@ pub trait Trace:
/// already applied.
fn apply_pre_execution_changes<DB: Send + Database<Error: Display> + DatabaseCommit>(
&self,
block: &SealedBlockWithSenders<ProviderBlock<Self::Provider>>,
block: &RecoveredBlock<ProviderBlock<Self::Provider>>,
db: &mut DB,
cfg: &CfgEnvWithHandlerCfg,
block_env: &BlockEnv,

View File

@ -14,7 +14,7 @@ use alloy_primitives::{Address, Bytes, TxHash, B256};
use alloy_rpc_types_eth::{transaction::TransactionRequest, BlockNumberOrTag, TransactionInfo};
use futures::Future;
use reth_node_api::BlockBody;
use reth_primitives::{transaction::SignedTransactionIntoRecoveredExt, SealedBlockWithSenders};
use reth_primitives::{transaction::SignedTransactionIntoRecoveredExt, RecoveredBlock};
use reth_primitives_traits::SignedTransaction;
use reth_provider::{
BlockNumReader, BlockReaderIdExt, ProviderBlock, ProviderReceipt, ProviderTx, ReceiptProvider,
@ -320,7 +320,7 @@ pub trait EthTransactions: LoadTransaction<Provider: BlockReaderIdExt> {
{
async move {
if let Some(block) = self.block_with_senders(block_id).await? {
if let Some(tx) = block.transactions().get(index) {
if let Some(tx) = block.body().transactions().get(index) {
return Ok(Some(tx.encoded_2718().into()))
}
}
@ -546,7 +546,7 @@ pub trait LoadTransaction: SpawnBlocking + FullEthApiTypes + RpcNodeCoreExt {
Output = Result<
Option<(
TransactionSource<ProviderTx<Self::Provider>>,
Arc<SealedBlockWithSenders<ProviderBlock<Self::Provider>>>,
Arc<RecoveredBlock<ProviderBlock<Self::Provider>>>,
)>,
Self::Error,
>,

View File

@ -7,7 +7,7 @@ use futures::{future::Either, Stream, StreamExt};
use reth_chain_state::CanonStateNotification;
use reth_errors::{ProviderError, ProviderResult};
use reth_execution_types::Chain;
use reth_primitives::{NodePrimitives, SealedBlockWithSenders};
use reth_primitives::{NodePrimitives, RecoveredBlock};
use reth_primitives_traits::{Block, BlockBody};
use reth_storage_api::{BlockReader, StateProviderFactory, TransactionVariant};
use reth_tasks::{TaskSpawner, TokioTaskExecutor};
@ -29,12 +29,12 @@ pub mod db;
pub mod metrics;
pub mod multi_consumer;
/// The type that can send the response to a requested [`SealedBlockWithSenders`]
/// The type that can send the response to a requested [`RecoveredBlock`]
type BlockTransactionsResponseSender<T> = oneshot::Sender<ProviderResult<Option<Vec<T>>>>;
/// The type that can send the response to a requested [`SealedBlockWithSenders`]
/// The type that can send the response to a requested [`RecoveredBlock`]
type BlockWithSendersResponseSender<B> =
oneshot::Sender<ProviderResult<Option<Arc<SealedBlockWithSenders<B>>>>>;
oneshot::Sender<ProviderResult<Option<Arc<RecoveredBlock<B>>>>>;
/// The type that can send the response to the requested receipts of a block.
type ReceiptsResponseSender<R> = oneshot::Sender<ProviderResult<Option<Arc<Vec<R>>>>>;
@ -44,7 +44,7 @@ type HeaderResponseSender<H> = oneshot::Sender<ProviderResult<H>>;
type BlockLruCache<B, L> = MultiConsumerLruCache<
B256,
Arc<SealedBlockWithSenders<B>>,
Arc<RecoveredBlock<B>>,
L,
Either<
BlockWithSendersResponseSender<B>,
@ -76,10 +76,7 @@ impl<R: Send + Sync, B: Block> ActionSender<B, R> {
const fn new(blockhash: B256, tx: Option<UnboundedSender<CacheAction<B, R>>>) -> Self {
Self { blockhash, tx }
}
fn send_block(
&mut self,
block_sender: Result<Option<Arc<SealedBlockWithSenders<B>>>, ProviderError>,
) {
fn send_block(&mut self, block_sender: Result<Option<Arc<RecoveredBlock<B>>>, ProviderError>) {
if let Some(tx) = self.tx.take() {
let _ = tx.send(CacheAction::BlockWithSendersResult {
block_hash: self.blockhash,
@ -191,13 +188,13 @@ impl<B: Block, R: Send + Sync> EthStateCache<B, R> {
this
}
/// Requests the [`SealedBlockWithSenders`] for the block hash
/// Requests the [`RecoveredBlock`] for the block hash
///
/// Returns `None` if the block does not exist.
pub async fn get_sealed_block_with_senders(
&self,
block_hash: B256,
) -> ProviderResult<Option<Arc<SealedBlockWithSenders<B>>>> {
) -> ProviderResult<Option<Arc<RecoveredBlock<B>>>> {
let (response_tx, rx) = oneshot::channel();
let _ = self.to_service.send(CacheAction::GetBlockWithSenders { block_hash, response_tx });
rx.await.map_err(|_| ProviderError::CacheServiceUnavailable)?
@ -216,7 +213,7 @@ impl<B: Block, R: Send + Sync> EthStateCache<B, R> {
pub async fn get_block_and_receipts(
&self,
block_hash: B256,
) -> ProviderResult<Option<(Arc<SealedBlockWithSenders<B>>, Arc<Vec<R>>)>> {
) -> ProviderResult<Option<(Arc<RecoveredBlock<B>>, Arc<Vec<R>>)>> {
let block = self.get_sealed_block_with_senders(block_hash);
let receipts = self.get_receipts(block_hash);
@ -260,7 +257,7 @@ pub(crate) struct EthStateCacheService<
LimitHeaders = ByLength,
> where
Provider: BlockReader,
LimitBlocks: Limiter<B256, Arc<SealedBlockWithSenders<Provider::Block>>>,
LimitBlocks: Limiter<B256, Arc<RecoveredBlock<Provider::Block>>>,
LimitReceipts: Limiter<B256, Arc<Vec<Provider::Receipt>>>,
LimitHeaders: Limiter<B256, Provider::Header>,
{
@ -293,7 +290,7 @@ where
fn on_new_block(
&mut self,
block_hash: B256,
res: ProviderResult<Option<Arc<SealedBlockWithSenders<Provider::Block>>>>,
res: ProviderResult<Option<Arc<RecoveredBlock<Provider::Block>>>>,
) {
if let Some(queued) = self.full_block_cache.remove(&block_hash) {
// send the response to queued senders
@ -304,7 +301,7 @@ where
}
Either::Right(transaction_tx) => {
let _ = transaction_tx.send(res.clone().map(|maybe_block| {
maybe_block.map(|block| block.block.body().transactions().to_vec())
maybe_block.map(|block| block.body().transactions().to_vec())
}));
}
}
@ -338,7 +335,7 @@ where
fn on_reorg_block(
&mut self,
block_hash: B256,
res: ProviderResult<Option<SealedBlockWithSenders<Provider::Block>>>,
res: ProviderResult<Option<RecoveredBlock<Provider::Block>>>,
) {
let res = res.map(|b| b.map(Arc::new));
if let Some(queued) = self.full_block_cache.remove(&block_hash) {
@ -350,7 +347,7 @@ where
}
Either::Right(transaction_tx) => {
let _ = transaction_tx.send(res.clone().map(|maybe_block| {
maybe_block.map(|block| block.block.body().transactions().to_vec())
maybe_block.map(|block| block.body().transactions().to_vec())
}));
}
}
@ -541,36 +538,14 @@ where
/// All message variants sent through the channel
enum CacheAction<B: Block, R> {
GetBlockWithSenders {
block_hash: B256,
response_tx: BlockWithSendersResponseSender<B>,
},
GetHeader {
block_hash: B256,
response_tx: HeaderResponseSender<B::Header>,
},
GetReceipts {
block_hash: B256,
response_tx: ReceiptsResponseSender<R>,
},
BlockWithSendersResult {
block_hash: B256,
res: ProviderResult<Option<Arc<SealedBlockWithSenders<B>>>>,
},
ReceiptsResult {
block_hash: B256,
res: ProviderResult<Option<Arc<Vec<R>>>>,
},
HeaderResult {
block_hash: B256,
res: Box<ProviderResult<B::Header>>,
},
CacheNewCanonicalChain {
chain_change: ChainChange<B, R>,
},
RemoveReorgedChain {
chain_change: ChainChange<B, R>,
},
GetBlockWithSenders { block_hash: B256, response_tx: BlockWithSendersResponseSender<B> },
GetHeader { block_hash: B256, response_tx: HeaderResponseSender<B::Header> },
GetReceipts { block_hash: B256, response_tx: ReceiptsResponseSender<R> },
BlockWithSendersResult { block_hash: B256, res: ProviderResult<Option<Arc<RecoveredBlock<B>>>> },
ReceiptsResult { block_hash: B256, res: ProviderResult<Option<Arc<Vec<R>>>> },
HeaderResult { block_hash: B256, res: Box<ProviderResult<B::Header>> },
CacheNewCanonicalChain { chain_change: ChainChange<B, R> },
RemoveReorgedChain { chain_change: ChainChange<B, R> },
}
struct BlockReceipts<R> {
@ -580,7 +555,7 @@ struct BlockReceipts<R> {
/// A change of the canonical chain
struct ChainChange<B: Block, R> {
blocks: Vec<SealedBlockWithSenders<B>>,
blocks: Vec<RecoveredBlock<B>>,
receipts: Vec<BlockReceipts<R>>,
}
@ -593,7 +568,7 @@ impl<B: Block, R: Clone> ChainChange<B, R> {
.blocks_and_receipts()
.map(|(block, receipts)| {
let block_receipts =
BlockReceipts { block_hash: block.block.hash(), receipts: receipts.clone() };
BlockReceipts { block_hash: block.hash(), receipts: receipts.clone() };
(block.clone(), block_receipts)
})
.unzip();

View File

@ -18,7 +18,7 @@ use metrics::atomics::AtomicU64;
use reth_chain_state::CanonStateNotification;
use reth_chainspec::{ChainSpecProvider, EthChainSpec};
use reth_primitives::{NodePrimitives, SealedBlock};
use reth_primitives_traits::BlockBody;
use reth_primitives_traits::{Block, BlockBody};
use reth_rpc_server_types::constants::gas_oracle::MAX_HEADER_HISTORY;
use reth_storage_api::BlockReaderIdExt;
use serde::{Deserialize, Serialize};
@ -72,12 +72,11 @@ impl FeeHistoryCache {
}
/// Insert block data into the cache.
async fn insert_blocks<'a, I, H, B, R>(&self, blocks: I)
async fn insert_blocks<'a, I, B, R>(&self, blocks: I)
where
H: BlockHeader + 'a,
B: BlockBody,
B: Block + 'a,
R: TxReceipt,
I: IntoIterator<Item = (&'a SealedBlock<H, B>, Arc<Vec<R>>)>,
I: IntoIterator<Item = (&'a SealedBlock<B>, Arc<Vec<R>>)>,
{
let mut entries = self.inner.entries.write().await;
@ -236,9 +235,9 @@ pub async fn fee_history_cache_new_blocks_task<St, Provider, N>(
tokio::select! {
res = &mut fetch_missing_block => {
if let Ok(res) = res {
fee_history_cache.insert_blocks(res.as_ref()
.map(|(b, r)| (&b.block, r.clone()))
.into_iter()).await;
let res = res.as_ref()
.map(|(b, r)| (b.sealed_block(), r.clone()));
fee_history_cache.insert_blocks(res).await;
}
}
event = events.next() => {
@ -251,10 +250,10 @@ pub async fn fee_history_cache_new_blocks_task<St, Provider, N>(
let (blocks, receipts): (Vec<_>, Vec<_>) = committed
.blocks_and_receipts()
.map(|(block, receipts)| {
(&block.block, Arc::new(receipts.iter().flatten().cloned().collect::<Vec<_>>()))
(block.clone_sealed_block(), Arc::new(receipts.iter().flatten().cloned().collect::<Vec<_>>()))
})
.unzip();
fee_history_cache.insert_blocks(blocks.into_iter().zip(receipts)).await;
fee_history_cache.insert_blocks(blocks.iter().zip(receipts)).await;
// keep track of missing blocks
missing_blocks = fee_history_cache.missing_consecutive_blocks().await;
@ -363,22 +362,23 @@ impl FeeHistoryEntry {
/// Creates a new entry from a sealed block.
///
/// Note: This does not calculate the rewards for the block.
pub fn new<H: BlockHeader, B: BlockBody>(block: &SealedBlock<H, B>) -> Self {
pub fn new<B: Block>(block: &SealedBlock<B>) -> Self {
Self {
base_fee_per_gas: block.base_fee_per_gas().unwrap_or_default(),
gas_used_ratio: block.gas_used() as f64 / block.gas_limit() as f64,
base_fee_per_gas: block.header().base_fee_per_gas().unwrap_or_default(),
gas_used_ratio: block.header().gas_used() as f64 / block.header().gas_limit() as f64,
base_fee_per_blob_gas: block
.header()
.excess_blob_gas()
.map(alloy_eips::eip4844::calc_blob_gasprice),
blob_gas_used_ratio: block.body().blob_gas_used() as f64 /
alloy_eips::eip4844::MAX_DATA_GAS_PER_BLOCK as f64,
excess_blob_gas: block.excess_blob_gas(),
blob_gas_used: block.blob_gas_used(),
gas_used: block.gas_used(),
excess_blob_gas: block.header().excess_blob_gas(),
blob_gas_used: block.header().blob_gas_used(),
gas_used: block.header().gas_used(),
header_hash: block.hash(),
gas_limit: block.gas_limit(),
gas_limit: block.header().gas_limit(),
rewards: Vec::new(),
timestamp: block.timestamp(),
timestamp: block.header().timestamp(),
}
}

View File

@ -8,7 +8,7 @@ use alloy_primitives::TxHash;
use alloy_rpc_types_eth::{FilteredParams, Log};
use reth_chainspec::ChainInfo;
use reth_errors::ProviderError;
use reth_primitives::SealedBlockWithSenders;
use reth_primitives::RecoveredBlock;
use reth_primitives_traits::{BlockBody, SignedTransaction};
use reth_storage_api::{BlockReader, ProviderBlock};
use std::sync::Arc;
@ -55,8 +55,8 @@ where
pub enum ProviderOrBlock<'a, P: BlockReader> {
/// Provider
Provider(&'a P),
/// [`SealedBlockWithSenders`]
Block(Arc<SealedBlockWithSenders<ProviderBlock<P>>>),
/// [`RecoveredBlock`]
Block(Arc<RecoveredBlock<ProviderBlock<P>>>),
}
/// Appends all matching logs of a block's receipts.

View File

@ -8,7 +8,7 @@ use alloy_consensus::BlockHeader;
use alloy_eips::{BlockId, BlockNumberOrTag};
use alloy_primitives::B256;
use derive_more::Constructor;
use reth_primitives::{Receipt, SealedBlockWithSenders};
use reth_primitives::{Receipt, RecoveredBlock};
use reth_primitives_traits::Block;
use revm_primitives::{BlockEnv, CfgEnvWithHandlerCfg};
@ -27,7 +27,7 @@ pub struct PendingBlockEnv<B: Block = reth_primitives::Block, R = Receipt> {
#[derive(Clone, Debug)]
pub enum PendingBlockEnvOrigin<B: Block = reth_primitives::Block, R = Receipt> {
/// The pending block as received from the CL.
ActualPending(SealedBlockWithSenders<B>, Vec<R>),
ActualPending(RecoveredBlock<B>, Vec<R>),
/// The _modified_ header of the latest block.
///
/// This derives the pending state based on the latest header by modifying:
@ -44,7 +44,7 @@ impl<B: Block, R> PendingBlockEnvOrigin<B, R> {
}
/// Consumes the type and returns the actual pending block.
pub fn into_actual_pending(self) -> Option<SealedBlockWithSenders<B>> {
pub fn into_actual_pending(self) -> Option<RecoveredBlock<B>> {
match self {
Self::ActualPending(block, _) => Some(block),
_ => None,
@ -81,7 +81,7 @@ pub struct PendingBlock<B: Block, R> {
/// Timestamp when the pending block is considered outdated.
pub expires_at: Instant,
/// The locally built pending block.
pub block: SealedBlockWithSenders<B>,
pub block: RecoveredBlock<B>,
/// The receipts for the pending block
pub receipts: Vec<R>,
}

View File

@ -7,7 +7,7 @@ use alloy_rpc_types_eth::{
Block, BlockTransactionsKind, Header,
};
use jsonrpsee_types::ErrorObject;
use reth_primitives::BlockWithSenders;
use reth_primitives::RecoveredBlock;
use reth_primitives_traits::{block::BlockTx, BlockBody as _, SignedTransaction};
use reth_rpc_server_types::result::rpc_err;
use reth_rpc_types_compat::{block::from_block, TransactionCompat};
@ -178,11 +178,11 @@ where
calls.push(call);
}
let block = BlockWithSenders::new_unchecked(block, senders);
let block = RecoveredBlock::new_unhashed(block, senders);
let txs_kind =
if full_transactions { BlockTransactionsKind::Full } else { BlockTransactionsKind::Hashes };
let block = from_block(block, txs_kind, None, tx_resp_builder)?;
let block = from_block(block, txs_kind, tx_resp_builder)?;
Ok(SimulatedBlock { inner: block, calls })
}

View File

@ -1,15 +1,14 @@
//! Compatibility functions for rpc `Block` type.
use alloy_consensus::{BlockHeader, Sealable, Sealed};
use crate::transaction::TransactionCompat;
use alloy_consensus::{BlockHeader, Sealable};
use alloy_eips::eip4895::Withdrawals;
use alloy_primitives::{B256, U256};
use alloy_primitives::U256;
use alloy_rpc_types_eth::{
Block, BlockTransactions, BlockTransactionsKind, Header, TransactionInfo,
};
use reth_primitives::{transaction::SignedTransactionIntoRecoveredExt, BlockWithSenders};
use reth_primitives_traits::{Block as BlockTrait, BlockBody, SignedTransaction};
use crate::transaction::TransactionCompat;
use reth_primitives::{transaction::SignedTransactionIntoRecoveredExt, RecoveredBlock};
use reth_primitives_traits::{Block as BlockTrait, BlockBody, SealedHeader, SignedTransaction};
/// Converts the given primitive block into a [`Block`] response with the given
/// [`BlockTransactionsKind`]
@ -17,9 +16,8 @@ use crate::transaction::TransactionCompat;
/// If a `block_hash` is provided, then this is used, otherwise the block hash is computed.
#[expect(clippy::type_complexity)]
pub fn from_block<T, B>(
block: BlockWithSenders<B>,
block: RecoveredBlock<B>,
kind: BlockTransactionsKind,
block_hash: Option<B256>,
tx_resp_builder: &T,
) -> Result<Block<T::Transaction, Header<B::Header>>, T::Error>
where
@ -27,10 +25,8 @@ where
B: BlockTrait,
{
match kind {
BlockTransactionsKind::Hashes => {
Ok(from_block_with_tx_hashes::<T::Transaction, B>(block, block_hash))
}
BlockTransactionsKind::Full => from_block_full::<T, B>(block, block_hash, tx_resp_builder),
BlockTransactionsKind::Hashes => Ok(from_block_with_tx_hashes::<T::Transaction, B>(block)),
BlockTransactionsKind::Full => from_block_full::<T, B>(block, tx_resp_builder),
}
}
@ -39,20 +35,17 @@ where
///
/// This will populate the `transactions` field with only the hashes of the transactions in the
/// block: [`BlockTransactions::Hashes`]
pub fn from_block_with_tx_hashes<T, B>(
block: BlockWithSenders<B>,
block_hash: Option<B256>,
) -> Block<T, Header<B::Header>>
pub fn from_block_with_tx_hashes<T, B>(block: RecoveredBlock<B>) -> Block<T, Header<B::Header>>
where
B: BlockTrait,
{
let block_hash = block_hash.unwrap_or_else(|| block.header().hash_slow());
let transactions = block.body().transaction_hashes_iter().copied().collect();
from_block_with_transactions(
block.length(),
block_hash,
block.block,
let rlp_length = block.rlp_length();
let (header, body) = block.into_sealed_block().split_sealed_header_body();
from_block_with_transactions::<T, B>(
rlp_length,
header,
body,
BlockTransactions::Hashes(transactions),
)
}
@ -64,23 +57,22 @@ where
/// [`TransactionCompat::Transaction`] objects: [`BlockTransactions::Full`]
#[expect(clippy::type_complexity)]
pub fn from_block_full<T, B>(
block: BlockWithSenders<B>,
block_hash: Option<B256>,
block: RecoveredBlock<B>,
tx_resp_builder: &T,
) -> Result<Block<T::Transaction, Header<B::Header>>, T::Error>
where
T: TransactionCompat<<<B as BlockTrait>::Body as BlockBody>::Transaction>,
B: BlockTrait,
{
let block_hash = block_hash.unwrap_or_else(|| block.block.header().hash_slow());
let block_number = block.block.header().number();
let base_fee_per_gas = block.block.header().base_fee_per_gas();
let block_number = block.header().number();
let base_fee_per_gas = block.header().base_fee_per_gas();
// NOTE: we can safely remove the body here because not needed to finalize the `Block` in
// `from_block_with_transactions`, however we need to compute the length before
let block_length = block.block.length();
let transactions = block.block.body().transactions().to_vec();
let block_length = block.rlp_length();
let transactions = block.body().transactions().to_vec();
let transactions_with_senders = transactions.into_iter().zip(block.senders_iter().copied());
let block_hash = Some(block.hash());
let transactions = transactions_with_senders
.enumerate()
.map(|(idx, (tx, sender))| {
@ -88,7 +80,7 @@ where
let signed_tx_ec_recovered = tx.with_signer(sender);
let tx_info = TransactionInfo {
hash: Some(tx_hash),
block_hash: Some(block_hash),
block_hash,
block_number: Some(block_number),
base_fee: base_fee_per_gas.map(u128::from),
index: Some(idx as u64),
@ -98,10 +90,11 @@ where
})
.collect::<Result<Vec<_>, T::Error>>()?;
Ok(from_block_with_transactions(
let (header, body) = block.into_sealed_block().split_sealed_header_body();
Ok(from_block_with_transactions::<_, B>(
block_length,
block_hash,
block.block,
header,
body,
BlockTransactions::Full(transactions),
))
}
@ -109,28 +102,19 @@ where
#[inline]
fn from_block_with_transactions<T, B: BlockTrait>(
block_length: usize,
block_hash: B256,
block: B,
header: SealedHeader<B::Header>,
body: B::Body,
transactions: BlockTransactions<T>,
) -> Block<T, Header<B::Header>> {
let withdrawals = block
.header()
let withdrawals = header
.withdrawals_root()
.is_some()
.then(|| block.body().withdrawals().cloned().map(Withdrawals::into_inner).map(Into::into))
.then(|| body.withdrawals().cloned().map(Withdrawals::into_inner).map(Into::into))
.flatten();
let uncles = block
.body()
.ommers()
.map(|o| o.iter().map(|h| h.hash_slow()).collect())
.unwrap_or_default();
let (header, _) = block.split();
let header = Header::from_consensus(
Sealed::new_unchecked(header, block_hash),
None,
Some(U256::from(block_length)),
);
let uncles =
body.ommers().map(|o| o.iter().map(|h| h.hash_slow()).collect()).unwrap_or_default();
let header = Header::from_consensus(header.into(), None, Some(U256::from(block_length)));
Block { header, uncles, transactions, withdrawals }
}

View File

@ -1,7 +1,6 @@
//! Standalone Conversion Functions for Handling Different Versions of Execution Payloads in
//! Ethereum's Engine
use alloy_consensus::Header;
use alloy_eips::{eip2718::Encodable2718, eip4895::Withdrawals, eip7685::RequestsOrHash};
use alloy_primitives::U256;
use alloy_rpc_types_engine::{
@ -9,12 +8,14 @@ use alloy_rpc_types_engine::{
CancunPayloadFields, ExecutionPayload, ExecutionPayloadSidecar, ExecutionPayloadV1,
ExecutionPayloadV2, ExecutionPayloadV3, PraguePayloadFields,
};
use reth_primitives::{BlockBody, SealedBlock};
use reth_primitives::{Block, SealedBlock};
use reth_primitives_traits::{BlockBody as _, SignedTransaction};
/// Converts [`SealedBlock`] to [`ExecutionPayload`]
/// Converts [`SealedBlock`] to [`ExecutionPayload`].
///
/// TODO(mattsse): remove after next alloy bump
pub fn block_to_payload<T: SignedTransaction>(
value: SealedBlock<Header, BlockBody<T>>,
value: SealedBlock<Block<T>>,
) -> (ExecutionPayload, ExecutionPayloadSidecar) {
let cancun =
value.parent_beacon_block_root.map(|parent_beacon_block_root| CancunPayloadFields {
@ -47,8 +48,8 @@ pub fn block_to_payload<T: SignedTransaction>(
}
/// Converts [`SealedBlock`] to [`ExecutionPayloadV1`]
pub fn block_to_payload_v1<T: Encodable2718>(
value: SealedBlock<Header, BlockBody<T>>,
pub fn block_to_payload_v1<T: SignedTransaction>(
value: SealedBlock<Block<T>>,
) -> ExecutionPayloadV1 {
let transactions =
value.body().transactions.iter().map(|tx| tx.encoded_2718().into()).collect::<Vec<_>>();
@ -71,8 +72,8 @@ pub fn block_to_payload_v1<T: Encodable2718>(
}
/// Converts [`SealedBlock`] to [`ExecutionPayloadV2`]
pub fn block_to_payload_v2<T: Encodable2718>(
value: SealedBlock<Header, BlockBody<T>>,
pub fn block_to_payload_v2<T: SignedTransaction>(
value: SealedBlock<Block<T>>,
) -> ExecutionPayloadV2 {
ExecutionPayloadV2 {
withdrawals: value.body().withdrawals.clone().unwrap_or_default().into_inner(),
@ -81,8 +82,8 @@ pub fn block_to_payload_v2<T: Encodable2718>(
}
/// Converts [`SealedBlock`] to [`ExecutionPayloadV3`], and returns the parent beacon block root.
pub fn block_to_payload_v3<T: Encodable2718>(
value: SealedBlock<Header, BlockBody<T>>,
pub fn block_to_payload_v3<T: SignedTransaction>(
value: SealedBlock<Block<T>>,
) -> ExecutionPayloadV3 {
ExecutionPayloadV3 {
blob_gas_used: value.blob_gas_used.unwrap_or_default(),
@ -92,8 +93,8 @@ pub fn block_to_payload_v3<T: Encodable2718>(
}
/// Converts [`SealedBlock`] to [`ExecutionPayloadFieldV2`]
pub fn convert_block_to_payload_field_v2<T: Encodable2718>(
value: SealedBlock<Header, BlockBody<T>>,
pub fn convert_block_to_payload_field_v2<T: SignedTransaction>(
value: SealedBlock<Block<T>>,
) -> ExecutionPayloadFieldV2 {
// if there are withdrawals, return V2
if value.body().withdrawals.is_some() {
@ -122,7 +123,8 @@ mod tests {
CancunPayloadFields, ExecutionPayload, ExecutionPayloadSidecar, ExecutionPayloadV1,
ExecutionPayloadV2, ExecutionPayloadV3,
};
use reth_primitives::{Block, BlockExt, TransactionSigned};
use reth_primitives::{Block, TransactionSigned};
use reth_primitives_traits::Block as _;
#[test]
fn roundtrip_payload_to_block() {

View File

@ -20,7 +20,7 @@ use reth_evm::{
execute::{BlockExecutorProvider, Executor},
ConfigureEvmEnv,
};
use reth_primitives::{BlockExt, NodePrimitives, ReceiptWithBloom, SealedBlockWithSenders};
use reth_primitives::{NodePrimitives, ReceiptWithBloom, RecoveredBlock};
use reth_primitives_traits::{Block as _, BlockBody, SignedTransaction};
use reth_provider::{
BlockIdReader, BlockReaderIdExt, ChainSpecProvider, HeaderProvider, ProviderBlock,
@ -94,7 +94,7 @@ where
/// Trace the entire block asynchronously
async fn trace_block(
&self,
block: Arc<SealedBlockWithSenders<ProviderBlock<Eth::Provider>>>,
block: Arc<RecoveredBlock<ProviderBlock<Eth::Provider>>>,
cfg: CfgEnvWithHandlerCfg,
block_env: BlockEnv,
opts: GethDebugTracingOptions,
@ -192,7 +192,7 @@ where
};
self.trace_block(
Arc::new(block.with_senders_unchecked(senders).seal_slow()),
Arc::new(block.with_senders_unchecked(senders)),
cfg_env_with_handler_cfg,
block_env,
opts,
@ -639,7 +639,7 @@ where
let mut witness_record = ExecutionWitnessRecord::default();
let _ = block_executor
.execute_with_state_closure(&(*block).clone().unseal(), |statedb: &State<_>| {
.execute_with_state_closure(&(*block).clone(), |statedb: &State<_>| {
witness_record.record_executed_state(statedb);
})
.map_err(|err| EthApiError::Internal(err.into()))?;

View File

@ -9,7 +9,7 @@ use alloy_rpc_types_eth::{
use async_trait::async_trait;
use jsonrpsee::{core::RpcResult, server::IdProvider};
use reth_chainspec::ChainInfo;
use reth_primitives::SealedBlockWithSenders;
use reth_primitives::RecoveredBlock;
use reth_provider::{
BlockHashReader, BlockIdReader, BlockNumReader, BlockReader, HeaderProvider, ProviderBlock,
ProviderError, ProviderReceipt,
@ -546,7 +546,7 @@ where
) -> Result<
Option<(
Arc<Vec<ProviderReceipt<Eth::Provider>>>,
Option<Arc<SealedBlockWithSenders<ProviderBlock<Eth::Provider>>>>,
Option<Arc<RecoveredBlock<ProviderBlock<Eth::Provider>>>>,
)>,
EthFilterError,
> {

View File

@ -17,8 +17,8 @@ use reth_consensus::{Consensus, FullConsensus, PostExecutionInput};
use reth_engine_primitives::PayloadValidator;
use reth_errors::{BlockExecutionError, ConsensusError, ProviderError};
use reth_evm::execute::{BlockExecutorProvider, Executor};
use reth_primitives::{GotExpected, NodePrimitives, SealedBlockWithSenders, SealedHeader};
use reth_primitives_traits::{constants::GAS_LIMIT_BOUND_DIVISOR, Block as _, BlockBody};
use reth_primitives::{GotExpected, NodePrimitives, RecoveredBlock, SealedHeader};
use reth_primitives_traits::{constants::GAS_LIMIT_BOUND_DIVISOR, BlockBody, SealedBlock};
use reth_provider::{BlockExecutionOutput, BlockReaderIdExt, StateProviderFactory};
use reth_revm::{cached::CachedReads, database::StateProviderDatabase};
use reth_rpc_api::BlockSubmissionValidationApiServer;
@ -98,7 +98,7 @@ where
/// Validates the given block and a [`BidTrace`] against it.
pub async fn validate_message_against_block(
&self,
block: SealedBlockWithSenders<<E::Primitives as NodePrimitives>::Block>,
block: RecoveredBlock<<E::Primitives as NodePrimitives>::Block>,
message: BidTrace,
registered_gas_limit: u64,
) -> Result<(), ValidationApiError> {
@ -106,7 +106,7 @@ where
self.consensus.validate_header_with_total_difficulty(block.sealed_header(), U256::MAX)?;
self.consensus.validate_header(block.sealed_header())?;
self.consensus.validate_block_pre_execution(&block)?;
self.consensus.validate_block_pre_execution(block.sealed_block())?;
if !self.disallow.is_empty() {
if self.disallow.contains(&block.beneficiary()) {
@ -115,7 +115,7 @@ where
if self.disallow.contains(&message.proposer_fee_recipient) {
return Err(ValidationApiError::Blacklist(message.proposer_fee_recipient))
}
for (sender, tx) in block.senders_iter().zip(block.transactions()) {
for (sender, tx) in block.senders_iter().zip(block.body().transactions()) {
if self.disallow.contains(sender) {
return Err(ValidationApiError::Blacklist(*sender))
}
@ -147,7 +147,6 @@ where
let cached_db = request_cache.as_db_mut(StateProviderDatabase::new(&state_provider));
let executor = self.executor_provider.executor(cached_db);
let block = block.unseal();
let mut accessed_blacklisted = None;
let output = executor.execute_with_state_closure(&block, |state| {
if !self.disallow.is_empty() {
@ -251,7 +250,7 @@ where
/// to checking the latest block transaction.
fn ensure_payment(
&self,
block: &<E::Primitives as NodePrimitives>::Block,
block: &SealedBlock<<E::Primitives as NodePrimitives>::Block>,
output: &BlockExecutionOutput<<E::Primitives as NodePrimitives>::Receipt>,
message: &BidTrace,
) -> Result<(), ValidationApiError> {
@ -349,7 +348,7 @@ where
versioned_hashes: self.validate_blobs_bundle(request.request.blobs_bundle)?,
}),
)?
.try_seal_with_senders()
.try_recover()
.map_err(|_| ValidationApiError::InvalidTransactionSignature)?;
self.validate_message_against_block(
@ -382,7 +381,7 @@ where
},
),
)?
.try_seal_with_senders()
.try_recover()
.map_err(|_| ValidationApiError::InvalidTransactionSignature)?;
self.validate_message_against_block(