mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 19:09:54 +00:00
fix(op): ensure EthApiServer helper trait method default impls, call OpEthApi overrides (#9879)
This commit is contained in:
7
Cargo.lock
generated
7
Cargo.lock
generated
@ -7558,6 +7558,7 @@ version = "1.0.3"
|
||||
dependencies = [
|
||||
"alloy-primitives",
|
||||
"alloy-rpc-types-admin",
|
||||
"auto_impl",
|
||||
"enr",
|
||||
"reth-eth-wire",
|
||||
"reth-network-peers",
|
||||
@ -7980,13 +7981,14 @@ name = "reth-optimism-rpc"
|
||||
version = "1.0.3"
|
||||
dependencies = [
|
||||
"alloy-primitives",
|
||||
"derive_more",
|
||||
"jsonrpsee",
|
||||
"jsonrpsee-types",
|
||||
"parking_lot 0.12.3",
|
||||
"reth-chainspec",
|
||||
"reth-errors",
|
||||
"reth-evm",
|
||||
"reth-evm-optimism",
|
||||
"reth-network",
|
||||
"reth-network-api",
|
||||
"reth-node-api",
|
||||
"reth-primitives",
|
||||
"reth-provider",
|
||||
@ -8407,6 +8409,7 @@ dependencies = [
|
||||
"reth-errors",
|
||||
"reth-evm",
|
||||
"reth-execution-types",
|
||||
"reth-network-api",
|
||||
"reth-primitives",
|
||||
"reth-provider",
|
||||
"reth-revm",
|
||||
|
||||
@ -27,6 +27,7 @@ enr = { workspace = true, default-features = false, features = ["rust-secp256k1"
|
||||
thiserror.workspace = true
|
||||
serde = { workspace = true, features = ["derive"], optional = true }
|
||||
tokio = { workspace = true, features = ["sync"] }
|
||||
auto_impl.workspace = true
|
||||
|
||||
[features]
|
||||
default = ["serde"]
|
||||
|
||||
@ -32,6 +32,7 @@ pub mod reputation;
|
||||
pub mod noop;
|
||||
|
||||
/// Provides general purpose information about the network.
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait NetworkInfo: Send + Sync {
|
||||
/// Returns the [`SocketAddr`] that listens for incoming connections.
|
||||
fn local_addr(&self) -> SocketAddr;
|
||||
|
||||
@ -19,7 +19,6 @@ use reth_optimism_consensus::OptimismBeaconConsensus;
|
||||
use reth_optimism_rpc::OpEthApi;
|
||||
use reth_payload_builder::{PayloadBuilderHandle, PayloadBuilderService};
|
||||
use reth_provider::CanonStateSubscriptions;
|
||||
use reth_rpc::EthApi;
|
||||
use reth_tracing::tracing::{debug, info};
|
||||
use reth_transaction_pool::{
|
||||
blobstore::DiskFileBlobStore, CoinbaseTipOrdering, TransactionPool,
|
||||
@ -105,7 +104,7 @@ impl NodeTypes for OptimismNode {
|
||||
pub struct OptimismAddOns;
|
||||
|
||||
impl<N: FullNodeComponents> NodeAddOns<N> for OptimismAddOns {
|
||||
type EthApi = OpEthApi<EthApi<N::Provider, N::Pool, NetworkHandle, N::Evm>>;
|
||||
type EthApi = OpEthApi<N>;
|
||||
}
|
||||
|
||||
/// A regular optimism evm and executor builder.
|
||||
|
||||
@ -13,8 +13,6 @@ workspace = true
|
||||
|
||||
[dependencies]
|
||||
# reth
|
||||
reth-chainspec.workspace = true
|
||||
reth-errors.workspace = true
|
||||
reth-evm-optimism.workspace = true
|
||||
reth-evm.workspace = true
|
||||
reth-primitives.workspace = true
|
||||
@ -27,6 +25,8 @@ reth-tasks = { workspace = true, features = ["rayon"] }
|
||||
reth-transaction-pool.workspace = true
|
||||
reth-rpc.workspace = true
|
||||
reth-node-api.workspace = true
|
||||
reth-network-api.workspace = true
|
||||
reth-network.workspace = true
|
||||
|
||||
# ethereum
|
||||
alloy-primitives.workspace = true
|
||||
@ -43,6 +43,7 @@ jsonrpsee-types.workspace = true
|
||||
# misc
|
||||
thiserror.workspace = true
|
||||
serde = { workspace = true, features = ["derive"] }
|
||||
derive_more.workspace = true
|
||||
|
||||
[dev-dependencies]
|
||||
serde_json.workspace = true
|
||||
@ -55,7 +56,6 @@ client = [
|
||||
]
|
||||
|
||||
optimism = [
|
||||
"reth-chainspec/optimism",
|
||||
"reth-evm-optimism/optimism",
|
||||
"reth-primitives/optimism",
|
||||
"reth-provider/optimism",
|
||||
|
||||
@ -1,22 +1,29 @@
|
||||
//! Loads and formats OP block RPC response.
|
||||
|
||||
use reth_node_api::FullNodeComponents;
|
||||
use reth_primitives::TransactionMeta;
|
||||
use reth_provider::{BlockReaderIdExt, HeaderProvider};
|
||||
use reth_rpc_eth_api::{
|
||||
helpers::{EthApiSpec, EthBlocks, LoadBlock, LoadReceipt, LoadTransaction},
|
||||
helpers::{
|
||||
EthApiSpec, EthBlocks, LoadBlock, LoadPendingBlock, LoadReceipt, LoadTransaction,
|
||||
SpawnBlocking,
|
||||
},
|
||||
FromEthApiError,
|
||||
};
|
||||
use reth_rpc_eth_types::{EthStateCache, ReceiptBuilder};
|
||||
use reth_rpc_types::{AnyTransactionReceipt, BlockId};
|
||||
|
||||
use crate::{op_receipt_fields, OpEthApi};
|
||||
use crate::{op_receipt_fields, OpEthApi, OpEthApiError};
|
||||
|
||||
impl<Eth> EthBlocks for OpEthApi<Eth>
|
||||
impl<N> EthBlocks for OpEthApi<N>
|
||||
where
|
||||
Eth: EthBlocks + EthApiSpec + LoadTransaction,
|
||||
Self: LoadBlock + EthApiSpec + LoadTransaction,
|
||||
Self::Error: From<OpEthApiError>,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
#[inline]
|
||||
fn provider(&self) -> impl HeaderProvider {
|
||||
EthBlocks::provider(&self.inner)
|
||||
self.inner.provider()
|
||||
}
|
||||
|
||||
async fn block_receipts(
|
||||
@ -69,11 +76,17 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<Eth: LoadBlock> LoadBlock for OpEthApi<Eth> {
|
||||
impl<N> LoadBlock for OpEthApi<N>
|
||||
where
|
||||
Self: LoadPendingBlock + SpawnBlocking,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
#[inline]
|
||||
fn provider(&self) -> impl BlockReaderIdExt {
|
||||
LoadBlock::provider(&self.inner)
|
||||
self.inner.provider()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn cache(&self) -> &EthStateCache {
|
||||
self.inner.cache()
|
||||
}
|
||||
|
||||
@ -1,28 +1,37 @@
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_node_api::FullNodeComponents;
|
||||
use reth_primitives::{
|
||||
revm_primitives::{BlockEnv, OptimismFields, TxEnv},
|
||||
Bytes,
|
||||
Bytes, TxKind, U256,
|
||||
};
|
||||
use reth_rpc_eth_api::{
|
||||
helpers::{Call, EthCall},
|
||||
EthApiTypes, FromEthApiError,
|
||||
helpers::{Call, EthCall, LoadState, SpawnBlocking},
|
||||
FromEthApiError, IntoEthApiError,
|
||||
};
|
||||
use reth_rpc_eth_types::EthApiError;
|
||||
use reth_rpc_eth_types::{revm_utils::CallFees, RpcInvalidTransactionError};
|
||||
use reth_rpc_types::TransactionRequest;
|
||||
|
||||
use crate::OpEthApi;
|
||||
use crate::{OpEthApi, OpEthApiError};
|
||||
|
||||
impl<Eth: EthCall> EthCall for OpEthApi<Eth> where EthApiError: From<Eth::Error> {}
|
||||
|
||||
impl<Eth> Call for OpEthApi<Eth>
|
||||
impl<N> EthCall for OpEthApi<N>
|
||||
where
|
||||
Eth: Call + EthApiTypes,
|
||||
EthApiError: From<Eth::Error>,
|
||||
Self: Call,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
}
|
||||
|
||||
impl<N> Call for OpEthApi<N>
|
||||
where
|
||||
Self: LoadState + SpawnBlocking,
|
||||
Self::Error: From<OpEthApiError>,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
#[inline]
|
||||
fn call_gas_limit(&self) -> u64 {
|
||||
self.inner.call_gas_limit()
|
||||
self.inner.gas_cap()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn evm_config(&self) -> &impl ConfigureEvm {
|
||||
self.inner.evm_config()
|
||||
}
|
||||
@ -32,10 +41,68 @@ where
|
||||
block_env: &BlockEnv,
|
||||
request: TransactionRequest,
|
||||
) -> Result<TxEnv, Self::Error> {
|
||||
let mut env =
|
||||
self.inner.create_txn_env(block_env, request).map_err(Self::Error::from_eth_err)?;
|
||||
// Ensure that if versioned hashes are set, they're not empty
|
||||
if request.blob_versioned_hashes.as_ref().map_or(false, |hashes| hashes.is_empty()) {
|
||||
return Err(RpcInvalidTransactionError::BlobTransactionMissingBlobHashes.into_eth_err())
|
||||
}
|
||||
|
||||
env.optimism = OptimismFields { enveloped_tx: Some(Bytes::new()), ..Default::default() };
|
||||
let TransactionRequest {
|
||||
from,
|
||||
to,
|
||||
gas_price,
|
||||
max_fee_per_gas,
|
||||
max_priority_fee_per_gas,
|
||||
gas,
|
||||
value,
|
||||
input,
|
||||
nonce,
|
||||
access_list,
|
||||
chain_id,
|
||||
blob_versioned_hashes,
|
||||
max_fee_per_blob_gas,
|
||||
// authorization_list,
|
||||
..
|
||||
} = request;
|
||||
|
||||
let CallFees { max_priority_fee_per_gas, gas_price, max_fee_per_blob_gas } =
|
||||
CallFees::ensure_fees(
|
||||
gas_price.map(U256::from),
|
||||
max_fee_per_gas.map(U256::from),
|
||||
max_priority_fee_per_gas.map(U256::from),
|
||||
block_env.basefee,
|
||||
blob_versioned_hashes.as_deref(),
|
||||
max_fee_per_blob_gas.map(U256::from),
|
||||
block_env.get_blob_gasprice().map(U256::from),
|
||||
)?;
|
||||
|
||||
let gas_limit = gas.unwrap_or_else(|| block_env.gas_limit.min(U256::from(u64::MAX)).to());
|
||||
|
||||
#[allow(clippy::needless_update)]
|
||||
let env = TxEnv {
|
||||
gas_limit: gas_limit
|
||||
.try_into()
|
||||
.map_err(|_| RpcInvalidTransactionError::GasUintOverflow)
|
||||
.map_err(Self::Error::from_eth_err)?,
|
||||
nonce,
|
||||
caller: from.unwrap_or_default(),
|
||||
gas_price,
|
||||
gas_priority_fee: max_priority_fee_per_gas,
|
||||
transact_to: to.unwrap_or(TxKind::Create),
|
||||
value: value.unwrap_or_default(),
|
||||
data: input
|
||||
.try_into_unique_input()
|
||||
.map_err(Self::Error::from_eth_err)?
|
||||
.unwrap_or_default(),
|
||||
chain_id,
|
||||
access_list: access_list.unwrap_or_default().into(),
|
||||
// EIP-4844 fields
|
||||
blob_hashes: blob_versioned_hashes.unwrap_or_default(),
|
||||
max_fee_per_blob_gas,
|
||||
// EIP-7702 fields
|
||||
// authorization_list: TODO
|
||||
authorization_list: Default::default(),
|
||||
optimism: OptimismFields { enveloped_tx: Some(Bytes::new()), ..Default::default() },
|
||||
};
|
||||
|
||||
Ok(env)
|
||||
}
|
||||
|
||||
@ -7,30 +7,53 @@ mod block;
|
||||
mod call;
|
||||
mod pending_block;
|
||||
|
||||
use std::{future::Future, sync::Arc};
|
||||
use std::{fmt, sync::Arc};
|
||||
|
||||
use alloy_primitives::{Address, U64};
|
||||
use reth_chainspec::{ChainInfo, ChainSpec};
|
||||
use reth_errors::RethResult;
|
||||
use alloy_primitives::U256;
|
||||
use derive_more::Deref;
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_node_api::{BuilderProvider, FullNodeComponents};
|
||||
use reth_provider::{BlockReaderIdExt, ChainSpecProvider, HeaderProvider, StateProviderFactory};
|
||||
use reth_rpc::eth::DevSigner;
|
||||
use reth_network::NetworkHandle;
|
||||
use reth_network_api::NetworkInfo;
|
||||
use reth_node_api::{BuilderProvider, FullNodeComponents, FullNodeTypes};
|
||||
use reth_provider::{
|
||||
BlockIdReader, BlockNumReader, BlockReaderIdExt, ChainSpecProvider, HeaderProvider,
|
||||
StateProviderFactory,
|
||||
};
|
||||
use reth_rpc::eth::{core::EthApiInner, DevSigner};
|
||||
use reth_rpc_eth_api::{
|
||||
helpers::{
|
||||
AddDevSigners, EthApiSpec, EthFees, EthSigner, EthState, LoadFee, LoadState, SpawnBlocking,
|
||||
Trace, UpdateRawTxForwarder,
|
||||
AddDevSigners, EthApiSpec, EthFees, EthState, LoadBlock, LoadFee, LoadState, SpawnBlocking,
|
||||
Trace,
|
||||
},
|
||||
EthApiTypes, RawTransactionForwarder,
|
||||
EthApiTypes,
|
||||
};
|
||||
use reth_rpc_eth_types::{EthStateCache, FeeHistoryCache, GasPriceOracle};
|
||||
use reth_tasks::{
|
||||
pool::{BlockingTaskGuard, BlockingTaskPool},
|
||||
TaskExecutor, TaskSpawner,
|
||||
};
|
||||
use reth_rpc_eth_types::EthStateCache;
|
||||
use reth_rpc_types::SyncStatus;
|
||||
use reth_tasks::{pool::BlockingTaskPool, TaskSpawner};
|
||||
use reth_transaction_pool::TransactionPool;
|
||||
use tokio::sync::{AcquireError, OwnedSemaphorePermit};
|
||||
|
||||
use crate::OpEthApiError;
|
||||
|
||||
/// Adapter for [`EthApiInner`], which holds all the data required to serve core `eth_` API.
|
||||
pub type EthApiNodeBackend<N> = EthApiInner<
|
||||
<N as FullNodeTypes>::Provider,
|
||||
<N as FullNodeComponents>::Pool,
|
||||
NetworkHandle,
|
||||
<N as FullNodeComponents>::Evm,
|
||||
>;
|
||||
|
||||
/// Adapter for [`EthApiBuilderCtx`].
|
||||
pub type EthApiBuilderCtx<N> = reth_rpc_eth_types::EthApiBuilderCtx<
|
||||
<N as FullNodeTypes>::Provider,
|
||||
<N as FullNodeComponents>::Pool,
|
||||
<N as FullNodeComponents>::Evm,
|
||||
NetworkHandle,
|
||||
TaskExecutor,
|
||||
<N as FullNodeTypes>::Provider,
|
||||
>;
|
||||
|
||||
/// OP-Reth `Eth` API implementation.
|
||||
///
|
||||
/// This type provides the functionality for handling `eth_` related requests.
|
||||
@ -41,148 +64,187 @@ use crate::OpEthApiError;
|
||||
///
|
||||
/// This type implements the [`FullEthApi`](reth_rpc_eth_api::helpers::FullEthApi) by implemented
|
||||
/// all the `Eth` helper traits and prerequisite traits.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct OpEthApi<Eth> {
|
||||
inner: Eth,
|
||||
#[derive(Clone, Deref)]
|
||||
pub struct OpEthApi<N: FullNodeComponents> {
|
||||
inner: Arc<EthApiNodeBackend<N>>,
|
||||
}
|
||||
|
||||
impl<Eth> OpEthApi<Eth> {
|
||||
/// Creates a new `OpEthApi` from the provided `Eth` implementation.
|
||||
pub const fn new(inner: Eth) -> Self {
|
||||
Self { inner }
|
||||
impl<N: FullNodeComponents> OpEthApi<N> {
|
||||
/// Creates a new instance for given context.
|
||||
#[allow(clippy::type_complexity)]
|
||||
pub fn with_spawner(ctx: &EthApiBuilderCtx<N>) -> Self {
|
||||
let blocking_task_pool =
|
||||
BlockingTaskPool::build().expect("failed to build blocking task pool");
|
||||
|
||||
let inner = EthApiInner::new(
|
||||
ctx.provider.clone(),
|
||||
ctx.pool.clone(),
|
||||
ctx.network.clone(),
|
||||
ctx.cache.clone(),
|
||||
ctx.new_gas_price_oracle(),
|
||||
ctx.config.rpc_gas_cap,
|
||||
ctx.config.eth_proof_window,
|
||||
blocking_task_pool,
|
||||
ctx.new_fee_history_cache(),
|
||||
ctx.evm_config.clone(),
|
||||
ctx.executor.clone(),
|
||||
None,
|
||||
ctx.config.proof_permits,
|
||||
);
|
||||
|
||||
Self { inner: Arc::new(inner) }
|
||||
}
|
||||
}
|
||||
|
||||
impl<Eth> EthApiTypes for OpEthApi<Eth>
|
||||
impl<N> EthApiTypes for OpEthApi<N>
|
||||
where
|
||||
Eth: Send + Sync,
|
||||
Self: Send + Sync,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
type Error = OpEthApiError;
|
||||
}
|
||||
|
||||
impl<Eth: EthApiSpec> EthApiSpec for OpEthApi<Eth> {
|
||||
fn protocol_version(&self) -> impl Future<Output = RethResult<U64>> + Send {
|
||||
self.inner.protocol_version()
|
||||
impl<N> EthApiSpec for OpEthApi<N>
|
||||
where
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
#[inline]
|
||||
fn provider(&self) -> impl ChainSpecProvider + BlockNumReader {
|
||||
self.inner.provider()
|
||||
}
|
||||
|
||||
fn chain_id(&self) -> U64 {
|
||||
self.inner.chain_id()
|
||||
#[inline]
|
||||
fn network(&self) -> impl NetworkInfo {
|
||||
self.inner.network()
|
||||
}
|
||||
|
||||
fn chain_info(&self) -> RethResult<ChainInfo> {
|
||||
self.inner.chain_info()
|
||||
#[inline]
|
||||
fn starting_block(&self) -> U256 {
|
||||
self.inner.starting_block()
|
||||
}
|
||||
|
||||
fn accounts(&self) -> Vec<Address> {
|
||||
self.inner.accounts()
|
||||
}
|
||||
|
||||
fn is_syncing(&self) -> bool {
|
||||
self.inner.is_syncing()
|
||||
}
|
||||
|
||||
fn sync_status(&self) -> RethResult<SyncStatus> {
|
||||
self.inner.sync_status()
|
||||
}
|
||||
|
||||
fn chain_spec(&self) -> Arc<ChainSpec> {
|
||||
self.inner.chain_spec()
|
||||
#[inline]
|
||||
fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn reth_rpc_eth_api::helpers::EthSigner>>> {
|
||||
self.inner.signers()
|
||||
}
|
||||
}
|
||||
|
||||
impl<Eth: SpawnBlocking> SpawnBlocking for OpEthApi<Eth> {
|
||||
impl<N> SpawnBlocking for OpEthApi<N>
|
||||
where
|
||||
Self: Send + Sync + Clone + 'static,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
#[inline]
|
||||
fn io_task_spawner(&self) -> impl TaskSpawner {
|
||||
self.inner.io_task_spawner()
|
||||
self.inner.task_spawner()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn tracing_task_pool(&self) -> &BlockingTaskPool {
|
||||
self.inner.tracing_task_pool()
|
||||
self.inner.blocking_task_pool()
|
||||
}
|
||||
|
||||
fn acquire_owned(
|
||||
&self,
|
||||
) -> impl Future<Output = Result<OwnedSemaphorePermit, AcquireError>> + Send {
|
||||
self.inner.acquire_owned()
|
||||
}
|
||||
|
||||
fn acquire_many_owned(
|
||||
&self,
|
||||
n: u32,
|
||||
) -> impl Future<Output = Result<OwnedSemaphorePermit, AcquireError>> + Send {
|
||||
self.inner.acquire_many_owned(n)
|
||||
#[inline]
|
||||
fn tracing_task_guard(&self) -> &BlockingTaskGuard {
|
||||
self.inner.blocking_task_guard()
|
||||
}
|
||||
}
|
||||
|
||||
impl<Eth: LoadFee> LoadFee for OpEthApi<Eth> {
|
||||
fn provider(&self) -> impl reth_provider::BlockIdReader + HeaderProvider + ChainSpecProvider {
|
||||
LoadFee::provider(&self.inner)
|
||||
impl<N> LoadFee for OpEthApi<N>
|
||||
where
|
||||
Self: LoadBlock,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
#[inline]
|
||||
fn provider(&self) -> impl BlockIdReader + HeaderProvider + ChainSpecProvider {
|
||||
self.inner.provider()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn cache(&self) -> &EthStateCache {
|
||||
LoadFee::cache(&self.inner)
|
||||
self.inner.cache()
|
||||
}
|
||||
|
||||
fn gas_oracle(&self) -> &reth_rpc_eth_types::GasPriceOracle<impl BlockReaderIdExt> {
|
||||
#[inline]
|
||||
fn gas_oracle(&self) -> &GasPriceOracle<impl BlockReaderIdExt> {
|
||||
self.inner.gas_oracle()
|
||||
}
|
||||
|
||||
fn fee_history_cache(&self) -> &reth_rpc_eth_types::FeeHistoryCache {
|
||||
#[inline]
|
||||
fn fee_history_cache(&self) -> &FeeHistoryCache {
|
||||
self.inner.fee_history_cache()
|
||||
}
|
||||
}
|
||||
|
||||
impl<Eth: LoadState> LoadState for OpEthApi<Eth> {
|
||||
impl<N> LoadState for OpEthApi<N>
|
||||
where
|
||||
Self: Send + Sync,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
#[inline]
|
||||
fn provider(&self) -> impl StateProviderFactory + ChainSpecProvider {
|
||||
LoadState::provider(&self.inner)
|
||||
self.inner.provider()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn cache(&self) -> &EthStateCache {
|
||||
LoadState::cache(&self.inner)
|
||||
self.inner.cache()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn pool(&self) -> impl TransactionPool {
|
||||
LoadState::pool(&self.inner)
|
||||
self.inner.pool()
|
||||
}
|
||||
}
|
||||
|
||||
impl<Eth: EthState> EthState for OpEthApi<Eth> {
|
||||
impl<N> EthState for OpEthApi<N>
|
||||
where
|
||||
Self: LoadState + SpawnBlocking,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
#[inline]
|
||||
fn max_proof_window(&self) -> u64 {
|
||||
self.inner.max_proof_window()
|
||||
self.inner.eth_proof_window()
|
||||
}
|
||||
}
|
||||
|
||||
impl<Eth: EthFees> EthFees for OpEthApi<Eth> {}
|
||||
impl<N> EthFees for OpEthApi<N>
|
||||
where
|
||||
Self: LoadFee,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
}
|
||||
|
||||
impl<Eth: Trace> Trace for OpEthApi<Eth> {
|
||||
impl<N> Trace for OpEthApi<N>
|
||||
where
|
||||
Self: LoadState,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
#[inline]
|
||||
fn evm_config(&self) -> &impl ConfigureEvm {
|
||||
self.inner.evm_config()
|
||||
}
|
||||
}
|
||||
|
||||
impl<Eth: AddDevSigners> AddDevSigners for OpEthApi<Eth> {
|
||||
fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn EthSigner>>> {
|
||||
self.inner.signers()
|
||||
}
|
||||
|
||||
impl<N: FullNodeComponents> AddDevSigners for OpEthApi<N> {
|
||||
fn with_dev_accounts(&self) {
|
||||
*self.signers().write() = DevSigner::random_signers(20)
|
||||
}
|
||||
}
|
||||
|
||||
impl<Eth: UpdateRawTxForwarder> UpdateRawTxForwarder for OpEthApi<Eth> {
|
||||
fn set_eth_raw_transaction_forwarder(&self, forwarder: Arc<dyn RawTransactionForwarder>) {
|
||||
self.inner.set_eth_raw_transaction_forwarder(forwarder);
|
||||
}
|
||||
}
|
||||
|
||||
impl<N, Eth> BuilderProvider<N> for OpEthApi<Eth>
|
||||
impl<N> BuilderProvider<N> for OpEthApi<N>
|
||||
where
|
||||
Eth: BuilderProvider<N>,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
type Ctx<'a> = <Eth as BuilderProvider<N>>::Ctx<'a>;
|
||||
type Ctx<'a> = &'a EthApiBuilderCtx<N>;
|
||||
|
||||
fn builder() -> Box<dyn for<'a> Fn(Self::Ctx<'a>) -> Self + Send> {
|
||||
Box::new(|ctx| Self { inner: Eth::builder()(ctx) })
|
||||
Box::new(|ctx| Self::with_spawner(ctx))
|
||||
}
|
||||
}
|
||||
|
||||
impl<N: FullNodeComponents> fmt::Debug for OpEthApi<N> {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("OpEthApi").finish_non_exhaustive()
|
||||
}
|
||||
}
|
||||
|
||||
@ -1,19 +1,21 @@
|
||||
//! Loads OP pending block for a RPC response.
|
||||
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_node_api::FullNodeComponents;
|
||||
use reth_primitives::{revm_primitives::BlockEnv, BlockNumber, B256};
|
||||
use reth_provider::{
|
||||
BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, ExecutionOutcome, StateProviderFactory,
|
||||
};
|
||||
use reth_rpc_eth_api::helpers::LoadPendingBlock;
|
||||
use reth_rpc_eth_api::helpers::{LoadPendingBlock, SpawnBlocking};
|
||||
use reth_rpc_eth_types::PendingBlock;
|
||||
use reth_transaction_pool::TransactionPool;
|
||||
|
||||
use crate::OpEthApi;
|
||||
|
||||
impl<Eth> LoadPendingBlock for OpEthApi<Eth>
|
||||
impl<N> LoadPendingBlock for OpEthApi<N>
|
||||
where
|
||||
Eth: LoadPendingBlock,
|
||||
Self: SpawnBlocking,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
#[inline]
|
||||
fn provider(
|
||||
|
||||
@ -1,5 +1,6 @@
|
||||
//! Loads and formats OP receipt RPC response.
|
||||
|
||||
use reth_node_api::FullNodeComponents;
|
||||
use reth_primitives::{Receipt, TransactionMeta, TransactionSigned};
|
||||
use reth_rpc_eth_api::{
|
||||
helpers::{EthApiSpec, LoadReceipt, LoadTransaction},
|
||||
@ -8,15 +9,17 @@ use reth_rpc_eth_api::{
|
||||
use reth_rpc_eth_types::{EthApiError, EthStateCache, ReceiptBuilder};
|
||||
use reth_rpc_types::{AnyTransactionReceipt, OptimismTransactionReceiptFields};
|
||||
|
||||
use crate::{OpEthApi, OptimismTxMeta};
|
||||
use crate::{OpEthApi, OpEthApiError, OptimismTxMeta};
|
||||
|
||||
impl<Eth> LoadReceipt for OpEthApi<Eth>
|
||||
impl<N> LoadReceipt for OpEthApi<N>
|
||||
where
|
||||
Eth: LoadReceipt + EthApiSpec + LoadTransaction,
|
||||
Self: EthApiSpec + LoadTransaction,
|
||||
Self::Error: From<OpEthApiError>,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
#[inline]
|
||||
fn cache(&self) -> &EthStateCache {
|
||||
LoadReceipt::cache(&self.inner)
|
||||
self.inner.cache()
|
||||
}
|
||||
|
||||
async fn build_transaction_receipt(
|
||||
|
||||
@ -3,10 +3,11 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use reth_evm_optimism::RethL1BlockInfo;
|
||||
use reth_node_api::FullNodeComponents;
|
||||
use reth_primitives::TransactionSigned;
|
||||
use reth_provider::{BlockReaderIdExt, TransactionsProvider};
|
||||
use reth_rpc_eth_api::{
|
||||
helpers::{EthApiSpec, EthSigner, EthTransactions, LoadTransaction},
|
||||
helpers::{EthApiSpec, EthSigner, EthTransactions, LoadTransaction, SpawnBlocking},
|
||||
EthApiTypes, RawTransactionForwarder,
|
||||
};
|
||||
use reth_rpc_eth_types::EthStateCache;
|
||||
@ -14,9 +15,13 @@ use revm::L1BlockInfo;
|
||||
|
||||
use crate::{OpEthApi, OpEthApiError};
|
||||
|
||||
impl<Eth: EthTransactions> EthTransactions for OpEthApi<Eth> {
|
||||
impl<N> EthTransactions for OpEthApi<N>
|
||||
where
|
||||
Self: LoadTransaction,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
fn provider(&self) -> impl BlockReaderIdExt {
|
||||
EthTransactions::provider(&self.inner)
|
||||
self.inner.provider()
|
||||
}
|
||||
|
||||
fn raw_tx_forwarder(&self) -> Option<Arc<dyn RawTransactionForwarder>> {
|
||||
@ -28,19 +33,23 @@ impl<Eth: EthTransactions> EthTransactions for OpEthApi<Eth> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<Eth: LoadTransaction> LoadTransaction for OpEthApi<Eth> {
|
||||
type Pool = Eth::Pool;
|
||||
impl<N> LoadTransaction for OpEthApi<N>
|
||||
where
|
||||
Self: SpawnBlocking,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
type Pool = N::Pool;
|
||||
|
||||
fn provider(&self) -> impl TransactionsProvider {
|
||||
LoadTransaction::provider(&self.inner)
|
||||
self.inner.provider()
|
||||
}
|
||||
|
||||
fn cache(&self) -> &EthStateCache {
|
||||
LoadTransaction::cache(&self.inner)
|
||||
self.inner.cache()
|
||||
}
|
||||
|
||||
fn pool(&self) -> &Self::Pool {
|
||||
LoadTransaction::pool(&self.inner)
|
||||
self.inner.pool()
|
||||
}
|
||||
}
|
||||
|
||||
@ -66,9 +75,11 @@ impl OptimismTxMeta {
|
||||
}
|
||||
}
|
||||
|
||||
impl<Eth> OpEthApi<Eth>
|
||||
impl<N> OpEthApi<N>
|
||||
where
|
||||
Eth: EthApiSpec + LoadTransaction,
|
||||
Self: EthApiSpec + LoadTransaction,
|
||||
<Self as EthApiTypes>::Error: From<OpEthApiError>,
|
||||
N: FullNodeComponents,
|
||||
{
|
||||
/// Builds [`OptimismTxMeta`] object using the provided [`TransactionSigned`], L1 block
|
||||
/// info and block timestamp. The [`L1BlockInfo`] is used to calculate the l1 fee and l1 data
|
||||
@ -86,15 +97,10 @@ where
|
||||
let envelope_buf = tx.envelope_encoded();
|
||||
|
||||
let inner_l1_fee = l1_block_info
|
||||
.l1_tx_data_fee(
|
||||
&self.inner.chain_spec(),
|
||||
block_timestamp,
|
||||
&envelope_buf,
|
||||
tx.is_deposit(),
|
||||
)
|
||||
.l1_tx_data_fee(&self.chain_spec(), block_timestamp, &envelope_buf, tx.is_deposit())
|
||||
.map_err(|_| OpEthApiError::L1BlockFeeError)?;
|
||||
let inner_l1_data_gas = l1_block_info
|
||||
.l1_data_gas(&self.inner.chain_spec(), block_timestamp, &envelope_buf)
|
||||
.l1_data_gas(&self.chain_spec(), block_timestamp, &envelope_buf)
|
||||
.map_err(|_| OpEthApiError::L1BlockGasError)?;
|
||||
(
|
||||
Some(inner_l1_fee.saturating_to::<u128>()),
|
||||
|
||||
@ -29,6 +29,7 @@ reth-chainspec.workspace = true
|
||||
reth-execution-types.workspace = true
|
||||
reth-rpc-eth-types.workspace = true
|
||||
reth-rpc-server-types.workspace = true
|
||||
reth-network-api.workspace = true
|
||||
|
||||
# ethereum
|
||||
alloy-dyn-abi = { workspace = true, features = ["eip712"] }
|
||||
|
||||
@ -3,7 +3,10 @@
|
||||
|
||||
use futures::Future;
|
||||
use reth_rpc_eth_types::EthApiError;
|
||||
use reth_tasks::{pool::BlockingTaskPool, TaskSpawner};
|
||||
use reth_tasks::{
|
||||
pool::{BlockingTaskGuard, BlockingTaskPool},
|
||||
TaskSpawner,
|
||||
};
|
||||
use tokio::sync::{oneshot, AcquireError, OwnedSemaphorePermit};
|
||||
|
||||
use crate::EthApiTypes;
|
||||
@ -20,16 +23,23 @@ pub trait SpawnBlocking: EthApiTypes + Clone + Send + Sync + 'static {
|
||||
/// Thread pool access in default trait method implementations.
|
||||
fn tracing_task_pool(&self) -> &BlockingTaskPool;
|
||||
|
||||
/// Returns handle to semaphore for pool of CPU heavy blocking tasks.
|
||||
fn tracing_task_guard(&self) -> &BlockingTaskGuard;
|
||||
|
||||
/// See also [`Semaphore::acquire_owned`](`tokio::sync::Semaphore::acquire_owned`).
|
||||
fn acquire_owned(
|
||||
&self,
|
||||
) -> impl Future<Output = Result<OwnedSemaphorePermit, AcquireError>> + Send;
|
||||
) -> impl Future<Output = Result<OwnedSemaphorePermit, AcquireError>> + Send {
|
||||
self.tracing_task_guard().clone().acquire_owned()
|
||||
}
|
||||
|
||||
/// See also [`Semaphore::acquire_many_owned`](`tokio::sync::Semaphore::acquire_many_owned`).
|
||||
fn acquire_many_owned(
|
||||
&self,
|
||||
n: u32,
|
||||
) -> impl Future<Output = Result<OwnedSemaphorePermit, AcquireError>> + Send;
|
||||
) -> impl Future<Output = Result<OwnedSemaphorePermit, AcquireError>> + Send {
|
||||
self.tracing_task_guard().clone().acquire_many_owned(n)
|
||||
}
|
||||
|
||||
/// Executes the future on a new blocking task.
|
||||
///
|
||||
|
||||
@ -41,9 +41,6 @@ dyn_clone::clone_trait_object!(EthSigner);
|
||||
/// Adds 20 random dev signers for access via the API. Used in dev mode.
|
||||
#[auto_impl::auto_impl(&)]
|
||||
pub trait AddDevSigners {
|
||||
/// Returns a handle to the signers.
|
||||
fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn EthSigner>>>;
|
||||
|
||||
/// Generates 20 random developer accounts.
|
||||
/// Used in DEV mode.
|
||||
fn with_dev_accounts(&self);
|
||||
|
||||
@ -4,33 +4,80 @@ use std::sync::Arc;
|
||||
|
||||
use futures::Future;
|
||||
use reth_chainspec::{ChainInfo, ChainSpec};
|
||||
use reth_errors::RethResult;
|
||||
use reth_primitives::{Address, U64};
|
||||
use reth_rpc_types::SyncStatus;
|
||||
use reth_errors::{RethError, RethResult};
|
||||
use reth_network_api::NetworkInfo;
|
||||
use reth_primitives::{Address, U256, U64};
|
||||
use reth_provider::{BlockNumReader, ChainSpecProvider};
|
||||
use reth_rpc_types::{SyncInfo, SyncStatus};
|
||||
|
||||
use super::EthSigner;
|
||||
|
||||
/// `Eth` API trait.
|
||||
///
|
||||
/// Defines core functionality of the `eth` API implementation.
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait EthApiSpec: Send + Sync {
|
||||
/// Returns a handle for reading data from disk.
|
||||
fn provider(&self) -> impl ChainSpecProvider + BlockNumReader;
|
||||
|
||||
/// Returns a handle for reading network data summary.
|
||||
fn network(&self) -> impl NetworkInfo;
|
||||
|
||||
/// Returns the block node is started on.
|
||||
fn starting_block(&self) -> U256;
|
||||
|
||||
/// Returns a handle to the signers owned by provider.
|
||||
fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn EthSigner>>>;
|
||||
|
||||
/// Returns the current ethereum protocol version.
|
||||
fn protocol_version(&self) -> impl Future<Output = RethResult<U64>> + Send;
|
||||
fn protocol_version(&self) -> impl Future<Output = RethResult<U64>> + Send {
|
||||
async move {
|
||||
let status = self.network().network_status().await.map_err(RethError::other)?;
|
||||
Ok(U64::from(status.protocol_version))
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the chain id
|
||||
fn chain_id(&self) -> U64;
|
||||
fn chain_id(&self) -> U64 {
|
||||
U64::from(self.network().chain_id())
|
||||
}
|
||||
|
||||
/// Returns provider chain info
|
||||
fn chain_info(&self) -> RethResult<ChainInfo>;
|
||||
fn chain_info(&self) -> RethResult<ChainInfo> {
|
||||
Ok(self.provider().chain_info()?)
|
||||
}
|
||||
|
||||
/// Returns a list of addresses owned by provider.
|
||||
fn accounts(&self) -> Vec<Address>;
|
||||
fn accounts(&self) -> Vec<Address> {
|
||||
self.signers().read().iter().flat_map(|s| s.accounts()).collect()
|
||||
}
|
||||
|
||||
/// Returns `true` if the network is undergoing sync.
|
||||
fn is_syncing(&self) -> bool;
|
||||
fn is_syncing(&self) -> bool {
|
||||
self.network().is_syncing()
|
||||
}
|
||||
|
||||
/// Returns the [`SyncStatus`] of the network
|
||||
fn sync_status(&self) -> RethResult<SyncStatus>;
|
||||
fn sync_status(&self) -> RethResult<SyncStatus> {
|
||||
let status = if self.is_syncing() {
|
||||
let current_block = U256::from(
|
||||
self.provider().chain_info().map(|info| info.best_number).unwrap_or_default(),
|
||||
);
|
||||
SyncStatus::Info(SyncInfo {
|
||||
starting_block: self.starting_block(),
|
||||
current_block,
|
||||
highest_block: current_block,
|
||||
warp_chunks_amount: None,
|
||||
warp_chunks_processed: None,
|
||||
})
|
||||
} else {
|
||||
SyncStatus::None
|
||||
};
|
||||
Ok(status)
|
||||
}
|
||||
|
||||
/// Returns the configured [`ChainSpec`].
|
||||
fn chain_spec(&self) -> Arc<ChainSpec>;
|
||||
fn chain_spec(&self) -> Arc<ChainSpec> {
|
||||
self.provider().chain_spec()
|
||||
}
|
||||
}
|
||||
|
||||
@ -102,8 +102,7 @@ pub trait EthState: LoadState + SpawnBlocking {
|
||||
let block_id = block_id.unwrap_or_default();
|
||||
|
||||
// Check whether the distance to the block exceeds the maximum configured window.
|
||||
let block_number = self
|
||||
.provider()
|
||||
let block_number = LoadState::provider(self)
|
||||
.block_number_for_id(block_id)
|
||||
.map_err(Self::Error::from_eth_err)?
|
||||
.ok_or(EthApiError::UnknownBlockNumber)?;
|
||||
|
||||
@ -4,7 +4,6 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use derive_more::Deref;
|
||||
use futures::Future;
|
||||
use reth_node_api::{BuilderProvider, FullNodeComponents};
|
||||
use reth_primitives::{BlockNumberOrTag, U256};
|
||||
use reth_provider::{BlockReaderIdExt, CanonStateSubscriptions, ChainSpecProvider};
|
||||
@ -20,7 +19,7 @@ use reth_tasks::{
|
||||
pool::{BlockingTaskGuard, BlockingTaskPool},
|
||||
TaskExecutor, TaskSpawner, TokioTaskExecutor,
|
||||
};
|
||||
use tokio::sync::{AcquireError, Mutex, OwnedSemaphorePermit};
|
||||
use tokio::sync::Mutex;
|
||||
|
||||
/// `Eth` API implementation.
|
||||
///
|
||||
@ -139,29 +138,21 @@ impl<Provider, Pool, Network, EvmConfig> Clone for EthApi<Provider, Pool, Networ
|
||||
impl<Provider, Pool, Network, EvmConfig> SpawnBlocking
|
||||
for EthApi<Provider, Pool, Network, EvmConfig>
|
||||
where
|
||||
Self: EthApiTypes + Clone + Send + Sync + 'static,
|
||||
Self: Clone + Send + Sync + 'static,
|
||||
{
|
||||
#[inline]
|
||||
fn io_task_spawner(&self) -> impl reth_tasks::TaskSpawner {
|
||||
fn io_task_spawner(&self) -> impl TaskSpawner {
|
||||
self.inner.task_spawner()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn tracing_task_pool(&self) -> &reth_tasks::pool::BlockingTaskPool {
|
||||
fn tracing_task_pool(&self) -> &BlockingTaskPool {
|
||||
self.inner.blocking_task_pool()
|
||||
}
|
||||
|
||||
fn acquire_owned(
|
||||
&self,
|
||||
) -> impl Future<Output = Result<OwnedSemaphorePermit, AcquireError>> + Send {
|
||||
self.blocking_task_guard.clone().acquire_owned()
|
||||
}
|
||||
|
||||
fn acquire_many_owned(
|
||||
&self,
|
||||
n: u32,
|
||||
) -> impl Future<Output = Result<OwnedSemaphorePermit, AcquireError>> + Send {
|
||||
self.blocking_task_guard.clone().acquire_many_owned(n)
|
||||
#[inline]
|
||||
fn tracing_task_guard(&self) -> &BlockingTaskGuard {
|
||||
self.inner.blocking_task_guard()
|
||||
}
|
||||
}
|
||||
|
||||
@ -358,6 +349,12 @@ impl<Provider, Pool, Network, EvmConfig> EthApiInner<Provider, Pool, Network, Ev
|
||||
pub const fn eth_proof_window(&self) -> u64 {
|
||||
self.eth_proof_window
|
||||
}
|
||||
|
||||
/// Returns reference to [`BlockingTaskGuard`].
|
||||
#[inline]
|
||||
pub const fn blocking_task_guard(&self) -> &BlockingTaskGuard {
|
||||
&self.blocking_task_guard
|
||||
}
|
||||
}
|
||||
|
||||
impl<Provider, Pool, Network, EvmConfig> UpdateRawTxForwarder
|
||||
|
||||
@ -14,7 +14,7 @@ where
|
||||
Self: SpawnBlocking,
|
||||
Provider: BlockReaderIdExt + EvmEnvProvider + ChainSpecProvider + StateProviderFactory,
|
||||
Pool: TransactionPool,
|
||||
EvmConfig: reth_evm::ConfigureEvm,
|
||||
EvmConfig: ConfigureEvm,
|
||||
{
|
||||
#[inline]
|
||||
fn provider(
|
||||
|
||||
@ -1,13 +1,13 @@
|
||||
//! Builds an RPC receipt response w.r.t. data layout of network.
|
||||
|
||||
use reth_rpc_eth_api::{helpers::LoadReceipt, EthApiTypes};
|
||||
use reth_rpc_eth_api::helpers::LoadReceipt;
|
||||
use reth_rpc_eth_types::EthStateCache;
|
||||
|
||||
use crate::EthApi;
|
||||
|
||||
impl<Provider, Pool, Network, EvmConfig> LoadReceipt for EthApi<Provider, Pool, Network, EvmConfig>
|
||||
where
|
||||
Self: EthApiTypes,
|
||||
Self: Send + Sync,
|
||||
{
|
||||
#[inline]
|
||||
fn cache(&self) -> &EthStateCache {
|
||||
|
||||
@ -17,12 +17,8 @@ use crate::EthApi;
|
||||
impl<Provider, Pool, Network, EvmConfig> AddDevSigners
|
||||
for EthApi<Provider, Pool, Network, EvmConfig>
|
||||
{
|
||||
fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn EthSigner>>> {
|
||||
self.inner.signers()
|
||||
}
|
||||
|
||||
fn with_dev_accounts(&self) {
|
||||
*self.signers().write() = DevSigner::random_signers(20)
|
||||
*self.inner.signers().write() = DevSigner::random_signers(20)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -1,13 +1,7 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use reth_chainspec::{ChainInfo, ChainSpec};
|
||||
use reth_errors::{RethError, RethResult};
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_network_api::NetworkInfo;
|
||||
use reth_primitives::{Address, U256, U64};
|
||||
use reth_provider::{BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, StateProviderFactory};
|
||||
use reth_primitives::U256;
|
||||
use reth_provider::{BlockNumReader, ChainSpecProvider};
|
||||
use reth_rpc_eth_api::helpers::EthApiSpec;
|
||||
use reth_rpc_types::{SyncInfo, SyncStatus};
|
||||
use reth_transaction_pool::TransactionPool;
|
||||
|
||||
use crate::EthApi;
|
||||
@ -15,57 +9,23 @@ use crate::EthApi;
|
||||
impl<Provider, Pool, Network, EvmConfig> EthApiSpec for EthApi<Provider, Pool, Network, EvmConfig>
|
||||
where
|
||||
Pool: TransactionPool + 'static,
|
||||
Provider:
|
||||
BlockReaderIdExt + ChainSpecProvider + StateProviderFactory + EvmEnvProvider + 'static,
|
||||
Provider: BlockNumReader + ChainSpecProvider + 'static,
|
||||
Network: NetworkInfo + 'static,
|
||||
EvmConfig: ConfigureEvm,
|
||||
EvmConfig: Send + Sync,
|
||||
{
|
||||
/// Returns the current ethereum protocol version.
|
||||
///
|
||||
/// Note: This returns an [`U64`], since this should return as hex string.
|
||||
async fn protocol_version(&self) -> RethResult<U64> {
|
||||
let status = self.network().network_status().await.map_err(RethError::other)?;
|
||||
Ok(U64::from(status.protocol_version))
|
||||
fn provider(&self) -> impl ChainSpecProvider + BlockNumReader {
|
||||
self.inner.provider()
|
||||
}
|
||||
|
||||
/// Returns the chain id
|
||||
fn chain_id(&self) -> U64 {
|
||||
U64::from(self.network().chain_id())
|
||||
fn network(&self) -> impl NetworkInfo {
|
||||
self.inner.network()
|
||||
}
|
||||
|
||||
/// Returns the current info for the chain
|
||||
fn chain_info(&self) -> RethResult<ChainInfo> {
|
||||
Ok(self.provider().chain_info()?)
|
||||
fn starting_block(&self) -> U256 {
|
||||
self.inner.starting_block()
|
||||
}
|
||||
|
||||
fn accounts(&self) -> Vec<Address> {
|
||||
self.inner.signers().read().iter().flat_map(|s| s.accounts()).collect()
|
||||
}
|
||||
|
||||
fn is_syncing(&self) -> bool {
|
||||
self.network().is_syncing()
|
||||
}
|
||||
|
||||
/// Returns the [`SyncStatus`] of the network
|
||||
fn sync_status(&self) -> RethResult<SyncStatus> {
|
||||
let status = if self.is_syncing() {
|
||||
let current_block = U256::from(
|
||||
self.provider().chain_info().map(|info| info.best_number).unwrap_or_default(),
|
||||
);
|
||||
SyncStatus::Info(SyncInfo {
|
||||
starting_block: self.inner.starting_block(),
|
||||
current_block,
|
||||
highest_block: current_block,
|
||||
warp_chunks_amount: None,
|
||||
warp_chunks_processed: None,
|
||||
})
|
||||
} else {
|
||||
SyncStatus::None
|
||||
};
|
||||
Ok(status)
|
||||
}
|
||||
|
||||
fn chain_spec(&self) -> Arc<ChainSpec> {
|
||||
self.inner.provider().chain_spec()
|
||||
fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn reth_rpc_eth_api::helpers::EthSigner>>> {
|
||||
self.inner.signers()
|
||||
}
|
||||
}
|
||||
|
||||
@ -3,10 +3,7 @@
|
||||
use reth_provider::{ChainSpecProvider, StateProviderFactory};
|
||||
use reth_transaction_pool::TransactionPool;
|
||||
|
||||
use reth_rpc_eth_api::{
|
||||
helpers::{EthState, LoadState, SpawnBlocking},
|
||||
EthApiTypes,
|
||||
};
|
||||
use reth_rpc_eth_api::helpers::{EthState, LoadState, SpawnBlocking};
|
||||
use reth_rpc_eth_types::EthStateCache;
|
||||
|
||||
use crate::EthApi;
|
||||
@ -16,13 +13,13 @@ where
|
||||
Self: LoadState + SpawnBlocking,
|
||||
{
|
||||
fn max_proof_window(&self) -> u64 {
|
||||
self.eth_proof_window()
|
||||
self.inner.eth_proof_window()
|
||||
}
|
||||
}
|
||||
|
||||
impl<Provider, Pool, Network, EvmConfig> LoadState for EthApi<Provider, Pool, Network, EvmConfig>
|
||||
where
|
||||
Self: EthApiTypes,
|
||||
Self: Send + Sync,
|
||||
Provider: StateProviderFactory + ChainSpecProvider,
|
||||
Pool: TransactionPool,
|
||||
{
|
||||
|
||||
Reference in New Issue
Block a user