chore: use chainspec blob_params for blob calculations (#14139)

This commit is contained in:
joshieDo
2025-02-01 16:41:22 +00:00
committed by GitHub
parent fd4ca7d6d6
commit a041b41b59
7 changed files with 55 additions and 25 deletions

View File

@ -424,7 +424,7 @@ where
} else { } else {
// for the first post-fork block, both parent.blob_gas_used and // for the first post-fork block, both parent.blob_gas_used and
// parent.excess_blob_gas are evaluated as 0 // parent.excess_blob_gas are evaluated as 0
Some(alloy_eips::eip4844::calc_excess_blob_gas(0, 0)) Some(alloy_eips::eip7840::BlobParams::cancun().next_block_excess_blob_gas(0, 0))
}; };
blob_gas_used = Some(sum_blob_gas_used); blob_gas_used = Some(sum_blob_gas_used);

View File

@ -19,6 +19,7 @@ use reth_node_core::{
}; };
use reth_payload_builder::PayloadStore; use reth_payload_builder::PayloadStore;
use reth_primitives::EthPrimitives; use reth_primitives::EthPrimitives;
use reth_provider::ChainSpecProvider;
use reth_rpc::{ use reth_rpc::{
eth::{EthApiTypes, FullEthApiServer}, eth::{EthApiTypes, FullEthApiServer},
EthApi, EthApi,
@ -571,8 +572,12 @@ pub trait EthApiBuilder<N: FullNodeComponents>: 'static {
fn build(ctx: &EthApiBuilderCtx<N>) -> Self; fn build(ctx: &EthApiBuilderCtx<N>) -> Self;
} }
impl<N: FullNodeComponents<Types: NodeTypes<Primitives = EthPrimitives>>> EthApiBuilder<N> impl<
for EthApi<N::Provider, N::Pool, N::Network, N::Evm> N: FullNodeComponents<
Provider: ChainSpecProvider,
Types: NodeTypes<Primitives = EthPrimitives>,
>,
> EthApiBuilder<N> for EthApi<N::Provider, N::Pool, N::Network, N::Evm>
{ {
fn build(ctx: &EthApiBuilderCtx<N>) -> Self { fn build(ctx: &EthApiBuilderCtx<N>) -> Self {
Self::with_spawner(ctx) Self::with_spawner(ctx)

View File

@ -203,7 +203,7 @@ impl OpReceiptBuilder {
) -> Result<Self, OpEthApiError> { ) -> Result<Self, OpEthApiError> {
let timestamp = meta.timestamp; let timestamp = meta.timestamp;
let core_receipt = let core_receipt =
build_receipt(transaction, meta, receipt, all_receipts, |receipt_with_bloom| { build_receipt(transaction, meta, receipt, all_receipts, None, |receipt_with_bloom| {
match receipt { match receipt {
OpReceipt::Legacy(_) => OpReceiptEnvelope::<Log>::Legacy(receipt_with_bloom), OpReceipt::Legacy(_) => OpReceiptEnvelope::<Log>::Legacy(receipt_with_bloom),
OpReceipt::Eip2930(_) => OpReceiptEnvelope::<Log>::Eip2930(receipt_with_bloom), OpReceipt::Eip2930(_) => OpReceiptEnvelope::<Log>::Eip2930(receipt_with_bloom),

View File

@ -7,7 +7,7 @@ use std::{
}; };
use alloy_consensus::{BlockHeader, Transaction, TxReceipt}; use alloy_consensus::{BlockHeader, Transaction, TxReceipt};
use alloy_eips::eip1559::calc_next_block_base_fee; use alloy_eips::{eip1559::calc_next_block_base_fee, eip7840::BlobParams};
use alloy_primitives::B256; use alloy_primitives::B256;
use alloy_rpc_types_eth::TxGasAndReward; use alloy_rpc_types_eth::TxGasAndReward;
use futures::{ use futures::{
@ -72,18 +72,22 @@ impl FeeHistoryCache {
} }
/// Insert block data into the cache. /// Insert block data into the cache.
async fn insert_blocks<'a, I, B, R>(&self, blocks: I) async fn insert_blocks<'a, I, B, R, C>(&self, blocks: I, chain_spec: &C)
where where
B: Block + 'a, B: Block + 'a,
R: TxReceipt, R: TxReceipt,
I: IntoIterator<Item = (&'a SealedBlock<B>, Arc<Vec<R>>)>, I: IntoIterator<Item = (&'a SealedBlock<B>, Arc<Vec<R>>)>,
C: EthChainSpec,
{ {
let mut entries = self.inner.entries.write().await; let mut entries = self.inner.entries.write().await;
let percentiles = self.predefined_percentiles(); let percentiles = self.predefined_percentiles();
// Insert all new blocks and calculate approximated rewards // Insert all new blocks and calculate approximated rewards
for (block, receipts) in blocks { for (block, receipts) in blocks {
let mut fee_history_entry = FeeHistoryEntry::new(block); let mut fee_history_entry = FeeHistoryEntry::new(
block,
chain_spec.blob_params_at_timestamp(block.header().timestamp()),
);
fee_history_entry.rewards = calculate_reward_percentiles_for_block( fee_history_entry.rewards = calculate_reward_percentiles_for_block(
&percentiles, &percentiles,
fee_history_entry.gas_used, fee_history_entry.gas_used,
@ -232,12 +236,14 @@ pub async fn fee_history_cache_new_blocks_task<St, Provider, N>(
} }
} }
let chain_spec = provider.chain_spec();
tokio::select! { tokio::select! {
res = &mut fetch_missing_block => { res = &mut fetch_missing_block => {
if let Ok(res) = res { if let Ok(res) = res {
let res = res.as_ref() let res = res.as_ref()
.map(|(b, r)| (b.sealed_block(), r.clone())); .map(|(b, r)| (b.sealed_block(), r.clone()));
fee_history_cache.insert_blocks(res).await; fee_history_cache.insert_blocks(res, &chain_spec).await;
} }
} }
event = events.next() => { event = events.next() => {
@ -253,7 +259,7 @@ pub async fn fee_history_cache_new_blocks_task<St, Provider, N>(
(block.clone_sealed_block(), Arc::new(receipts.clone())) (block.clone_sealed_block(), Arc::new(receipts.clone()))
}) })
.unzip(); .unzip();
fee_history_cache.insert_blocks(blocks.iter().zip(receipts)).await; fee_history_cache.insert_blocks(blocks.iter().zip(receipts), &chain_spec).await;
// keep track of missing blocks // keep track of missing blocks
missing_blocks = fee_history_cache.missing_consecutive_blocks().await; missing_blocks = fee_history_cache.missing_consecutive_blocks().await;
@ -356,20 +362,22 @@ pub struct FeeHistoryEntry {
pub rewards: Vec<u128>, pub rewards: Vec<u128>,
/// The timestamp of the block. /// The timestamp of the block.
pub timestamp: u64, pub timestamp: u64,
/// Blob parameters for this block.
pub blob_params: Option<BlobParams>,
} }
impl FeeHistoryEntry { impl FeeHistoryEntry {
/// Creates a new entry from a sealed block. /// Creates a new entry from a sealed block.
/// ///
/// Note: This does not calculate the rewards for the block. /// Note: This does not calculate the rewards for the block.
pub fn new<B: Block>(block: &SealedBlock<B>) -> Self { pub fn new<B: Block>(block: &SealedBlock<B>, blob_params: Option<BlobParams>) -> Self {
Self { Self {
base_fee_per_gas: block.header().base_fee_per_gas().unwrap_or_default(), 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, gas_used_ratio: block.header().gas_used() as f64 / block.header().gas_limit() as f64,
base_fee_per_blob_gas: block base_fee_per_blob_gas: block
.header() .header()
.excess_blob_gas() .excess_blob_gas()
.map(alloy_eips::eip4844::calc_blob_gasprice), .and_then(|excess_blob_gas| Some(blob_params?.calc_blob_fee(excess_blob_gas))),
blob_gas_used_ratio: block.body().blob_gas_used() as f64 / blob_gas_used_ratio: block.body().blob_gas_used() as f64 /
alloy_eips::eip4844::MAX_DATA_GAS_PER_BLOCK as f64, alloy_eips::eip4844::MAX_DATA_GAS_PER_BLOCK as f64,
excess_blob_gas: block.header().excess_blob_gas(), excess_blob_gas: block.header().excess_blob_gas(),
@ -379,6 +387,7 @@ impl FeeHistoryEntry {
gas_limit: block.header().gas_limit(), gas_limit: block.header().gas_limit(),
rewards: Vec::new(), rewards: Vec::new(),
timestamp: block.header().timestamp(), timestamp: block.header().timestamp(),
blob_params,
} }
} }
@ -398,13 +407,16 @@ impl FeeHistoryEntry {
/// ///
/// See also [`Self::next_block_excess_blob_gas`] /// See also [`Self::next_block_excess_blob_gas`]
pub fn next_block_blob_fee(&self) -> Option<u128> { pub fn next_block_blob_fee(&self) -> Option<u128> {
self.next_block_excess_blob_gas().map(alloy_eips::eip4844::calc_blob_gasprice) self.next_block_excess_blob_gas()
.and_then(|excess_blob_gas| Some(self.blob_params?.calc_blob_fee(excess_blob_gas)))
} }
/// Calculate excess blob gas for the next block according to the EIP-4844 spec. /// Calculate excess blob gas for the next block according to the EIP-4844 spec.
/// ///
/// Returns a `None` if no excess blob gas is set, no EIP-4844 support /// Returns a `None` if no excess blob gas is set, no EIP-4844 support
pub fn next_block_excess_blob_gas(&self) -> Option<u64> { pub fn next_block_excess_blob_gas(&self) -> Option<u64> {
Some(alloy_eips::eip4844::calc_excess_blob_gas(self.excess_blob_gas?, self.blob_gas_used?)) self.excess_blob_gas.and_then(|excess_blob_gas| {
Some(self.blob_params?.next_block_excess_blob_gas(excess_blob_gas, self.blob_gas_used?))
})
} }
} }

View File

@ -2,6 +2,7 @@
use super::EthResult; use super::EthResult;
use alloy_consensus::{transaction::TransactionMeta, ReceiptEnvelope, TxReceipt}; use alloy_consensus::{transaction::TransactionMeta, ReceiptEnvelope, TxReceipt};
use alloy_eips::eip7840::BlobParams;
use alloy_primitives::{Address, TxKind}; use alloy_primitives::{Address, TxKind};
use alloy_rpc_types_eth::{Log, ReceiptWithBloom, TransactionReceipt}; use alloy_rpc_types_eth::{Log, ReceiptWithBloom, TransactionReceipt};
use reth_primitives::{Receipt, TransactionSigned, TxType}; use reth_primitives::{Receipt, TransactionSigned, TxType};
@ -13,6 +14,7 @@ pub fn build_receipt<R, T, E>(
meta: TransactionMeta, meta: TransactionMeta,
receipt: &R, receipt: &R,
all_receipts: &[R], all_receipts: &[R],
blob_params: Option<BlobParams>,
build_envelope: impl FnOnce(ReceiptWithBloom<alloy_consensus::Receipt<Log>>) -> E, build_envelope: impl FnOnce(ReceiptWithBloom<alloy_consensus::Receipt<Log>>) -> E,
) -> EthResult<TransactionReceipt<E>> ) -> EthResult<TransactionReceipt<E>>
where where
@ -36,8 +38,9 @@ where
let blob_gas_used = transaction.blob_gas_used(); let blob_gas_used = transaction.blob_gas_used();
// Blob gas price should only be present if the transaction is a blob transaction // Blob gas price should only be present if the transaction is a blob transaction
let blob_gas_price = blob_gas_used let blob_gas_price =
.and_then(|_| meta.excess_blob_gas.map(alloy_eips::eip4844::calc_blob_gasprice)); blob_gas_used.and_then(|_| Some(blob_params?.calc_blob_fee(meta.excess_blob_gas?)));
let logs_bloom = receipt.bloom(); let logs_bloom = receipt.bloom();
// get number of logs in the block // get number of logs in the block
@ -107,9 +110,15 @@ impl EthReceiptBuilder {
meta: TransactionMeta, meta: TransactionMeta,
receipt: &Receipt, receipt: &Receipt,
all_receipts: &[Receipt], all_receipts: &[Receipt],
blob_params: Option<BlobParams>,
) -> EthResult<Self> { ) -> EthResult<Self> {
let base = build_receipt(transaction, meta, receipt, all_receipts, |receipt_with_bloom| { let base = build_receipt(
match receipt.tx_type { transaction,
meta,
receipt,
all_receipts,
blob_params,
|receipt_with_bloom| match receipt.tx_type {
TxType::Legacy => ReceiptEnvelope::Legacy(receipt_with_bloom), TxType::Legacy => ReceiptEnvelope::Legacy(receipt_with_bloom),
TxType::Eip2930 => ReceiptEnvelope::Eip2930(receipt_with_bloom), TxType::Eip2930 => ReceiptEnvelope::Eip2930(receipt_with_bloom),
TxType::Eip1559 => ReceiptEnvelope::Eip1559(receipt_with_bloom), TxType::Eip1559 => ReceiptEnvelope::Eip1559(receipt_with_bloom),
@ -117,8 +126,8 @@ impl EthReceiptBuilder {
TxType::Eip7702 => ReceiptEnvelope::Eip7702(receipt_with_bloom), TxType::Eip7702 => ReceiptEnvelope::Eip7702(receipt_with_bloom),
#[allow(unreachable_patterns)] #[allow(unreachable_patterns)]
_ => unreachable!(), _ => unreachable!(),
} },
})?; )?;
Ok(Self { base }) Ok(Self { base })
} }

View File

@ -2,8 +2,9 @@
use alloy_consensus::{transaction::TransactionMeta, BlockHeader}; use alloy_consensus::{transaction::TransactionMeta, BlockHeader};
use alloy_rpc_types_eth::{BlockId, TransactionReceipt}; use alloy_rpc_types_eth::{BlockId, TransactionReceipt};
use reth_chainspec::EthChainSpec;
use reth_primitives_traits::{BlockBody, SignedTransaction}; use reth_primitives_traits::{BlockBody, SignedTransaction};
use reth_provider::BlockReader; use reth_provider::{BlockReader, ChainSpecProvider};
use reth_rpc_eth_api::{ use reth_rpc_eth_api::{
helpers::{EthBlocks, LoadBlock, LoadPendingBlock, LoadReceipt, SpawnBlocking}, helpers::{EthBlocks, LoadBlock, LoadPendingBlock, LoadReceipt, SpawnBlocking},
RpcNodeCoreExt, RpcReceipt, RpcNodeCoreExt, RpcReceipt,
@ -22,7 +23,7 @@ where
Receipt = reth_primitives::Receipt, Receipt = reth_primitives::Receipt,
>, >,
>, >,
Provider: BlockReader, Provider: BlockReader + ChainSpecProvider,
{ {
async fn block_receipts( async fn block_receipts(
&self, &self,
@ -37,6 +38,7 @@ where
let block_hash = block.hash(); let block_hash = block.hash();
let excess_blob_gas = block.excess_blob_gas(); let excess_blob_gas = block.excess_blob_gas();
let timestamp = block.timestamp(); let timestamp = block.timestamp();
let blob_params = self.provider().chain_spec().blob_params_at_timestamp(timestamp);
return block return block
.body() .body()
@ -54,7 +56,7 @@ where
excess_blob_gas, excess_blob_gas,
timestamp, timestamp,
}; };
EthReceiptBuilder::new(tx, meta, receipt, &receipts) EthReceiptBuilder::new(tx, meta, receipt, &receipts, blob_params)
.map(|builder| builder.build()) .map(|builder| builder.build())
}) })
.collect::<Result<Vec<_>, Self::Error>>() .collect::<Result<Vec<_>, Self::Error>>()

View File

@ -1,8 +1,9 @@
//! Builds an RPC receipt response w.r.t. data layout of network. //! Builds an RPC receipt response w.r.t. data layout of network.
use alloy_consensus::transaction::TransactionMeta; use alloy_consensus::transaction::TransactionMeta;
use reth_chainspec::EthChainSpec;
use reth_primitives::{Receipt, TransactionSigned}; use reth_primitives::{Receipt, TransactionSigned};
use reth_provider::{BlockReader, ReceiptProvider, TransactionsProvider}; use reth_provider::{BlockReader, ChainSpecProvider, ReceiptProvider, TransactionsProvider};
use reth_rpc_eth_api::{helpers::LoadReceipt, FromEthApiError, RpcNodeCoreExt, RpcReceipt}; use reth_rpc_eth_api::{helpers::LoadReceipt, FromEthApiError, RpcNodeCoreExt, RpcReceipt};
use reth_rpc_eth_types::{EthApiError, EthReceiptBuilder}; use reth_rpc_eth_types::{EthApiError, EthReceiptBuilder};
@ -14,7 +15,7 @@ where
Provider: TransactionsProvider<Transaction = TransactionSigned> Provider: TransactionsProvider<Transaction = TransactionSigned>
+ ReceiptProvider<Receipt = reth_primitives::Receipt>, + ReceiptProvider<Receipt = reth_primitives::Receipt>,
>, >,
Provider: BlockReader, Provider: BlockReader + ChainSpecProvider,
{ {
async fn build_transaction_receipt( async fn build_transaction_receipt(
&self, &self,
@ -30,7 +31,8 @@ where
.await .await
.map_err(Self::Error::from_eth_err)? .map_err(Self::Error::from_eth_err)?
.ok_or(EthApiError::HeaderNotFound(hash.into()))?; .ok_or(EthApiError::HeaderNotFound(hash.into()))?;
let blob_params = self.provider().chain_spec().blob_params_at_timestamp(meta.timestamp);
Ok(EthReceiptBuilder::new(&tx, meta, &receipt, &all_receipts)?.build()) Ok(EthReceiptBuilder::new(&tx, meta, &receipt, &all_receipts, blob_params)?.build())
} }
} }