mirror of
https://github.com/paradigmxyz/reth.git
synced 2026-01-10 07:48:19 -05:00
feat(rpc): Replace associated type Transaction with Network in TransactionCompat (#16973)
This commit is contained in:
2
Cargo.lock
generated
2
Cargo.lock
generated
@@ -10082,6 +10082,7 @@ name = "reth-rpc-types-compat"
|
||||
version = "1.4.8"
|
||||
dependencies = [
|
||||
"alloy-consensus",
|
||||
"alloy-json-rpc",
|
||||
"alloy-network",
|
||||
"alloy-primitives",
|
||||
"alloy-rpc-types-eth",
|
||||
@@ -10094,7 +10095,6 @@ dependencies = [
|
||||
"reth-primitives-traits",
|
||||
"reth-storage-api",
|
||||
"revm-context",
|
||||
"serde",
|
||||
"thiserror 2.0.12",
|
||||
]
|
||||
|
||||
|
||||
@@ -13,7 +13,7 @@ use reth_primitives_traits::{RecoveredBlock, SealedHeader};
|
||||
use reth_rpc_eth_api::{
|
||||
helpers::{LoadPendingBlock, SpawnBlocking},
|
||||
types::RpcTypes,
|
||||
EthApiTypes, FromEthApiError, FromEvmError, RpcNodeCore,
|
||||
EthApiTypes, FromEthApiError, FromEvmError, RpcNodeCore, TransactionCompat,
|
||||
};
|
||||
use reth_rpc_eth_types::{EthApiError, PendingBlock};
|
||||
use reth_storage_api::{
|
||||
@@ -30,6 +30,7 @@ where
|
||||
Header = alloy_rpc_types_eth::Header<ProviderHeader<Self::Provider>>,
|
||||
>,
|
||||
Error: FromEvmError<Self::Evm>,
|
||||
TransactionCompat: TransactionCompat<Network = Self::NetworkTypes>,
|
||||
>,
|
||||
N: RpcNodeCore<
|
||||
Provider: BlockReaderIdExt
|
||||
|
||||
@@ -19,6 +19,7 @@ use reth_primitives_traits::{
|
||||
};
|
||||
use reth_revm::{database::StateProviderDatabase, db::State};
|
||||
use reth_rpc_eth_types::{EthApiError, PendingBlock, PendingBlockEnv, PendingBlockEnvOrigin};
|
||||
use reth_rpc_types_compat::TransactionCompat;
|
||||
use reth_storage_api::{
|
||||
BlockReader, BlockReaderIdExt, ProviderBlock, ProviderHeader, ProviderReceipt, ProviderTx,
|
||||
ReceiptProvider, StateProviderFactory,
|
||||
@@ -41,6 +42,7 @@ pub trait LoadPendingBlock:
|
||||
Header = alloy_rpc_types_eth::Header<ProviderHeader<Self::Provider>>,
|
||||
>,
|
||||
Error: FromEvmError<Self::Evm>,
|
||||
TransactionCompat: TransactionCompat<Network = Self::NetworkTypes>,
|
||||
> + RpcNodeCore<
|
||||
Provider: BlockReaderIdExt<Receipt: Receipt>
|
||||
+ ChainSpecProvider<ChainSpec: EthChainSpec + EthereumHardforks>
|
||||
|
||||
@@ -1,8 +1,6 @@
|
||||
//! Trait for specifying `eth` network dependent API types.
|
||||
|
||||
use crate::{AsEthApiError, FromEthApiError, RpcNodeCore};
|
||||
use alloy_json_rpc::RpcObject;
|
||||
use alloy_network::{Network, ReceiptResponse, TransactionResponse};
|
||||
use alloy_rpc_types_eth::Block;
|
||||
use reth_chain_state::CanonStateSubscriptions;
|
||||
use reth_rpc_types_compat::TransactionCompat;
|
||||
@@ -13,32 +11,13 @@ use std::{
|
||||
fmt::{self},
|
||||
};
|
||||
|
||||
/// RPC types used by the `eth_` RPC API.
|
||||
///
|
||||
/// This is a subset of [`alloy_network::Network`] trait with only RPC response types kept.
|
||||
pub trait RpcTypes {
|
||||
/// Header response type.
|
||||
type Header: RpcObject;
|
||||
/// Receipt response type.
|
||||
type Receipt: RpcObject + ReceiptResponse;
|
||||
/// Transaction response type.
|
||||
type Transaction: RpcObject + TransactionResponse;
|
||||
}
|
||||
|
||||
impl<T> RpcTypes for T
|
||||
where
|
||||
T: Network,
|
||||
{
|
||||
type Header = T::HeaderResponse;
|
||||
type Receipt = T::ReceiptResponse;
|
||||
type Transaction = T::TransactionResponse;
|
||||
}
|
||||
pub use reth_rpc_types_compat::{RpcTransaction, RpcTypes};
|
||||
|
||||
/// Network specific `eth` API types.
|
||||
///
|
||||
/// This trait defines the network specific rpc types and helpers required for the `eth_` and
|
||||
/// adjacent endpoints. `NetworkTypes` is [`Network`] as defined by the alloy crate, see also
|
||||
/// [`alloy_network::Ethereum`].
|
||||
/// adjacent endpoints. `NetworkTypes` is [`alloy_network::Network`] as defined by the alloy crate,
|
||||
/// see also [`alloy_network::Ethereum`].
|
||||
///
|
||||
/// This type is stateful so that it can provide additional context if necessary, e.g. populating
|
||||
/// receipts with additional data.
|
||||
@@ -59,9 +38,6 @@ pub trait EthApiTypes: Send + Sync + Clone {
|
||||
fn tx_resp_builder(&self) -> &Self::TransactionCompat;
|
||||
}
|
||||
|
||||
/// Adapter for network specific transaction type.
|
||||
pub type RpcTransaction<T> = <T as RpcTypes>::Transaction;
|
||||
|
||||
/// Adapter for network specific block type.
|
||||
pub type RpcBlock<T> = Block<RpcTransaction<T>, RpcHeader<T>>;
|
||||
|
||||
@@ -85,7 +61,7 @@ where
|
||||
> + EthApiTypes<
|
||||
TransactionCompat: TransactionCompat<
|
||||
Primitives = <Self as RpcNodeCore>::Primitives,
|
||||
Transaction = RpcTransaction<Self::NetworkTypes>,
|
||||
Network = Self::NetworkTypes,
|
||||
Error = RpcError<Self>,
|
||||
>,
|
||||
>,
|
||||
@@ -101,7 +77,7 @@ impl<T> FullEthApiTypes for T where
|
||||
> + EthApiTypes<
|
||||
TransactionCompat: TransactionCompat<
|
||||
Primitives = <Self as RpcNodeCore>::Primitives,
|
||||
Transaction = RpcTransaction<T::NetworkTypes>,
|
||||
Network = Self::NetworkTypes,
|
||||
Error = RpcError<T>,
|
||||
>,
|
||||
>
|
||||
|
||||
@@ -23,7 +23,7 @@ use reth_primitives_traits::{
|
||||
block::BlockTx, BlockBody as _, NodePrimitives, Recovered, RecoveredBlock, SignedTransaction,
|
||||
};
|
||||
use reth_rpc_server_types::result::rpc_err;
|
||||
use reth_rpc_types_compat::TransactionCompat;
|
||||
use reth_rpc_types_compat::{RpcTransaction, TransactionCompat};
|
||||
use reth_storage_api::noop::NoopProvider;
|
||||
use revm::{
|
||||
context_interface::result::ExecutionResult,
|
||||
@@ -191,7 +191,7 @@ pub fn build_simulated_block<T, B, Halt: Clone>(
|
||||
results: Vec<ExecutionResult<Halt>>,
|
||||
full_transactions: bool,
|
||||
tx_resp_builder: &T,
|
||||
) -> Result<SimulatedBlock<Block<T::Transaction, Header<B::Header>>>, T::Error>
|
||||
) -> Result<SimulatedBlock<Block<RpcTransaction<T::Network>, Header<B::Header>>>, T::Error>
|
||||
where
|
||||
T: TransactionCompat<
|
||||
Primitives: NodePrimitives<SignedTx = BlockTx<B>>,
|
||||
|
||||
@@ -6,7 +6,7 @@ use alloy_primitives::B256;
|
||||
use alloy_rpc_types_eth::TransactionInfo;
|
||||
use reth_ethereum_primitives::TransactionSigned;
|
||||
use reth_primitives_traits::{NodePrimitives, Recovered, SignedTransaction};
|
||||
use reth_rpc_types_compat::TransactionCompat;
|
||||
use reth_rpc_types_compat::{RpcTransaction, TransactionCompat};
|
||||
|
||||
/// Represents from where a transaction was fetched.
|
||||
#[derive(Debug, Clone, Eq, PartialEq)]
|
||||
@@ -42,7 +42,7 @@ impl<T: SignedTransaction> TransactionSource<T> {
|
||||
pub fn into_transaction<Builder>(
|
||||
self,
|
||||
resp_builder: &Builder,
|
||||
) -> Result<Builder::Transaction, Builder::Error>
|
||||
) -> Result<RpcTransaction<Builder::Network>, Builder::Error>
|
||||
where
|
||||
Builder: TransactionCompat<Primitives: NodePrimitives<SignedTx = T>>,
|
||||
{
|
||||
|
||||
@@ -22,6 +22,7 @@ alloy-primitives.workspace = true
|
||||
alloy-rpc-types-eth = { workspace = true, features = ["serde"] }
|
||||
alloy-consensus.workspace = true
|
||||
alloy-network.workspace = true
|
||||
alloy-json-rpc.workspace = true
|
||||
|
||||
# optimism
|
||||
op-alloy-consensus = { workspace = true, optional = true }
|
||||
@@ -33,7 +34,6 @@ op-revm = { workspace = true, optional = true }
|
||||
revm-context.workspace = true
|
||||
|
||||
# io
|
||||
serde.workspace = true
|
||||
jsonrpsee-types.workspace = true
|
||||
|
||||
# error
|
||||
|
||||
@@ -11,8 +11,11 @@
|
||||
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
|
||||
|
||||
mod fees;
|
||||
mod rpc;
|
||||
pub mod transaction;
|
||||
|
||||
pub use fees::{CallFees, CallFeesError};
|
||||
pub use rpc::*;
|
||||
pub use transaction::{
|
||||
EthTxEnvError, IntoRpcTx, RpcConverter, TransactionCompat, TransactionConversionError,
|
||||
TryIntoSimTx, TxInfoMapper,
|
||||
|
||||
26
crates/rpc/rpc-types-compat/src/rpc.rs
Normal file
26
crates/rpc/rpc-types-compat/src/rpc.rs
Normal file
@@ -0,0 +1,26 @@
|
||||
use alloy_json_rpc::RpcObject;
|
||||
use alloy_network::{Network, ReceiptResponse, TransactionResponse};
|
||||
|
||||
/// RPC types used by the `eth_` RPC API.
|
||||
///
|
||||
/// This is a subset of [`Network`] trait with only RPC response types kept.
|
||||
pub trait RpcTypes {
|
||||
/// Header response type.
|
||||
type Header: RpcObject;
|
||||
/// Receipt response type.
|
||||
type Receipt: RpcObject + ReceiptResponse;
|
||||
/// Transaction response type.
|
||||
type Transaction: RpcObject + TransactionResponse;
|
||||
}
|
||||
|
||||
impl<T> RpcTypes for T
|
||||
where
|
||||
T: Network,
|
||||
{
|
||||
type Header = T::HeaderResponse;
|
||||
type Receipt = T::ReceiptResponse;
|
||||
type Transaction = T::TransactionResponse;
|
||||
}
|
||||
|
||||
/// Adapter for network specific transaction type.
|
||||
pub type RpcTransaction<T> = <T as RpcTypes>::Transaction;
|
||||
@@ -1,8 +1,10 @@
|
||||
//! Compatibility functions for rpc `Transaction` type.
|
||||
|
||||
use crate::fees::{CallFees, CallFeesError};
|
||||
use crate::{
|
||||
fees::{CallFees, CallFeesError},
|
||||
RpcTransaction, RpcTypes,
|
||||
};
|
||||
use alloy_consensus::{error::ValueError, transaction::Recovered, EthereumTxEnvelope, TxEip4844};
|
||||
use alloy_network::Network;
|
||||
use alloy_primitives::{Address, TxKind, U256};
|
||||
use alloy_rpc_types_eth::{
|
||||
request::{TransactionInputError, TransactionRequest},
|
||||
@@ -15,17 +17,17 @@ use reth_evm::{
|
||||
};
|
||||
use reth_primitives_traits::{NodePrimitives, TxTy};
|
||||
use revm_context::{BlockEnv, CfgEnv, TxEnv};
|
||||
use serde::{Deserialize, Serialize};
|
||||
use std::{convert::Infallible, error::Error, fmt::Debug, marker::PhantomData};
|
||||
use thiserror::Error;
|
||||
|
||||
/// Builds RPC transaction w.r.t. network.
|
||||
pub trait TransactionCompat: Send + Sync + Unpin + Clone + Debug {
|
||||
/// The lower layer consensus types to convert from.
|
||||
/// Associated lower layer consensus types to convert from and into types of [`Self::Network`].
|
||||
type Primitives: NodePrimitives;
|
||||
|
||||
/// RPC transaction response type.
|
||||
type Transaction: Serialize + for<'de> Deserialize<'de> + Send + Sync + Unpin + Clone + Debug;
|
||||
/// Associated upper layer JSON-RPC API network requests and responses to convert from and into
|
||||
/// types of [`Self::Primitives`].
|
||||
type Network: RpcTypes + Send + Sync + Unpin + Clone + Debug;
|
||||
|
||||
/// A set of variables for executing a transaction.
|
||||
type TxEnv;
|
||||
@@ -39,7 +41,7 @@ pub trait TransactionCompat: Send + Sync + Unpin + Clone + Debug {
|
||||
fn fill_pending(
|
||||
&self,
|
||||
tx: Recovered<TxTy<Self::Primitives>>,
|
||||
) -> Result<Self::Transaction, Self::Error> {
|
||||
) -> Result<RpcTransaction<Self::Network>, Self::Error> {
|
||||
self.fill(tx, TransactionInfo::default())
|
||||
}
|
||||
|
||||
@@ -52,7 +54,7 @@ pub trait TransactionCompat: Send + Sync + Unpin + Clone + Debug {
|
||||
&self,
|
||||
tx: Recovered<TxTy<Self::Primitives>>,
|
||||
tx_inf: TransactionInfo,
|
||||
) -> Result<Self::Transaction, Self::Error>;
|
||||
) -> Result<RpcTransaction<Self::Network>, Self::Error>;
|
||||
|
||||
/// Builds a fake transaction from a transaction request for inclusion into block built in
|
||||
/// `eth_simulateV1`.
|
||||
@@ -353,9 +355,9 @@ impl<N, E, Evm, Err> Default for RpcConverter<N, E, Evm, Err> {
|
||||
impl<N, E, Evm, Err, Map> TransactionCompat for RpcConverter<N, E, Evm, Err, Map>
|
||||
where
|
||||
N: NodePrimitives,
|
||||
E: Network + Unpin,
|
||||
E: RpcTypes + Send + Sync + Unpin + Clone + Debug,
|
||||
Evm: ConfigureEvm,
|
||||
TxTy<N>: IntoRpcTx<<E as Network>::TransactionResponse> + Clone + Debug,
|
||||
TxTy<N>: IntoRpcTx<E::Transaction> + Clone + Debug,
|
||||
TransactionRequest: TryIntoSimTx<TxTy<N>> + TryIntoTxEnv<TxEnvFor<Evm>>,
|
||||
Err: From<TransactionConversionError>
|
||||
+ From<<TransactionRequest as TryIntoTxEnv<TxEnvFor<Evm>>>::Err>
|
||||
@@ -365,17 +367,15 @@ where
|
||||
+ Sync
|
||||
+ Send
|
||||
+ Into<jsonrpsee_types::ErrorObject<'static>>,
|
||||
Map: for<'a> TxInfoMapper<
|
||||
&'a TxTy<N>,
|
||||
Out = <TxTy<N> as IntoRpcTx<<E as Network>::TransactionResponse>>::TxInfo,
|
||||
> + Clone
|
||||
Map: for<'a> TxInfoMapper<&'a TxTy<N>, Out = <TxTy<N> as IntoRpcTx<E::Transaction>>::TxInfo>
|
||||
+ Clone
|
||||
+ Debug
|
||||
+ Unpin
|
||||
+ Send
|
||||
+ Sync,
|
||||
{
|
||||
type Primitives = N;
|
||||
type Transaction = <E as Network>::TransactionResponse;
|
||||
type Network = E;
|
||||
type TxEnv = TxEnvFor<Evm>;
|
||||
type Error = Err;
|
||||
|
||||
@@ -383,7 +383,7 @@ where
|
||||
&self,
|
||||
tx: Recovered<TxTy<N>>,
|
||||
tx_info: TransactionInfo,
|
||||
) -> Result<Self::Transaction, Self::Error> {
|
||||
) -> Result<E::Transaction, Self::Error> {
|
||||
let (tx, signer) = tx.into_parts();
|
||||
let tx_info = self.mapper.try_map(&tx, tx_info)?;
|
||||
|
||||
|
||||
@@ -415,7 +415,9 @@ struct EthFilterInner<Eth: EthApiTypes> {
|
||||
|
||||
impl<Eth> EthFilterInner<Eth>
|
||||
where
|
||||
Eth: RpcNodeCoreExt<Provider: BlockIdReader, Pool: TransactionPool> + EthApiTypes + 'static,
|
||||
Eth: RpcNodeCoreExt<Provider: BlockIdReader, Pool: TransactionPool>
|
||||
+ EthApiTypes<NetworkTypes: reth_rpc_eth_api::types::RpcTypes>
|
||||
+ 'static,
|
||||
{
|
||||
/// Access the underlying provider.
|
||||
fn provider(&self) -> &Eth::Provider {
|
||||
@@ -692,7 +694,7 @@ where
|
||||
}
|
||||
|
||||
/// Returns all new pending transactions received since the last poll.
|
||||
async fn drain(&self) -> FilterChanges<TxCompat::Transaction> {
|
||||
async fn drain(&self) -> FilterChanges<RpcTransaction<TxCompat::Network>> {
|
||||
let mut pending_txs = Vec::new();
|
||||
let mut prepared_stream = self.txs_stream.lock().await;
|
||||
|
||||
@@ -718,13 +720,13 @@ trait FullTransactionsFilter<T>: fmt::Debug + Send + Sync + Unpin + 'static {
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl<T, TxCompat> FullTransactionsFilter<TxCompat::Transaction>
|
||||
impl<T, TxCompat> FullTransactionsFilter<RpcTransaction<TxCompat::Network>>
|
||||
for FullTransactionsReceiver<T, TxCompat>
|
||||
where
|
||||
T: PoolTransaction + 'static,
|
||||
TxCompat: TransactionCompat<Primitives: NodePrimitives<SignedTx = T::Consensus>> + 'static,
|
||||
{
|
||||
async fn drain(&self) -> FilterChanges<TxCompat::Transaction> {
|
||||
async fn drain(&self) -> FilterChanges<RpcTransaction<TxCompat::Network>> {
|
||||
Self::drain(self).await
|
||||
}
|
||||
}
|
||||
|
||||
@@ -11,6 +11,7 @@ use reth_rpc_eth_api::{
|
||||
RpcNodeCore, RpcNodeCoreExt, RpcReceipt,
|
||||
};
|
||||
use reth_rpc_eth_types::{EthApiError, EthReceiptBuilder};
|
||||
use reth_rpc_types_compat::TransactionCompat;
|
||||
use reth_storage_api::{BlockReader, ProviderTx};
|
||||
use reth_transaction_pool::{PoolTransaction, TransactionPool};
|
||||
|
||||
@@ -21,6 +22,7 @@ where
|
||||
Self: LoadBlock<
|
||||
Error = EthApiError,
|
||||
NetworkTypes: RpcTypes<Receipt = TransactionReceipt>,
|
||||
TransactionCompat: TransactionCompat<Network = Self::NetworkTypes>,
|
||||
Provider: BlockReader<
|
||||
Transaction = reth_ethereum_primitives::TransactionSigned,
|
||||
Receipt = reth_ethereum_primitives::Receipt,
|
||||
|
||||
@@ -1,5 +1,6 @@
|
||||
//! Support for building a pending block with transactions from local view of mempool.
|
||||
|
||||
use crate::EthApi;
|
||||
use alloy_consensus::BlockHeader;
|
||||
use reth_chainspec::{ChainSpecProvider, EthChainSpec, EthereumHardforks};
|
||||
use reth_evm::{ConfigureEvm, NextBlockEnvAttributes};
|
||||
@@ -11,6 +12,7 @@ use reth_rpc_eth_api::{
|
||||
FromEvmError, RpcNodeCore,
|
||||
};
|
||||
use reth_rpc_eth_types::PendingBlock;
|
||||
use reth_rpc_types_compat::TransactionCompat;
|
||||
use reth_storage_api::{
|
||||
BlockReader, BlockReaderIdExt, ProviderBlock, ProviderHeader, ProviderReceipt, ProviderTx,
|
||||
StateProviderFactory,
|
||||
@@ -18,8 +20,6 @@ use reth_storage_api::{
|
||||
use reth_transaction_pool::{PoolTransaction, TransactionPool};
|
||||
use revm_primitives::B256;
|
||||
|
||||
use crate::EthApi;
|
||||
|
||||
impl<Provider, Pool, Network, EvmConfig> LoadPendingBlock
|
||||
for EthApi<Provider, Pool, Network, EvmConfig>
|
||||
where
|
||||
@@ -28,6 +28,7 @@ where
|
||||
Header = alloy_rpc_types_eth::Header<ProviderHeader<Self::Provider>>,
|
||||
>,
|
||||
Error: FromEvmError<Self::Evm>,
|
||||
TransactionCompat: TransactionCompat<Network = Self::NetworkTypes>,
|
||||
> + RpcNodeCore<
|
||||
Provider: BlockReaderIdExt<Receipt = Provider::Receipt, Block = Provider::Block>
|
||||
+ ChainSpecProvider<ChainSpec: EthChainSpec + EthereumHardforks>
|
||||
|
||||
@@ -10,7 +10,8 @@ use async_trait::async_trait;
|
||||
use jsonrpsee::core::RpcResult;
|
||||
use reth_primitives_traits::NodePrimitives;
|
||||
use reth_rpc_api::TxPoolApiServer;
|
||||
use reth_rpc_types_compat::TransactionCompat;
|
||||
use reth_rpc_eth_api::RpcTransaction;
|
||||
use reth_rpc_types_compat::{RpcTypes, TransactionCompat};
|
||||
use reth_transaction_pool::{
|
||||
AllPoolTransactions, PoolConsensusTx, PoolTransaction, TransactionPool,
|
||||
};
|
||||
@@ -38,11 +39,14 @@ where
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus: Transaction>> + 'static,
|
||||
Eth: TransactionCompat<Primitives: NodePrimitives<SignedTx = PoolConsensusTx<Pool>>>,
|
||||
{
|
||||
fn content(&self) -> Result<TxpoolContent<Eth::Transaction>, Eth::Error> {
|
||||
fn content(&self) -> Result<TxpoolContent<RpcTransaction<Eth::Network>>, Eth::Error> {
|
||||
#[inline]
|
||||
fn insert<Tx, RpcTxB>(
|
||||
tx: &Tx,
|
||||
content: &mut BTreeMap<Address, BTreeMap<String, RpcTxB::Transaction>>,
|
||||
content: &mut BTreeMap<
|
||||
Address,
|
||||
BTreeMap<String, <RpcTxB::Network as RpcTypes>::Transaction>,
|
||||
>,
|
||||
resp_builder: &RpcTxB,
|
||||
) -> Result<(), RpcTxB::Error>
|
||||
where
|
||||
@@ -72,7 +76,7 @@ where
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl<Pool, Eth> TxPoolApiServer<Eth::Transaction> for TxPoolApi<Pool, Eth>
|
||||
impl<Pool, Eth> TxPoolApiServer<RpcTransaction<Eth::Network>> for TxPoolApi<Pool, Eth>
|
||||
where
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus: Transaction>> + 'static,
|
||||
Eth: TransactionCompat<Primitives: NodePrimitives<SignedTx = PoolConsensusTx<Pool>>> + 'static,
|
||||
@@ -129,7 +133,7 @@ where
|
||||
async fn txpool_content_from(
|
||||
&self,
|
||||
from: Address,
|
||||
) -> RpcResult<TxpoolContentFrom<Eth::Transaction>> {
|
||||
) -> RpcResult<TxpoolContentFrom<RpcTransaction<Eth::Network>>> {
|
||||
trace!(target: "rpc::eth", ?from, "Serving txpool_contentFrom");
|
||||
Ok(self.content().map_err(Into::into)?.remove_from(&from))
|
||||
}
|
||||
@@ -139,7 +143,7 @@ where
|
||||
///
|
||||
/// See [here](https://geth.ethereum.org/docs/rpc/ns-txpool#txpool_content) for more details
|
||||
/// Handler for `txpool_content`
|
||||
async fn txpool_content(&self) -> RpcResult<TxpoolContent<Eth::Transaction>> {
|
||||
async fn txpool_content(&self) -> RpcResult<TxpoolContent<RpcTransaction<Eth::Network>>> {
|
||||
trace!(target: "rpc::eth", "Serving txpool_content");
|
||||
Ok(self.content().map_err(Into::into)?)
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user