feat: EthApi traits abstraction (#13170)

This commit is contained in:
Arsenii Kulikov
2024-12-06 16:30:50 +04:00
committed by GitHub
parent ab87f22cab
commit cf2a6a1ee8
53 changed files with 851 additions and 511 deletions

View File

@ -1,7 +1,6 @@
use alloy_eips::BlockId;
use alloy_json_rpc::RpcObject;
use alloy_primitives::{Address, Bytes, TxHash, B256};
use alloy_rpc_types_eth::Header;
use alloy_rpc_types_trace::otterscan::{
BlockDetails, ContractCreator, InternalOperation, OtsBlockTransactions, TraceEntry,
TransactionsWithReceipts,
@ -11,7 +10,7 @@ use jsonrpsee::{core::RpcResult, proc_macros::rpc};
/// Otterscan rpc interface.
#[cfg_attr(not(feature = "client"), rpc(server, namespace = "ots"))]
#[cfg_attr(feature = "client", rpc(server, client, namespace = "ots"))]
pub trait Otterscan<T: RpcObject> {
pub trait Otterscan<T: RpcObject, H: RpcObject> {
/// Get the block header by block number, required by otterscan.
/// Otterscan currently requires this endpoint, used as:
///
@ -20,7 +19,7 @@ pub trait Otterscan<T: RpcObject> {
///
/// Ref: <https://github.com/otterscan/otterscan/blob/071d8c55202badf01804f6f8d53ef9311d4a9e47/src/useProvider.ts#L71>
#[method(name = "getHeaderByNumber", aliases = ["erigon_getHeaderByNumber"])]
async fn get_header_by_number(&self, block_number: u64) -> RpcResult<Option<Header>>;
async fn get_header_by_number(&self, block_number: u64) -> RpcResult<Option<H>>;
/// Check if a certain address contains a deployed code.
#[method(name = "hasCode")]
@ -48,11 +47,11 @@ pub trait Otterscan<T: RpcObject> {
/// Tailor-made and expanded version of eth_getBlockByNumber for block details page in
/// Otterscan.
#[method(name = "getBlockDetails")]
async fn get_block_details(&self, block_number: u64) -> RpcResult<BlockDetails>;
async fn get_block_details(&self, block_number: u64) -> RpcResult<BlockDetails<H>>;
/// Tailor-made and expanded version of eth_getBlockByHash for block details page in Otterscan.
#[method(name = "getBlockDetailsByHash")]
async fn get_block_details_by_hash(&self, block_hash: B256) -> RpcResult<BlockDetails>;
async fn get_block_details_by_hash(&self, block_hash: B256) -> RpcResult<BlockDetails<H>>;
/// Get paginated transactions for a certain block. Also remove some verbose fields like logs.
#[method(name = "getBlockTransactions")]
@ -61,7 +60,7 @@ pub trait Otterscan<T: RpcObject> {
block_number: u64,
page_number: usize,
page_size: usize,
) -> RpcResult<OtsBlockTransactions<T>>;
) -> RpcResult<OtsBlockTransactions<T, H>>;
/// Gets paginated inbound/outbound transaction calls for a certain address.
#[method(name = "searchTransactionsBefore")]

View File

@ -207,7 +207,8 @@ use reth_network_api::{noop::NoopNetwork, NetworkInfo, Peers};
use reth_primitives::{EthPrimitives, NodePrimitives};
use reth_provider::{
AccountReader, BlockReader, CanonStateSubscriptions, ChainSpecProvider, ChangeSetReader,
EvmEnvProvider, FullRpcProvider, HeaderProvider, ReceiptProvider, StateProviderFactory,
EvmEnvProvider, FullRpcProvider, HeaderProvider, ProviderBlock, ProviderHeader,
ProviderReceipt, ReceiptProvider, StateProviderFactory,
};
use reth_rpc::{
AdminApi, DebugApi, EngineEthApi, EthBundle, MinerApi, NetApi, OtterscanApi, RPCApi, RethApi,
@ -216,7 +217,7 @@ use reth_rpc::{
use reth_rpc_api::servers::*;
use reth_rpc_eth_api::{
helpers::{Call, EthApiSpec, EthTransactions, LoadPendingBlock, TraceExt},
EthApiServer, EthApiTypes, FullEthApiServer, RpcBlock, RpcReceipt, RpcTransaction,
EthApiServer, EthApiTypes, FullEthApiServer, RpcBlock, RpcHeader, RpcReceipt, RpcTransaction,
};
use reth_rpc_eth_types::{EthConfig, EthStateCache, EthSubscriptionIdProvider};
use reth_rpc_layer::{AuthLayer, Claims, CompressionLayer, JwtAuthValidator, JwtSecret};
@ -276,9 +277,9 @@ pub async fn launch<Provider, Pool, Network, Tasks, Events, EvmConfig, EthApi, B
) -> Result<RpcServerHandle, RpcError>
where
Provider: FullRpcProvider<
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
Header = reth_primitives::Header,
Block = ProviderBlock<EthApi::Provider>,
Receipt = ProviderReceipt<EthApi::Provider>,
Header = ProviderHeader<EthApi::Provider>,
> + AccountReader
+ ChangeSetReader,
Pool: TransactionPool<Transaction = <EthApi::Pool as TransactionPool>::Transaction> + 'static,
@ -286,7 +287,13 @@ where
Tasks: TaskSpawner + Clone + 'static,
Events: CanonStateSubscriptions<Primitives = EthPrimitives> + Clone + 'static,
EvmConfig: ConfigureEvm<Header = alloy_consensus::Header>,
EthApi: FullEthApiServer,
EthApi: FullEthApiServer<
Provider: BlockReader<
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
Header = reth_primitives::Header,
>,
>,
BlockExecutor: BlockExecutorProvider<
Primitives: NodePrimitives<
Block = reth_primitives::Block,
@ -672,7 +679,13 @@ where
where
EngineT: EngineTypes,
EngineApi: EngineApiServer<EngineT>,
EthApi: FullEthApiServer,
EthApi: FullEthApiServer<
Provider: BlockReader<
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
Header = reth_primitives::Header,
>,
>,
Provider: BlockReader<
Block = <EthApi::Provider as BlockReader>::Block,
Receipt = <EthApi::Provider as ReceiptProvider>::Receipt,
@ -792,7 +805,13 @@ where
eth: DynEthApiBuilder<Provider, Pool, EvmConfig, Network, Tasks, Events, EthApi>,
) -> TransportRpcModules<()>
where
EthApi: FullEthApiServer,
EthApi: FullEthApiServer<
Provider: BlockReader<
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
Header = reth_primitives::Header,
>,
>,
Provider: BlockReader<
Block = <EthApi::Provider as BlockReader>::Block,
Receipt = <EthApi::Provider as ReceiptProvider>::Receipt,
@ -1134,6 +1153,7 @@ where
RpcTransaction<EthApi::NetworkTypes>,
RpcBlock<EthApi::NetworkTypes>,
RpcReceipt<EthApi::NetworkTypes>,
RpcHeader<EthApi::NetworkTypes>,
> + EthApiTypes,
BlockExecutor:
BlockExecutorProvider<Primitives: NodePrimitives<Block = reth_primitives::Block>>,
@ -1170,10 +1190,16 @@ where
/// If called outside of the tokio runtime. See also [`Self::eth_api`]
pub fn register_debug(&mut self) -> &mut Self
where
EthApi: EthApiSpec + EthTransactions + TraceExt,
EthApi: EthApiSpec
+ EthTransactions<
Provider: BlockReader<
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
>,
> + TraceExt,
Provider: BlockReader<
Block = <EthApi::Provider as BlockReader>::Block,
Receipt = reth_primitives::Receipt,
Receipt = <EthApi::Provider as ReceiptProvider>::Receipt,
>,
{
let debug_api = self.debug_api();
@ -1339,7 +1365,13 @@ where
Network: NetworkInfo + Peers + Clone + 'static,
Tasks: TaskSpawner + Clone + 'static,
Events: CanonStateSubscriptions<Primitives = EthPrimitives> + Clone + 'static,
EthApi: FullEthApiServer,
EthApi: FullEthApiServer<
Provider: BlockReader<
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
Header = reth_primitives::Header,
>,
>,
BlockExecutor: BlockExecutorProvider<
Primitives: NodePrimitives<
Block = reth_primitives::Block,

View File

@ -5,7 +5,7 @@ use crate::utils::{launch_http, launch_http_ws, launch_ws};
use alloy_eips::{BlockId, BlockNumberOrTag};
use alloy_primitives::{hex_literal::hex, Address, Bytes, TxHash, B256, B64, U256, U64};
use alloy_rpc_types_eth::{
transaction::TransactionRequest, Block, FeeHistory, Filter, Index, Log,
transaction::TransactionRequest, Block, FeeHistory, Filter, Header, Index, Log,
PendingTransactionFilterKind, SyncStatus, Transaction, TransactionReceipt,
};
use alloy_rpc_types_trace::filter::TraceFilter;
@ -174,16 +174,24 @@ where
.unwrap();
// Implemented
EthApiClient::<Transaction, Block, Receipt>::protocol_version(client).await.unwrap();
EthApiClient::<Transaction, Block, Receipt>::chain_id(client).await.unwrap();
EthApiClient::<Transaction, Block, Receipt>::accounts(client).await.unwrap();
EthApiClient::<Transaction, Block, Receipt>::get_account(client, address, block_number.into())
EthApiClient::<Transaction, Block, Receipt, Header>::protocol_version(client).await.unwrap();
EthApiClient::<Transaction, Block, Receipt, Header>::chain_id(client).await.unwrap();
EthApiClient::<Transaction, Block, Receipt, Header>::accounts(client).await.unwrap();
EthApiClient::<Transaction, Block, Receipt, Header>::get_account(
client,
address,
block_number.into(),
)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt, Header>::block_number(client).await.unwrap();
EthApiClient::<Transaction, Block, Receipt, Header>::get_code(client, address, None)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt>::block_number(client).await.unwrap();
EthApiClient::<Transaction, Block, Receipt>::get_code(client, address, None).await.unwrap();
EthApiClient::<Transaction, Block, Receipt>::send_raw_transaction(client, tx).await.unwrap();
EthApiClient::<Transaction, Block, Receipt>::fee_history(
EthApiClient::<Transaction, Block, Receipt, Header>::send_raw_transaction(client, tx)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt, Header>::fee_history(
client,
U64::from(0),
block_number,
@ -191,11 +199,13 @@ where
)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt>::balance(client, address, None).await.unwrap();
EthApiClient::<Transaction, Block, Receipt>::transaction_count(client, address, None)
EthApiClient::<Transaction, Block, Receipt, Header>::balance(client, address, None)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt>::storage_at(
EthApiClient::<Transaction, Block, Receipt, Header>::transaction_count(client, address, None)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt, Header>::storage_at(
client,
address,
U256::default().into(),
@ -203,64 +213,79 @@ where
)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt>::block_by_hash(client, hash, false).await.unwrap();
EthApiClient::<Transaction, Block, Receipt>::block_by_number(client, block_number, false)
EthApiClient::<Transaction, Block, Receipt, Header>::block_by_hash(client, hash, false)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt>::block_transaction_count_by_number(
EthApiClient::<Transaction, Block, Receipt, Header>::block_by_number(
client,
block_number,
false,
)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt, Header>::block_transaction_count_by_number(
client,
block_number,
)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt>::block_transaction_count_by_hash(client, hash)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt>::block_uncles_count_by_hash(client, hash)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt>::block_uncles_count_by_number(client, block_number)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt>::uncle_by_block_hash_and_index(client, hash, index)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt>::uncle_by_block_number_and_index(
client,
block_number,
index,
EthApiClient::<Transaction, Block, Receipt, Header>::block_transaction_count_by_hash(
client, hash,
)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt>::sign(client, address, bytes.clone())
.await
.unwrap_err();
EthApiClient::<Transaction, Block, Receipt>::sign_typed_data(client, address, typed_data)
.await
.unwrap_err();
EthApiClient::<Transaction, Block, Receipt>::transaction_by_hash(client, tx_hash)
EthApiClient::<Transaction, Block, Receipt, Header>::block_uncles_count_by_hash(client, hash)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt>::transaction_by_block_hash_and_index(
EthApiClient::<Transaction, Block, Receipt, Header>::block_uncles_count_by_number(
client,
block_number,
)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt, Header>::uncle_by_block_hash_and_index(
client, hash, index,
)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt>::transaction_by_block_number_and_index(
EthApiClient::<Transaction, Block, Receipt, Header>::uncle_by_block_number_and_index(
client,
block_number,
index,
)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt>::create_access_list(
EthApiClient::<Transaction, Block, Receipt, Header>::sign(client, address, bytes.clone())
.await
.unwrap_err();
EthApiClient::<Transaction, Block, Receipt, Header>::sign_typed_data(
client, address, typed_data,
)
.await
.unwrap_err();
EthApiClient::<Transaction, Block, Receipt, Header>::transaction_by_hash(client, tx_hash)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt, Header>::transaction_by_block_hash_and_index(
client, hash, index,
)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt, Header>::transaction_by_block_number_and_index(
client,
block_number,
index,
)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt, Header>::create_access_list(
client,
call_request.clone(),
Some(block_number.into()),
)
.await
.unwrap_err();
EthApiClient::<Transaction, Block, Receipt>::estimate_gas(
EthApiClient::<Transaction, Block, Receipt, Header>::estimate_gas(
client,
call_request.clone(),
Some(block_number.into()),
@ -268,7 +293,7 @@ where
)
.await
.unwrap_err();
EthApiClient::<Transaction, Block, Receipt>::call(
EthApiClient::<Transaction, Block, Receipt, Header>::call(
client,
call_request.clone(),
Some(block_number.into()),
@ -277,44 +302,47 @@ where
)
.await
.unwrap_err();
EthApiClient::<Transaction, Block, Receipt>::syncing(client).await.unwrap();
EthApiClient::<Transaction, Block, Receipt>::send_transaction(
EthApiClient::<Transaction, Block, Receipt, Header>::syncing(client).await.unwrap();
EthApiClient::<Transaction, Block, Receipt, Header>::send_transaction(
client,
transaction_request.clone(),
)
.await
.unwrap_err();
EthApiClient::<Transaction, Block, Receipt>::sign_transaction(client, transaction_request)
.await
.unwrap_err();
EthApiClient::<Transaction, Block, Receipt>::hashrate(client).await.unwrap();
EthApiClient::<Transaction, Block, Receipt>::submit_hashrate(
EthApiClient::<Transaction, Block, Receipt, Header>::sign_transaction(
client,
transaction_request,
)
.await
.unwrap_err();
EthApiClient::<Transaction, Block, Receipt, Header>::hashrate(client).await.unwrap();
EthApiClient::<Transaction, Block, Receipt, Header>::submit_hashrate(
client,
U256::default(),
B256::default(),
)
.await
.unwrap();
EthApiClient::<Transaction, Block, Receipt>::gas_price(client).await.unwrap_err();
EthApiClient::<Transaction, Block, Receipt>::max_priority_fee_per_gas(client)
EthApiClient::<Transaction, Block, Receipt, Header>::gas_price(client).await.unwrap_err();
EthApiClient::<Transaction, Block, Receipt, Header>::max_priority_fee_per_gas(client)
.await
.unwrap_err();
EthApiClient::<Transaction, Block, Receipt>::get_proof(client, address, vec![], None)
EthApiClient::<Transaction, Block, Receipt, Header>::get_proof(client, address, vec![], None)
.await
.unwrap();
// Unimplemented
assert!(is_unimplemented(
EthApiClient::<Transaction, Block, Receipt>::author(client).await.err().unwrap()
EthApiClient::<Transaction, Block, Receipt, Header>::author(client).await.err().unwrap()
));
assert!(is_unimplemented(
EthApiClient::<Transaction, Block, Receipt>::is_mining(client).await.err().unwrap()
EthApiClient::<Transaction, Block, Receipt, Header>::is_mining(client).await.err().unwrap()
));
assert!(is_unimplemented(
EthApiClient::<Transaction, Block, Receipt>::get_work(client).await.err().unwrap()
EthApiClient::<Transaction, Block, Receipt, Header>::get_work(client).await.err().unwrap()
));
assert!(is_unimplemented(
EthApiClient::<Transaction, Block, Receipt>::submit_work(
EthApiClient::<Transaction, Block, Receipt, Header>::submit_work(
client,
B64::default(),
B256::default(),
@ -402,28 +430,32 @@ where
let nonce = 1;
let block_hash = B256::default();
OtterscanClient::<Transaction>::get_header_by_number(client, block_number).await.unwrap();
OtterscanClient::<Transaction>::has_code(client, address, None).await.unwrap();
OtterscanClient::<Transaction>::has_code(client, address, Some(block_number.into()))
OtterscanClient::<Transaction, Header>::get_header_by_number(client, block_number)
.await
.unwrap();
OtterscanClient::<Transaction>::get_api_level(client).await.unwrap();
OtterscanClient::<Transaction, Header>::has_code(client, address, None).await.unwrap();
OtterscanClient::<Transaction, Header>::has_code(client, address, Some(block_number.into()))
.await
.unwrap();
OtterscanClient::<Transaction>::get_internal_operations(client, tx_hash).await.unwrap();
OtterscanClient::<Transaction, Header>::get_api_level(client).await.unwrap();
OtterscanClient::<Transaction>::get_transaction_error(client, tx_hash).await.unwrap();
OtterscanClient::<Transaction, Header>::get_internal_operations(client, tx_hash).await.unwrap();
OtterscanClient::<Transaction>::trace_transaction(client, tx_hash).await.unwrap();
OtterscanClient::<Transaction, Header>::get_transaction_error(client, tx_hash).await.unwrap();
OtterscanClient::<Transaction>::get_block_details(client, block_number).await.unwrap_err();
OtterscanClient::<Transaction, Header>::trace_transaction(client, tx_hash).await.unwrap();
OtterscanClient::<Transaction>::get_block_details_by_hash(client, block_hash)
OtterscanClient::<Transaction, Header>::get_block_details(client, block_number)
.await
.unwrap_err();
OtterscanClient::<Transaction>::get_block_transactions(
OtterscanClient::<Transaction, Header>::get_block_details_by_hash(client, block_hash)
.await
.unwrap_err();
OtterscanClient::<Transaction, Header>::get_block_transactions(
client,
block_number,
page_number,
@ -434,7 +466,7 @@ where
.unwrap();
assert!(is_unimplemented(
OtterscanClient::<Transaction>::search_transactions_before(
OtterscanClient::<Transaction, Header>::search_transactions_before(
client,
address,
block_number,
@ -445,7 +477,7 @@ where
.unwrap()
));
assert!(is_unimplemented(
OtterscanClient::<Transaction>::search_transactions_after(
OtterscanClient::<Transaction, Header>::search_transactions_after(
client,
address,
block_number,
@ -455,13 +487,13 @@ where
.err()
.unwrap()
));
assert!(OtterscanClient::<Transaction>::get_transaction_by_sender_and_nonce(
assert!(OtterscanClient::<Transaction, Header>::get_transaction_by_sender_and_nonce(
client, sender, nonce
)
.await
.err()
.is_none());
assert!(OtterscanClient::<Transaction>::get_contract_creator(client, address)
assert!(OtterscanClient::<Transaction, Header>::get_contract_creator(client, address)
.await
.unwrap()
.is_none());

View File

@ -1,5 +1,5 @@
use crate::utils::{test_address, test_rpc_builder};
use alloy_rpc_types_eth::{Block, Receipt, Transaction};
use alloy_rpc_types_eth::{Block, Header, Receipt, Transaction};
use jsonrpsee::{
server::{middleware::rpc::RpcServiceT, RpcServiceBuilder},
types::Request,
@ -75,7 +75,7 @@ async fn test_rpc_middleware() {
.unwrap();
let client = handle.http_client().unwrap();
EthApiClient::<Transaction, Block, Receipt>::protocol_version(&client).await.unwrap();
EthApiClient::<Transaction, Block, Receipt, Header>::protocol_version(&client).await.unwrap();
let count = mylayer.count.load(Ordering::Relaxed);
assert_eq!(count, 1);
}

View File

@ -34,6 +34,7 @@ reth-node-api.workspace = true
reth-trie-common = { workspace = true, features = ["eip1186"] }
# ethereum
alloy-rlp.workspace = true
alloy-serde.workspace = true
alloy-eips.workspace = true
alloy-dyn-abi = { workspace = true, features = ["eip712"] }

View File

@ -8,17 +8,18 @@ use alloy_rpc_types_eth::{
simulate::{SimulatePayload, SimulatedBlock},
state::{EvmOverrides, StateOverride},
transaction::TransactionRequest,
BlockOverrides, Bundle, EIP1186AccountProofResponse, EthCallResponse, FeeHistory, Header,
Index, StateContext, SyncStatus, Work,
BlockOverrides, Bundle, EIP1186AccountProofResponse, EthCallResponse, FeeHistory, Index,
StateContext, SyncStatus, Work,
};
use alloy_serde::JsonStorageKey;
use jsonrpsee::{core::RpcResult, proc_macros::rpc};
use reth_provider::BlockReader;
use reth_rpc_server_types::{result::internal_rpc_err, ToRpcResult};
use tracing::trace;
use crate::{
helpers::{EthApiSpec, EthBlocks, EthCall, EthFees, EthState, EthTransactions, FullEthApi},
RpcBlock, RpcReceipt, RpcTransaction,
RpcBlock, RpcHeader, RpcReceipt, RpcTransaction,
};
/// Helper trait, unifies functionality that must be supported to implement all RPC methods for
@ -28,6 +29,7 @@ pub trait FullEthApiServer:
RpcTransaction<Self::NetworkTypes>,
RpcBlock<Self::NetworkTypes>,
RpcReceipt<Self::NetworkTypes>,
RpcHeader<Self::NetworkTypes>,
> + FullEthApi
+ Clone
{
@ -38,6 +40,7 @@ impl<T> FullEthApiServer for T where
RpcTransaction<T::NetworkTypes>,
RpcBlock<T::NetworkTypes>,
RpcReceipt<T::NetworkTypes>,
RpcHeader<T::NetworkTypes>,
> + FullEthApi
+ Clone
{
@ -46,7 +49,7 @@ impl<T> FullEthApiServer for T where
/// Eth rpc interface: <https://ethereum.github.io/execution-apis/api-documentation/>
#[cfg_attr(not(feature = "client"), rpc(server, namespace = "eth"))]
#[cfg_attr(feature = "client", rpc(server, client, namespace = "eth"))]
pub trait EthApi<T: RpcObject, B: RpcObject, R: RpcObject> {
pub trait EthApi<T: RpcObject, B: RpcObject, R: RpcObject, H: RpcObject> {
/// Returns the protocol version encoded as a string.
#[method(name = "protocolVersion")]
async fn protocol_version(&self) -> RpcResult<U64>;
@ -200,11 +203,11 @@ pub trait EthApi<T: RpcObject, B: RpcObject, R: RpcObject> {
/// Returns the block's header at given number.
#[method(name = "getHeaderByNumber")]
async fn header_by_number(&self, hash: BlockNumberOrTag) -> RpcResult<Option<Header>>;
async fn header_by_number(&self, hash: BlockNumberOrTag) -> RpcResult<Option<H>>;
/// Returns the block's header at given hash.
#[method(name = "getHeaderByHash")]
async fn header_by_hash(&self, hash: B256) -> RpcResult<Option<Header>>;
async fn header_by_hash(&self, hash: B256) -> RpcResult<Option<H>>;
/// `eth_simulateV1` executes an arbitrary number of transactions on top of the requested state.
/// The transactions are packed into individual blocks. Overrides can be provided.
@ -366,9 +369,15 @@ impl<T>
RpcTransaction<T::NetworkTypes>,
RpcBlock<T::NetworkTypes>,
RpcReceipt<T::NetworkTypes>,
RpcHeader<T::NetworkTypes>,
> for T
where
T: FullEthApi,
T: FullEthApi<
Provider: BlockReader<
Header = alloy_consensus::Header,
Transaction = reth_primitives::TransactionSigned,
>,
>,
jsonrpsee_types::error::ErrorObject<'static>: From<T::Error>,
{
/// Handler for: `eth_protocolVersion`
@ -607,13 +616,16 @@ where
}
/// Handler for: `eth_getHeaderByNumber`
async fn header_by_number(&self, block_number: BlockNumberOrTag) -> RpcResult<Option<Header>> {
async fn header_by_number(
&self,
block_number: BlockNumberOrTag,
) -> RpcResult<Option<RpcHeader<T::NetworkTypes>>> {
trace!(target: "rpc::eth", ?block_number, "Serving eth_getHeaderByNumber");
Ok(EthBlocks::rpc_block_header(self, block_number.into()).await?)
}
/// Handler for: `eth_getHeaderByHash`
async fn header_by_hash(&self, hash: B256) -> RpcResult<Option<Header>> {
async fn header_by_hash(&self, hash: B256) -> RpcResult<Option<RpcHeader<T::NetworkTypes>>> {
trace!(target: "rpc::eth", ?hash, "Serving eth_getHeaderByHash");
Ok(EthBlocks::rpc_block_header(self, hash.into()).await?)
}

View File

@ -4,14 +4,17 @@ use std::sync::Arc;
use alloy_consensus::BlockHeader;
use alloy_eips::BlockId;
use alloy_rpc_types_eth::{Block, Header, Index};
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_provider::{
BlockIdReader, BlockReader, BlockReaderIdExt, HeaderProvider, ProviderReceipt,
BlockIdReader, BlockReader, BlockReaderIdExt, HeaderProvider, ProviderHeader, ProviderReceipt,
};
use reth_rpc_types_compat::block::from_block;
use revm_primitives::U256;
use crate::{
node::RpcNodeCoreExt, EthApiTypes, FromEthApiError, FullEthApiTypes, RpcBlock, RpcNodeCore,
@ -35,10 +38,11 @@ pub type BlockAndReceiptsResult<Eth> = Result<
/// `eth_` namespace.
pub trait EthBlocks: LoadBlock {
/// Returns the block header for the given block id.
#[expect(clippy::type_complexity)]
fn rpc_block_header(
&self,
block_id: BlockId,
) -> impl Future<Output = Result<Option<Header>, Self::Error>> + Send
) -> impl Future<Output = Result<Option<Header<ProviderHeader<Self::Provider>>>, Self::Error>> + Send
where
Self: FullEthApiTypes,
{
@ -113,7 +117,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.transactions().len()))
}
}
@ -173,10 +177,11 @@ pub trait EthBlocks: LoadBlock {
/// Returns uncle headers of given block.
///
/// Returns an empty vec if there are none.
#[expect(clippy::type_complexity)]
fn ommers(
&self,
block_id: BlockId,
) -> Result<Option<Vec<alloy_consensus::Header>>, Self::Error> {
) -> Result<Option<Vec<ProviderHeader<Self::Provider>>>, Self::Error> {
self.provider().ommers_by_id(block_id).map_err(Self::Error::from_eth_err)
}
@ -195,13 +200,22 @@ pub trait EthBlocks: LoadBlock {
self.provider()
.pending_block()
.map_err(Self::Error::from_eth_err)?
.map(|block| block.body.ommers)
.and_then(|block| block.body.ommers().map(|o| o.to_vec()))
} else {
self.provider().ommers_by_id(block_id).map_err(Self::Error::from_eth_err)?
}
.unwrap_or_default();
Ok(uncles.into_iter().nth(index.into()).map(Block::uncle_from_header))
Ok(uncles.into_iter().nth(index.into()).map(|header| {
let block = alloy_consensus::Block::<alloy_consensus::TxEnvelope, _>::uncle(header);
let size = U256::from(block.length());
Block {
uncles: vec![],
header: Header::from_consensus(block.header.seal_slow(), None, Some(size)),
transactions: BlockTransactions::Uncle,
withdrawals: None,
}
}))
}
}
}

View File

@ -6,7 +6,7 @@ use crate::{
helpers::estimate::EstimateCall, FromEthApiError, FromEvmError, FullEthApiTypes,
IntoEthApiError, RpcBlock, RpcNodeCore,
};
use alloy_consensus::{BlockHeader, Header};
use alloy_consensus::BlockHeader;
use alloy_eips::{eip1559::calc_next_block_base_fee, eip2930::AccessListResult};
use alloy_primitives::{Address, Bytes, TxKind, B256, U256};
use alloy_rpc_types_eth::{
@ -20,7 +20,9 @@ use reth_chainspec::EthChainSpec;
use reth_evm::{ConfigureEvm, ConfigureEvmEnv};
use reth_node_api::BlockBody;
use reth_primitives_traits::SignedTransaction;
use reth_provider::{BlockIdReader, ChainSpecProvider, HeaderProvider};
use reth_provider::{
BlockIdReader, BlockReader, ChainSpecProvider, HeaderProvider, ProviderHeader,
};
use reth_revm::{
database::StateProviderDatabase,
db::CacheDB,
@ -70,7 +72,12 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock {
block: Option<BlockId>,
) -> impl Future<Output = SimulatedBlocksResult<Self::NetworkTypes, Self::Error>> + Send
where
Self: LoadBlock + FullEthApiTypes,
Self: LoadBlock<
Provider: BlockReader<
Header = alloy_consensus::Header,
Transaction = reth_primitives::TransactionSigned,
>,
> + FullEthApiTypes,
{
async move {
if payload.block_state_calls.len() > self.max_simulate_blocks() as usize {
@ -456,7 +463,9 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock {
}
/// Executes code on state.
pub trait Call: LoadState<Evm: ConfigureEvm<Header = Header>> + SpawnBlocking {
pub trait Call:
LoadState<Evm: ConfigureEvm<Header = ProviderHeader<Self::Provider>>> + SpawnBlocking
{
/// Returns default gas limit to use for `eth_call` and tracing RPC methods.
///
/// Data access in default trait method implementations.
@ -616,7 +625,7 @@ pub trait Call: LoadState<Evm: ConfigureEvm<Header = Header>> + SpawnBlocking {
// we need to get the state of the parent block because we're essentially replaying the
// block the transaction is included in
let parent_block = block.parent_hash;
let parent_block = block.parent_hash();
let this = self.clone();
self.spawn_with_state_at_block(parent_block.into(), move |state| {
@ -629,7 +638,7 @@ pub trait Call: LoadState<Evm: ConfigureEvm<Header = Header>> + SpawnBlocking {
cfg.clone(),
block_env.clone(),
block_txs,
tx.hash(),
*tx.tx_hash(),
)?;
let env = EnvWithHandlerCfg::new_with_cfg_env(

View File

@ -5,6 +5,7 @@ use alloy_primitives::U256;
use alloy_rpc_types_eth::{BlockNumberOrTag, FeeHistory};
use futures::Future;
use reth_chainspec::EthChainSpec;
use reth_primitives_traits::BlockBody;
use reth_provider::{BlockIdReader, ChainSpecProvider, HeaderProvider};
use reth_rpc_eth_types::{
fee_history::calculate_reward_percentiles_for_block, EthApiError, FeeHistoryCache,
@ -183,7 +184,7 @@ pub trait EthFees: LoadFee {
percentiles,
header.gas_used(),
header.base_fee_per_gas().unwrap_or_default(),
&block.body.transactions,
block.body.transactions(),
&receipts,
)
.unwrap_or_default(),

View File

@ -3,12 +3,11 @@
use super::SpawnBlocking;
use crate::{EthApiTypes, FromEthApiError, FromEvmError, RpcNodeCore};
use alloy_consensus::{BlockHeader, Header, EMPTY_OMMER_ROOT_HASH};
use alloy_eips::{
eip4844::MAX_DATA_GAS_PER_BLOCK, eip7685::EMPTY_REQUESTS_HASH, merge::BEACON_NONCE,
};
use alloy_primitives::{BlockNumber, B256, U256};
use alloy_rpc_types_eth::{BlockNumberOrTag, Withdrawals};
use alloy_consensus::{BlockHeader, Transaction};
use alloy_eips::eip4844::MAX_DATA_GAS_PER_BLOCK;
use alloy_network::Network;
use alloy_primitives::B256;
use alloy_rpc_types_eth::BlockNumberOrTag;
use futures::Future;
use reth_chainspec::{EthChainSpec, EthereumHardforks};
use reth_errors::RethError;
@ -17,19 +16,17 @@ use reth_evm::{
ConfigureEvm, ConfigureEvmEnv, NextBlockEnvAttributes,
};
use reth_execution_types::ExecutionOutcome;
use reth_primitives::{
proofs::calculate_transaction_root, Block, BlockBody, BlockExt, InvalidTransactionError,
Receipt, RecoveredTx, SealedBlockWithSenders,
};
use reth_primitives::{BlockExt, InvalidTransactionError, RecoveredTx, SealedBlockWithSenders};
use reth_primitives_traits::receipt::ReceiptExt;
use reth_provider::{
BlockReader, BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, ProviderError,
ProviderReceipt, ProviderTx, ReceiptProvider, StateProviderFactory,
BlockReader, BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, ProviderBlock, ProviderError,
ProviderHeader, ProviderReceipt, ProviderTx, ReceiptProvider, StateProviderFactory,
};
use reth_revm::{
database::StateProviderDatabase,
primitives::{
BlockEnv, CfgEnvWithHandlerCfg, EVMError, Env, ExecutionResult, InvalidTransaction,
ResultAndState, SpecId,
ResultAndState,
},
};
use reth_rpc_eth_types::{EthApiError, PendingBlock, PendingBlockEnv, PendingBlockEnvOrigin};
@ -46,29 +43,40 @@ use tracing::debug;
///
/// Behaviour shared by several `eth_` RPC methods, not exclusive to `eth_` blocks RPC methods.
pub trait LoadPendingBlock:
EthApiTypes
+ RpcNodeCore<
Provider: BlockReaderIdExt<
Transaction = reth_primitives::TransactionSigned,
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
Header = reth_primitives::Header,
> + EvmEnvProvider
EthApiTypes<
NetworkTypes: Network<
HeaderResponse = alloy_rpc_types_eth::Header<ProviderHeader<Self::Provider>>,
>,
> + RpcNodeCore<
Provider: BlockReaderIdExt<Receipt: ReceiptExt>
+ EvmEnvProvider<ProviderHeader<Self::Provider>>
+ ChainSpecProvider<ChainSpec: EthChainSpec + EthereumHardforks>
+ StateProviderFactory,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = ProviderTx<Self::Provider>>>,
Evm: ConfigureEvm<Header = Header, Transaction = ProviderTx<Self::Provider>>,
Evm: ConfigureEvm<
Header = ProviderHeader<Self::Provider>,
Transaction = ProviderTx<Self::Provider>,
>,
>
{
/// Returns a handle to the pending block.
///
/// Data access in default (L1) trait method implementations.
fn pending_block(&self) -> &Mutex<Option<PendingBlock>>;
#[expect(clippy::type_complexity)]
fn pending_block(
&self,
) -> &Mutex<Option<PendingBlock<ProviderBlock<Self::Provider>, ProviderReceipt<Self::Provider>>>>;
/// Configures the [`CfgEnvWithHandlerCfg`] and [`BlockEnv`] for the pending block
///
/// If no pending block is available, this will derive it from the `latest` block
fn pending_block_env_and_cfg(&self) -> Result<PendingBlockEnv, Self::Error> {
#[expect(clippy::type_complexity)]
fn pending_block_env_and_cfg(
&self,
) -> Result<
PendingBlockEnv<ProviderBlock<Self::Provider>, ProviderReceipt<Self::Provider>>,
Self::Error,
> {
if let Some(block) =
self.provider().pending_block_with_senders().map_err(Self::Error::from_eth_err)?
{
@ -154,8 +162,8 @@ pub trait LoadPendingBlock:
// check if the block is still good
if let Some(pending_block) = lock.as_ref() {
// this is guaranteed to be the `latest` header
if pending.block_env.number.to::<u64>() == pending_block.block.number &&
parent_hash == pending_block.block.parent_hash &&
if pending.block_env.number.to::<u64>() == pending_block.block.number() &&
parent_hash == pending_block.block.parent_hash() &&
now <= pending_block.expires_at
{
return Ok(Some((pending_block.block.clone(), pending_block.receipts.clone())));
@ -188,34 +196,24 @@ pub trait LoadPendingBlock:
}
}
/// Assembles a [`Receipt`] for a transaction, based on its [`ExecutionResult`].
/// Assembles a receipt for a transaction, based on its [`ExecutionResult`].
fn assemble_receipt(
&self,
tx: &RecoveredTx,
tx: &RecoveredTx<ProviderTx<Self::Provider>>,
result: ExecutionResult,
cumulative_gas_used: u64,
) -> Receipt {
#[allow(clippy::needless_update)]
Receipt {
tx_type: tx.tx_type(),
success: result.is_success(),
cumulative_gas_used,
logs: result.into_logs().into_iter().map(Into::into).collect(),
..Default::default()
}
}
) -> ProviderReceipt<Self::Provider>;
/// Calculates receipts root in block building.
///
/// Panics if block is not in the [`ExecutionOutcome`]'s block range.
fn receipts_root(
/// Assembles a pending block.
fn assemble_block(
&self,
_block_env: &BlockEnv,
execution_outcome: &ExecutionOutcome,
block_number: BlockNumber,
) -> B256 {
execution_outcome.receipts_root_slow(block_number).expect("Block is present")
}
cfg: CfgEnvWithHandlerCfg,
block_env: BlockEnv,
parent_hash: revm_primitives::B256,
state_root: revm_primitives::B256,
transactions: Vec<ProviderTx<Self::Provider>>,
receipts: &[ProviderReceipt<Self::Provider>],
) -> ProviderBlock<Self::Provider>;
/// Builds a pending block using the configured provider and pool.
///
@ -223,12 +221,19 @@ pub trait LoadPendingBlock:
///
/// After Cancun, if the origin is the actual pending block, the block includes the EIP-4788 pre
/// block contract call using the parent beacon block root received from the CL.
#[expect(clippy::type_complexity)]
fn build_block(
&self,
cfg: CfgEnvWithHandlerCfg,
block_env: BlockEnv,
parent_hash: B256,
) -> Result<(SealedBlockWithSenders, Vec<Receipt>), Self::Error>
) -> Result<
(
SealedBlockWithSenders<ProviderBlock<Self::Provider>>,
Vec<ProviderReceipt<Self::Provider>>,
),
Self::Error,
>
where
EthApiError: From<ProviderError>,
{
@ -253,14 +258,10 @@ pub trait LoadPendingBlock:
block_env.get_blob_gasprice().map(|gasprice| gasprice as u64),
));
let withdrawals: Option<Withdrawals> = None;
let withdrawals_root = None;
let chain_spec = self.provider().chain_spec();
let mut system_caller = SystemCaller::new(self.evm_config().clone(), chain_spec.clone());
let parent_beacon_block_root = None;
system_caller
.pre_block_blockhashes_contract_call(&mut db, &cfg, &block_env, parent_hash)
.map_err(|err| EthApiError::Internal(err.into()))?;
@ -301,8 +302,7 @@ pub trait LoadPendingBlock:
// There's only limited amount of blob space available per block, so we need to check if
// the EIP-4844 can still fit in the block
if let Some(blob_tx) = tx.transaction.as_eip4844() {
let tx_blob_gas = blob_tx.blob_gas();
if let Some(tx_blob_gas) = tx.blob_gas_used() {
if sum_blob_gas_used + tx_blob_gas > MAX_DATA_GAS_PER_BLOCK {
// we can't fit this _blob_ transaction into the block, so we mark it as
// invalid, which removes its dependent transactions from
@ -360,8 +360,7 @@ pub trait LoadPendingBlock:
db.commit(state);
// add to the total blob gas used if the transaction successfully executed
if let Some(blob_tx) = tx.transaction.as_eip4844() {
let tx_blob_gas = blob_tx.blob_gas();
if let Some(tx_blob_gas) = tx.blob_gas_used() {
sum_blob_gas_used += tx_blob_gas;
// if we've reached the max data gas per block, we can skip blob txs entirely
@ -388,7 +387,7 @@ pub trait LoadPendingBlock:
let balance_increments = post_block_withdrawals_balance_increments(
chain_spec.as_ref(),
block_env.timestamp.try_into().unwrap_or(u64::MAX),
&withdrawals.clone().unwrap_or_default(),
&[],
);
// increment account balances for withdrawals
@ -397,66 +396,23 @@ pub trait LoadPendingBlock:
// merge all transitions into bundle state.
db.merge_transitions(BundleRetention::PlainState);
let execution_outcome = ExecutionOutcome::new(
db.take_bundle(),
vec![receipts.clone()].into(),
block_number,
Vec::new(),
);
let execution_outcome: ExecutionOutcome<ProviderReceipt<Self::Provider>> =
ExecutionOutcome::new(
db.take_bundle(),
vec![receipts.clone()].into(),
block_number,
Vec::new(),
);
let hashed_state = db.database.hashed_post_state(execution_outcome.state());
let receipts_root = self.receipts_root(&block_env, &execution_outcome, block_number);
let logs_bloom =
execution_outcome.block_logs_bloom(block_number).expect("Block is present");
// calculate the state root
let state_root = db.database.state_root(hashed_state).map_err(Self::Error::from_eth_err)?;
// create the block header
let transactions_root = calculate_transaction_root(&executed_txs);
// check if cancun is activated to set eip4844 header fields correctly
let blob_gas_used =
(cfg.handler_cfg.spec_id >= SpecId::CANCUN).then_some(sum_blob_gas_used);
let requests_hash = chain_spec
.is_prague_active_at_timestamp(block_env.timestamp.to::<u64>())
.then_some(EMPTY_REQUESTS_HASH);
let header = Header {
parent_hash,
ommers_hash: EMPTY_OMMER_ROOT_HASH,
beneficiary: block_env.coinbase,
state_root,
transactions_root,
receipts_root,
withdrawals_root,
logs_bloom,
timestamp: block_env.timestamp.to::<u64>(),
mix_hash: block_env.prevrandao.unwrap_or_default(),
nonce: BEACON_NONCE.into(),
base_fee_per_gas: Some(base_fee),
number: block_number,
gas_limit: block_gas_limit,
difficulty: U256::ZERO,
gas_used: cumulative_gas_used,
blob_gas_used: blob_gas_used.map(Into::into),
excess_blob_gas: block_env.get_blob_excess_gas().map(Into::into),
extra_data: Default::default(),
parent_beacon_block_root,
requests_hash,
target_blobs_per_block: None,
};
// Convert Vec<Option<Receipt>> to Vec<Receipt>
let receipts: Vec<Receipt> = receipts.into_iter().flatten().collect();
let receipts: Vec<_> = receipts.into_iter().flatten().collect();
let block =
self.assemble_block(cfg, block_env, parent_hash, state_root, executed_txs, &receipts);
// seal the block
let block = Block {
header,
body: BlockBody { transactions: executed_txs, ommers: vec![], withdrawals },
};
Ok((SealedBlockWithSenders { block: block.seal_slow(), senders }, receipts))
}
}

View File

@ -4,7 +4,6 @@ use alloy_dyn_abi::TypedData;
use alloy_primitives::{Address, PrimitiveSignature as Signature};
use alloy_rpc_types_eth::TransactionRequest;
use dyn_clone::DynClone;
use reth_primitives::TransactionSigned;
use reth_rpc_eth_types::SignError;
use std::result;
@ -13,7 +12,7 @@ pub type Result<T> = result::Result<T, SignError>;
/// An Ethereum Signer used via RPC.
#[async_trait::async_trait]
pub trait EthSigner: Send + Sync + DynClone {
pub trait EthSigner<T>: Send + Sync + DynClone {
/// Returns the available accounts for this signer.
fn accounts(&self) -> Vec<Address>;
@ -26,17 +25,13 @@ pub trait EthSigner: Send + Sync + DynClone {
async fn sign(&self, address: Address, message: &[u8]) -> Result<Signature>;
/// signs a transaction request using the given account in request
async fn sign_transaction(
&self,
request: TransactionRequest,
address: &Address,
) -> Result<TransactionSigned>;
async fn sign_transaction(&self, request: TransactionRequest, address: &Address) -> Result<T>;
/// Encodes and signs the typed data according EIP-712. Payload must implement Eip712 trait.
fn sign_typed_data(&self, address: Address, payload: &TypedData) -> Result<Signature>;
}
dyn_clone::clone_trait_object!(EthSigner);
dyn_clone::clone_trait_object!(<T> EthSigner<T>);
/// Adds 20 random dev signers for access via the API. Used in dev mode.
#[auto_impl::auto_impl(&)]

View File

@ -22,11 +22,14 @@ pub trait EthApiSpec:
Network: NetworkInfo,
>
{
/// The transaction type signers are using.
type Transaction;
/// Returns the block node is started on.
fn starting_block(&self) -> U256;
/// Returns a handle to the signers owned by provider.
fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn EthSigner>>>;
fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn EthSigner<Self::Transaction>>>>;
/// Returns the current ethereum protocol version.
fn protocol_version(&self) -> impl Future<Output = RethResult<U64>> + Send {

View File

@ -2,7 +2,7 @@
//! RPC methods.
use super::{EthApiSpec, LoadPendingBlock, SpawnBlocking};
use crate::{EthApiTypes, FromEthApiError, RpcNodeCore, RpcNodeCoreExt};
use alloy_consensus::{constants::KECCAK_EMPTY, Header};
use alloy_consensus::{constants::KECCAK_EMPTY, BlockHeader};
use alloy_eips::BlockId;
use alloy_primitives::{Address, Bytes, B256, U256};
use alloy_rpc_types_eth::{Account, EIP1186AccountProofResponse};
@ -12,8 +12,8 @@ use reth_chainspec::{EthChainSpec, EthereumHardforks};
use reth_errors::RethError;
use reth_evm::ConfigureEvmEnv;
use reth_provider::{
BlockIdReader, BlockNumReader, ChainSpecProvider, EvmEnvProvider as _, StateProvider,
StateProviderBox, StateProviderFactory,
BlockIdReader, BlockNumReader, ChainSpecProvider, EvmEnvProvider as _, ProviderHeader,
StateProvider, StateProviderBox, StateProviderFactory,
};
use reth_rpc_eth_types::{EthApiError, PendingBlockEnv, RpcInvalidTransactionError};
use reth_transaction_pool::TransactionPool;
@ -247,14 +247,14 @@ pub trait LoadState:
/// This is used for tracing raw blocks
fn evm_env_for_raw_block(
&self,
header: &Header,
header: &ProviderHeader<Self::Provider>,
) -> impl Future<Output = Result<(CfgEnvWithHandlerCfg, BlockEnv), Self::Error>> + Send
where
Self: LoadPendingBlock + SpawnBlocking,
{
async move {
// get the parent config first
let (cfg, mut block_env, _) = self.evm_env_at(header.parent_hash.into()).await?;
let (cfg, mut block_env, _) = self.evm_env_at(header.parent_hash().into()).await?;
let after_merge = cfg.handler_cfg.spec_id >= SpecId::MERGE;
self.evm_config().fill_block_env(&mut block_env, header, after_merge);

View File

@ -3,14 +3,15 @@
use std::{fmt::Display, sync::Arc};
use crate::{FromEvmError, RpcNodeCore};
use alloy_consensus::Header;
use alloy_consensus::BlockHeader;
use alloy_primitives::B256;
use alloy_rpc_types_eth::{BlockId, TransactionInfo};
use futures::Future;
use reth_chainspec::ChainSpecProvider;
use reth_evm::{system_calls::SystemCaller, ConfigureEvm, ConfigureEvmEnv};
use reth_primitives::SealedBlockWithSenders;
use reth_provider::BlockReader;
use reth_primitives_traits::{BlockBody, SignedTransaction};
use reth_provider::{BlockReader, ProviderBlock, ProviderHeader, ProviderTx};
use reth_revm::database::StateProviderDatabase;
use reth_rpc_eth_types::{
cache::db::{StateCacheDb, StateCacheDbRefMutWrapper, StateProviderTraitObjWrapper},
@ -25,7 +26,15 @@ use revm_primitives::{
use super::{Call, LoadBlock, LoadPendingBlock, LoadState, LoadTransaction};
/// Executes CPU heavy tasks.
pub trait Trace: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Header>> {
pub trait Trace:
LoadState<
Provider: BlockReader,
Evm: ConfigureEvm<
Header = ProviderHeader<Self::Provider>,
Transaction = ProviderTx<Self::Provider>,
>,
>
{
/// Executes the [`EnvWithHandlerCfg`] against the given [Database] without committing state
/// changes.
fn inspect<DB, I>(
@ -190,7 +199,7 @@ pub trait Trace: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Hea
// we need to get the state of the parent block because we're essentially replaying the
// block the transaction is included in
let parent_block = block.parent_hash;
let parent_block = block.parent_hash();
let this = self.clone();
self.spawn_with_state_at_block(parent_block.into(), move |state| {
@ -205,7 +214,7 @@ pub trait Trace: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Hea
cfg.clone(),
block_env.clone(),
block_txs,
tx.hash(),
*tx.tx_hash(),
)?;
let env = EnvWithHandlerCfg::new_with_cfg_env(
@ -231,7 +240,7 @@ pub trait Trace: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Hea
fn trace_block_until<F, R>(
&self,
block_id: BlockId,
block: Option<Arc<SealedBlockWithSenders<<Self::Provider as BlockReader>::Block>>>,
block: Option<Arc<SealedBlockWithSenders<ProviderBlock<Self::Provider>>>>,
highest_index: Option<u64>,
config: TracingInspectorConfig,
f: F,
@ -271,7 +280,7 @@ pub trait Trace: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Hea
fn trace_block_until_with_inspector<Setup, Insp, F, R>(
&self,
block_id: BlockId,
block: Option<Arc<SealedBlockWithSenders>>,
block: Option<Arc<SealedBlockWithSenders<ProviderBlock<Self::Provider>>>>,
highest_index: Option<u64>,
mut inspector_setup: Setup,
f: F,
@ -304,7 +313,7 @@ pub trait Trace: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Hea
let Some(block) = block else { return Ok(None) };
if block.body.transactions.is_empty() {
if block.body.transactions().is_empty() {
// nothing to trace
return Ok(Some(Vec::new()))
}
@ -313,7 +322,7 @@ pub trait Trace: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Hea
self.spawn_tracing(move |this| {
// we need to get the state of the parent block because we're replaying this block
// on top of its parent block's state
let state_at = block.parent_hash;
let state_at = block.parent_hash();
let block_hash = block.hash();
let block_number = block_env.number.saturating_to::<u64>();
@ -329,7 +338,7 @@ pub trait Trace: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Hea
// 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.transactions().len(), |highest| {
// we need + 1 because the index is 0-based
highest as usize + 1
});
@ -341,7 +350,7 @@ pub trait Trace: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Hea
.enumerate()
.map(|(idx, (signer, tx))| {
let tx_info = TransactionInfo {
hash: Some(tx.hash()),
hash: Some(*tx.tx_hash()),
index: Some(idx as u64),
block_hash: Some(block_hash),
block_number: Some(block_number),
@ -389,7 +398,7 @@ pub trait Trace: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Hea
fn trace_block_with<F, R>(
&self,
block_id: BlockId,
block: Option<Arc<SealedBlockWithSenders>>,
block: Option<Arc<SealedBlockWithSenders<ProviderBlock<Self::Provider>>>>,
config: TracingInspectorConfig,
f: F,
) -> impl Future<Output = Result<Option<Vec<R>>, Self::Error>> + Send
@ -428,7 +437,7 @@ pub trait Trace: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Hea
fn trace_block_inspector<Setup, Insp, F, R>(
&self,
block_id: BlockId,
block: Option<Arc<SealedBlockWithSenders>>,
block: Option<Arc<SealedBlockWithSenders<ProviderBlock<Self::Provider>>>>,
insp_setup: Setup,
f: F,
) -> impl Future<Output = Result<Option<Vec<R>>, Self::Error>> + Send
@ -459,7 +468,7 @@ pub trait Trace: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Hea
/// already applied.
fn apply_pre_execution_changes<DB: Send + Database<Error: Display> + DatabaseCommit>(
&self,
block: &SealedBlockWithSenders,
block: &SealedBlockWithSenders<ProviderBlock<Self::Provider>>,
db: &mut DB,
cfg: &CfgEnvWithHandlerCfg,
block_env: &BlockEnv,
@ -472,12 +481,12 @@ pub trait Trace: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Hea
db,
cfg,
block_env,
block.header.parent_beacon_block_root,
block.header.parent_beacon_block_root(),
)
.map_err(|_| EthApiError::EvmCustom("failed to apply 4788 system call".to_string()))?;
system_caller
.pre_block_blockhashes_contract_call(db, cfg, block_env, block.header.parent_hash)
.pre_block_blockhashes_contract_call(db, cfg, block_env, block.header.parent_hash())
.map_err(|_| {
EthApiError::EvmCustom("failed to apply blockhashes system call".to_string())
})?;

View File

@ -11,8 +11,8 @@ use futures::Future;
use reth_node_api::BlockBody;
use reth_primitives::{
transaction::SignedTransactionIntoRecoveredExt, SealedBlockWithSenders, TransactionMeta,
TransactionSigned,
};
use reth_primitives_traits::SignedTransaction;
use reth_provider::{
BlockNumReader, BlockReaderIdExt, ProviderBlock, ProviderReceipt, ProviderTx, ReceiptProvider,
TransactionsProvider,
@ -60,7 +60,8 @@ pub trait EthTransactions: LoadTransaction<Provider: BlockReaderIdExt> {
/// Returns a handle for signing data.
///
/// Singer access in default (L1) trait method implementations.
fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn EthSigner>>>;
#[expect(clippy::type_complexity)]
fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn EthSigner<ProviderTx<Self::Provider>>>>>;
/// Returns the transaction by hash.
///
@ -213,7 +214,7 @@ pub trait EthTransactions: LoadTransaction<Provider: BlockReaderIdExt> {
let base_fee_per_gas = block.base_fee_per_gas();
if let Some((signer, tx)) = block.transactions_with_sender().nth(index) {
let tx_info = TransactionInfo {
hash: Some(tx.hash()),
hash: Some(*tx.tx_hash()),
block_hash: Some(block_hash),
block_number: Some(block_number),
base_fee: base_fee_per_gas.map(u128::from),
@ -294,7 +295,7 @@ pub trait EthTransactions: LoadTransaction<Provider: BlockReaderIdExt> {
.find(|(_, (signer, tx))| **signer == sender && (*tx).nonce() == nonce)
.map(|(index, (signer, tx))| {
let tx_info = TransactionInfo {
hash: Some(tx.hash()),
hash: Some(*tx.tx_hash()),
block_hash: Some(block_hash),
block_number: Some(block_number),
base_fee: base_fee_per_gas.map(u128::from),
@ -414,7 +415,7 @@ pub trait EthTransactions: LoadTransaction<Provider: BlockReaderIdExt> {
&self,
from: &Address,
txn: TransactionRequest,
) -> impl Future<Output = Result<TransactionSigned, Self::Error>> + Send {
) -> impl Future<Output = Result<ProviderTx<Self::Provider>, Self::Error>> + Send {
async move {
self.find_signer(from)?
.sign_transaction(txn, from)
@ -467,10 +468,11 @@ pub trait EthTransactions: LoadTransaction<Provider: BlockReaderIdExt> {
}
/// Returns the signer for the given account, if found in configured signers.
#[expect(clippy::type_complexity)]
fn find_signer(
&self,
account: &Address,
) -> Result<Box<(dyn EthSigner + 'static)>, Self::Error> {
) -> Result<Box<(dyn EthSigner<ProviderTx<Self::Provider>> + 'static)>, Self::Error> {
self.signers()
.read()
.iter()

View File

@ -29,7 +29,7 @@ pub use reth_rpc_eth_types::error::{
AsEthApiError, FromEthApiError, FromEvmError, IntoEthApiError,
};
pub use reth_rpc_types_compat::TransactionCompat;
pub use types::{EthApiTypes, FullEthApiTypes, RpcBlock, RpcReceipt, RpcTransaction};
pub use types::{EthApiTypes, FullEthApiTypes, RpcBlock, RpcHeader, RpcReceipt, RpcTransaction};
#[cfg(feature = "client")]
pub use bundle::{EthBundleApiClient, EthCallBundleApiClient};

View File

@ -24,7 +24,7 @@ pub trait EthApiTypes: Send + Sync + Clone {
+ Send
+ Sync;
/// Blockchain primitive types, specific to network, e.g. block and transaction.
type NetworkTypes: Network<HeaderResponse = alloy_rpc_types_eth::Header>;
type NetworkTypes: Network;
/// Conversion methods for transaction RPC type.
type TransactionCompat: Send + Sync + Clone + fmt::Debug;
@ -41,6 +41,9 @@ pub type RpcBlock<T> = Block<RpcTransaction<T>, <T as Network>::HeaderResponse>;
/// Adapter for network specific receipt type.
pub type RpcReceipt<T> = <T as Network>::ReceiptResponse;
/// Adapter for network specific header type.
pub type RpcHeader<T> = <T as Network>::HeaderResponse;
/// Adapter for network specific error type.
pub type RpcError<T> = <T as EthApiTypes>::Error;

View File

@ -14,13 +14,13 @@ use revm_primitives::{BlockEnv, CfgEnvWithHandlerCfg};
/// Configured [`BlockEnv`] and [`CfgEnvWithHandlerCfg`] for a pending block.
#[derive(Debug, Clone, Constructor)]
pub struct PendingBlockEnv {
pub struct PendingBlockEnv<B: Block = reth_primitives::Block, R = Receipt> {
/// Configured [`CfgEnvWithHandlerCfg`] for the pending block.
pub cfg: CfgEnvWithHandlerCfg,
/// Configured [`BlockEnv`] for the pending block.
pub block_env: BlockEnv,
/// Origin block for the config
pub origin: PendingBlockEnvOrigin,
pub origin: PendingBlockEnvOrigin<B, R>,
}
/// The origin for a configured [`PendingBlockEnv`]
@ -77,11 +77,11 @@ impl<B: Block, R> PendingBlockEnvOrigin<B, R> {
/// Locally built pending block for `pending` tag.
#[derive(Debug, Constructor)]
pub struct PendingBlock {
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,
pub block: SealedBlockWithSenders<B>,
/// The receipts for the pending block
pub receipts: Vec<Receipt>,
pub receipts: Vec<R>,
}

View File

@ -8,7 +8,7 @@ use std::{
use alloy_eips::BlockId;
use alloy_primitives::{TxHash, B256};
use alloy_rpc_types_eth::{transaction::TransactionRequest, Block, Transaction};
use alloy_rpc_types_eth::{transaction::TransactionRequest, Block, Header, Transaction};
use alloy_rpc_types_trace::{
common::TraceResult,
geth::{GethDebugTracerType, GethDebugTracingOptions, GethTrace},
@ -77,7 +77,7 @@ pub trait DebugApiExt {
impl<T> DebugApiExt for T
where
T: EthApiClient<Transaction, Block, Receipt> + DebugApiClient + Sync,
T: EthApiClient<Transaction, Block, Receipt, Header> + DebugApiClient + Sync,
{
type Provider = T;

View File

@ -1,7 +1,7 @@
//! Integration tests for the trace API.
use alloy_primitives::map::HashSet;
use alloy_rpc_types_eth::{Block, Transaction};
use alloy_rpc_types_eth::{Block, Header, Transaction};
use alloy_rpc_types_trace::{
filter::TraceFilter, parity::TraceType, tracerequest::TraceCallRequest,
};
@ -113,7 +113,7 @@ async fn debug_trace_block_entire_chain() {
let client = HttpClientBuilder::default().build(url).unwrap();
let current_block: u64 =
<HttpClient as EthApiClient<Transaction, Block, Receipt>>::block_number(&client)
<HttpClient as EthApiClient<Transaction, Block, Receipt, Header>>::block_number(&client)
.await
.unwrap()
.try_into()

View File

@ -14,6 +14,7 @@ workspace = true
[dependencies]
# reth
reth-primitives.workspace = true
reth-primitives-traits.workspace = true
# ethereum
alloy-eips.workspace = true

View File

@ -1,13 +1,13 @@
//! Compatibility functions for rpc `Block` type.
use alloy_consensus::Sealed;
use alloy_consensus::{BlockHeader, Sealable, Sealed};
use alloy_eips::eip4895::Withdrawals;
use alloy_primitives::{B256, U256};
use alloy_rlp::Encodable;
use alloy_rpc_types_eth::{
Block, BlockTransactions, BlockTransactionsKind, Header, TransactionInfo,
};
use reth_primitives::{Block as PrimitiveBlock, BlockWithSenders, TransactionSigned};
use reth_primitives::{transaction::SignedTransactionIntoRecoveredExt, BlockWithSenders};
use reth_primitives_traits::{Block as BlockTrait, BlockBody, SignedTransaction};
use crate::{transaction::from_recovered_with_block_context, TransactionCompat};
@ -15,19 +15,24 @@ use crate::{transaction::from_recovered_with_block_context, TransactionCompat};
/// [`BlockTransactionsKind`]
///
/// If a `block_hash` is provided, then this is used, otherwise the block hash is computed.
pub fn from_block<T: TransactionCompat>(
block: BlockWithSenders,
#[expect(clippy::type_complexity)]
pub fn from_block<T, B>(
block: BlockWithSenders<B>,
total_difficulty: U256,
kind: BlockTransactionsKind,
block_hash: Option<B256>,
tx_resp_builder: &T,
) -> Result<Block<T::Transaction>, T::Error> {
) -> Result<Block<T::Transaction, Header<B::Header>>, T::Error>
where
T: TransactionCompat<<<B as BlockTrait>::Body as BlockBody>::Transaction>,
B: BlockTrait,
{
match kind {
BlockTransactionsKind::Hashes => {
Ok(from_block_with_tx_hashes::<T::Transaction>(block, total_difficulty, block_hash))
Ok(from_block_with_tx_hashes::<T::Transaction, B>(block, total_difficulty, block_hash))
}
BlockTransactionsKind::Full => {
from_block_full::<T>(block, total_difficulty, block_hash, tx_resp_builder)
from_block_full::<T, B>(block, total_difficulty, block_hash, tx_resp_builder)
}
}
}
@ -37,13 +42,16 @@ pub fn from_block<T: TransactionCompat>(
///
/// 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>(
block: BlockWithSenders,
pub fn from_block_with_tx_hashes<T, B>(
block: BlockWithSenders<B>,
total_difficulty: U256,
block_hash: Option<B256>,
) -> Block<T> {
let block_hash = block_hash.unwrap_or_else(|| block.header.hash_slow());
let transactions = block.body.transactions.iter().map(|tx| tx.hash()).collect();
) -> Block<T, Header<B::Header>>
where
B: BlockTrait,
{
let block_hash = block_hash.unwrap_or_else(|| block.header().hash_slow());
let transactions = block.body().transactions().iter().map(|tx| *tx.tx_hash()).collect();
from_block_with_transactions(
block.length(),
@ -59,25 +67,30 @@ pub fn from_block_with_tx_hashes<T>(
///
/// This will populate the `transactions` field with the _full_
/// [`TransactionCompat::Transaction`] objects: [`BlockTransactions::Full`]
pub fn from_block_full<T: TransactionCompat>(
mut block: BlockWithSenders,
#[expect(clippy::type_complexity)]
pub fn from_block_full<T, B>(
block: BlockWithSenders<B>,
total_difficulty: U256,
block_hash: Option<B256>,
tx_resp_builder: &T,
) -> Result<Block<T::Transaction>, T::Error> {
let block_hash = block_hash.unwrap_or_else(|| block.block.header.hash_slow());
let block_number = block.block.number;
let base_fee_per_gas = block.block.base_fee_per_gas;
) -> 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();
// 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 = std::mem::take(&mut block.block.body.transactions);
let transactions = block.block.body().transactions().to_vec();
let transactions_with_senders = transactions.into_iter().zip(block.senders);
let transactions = transactions_with_senders
.enumerate()
.map(|(idx, (tx, sender))| {
let tx_hash = tx.hash();
let tx_hash = *tx.tx_hash();
let signed_tx_ec_recovered = tx.with_signer(sender);
let tx_info = TransactionInfo {
hash: Some(tx_hash),
@ -87,7 +100,7 @@ pub fn from_block_full<T: TransactionCompat>(
index: Some(idx as u64),
};
from_recovered_with_block_context::<TransactionSigned, T>(
from_recovered_with_block_context::<_, T>(
signed_tx_ec_recovered,
tx_info,
tx_resp_builder,
@ -105,23 +118,28 @@ pub fn from_block_full<T: TransactionCompat>(
}
#[inline]
fn from_block_with_transactions<T>(
fn from_block_with_transactions<T, B: BlockTrait>(
block_length: usize,
block_hash: B256,
block: PrimitiveBlock,
block: B,
total_difficulty: U256,
transactions: BlockTransactions<T>,
) -> Block<T> {
) -> Block<T, Header<B::Header>> {
let withdrawals = block
.header
.withdrawals_root
.header()
.withdrawals_root()
.is_some()
.then(|| block.body.withdrawals.map(Withdrawals::into_inner).map(Into::into))
.then(|| block.body().withdrawals().cloned().map(Withdrawals::into_inner).map(Into::into))
.flatten();
let uncles = block.body.ommers.into_iter().map(|h| h.hash_slow()).collect();
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(block.header, block_hash),
Sealed::new_unchecked(header, block_hash),
Some(total_difficulty),
Some(U256::from(block_length)),
);

View File

@ -1,3 +1,4 @@
use alloy_consensus::BlockHeader;
use alloy_eips::{eip2718::Encodable2718, BlockId, BlockNumberOrTag};
use alloy_primitives::{Address, Bytes, B256, U256};
use alloy_rlp::{Decodable, Encodable};
@ -18,11 +19,11 @@ use reth_evm::{
execute::{BlockExecutorProvider, Executor},
ConfigureEvmEnv,
};
use reth_primitives::{Block, BlockExt, NodePrimitives, SealedBlockWithSenders};
use reth_primitives_traits::SignedTransaction;
use reth_primitives::{BlockExt, NodePrimitives, SealedBlockWithSenders};
use reth_primitives_traits::{Block as _, BlockBody, SignedTransaction};
use reth_provider::{
BlockReader, BlockReaderIdExt, ChainSpecProvider, HeaderProvider, StateProofProvider,
StateProviderFactory, TransactionVariant,
BlockReader, BlockReaderIdExt, ChainSpecProvider, HeaderProvider, ProviderBlock,
StateProofProvider, StateProviderFactory, TransactionVariant,
};
use reth_revm::{database::StateProviderDatabase, witness::ExecutionWitnessRecord};
use reth_rpc_api::DebugApiServer;
@ -81,9 +82,8 @@ where
+ StateProviderFactory
+ 'static,
Eth: EthApiTypes + TraceExt + 'static,
BlockExecutor: BlockExecutorProvider<
Primitives: NodePrimitives<Block = <<Eth as RpcNodeCore>::Provider as BlockReader>::Block>,
>,
BlockExecutor:
BlockExecutorProvider<Primitives: NodePrimitives<Block = ProviderBlock<Eth::Provider>>>,
{
/// Acquires a permit to execute a tracing call.
async fn acquire_trace_permit(&self) -> Result<OwnedSemaphorePermit, AcquireError> {
@ -93,7 +93,7 @@ where
/// Trace the entire block asynchronously
async fn trace_block(
&self,
block: Arc<SealedBlockWithSenders>,
block: Arc<SealedBlockWithSenders<ProviderBlock<Eth::Provider>>>,
cfg: CfgEnvWithHandlerCfg,
block_env: BlockEnv,
opts: GethDebugTracingOptions,
@ -101,8 +101,8 @@ where
// replay all transactions of the block
let this = self.clone();
self.eth_api()
.spawn_with_state_at_block(block.parent_hash.into(), move |state| {
let mut results = Vec::with_capacity(block.body.transactions.len());
.spawn_with_state_at_block(block.parent_hash().into(), move |state| {
let mut results = Vec::with_capacity(block.body.transactions().len());
let mut db = CacheDB::new(StateProviderDatabase::new(state));
this.eth_api().apply_pre_execution_changes(&block, &mut db, &cfg, &block_env)?;
@ -110,7 +110,7 @@ where
let mut transactions = block.transactions_with_sender().enumerate().peekable();
let mut inspector = None;
while let Some((index, (signer, tx))) = transactions.next() {
let tx_hash = tx.hash();
let tx_hash = *tx.tx_hash();
let env = EnvWithHandlerCfg {
env: Env::boxed(
@ -157,18 +157,22 @@ where
rlp_block: Bytes,
opts: GethDebugTracingOptions,
) -> Result<Vec<TraceResult>, Eth::Error> {
let block = Block::decode(&mut rlp_block.as_ref())
let block: ProviderBlock<Eth::Provider> = Decodable::decode(&mut rlp_block.as_ref())
.map_err(BlockError::RlpDecodeRawBlock)
.map_err(Eth::Error::from_eth_err)?;
let (cfg, block_env) = self.eth_api().evm_env_for_raw_block(&block.header).await?;
let (cfg, block_env) = self.eth_api().evm_env_for_raw_block(block.header()).await?;
// Depending on EIP-2 we need to recover the transactions differently
let senders = if self.inner.provider.chain_spec().is_homestead_active_at_block(block.number)
let senders = if self
.inner
.provider
.chain_spec()
.is_homestead_active_at_block(block.header().number())
{
block
.body
.transactions
.body()
.transactions()
.iter()
.map(|tx| {
tx.recover_signer()
@ -178,8 +182,8 @@ where
.collect::<Result<Vec<_>, Eth::Error>>()?
} else {
block
.body
.transactions
.body()
.transactions()
.iter()
.map(|tx| {
tx.recover_signer_unchecked()
@ -237,7 +241,7 @@ where
// we need to get the state of the parent block because we're essentially replaying the
// block the transaction is included in
let state_at: BlockId = block.parent_hash.into();
let state_at: BlockId = block.parent_hash().into();
let block_hash = block.hash();
let this = self.clone();
@ -258,7 +262,7 @@ where
cfg.clone(),
block_env.clone(),
block_txs,
tx.hash(),
*tx.tx_hash(),
)?;
let env = EnvWithHandlerCfg {
@ -277,7 +281,7 @@ where
Some(TransactionContext {
block_hash: Some(block_hash),
tx_index: Some(index),
tx_hash: Some(tx.hash()),
tx_hash: Some(*tx.tx_hash()),
}),
&mut None,
)
@ -514,15 +518,15 @@ where
// we're essentially replaying the transactions in the block here, hence we need the state
// that points to the beginning of the block, which is the state at the parent block
let mut at = block.parent_hash;
let mut at = block.parent_hash();
let mut replay_block_txs = true;
// if a transaction index is provided, we need to replay the transactions until the index
let num_txs = transaction_index.index().unwrap_or(block.body.transactions.len());
let num_txs = transaction_index.index().unwrap_or_else(|| block.body.transactions().len());
// but if all transactions are to be replayed, we can use the state at the block itself
// this works with the exception of the PENDING block, because its state might not exist if
// built locally
if !target_block.is_pending() && num_txs == block.body.transactions.len() {
if !target_block.is_pending() && num_txs == block.body.transactions().len() {
at = block.hash();
replay_block_txs = false;
}
@ -622,7 +626,7 @@ where
.ok_or(EthApiError::HeaderNotFound(block_id.into()))?;
self.eth_api()
.spawn_with_state_at_block(block.parent_hash.into(), move |state_provider| {
.spawn_with_state_at_block(block.parent_hash().into(), move |state_provider| {
let db = StateProviderDatabase::new(&state_provider);
let block_executor = this.inner.block_executor.executor(db);
@ -630,7 +634,7 @@ where
let _ = block_executor
.execute_with_state_closure(
(&(*block).clone().unseal(), block.difficulty).into(),
(&(*block).clone().unseal(), block.difficulty()).into(),
|statedb: &State<_>| {
witness_record.record_executed_state(statedb);
},

View File

@ -9,7 +9,7 @@ use jsonrpsee::core::RpcResult as Result;
use reth_rpc_api::{EngineEthApiServer, EthApiServer, EthFilterApiServer};
/// Re-export for convenience
pub use reth_rpc_engine_api::EngineApi;
use reth_rpc_eth_api::{FullEthApiTypes, RpcBlock, RpcReceipt, RpcTransaction};
use reth_rpc_eth_api::{FullEthApiTypes, RpcBlock, RpcHeader, RpcReceipt, RpcTransaction};
use tracing_futures::Instrument;
macro_rules! engine_span {
@ -41,6 +41,7 @@ where
RpcTransaction<Eth::NetworkTypes>,
RpcBlock<Eth::NetworkTypes>,
RpcReceipt<Eth::NetworkTypes>,
RpcHeader<Eth::NetworkTypes>,
> + FullEthApiTypes,
EthFilter: EthFilterApiServer<RpcTransaction<Eth::NetworkTypes>>,
{

View File

@ -1,12 +1,13 @@
//! `Eth` bundle implementation and helpers.
use alloy_consensus::Transaction as _;
use alloy_consensus::{BlockHeader, Transaction as _};
use alloy_primitives::{Keccak256, U256};
use alloy_rpc_types_mev::{EthCallBundle, EthCallBundleResponse, EthCallBundleTransactionResult};
use jsonrpsee::core::RpcResult;
use reth_chainspec::EthChainSpec;
use reth_evm::{ConfigureEvm, ConfigureEvmEnv};
use reth_primitives::{PooledTransactionsElement, Transaction};
use reth_primitives::PooledTransactionsElement;
use reth_primitives_traits::SignedTransaction;
use reth_provider::{ChainSpecProvider, HeaderProvider};
use reth_revm::database::StateProviderDatabase;
use reth_rpc_eth_api::{
@ -15,12 +16,13 @@ use reth_rpc_eth_api::{
};
use reth_rpc_eth_types::{utils::recover_raw_transaction, EthApiError, RpcInvalidTransactionError};
use reth_tasks::pool::BlockingTaskGuard;
use reth_transaction_pool::{PoolConsensusTx, PoolPooledTx, PoolTransaction, TransactionPool};
use revm::{
db::{CacheDB, DatabaseCommit, DatabaseRef},
primitives::{ResultAndState, TxEnv},
};
use revm_primitives::{EnvKzgSettings, EnvWithHandlerCfg, SpecId, MAX_BLOB_GAS_PER_BLOCK};
use std::{ops::Deref, sync::Arc};
use std::sync::Arc;
/// `Eth` bundle implementation.
pub struct EthBundle<Eth> {
@ -42,7 +44,16 @@ impl<Eth> EthBundle<Eth> {
impl<Eth> EthBundle<Eth>
where
Eth: EthTransactions + LoadPendingBlock + Call + 'static,
Eth: EthTransactions<
Pool: TransactionPool<
Transaction: PoolTransaction<
Consensus: From<PooledTransactionsElement>,
Pooled = PooledTransactionsElement,
>,
>,
> + LoadPendingBlock
+ Call
+ 'static,
{
/// Simulates a bundle of transactions at the top of a given block number with the state of
/// another (or the same) block. This can be used to simulate future blocks with the current
@ -79,7 +90,7 @@ where
let transactions = txs
.into_iter()
.map(recover_raw_transaction::<PooledTransactionsElement>)
.map(recover_raw_transaction::<PoolPooledTx<Eth::Pool>>)
.collect::<Result<Vec<_>, _>>()?
.into_iter()
.map(|tx| tx.to_components())
@ -192,12 +203,10 @@ where
})?;
}
let tx = tx.into_transaction();
let tx: PoolConsensusTx<Eth::Pool> = tx.into();
hasher.update(tx.hash());
let gas_price = Transaction::effective_tip_per_gas(tx.deref(), basefee)
.ok_or_else(|| RpcInvalidTransactionError::FeeCapTooLow)
.map_err(Eth::Error::from_eth_err)?;
hasher.update(*tx.tx_hash());
let gas_price = tx.effective_gas_price(basefee);
eth_api.evm_config().fill_tx_env(evm.tx_mut(), &tx, signer);
let ResultAndState { result, state } =
evm.transact().map_err(Eth::Error::from_evm_err)?;
@ -235,7 +244,7 @@ where
gas_price: U256::from(gas_price),
gas_used,
to_address: tx.to(),
tx_hash: tx.hash(),
tx_hash: *tx.tx_hash(),
value,
revert,
};

View File

@ -244,7 +244,7 @@ pub struct EthApiInner<Provider: BlockReader, Pool, Network, EvmConfig> {
/// An interface to interact with the network
network: Network,
/// All configured Signers
signers: parking_lot::RwLock<Vec<Box<dyn EthSigner>>>,
signers: parking_lot::RwLock<Vec<Box<dyn EthSigner<Provider::Transaction>>>>,
/// The async cache frontend for eth related data
eth_cache: EthStateCache<Provider::Block, Provider::Receipt>,
/// The async gas oracle frontend for gas price suggestions
@ -260,7 +260,7 @@ pub struct EthApiInner<Provider: BlockReader, Pool, Network, EvmConfig> {
/// The type that can spawn tasks which would otherwise block.
task_spawner: Box<dyn TaskSpawner>,
/// Cached pending block if any
pending_block: Mutex<Option<PendingBlock>>,
pending_block: Mutex<Option<PendingBlock<Provider::Block, Provider::Receipt>>>,
/// A pool dedicated to CPU heavy blocking tasks.
blocking_task_pool: BlockingTaskPool,
/// Cache for block fees history
@ -343,7 +343,9 @@ where
/// Returns a handle to the pending block.
#[inline]
pub const fn pending_block(&self) -> &Mutex<Option<PendingBlock>> {
pub const fn pending_block(
&self,
) -> &Mutex<Option<PendingBlock<Provider::Block, Provider::Receipt>>> {
&self.pending_block
}
@ -397,7 +399,9 @@ where
/// Returns a handle to the signers.
#[inline]
pub const fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn EthSigner>>> {
pub const fn signers(
&self,
) -> &parking_lot::RwLock<Vec<Box<dyn EthSigner<Provider::Transaction>>>> {
&self.signers
}
@ -579,7 +583,7 @@ mod tests {
/// Invalid block range
#[tokio::test]
async fn test_fee_history_empty() {
let response = <EthApi<_, _, _, _> as EthApiServer<_, _, _>>::fee_history(
let response = <EthApi<_, _, _, _> as EthApiServer<_, _, _, _>>::fee_history(
&build_test_eth_api(NoopProvider::default()),
U64::from(1),
BlockNumberOrTag::Latest,
@ -601,7 +605,7 @@ mod tests {
let (eth_api, _, _) =
prepare_eth_api(newest_block, oldest_block, block_count, MockEthProvider::default());
let response = <EthApi<_, _, _, _> as EthApiServer<_, _, _>>::fee_history(
let response = <EthApi<_, _, _, _> as EthApiServer<_, _, _, _>>::fee_history(
&eth_api,
U64::from(newest_block + 1),
newest_block.into(),
@ -624,7 +628,7 @@ mod tests {
let (eth_api, _, _) =
prepare_eth_api(newest_block, oldest_block, block_count, MockEthProvider::default());
let response = <EthApi<_, _, _, _> as EthApiServer<_, _, _>>::fee_history(
let response = <EthApi<_, _, _, _> as EthApiServer<_, _, _, _>>::fee_history(
&eth_api,
U64::from(1),
(newest_block + 1000).into(),
@ -647,7 +651,7 @@ mod tests {
let (eth_api, _, _) =
prepare_eth_api(newest_block, oldest_block, block_count, MockEthProvider::default());
let response = <EthApi<_, _, _, _> as EthApiServer<_, _, _>>::fee_history(
let response = <EthApi<_, _, _, _> as EthApiServer<_, _, _, _>>::fee_history(
&eth_api,
U64::from(0),
newest_block.into(),

View File

@ -1,8 +1,10 @@
//! Contains RPC handler implementations specific to blocks.
use alloy_consensus::BlockHeader;
use alloy_rpc_types_eth::{BlockId, TransactionReceipt};
use reth_primitives::TransactionMeta;
use reth_provider::{BlockReader, HeaderProvider};
use reth_primitives_traits::{BlockBody, SignedTransaction};
use reth_provider::BlockReader;
use reth_rpc_eth_api::{
helpers::{EthBlocks, LoadBlock, LoadPendingBlock, LoadReceipt, SpawnBlocking},
RpcNodeCoreExt, RpcReceipt,
@ -16,7 +18,10 @@ where
Self: LoadBlock<
Error = EthApiError,
NetworkTypes: alloy_network::Network<ReceiptResponse = TransactionReceipt>,
Provider: HeaderProvider,
Provider: BlockReader<
Transaction = reth_primitives::TransactionSigned,
Receipt = reth_primitives::Receipt,
>,
>,
Provider: BlockReader,
{
@ -28,21 +33,21 @@ where
Self: LoadReceipt,
{
if let Some((block, receipts)) = self.load_block_and_receipts(block_id).await? {
let block_number = block.number;
let base_fee = block.base_fee_per_gas;
let block_number = block.number();
let base_fee = block.base_fee_per_gas();
let block_hash = block.hash();
let excess_blob_gas = block.excess_blob_gas;
let timestamp = block.timestamp;
let excess_blob_gas = block.excess_blob_gas();
let timestamp = block.timestamp();
return block
.body
.transactions
.into_iter()
.transactions()
.iter()
.zip(receipts.iter())
.enumerate()
.map(|(idx, (tx, receipt))| {
let meta = TransactionMeta {
tx_hash: tx.hash(),
tx_hash: *tx.tx_hash(),
index: idx as u64,
block_hash,
block_number,
@ -50,7 +55,7 @@ where
excess_blob_gas,
timestamp,
};
EthReceiptBuilder::new(&tx, meta, receipt, &receipts)
EthReceiptBuilder::new(tx, meta, receipt, &receipts)
.map(|builder| builder.build())
})
.collect::<Result<Vec<_>, Self::Error>>()

View File

@ -3,7 +3,7 @@
use crate::EthApi;
use alloy_consensus::Header;
use reth_evm::ConfigureEvm;
use reth_provider::BlockReader;
use reth_provider::{BlockReader, ProviderHeader};
use reth_rpc_eth_api::helpers::{
estimate::EstimateCall, Call, EthCall, LoadPendingBlock, LoadState, SpawnBlocking,
};
@ -17,7 +17,7 @@ where
impl<Provider, Pool, Network, EvmConfig> Call for EthApi<Provider, Pool, Network, EvmConfig>
where
Self: LoadState<Evm: ConfigureEvm<Header = Header>> + SpawnBlocking,
Self: LoadState<Evm: ConfigureEvm<Header = ProviderHeader<Self::Provider>>> + SpawnBlocking,
EvmConfig: ConfigureEvm<Header = Header>,
Provider: BlockReader,
{

View File

@ -1,11 +1,18 @@
//! Support for building a pending block with transactions from local view of mempool.
use alloy_consensus::Header;
use alloy_consensus::{constants::EMPTY_WITHDRAWALS, Header, EMPTY_OMMER_ROOT_HASH};
use alloy_eips::{eip7685::EMPTY_REQUESTS_HASH, merge::BEACON_NONCE};
use alloy_primitives::U256;
use reth_chainspec::{EthChainSpec, EthereumHardforks};
use reth_evm::ConfigureEvm;
use reth_primitives::{
logs_bloom,
proofs::{calculate_receipt_root_no_memo, calculate_transaction_root},
BlockBody, Receipt,
};
use reth_provider::{
BlockReader, BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, ProviderTx,
StateProviderFactory,
BlockReader, BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, ProviderBlock,
ProviderReceipt, ProviderTx, StateProviderFactory,
};
use reth_rpc_eth_api::{
helpers::{LoadPendingBlock, SpawnBlocking},
@ -13,14 +20,16 @@ use reth_rpc_eth_api::{
};
use reth_rpc_eth_types::PendingBlock;
use reth_transaction_pool::{PoolTransaction, TransactionPool};
use revm_primitives::{BlockEnv, CfgEnvWithHandlerCfg, SpecId, B256};
use crate::EthApi;
impl<Provider, Pool, Network, EvmConfig> LoadPendingBlock
for EthApi<Provider, Pool, Network, EvmConfig>
where
Self: SpawnBlocking
+ RpcNodeCore<
Self: SpawnBlocking<
NetworkTypes: alloy_network::Network<HeaderResponse = alloy_rpc_types_eth::Header>,
> + RpcNodeCore<
Provider: BlockReaderIdExt<
Transaction = reth_primitives::TransactionSigned,
Block = reth_primitives::Block,
@ -34,10 +43,81 @@ where
>,
Evm: ConfigureEvm<Header = Header, Transaction = ProviderTx<Self::Provider>>,
>,
Provider: BlockReader,
Provider: BlockReader<Block = reth_primitives::Block, Receipt = reth_primitives::Receipt>,
{
#[inline]
fn pending_block(&self) -> &tokio::sync::Mutex<Option<PendingBlock>> {
fn pending_block(
&self,
) -> &tokio::sync::Mutex<
Option<PendingBlock<ProviderBlock<Self::Provider>, ProviderReceipt<Self::Provider>>>,
> {
self.inner.pending_block()
}
fn assemble_block(
&self,
cfg: CfgEnvWithHandlerCfg,
block_env: BlockEnv,
parent_hash: revm_primitives::B256,
state_root: revm_primitives::B256,
transactions: Vec<ProviderTx<Self::Provider>>,
receipts: &[ProviderReceipt<Self::Provider>],
) -> reth_provider::ProviderBlock<Self::Provider> {
let transactions_root = calculate_transaction_root(&transactions);
let receipts_root = calculate_receipt_root_no_memo(&receipts.iter().collect::<Vec<_>>());
let logs_bloom = logs_bloom(receipts.iter().flat_map(|r| &r.logs));
let header = Header {
parent_hash,
ommers_hash: EMPTY_OMMER_ROOT_HASH,
beneficiary: block_env.coinbase,
state_root,
transactions_root,
receipts_root,
withdrawals_root: (cfg.handler_cfg.spec_id >= SpecId::SHANGHAI)
.then_some(EMPTY_WITHDRAWALS),
logs_bloom,
timestamp: block_env.timestamp.to::<u64>(),
mix_hash: block_env.prevrandao.unwrap_or_default(),
nonce: BEACON_NONCE.into(),
base_fee_per_gas: Some(block_env.basefee.to::<u64>()),
number: block_env.number.to::<u64>(),
gas_limit: block_env.gas_limit.to::<u64>(),
difficulty: U256::ZERO,
gas_used: receipts.last().map(|r| r.cumulative_gas_used).unwrap_or_default(),
blob_gas_used: (cfg.handler_cfg.spec_id >= SpecId::CANCUN).then(|| {
transactions.iter().map(|tx| tx.blob_gas_used().unwrap_or_default()).sum::<u64>()
}),
excess_blob_gas: block_env.get_blob_excess_gas().map(Into::into),
extra_data: Default::default(),
parent_beacon_block_root: (cfg.handler_cfg.spec_id >= SpecId::CANCUN)
.then_some(B256::ZERO),
requests_hash: (cfg.handler_cfg.spec_id >= SpecId::PRAGUE)
.then_some(EMPTY_REQUESTS_HASH),
target_blobs_per_block: None,
};
// seal the block
reth_primitives::Block {
header,
body: BlockBody { transactions, ommers: vec![], withdrawals: None },
}
}
fn assemble_receipt(
&self,
tx: &reth_primitives::RecoveredTx<ProviderTx<Self::Provider>>,
result: revm_primitives::ExecutionResult,
cumulative_gas_used: u64,
) -> reth_provider::ProviderReceipt<Self::Provider> {
#[allow(clippy::needless_update)]
Receipt {
tx_type: tx.tx_type(),
success: result.is_success(),
cumulative_gas_used,
logs: result.into_logs().into_iter().map(Into::into).collect(),
..Default::default()
}
}
}

View File

@ -10,7 +10,6 @@ use alloy_primitives::{eip191_hash_message, Address, PrimitiveSignature as Signa
use alloy_rpc_types_eth::TransactionRequest;
use alloy_signer::SignerSync;
use alloy_signer_local::PrivateKeySigner;
use reth_primitives::TransactionSigned;
use reth_provider::BlockReader;
use reth_rpc_eth_api::helpers::{signer::Result, AddDevSigners, EthSigner};
use reth_rpc_eth_types::SignError;
@ -35,14 +34,14 @@ pub struct DevSigner {
#[allow(dead_code)]
impl DevSigner {
/// Generates a random dev signer which satisfies [`EthSigner`] trait
pub fn random() -> Box<dyn EthSigner> {
pub fn random<T: Decodable2718>() -> Box<dyn EthSigner<T>> {
let mut signers = Self::random_signers(1);
signers.pop().expect("expect to generate at least one signer")
}
/// Generates provided number of random dev signers
/// which satisfy [`EthSigner`] trait
pub fn random_signers(num: u32) -> Vec<Box<dyn EthSigner + 'static>> {
pub fn random_signers<T: Decodable2718>(num: u32) -> Vec<Box<dyn EthSigner<T> + 'static>> {
let mut signers = Vec::with_capacity(num as usize);
for _ in 0..num {
let sk = PrivateKeySigner::random_with(&mut rand::thread_rng());
@ -51,7 +50,7 @@ impl DevSigner {
let addresses = vec![address];
let accounts = HashMap::from([(address, sk)]);
signers.push(Box::new(Self { addresses, accounts }) as Box<dyn EthSigner>);
signers.push(Box::new(Self { addresses, accounts }) as Box<dyn EthSigner<T>>);
}
signers
}
@ -67,7 +66,7 @@ impl DevSigner {
}
#[async_trait::async_trait]
impl EthSigner for DevSigner {
impl<T: Decodable2718> EthSigner<T> for DevSigner {
fn accounts(&self) -> Vec<Address> {
self.addresses.clone()
}
@ -83,11 +82,7 @@ impl EthSigner for DevSigner {
self.sign_hash(hash, address)
}
async fn sign_transaction(
&self,
request: TransactionRequest,
address: &Address,
) -> Result<TransactionSigned> {
async fn sign_transaction(&self, request: TransactionRequest, address: &Address) -> Result<T> {
// create local signer wallet from signing key
let signer = self.accounts.get(address).ok_or(SignError::NoAccount)?.clone();
let wallet = EthereumWallet::from(signer);
@ -98,7 +93,7 @@ impl EthSigner for DevSigner {
// decode transaction into signed transaction type
let encoded = txn_envelope.encoded_2718();
let txn_signed = TransactionSigned::decode_2718(&mut encoded.as_ref())
let txn_signed = T::decode_2718(&mut encoded.as_ref())
.map_err(|_| SignError::InvalidTransactionRequest)?;
Ok(txn_signed)
@ -115,6 +110,7 @@ mod tests {
use alloy_consensus::Transaction;
use alloy_primitives::{Bytes, U256};
use alloy_rpc_types_eth::TransactionInput;
use reth_primitives::TransactionSigned;
use revm_primitives::TxKind;
use super::*;
@ -197,7 +193,9 @@ mod tests {
let data: TypedData = serde_json::from_str(eip_712_example).unwrap();
let signer = build_signer();
let from = *signer.addresses.first().unwrap();
let sig = signer.sign_typed_data(from, &data).unwrap();
let sig =
EthSigner::<reth_primitives::TransactionSigned>::sign_typed_data(&signer, from, &data)
.unwrap();
let expected = Signature::new(
U256::from_str_radix(
"5318aee9942b84885761bb20e768372b76e7ee454fc4d39b59ce07338d15a06c",
@ -219,7 +217,9 @@ mod tests {
let message = b"Test message";
let signer = build_signer();
let from = *signer.addresses.first().unwrap();
let sig = signer.sign(from, message).await.unwrap();
let sig = EthSigner::<reth_primitives::TransactionSigned>::sign(&signer, from, message)
.await
.unwrap();
let expected = Signature::new(
U256::from_str_radix(
"54313da7432e4058b8d22491b2e7dbb19c7186c35c24155bec0820a8a2bfe0c1",
@ -255,7 +255,8 @@ mod tests {
nonce: Some(0u64),
..Default::default()
};
let txn_signed = signer.sign_transaction(request, &from).await;
let txn_signed: std::result::Result<TransactionSigned, SignError> =
signer.sign_transaction(request, &from).await;
assert!(txn_signed.is_ok());
assert_eq!(Bytes::from(message.to_vec()), txn_signed.unwrap().input().0);

View File

@ -1,7 +1,9 @@
use alloy_primitives::U256;
use reth_chainspec::EthereumHardforks;
use reth_network_api::NetworkInfo;
use reth_provider::{BlockNumReader, BlockReader, ChainSpecProvider, StageCheckpointReader};
use reth_provider::{
BlockNumReader, BlockReader, ChainSpecProvider, ProviderTx, StageCheckpointReader,
};
use reth_rpc_eth_api::{helpers::EthApiSpec, RpcNodeCore};
use crate::EthApi;
@ -16,11 +18,16 @@ where
>,
Provider: BlockReader,
{
type Transaction = ProviderTx<Provider>;
fn starting_block(&self) -> U256 {
self.inner.starting_block()
}
fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn reth_rpc_eth_api::helpers::EthSigner>>> {
fn signers(
&self,
) -> &parking_lot::RwLock<Vec<Box<dyn reth_rpc_eth_api::helpers::EthSigner<Self::Transaction>>>>
{
self.inner.signers()
}
}

View File

@ -1,15 +1,20 @@
//! Contains RPC handler implementations specific to tracing.
use alloy_consensus::Header;
use reth_evm::ConfigureEvm;
use reth_provider::BlockReader;
use reth_provider::{BlockReader, ProviderHeader, ProviderTx};
use reth_rpc_eth_api::helpers::{LoadState, Trace};
use crate::EthApi;
impl<Provider, Pool, Network, EvmConfig> Trace for EthApi<Provider, Pool, Network, EvmConfig>
where
Self: LoadState<Provider: BlockReader, Evm: ConfigureEvm<Header = Header>>,
Self: LoadState<
Provider: BlockReader,
Evm: ConfigureEvm<
Header = ProviderHeader<Self::Provider>,
Transaction = ProviderTx<Self::Provider>,
>,
>,
Provider: BlockReader,
{
}

View File

@ -1,6 +1,6 @@
//! Contains RPC handler implementations specific to transactions
use reth_provider::{BlockReader, BlockReaderIdExt, TransactionsProvider};
use reth_provider::{BlockReader, BlockReaderIdExt, ProviderTx, TransactionsProvider};
use reth_rpc_eth_api::{
helpers::{EthSigner, EthTransactions, LoadTransaction, SpawnBlocking},
FullEthApiTypes, RpcNodeCoreExt,
@ -13,10 +13,10 @@ impl<Provider, Pool, Network, EvmConfig> EthTransactions
for EthApi<Provider, Pool, Network, EvmConfig>
where
Self: LoadTransaction<Provider: BlockReaderIdExt>,
Provider: BlockReader,
Provider: BlockReader<Transaction = ProviderTx<Self::Provider>>,
{
#[inline]
fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn EthSigner>>> {
fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn EthSigner<ProviderTx<Self::Provider>>>>> {
self.inner.signers()
}
}

View File

@ -1,5 +1,6 @@
//! `Eth` Sim bundle implementation and helpers.
use alloy_consensus::BlockHeader;
use alloy_eips::BlockNumberOrTag;
use alloy_primitives::U256;
use alloy_rpc_types_eth::BlockId;
@ -10,8 +11,7 @@ use alloy_rpc_types_mev::{
use jsonrpsee::core::RpcResult;
use reth_chainspec::EthChainSpec;
use reth_evm::{ConfigureEvm, ConfigureEvmEnv};
use reth_primitives::{PooledTransactionsElement, TransactionSigned};
use reth_provider::{ChainSpecProvider, HeaderProvider};
use reth_provider::{ChainSpecProvider, HeaderProvider, ProviderTx};
use reth_revm::database::StateProviderDatabase;
use reth_rpc_api::MevSimApiServer;
use reth_rpc_eth_api::{
@ -20,6 +20,7 @@ use reth_rpc_eth_api::{
};
use reth_rpc_eth_types::{utils::recover_raw_transaction, EthApiError};
use reth_tasks::pool::BlockingTaskGuard;
use reth_transaction_pool::{PoolConsensusTx, PoolPooledTx, PoolTransaction, TransactionPool};
use revm::{
db::CacheDB,
primitives::{Address, EnvWithHandlerCfg, ResultAndState, SpecId, TxEnv},
@ -45,9 +46,9 @@ const SBUNDLE_PAYOUT_MAX_COST: u64 = 30_000;
/// A flattened representation of a bundle item containing transaction and associated metadata.
#[derive(Clone, Debug)]
pub struct FlattenedBundleItem {
pub struct FlattenedBundleItem<T> {
/// The signed transaction
pub tx: TransactionSigned,
pub tx: T,
/// The address that signed the transaction
pub signer: Address,
/// Whether the transaction is allowed to revert
@ -93,7 +94,7 @@ where
fn parse_and_flatten_bundle(
&self,
request: &SendBundleRequest,
) -> Result<Vec<FlattenedBundleItem>, EthApiError> {
) -> Result<Vec<FlattenedBundleItem<ProviderTx<Eth::Provider>>>, EthApiError> {
let mut items = Vec::new();
// Stack for processing bundles
@ -171,10 +172,11 @@ where
match &body[idx] {
BundleItem::Tx { tx, can_revert } => {
let recovered_tx =
recover_raw_transaction::<PooledTransactionsElement>(tx.clone())
recover_raw_transaction::<PoolPooledTx<Eth::Pool>>(tx.clone())
.map_err(EthApiError::from)?;
let (tx, signer) = recovered_tx.to_components();
let tx = tx.into_transaction();
let tx: PoolConsensusTx<Eth::Pool> =
<Eth::Pool as TransactionPool>::Transaction::pooled_into_consensus(tx);
let refund_percent =
validity.as_ref().and_then(|v| v.refund.as_ref()).and_then(|refunds| {

View File

@ -1,8 +1,8 @@
use alloy_consensus::Transaction;
use alloy_consensus::{BlockHeader, Transaction};
use alloy_eips::{BlockId, BlockNumberOrTag};
use alloy_network::{ReceiptResponse, TransactionResponse};
use alloy_primitives::{Address, Bytes, TxHash, B256, U256};
use alloy_rpc_types_eth::{BlockTransactions, Header, TransactionReceipt};
use alloy_rpc_types_eth::{BlockTransactions, TransactionReceipt};
use alloy_rpc_types_trace::{
otterscan::{
BlockDetails, ContractCreator, InternalOperation, OperationType, OtsBlockTransactions,
@ -15,7 +15,7 @@ use jsonrpsee::{core::RpcResult, types::ErrorObjectOwned};
use reth_rpc_api::{EthApiServer, OtterscanServer};
use reth_rpc_eth_api::{
helpers::{EthTransactions, TraceExt},
FullEthApiTypes, RpcBlock, RpcReceipt, RpcTransaction, TransactionCompat,
FullEthApiTypes, RpcBlock, RpcHeader, RpcReceipt, RpcTransaction, TransactionCompat,
};
use reth_rpc_eth_types::{utils::binary_search, EthApiError};
use reth_rpc_server_types::result::internal_rpc_err;
@ -49,7 +49,7 @@ where
&self,
block: RpcBlock<Eth::NetworkTypes>,
receipts: Vec<RpcReceipt<Eth::NetworkTypes>>,
) -> RpcResult<BlockDetails> {
) -> RpcResult<BlockDetails<RpcHeader<Eth::NetworkTypes>>> {
// blob fee is burnt, so we don't need to calculate it
let total_fees = receipts
.iter()
@ -61,18 +61,23 @@ where
}
#[async_trait]
impl<Eth> OtterscanServer<RpcTransaction<Eth::NetworkTypes>> for OtterscanApi<Eth>
impl<Eth> OtterscanServer<RpcTransaction<Eth::NetworkTypes>, RpcHeader<Eth::NetworkTypes>>
for OtterscanApi<Eth>
where
Eth: EthApiServer<
RpcTransaction<Eth::NetworkTypes>,
RpcBlock<Eth::NetworkTypes>,
RpcReceipt<Eth::NetworkTypes>,
RpcHeader<Eth::NetworkTypes>,
> + EthTransactions
+ TraceExt
+ 'static,
{
/// Handler for `{ots,erigon}_getHeaderByNumber`
async fn get_header_by_number(&self, block_number: u64) -> RpcResult<Option<Header>> {
async fn get_header_by_number(
&self,
block_number: u64,
) -> RpcResult<Option<RpcHeader<Eth::NetworkTypes>>> {
self.eth.header_by_number(BlockNumberOrTag::Number(block_number)).await
}
@ -165,7 +170,10 @@ where
}
/// Handler for `ots_getBlockDetails`
async fn get_block_details(&self, block_number: u64) -> RpcResult<BlockDetails<Header>> {
async fn get_block_details(
&self,
block_number: u64,
) -> RpcResult<BlockDetails<RpcHeader<Eth::NetworkTypes>>> {
let block_id = block_number.into();
let block = self.eth.block_by_number(block_id, true);
let block_id = block_id.into();
@ -178,7 +186,10 @@ where
}
/// Handler for `getBlockDetailsByHash`
async fn get_block_details_by_hash(&self, block_hash: B256) -> RpcResult<BlockDetails<Header>> {
async fn get_block_details_by_hash(
&self,
block_hash: B256,
) -> RpcResult<BlockDetails<RpcHeader<Eth::NetworkTypes>>> {
let block = self.eth.block_by_hash(block_hash, true);
let block_id = block_hash.into();
let receipts = self.eth.block_receipts(block_id);
@ -195,7 +206,9 @@ where
block_number: u64,
page_number: usize,
page_size: usize,
) -> RpcResult<OtsBlockTransactions<RpcTransaction<Eth::NetworkTypes>, Header>> {
) -> RpcResult<
OtsBlockTransactions<RpcTransaction<Eth::NetworkTypes>, RpcHeader<Eth::NetworkTypes>>,
> {
let block_id = block_number.into();
// retrieve full block and its receipts
let block = self.eth.block_by_number(block_id, true);
@ -236,7 +249,7 @@ where
}
// Crop receipts and transform them into OtsTransactionReceipt
let timestamp = Some(block.header.timestamp);
let timestamp = Some(block.header.timestamp());
let receipts = receipts
.drain(page_start..page_end)
.zip(transactions.iter().map(Transaction::ty))

View File

@ -1,4 +1,4 @@
use alloy_consensus::Header;
use alloy_consensus::BlockHeader as _;
use alloy_eips::BlockId;
use alloy_primitives::{map::HashSet, Bytes, B256, U256};
use alloy_rpc_types_eth::{
@ -19,13 +19,14 @@ use reth_consensus_common::calc::{
base_block_reward, base_block_reward_pre_merge, block_reward, ommer_reward,
};
use reth_evm::ConfigureEvmEnv;
use reth_primitives::PooledTransactionsElement;
use reth_primitives_traits::{BlockBody, BlockHeader};
use reth_provider::{BlockReader, ChainSpecProvider, EvmEnvProvider, StateProviderFactory};
use reth_revm::database::StateProviderDatabase;
use reth_rpc_api::TraceApiServer;
use reth_rpc_eth_api::{helpers::TraceExt, FromEthApiError};
use reth_rpc_eth_types::{error::EthApiError, utils::recover_raw_transaction};
use reth_tasks::pool::BlockingTaskGuard;
use reth_transaction_pool::{PoolPooledTx, PoolTransaction, TransactionPool};
use revm::{
db::{CacheDB, DatabaseCommit},
primitives::EnvWithHandlerCfg,
@ -116,8 +117,8 @@ where
trace_types: HashSet<TraceType>,
block_id: Option<BlockId>,
) -> Result<TraceResults, Eth::Error> {
let tx = recover_raw_transaction::<PooledTransactionsElement>(tx)?
.into_ecrecovered_transaction();
let tx = recover_raw_transaction::<PoolPooledTx<Eth::Pool>>(tx)?
.map_transaction(<Eth::Pool as TransactionPool>::Transaction::pooled_into_consensus);
let (cfg, block, at) = self.eth_api().evm_env_at(block_id.unwrap_or_default()).await?;
@ -313,11 +314,13 @@ where
// add reward traces for all blocks
for block in &blocks {
if let Some(base_block_reward) = self.calculate_base_block_reward(&block.header)? {
if let Some(base_block_reward) =
self.calculate_base_block_reward(block.header.header())?
{
all_traces.extend(
self.extract_reward_traces(
&block.header,
&block.body.ommers,
block.header.header(),
block.body.ommers(),
base_block_reward,
)
.into_iter()
@ -393,10 +396,12 @@ where
maybe_traces.map(|traces| traces.into_iter().flatten().collect::<Vec<_>>());
if let (Some(block), Some(traces)) = (maybe_block, maybe_traces.as_mut()) {
if let Some(base_block_reward) = self.calculate_base_block_reward(&block.header)? {
if let Some(base_block_reward) =
self.calculate_base_block_reward(block.header.header())?
{
traces.extend(self.extract_reward_traces(
&block.header,
&block.body.ommers,
block.block.header(),
block.body.ommers(),
base_block_reward,
));
}
@ -490,7 +495,7 @@ where
Ok(Some(BlockOpcodeGas {
block_hash: block.hash(),
block_number: block.header.number,
block_number: block.header.number(),
transactions,
}))
}
@ -500,25 +505,28 @@ where
/// - if Paris hardfork is activated, no block rewards are given
/// - if Paris hardfork is not activated, calculate block rewards with block number only
/// - if Paris hardfork is unknown, calculate block rewards with block number and ttd
fn calculate_base_block_reward(&self, header: &Header) -> Result<Option<u128>, Eth::Error> {
fn calculate_base_block_reward<H: BlockHeader>(
&self,
header: &H,
) -> Result<Option<u128>, Eth::Error> {
let chain_spec = self.provider().chain_spec();
let is_paris_activated = chain_spec.is_paris_active_at_block(header.number);
let is_paris_activated = chain_spec.is_paris_active_at_block(header.number());
Ok(match is_paris_activated {
Some(true) => None,
Some(false) => Some(base_block_reward_pre_merge(&chain_spec, header.number)),
Some(false) => Some(base_block_reward_pre_merge(&chain_spec, header.number())),
None => {
// if Paris hardfork is unknown, we need to fetch the total difficulty at the
// block's height and check if it is pre-merge to calculate the base block reward
if let Some(header_td) = self
.provider()
.header_td_by_number(header.number)
.header_td_by_number(header.number())
.map_err(Eth::Error::from_eth_err)?
{
base_block_reward(
chain_spec.as_ref(),
header.number,
header.difficulty,
header.number(),
header.difficulty(),
header_td,
)
} else {
@ -531,30 +539,33 @@ where
/// Extracts the reward traces for the given block:
/// - block reward
/// - uncle rewards
fn extract_reward_traces(
fn extract_reward_traces<H: BlockHeader>(
&self,
header: &Header,
ommers: &[Header],
header: &H,
ommers: Option<&[H]>,
base_block_reward: u128,
) -> Vec<LocalizedTransactionTrace> {
let mut traces = Vec::with_capacity(ommers.len() + 1);
let ommers_cnt = ommers.map(|o| o.len()).unwrap_or_default();
let mut traces = Vec::with_capacity(ommers_cnt + 1);
let block_reward = block_reward(base_block_reward, ommers.len());
let block_reward = block_reward(base_block_reward, ommers_cnt);
traces.push(reward_trace(
header,
RewardAction {
author: header.beneficiary,
author: header.beneficiary(),
reward_type: RewardType::Block,
value: U256::from(block_reward),
},
));
let Some(ommers) = ommers else { return traces };
for uncle in ommers {
let uncle_reward = ommer_reward(base_block_reward, header.number, uncle.number);
let uncle_reward = ommer_reward(base_block_reward, header.number(), uncle.number());
traces.push(reward_trace(
header,
RewardAction {
author: uncle.beneficiary,
author: uncle.beneficiary(),
reward_type: RewardType::Uncle,
value: U256::from(uncle_reward),
},
@ -715,10 +726,10 @@ struct TraceApiInner<Provider, Eth> {
/// Helper to construct a [`LocalizedTransactionTrace`] that describes a reward to the block
/// beneficiary.
fn reward_trace(header: &Header, reward: RewardAction) -> LocalizedTransactionTrace {
fn reward_trace<H: BlockHeader>(header: &H, reward: RewardAction) -> LocalizedTransactionTrace {
LocalizedTransactionTrace {
block_hash: Some(header.hash_slow()),
block_number: Some(header.number),
block_number: Some(header.number()),
transaction_hash: None,
transaction_position: None,
trace: TransactionTrace {