feat: more flexible rpc receipts (#18501)

This commit is contained in:
Arsenii Kulikov
2025-09-16 22:49:41 +04:00
committed by GitHub
parent 5274f095fe
commit bf58089286
17 changed files with 211 additions and 80 deletions

7
Cargo.lock generated
View File

@@ -8365,6 +8365,8 @@ dependencies = [
"alloy-eips",
"alloy-primitives",
"alloy-rlp",
"alloy-rpc-types-eth",
"alloy-serde",
"arbitrary",
"bincode 1.3.3",
"derive_more",
@@ -8378,6 +8380,7 @@ dependencies = [
"reth-zstd-compressors",
"secp256k1 0.30.0",
"serde",
"serde_json",
"serde_with",
]
@@ -9886,6 +9889,7 @@ dependencies = [
"alloy-signer-local",
"async-trait",
"derive_more",
"dyn-clone",
"futures",
"http",
"http-body",
@@ -9999,6 +10003,7 @@ dependencies = [
"alloy-rpc-types-eth",
"alloy-rpc-types-trace",
"clap",
"dyn-clone",
"http",
"jsonrpsee",
"metrics",
@@ -10051,6 +10056,8 @@ dependencies = [
"alloy-primitives",
"alloy-rpc-types-eth",
"alloy-signer",
"auto_impl",
"dyn-clone",
"jsonrpsee-types",
"op-alloy-consensus",
"op-alloy-network",

View File

@@ -21,6 +21,8 @@ reth-zstd-compressors = { workspace = true, optional = true }
alloy-eips = { workspace = true, features = ["k256"] }
alloy-primitives.workspace = true
alloy-consensus = { workspace = true, features = ["serde"] }
alloy-serde = { workspace = true, optional = true }
alloy-rpc-types-eth = { workspace = true, optional = true }
alloy-rlp.workspace = true
# misc
@@ -41,6 +43,7 @@ reth-codecs = { workspace = true, features = ["test-utils"] }
reth-zstd-compressors.workspace = true
secp256k1 = { workspace = true, features = ["rand"] }
alloy-consensus = { workspace = true, features = ["serde", "arbitrary"] }
serde_json.workspace = true
[features]
default = ["std"]
@@ -59,6 +62,9 @@ std = [
"derive_more/std",
"serde_with?/std",
"secp256k1/std",
"alloy-rpc-types-eth?/std",
"alloy-serde?/std",
"serde_json/std",
]
reth-codec = [
"std",
@@ -74,15 +80,19 @@ arbitrary = [
"reth-codecs?/arbitrary",
"reth-primitives-traits/arbitrary",
"alloy-eips/arbitrary",
"alloy-rpc-types-eth?/arbitrary",
"alloy-serde?/arbitrary",
]
serde-bincode-compat = [
"dep:serde_with",
"alloy-consensus/serde-bincode-compat",
"alloy-eips/serde-bincode-compat",
"reth-primitives-traits/serde-bincode-compat",
"alloy-rpc-types-eth?/serde-bincode-compat",
]
serde = [
"dep:serde",
"dep:alloy-serde",
"alloy-consensus/serde",
"alloy-eips/serde",
"alloy-primitives/serde",
@@ -91,4 +101,6 @@ serde = [
"rand/serde",
"rand_08/serde",
"secp256k1/serde",
"alloy-rpc-types-eth?/serde",
]
rpc = ["dep:alloy-rpc-types-eth"]

View File

@@ -2,7 +2,7 @@ use core::fmt::Debug;
use alloc::vec::Vec;
use alloy_consensus::{
Eip2718EncodableReceipt, Eip658Value, ReceiptWithBloom, RlpDecodableReceipt,
Eip2718EncodableReceipt, Eip658Value, ReceiptEnvelope, ReceiptWithBloom, RlpDecodableReceipt,
RlpEncodableReceipt, TxReceipt, TxType, Typed2718,
};
use alloy_eips::{
@@ -41,23 +41,48 @@ impl<T> TxTy for T where
{
}
/// Raw ethereum receipt.
pub type Receipt<T = TxType> = EthereumReceipt<T>;
#[cfg(feature = "rpc")]
/// Receipt representation for RPC.
pub type RpcReceipt<T = TxType> = EthereumReceipt<T, alloy_rpc_types_eth::Log>;
/// Typed ethereum transaction receipt.
/// Receipt containing result of transaction execution.
#[derive(Clone, Debug, PartialEq, Eq, Default)]
#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))]
#[cfg_attr(feature = "arbitrary", derive(arbitrary::Arbitrary))]
#[cfg_attr(feature = "reth-codec", reth_codecs::add_arbitrary_tests(compact, rlp))]
pub struct Receipt<T = TxType> {
#[cfg_attr(feature = "serde", serde(rename_all = "camelCase"))]
pub struct EthereumReceipt<T = TxType, L = Log> {
/// Receipt type.
#[cfg_attr(feature = "serde", serde(rename = "type"))]
pub tx_type: T,
/// If transaction is executed successfully.
///
/// This is the `statusCode`
#[cfg_attr(feature = "serde", serde(with = "alloy_serde::quantity", rename = "status"))]
pub success: bool,
/// Gas used
#[cfg_attr(feature = "serde", serde(with = "alloy_serde::quantity"))]
pub cumulative_gas_used: u64,
/// Log send from contracts.
pub logs: Vec<Log>,
pub logs: Vec<L>,
}
#[cfg(feature = "rpc")]
impl<T> Receipt<T> {
/// Converts the logs of the receipt to RPC logs.
pub fn into_rpc(
self,
next_log_index: usize,
meta: alloy_consensus::transaction::TransactionMeta,
) -> RpcReceipt<T> {
let Self { tx_type, success, cumulative_gas_used, logs } = self;
let logs = alloy_rpc_types_eth::Log::collect_for_receipt(next_log_index, meta, logs);
RpcReceipt { tx_type, success, cumulative_gas_used, logs }
}
}
impl<T: TxTy> Receipt<T> {
@@ -260,8 +285,12 @@ impl<T: TxTy> Decodable for Receipt<T> {
}
}
impl<T: TxTy> TxReceipt for Receipt<T> {
type Log = Log;
impl<T, L> TxReceipt for EthereumReceipt<T, L>
where
T: TxTy,
L: Send + Sync + Clone + Debug + Eq + AsRef<Log>,
{
type Log = L;
fn status_or_post_state(&self) -> Eip658Value {
self.success.into()
@@ -272,18 +301,18 @@ impl<T: TxTy> TxReceipt for Receipt<T> {
}
fn bloom(&self) -> Bloom {
alloy_primitives::logs_bloom(self.logs())
alloy_primitives::logs_bloom(self.logs.iter().map(|l| l.as_ref()))
}
fn cumulative_gas_used(&self) -> u64 {
self.cumulative_gas_used
}
fn logs(&self) -> &[Log] {
fn logs(&self) -> &[L] {
&self.logs
}
fn into_logs(self) -> Vec<Log> {
fn into_logs(self) -> Vec<L> {
self.logs
}
}
@@ -309,11 +338,11 @@ impl<T: TxTy> InMemorySize for Receipt<T> {
}
}
impl<T> From<alloy_consensus::ReceiptEnvelope<T>> for Receipt<TxType>
impl<T> From<ReceiptEnvelope<T>> for Receipt<TxType>
where
T: Into<Log>,
{
fn from(value: alloy_consensus::ReceiptEnvelope<T>) -> Self {
fn from(value: ReceiptEnvelope<T>) -> Self {
let value = value.into_primitives_receipt();
Self {
tx_type: value.tx_type(),
@@ -324,8 +353,8 @@ where
}
}
impl<T> From<Receipt<T>> for alloy_consensus::Receipt<Log> {
fn from(value: Receipt<T>) -> Self {
impl<T, L> From<EthereumReceipt<T, L>> for alloy_consensus::Receipt<L> {
fn from(value: EthereumReceipt<T, L>) -> Self {
Self {
status: value.success.into(),
cumulative_gas_used: value.cumulative_gas_used,
@@ -334,8 +363,11 @@ impl<T> From<Receipt<T>> for alloy_consensus::Receipt<Log> {
}
}
impl From<Receipt<TxType>> for alloy_consensus::ReceiptEnvelope<Log> {
fn from(value: Receipt<TxType>) -> Self {
impl<L> From<EthereumReceipt<TxType, L>> for ReceiptEnvelope<L>
where
L: Send + Sync + Clone + Debug + Eq + AsRef<Log>,
{
fn from(value: EthereumReceipt<TxType, L>) -> Self {
let tx_type = value.tx_type;
let receipt = value.into_with_bloom().map_receipt(Into::into);
match tx_type {
@@ -624,6 +656,7 @@ mod tests {
pub(crate) type Block<T = TransactionSigned> = alloy_consensus::Block<T>;
#[test]
#[cfg(feature = "reth-codec")]
fn test_decode_receipt() {
reth_codecs::test_utils::test_decode::<Receipt<TxType>>(&hex!(
"c428b52ffd23fc42696156b10200f034792b6a94c3850215c2fef7aea361a0c31b79d9a32652eefc0d4e2e730036061cff7344b6fc6132b50cda0ed810a991ae58ef013150c12b2522533cb3b3a8b19b7786a8b5ff1d3cdc84225e22b02def168c8858df"
@@ -824,4 +857,20 @@ mod tests {
b256!("0xfe70ae4a136d98944951b2123859698d59ad251a381abc9960fa81cae3d0d4a0")
);
}
// Ensures that reth and alloy receipts encode to the same JSON
#[test]
#[cfg(feature = "rpc")]
fn test_receipt_serde() {
let input = r#"{"status":"0x1","cumulativeGasUsed":"0x175cc0e","logs":[{"address":"0xa18b9ca2a78660d44ab38ae72e72b18792ffe413","topics":["0x8c5be1e5ebec7d5bd14f71427d1e84f3dd0314c0f7b2291e5b200ac8c7c3b925","0x000000000000000000000000e7e7d8006cbff47bc6ac2dabf592c98e97502708","0x0000000000000000000000007a250d5630b4cf539739df2c5dacb4c659f2488d"],"data":"0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff","blockHash":"0xbf9e6a368a399f996a0f0b27cab4191c028c3c99f5f76ea08a5b70b961475fcb","blockNumber":"0x164b59f","blockTimestamp":"0x68c9a713","transactionHash":"0x533aa9e57865675bb94f41aa2895c0ac81eee69686c77af16149c301e19805f1","transactionIndex":"0x14d","logIndex":"0x238","removed":false}],"logsBloom":"0x00000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000400000040000000000000004000000000000000000000000000000000000000000000020000000000000000000000000080000000000000000000000000200000020000000000000000000000000000000000000000000000000000000000000020000010000000000000000000000000000000000000000000000000000000000000","type":"0x2","transactionHash":"0x533aa9e57865675bb94f41aa2895c0ac81eee69686c77af16149c301e19805f1","transactionIndex":"0x14d","blockHash":"0xbf9e6a368a399f996a0f0b27cab4191c028c3c99f5f76ea08a5b70b961475fcb","blockNumber":"0x164b59f","gasUsed":"0xb607","effectiveGasPrice":"0x4a3ee768","from":"0xe7e7d8006cbff47bc6ac2dabf592c98e97502708","to":"0xa18b9ca2a78660d44ab38ae72e72b18792ffe413","contractAddress":null}"#;
let receipt: RpcReceipt = serde_json::from_str(input).unwrap();
let envelope: ReceiptEnvelope<alloy_rpc_types_eth::Log> =
serde_json::from_str(input).unwrap();
assert_eq!(envelope, receipt.clone().into());
let json_envelope = serde_json::to_value(&envelope).unwrap();
let json_receipt = serde_json::to_value(receipt.into_with_bloom()).unwrap();
assert_eq!(json_envelope, json_receipt);
}
}

View File

@@ -1,11 +1,10 @@
//! Loads and formats OP receipt RPC response.
use crate::{eth::RpcNodeCore, OpEthApi, OpEthApiError};
use alloy_consensus::{Receipt, TxReceipt};
use alloy_eips::eip2718::Encodable2718;
use alloy_rpc_types_eth::{Log, TransactionReceipt};
use op_alloy_consensus::{
OpDepositReceipt, OpDepositReceiptWithBloom, OpReceiptEnvelope, OpTransaction,
};
use op_alloy_consensus::{OpReceiptEnvelope, OpTransaction};
use op_alloy_rpc_types::{L1BlockInfo, OpTransactionReceipt, OpTransactionReceiptFields};
use reth_chainspec::ChainSpecProvider;
use reth_node_api::NodePrimitives;
@@ -270,23 +269,30 @@ impl OpReceiptBuilder {
let timestamp = input.meta.timestamp;
let block_number = input.meta.block_number;
let tx_signed = *input.tx.inner();
let core_receipt =
build_receipt(&input, None, |receipt_with_bloom| match input.receipt.as_ref() {
OpReceipt::Legacy(_) => OpReceiptEnvelope::Legacy(receipt_with_bloom),
OpReceipt::Eip2930(_) => OpReceiptEnvelope::Eip2930(receipt_with_bloom),
OpReceipt::Eip1559(_) => OpReceiptEnvelope::Eip1559(receipt_with_bloom),
OpReceipt::Eip7702(_) => OpReceiptEnvelope::Eip7702(receipt_with_bloom),
OpReceipt::Deposit(receipt) => {
OpReceiptEnvelope::Deposit(OpDepositReceiptWithBloom {
receipt: OpDepositReceipt {
inner: receipt_with_bloom.receipt,
deposit_nonce: receipt.deposit_nonce,
deposit_receipt_version: receipt.deposit_receipt_version,
},
logs_bloom: receipt_with_bloom.logs_bloom,
})
let core_receipt = build_receipt(input, None, |receipt, next_log_index, meta| {
let map_logs = move |receipt: alloy_consensus::Receipt| {
let Receipt { status, cumulative_gas_used, logs } = receipt;
let logs = Log::collect_for_receipt(next_log_index, meta, logs);
Receipt { status, cumulative_gas_used, logs }
};
match receipt {
OpReceipt::Legacy(receipt) => {
OpReceiptEnvelope::Legacy(map_logs(receipt).into_with_bloom())
}
});
OpReceipt::Eip2930(receipt) => {
OpReceiptEnvelope::Eip2930(map_logs(receipt).into_with_bloom())
}
OpReceipt::Eip1559(receipt) => {
OpReceiptEnvelope::Eip1559(map_logs(receipt).into_with_bloom())
}
OpReceipt::Eip7702(receipt) => {
OpReceiptEnvelope::Eip7702(map_logs(receipt).into_with_bloom())
}
OpReceipt::Deposit(receipt) => {
OpReceiptEnvelope::Deposit(receipt.map_inner(map_logs).into_with_bloom())
}
}
});
let op_receipt_fields = OpReceiptFieldsBuilder::new(timestamp, block_number)
.l1_block_info(chain_spec, tx_signed, l1_block_info)?

View File

@@ -43,6 +43,7 @@ reth-metrics = { workspace = true, features = ["common"] }
metrics.workspace = true
# misc
dyn-clone.workspace = true
serde = { workspace = true, features = ["derive"] }
thiserror.workspace = true
tracing.workspace = true

View File

@@ -963,7 +963,7 @@ where
RethRpcModule::Web3 => Web3Api::new(self.network.clone()).into_rpc().into(),
RethRpcModule::Txpool => TxPoolApi::new(
self.eth.api.pool().clone(),
self.eth.api.tx_resp_builder().clone(),
dyn_clone::clone(self.eth.api.tx_resp_builder()),
)
.into_rpc()
.into(),

View File

@@ -42,6 +42,9 @@ jsonrpsee-types.workspace = true
# error
thiserror.workspace = true
auto_impl.workspace = true
dyn-clone.workspace = true
[features]
default = []
op = [

View File

@@ -14,6 +14,7 @@ use alloy_rpc_types_eth::{
Transaction, TransactionInfo,
};
use core::error;
use dyn_clone::DynClone;
use reth_evm::{
revm::context_interface::{either::Either, Block},
ConfigureEvm, SpecFor, TxEnvFor,
@@ -22,14 +23,14 @@ use reth_primitives_traits::{
HeaderTy, NodePrimitives, SealedHeader, SealedHeaderFor, TransactionMeta, TxTy,
};
use revm_context::{BlockEnv, CfgEnv, TxEnv};
use std::{borrow::Cow, convert::Infallible, error::Error, fmt::Debug, marker::PhantomData};
use std::{convert::Infallible, error::Error, fmt::Debug, marker::PhantomData};
use thiserror::Error;
/// Input for [`RpcConvert::convert_receipts`].
#[derive(Debug, Clone)]
pub struct ConvertReceiptInput<'a, N: NodePrimitives> {
/// Primitive receipt.
pub receipt: Cow<'a, N::Receipt>,
pub receipt: N::Receipt,
/// Transaction the receipt corresponds to.
pub tx: Recovered<&'a N::SignedTx>,
/// Gas used by the transaction.
@@ -93,7 +94,8 @@ impl<T: Sealable> FromConsensusHeader<T> for alloy_rpc_types_eth::Header<T> {
/// A generic implementation [`RpcConverter`] should be preferred over a manual implementation. As
/// long as its trait bound requirements are met, the implementation is created automatically and
/// can be used in RPC method handlers for all the conversions.
pub trait RpcConvert: Send + Sync + Unpin + Clone + Debug + 'static {
#[auto_impl::auto_impl(&, Box, Arc)]
pub trait RpcConvert: Send + Sync + Unpin + Debug + DynClone + 'static {
/// Associated lower layer consensus types to convert from and into types of [`Self::Network`].
type Primitives: NodePrimitives;
@@ -162,6 +164,11 @@ pub trait RpcConvert: Send + Sync + Unpin + Clone + Debug + 'static {
) -> Result<RpcHeader<Self::Network>, Self::Error>;
}
dyn_clone::clone_trait_object!(
<Primitives, Network, Error, TxEnv, Spec>
RpcConvert<Primitives = Primitives, Network = Network, Error = Error, TxEnv = TxEnv, Spec = Spec>
);
/// Converts `self` into `T`. The opposite of [`FromConsensusTx`].
///
/// Should create an RPC transaction response object based on a consensus transaction, its signer
@@ -782,6 +789,25 @@ impl<Network, Evm, Receipt, Header, Map, SimTx, RpcTx, TxEnv>
tx_env_converter,
}
}
/// Converts `self` into a boxed converter.
#[expect(clippy::type_complexity)]
pub fn erased(
self,
) -> Box<
dyn RpcConvert<
Primitives = <Self as RpcConvert>::Primitives,
Network = <Self as RpcConvert>::Network,
Error = <Self as RpcConvert>::Error,
TxEnv = <Self as RpcConvert>::TxEnv,
Spec = <Self as RpcConvert>::Spec,
>,
>
where
Self: RpcConvert,
{
Box::new(self)
}
}
impl<Network, Evm, Receipt, Header, Map, SimTx, RpcTx, TxEnv> Default

View File

@@ -17,7 +17,7 @@ use reth_primitives_traits::{
use reth_rpc_convert::{transaction::ConvertReceiptInput, RpcConvert, RpcHeader};
use reth_storage_api::{BlockIdReader, BlockReader, ProviderHeader, ProviderReceipt, ProviderTx};
use reth_transaction_pool::{PoolTransaction, TransactionPool};
use std::{borrow::Cow, sync::Arc};
use std::sync::Arc;
/// Result type of the fetched block receipts.
pub type BlockReceiptsResult<N, E> = Result<Option<Vec<RpcReceipt<N>>>, E>;
@@ -127,7 +127,7 @@ pub trait EthBlocks:
let inputs = block
.transactions_recovered()
.zip(receipts.iter())
.zip(Arc::unwrap_or_clone(receipts))
.enumerate()
.map(|(idx, (tx, receipt))| {
let meta = TransactionMeta {
@@ -140,16 +140,19 @@ pub trait EthBlocks:
timestamp,
};
let cumulative_gas_used = receipt.cumulative_gas_used();
let logs_len = receipt.logs().len();
let input = ConvertReceiptInput {
receipt: Cow::Borrowed(receipt),
tx,
gas_used: receipt.cumulative_gas_used() - gas_used,
gas_used: cumulative_gas_used - gas_used,
next_log_index,
meta,
receipt,
};
gas_used = receipt.cumulative_gas_used();
next_log_index += receipt.logs().len();
gas_used = cumulative_gas_used;
next_log_index += logs_len;
input
})

View File

@@ -8,7 +8,6 @@ use reth_primitives_traits::SignerRecoverable;
use reth_rpc_convert::{transaction::ConvertReceiptInput, RpcConvert};
use reth_rpc_eth_types::{error::FromEthApiError, EthApiError};
use reth_storage_api::{ProviderReceipt, ProviderTx};
use std::borrow::Cow;
/// Assembles transaction receipt data w.r.t to network.
///
@@ -60,7 +59,7 @@ pub trait LoadReceipt:
.map_err(Self::Error::from_eth_err)?
.as_recovered_ref(),
gas_used: receipt.cumulative_gas_used() - gas_used,
receipt: Cow::Owned(receipt),
receipt,
next_log_index,
meta,
}])?

View File

@@ -31,7 +31,7 @@ pub trait EthApiTypes: Send + Sync + Clone {
/// Blockchain primitive types, specific to network, e.g. block and transaction.
type NetworkTypes: RpcTypes;
/// Conversion methods for transaction RPC type.
type RpcConvert: Send + Sync + Clone + fmt::Debug;
type RpcConvert: Send + Sync + fmt::Debug;
/// Returns reference to transaction response builder.
fn tx_resp_builder(&self) -> &Self::RpcConvert;

View File

@@ -18,7 +18,7 @@ reth-errors.workspace = true
reth-evm.workspace = true
reth-execution-types.workspace = true
reth-metrics.workspace = true
reth-ethereum-primitives.workspace = true
reth-ethereum-primitives = { workspace = true, features = ["rpc"] }
reth-primitives-traits = { workspace = true, features = ["rpc-compat"] }
reth-storage-api.workspace = true
reth-revm.workspace = true

View File

@@ -1,21 +1,21 @@
//! RPC receipt response builder, extends a layer one receipt with layer two data.
use crate::EthApiError;
use alloy_consensus::{ReceiptEnvelope, Transaction, TxReceipt};
use alloy_consensus::{ReceiptEnvelope, Transaction};
use alloy_eips::eip7840::BlobParams;
use alloy_primitives::{Address, TxKind};
use alloy_rpc_types_eth::{Log, ReceiptWithBloom, TransactionReceipt};
use alloy_rpc_types_eth::{Log, TransactionReceipt};
use reth_chainspec::EthChainSpec;
use reth_ethereum_primitives::Receipt;
use reth_primitives_traits::NodePrimitives;
use reth_primitives_traits::{NodePrimitives, TransactionMeta};
use reth_rpc_convert::transaction::{ConvertReceiptInput, ReceiptConverter};
use std::{borrow::Cow, sync::Arc};
use std::{fmt::Debug, sync::Arc};
/// Builds an [`TransactionReceipt`] obtaining the inner receipt envelope from the given closure.
pub fn build_receipt<N, E>(
input: &ConvertReceiptInput<'_, N>,
input: ConvertReceiptInput<'_, N>,
blob_params: Option<BlobParams>,
build_envelope: impl FnOnce(ReceiptWithBloom<alloy_consensus::Receipt<Log>>) -> E,
build_rpc_receipt: impl FnOnce(N::Receipt, usize, TransactionMeta) -> E,
) -> TransactionReceipt<E>
where
N: NodePrimitives,
@@ -28,33 +28,20 @@ where
let blob_gas_price =
blob_gas_used.and_then(|_| Some(blob_params?.calc_blob_fee(meta.excess_blob_gas?)));
let status = receipt.status_or_post_state();
let cumulative_gas_used = receipt.cumulative_gas_used();
let logs_bloom = receipt.bloom();
let logs = match receipt {
Cow::Borrowed(r) => {
Log::collect_for_receipt(*next_log_index, *meta, r.logs().iter().cloned())
}
Cow::Owned(r) => Log::collect_for_receipt(*next_log_index, *meta, r.into_logs()),
};
let rpc_receipt = alloy_rpc_types_eth::Receipt { status, cumulative_gas_used, logs };
let (contract_address, to) = match tx.kind() {
TxKind::Create => (Some(from.create(tx.nonce())), None),
TxKind::Call(addr) => (None, Some(Address(*addr))),
};
TransactionReceipt {
inner: build_envelope(ReceiptWithBloom { receipt: rpc_receipt, logs_bloom }),
inner: build_rpc_receipt(receipt, next_log_index, meta),
transaction_hash: meta.tx_hash,
transaction_index: Some(meta.index),
block_hash: Some(meta.block_hash),
block_number: Some(meta.block_number),
from,
to,
gas_used: *gas_used,
gas_used,
contract_address,
effective_gas_price: tx.effective_gas_price(meta.base_fee),
// EIP-4844 fields
@@ -65,29 +52,53 @@ where
/// Converter for Ethereum receipts.
#[derive(Debug)]
pub struct EthReceiptConverter<ChainSpec> {
pub struct EthReceiptConverter<
ChainSpec,
Builder = fn(Receipt, usize, TransactionMeta) -> ReceiptEnvelope<Log>,
> {
chain_spec: Arc<ChainSpec>,
build_rpc_receipt: Builder,
}
impl<ChainSpec> Clone for EthReceiptConverter<ChainSpec> {
impl<ChainSpec, Builder> Clone for EthReceiptConverter<ChainSpec, Builder>
where
Builder: Clone,
{
fn clone(&self) -> Self {
Self { chain_spec: self.chain_spec.clone() }
Self {
chain_spec: self.chain_spec.clone(),
build_rpc_receipt: self.build_rpc_receipt.clone(),
}
}
}
impl<ChainSpec> EthReceiptConverter<ChainSpec> {
/// Creates a new converter with the given chain spec.
pub const fn new(chain_spec: Arc<ChainSpec>) -> Self {
Self { chain_spec }
Self {
chain_spec,
build_rpc_receipt: |receipt, next_log_index, meta| {
receipt.into_rpc(next_log_index, meta).into()
},
}
}
/// Sets new builder for the converter.
pub fn with_builder<Builder>(
self,
build_rpc_receipt: Builder,
) -> EthReceiptConverter<ChainSpec, Builder> {
EthReceiptConverter { chain_spec: self.chain_spec, build_rpc_receipt }
}
}
impl<N, ChainSpec> ReceiptConverter<N> for EthReceiptConverter<ChainSpec>
impl<N, ChainSpec, Builder, Rpc> ReceiptConverter<N> for EthReceiptConverter<ChainSpec, Builder>
where
N: NodePrimitives<Receipt = Receipt>,
N: NodePrimitives,
ChainSpec: EthChainSpec + 'static,
Builder: Debug + Fn(N::Receipt, usize, TransactionMeta) -> Rpc + 'static,
{
type RpcReceipt = TransactionReceipt;
type RpcReceipt = TransactionReceipt<Rpc>;
type Error = EthApiError;
fn convert_receipts(
@@ -97,11 +108,8 @@ where
let mut receipts = Vec::with_capacity(inputs.len());
for input in inputs {
let tx_type = input.receipt.tx_type;
let blob_params = self.chain_spec.blob_params_at_timestamp(input.meta.timestamp);
receipts.push(build_receipt(&input, blob_params, |receipt_with_bloom| {
ReceiptEnvelope::from_typed(tx_type, receipt_with_bloom)
}));
receipts.push(build_receipt(input, blob_params, &self.build_rpc_receipt));
}
Ok(receipts)

View File

@@ -84,6 +84,7 @@ pin-project.workspace = true
parking_lot.workspace = true
# misc
dyn-clone.workspace = true
tracing.workspace = true
tracing-futures.workspace = true
futures.workspace = true

View File

@@ -0,0 +1,14 @@
use reth_evm::{ConfigureEvm, SpecFor, TxEnvFor};
use reth_rpc_convert::RpcConvert;
use reth_rpc_eth_types::EthApiError;
/// Boxed RPC converter.
pub type DynRpcConverter<Evm, Network, Error = EthApiError> = Box<
dyn RpcConvert<
Primitives = <Evm as ConfigureEvm>::Primitives,
Network = Network,
Error = Error,
TxEnv = TxEnvFor<Evm>,
Spec = SpecFor<Evm>,
>,
>;

View File

@@ -348,7 +348,7 @@ where
let stream = self.pool().new_pending_pool_transactions_listener();
let full_txs_receiver = FullTransactionsReceiver::new(
stream,
self.inner.eth_api.tx_resp_builder().clone(),
dyn_clone::clone(self.inner.eth_api.tx_resp_builder()),
);
FilterKind::PendingTransaction(PendingTransactionKind::FullTransaction(Arc::new(
full_txs_receiver,

View File

@@ -33,6 +33,7 @@ use pin_project as _;
use tower as _;
mod admin;
mod aliases;
mod debug;
mod engine;
pub mod eth;
@@ -47,6 +48,7 @@ mod validation;
mod web3;
pub use admin::AdminApi;
pub use aliases::*;
pub use debug::DebugApi;
pub use engine::{EngineApi, EngineEthApi};
pub use eth::{helpers::SyncListener, EthApi, EthApiBuilder, EthBundle, EthFilter, EthPubSub};