mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat(rpc): enable eth_getProof (#5071)
This commit is contained in:
@ -33,7 +33,7 @@ use reth_rpc_types::engine::{
|
|||||||
CancunPayloadFields, ExecutionPayload, PayloadAttributes, PayloadError, PayloadStatus,
|
CancunPayloadFields, ExecutionPayload, PayloadAttributes, PayloadError, PayloadStatus,
|
||||||
PayloadStatusEnum, PayloadValidationError,
|
PayloadStatusEnum, PayloadValidationError,
|
||||||
};
|
};
|
||||||
use reth_rpc_types_compat::payload::{try_into_block, validate_block_hash};
|
use reth_rpc_types_compat::engine::payload::{try_into_block, validate_block_hash};
|
||||||
use reth_stages::{ControlFlow, Pipeline, PipelineError};
|
use reth_stages::{ControlFlow, Pipeline, PipelineError};
|
||||||
use reth_tasks::TaskSpawner;
|
use reth_tasks::TaskSpawner;
|
||||||
use std::{
|
use std::{
|
||||||
|
|||||||
@ -553,7 +553,9 @@ mod tests {
|
|||||||
use reth_primitives::{
|
use reth_primitives::{
|
||||||
bytes,
|
bytes,
|
||||||
constants::{BEACON_ROOTS_ADDRESS, SYSTEM_ADDRESS},
|
constants::{BEACON_ROOTS_ADDRESS, SYSTEM_ADDRESS},
|
||||||
keccak256, Account, Bytecode, Bytes, ChainSpecBuilder, ForkCondition, StorageKey, MAINNET,
|
keccak256,
|
||||||
|
trie::AccountProof,
|
||||||
|
Account, Bytecode, Bytes, ChainSpecBuilder, ForkCondition, StorageKey, MAINNET,
|
||||||
};
|
};
|
||||||
use reth_provider::{AccountReader, BlockHashReader, StateRootProvider};
|
use reth_provider::{AccountReader, BlockHashReader, StateRootProvider};
|
||||||
use revm::{Database, TransitionState};
|
use revm::{Database, TransitionState};
|
||||||
@ -634,12 +636,8 @@ mod tests {
|
|||||||
Ok(self.contracts.get(&code_hash).cloned())
|
Ok(self.contracts.get(&code_hash).cloned())
|
||||||
}
|
}
|
||||||
|
|
||||||
fn proof(
|
fn proof(&self, _address: Address, _keys: &[B256]) -> RethResult<AccountProof> {
|
||||||
&self,
|
unimplemented!("proof generation is not supported")
|
||||||
_address: Address,
|
|
||||||
_keys: &[B256],
|
|
||||||
) -> RethResult<(Vec<Bytes>, B256, Vec<Vec<Bytes>>)> {
|
|
||||||
todo!()
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -17,8 +17,8 @@ use reth_provider::{
|
|||||||
};
|
};
|
||||||
use reth_rpc::{
|
use reth_rpc::{
|
||||||
eth::{cache::EthStateCache, gas_oracle::GasPriceOracle},
|
eth::{cache::EthStateCache, gas_oracle::GasPriceOracle},
|
||||||
AuthLayer, Claims, EngineEthApi, EthApi, EthFilter, EthSubscriptionIdProvider,
|
AuthLayer, BlockingTaskPool, Claims, EngineEthApi, EthApi, EthFilter,
|
||||||
JwtAuthValidator, JwtSecret, TracingCallPool,
|
EthSubscriptionIdProvider, JwtAuthValidator, JwtSecret,
|
||||||
};
|
};
|
||||||
use reth_rpc_api::{servers::*, EngineApiServer};
|
use reth_rpc_api::{servers::*, EngineApiServer};
|
||||||
use reth_tasks::TaskSpawner;
|
use reth_tasks::TaskSpawner;
|
||||||
@ -66,7 +66,7 @@ where
|
|||||||
gas_oracle,
|
gas_oracle,
|
||||||
EthConfig::default().rpc_gas_cap,
|
EthConfig::default().rpc_gas_cap,
|
||||||
Box::new(executor.clone()),
|
Box::new(executor.clone()),
|
||||||
TracingCallPool::build().expect("failed to build tracing pool"),
|
BlockingTaskPool::build().expect("failed to build tracing pool"),
|
||||||
);
|
);
|
||||||
let eth_filter = EthFilter::new(
|
let eth_filter = EthFilter::new(
|
||||||
provider,
|
provider,
|
||||||
|
|||||||
@ -5,7 +5,7 @@ use reth_rpc::{
|
|||||||
gas_oracle::GasPriceOracleConfig,
|
gas_oracle::GasPriceOracleConfig,
|
||||||
RPC_DEFAULT_GAS_CAP,
|
RPC_DEFAULT_GAS_CAP,
|
||||||
},
|
},
|
||||||
EthApi, EthFilter, EthPubSub, TracingCallPool,
|
BlockingTaskPool, EthApi, EthFilter, EthPubSub,
|
||||||
};
|
};
|
||||||
use serde::{Deserialize, Serialize};
|
use serde::{Deserialize, Serialize};
|
||||||
|
|
||||||
@ -21,7 +21,7 @@ pub struct EthHandlers<Provider, Pool, Network, Events> {
|
|||||||
/// Handler for subscriptions only available for transports that support it (ws, ipc)
|
/// Handler for subscriptions only available for transports that support it (ws, ipc)
|
||||||
pub pubsub: EthPubSub<Provider, Pool, Events, Network>,
|
pub pubsub: EthPubSub<Provider, Pool, Events, Network>,
|
||||||
/// The configured tracing call pool
|
/// The configured tracing call pool
|
||||||
pub tracing_call_pool: TracingCallPool,
|
pub blocking_task_pool: BlockingTaskPool,
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Additional config values for the eth namespace
|
/// Additional config values for the eth namespace
|
||||||
|
|||||||
@ -117,9 +117,9 @@ use reth_rpc::{
|
|||||||
cache::{cache_new_blocks_task, EthStateCache},
|
cache::{cache_new_blocks_task, EthStateCache},
|
||||||
gas_oracle::GasPriceOracle,
|
gas_oracle::GasPriceOracle,
|
||||||
},
|
},
|
||||||
AdminApi, DebugApi, EngineEthApi, EthApi, EthFilter, EthPubSub, EthSubscriptionIdProvider,
|
AdminApi, BlockingTaskGuard, BlockingTaskPool, DebugApi, EngineEthApi, EthApi, EthFilter,
|
||||||
NetApi, OtterscanApi, RPCApi, RethApi, TraceApi, TracingCallGuard, TracingCallPool, TxPoolApi,
|
EthPubSub, EthSubscriptionIdProvider, NetApi, OtterscanApi, RPCApi, RethApi, TraceApi,
|
||||||
Web3Api,
|
TxPoolApi, Web3Api,
|
||||||
};
|
};
|
||||||
use reth_rpc_api::{servers::*, EngineApiServer};
|
use reth_rpc_api::{servers::*, EngineApiServer};
|
||||||
use reth_tasks::{TaskSpawner, TokioTaskExecutor};
|
use reth_tasks::{TaskSpawner, TokioTaskExecutor};
|
||||||
@ -719,7 +719,7 @@ pub struct RethModuleRegistry<Provider, Pool, Network, Tasks, Events> {
|
|||||||
/// Holds a clone of all the eth namespace handlers
|
/// Holds a clone of all the eth namespace handlers
|
||||||
eth: Option<EthHandlers<Provider, Pool, Network, Events>>,
|
eth: Option<EthHandlers<Provider, Pool, Network, Events>>,
|
||||||
/// to put trace calls behind semaphore
|
/// to put trace calls behind semaphore
|
||||||
tracing_call_guard: TracingCallGuard,
|
blocking_pool_guard: BlockingTaskGuard,
|
||||||
/// Contains the [Methods] of a module
|
/// Contains the [Methods] of a module
|
||||||
modules: HashMap<RethRpcModule, Methods>,
|
modules: HashMap<RethRpcModule, Methods>,
|
||||||
}
|
}
|
||||||
@ -745,7 +745,7 @@ impl<Provider, Pool, Network, Tasks, Events>
|
|||||||
eth: None,
|
eth: None,
|
||||||
executor,
|
executor,
|
||||||
modules: Default::default(),
|
modules: Default::default(),
|
||||||
tracing_call_guard: TracingCallGuard::new(config.eth.max_tracing_requests),
|
blocking_pool_guard: BlockingTaskGuard::new(config.eth.max_tracing_requests),
|
||||||
config,
|
config,
|
||||||
events,
|
events,
|
||||||
}
|
}
|
||||||
@ -927,7 +927,7 @@ where
|
|||||||
filter: eth_filter,
|
filter: eth_filter,
|
||||||
pubsub: eth_pubsub,
|
pubsub: eth_pubsub,
|
||||||
cache: _,
|
cache: _,
|
||||||
tracing_call_pool: _,
|
blocking_task_pool: _,
|
||||||
} = self.with_eth(|eth| eth.clone());
|
} = self.with_eth(|eth| eth.clone());
|
||||||
|
|
||||||
// Create a copy, so we can list out all the methods for rpc_ api
|
// Create a copy, so we can list out all the methods for rpc_ api
|
||||||
@ -946,7 +946,7 @@ where
|
|||||||
self.provider.clone(),
|
self.provider.clone(),
|
||||||
eth_api.clone(),
|
eth_api.clone(),
|
||||||
Box::new(self.executor.clone()),
|
Box::new(self.executor.clone()),
|
||||||
self.tracing_call_guard.clone(),
|
self.blocking_pool_guard.clone(),
|
||||||
)
|
)
|
||||||
.into_rpc()
|
.into_rpc()
|
||||||
.into(),
|
.into(),
|
||||||
@ -964,7 +964,7 @@ where
|
|||||||
RethRpcModule::Trace => TraceApi::new(
|
RethRpcModule::Trace => TraceApi::new(
|
||||||
self.provider.clone(),
|
self.provider.clone(),
|
||||||
eth_api.clone(),
|
eth_api.clone(),
|
||||||
self.tracing_call_guard.clone(),
|
self.blocking_pool_guard.clone(),
|
||||||
)
|
)
|
||||||
.into_rpc()
|
.into_rpc()
|
||||||
.into(),
|
.into(),
|
||||||
@ -1026,7 +1026,8 @@ where
|
|||||||
);
|
);
|
||||||
|
|
||||||
let executor = Box::new(self.executor.clone());
|
let executor = Box::new(self.executor.clone());
|
||||||
let tracing_call_pool = TracingCallPool::build().expect("failed to build tracing pool");
|
let blocking_task_pool =
|
||||||
|
BlockingTaskPool::build().expect("failed to build tracing pool");
|
||||||
let api = EthApi::with_spawner(
|
let api = EthApi::with_spawner(
|
||||||
self.provider.clone(),
|
self.provider.clone(),
|
||||||
self.pool.clone(),
|
self.pool.clone(),
|
||||||
@ -1035,7 +1036,7 @@ where
|
|||||||
gas_oracle,
|
gas_oracle,
|
||||||
self.config.eth.rpc_gas_cap,
|
self.config.eth.rpc_gas_cap,
|
||||||
executor.clone(),
|
executor.clone(),
|
||||||
tracing_call_pool.clone(),
|
blocking_task_pool.clone(),
|
||||||
);
|
);
|
||||||
let filter = EthFilter::new(
|
let filter = EthFilter::new(
|
||||||
self.provider.clone(),
|
self.provider.clone(),
|
||||||
@ -1053,7 +1054,7 @@ where
|
|||||||
executor,
|
executor,
|
||||||
);
|
);
|
||||||
|
|
||||||
let eth = EthHandlers { api, cache, filter, pubsub, tracing_call_pool };
|
let eth = EthHandlers { api, cache, filter, pubsub, blocking_task_pool };
|
||||||
self.eth = Some(eth);
|
self.eth = Some(eth);
|
||||||
}
|
}
|
||||||
f(self.eth.as_ref().expect("exists; qed"))
|
f(self.eth.as_ref().expect("exists; qed"))
|
||||||
@ -1071,7 +1072,7 @@ where
|
|||||||
/// Instantiates TraceApi
|
/// Instantiates TraceApi
|
||||||
pub fn trace_api(&mut self) -> TraceApi<Provider, EthApi<Provider, Pool, Network>> {
|
pub fn trace_api(&mut self) -> TraceApi<Provider, EthApi<Provider, Pool, Network>> {
|
||||||
let eth = self.eth_handlers();
|
let eth = self.eth_handlers();
|
||||||
TraceApi::new(self.provider.clone(), eth.api, self.tracing_call_guard.clone())
|
TraceApi::new(self.provider.clone(), eth.api, self.blocking_pool_guard.clone())
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Instantiates OtterscanApi
|
/// Instantiates OtterscanApi
|
||||||
@ -1087,7 +1088,7 @@ where
|
|||||||
self.provider.clone(),
|
self.provider.clone(),
|
||||||
eth_api,
|
eth_api,
|
||||||
Box::new(self.executor.clone()),
|
Box::new(self.executor.clone()),
|
||||||
self.tracing_call_guard.clone(),
|
self.blocking_pool_guard.clone(),
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -114,11 +114,9 @@ where
|
|||||||
EthApiClient::submit_hashrate(client, U256::default(), B256::default()).await.unwrap();
|
EthApiClient::submit_hashrate(client, U256::default(), B256::default()).await.unwrap();
|
||||||
EthApiClient::gas_price(client).await.unwrap_err();
|
EthApiClient::gas_price(client).await.unwrap_err();
|
||||||
EthApiClient::max_priority_fee_per_gas(client).await.unwrap_err();
|
EthApiClient::max_priority_fee_per_gas(client).await.unwrap_err();
|
||||||
|
EthApiClient::get_proof(client, address, vec![], None).await.unwrap();
|
||||||
|
|
||||||
// Unimplemented
|
// Unimplemented
|
||||||
assert!(is_unimplemented(
|
|
||||||
EthApiClient::get_proof(client, address, vec![], None).await.err().unwrap()
|
|
||||||
));
|
|
||||||
assert!(is_unimplemented(EthApiClient::author(client).await.err().unwrap()));
|
assert!(is_unimplemented(EthApiClient::author(client).await.err().unwrap()));
|
||||||
assert!(is_unimplemented(EthApiClient::is_mining(client).await.err().unwrap()));
|
assert!(is_unimplemented(EthApiClient::is_mining(client).await.err().unwrap()));
|
||||||
assert!(is_unimplemented(EthApiClient::get_work(client).await.err().unwrap()));
|
assert!(is_unimplemented(EthApiClient::get_work(client).await.err().unwrap()));
|
||||||
|
|||||||
@ -12,10 +12,7 @@
|
|||||||
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
|
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
|
||||||
|
|
||||||
pub mod block;
|
pub mod block;
|
||||||
pub use block::*;
|
|
||||||
pub mod transaction;
|
|
||||||
pub use transaction::*;
|
|
||||||
pub mod engine;
|
pub mod engine;
|
||||||
pub use engine::*;
|
|
||||||
pub mod log;
|
pub mod log;
|
||||||
pub use log::*;
|
pub mod proof;
|
||||||
|
pub mod transaction;
|
||||||
|
|||||||
27
crates/rpc/rpc-types-compat/src/proof.rs
Normal file
27
crates/rpc/rpc-types-compat/src/proof.rs
Normal file
@ -0,0 +1,27 @@
|
|||||||
|
//! Compatibility functions for rpc proof related types.
|
||||||
|
|
||||||
|
use reth_primitives::{
|
||||||
|
serde_helper::JsonStorageKey,
|
||||||
|
trie::{AccountProof, StorageProof},
|
||||||
|
U64,
|
||||||
|
};
|
||||||
|
use reth_rpc_types::{EIP1186AccountProofResponse, EIP1186StorageProof};
|
||||||
|
|
||||||
|
/// Creates a new rpc storage proof from a primitive storage proof type.
|
||||||
|
pub fn from_primitive_storage_proof(proof: StorageProof) -> EIP1186StorageProof {
|
||||||
|
EIP1186StorageProof { key: JsonStorageKey(proof.key), value: proof.value, proof: proof.proof }
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Creates a new rpc account proof from a primitive account proof type.
|
||||||
|
pub fn from_primitive_account_proof(proof: AccountProof) -> EIP1186AccountProofResponse {
|
||||||
|
let info = proof.info.unwrap_or_default();
|
||||||
|
EIP1186AccountProofResponse {
|
||||||
|
address: proof.address,
|
||||||
|
balance: info.balance,
|
||||||
|
code_hash: info.get_bytecode_hash(),
|
||||||
|
nonce: U64::from(info.nonce),
|
||||||
|
storage_hash: proof.storage_root,
|
||||||
|
account_proof: proof.proof,
|
||||||
|
storage_proof: proof.storage_proofs.into_iter().map(from_primitive_storage_proof).collect(),
|
||||||
|
}
|
||||||
|
}
|
||||||
@ -12,7 +12,7 @@ pub struct AccountInfo {
|
|||||||
/// Data structure with proof for one single storage-entry
|
/// Data structure with proof for one single storage-entry
|
||||||
#[derive(Clone, Debug, Default, Eq, PartialEq, Serialize, Deserialize)]
|
#[derive(Clone, Debug, Default, Eq, PartialEq, Serialize, Deserialize)]
|
||||||
#[serde(rename_all = "camelCase")]
|
#[serde(rename_all = "camelCase")]
|
||||||
pub struct StorageProof {
|
pub struct EIP1186StorageProof {
|
||||||
/// Storage key.
|
/// Storage key.
|
||||||
pub key: JsonStorageKey,
|
pub key: JsonStorageKey,
|
||||||
/// Value that the key holds
|
/// Value that the key holds
|
||||||
@ -31,7 +31,7 @@ pub struct EIP1186AccountProofResponse {
|
|||||||
pub nonce: U64,
|
pub nonce: U64,
|
||||||
pub storage_hash: B256,
|
pub storage_hash: B256,
|
||||||
pub account_proof: Vec<Bytes>,
|
pub account_proof: Vec<Bytes>,
|
||||||
pub storage_proof: Vec<StorageProof>,
|
pub storage_proof: Vec<EIP1186StorageProof>,
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Extended account information (used by `parity_allAccountInfo`).
|
/// Extended account information (used by `parity_allAccountInfo`).
|
||||||
|
|||||||
@ -13,17 +13,19 @@ use tokio::sync::{oneshot, AcquireError, OwnedSemaphorePermit, Semaphore};
|
|||||||
/// RPC Tracing call guard semaphore.
|
/// RPC Tracing call guard semaphore.
|
||||||
///
|
///
|
||||||
/// This is used to restrict the number of concurrent RPC requests to tracing methods like
|
/// This is used to restrict the number of concurrent RPC requests to tracing methods like
|
||||||
/// `debug_traceTransaction` because they can consume a lot of memory and CPU.
|
/// `debug_traceTransaction` as well as `eth_getProof` because they can consume a lot of
|
||||||
|
/// memory and CPU.
|
||||||
///
|
///
|
||||||
/// This types serves as an entry guard for the [TracingCallPool] and is used to rate limit parallel
|
/// This types serves as an entry guard for the [BlockingTaskPool] and is used to rate limit
|
||||||
/// tracing calls on the pool.
|
/// parallel blocking tasks in the pool.
|
||||||
#[derive(Clone, Debug)]
|
#[derive(Clone, Debug)]
|
||||||
pub struct TracingCallGuard(Arc<Semaphore>);
|
pub struct BlockingTaskGuard(Arc<Semaphore>);
|
||||||
|
|
||||||
impl TracingCallGuard {
|
impl BlockingTaskGuard {
|
||||||
/// Create a new `TracingCallGuard` with the given maximum number of tracing calls in parallel.
|
/// Create a new `BlockingTaskGuard` with the given maximum number of blocking tasks in
|
||||||
pub fn new(max_tracing_requests: u32) -> Self {
|
/// parallel.
|
||||||
Self(Arc::new(Semaphore::new(max_tracing_requests as usize)))
|
pub fn new(max_blocking_tasks: u32) -> Self {
|
||||||
|
Self(Arc::new(Semaphore::new(max_blocking_tasks as usize)))
|
||||||
}
|
}
|
||||||
|
|
||||||
/// See also [Semaphore::acquire_owned]
|
/// See also [Semaphore::acquire_owned]
|
||||||
@ -37,24 +39,24 @@ impl TracingCallGuard {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Used to execute tracing calls on a rayon threadpool from within a tokio runtime.
|
/// Used to execute blocking tasks on a rayon threadpool from within a tokio runtime.
|
||||||
///
|
///
|
||||||
/// This is a dedicated threadpool for tracing calls which are CPU bound.
|
/// This is a dedicated threadpool for blocking tasks which are CPU bound.
|
||||||
/// RPC calls that perform blocking IO (disk lookups) are not executed on this pool but on the tokio
|
/// RPC calls that perform blocking IO (disk lookups) are not executed on this pool but on the tokio
|
||||||
/// runtime's blocking pool, which performs poorly with CPU bound tasks. Once the tokio blocking
|
/// runtime's blocking pool, which performs poorly with CPU bound tasks. Once the tokio blocking
|
||||||
/// pool is saturated it is converted into a queue, tracing calls could then interfere with the
|
/// pool is saturated it is converted into a queue, blocking tasks could then interfere with the
|
||||||
/// queue and block other RPC calls.
|
/// queue and block other RPC calls.
|
||||||
///
|
///
|
||||||
/// See also [tokio-docs] for more information.
|
/// See also [tokio-docs] for more information.
|
||||||
///
|
///
|
||||||
/// [tokio-docs]: https://docs.rs/tokio/latest/tokio/index.html#cpu-bound-tasks-and-blocking-code
|
/// [tokio-docs]: https://docs.rs/tokio/latest/tokio/index.html#cpu-bound-tasks-and-blocking-code
|
||||||
#[derive(Clone, Debug)]
|
#[derive(Clone, Debug)]
|
||||||
pub struct TracingCallPool {
|
pub struct BlockingTaskPool {
|
||||||
pool: Arc<rayon::ThreadPool>,
|
pool: Arc<rayon::ThreadPool>,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl TracingCallPool {
|
impl BlockingTaskPool {
|
||||||
/// Create a new `TracingCallPool` with the given threadpool.
|
/// Create a new `BlockingTaskPool` with the given threadpool.
|
||||||
pub fn new(pool: rayon::ThreadPool) -> Self {
|
pub fn new(pool: rayon::ThreadPool) -> Self {
|
||||||
Self { pool: Arc::new(pool) }
|
Self { pool: Arc::new(pool) }
|
||||||
}
|
}
|
||||||
@ -83,7 +85,7 @@ impl TracingCallPool {
|
|||||||
/// function's return value.
|
/// function's return value.
|
||||||
///
|
///
|
||||||
/// If the function panics, the future will resolve to an error.
|
/// If the function panics, the future will resolve to an error.
|
||||||
pub fn spawn<F, R>(&self, func: F) -> TracingCallHandle<R>
|
pub fn spawn<F, R>(&self, func: F) -> BlockingTaskHandle<R>
|
||||||
where
|
where
|
||||||
F: FnOnce() -> R + Send + 'static,
|
F: FnOnce() -> R + Send + 'static,
|
||||||
R: Send + 'static,
|
R: Send + 'static,
|
||||||
@ -94,7 +96,7 @@ impl TracingCallPool {
|
|||||||
let _result = tx.send(catch_unwind(AssertUnwindSafe(func)));
|
let _result = tx.send(catch_unwind(AssertUnwindSafe(func)));
|
||||||
});
|
});
|
||||||
|
|
||||||
TracingCallHandle { rx }
|
BlockingTaskHandle { rx }
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Asynchronous wrapper around Rayon's
|
/// Asynchronous wrapper around Rayon's
|
||||||
@ -104,7 +106,7 @@ impl TracingCallPool {
|
|||||||
/// function's return value.
|
/// function's return value.
|
||||||
///
|
///
|
||||||
/// If the function panics, the future will resolve to an error.
|
/// If the function panics, the future will resolve to an error.
|
||||||
pub fn spawn_fifo<F, R>(&self, func: F) -> TracingCallHandle<R>
|
pub fn spawn_fifo<F, R>(&self, func: F) -> BlockingTaskHandle<R>
|
||||||
where
|
where
|
||||||
F: FnOnce() -> R + Send + 'static,
|
F: FnOnce() -> R + Send + 'static,
|
||||||
R: Send + 'static,
|
R: Send + 'static,
|
||||||
@ -115,11 +117,11 @@ impl TracingCallPool {
|
|||||||
let _result = tx.send(catch_unwind(AssertUnwindSafe(func)));
|
let _result = tx.send(catch_unwind(AssertUnwindSafe(func)));
|
||||||
});
|
});
|
||||||
|
|
||||||
TracingCallHandle { rx }
|
BlockingTaskHandle { rx }
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Async handle for a blocking tracing task running in a Rayon thread pool.
|
/// Async handle for a blocking task running in a Rayon thread pool.
|
||||||
///
|
///
|
||||||
/// ## Panics
|
/// ## Panics
|
||||||
///
|
///
|
||||||
@ -127,18 +129,18 @@ impl TracingCallPool {
|
|||||||
#[derive(Debug)]
|
#[derive(Debug)]
|
||||||
#[must_use = "futures do nothing unless you `.await` or poll them"]
|
#[must_use = "futures do nothing unless you `.await` or poll them"]
|
||||||
#[pin_project::pin_project]
|
#[pin_project::pin_project]
|
||||||
pub struct TracingCallHandle<T> {
|
pub struct BlockingTaskHandle<T> {
|
||||||
#[pin]
|
#[pin]
|
||||||
pub(crate) rx: oneshot::Receiver<thread::Result<T>>,
|
pub(crate) rx: oneshot::Receiver<thread::Result<T>>,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<T> Future for TracingCallHandle<T> {
|
impl<T> Future for BlockingTaskHandle<T> {
|
||||||
type Output = thread::Result<T>;
|
type Output = thread::Result<T>;
|
||||||
|
|
||||||
fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
|
fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
|
||||||
match ready!(self.project().rx.poll(cx)) {
|
match ready!(self.project().rx.poll(cx)) {
|
||||||
Ok(res) => Poll::Ready(res),
|
Ok(res) => Poll::Ready(res),
|
||||||
Err(_) => Poll::Ready(Err(Box::<TokioTracingCallError>::default())),
|
Err(_) => Poll::Ready(Err(Box::<TokioBlockingTaskError>::default())),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -149,23 +151,23 @@ impl<T> Future for TracingCallHandle<T> {
|
|||||||
#[derive(Debug, Default, thiserror::Error)]
|
#[derive(Debug, Default, thiserror::Error)]
|
||||||
#[error("Tokio channel dropped while awaiting result")]
|
#[error("Tokio channel dropped while awaiting result")]
|
||||||
#[non_exhaustive]
|
#[non_exhaustive]
|
||||||
pub struct TokioTracingCallError;
|
pub struct TokioBlockingTaskError;
|
||||||
|
|
||||||
#[cfg(test)]
|
#[cfg(test)]
|
||||||
mod tests {
|
mod tests {
|
||||||
use super::*;
|
use super::*;
|
||||||
|
|
||||||
#[tokio::test]
|
#[tokio::test]
|
||||||
async fn tracing_pool() {
|
async fn blocking_pool() {
|
||||||
let pool = TracingCallPool::build().unwrap();
|
let pool = BlockingTaskPool::build().unwrap();
|
||||||
let res = pool.spawn(move || 5);
|
let res = pool.spawn(move || 5);
|
||||||
let res = res.await.unwrap();
|
let res = res.await.unwrap();
|
||||||
assert_eq!(res, 5);
|
assert_eq!(res, 5);
|
||||||
}
|
}
|
||||||
|
|
||||||
#[tokio::test]
|
#[tokio::test]
|
||||||
async fn tracing_pool_panic() {
|
async fn blocking_pool_panic() {
|
||||||
let pool = TracingCallPool::build().unwrap();
|
let pool = BlockingTaskPool::build().unwrap();
|
||||||
let res = pool.spawn(move || -> i32 {
|
let res = pool.spawn(move || -> i32 {
|
||||||
panic!();
|
panic!();
|
||||||
});
|
});
|
||||||
@ -8,7 +8,7 @@ use crate::{
|
|||||||
EthTransactions, TransactionSource,
|
EthTransactions, TransactionSource,
|
||||||
},
|
},
|
||||||
result::{internal_rpc_err, ToRpcResult},
|
result::{internal_rpc_err, ToRpcResult},
|
||||||
EthApiSpec, TracingCallGuard,
|
BlockingTaskGuard, EthApiSpec,
|
||||||
};
|
};
|
||||||
use alloy_rlp::{Decodable, Encodable};
|
use alloy_rlp::{Decodable, Encodable};
|
||||||
use async_trait::async_trait;
|
use async_trait::async_trait;
|
||||||
@ -61,10 +61,10 @@ impl<Provider, Eth> DebugApi<Provider, Eth> {
|
|||||||
provider: Provider,
|
provider: Provider,
|
||||||
eth: Eth,
|
eth: Eth,
|
||||||
task_spawner: Box<dyn TaskSpawner>,
|
task_spawner: Box<dyn TaskSpawner>,
|
||||||
tracing_call_guard: TracingCallGuard,
|
blocking_task_guard: BlockingTaskGuard,
|
||||||
) -> Self {
|
) -> Self {
|
||||||
let inner =
|
let inner =
|
||||||
Arc::new(DebugApiInner { provider, eth_api: eth, task_spawner, tracing_call_guard });
|
Arc::new(DebugApiInner { provider, eth_api: eth, task_spawner, blocking_task_guard });
|
||||||
Self { inner }
|
Self { inner }
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -78,7 +78,7 @@ where
|
|||||||
{
|
{
|
||||||
/// Acquires a permit to execute a tracing call.
|
/// Acquires a permit to execute a tracing call.
|
||||||
async fn acquire_trace_permit(&self) -> Result<OwnedSemaphorePermit, AcquireError> {
|
async fn acquire_trace_permit(&self) -> Result<OwnedSemaphorePermit, AcquireError> {
|
||||||
self.inner.tracing_call_guard.clone().acquire_owned().await
|
self.inner.blocking_task_guard.clone().acquire_owned().await
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Trace the entire block asynchronously
|
/// Trace the entire block asynchronously
|
||||||
@ -1010,8 +1010,8 @@ struct DebugApiInner<Provider, Eth> {
|
|||||||
provider: Provider,
|
provider: Provider,
|
||||||
/// The implementation of `eth` API
|
/// The implementation of `eth` API
|
||||||
eth_api: Eth,
|
eth_api: Eth,
|
||||||
// restrict the number of concurrent calls to tracing calls
|
// restrict the number of concurrent calls to blocking calls
|
||||||
tracing_call_guard: TracingCallGuard,
|
blocking_task_guard: BlockingTaskGuard,
|
||||||
/// The type that can spawn tasks which would otherwise block.
|
/// The type that can spawn tasks which would otherwise block.
|
||||||
task_spawner: Box<dyn TaskSpawner>,
|
task_spawner: Box<dyn TaskSpawner>,
|
||||||
}
|
}
|
||||||
|
|||||||
@ -39,7 +39,7 @@ mod sign;
|
|||||||
mod state;
|
mod state;
|
||||||
mod transactions;
|
mod transactions;
|
||||||
|
|
||||||
use crate::TracingCallPool;
|
use crate::BlockingTaskPool;
|
||||||
pub use transactions::{EthTransactions, TransactionSource};
|
pub use transactions::{EthTransactions, TransactionSource};
|
||||||
|
|
||||||
/// `Eth` API trait.
|
/// `Eth` API trait.
|
||||||
@ -91,7 +91,7 @@ where
|
|||||||
eth_cache: EthStateCache,
|
eth_cache: EthStateCache,
|
||||||
gas_oracle: GasPriceOracle<Provider>,
|
gas_oracle: GasPriceOracle<Provider>,
|
||||||
gas_cap: impl Into<GasCap>,
|
gas_cap: impl Into<GasCap>,
|
||||||
tracing_call_pool: TracingCallPool,
|
blocking_task_pool: BlockingTaskPool,
|
||||||
) -> Self {
|
) -> Self {
|
||||||
Self::with_spawner(
|
Self::with_spawner(
|
||||||
provider,
|
provider,
|
||||||
@ -101,7 +101,7 @@ where
|
|||||||
gas_oracle,
|
gas_oracle,
|
||||||
gas_cap.into().into(),
|
gas_cap.into().into(),
|
||||||
Box::<TokioTaskExecutor>::default(),
|
Box::<TokioTaskExecutor>::default(),
|
||||||
tracing_call_pool,
|
blocking_task_pool,
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -115,7 +115,7 @@ where
|
|||||||
gas_oracle: GasPriceOracle<Provider>,
|
gas_oracle: GasPriceOracle<Provider>,
|
||||||
gas_cap: u64,
|
gas_cap: u64,
|
||||||
task_spawner: Box<dyn TaskSpawner>,
|
task_spawner: Box<dyn TaskSpawner>,
|
||||||
tracing_call_pool: TracingCallPool,
|
blocking_task_pool: BlockingTaskPool,
|
||||||
) -> Self {
|
) -> Self {
|
||||||
// get the block number of the latest block
|
// get the block number of the latest block
|
||||||
let latest_block = provider
|
let latest_block = provider
|
||||||
@ -136,7 +136,7 @@ where
|
|||||||
starting_block: U256::from(latest_block),
|
starting_block: U256::from(latest_block),
|
||||||
task_spawner,
|
task_spawner,
|
||||||
pending_block: Default::default(),
|
pending_block: Default::default(),
|
||||||
tracing_call_pool,
|
blocking_task_pool,
|
||||||
};
|
};
|
||||||
Self { inner: Arc::new(inner) }
|
Self { inner: Arc::new(inner) }
|
||||||
}
|
}
|
||||||
@ -436,6 +436,6 @@ struct EthApiInner<Provider, Pool, Network> {
|
|||||||
task_spawner: Box<dyn TaskSpawner>,
|
task_spawner: Box<dyn TaskSpawner>,
|
||||||
/// Cached pending block if any
|
/// Cached pending block if any
|
||||||
pending_block: Mutex<Option<PendingBlock>>,
|
pending_block: Mutex<Option<PendingBlock>>,
|
||||||
/// A pool dedicated to tracing calls
|
/// A pool dedicated to blocking tasks.
|
||||||
tracing_call_pool: TracingCallPool,
|
blocking_task_pool: BlockingTaskPool,
|
||||||
}
|
}
|
||||||
|
|||||||
@ -5,6 +5,7 @@ use super::EthApiSpec;
|
|||||||
use crate::{
|
use crate::{
|
||||||
eth::{
|
eth::{
|
||||||
api::{EthApi, EthTransactions},
|
api::{EthApi, EthTransactions},
|
||||||
|
error::EthApiError,
|
||||||
revm_utils::EvmOverrides,
|
revm_utils::EvmOverrides,
|
||||||
},
|
},
|
||||||
result::{internal_rpc_err, ToRpcResult},
|
result::{internal_rpc_err, ToRpcResult},
|
||||||
@ -368,21 +369,19 @@ where
|
|||||||
/// Handler for: `eth_getProof`
|
/// Handler for: `eth_getProof`
|
||||||
async fn get_proof(
|
async fn get_proof(
|
||||||
&self,
|
&self,
|
||||||
_address: Address,
|
address: Address,
|
||||||
_keys: Vec<JsonStorageKey>,
|
keys: Vec<JsonStorageKey>,
|
||||||
_block_number: Option<BlockId>,
|
block_number: Option<BlockId>,
|
||||||
) -> Result<EIP1186AccountProofResponse> {
|
) -> Result<EIP1186AccountProofResponse> {
|
||||||
// TODO: uncomment when implemented
|
trace!(target: "rpc::eth", ?address, ?keys, ?block_number, "Serving eth_getProof");
|
||||||
// trace!(target: "rpc::eth", ?address, ?keys, ?block_number, "Serving eth_getProof");
|
let res = EthApi::get_proof(self, address, keys, block_number).await;
|
||||||
// let res = EthApi::get_proof(self, address, keys, block_number);
|
|
||||||
|
|
||||||
// Ok(res.map_err(|e| match e {
|
Ok(res.map_err(|e| match e {
|
||||||
// EthApiError::InvalidBlockRange => {
|
EthApiError::InvalidBlockRange => {
|
||||||
// internal_rpc_err("eth_getProof is unimplemented for historical blocks")
|
internal_rpc_err("eth_getProof is unimplemented for historical blocks")
|
||||||
// }
|
}
|
||||||
// _ => e.into(),
|
_ => e.into(),
|
||||||
// })?)
|
})?)
|
||||||
Err(internal_rpc_err("unimplemented"))
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -390,7 +389,7 @@ where
|
|||||||
mod tests {
|
mod tests {
|
||||||
use crate::{
|
use crate::{
|
||||||
eth::{cache::EthStateCache, gas_oracle::GasPriceOracle},
|
eth::{cache::EthStateCache, gas_oracle::GasPriceOracle},
|
||||||
EthApi, TracingCallPool,
|
BlockingTaskPool, EthApi,
|
||||||
};
|
};
|
||||||
use jsonrpsee::types::error::INVALID_PARAMS_CODE;
|
use jsonrpsee::types::error::INVALID_PARAMS_CODE;
|
||||||
use reth_interfaces::test_utils::{generators, generators::Rng};
|
use reth_interfaces::test_utils::{generators, generators::Rng};
|
||||||
@ -428,7 +427,7 @@ mod tests {
|
|||||||
cache.clone(),
|
cache.clone(),
|
||||||
GasPriceOracle::new(provider, Default::default(), cache),
|
GasPriceOracle::new(provider, Default::default(), cache),
|
||||||
ETHEREUM_BLOCK_GAS_LIMIT,
|
ETHEREUM_BLOCK_GAS_LIMIT,
|
||||||
TracingCallPool::build().expect("failed to build tracing pool"),
|
BlockingTaskPool::build().expect("failed to build tracing pool"),
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -5,14 +5,13 @@ use crate::{
|
|||||||
EthApi,
|
EthApi,
|
||||||
};
|
};
|
||||||
use reth_primitives::{
|
use reth_primitives::{
|
||||||
serde_helper::JsonStorageKey, Address, BlockId, BlockNumberOrTag, Bytes, B256, KECCAK_EMPTY,
|
serde_helper::JsonStorageKey, Address, BlockId, BlockNumberOrTag, Bytes, B256, U256,
|
||||||
U256, U64,
|
|
||||||
};
|
};
|
||||||
use reth_provider::{
|
use reth_provider::{
|
||||||
AccountReader, BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, StateProvider,
|
BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, StateProvider, StateProviderFactory,
|
||||||
StateProviderFactory,
|
|
||||||
};
|
};
|
||||||
use reth_rpc_types::{EIP1186AccountProofResponse, StorageProof};
|
use reth_rpc_types::EIP1186AccountProofResponse;
|
||||||
|
use reth_rpc_types_compat::proof::from_primitive_account_proof;
|
||||||
use reth_transaction_pool::{PoolTransaction, TransactionPool};
|
use reth_transaction_pool::{PoolTransaction, TransactionPool};
|
||||||
|
|
||||||
impl<Provider, Pool, Network> EthApi<Provider, Pool, Network>
|
impl<Provider, Pool, Network> EthApi<Provider, Pool, Network>
|
||||||
@ -84,8 +83,7 @@ where
|
|||||||
Ok(B256::new(value.to_be_bytes()))
|
Ok(B256::new(value.to_be_bytes()))
|
||||||
}
|
}
|
||||||
|
|
||||||
#[allow(unused)]
|
pub(crate) async fn get_proof(
|
||||||
pub(crate) fn get_proof(
|
|
||||||
&self,
|
&self,
|
||||||
address: Address,
|
address: Address,
|
||||||
keys: Vec<JsonStorageKey>,
|
keys: Vec<JsonStorageKey>,
|
||||||
@ -97,7 +95,7 @@ where
|
|||||||
// if we are trying to create a proof for the latest block, but have a BlockId as input
|
// if we are trying to create a proof for the latest block, but have a BlockId as input
|
||||||
// that is not BlockNumberOrTag::Latest, then we need to figure out whether or not the
|
// that is not BlockNumberOrTag::Latest, then we need to figure out whether or not the
|
||||||
// BlockId corresponds to the latest block
|
// BlockId corresponds to the latest block
|
||||||
let is_blockid_latest = match block_id {
|
let is_latest_block = match block_id {
|
||||||
BlockId::Number(BlockNumberOrTag::Number(num)) => num == chain_info.best_number,
|
BlockId::Number(BlockNumberOrTag::Number(num)) => num == chain_info.best_number,
|
||||||
BlockId::Hash(hash) => hash == chain_info.best_hash.into(),
|
BlockId::Hash(hash) => hash == chain_info.best_hash.into(),
|
||||||
BlockId::Number(BlockNumberOrTag::Latest) => true,
|
BlockId::Number(BlockNumberOrTag::Latest) => true,
|
||||||
@ -105,43 +103,21 @@ where
|
|||||||
};
|
};
|
||||||
|
|
||||||
// TODO: remove when HistoricalStateProviderRef::proof is implemented
|
// TODO: remove when HistoricalStateProviderRef::proof is implemented
|
||||||
if !is_blockid_latest {
|
if !is_latest_block {
|
||||||
return Err(EthApiError::InvalidBlockRange)
|
return Err(EthApiError::InvalidBlockRange)
|
||||||
}
|
}
|
||||||
|
|
||||||
let state = self.state_at_block_id(block_id)?;
|
let this = self.clone();
|
||||||
|
self.inner
|
||||||
let hash_keys = keys.iter().map(|key| key.0).collect::<Vec<_>>();
|
.blocking_task_pool
|
||||||
let (account_proof, storage_hash, stg_proofs) = state.proof(address, &hash_keys)?;
|
.spawn(move || {
|
||||||
|
let state = this.state_at_block_id(block_id)?;
|
||||||
let storage_proof = keys
|
let storage_keys = keys.iter().map(|key| key.0).collect::<Vec<_>>();
|
||||||
.into_iter()
|
let proof = state.proof(address, &storage_keys)?;
|
||||||
.zip(stg_proofs)
|
Ok(from_primitive_account_proof(proof))
|
||||||
.map(|(key, proof)| {
|
|
||||||
state.storage(address, key.0).map(|op| StorageProof {
|
|
||||||
key,
|
|
||||||
value: op.unwrap_or_default(),
|
|
||||||
proof,
|
|
||||||
})
|
|
||||||
})
|
})
|
||||||
.collect::<Result<_, _>>()?;
|
.await
|
||||||
|
.map_err(|_| EthApiError::InternalBlockingTaskError)?
|
||||||
let mut proof = EIP1186AccountProofResponse {
|
|
||||||
address,
|
|
||||||
code_hash: KECCAK_EMPTY,
|
|
||||||
account_proof,
|
|
||||||
storage_hash,
|
|
||||||
storage_proof,
|
|
||||||
..Default::default()
|
|
||||||
};
|
|
||||||
|
|
||||||
if let Some(account) = state.basic_account(proof.address)? {
|
|
||||||
proof.balance = account.balance;
|
|
||||||
proof.nonce = U64::from(account.nonce);
|
|
||||||
proof.code_hash = account.get_bytecode_hash();
|
|
||||||
}
|
|
||||||
|
|
||||||
Ok(proof)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -150,7 +126,7 @@ mod tests {
|
|||||||
use super::*;
|
use super::*;
|
||||||
use crate::{
|
use crate::{
|
||||||
eth::{cache::EthStateCache, gas_oracle::GasPriceOracle},
|
eth::{cache::EthStateCache, gas_oracle::GasPriceOracle},
|
||||||
TracingCallPool,
|
BlockingTaskPool,
|
||||||
};
|
};
|
||||||
use reth_primitives::{constants::ETHEREUM_BLOCK_GAS_LIMIT, StorageKey, StorageValue};
|
use reth_primitives::{constants::ETHEREUM_BLOCK_GAS_LIMIT, StorageKey, StorageValue};
|
||||||
use reth_provider::test_utils::{ExtendedAccount, MockEthProvider, NoopProvider};
|
use reth_provider::test_utils::{ExtendedAccount, MockEthProvider, NoopProvider};
|
||||||
@ -170,7 +146,7 @@ mod tests {
|
|||||||
cache.clone(),
|
cache.clone(),
|
||||||
GasPriceOracle::new(NoopProvider::default(), Default::default(), cache),
|
GasPriceOracle::new(NoopProvider::default(), Default::default(), cache),
|
||||||
ETHEREUM_BLOCK_GAS_LIMIT,
|
ETHEREUM_BLOCK_GAS_LIMIT,
|
||||||
TracingCallPool::build().expect("failed to build tracing pool"),
|
BlockingTaskPool::build().expect("failed to build tracing pool"),
|
||||||
);
|
);
|
||||||
let address = Address::random();
|
let address = Address::random();
|
||||||
let storage = eth_api.storage_at(address, U256::ZERO.into(), None).unwrap();
|
let storage = eth_api.storage_at(address, U256::ZERO.into(), None).unwrap();
|
||||||
@ -192,7 +168,7 @@ mod tests {
|
|||||||
cache.clone(),
|
cache.clone(),
|
||||||
GasPriceOracle::new(mock_provider, Default::default(), cache),
|
GasPriceOracle::new(mock_provider, Default::default(), cache),
|
||||||
ETHEREUM_BLOCK_GAS_LIMIT,
|
ETHEREUM_BLOCK_GAS_LIMIT,
|
||||||
TracingCallPool::build().expect("failed to build tracing pool"),
|
BlockingTaskPool::build().expect("failed to build tracing pool"),
|
||||||
);
|
);
|
||||||
|
|
||||||
let storage_key: U256 = storage_key.into();
|
let storage_key: U256 = storage_key.into();
|
||||||
|
|||||||
@ -32,7 +32,7 @@ use reth_rpc_types::{
|
|||||||
BlockError, CallRequest, Index, Log, Transaction, TransactionInfo, TransactionReceipt,
|
BlockError, CallRequest, Index, Log, Transaction, TransactionInfo, TransactionReceipt,
|
||||||
TransactionRequest, TypedTransactionRequest,
|
TransactionRequest, TypedTransactionRequest,
|
||||||
};
|
};
|
||||||
use reth_rpc_types_compat::from_recovered_with_block_context;
|
use reth_rpc_types_compat::transaction::from_recovered_with_block_context;
|
||||||
use reth_transaction_pool::{TransactionOrigin, TransactionPool};
|
use reth_transaction_pool::{TransactionOrigin, TransactionPool};
|
||||||
use revm::{
|
use revm::{
|
||||||
db::CacheDB,
|
db::CacheDB,
|
||||||
@ -50,7 +50,7 @@ pub(crate) type StateCacheDB<'r> = CacheDB<StateProviderDatabase<StateProviderBo
|
|||||||
/// Commonly used transaction related functions for the [EthApi] type in the `eth_` namespace.
|
/// Commonly used transaction related functions for the [EthApi] type in the `eth_` namespace.
|
||||||
///
|
///
|
||||||
/// Async functions that are spawned onto the
|
/// Async functions that are spawned onto the
|
||||||
/// [TracingCallPool](crate::tracing_call::TracingCallPool) begin with `spawn_`
|
/// [BlockingTaskPool](crate::blocking_pool::BlockingTaskPool) begin with `spawn_`
|
||||||
#[async_trait::async_trait]
|
#[async_trait::async_trait]
|
||||||
pub trait EthTransactions: Send + Sync {
|
pub trait EthTransactions: Send + Sync {
|
||||||
/// Returns default gas limit to use for `eth_call` and tracing RPC methods.
|
/// Returns default gas limit to use for `eth_call` and tracing RPC methods.
|
||||||
@ -222,7 +222,7 @@ pub trait EthTransactions: Send + Sync {
|
|||||||
/// the database that points to the beginning of the transaction.
|
/// the database that points to the beginning of the transaction.
|
||||||
///
|
///
|
||||||
/// Note: Implementers should use a threadpool where blocking is allowed, such as
|
/// Note: Implementers should use a threadpool where blocking is allowed, such as
|
||||||
/// [TracingCallPool](crate::tracing_call::TracingCallPool).
|
/// [BlockingTaskPool](crate::blocking_pool::BlockingTaskPool).
|
||||||
async fn spawn_trace_transaction_in_block<F, R>(
|
async fn spawn_trace_transaction_in_block<F, R>(
|
||||||
&self,
|
&self,
|
||||||
hash: B256,
|
hash: B256,
|
||||||
@ -325,13 +325,13 @@ where
|
|||||||
{
|
{
|
||||||
let this = self.clone();
|
let this = self.clone();
|
||||||
self.inner
|
self.inner
|
||||||
.tracing_call_pool
|
.blocking_task_pool
|
||||||
.spawn(move || {
|
.spawn(move || {
|
||||||
let state = this.state_at(at)?;
|
let state = this.state_at(at)?;
|
||||||
f(state)
|
f(state)
|
||||||
})
|
})
|
||||||
.await
|
.await
|
||||||
.map_err(|_| EthApiError::InternalTracingError)?
|
.map_err(|_| EthApiError::InternalBlockingTaskError)?
|
||||||
}
|
}
|
||||||
|
|
||||||
async fn evm_env_at(&self, at: BlockId) -> EthResult<(CfgEnv, BlockEnv, BlockId)> {
|
async fn evm_env_at(&self, at: BlockId) -> EthResult<(CfgEnv, BlockEnv, BlockId)> {
|
||||||
@ -594,7 +594,7 @@ where
|
|||||||
let (cfg, block_env, at) = self.evm_env_at(at).await?;
|
let (cfg, block_env, at) = self.evm_env_at(at).await?;
|
||||||
let this = self.clone();
|
let this = self.clone();
|
||||||
self.inner
|
self.inner
|
||||||
.tracing_call_pool
|
.blocking_task_pool
|
||||||
.spawn(move || {
|
.spawn(move || {
|
||||||
let state = this.state_at(at)?;
|
let state = this.state_at(at)?;
|
||||||
let mut db = CacheDB::new(StateProviderDatabase::new(state));
|
let mut db = CacheDB::new(StateProviderDatabase::new(state));
|
||||||
@ -610,7 +610,7 @@ where
|
|||||||
f(db, env)
|
f(db, env)
|
||||||
})
|
})
|
||||||
.await
|
.await
|
||||||
.map_err(|_| EthApiError::InternalTracingError)?
|
.map_err(|_| EthApiError::InternalBlockingTaskError)?
|
||||||
}
|
}
|
||||||
|
|
||||||
async fn transact_call_at(
|
async fn transact_call_at(
|
||||||
@ -1094,7 +1094,7 @@ mod tests {
|
|||||||
use super::*;
|
use super::*;
|
||||||
use crate::{
|
use crate::{
|
||||||
eth::{cache::EthStateCache, gas_oracle::GasPriceOracle},
|
eth::{cache::EthStateCache, gas_oracle::GasPriceOracle},
|
||||||
EthApi, TracingCallPool,
|
BlockingTaskPool, EthApi,
|
||||||
};
|
};
|
||||||
use reth_network_api::noop::NoopNetwork;
|
use reth_network_api::noop::NoopNetwork;
|
||||||
use reth_primitives::{constants::ETHEREUM_BLOCK_GAS_LIMIT, hex_literal::hex, Bytes};
|
use reth_primitives::{constants::ETHEREUM_BLOCK_GAS_LIMIT, hex_literal::hex, Bytes};
|
||||||
@ -1116,7 +1116,7 @@ mod tests {
|
|||||||
cache.clone(),
|
cache.clone(),
|
||||||
GasPriceOracle::new(noop_provider, Default::default(), cache),
|
GasPriceOracle::new(noop_provider, Default::default(), cache),
|
||||||
ETHEREUM_BLOCK_GAS_LIMIT,
|
ETHEREUM_BLOCK_GAS_LIMIT,
|
||||||
TracingCallPool::build().expect("failed to build tracing pool"),
|
BlockingTaskPool::build().expect("failed to build tracing pool"),
|
||||||
);
|
);
|
||||||
|
|
||||||
// https://etherscan.io/tx/0xa694b71e6c128a2ed8e2e0f6770bddbe52e3bb8f10e8472f9a79ab81497a8b5d
|
// https://etherscan.io/tx/0xa694b71e6c128a2ed8e2e0f6770bddbe52e3bb8f10e8472f9a79ab81497a8b5d
|
||||||
|
|||||||
@ -83,12 +83,12 @@ pub enum EthApiError {
|
|||||||
/// Percentile array is invalid
|
/// Percentile array is invalid
|
||||||
#[error("invalid reward percentiles")]
|
#[error("invalid reward percentiles")]
|
||||||
InvalidRewardPercentiles,
|
InvalidRewardPercentiles,
|
||||||
/// Error thrown when a spawned tracing task failed to deliver an anticipated response.
|
/// Error thrown when a spawned blocking task failed to deliver an anticipated response.
|
||||||
///
|
///
|
||||||
/// This only happens if the tracing task panics and is aborted before it can return a response
|
/// This only happens if the blocking task panics and is aborted before it can return a
|
||||||
/// back to the request handler.
|
/// response back to the request handler.
|
||||||
#[error("internal error while tracing")]
|
#[error("internal blocking task error")]
|
||||||
InternalTracingError,
|
InternalBlockingTaskError,
|
||||||
/// Error thrown when a spawned blocking task failed to deliver an anticipated response.
|
/// Error thrown when a spawned blocking task failed to deliver an anticipated response.
|
||||||
#[error("internal eth error")]
|
#[error("internal eth error")]
|
||||||
InternalEthError,
|
InternalEthError,
|
||||||
@ -133,7 +133,7 @@ impl From<EthApiError> for ErrorObject<'static> {
|
|||||||
err @ EthApiError::ExecutionTimedOut(_) => {
|
err @ EthApiError::ExecutionTimedOut(_) => {
|
||||||
rpc_error_with_code(CALL_EXECUTION_FAILED_CODE, err.to_string())
|
rpc_error_with_code(CALL_EXECUTION_FAILED_CODE, err.to_string())
|
||||||
}
|
}
|
||||||
err @ EthApiError::InternalTracingError => internal_rpc_err(err.to_string()),
|
err @ EthApiError::InternalBlockingTaskError => internal_rpc_err(err.to_string()),
|
||||||
err @ EthApiError::InternalEthError => internal_rpc_err(err.to_string()),
|
err @ EthApiError::InternalEthError => internal_rpc_err(err.to_string()),
|
||||||
err @ EthApiError::CallInputError(_) => invalid_params_rpc_err(err.to_string()),
|
err @ EthApiError::CallInputError(_) => invalid_params_rpc_err(err.to_string()),
|
||||||
}
|
}
|
||||||
|
|||||||
@ -36,11 +36,11 @@ mod otterscan;
|
|||||||
mod reth;
|
mod reth;
|
||||||
mod rpc;
|
mod rpc;
|
||||||
mod trace;
|
mod trace;
|
||||||
pub mod tracing_call;
|
|
||||||
mod txpool;
|
mod txpool;
|
||||||
mod web3;
|
mod web3;
|
||||||
|
|
||||||
pub use admin::AdminApi;
|
pub use admin::AdminApi;
|
||||||
|
pub use blocking_pool::{BlockingTaskGuard, BlockingTaskPool};
|
||||||
pub use debug::DebugApi;
|
pub use debug::DebugApi;
|
||||||
pub use engine::{EngineApi, EngineEthApi};
|
pub use engine::{EngineApi, EngineEthApi};
|
||||||
pub use eth::{EthApi, EthApiSpec, EthFilter, EthPubSub, EthSubscriptionIdProvider};
|
pub use eth::{EthApi, EthApiSpec, EthFilter, EthPubSub, EthSubscriptionIdProvider};
|
||||||
@ -50,8 +50,8 @@ pub use otterscan::OtterscanApi;
|
|||||||
pub use reth::RethApi;
|
pub use reth::RethApi;
|
||||||
pub use rpc::RPCApi;
|
pub use rpc::RPCApi;
|
||||||
pub use trace::TraceApi;
|
pub use trace::TraceApi;
|
||||||
pub use tracing_call::{TracingCallGuard, TracingCallPool};
|
|
||||||
pub use txpool::TxPoolApi;
|
pub use txpool::TxPoolApi;
|
||||||
pub use web3::Web3Api;
|
pub use web3::Web3Api;
|
||||||
|
|
||||||
|
pub mod blocking_pool;
|
||||||
pub mod result;
|
pub mod result;
|
||||||
|
|||||||
@ -5,7 +5,7 @@ use crate::{
|
|||||||
utils::recover_raw_transaction,
|
utils::recover_raw_transaction,
|
||||||
EthTransactions,
|
EthTransactions,
|
||||||
},
|
},
|
||||||
TracingCallGuard,
|
BlockingTaskGuard,
|
||||||
};
|
};
|
||||||
use async_trait::async_trait;
|
use async_trait::async_trait;
|
||||||
use jsonrpsee::core::RpcResult as Result;
|
use jsonrpsee::core::RpcResult as Result;
|
||||||
@ -44,8 +44,8 @@ impl<Provider, Eth> TraceApi<Provider, Eth> {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Create a new instance of the [TraceApi]
|
/// Create a new instance of the [TraceApi]
|
||||||
pub fn new(provider: Provider, eth_api: Eth, tracing_call_guard: TracingCallGuard) -> Self {
|
pub fn new(provider: Provider, eth_api: Eth, blocking_task_guard: BlockingTaskGuard) -> Self {
|
||||||
let inner = Arc::new(TraceApiInner { provider, eth_api, tracing_call_guard });
|
let inner = Arc::new(TraceApiInner { provider, eth_api, blocking_task_guard });
|
||||||
Self { inner }
|
Self { inner }
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -53,7 +53,7 @@ impl<Provider, Eth> TraceApi<Provider, Eth> {
|
|||||||
async fn acquire_trace_permit(
|
async fn acquire_trace_permit(
|
||||||
&self,
|
&self,
|
||||||
) -> std::result::Result<OwnedSemaphorePermit, AcquireError> {
|
) -> std::result::Result<OwnedSemaphorePermit, AcquireError> {
|
||||||
self.inner.tracing_call_guard.clone().acquire_owned().await
|
self.inner.blocking_task_guard.clone().acquire_owned().await
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -557,7 +557,7 @@ struct TraceApiInner<Provider, Eth> {
|
|||||||
/// Access to commonly used code of the `eth` namespace
|
/// Access to commonly used code of the `eth` namespace
|
||||||
eth_api: Eth,
|
eth_api: Eth,
|
||||||
// restrict the number of concurrent calls to `trace_*`
|
// restrict the number of concurrent calls to `trace_*`
|
||||||
tracing_call_guard: TracingCallGuard,
|
blocking_task_guard: BlockingTaskGuard,
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Returns the [TracingInspectorConfig] depending on the enabled [TraceType]s
|
/// Returns the [TracingInspectorConfig] depending on the enabled [TraceType]s
|
||||||
|
|||||||
@ -3,7 +3,7 @@ use crate::{
|
|||||||
StateProvider, StateRootProvider,
|
StateProvider, StateRootProvider,
|
||||||
};
|
};
|
||||||
use reth_interfaces::{provider::ProviderError, RethResult};
|
use reth_interfaces::{provider::ProviderError, RethResult};
|
||||||
use reth_primitives::{Account, Address, BlockNumber, Bytecode, Bytes, B256};
|
use reth_primitives::{trie::AccountProof, Account, Address, BlockNumber, Bytecode, B256};
|
||||||
|
|
||||||
/// A state provider that either resolves to data in a wrapped [`crate::BundleStateWithReceipts`],
|
/// A state provider that either resolves to data in a wrapped [`crate::BundleStateWithReceipts`],
|
||||||
/// or an underlying state provider.
|
/// or an underlying state provider.
|
||||||
@ -92,11 +92,7 @@ impl<SP: StateProvider, BSDP: BundleStateDataProvider> StateProvider
|
|||||||
self.state_provider.bytecode_by_hash(code_hash)
|
self.state_provider.bytecode_by_hash(code_hash)
|
||||||
}
|
}
|
||||||
|
|
||||||
fn proof(
|
fn proof(&self, _address: Address, _keys: &[B256]) -> RethResult<AccountProof> {
|
||||||
&self,
|
|
||||||
_address: Address,
|
|
||||||
_keys: &[B256],
|
|
||||||
) -> RethResult<(Vec<Bytes>, B256, Vec<Vec<Bytes>>)> {
|
|
||||||
Err(ProviderError::StateRootNotAvailableForHistoricalBlock.into())
|
Err(ProviderError::StateRootNotAvailableForHistoricalBlock.into())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@ -12,7 +12,7 @@ use reth_db::{
|
|||||||
};
|
};
|
||||||
use reth_interfaces::RethResult;
|
use reth_interfaces::RethResult;
|
||||||
use reth_primitives::{
|
use reth_primitives::{
|
||||||
Account, Address, BlockNumber, Bytecode, Bytes, StorageKey, StorageValue, B256,
|
trie::AccountProof, Account, Address, BlockNumber, Bytecode, StorageKey, StorageValue, B256,
|
||||||
};
|
};
|
||||||
|
|
||||||
/// State provider for a given block number which takes a tx reference.
|
/// State provider for a given block number which takes a tx reference.
|
||||||
@ -240,11 +240,7 @@ impl<'b, TX: DbTx> StateProvider for HistoricalStateProviderRef<'b, TX> {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Get account and storage proofs.
|
/// Get account and storage proofs.
|
||||||
fn proof(
|
fn proof(&self, _address: Address, _keys: &[B256]) -> RethResult<AccountProof> {
|
||||||
&self,
|
|
||||||
_address: Address,
|
|
||||||
_keys: &[B256],
|
|
||||||
) -> RethResult<(Vec<Bytes>, B256, Vec<Vec<Bytes>>)> {
|
|
||||||
Err(ProviderError::StateRootNotAvailableForHistoricalBlock.into())
|
Err(ProviderError::StateRootNotAvailableForHistoricalBlock.into())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@ -9,7 +9,8 @@ use reth_db::{
|
|||||||
};
|
};
|
||||||
use reth_interfaces::{provider::ProviderError, RethError, RethResult};
|
use reth_interfaces::{provider::ProviderError, RethError, RethResult};
|
||||||
use reth_primitives::{
|
use reth_primitives::{
|
||||||
keccak256, Account, Address, BlockNumber, Bytecode, Bytes, StorageKey, StorageValue, B256,
|
keccak256, trie::AccountProof, Account, Address, BlockNumber, Bytecode, StorageKey,
|
||||||
|
StorageValue, B256,
|
||||||
};
|
};
|
||||||
|
|
||||||
/// State provider over latest state that takes tx reference.
|
/// State provider over latest state that takes tx reference.
|
||||||
@ -84,11 +85,7 @@ impl<'b, TX: DbTx> StateProvider for LatestStateProviderRef<'b, TX> {
|
|||||||
self.db.get::<tables::Bytecodes>(code_hash).map_err(Into::into)
|
self.db.get::<tables::Bytecodes>(code_hash).map_err(Into::into)
|
||||||
}
|
}
|
||||||
|
|
||||||
fn proof(
|
fn proof(&self, address: Address, _keys: &[B256]) -> RethResult<AccountProof> {
|
||||||
&self,
|
|
||||||
address: Address,
|
|
||||||
_keys: &[B256],
|
|
||||||
) -> RethResult<(Vec<Bytes>, B256, Vec<Vec<Bytes>>)> {
|
|
||||||
let _hashed_address = keccak256(address);
|
let _hashed_address = keccak256(address);
|
||||||
let _root = self
|
let _root = self
|
||||||
.db
|
.db
|
||||||
|
|||||||
@ -42,7 +42,7 @@ macro_rules! delegate_provider_impls {
|
|||||||
}
|
}
|
||||||
StateProvider $(where [$($generics)*])?{
|
StateProvider $(where [$($generics)*])?{
|
||||||
fn storage(&self, account: reth_primitives::Address, storage_key: reth_primitives::StorageKey) -> reth_interfaces::RethResult<Option<reth_primitives::StorageValue>>;
|
fn storage(&self, account: reth_primitives::Address, storage_key: reth_primitives::StorageKey) -> reth_interfaces::RethResult<Option<reth_primitives::StorageValue>>;
|
||||||
fn proof(&self, address: reth_primitives::Address, keys: &[reth_primitives::B256]) -> reth_interfaces::RethResult<(Vec<reth_primitives::Bytes>, reth_primitives::B256, Vec<Vec<reth_primitives::Bytes>>)>;
|
fn proof(&self, address: reth_primitives::Address, keys: &[reth_primitives::B256]) -> reth_interfaces::RethResult<reth_primitives::trie::AccountProof>;
|
||||||
fn bytecode_by_hash(&self, code_hash: reth_primitives::B256) -> reth_interfaces::RethResult<Option<reth_primitives::Bytecode>>;
|
fn bytecode_by_hash(&self, code_hash: reth_primitives::B256) -> reth_interfaces::RethResult<Option<reth_primitives::Bytecode>>;
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
|||||||
@ -10,9 +10,9 @@ use parking_lot::Mutex;
|
|||||||
use reth_db::models::StoredBlockBodyIndices;
|
use reth_db::models::StoredBlockBodyIndices;
|
||||||
use reth_interfaces::{provider::ProviderError, RethResult};
|
use reth_interfaces::{provider::ProviderError, RethResult};
|
||||||
use reth_primitives::{
|
use reth_primitives::{
|
||||||
keccak256, Account, Address, Block, BlockHash, BlockHashOrNumber, BlockId, BlockNumber,
|
keccak256, trie::AccountProof, Account, Address, Block, BlockHash, BlockHashOrNumber, BlockId,
|
||||||
BlockWithSenders, Bytecode, Bytes, ChainInfo, ChainSpec, Header, Receipt, SealedBlock,
|
BlockNumber, BlockWithSenders, Bytecode, Bytes, ChainInfo, ChainSpec, Header, Receipt,
|
||||||
SealedHeader, StorageKey, StorageValue, TransactionMeta, TransactionSigned,
|
SealedBlock, SealedHeader, StorageKey, StorageValue, TransactionMeta, TransactionSigned,
|
||||||
TransactionSignedNoHash, TxHash, TxNumber, B256, U256,
|
TransactionSignedNoHash, TxHash, TxNumber, B256, U256,
|
||||||
};
|
};
|
||||||
use revm::primitives::{BlockEnv, CfgEnv};
|
use revm::primitives::{BlockEnv, CfgEnv};
|
||||||
@ -507,11 +507,7 @@ impl StateProvider for MockEthProvider {
|
|||||||
}))
|
}))
|
||||||
}
|
}
|
||||||
|
|
||||||
fn proof(
|
fn proof(&self, _address: Address, _keys: &[B256]) -> RethResult<AccountProof> {
|
||||||
&self,
|
|
||||||
_address: Address,
|
|
||||||
_keys: &[B256],
|
|
||||||
) -> RethResult<(Vec<Bytes>, B256, Vec<Vec<Bytes>>)> {
|
|
||||||
todo!()
|
todo!()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@ -10,10 +10,11 @@ use reth_db::models::{AccountBeforeTx, StoredBlockBodyIndices};
|
|||||||
use reth_interfaces::RethResult;
|
use reth_interfaces::RethResult;
|
||||||
use reth_primitives::{
|
use reth_primitives::{
|
||||||
stage::{StageCheckpoint, StageId},
|
stage::{StageCheckpoint, StageId},
|
||||||
Account, Address, Block, BlockHash, BlockHashOrNumber, BlockId, BlockNumber, Bytecode, Bytes,
|
trie::AccountProof,
|
||||||
|
Account, Address, Block, BlockHash, BlockHashOrNumber, BlockId, BlockNumber, Bytecode,
|
||||||
ChainInfo, ChainSpec, Header, PruneCheckpoint, PruneSegment, Receipt, SealedBlock,
|
ChainInfo, ChainSpec, Header, PruneCheckpoint, PruneSegment, Receipt, SealedBlock,
|
||||||
SealedHeader, StorageKey, StorageValue, TransactionMeta, TransactionSigned,
|
SealedHeader, StorageKey, StorageValue, TransactionMeta, TransactionSigned,
|
||||||
TransactionSignedNoHash, TxHash, TxNumber, B256, KECCAK_EMPTY, MAINNET, U256,
|
TransactionSignedNoHash, TxHash, TxNumber, B256, MAINNET, U256,
|
||||||
};
|
};
|
||||||
use revm::primitives::{BlockEnv, CfgEnv};
|
use revm::primitives::{BlockEnv, CfgEnv};
|
||||||
use std::{
|
use std::{
|
||||||
@ -278,12 +279,8 @@ impl StateProvider for NoopProvider {
|
|||||||
Ok(None)
|
Ok(None)
|
||||||
}
|
}
|
||||||
|
|
||||||
fn proof(
|
fn proof(&self, _address: Address, _keys: &[B256]) -> RethResult<AccountProof> {
|
||||||
&self,
|
Ok(AccountProof::default())
|
||||||
_address: Address,
|
|
||||||
_keys: &[B256],
|
|
||||||
) -> RethResult<(Vec<Bytes>, B256, Vec<Vec<Bytes>>)> {
|
|
||||||
Ok((vec![], KECCAK_EMPTY, vec![]))
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -3,8 +3,8 @@ use crate::{BlockHashReader, BlockIdReader, BundleStateWithReceipts};
|
|||||||
use auto_impl::auto_impl;
|
use auto_impl::auto_impl;
|
||||||
use reth_interfaces::{provider::ProviderError, RethResult};
|
use reth_interfaces::{provider::ProviderError, RethResult};
|
||||||
use reth_primitives::{
|
use reth_primitives::{
|
||||||
Address, BlockHash, BlockId, BlockNumHash, BlockNumber, BlockNumberOrTag, Bytecode, Bytes,
|
trie::AccountProof, Address, BlockHash, BlockId, BlockNumHash, BlockNumber, BlockNumberOrTag,
|
||||||
StorageKey, StorageValue, B256, KECCAK_EMPTY, U256,
|
Bytecode, StorageKey, StorageValue, B256, KECCAK_EMPTY, U256,
|
||||||
};
|
};
|
||||||
|
|
||||||
/// Type alias of boxed [StateProvider].
|
/// Type alias of boxed [StateProvider].
|
||||||
@ -24,11 +24,7 @@ pub trait StateProvider: BlockHashReader + AccountReader + StateRootProvider + S
|
|||||||
fn bytecode_by_hash(&self, code_hash: B256) -> RethResult<Option<Bytecode>>;
|
fn bytecode_by_hash(&self, code_hash: B256) -> RethResult<Option<Bytecode>>;
|
||||||
|
|
||||||
/// Get account and storage proofs.
|
/// Get account and storage proofs.
|
||||||
fn proof(
|
fn proof(&self, address: Address, keys: &[B256]) -> RethResult<AccountProof>;
|
||||||
&self,
|
|
||||||
address: Address,
|
|
||||||
keys: &[B256],
|
|
||||||
) -> RethResult<(Vec<Bytes>, B256, Vec<Vec<Bytes>>)>;
|
|
||||||
|
|
||||||
/// Get account code by its address.
|
/// Get account code by its address.
|
||||||
///
|
///
|
||||||
|
|||||||
@ -17,7 +17,7 @@ use reth::{
|
|||||||
Cli,
|
Cli,
|
||||||
},
|
},
|
||||||
primitives::{Address, IntoRecoveredTransaction},
|
primitives::{Address, IntoRecoveredTransaction},
|
||||||
rpc::{compat::transaction_to_call_request, types::trace::parity::TraceType},
|
rpc::{compat::transaction::transaction_to_call_request, types::trace::parity::TraceType},
|
||||||
tasks::TaskSpawner,
|
tasks::TaskSpawner,
|
||||||
transaction_pool::TransactionPool,
|
transaction_pool::TransactionPool,
|
||||||
};
|
};
|
||||||
|
|||||||
Reference in New Issue
Block a user