From a24fe46b133a784842ef789946f4e0c15bc63ff9 Mon Sep 17 00:00:00 2001 From: nk_ysg Date: Tue, 27 Aug 2024 02:17:34 +0800 Subject: [PATCH] remove transaction forwarder trait (#9678) --- Cargo.lock | 5 +- bin/reth/Cargo.toml | 1 + bin/reth/src/optimism.rs | 18 +++---- crates/optimism/node/src/lib.rs | 2 - crates/optimism/rpc/Cargo.toml | 4 +- crates/optimism/rpc/src/eth/mod.rs | 22 ++++++-- .../optimism/{node/src => rpc/src/eth}/rpc.rs | 11 +--- crates/optimism/rpc/src/eth/transaction.rs | 54 ++++++++++++++++--- crates/rpc/rpc-builder/src/lib.rs | 21 +------- crates/rpc/rpc-eth-api/src/core.rs | 9 +--- crates/rpc/rpc-eth-api/src/helpers/mod.rs | 2 +- .../rpc-eth-api/src/helpers/transaction.rs | 46 +--------------- crates/rpc/rpc-eth-api/src/lib.rs | 5 +- crates/rpc/rpc/src/eth/core.rs | 27 ++-------- crates/rpc/rpc/src/eth/helpers/state.rs | 2 - crates/rpc/rpc/src/eth/helpers/transaction.rs | 13 +---- crates/rpc/rpc/src/eth/mod.rs | 2 +- 17 files changed, 92 insertions(+), 152 deletions(-) rename crates/optimism/{node/src => rpc/src/eth}/rpc.rs (91%) diff --git a/Cargo.lock b/Cargo.lock index 224186f3114a..e4f48ea83542 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6297,6 +6297,7 @@ dependencies = [ "reth-node-optimism", "reth-optimism-cli", "reth-optimism-primitives", + "reth-optimism-rpc", "reth-payload-builder", "reth-payload-primitives", "reth-payload-validator", @@ -8066,10 +8067,10 @@ name = "reth-optimism-rpc" version = "1.0.5" dependencies = [ "alloy-primitives", - "derive_more 1.0.0", "jsonrpsee-types", "op-alloy-network", "parking_lot 0.12.3", + "reqwest", "reth-chainspec", "reth-evm", "reth-evm-optimism", @@ -8086,8 +8087,10 @@ dependencies = [ "reth-tasks", "reth-transaction-pool", "revm", + "serde_json", "thiserror", "tokio", + "tracing", ] [[package]] diff --git a/bin/reth/Cargo.toml b/bin/reth/Cargo.toml index 2c164530fdaa..49ea27b8f2fc 100644 --- a/bin/reth/Cargo.toml +++ b/bin/reth/Cargo.toml @@ -72,6 +72,7 @@ reth-engine-util.workspace = true reth-prune.workspace = true reth-stages-api.workspace = true reth-optimism-cli = { workspace = true, optional = true } +reth-optimism-rpc.workspace = true # crypto alloy-rlp.workspace = true diff --git a/bin/reth/src/optimism.rs b/bin/reth/src/optimism.rs index 312c2998d61d..94a533822dce 100644 --- a/bin/reth/src/optimism.rs +++ b/bin/reth/src/optimism.rs @@ -3,11 +3,9 @@ use clap::Parser; use reth::cli::Cli; use reth_node_builder::EngineNodeLauncher; -use reth_node_optimism::{ - args::RollupArgs, node::OptimismAddOns, rpc::SequencerClient, OptimismNode, -}; +use reth_node_optimism::{args::RollupArgs, node::OptimismAddOns, OptimismNode}; +use reth_optimism_rpc::eth::rpc::SequencerClient; use reth_provider::providers::BlockchainProvider2; -use std::sync::Arc; // We use jemalloc for performance reasons #[cfg(all(feature = "jemalloc", unix))] @@ -38,9 +36,9 @@ fn main() { .extend_rpc_modules(move |ctx| { // register sequencer tx forwarder if let Some(sequencer_http) = sequencer_http_arg { - ctx.registry.set_eth_raw_transaction_forwarder(Arc::new( - SequencerClient::new(sequencer_http), - )); + ctx.registry + .eth_api() + .set_sequencer_client(SequencerClient::new(sequencer_http)); } Ok(()) @@ -62,9 +60,9 @@ fn main() { .extend_rpc_modules(move |ctx| { // register sequencer tx forwarder if let Some(sequencer_http) = sequencer_http_arg { - ctx.registry.set_eth_raw_transaction_forwarder(Arc::new( - SequencerClient::new(sequencer_http), - )); + ctx.registry + .eth_api() + .set_sequencer_client(SequencerClient::new(sequencer_http)); } Ok(()) diff --git a/crates/optimism/node/src/lib.rs b/crates/optimism/node/src/lib.rs index 68aebd0835fb..9d7d6f3554c8 100644 --- a/crates/optimism/node/src/lib.rs +++ b/crates/optimism/node/src/lib.rs @@ -22,8 +22,6 @@ pub use node::OptimismNode; pub mod txpool; -pub mod rpc; - pub use reth_optimism_payload_builder::{ OptimismBuiltPayload, OptimismPayloadBuilder, OptimismPayloadBuilderAttributes, }; diff --git a/crates/optimism/rpc/Cargo.toml b/crates/optimism/rpc/Cargo.toml index 7f498a7e4b77..8b1f64330072 100644 --- a/crates/optimism/rpc/Cargo.toml +++ b/crates/optimism/rpc/Cargo.toml @@ -37,13 +37,15 @@ revm.workspace = true # async parking_lot.workspace = true tokio.workspace = true +reqwest = { workspace = true, features = ["rustls-tls-native-roots"] } # rpc jsonrpsee-types.workspace = true +serde_json.workspace = true # misc thiserror.workspace = true -derive_more.workspace = true +tracing.workspace = true [features] optimism = [ diff --git a/crates/optimism/rpc/src/eth/mod.rs b/crates/optimism/rpc/src/eth/mod.rs index 9fd6d0e17662..40abdfc83998 100644 --- a/crates/optimism/rpc/src/eth/mod.rs +++ b/crates/optimism/rpc/src/eth/mod.rs @@ -6,11 +6,12 @@ pub mod transaction; mod block; mod call; mod pending_block; +pub mod rpc; use std::{fmt, sync::Arc}; +use crate::eth::rpc::SequencerClient; use alloy_primitives::U256; -use derive_more::Deref; use op_alloy_network::Optimism; use reth_chainspec::ChainSpec; use reth_evm::ConfigureEvm; @@ -56,10 +57,9 @@ pub type EthApiNodeBackend = EthApiInner< /// /// This type implements the [`FullEthApi`](reth_rpc_eth_api::helpers::FullEthApi) by implemented /// all the `Eth` helper traits and prerequisite traits. -#[derive(Clone, Deref)] pub struct OpEthApi { - #[deref] inner: Arc>, + sequencer_client: parking_lot::RwLock>, } impl OpEthApi { @@ -81,11 +81,23 @@ impl OpEthApi { ctx.new_fee_history_cache(), ctx.evm_config.clone(), ctx.executor.clone(), - None, ctx.config.proof_permits, ); - Self { inner: Arc::new(inner) } + Self { inner: Arc::new(inner), sequencer_client: parking_lot::RwLock::new(None) } + } +} + +impl Clone for OpEthApi +where + N: FullNodeComponents, + Self: Send + Sync, +{ + fn clone(&self) -> Self { + Self { + inner: self.inner.clone(), + sequencer_client: parking_lot::RwLock::new(self.sequencer_client.read().clone()), + } } } diff --git a/crates/optimism/node/src/rpc.rs b/crates/optimism/rpc/src/eth/rpc.rs similarity index 91% rename from crates/optimism/node/src/rpc.rs rename to crates/optimism/rpc/src/eth/rpc.rs index 63e28300b52d..02f227cbe9ae 100644 --- a/crates/optimism/node/src/rpc.rs +++ b/crates/optimism/rpc/src/eth/rpc.rs @@ -4,8 +4,7 @@ use std::sync::{atomic::AtomicUsize, Arc}; use jsonrpsee_types::error::{ErrorObject, INTERNAL_ERROR_CODE}; use reqwest::Client; -use reth_rpc_eth_api::RawTransactionForwarder; -use reth_rpc_eth_types::error::{EthApiError, EthResult}; +use reth_rpc_eth_types::error::EthApiError; use reth_rpc_types::ToRpcError; /// Error type when interacting with the Sequencer @@ -104,14 +103,6 @@ impl SequencerClient { } } -#[async_trait::async_trait] -impl RawTransactionForwarder for SequencerClient { - async fn forward_raw_transaction(&self, tx: &[u8]) -> EthResult<()> { - Self::forward_raw_transaction(self, tx).await?; - Ok(()) - } -} - #[derive(Debug, Default)] struct SequencerClientInner { /// The endpoint of the sequencer diff --git a/crates/optimism/rpc/src/eth/transaction.rs b/crates/optimism/rpc/src/eth/transaction.rs index 705f24eafca4..40da80c96a35 100644 --- a/crates/optimism/rpc/src/eth/transaction.rs +++ b/crates/optimism/rpc/src/eth/transaction.rs @@ -1,6 +1,6 @@ //! Loads and formats OP transaction RPC response. -use std::sync::Arc; +use alloy_primitives::{Bytes, B256}; use reth_evm_optimism::RethL1BlockInfo; use reth_node_api::FullNodeComponents; @@ -8,12 +8,13 @@ use reth_primitives::TransactionSigned; use reth_provider::{BlockReaderIdExt, TransactionsProvider}; use reth_rpc_eth_api::{ helpers::{EthApiSpec, EthSigner, EthTransactions, LoadTransaction, SpawnBlocking}, - EthApiTypes, RawTransactionForwarder, + EthApiTypes, FromEthApiError, }; -use reth_rpc_eth_types::EthStateCache; +use reth_rpc_eth_types::{utils::recover_raw_transaction, EthStateCache}; +use reth_transaction_pool::{PoolTransaction, TransactionOrigin, TransactionPool}; use revm::L1BlockInfo; -use crate::{OpEthApi, OpEthApiError}; +use crate::{eth::rpc::SequencerClient, OpEthApi, OpEthApiError}; impl EthTransactions for OpEthApi where @@ -24,13 +25,35 @@ where self.inner.provider() } - fn raw_tx_forwarder(&self) -> Option> { - self.inner.raw_tx_forwarder() - } - fn signers(&self) -> &parking_lot::RwLock>> { self.inner.signers() } + + /// Decodes and recovers the transaction and submits it to the pool. + /// + /// Returns the hash of the transaction. + async fn send_raw_transaction(&self, tx: Bytes) -> Result { + let recovered = recover_raw_transaction(tx.clone())?; + let pool_transaction = ::Transaction::from_pooled(recovered); + + // On optimism, transactions are forwarded directly to the sequencer to be included in + // blocks that it builds. + if let Some(client) = self.raw_tx_forwarder().as_ref() { + tracing::debug!( target: "rpc::eth", "forwarding raw transaction to"); + let _ = client.forward_raw_transaction(&tx).await.inspect_err(|err| { + tracing::debug!(target: "rpc::eth", %err, hash=% *pool_transaction.hash(), "failed to forward raw transaction"); + }); + } + + // submit the transaction to the pool with a `Local` origin + let hash = self + .pool() + .add_transaction(TransactionOrigin::Local, pool_transaction) + .await + .map_err(Self::Error::from_eth_err)?; + + Ok(hash) + } } impl LoadTransaction for OpEthApi @@ -113,3 +136,18 @@ where Ok(OptimismTxMeta::new(Some(l1_block_info), l1_fee, l1_data_gas)) } } + +impl OpEthApi +where + N: FullNodeComponents, +{ + /// Sets a `SequencerClient` for `eth_sendRawTransaction` to forward transactions to. + pub fn set_sequencer_client(&self, sequencer_client: SequencerClient) { + *self.sequencer_client.write() = Some(sequencer_client); + } + + /// Returns the `SequencerClient` if one is set. + pub fn raw_tx_forwarder(&self) -> Option { + self.sequencer_client.read().clone() + } +} diff --git a/crates/rpc/rpc-builder/src/lib.rs b/crates/rpc/rpc-builder/src/lib.rs index e1712ec091af..0d271c4b07de 100644 --- a/crates/rpc/rpc-builder/src/lib.rs +++ b/crates/rpc/rpc-builder/src/lib.rs @@ -136,7 +136,6 @@ use std::{ collections::HashMap, net::{Ipv4Addr, SocketAddr, SocketAddrV4}, - sync::Arc, time::{Duration, SystemTime, UNIX_EPOCH}, }; @@ -165,10 +164,8 @@ use reth_rpc::{ }; use reth_rpc_api::servers::*; use reth_rpc_eth_api::{ - helpers::{ - Call, EthApiSpec, EthTransactions, LoadPendingBlock, TraceExt, UpdateRawTxForwarder, - }, - EthApiServer, FullEthApiServer, RawTransactionForwarder, + helpers::{Call, EthApiSpec, EthTransactions, LoadPendingBlock, TraceExt}, + EthApiServer, FullEthApiServer, }; use reth_rpc_eth_types::{EthConfig, EthStateCache, EthSubscriptionIdProvider}; use reth_rpc_layer::{AuthLayer, Claims, JwtAuthValidator, JwtSecret}; @@ -741,20 +738,6 @@ impl } } -impl - RpcRegistryInner -where - EthApi: UpdateRawTxForwarder, -{ - /// Sets a forwarder for `eth_sendRawTransaction` - /// - /// Note: this might be removed in the future in favor of a more generic approach. - pub fn set_eth_raw_transaction_forwarder(&self, forwarder: Arc) { - // in case the eth api has been created before the forwarder was set: - self.eth.api.set_eth_raw_transaction_forwarder(forwarder.clone()); - } -} - impl RpcRegistryInner where diff --git a/crates/rpc/rpc-eth-api/src/core.rs b/crates/rpc/rpc-eth-api/src/core.rs index a18bd579598d..409dc0076791 100644 --- a/crates/rpc/rpc-eth-api/src/core.rs +++ b/crates/rpc/rpc-eth-api/src/core.rs @@ -21,8 +21,7 @@ use tracing::trace; use crate::{ helpers::{ - transaction::UpdateRawTxForwarder, EthApiSpec, EthBlocks, EthCall, EthFees, EthState, - EthTransactions, FullEthApi, LoadState, + EthApiSpec, EthBlocks, EthCall, EthFees, EthState, EthTransactions, FullEthApi, LoadState, }, RpcBlock, RpcTransaction, }; @@ -30,17 +29,13 @@ use crate::{ /// Helper trait, unifies functionality that must be supported to implement all RPC methods for /// server. pub trait FullEthApiServer: - EthApiServer, RpcBlock> - + FullEthApi - + UpdateRawTxForwarder - + Clone + EthApiServer, RpcBlock> + FullEthApi + Clone { } impl FullEthApiServer for T where T: EthApiServer, RpcBlock> + FullEthApi - + UpdateRawTxForwarder + Clone { } diff --git a/crates/rpc/rpc-eth-api/src/helpers/mod.rs b/crates/rpc/rpc-eth-api/src/helpers/mod.rs index 4f12b7dac861..73a64078da54 100644 --- a/crates/rpc/rpc-eth-api/src/helpers/mod.rs +++ b/crates/rpc/rpc-eth-api/src/helpers/mod.rs @@ -37,7 +37,7 @@ pub use signer::{AddDevSigners, EthSigner}; pub use spec::EthApiSpec; pub use state::{EthState, LoadState}; pub use trace::Trace; -pub use transaction::{EthTransactions, LoadTransaction, UpdateRawTxForwarder}; +pub use transaction::{EthTransactions, LoadTransaction}; use crate::EthApiTypes; diff --git a/crates/rpc/rpc-eth-api/src/helpers/transaction.rs b/crates/rpc/rpc-eth-api/src/helpers/transaction.rs index 6d5338b07350..cf7bbf025b05 100644 --- a/crates/rpc/rpc-eth-api/src/helpers/transaction.rs +++ b/crates/rpc/rpc-eth-api/src/helpers/transaction.rs @@ -1,8 +1,6 @@ //! Database access for `eth_` transaction RPC methods. Loads transaction and receipt data w.r.t. //! network. -use std::{fmt, ops::Deref, sync::Arc}; - use alloy_dyn_abi::TypedData; use futures::Future; use reth_primitives::{ @@ -11,8 +9,7 @@ use reth_primitives::{ }; use reth_provider::{BlockReaderIdExt, ReceiptProvider, TransactionsProvider}; use reth_rpc_eth_types::{ - utils::recover_raw_transaction, EthApiError, EthResult, EthStateCache, SignError, - TransactionSource, + utils::recover_raw_transaction, EthApiError, EthStateCache, SignError, TransactionSource, }; use reth_rpc_types::{ transaction::{ @@ -59,11 +56,6 @@ pub trait EthTransactions: LoadTransaction { /// Data access in default (L1) trait method implementations. fn provider(&self) -> impl BlockReaderIdExt; - /// Returns a handle for forwarding received raw transactions. - /// - /// Access to transaction forwarder in default (L1) trait method implementations. - fn raw_tx_forwarder(&self) -> Option>; - /// Returns a handle for signing data. /// /// Singer access in default (L1) trait method implementations. @@ -254,15 +246,6 @@ pub trait EthTransactions: LoadTransaction { let pool_transaction = ::Transaction::from_pooled(recovered); - // On optimism, transactions are forwarded directly to the sequencer to be included in - // blocks that it builds. - if let Some(client) = self.raw_tx_forwarder().as_ref() { - tracing::debug!( target: "rpc::eth", "forwarding raw transaction to"); - let _ = client.forward_raw_transaction(&tx).await.inspect_err(|err| { - tracing::debug!(target: "rpc::eth", %err, hash=% *pool_transaction.hash(), "failed to forward raw transaction"); - }); - } - // submit the transaction to the pool with a `Local` origin let hash = self .pool() @@ -660,30 +643,3 @@ pub trait LoadTransaction: SpawnBlocking { } } } - -/// A trait that allows for forwarding raw transactions. -/// -/// For example to a sequencer. -#[async_trait::async_trait] -pub trait RawTransactionForwarder: fmt::Debug + Send + Sync + 'static { - /// Forwards raw transaction bytes for `eth_sendRawTransaction` - async fn forward_raw_transaction(&self, raw: &[u8]) -> EthResult<()>; -} - -/// Configure server's forwarder for `eth_sendRawTransaction`, at runtime. -pub trait UpdateRawTxForwarder { - /// Sets a forwarder for `eth_sendRawTransaction` - /// - /// Note: this might be removed in the future in favor of a more generic approach. - fn set_eth_raw_transaction_forwarder(&self, forwarder: Arc); -} - -impl UpdateRawTxForwarder for T -where - T: Deref>, - K: UpdateRawTxForwarder, -{ - fn set_eth_raw_transaction_forwarder(&self, forwarder: Arc) { - self.deref().deref().set_eth_raw_transaction_forwarder(forwarder); - } -} diff --git a/crates/rpc/rpc-eth-api/src/lib.rs b/crates/rpc/rpc-eth-api/src/lib.rs index c3ea0746a119..30623d6b8e2f 100644 --- a/crates/rpc/rpc-eth-api/src/lib.rs +++ b/crates/rpc/rpc-eth-api/src/lib.rs @@ -22,10 +22,7 @@ pub mod types; pub use bundle::{EthBundleApiServer, EthCallBundleApiServer}; pub use core::{EthApiServer, FullEthApiServer}; pub use filter::EthFilterApiServer; -pub use helpers::{ - error::{AsEthApiError, FromEthApiError, FromEvmError, IntoEthApiError}, - transaction::RawTransactionForwarder, -}; +pub use helpers::error::{AsEthApiError, FromEthApiError, FromEvmError, IntoEthApiError}; pub use pubsub::EthPubSubApiServer; pub use types::{EthApiTypes, RpcBlock, RpcTransaction}; diff --git a/crates/rpc/rpc/src/eth/core.rs b/crates/rpc/rpc/src/eth/core.rs index 774f1e71baa6..6d3cd1301b11 100644 --- a/crates/rpc/rpc/src/eth/core.rs +++ b/crates/rpc/rpc/src/eth/core.rs @@ -9,8 +9,8 @@ use reth_node_api::{BuilderProvider, FullNodeComponents}; use reth_primitives::{BlockNumberOrTag, U256}; use reth_provider::{BlockReaderIdExt, CanonStateSubscriptions, ChainSpecProvider}; use reth_rpc_eth_api::{ - helpers::{transaction::UpdateRawTxForwarder, EthSigner, SpawnBlocking}, - EthApiTypes, RawTransactionForwarder, + helpers::{EthSigner, SpawnBlocking}, + EthApiTypes, }; use reth_rpc_eth_types::{ EthApiBuilderCtx, EthApiError, EthStateCache, FeeHistoryCache, GasCap, GasPriceOracle, @@ -60,7 +60,6 @@ where blocking_task_pool: BlockingTaskPool, fee_history_cache: FeeHistoryCache, evm_config: EvmConfig, - raw_transaction_forwarder: Option>, proof_permits: usize, ) -> Self { let inner = EthApiInner::new( @@ -75,7 +74,6 @@ where fee_history_cache, evm_config, TokioTaskExecutor::default(), - raw_transaction_forwarder, proof_permits, ); @@ -113,7 +111,6 @@ where ctx.new_fee_history_cache(), ctx.evm_config.clone(), ctx.executor.clone(), - None, ctx.config.proof_permits, ); @@ -202,8 +199,7 @@ pub struct EthApiInner { fee_history_cache: FeeHistoryCache, /// The type that defines how to configure the EVM evm_config: EvmConfig, - /// Allows forwarding received raw transactions - raw_transaction_forwarder: parking_lot::RwLock>>, + /// Guard for getproof calls blocking_task_guard: BlockingTaskGuard, } @@ -226,7 +222,6 @@ where fee_history_cache: FeeHistoryCache, evm_config: EvmConfig, task_spawner: impl TaskSpawner + 'static, - raw_transaction_forwarder: Option>, proof_permits: usize, ) -> Self { let signers = parking_lot::RwLock::new(Default::default()); @@ -255,7 +250,6 @@ where blocking_task_pool, fee_history_cache, evm_config, - raw_transaction_forwarder: parking_lot::RwLock::new(raw_transaction_forwarder), blocking_task_guard: BlockingTaskGuard::new(proof_permits), } } @@ -304,12 +298,6 @@ impl EthApiInner Option> { - self.raw_transaction_forwarder.read().clone() - } - /// Returns the gas cap. #[inline] pub const fn gas_cap(&self) -> u64 { @@ -359,14 +347,6 @@ impl EthApiInner UpdateRawTxForwarder - for EthApiInner -{ - fn set_eth_raw_transaction_forwarder(&self, forwarder: Arc) { - self.raw_transaction_forwarder.write().replace(forwarder); - } -} - #[cfg(test)] mod tests { use jsonrpsee_types::error::INVALID_PARAMS_CODE; @@ -419,7 +399,6 @@ mod tests { BlockingTaskPool::build().expect("failed to build tracing pool"), fee_history_cache, evm_config, - None, DEFAULT_PROOF_PERMITS, ) } diff --git a/crates/rpc/rpc/src/eth/helpers/state.rs b/crates/rpc/rpc/src/eth/helpers/state.rs index a63ce5847c84..059db3fe59bd 100644 --- a/crates/rpc/rpc/src/eth/helpers/state.rs +++ b/crates/rpc/rpc/src/eth/helpers/state.rs @@ -73,7 +73,6 @@ mod tests { BlockingTaskPool::build().expect("failed to build tracing pool"), FeeHistoryCache::new(cache, FeeHistoryCacheConfig::default()), evm_config, - None, DEFAULT_PROOF_PERMITS, ) } @@ -99,7 +98,6 @@ mod tests { BlockingTaskPool::build().expect("failed to build tracing pool"), FeeHistoryCache::new(cache, FeeHistoryCacheConfig::default()), evm_config, - None, DEFAULT_PROOF_PERMITS, ) } diff --git a/crates/rpc/rpc/src/eth/helpers/transaction.rs b/crates/rpc/rpc/src/eth/helpers/transaction.rs index 635281c08e76..42af336811d5 100644 --- a/crates/rpc/rpc/src/eth/helpers/transaction.rs +++ b/crates/rpc/rpc/src/eth/helpers/transaction.rs @@ -1,12 +1,7 @@ //! Contains RPC handler implementations specific to transactions -use std::sync::Arc; - use reth_provider::{BlockReaderIdExt, TransactionsProvider}; -use reth_rpc_eth_api::{ - helpers::{EthSigner, EthTransactions, LoadTransaction, SpawnBlocking}, - RawTransactionForwarder, -}; +use reth_rpc_eth_api::helpers::{EthSigner, EthTransactions, LoadTransaction, SpawnBlocking}; use reth_rpc_eth_types::EthStateCache; use reth_transaction_pool::TransactionPool; @@ -24,11 +19,6 @@ where self.inner.provider() } - #[inline] - fn raw_tx_forwarder(&self) -> Option> { - self.inner.raw_tx_forwarder() - } - #[inline] fn signers(&self) -> &parking_lot::RwLock>> { self.inner.signers() @@ -98,7 +88,6 @@ mod tests { BlockingTaskPool::build().expect("failed to build tracing pool"), fee_history_cache, evm_config, - None, DEFAULT_PROOF_PERMITS, ); diff --git a/crates/rpc/rpc/src/eth/mod.rs b/crates/rpc/rpc/src/eth/mod.rs index 52b10c08c2bf..0addeb75ae77 100644 --- a/crates/rpc/rpc/src/eth/mod.rs +++ b/crates/rpc/rpc/src/eth/mod.rs @@ -14,4 +14,4 @@ pub use pubsub::EthPubSub; pub use helpers::signer::DevSigner; -pub use reth_rpc_eth_api::{EthApiServer, RawTransactionForwarder}; +pub use reth_rpc_eth_api::EthApiServer;