2023-05-31 02:32:34 +03:00
|
|
|
//! Load balanced communication with a group of web3 rpc providers
|
2023-05-18 23:34:22 +03:00
|
|
|
use super::blockchain::{BlocksByHashCache, BlocksByNumberCache, Web3ProxyBlock};
|
2023-06-27 22:36:41 +03:00
|
|
|
use super::consensus::{RankedRpcs, ShouldWaitForBlock};
|
2023-02-06 20:55:27 +03:00
|
|
|
use super::one::Web3Rpc;
|
2023-04-06 01:34:28 +03:00
|
|
|
use super::request::{OpenRequestHandle, OpenRequestResult, RequestErrorHandler};
|
2023-05-24 00:40:34 +03:00
|
|
|
use crate::app::{flatten_handle, Web3ProxyApp, Web3ProxyJoinHandle};
|
2023-07-13 20:58:22 +03:00
|
|
|
use crate::config::{average_block_interval, BlockAndRpc, Web3RpcConfig};
|
2023-05-31 07:26:11 +03:00
|
|
|
use crate::errors::{Web3ProxyError, Web3ProxyResult};
|
2022-11-08 22:58:11 +03:00
|
|
|
use crate::frontend::authorization::{Authorization, RequestMetadata};
|
2023-01-17 09:54:40 +03:00
|
|
|
use crate::frontend::rpc_proxy_ws::ProxyMode;
|
2023-06-08 03:26:38 +03:00
|
|
|
use crate::frontend::status::MokaCacheSerializer;
|
2023-05-31 02:32:34 +03:00
|
|
|
use crate::jsonrpc::{JsonRpcErrorData, JsonRpcParams, JsonRpcResultData};
|
2022-05-28 07:26:24 +03:00
|
|
|
use counter::Counter;
|
2022-05-05 22:07:09 +03:00
|
|
|
use derive_more::From;
|
2023-07-13 20:58:22 +03:00
|
|
|
use ethers::prelude::{ProviderError, U64};
|
2023-02-09 22:56:07 +03:00
|
|
|
use futures::future::try_join_all;
|
2022-05-05 22:07:09 +03:00
|
|
|
use futures::stream::FuturesUnordered;
|
|
|
|
use futures::StreamExt;
|
2023-06-27 22:36:41 +03:00
|
|
|
use hashbrown::HashMap;
|
2023-02-15 04:41:40 +03:00
|
|
|
use itertools::Itertools;
|
2023-07-13 20:58:22 +03:00
|
|
|
use moka::future::CacheBuilder;
|
2023-07-20 09:09:39 +03:00
|
|
|
use parking_lot::RwLock;
|
2022-05-21 01:16:15 +03:00
|
|
|
use serde::ser::{SerializeStruct, Serializer};
|
2023-07-20 09:09:39 +03:00
|
|
|
use serde::Serialize;
|
2022-09-24 05:47:44 +03:00
|
|
|
use serde_json::json;
|
2022-05-05 22:07:09 +03:00
|
|
|
use serde_json::value::RawValue;
|
2023-07-15 04:35:40 +03:00
|
|
|
use std::borrow::Cow;
|
2023-06-09 23:09:58 +03:00
|
|
|
use std::cmp::min_by_key;
|
2023-05-17 02:04:17 +03:00
|
|
|
use std::fmt::{self, Display};
|
2023-06-09 23:09:58 +03:00
|
|
|
use std::sync::atomic::Ordering;
|
2022-05-05 22:07:09 +03:00
|
|
|
use std::sync::Arc;
|
2023-05-24 06:46:27 +03:00
|
|
|
use tokio::select;
|
2023-07-20 09:09:39 +03:00
|
|
|
use tokio::sync::{mpsc, watch};
|
2023-05-24 00:40:34 +03:00
|
|
|
use tokio::time::{sleep, sleep_until, Duration, Instant};
|
2023-06-24 02:28:45 +03:00
|
|
|
use tracing::{debug, error, info, trace, warn};
|
2022-05-05 22:07:09 +03:00
|
|
|
|
|
|
|
/// A collection of web3 connections. Sends requests either the current best server or all servers.
|
|
|
|
#[derive(From)]
|
2023-02-06 20:55:27 +03:00
|
|
|
pub struct Web3Rpcs {
|
2023-07-15 04:35:40 +03:00
|
|
|
pub(crate) name: Cow<'static, str>,
|
2023-06-21 00:22:14 +03:00
|
|
|
pub(crate) chain_id: u64,
|
2023-02-26 10:52:33 +03:00
|
|
|
/// if watch_consensus_head_sender is some, Web3Rpc inside self will send blocks here when they get them
|
2023-07-11 09:08:06 +03:00
|
|
|
pub(crate) block_sender: mpsc::UnboundedSender<(Option<Web3ProxyBlock>, Arc<Web3Rpc>)>,
|
2022-08-27 02:44:25 +03:00
|
|
|
/// any requests will be forwarded to one (or more) of these connections
|
2023-06-16 20:40:02 +03:00
|
|
|
/// TODO: hopefully this not being an async lock will be okay. if you need it across awaits, clone the arc
|
2023-07-11 04:09:58 +03:00
|
|
|
pub(crate) by_name: RwLock<HashMap<String, Arc<Web3Rpc>>>,
|
2023-01-23 09:02:08 +03:00
|
|
|
/// all providers with the same consensus head block. won't update if there is no `self.watch_consensus_head_sender`
|
2023-02-26 10:52:33 +03:00
|
|
|
/// TODO: document that this is a watch sender and not a broadcast! if things get busy, blocks might get missed
|
|
|
|
/// TODO: why is watch_consensus_head_sender in an Option, but this one isn't?
|
|
|
|
/// Geth's subscriptions have the same potential for skipping blocks.
|
2023-06-27 22:36:41 +03:00
|
|
|
pub(crate) watch_ranked_rpcs: watch::Sender<Option<Arc<RankedRpcs>>>,
|
2023-01-23 09:02:08 +03:00
|
|
|
/// this head receiver makes it easy to wait until there is a new block
|
2023-06-27 22:36:41 +03:00
|
|
|
pub(super) watch_head_block: Option<watch::Sender<Option<Web3ProxyBlock>>>,
|
2022-08-24 02:56:47 +03:00
|
|
|
/// TODO: this map is going to grow forever unless we do some sort of pruning. maybe store pruned in redis?
|
|
|
|
/// all blocks, including orphans
|
2023-02-26 10:52:33 +03:00
|
|
|
pub(super) blocks_by_hash: BlocksByHashCache,
|
2022-09-01 08:58:55 +03:00
|
|
|
/// blocks on the heaviest chain
|
2023-05-18 23:34:22 +03:00
|
|
|
pub(super) blocks_by_number: BlocksByNumberCache,
|
2023-02-14 23:14:50 +03:00
|
|
|
/// the number of rpcs required to agree on consensus for the head block (thundering herd protection)
|
2023-06-09 23:30:00 +03:00
|
|
|
pub(super) min_synced_rpcs: usize,
|
2023-02-14 23:14:50 +03:00
|
|
|
/// the soft limit required to agree on consensus for the head block. (thundering herd protection)
|
2022-08-27 06:11:58 +03:00
|
|
|
pub(super) min_sum_soft_limit: u32,
|
2023-02-14 23:14:50 +03:00
|
|
|
/// how far behind the highest known block height we can be before we stop serving requests
|
2023-06-17 20:11:48 +03:00
|
|
|
pub(super) max_head_block_lag: U64,
|
2023-02-14 23:14:50 +03:00
|
|
|
/// how old our consensus head block we can be before we stop serving requests
|
2023-06-17 20:11:48 +03:00
|
|
|
/// calculated based on max_head_block_lag and averge block times
|
|
|
|
pub(super) max_head_block_age: Duration,
|
2022-05-05 22:07:09 +03:00
|
|
|
}
|
|
|
|
|
2023-02-06 20:55:27 +03:00
|
|
|
impl Web3Rpcs {
|
2022-08-24 02:56:47 +03:00
|
|
|
/// Spawn durable connections to multiple Web3 providers.
|
2022-06-14 07:04:14 +03:00
|
|
|
pub async fn spawn(
|
2023-06-17 20:11:48 +03:00
|
|
|
chain_id: u64,
|
|
|
|
max_head_block_lag: Option<U64>,
|
2022-11-23 01:45:22 +03:00
|
|
|
min_head_rpcs: usize,
|
2023-02-12 12:22:53 +03:00
|
|
|
min_sum_soft_limit: u32,
|
2023-07-15 04:35:40 +03:00
|
|
|
name: Cow<'static, str>,
|
2023-02-15 04:41:40 +03:00
|
|
|
watch_consensus_head_sender: Option<watch::Sender<Option<Web3ProxyBlock>>>,
|
2023-01-26 08:24:09 +03:00
|
|
|
) -> anyhow::Result<(
|
|
|
|
Arc<Self>,
|
2023-05-24 00:40:34 +03:00
|
|
|
Web3ProxyJoinHandle<()>,
|
2023-06-27 22:36:41 +03:00
|
|
|
watch::Receiver<Option<Arc<RankedRpcs>>>,
|
2023-01-26 08:24:09 +03:00
|
|
|
)> {
|
2023-07-11 09:08:06 +03:00
|
|
|
let (block_sender, block_receiver) = mpsc::unbounded_channel::<BlockAndRpc>();
|
2022-05-05 22:07:09 +03:00
|
|
|
|
2023-02-26 10:52:33 +03:00
|
|
|
// these blocks don't have full transactions, but they do have rather variable amounts of transaction hashes
|
2023-05-24 00:40:34 +03:00
|
|
|
// TODO: actual weighter on this
|
2023-04-24 21:00:12 +03:00
|
|
|
// TODO: time_to_idle instead?
|
2023-06-08 03:26:38 +03:00
|
|
|
let blocks_by_hash: BlocksByHashCache = CacheBuilder::new(1_000)
|
2023-06-08 03:55:34 +03:00
|
|
|
.name("blocks_by_hash")
|
2023-06-08 03:26:38 +03:00
|
|
|
.time_to_idle(Duration::from_secs(30 * 60))
|
|
|
|
.build();
|
2023-02-26 10:52:33 +03:00
|
|
|
|
2022-09-17 05:30:06 +03:00
|
|
|
// all block numbers are the same size, so no need for weigher
|
2023-02-26 10:52:33 +03:00
|
|
|
// TODO: limits from config
|
2023-04-24 21:00:12 +03:00
|
|
|
// TODO: time_to_idle instead?
|
2023-06-08 03:26:38 +03:00
|
|
|
let blocks_by_number = CacheBuilder::new(1_000)
|
2023-06-08 03:55:34 +03:00
|
|
|
.name("blocks_by_number")
|
2023-06-08 03:26:38 +03:00
|
|
|
.time_to_idle(Duration::from_secs(30 * 60))
|
|
|
|
.build();
|
2022-09-05 08:53:58 +03:00
|
|
|
|
2023-03-25 19:56:45 +03:00
|
|
|
let (watch_consensus_rpcs_sender, consensus_connections_watcher) =
|
|
|
|
watch::channel(Default::default());
|
2023-01-23 09:02:08 +03:00
|
|
|
|
2023-02-26 10:52:33 +03:00
|
|
|
// by_name starts empty. self.apply_server_configs will add to it
|
2023-07-11 04:09:58 +03:00
|
|
|
let by_name = RwLock::new(HashMap::new());
|
2023-01-23 09:02:08 +03:00
|
|
|
|
2023-06-17 20:11:48 +03:00
|
|
|
let max_head_block_lag = max_head_block_lag.unwrap_or(5.into());
|
|
|
|
|
2023-06-17 21:33:33 +03:00
|
|
|
let max_head_block_age =
|
|
|
|
average_block_interval(chain_id).mul_f32((max_head_block_lag.as_u64() * 10) as f32);
|
2023-06-17 20:11:48 +03:00
|
|
|
|
2022-05-16 01:02:14 +03:00
|
|
|
let connections = Arc::new(Self {
|
2023-02-26 10:52:33 +03:00
|
|
|
block_sender,
|
|
|
|
blocks_by_hash,
|
|
|
|
blocks_by_number,
|
2023-05-17 02:04:17 +03:00
|
|
|
by_name,
|
2023-06-21 00:22:14 +03:00
|
|
|
chain_id,
|
2023-06-17 19:19:05 +03:00
|
|
|
max_head_block_age,
|
2023-06-17 20:11:48 +03:00
|
|
|
max_head_block_lag,
|
2023-06-09 23:30:00 +03:00
|
|
|
min_synced_rpcs: min_head_rpcs,
|
2023-05-17 02:04:17 +03:00
|
|
|
min_sum_soft_limit,
|
|
|
|
name,
|
2023-06-27 22:36:41 +03:00
|
|
|
watch_head_block: watch_consensus_head_sender,
|
|
|
|
watch_ranked_rpcs: watch_consensus_rpcs_sender,
|
2022-05-16 01:02:14 +03:00
|
|
|
});
|
|
|
|
|
2022-06-14 07:04:14 +03:00
|
|
|
let handle = {
|
|
|
|
let connections = connections.clone();
|
|
|
|
|
2023-07-13 20:58:22 +03:00
|
|
|
tokio::spawn(connections.subscribe(block_receiver))
|
2022-06-14 07:04:14 +03:00
|
|
|
};
|
|
|
|
|
2023-01-26 08:24:09 +03:00
|
|
|
Ok((connections, handle, consensus_connections_watcher))
|
2022-05-18 19:35:06 +03:00
|
|
|
}
|
|
|
|
|
2023-02-26 10:52:33 +03:00
|
|
|
/// update the rpcs in this group
|
|
|
|
pub async fn apply_server_configs(
|
|
|
|
&self,
|
|
|
|
app: &Web3ProxyApp,
|
|
|
|
rpc_configs: HashMap<String, Web3RpcConfig>,
|
2023-05-24 00:40:34 +03:00
|
|
|
) -> Web3ProxyResult<()> {
|
2023-02-26 10:52:33 +03:00
|
|
|
// safety checks
|
|
|
|
if rpc_configs.len() < app.config.min_synced_rpcs {
|
2023-04-14 10:41:51 +03:00
|
|
|
// TODO: don't count disabled servers!
|
|
|
|
// TODO: include if this is balanced, private, or 4337
|
|
|
|
warn!(
|
2023-02-26 10:52:33 +03:00
|
|
|
"Only {}/{} rpcs! Add more rpcs or reduce min_synced_rpcs.",
|
|
|
|
rpc_configs.len(),
|
|
|
|
app.config.min_synced_rpcs
|
2023-04-14 10:41:51 +03:00
|
|
|
);
|
|
|
|
return Ok(());
|
2023-02-26 10:52:33 +03:00
|
|
|
}
|
|
|
|
|
|
|
|
// safety check on sum soft limit
|
|
|
|
// TODO: will need to think about this more once sum_soft_limit is dynamic
|
|
|
|
let sum_soft_limit = rpc_configs.values().fold(0, |acc, x| acc + x.soft_limit);
|
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
// TODO: require a buffer?
|
2023-05-24 00:40:34 +03:00
|
|
|
if sum_soft_limit < self.min_sum_soft_limit {
|
|
|
|
return Err(Web3ProxyError::NotEnoughSoftLimit {
|
|
|
|
available: sum_soft_limit,
|
|
|
|
needed: self.min_sum_soft_limit,
|
|
|
|
});
|
|
|
|
}
|
2023-02-26 10:52:33 +03:00
|
|
|
|
2023-06-17 20:11:48 +03:00
|
|
|
let chain_id = app.config.chain_id;
|
|
|
|
|
|
|
|
let block_interval = average_block_interval(chain_id);
|
|
|
|
|
2023-07-11 06:59:03 +03:00
|
|
|
let mut names_to_keep = vec![];
|
|
|
|
|
2023-02-26 10:52:33 +03:00
|
|
|
// turn configs into connections (in parallel)
|
|
|
|
let mut spawn_handles: FuturesUnordered<_> = rpc_configs
|
|
|
|
.into_iter()
|
|
|
|
.filter_map(|(server_name, server_config)| {
|
|
|
|
if server_config.disabled {
|
|
|
|
info!("{} is disabled", server_name);
|
|
|
|
return None;
|
|
|
|
}
|
|
|
|
|
|
|
|
let http_client = app.http_client.clone();
|
|
|
|
let vredis_pool = app.vredis_pool.clone();
|
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
let block_sender = if self.watch_head_block.is_some() {
|
2023-02-26 10:52:33 +03:00
|
|
|
Some(self.block_sender.clone())
|
|
|
|
} else {
|
|
|
|
None
|
|
|
|
};
|
|
|
|
|
2023-05-18 23:34:22 +03:00
|
|
|
let blocks_by_hash_cache = self.blocks_by_hash.clone();
|
2023-02-26 10:52:33 +03:00
|
|
|
|
2023-06-19 02:41:01 +03:00
|
|
|
debug!("spawning tasks for {}", server_name);
|
2023-02-26 10:52:33 +03:00
|
|
|
|
2023-07-11 06:59:03 +03:00
|
|
|
names_to_keep.push(server_name.clone());
|
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
let handle = tokio::spawn(server_config.spawn(
|
|
|
|
server_name,
|
|
|
|
vredis_pool,
|
|
|
|
chain_id,
|
2023-06-17 20:11:48 +03:00
|
|
|
block_interval,
|
2023-05-13 01:15:32 +03:00
|
|
|
http_client,
|
2023-05-18 23:34:22 +03:00
|
|
|
blocks_by_hash_cache,
|
2023-05-13 01:15:32 +03:00
|
|
|
block_sender,
|
2023-07-11 08:17:15 +03:00
|
|
|
self.max_head_block_age,
|
2023-05-13 01:15:32 +03:00
|
|
|
));
|
2023-02-26 10:52:33 +03:00
|
|
|
|
|
|
|
Some(handle)
|
|
|
|
})
|
|
|
|
.collect();
|
|
|
|
|
|
|
|
while let Some(x) = spawn_handles.next().await {
|
|
|
|
match x {
|
2023-07-11 04:09:58 +03:00
|
|
|
Ok(Ok((new_rpc, _handle))) => {
|
2023-02-26 10:52:33 +03:00
|
|
|
// web3 connection worked
|
2023-05-13 01:15:32 +03:00
|
|
|
|
2023-07-20 09:09:39 +03:00
|
|
|
let old_rpc = self.by_name.read().get(&new_rpc.name).map(Arc::clone);
|
2023-07-11 04:09:58 +03:00
|
|
|
|
2023-06-16 20:40:02 +03:00
|
|
|
// clean up the old rpc if it exists
|
2023-07-11 04:09:58 +03:00
|
|
|
if let Some(old_rpc) = old_rpc {
|
2023-06-16 10:46:27 +03:00
|
|
|
trace!("old_rpc: {}", old_rpc);
|
|
|
|
|
|
|
|
// if the old rpc was synced, wait for the new one to sync
|
2023-06-13 20:00:08 +03:00
|
|
|
if old_rpc.head_block.as_ref().unwrap().borrow().is_some() {
|
2023-05-13 01:15:32 +03:00
|
|
|
let mut new_head_receiver =
|
2023-07-11 04:09:58 +03:00
|
|
|
new_rpc.head_block.as_ref().unwrap().subscribe();
|
|
|
|
trace!("waiting for new {} connection to sync", new_rpc);
|
2023-05-13 01:15:32 +03:00
|
|
|
|
2023-06-16 10:46:27 +03:00
|
|
|
// TODO: maximum wait time
|
2023-05-13 01:15:32 +03:00
|
|
|
while new_head_receiver.borrow_and_update().is_none() {
|
2023-06-16 10:46:27 +03:00
|
|
|
if new_head_receiver.changed().await.is_err() {
|
|
|
|
break;
|
|
|
|
};
|
2023-02-28 00:40:13 +03:00
|
|
|
}
|
2023-06-16 10:46:27 +03:00
|
|
|
}
|
2023-02-28 00:40:13 +03:00
|
|
|
|
2023-06-16 20:40:02 +03:00
|
|
|
// new rpc is synced (or old one was not synced). update the local map
|
|
|
|
// make sure that any new requests use the new connection
|
2023-07-20 09:09:39 +03:00
|
|
|
self.by_name.write().insert(new_rpc.name.clone(), new_rpc);
|
2023-06-16 20:40:02 +03:00
|
|
|
|
2023-06-16 10:46:27 +03:00
|
|
|
// tell the old rpc to disconnect
|
|
|
|
if let Some(ref disconnect_sender) = old_rpc.disconnect_watch {
|
2023-07-11 06:59:03 +03:00
|
|
|
debug!("telling old {} to disconnect", old_rpc);
|
2023-06-16 10:46:27 +03:00
|
|
|
disconnect_sender.send_replace(true);
|
2023-05-23 01:32:15 +03:00
|
|
|
}
|
2023-06-16 20:40:02 +03:00
|
|
|
} else {
|
2023-07-20 09:09:39 +03:00
|
|
|
self.by_name.write().insert(new_rpc.name.clone(), new_rpc);
|
2023-02-26 10:52:33 +03:00
|
|
|
}
|
|
|
|
}
|
|
|
|
Ok(Err(err)) => {
|
|
|
|
// if we got an error here, the app can continue on
|
|
|
|
// TODO: include context about which connection failed
|
2023-05-24 00:40:34 +03:00
|
|
|
// TODO: retry automatically
|
2023-02-26 10:52:33 +03:00
|
|
|
error!("Unable to create connection. err={:?}", err);
|
|
|
|
}
|
|
|
|
Err(err) => {
|
|
|
|
// something actually bad happened. exit with an error
|
|
|
|
return Err(err.into());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-07-11 06:59:03 +03:00
|
|
|
// TODO: remove any RPCs that were part of the config, but are now removed
|
2023-07-20 09:09:39 +03:00
|
|
|
let active_names: Vec<_> = self.by_name.read().keys().cloned().collect();
|
2023-07-11 06:59:03 +03:00
|
|
|
for name in active_names {
|
|
|
|
if names_to_keep.contains(&name) {
|
|
|
|
continue;
|
|
|
|
}
|
2023-07-20 09:09:39 +03:00
|
|
|
if let Some(old_rpc) = self.by_name.write().remove(&name) {
|
2023-07-11 06:59:03 +03:00
|
|
|
if let Some(ref disconnect_sender) = old_rpc.disconnect_watch {
|
|
|
|
debug!("telling {} to disconnect. no longer needed", old_rpc);
|
|
|
|
disconnect_sender.send_replace(true);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-07-20 09:09:39 +03:00
|
|
|
let num_rpcs = self.len();
|
2023-05-24 00:40:34 +03:00
|
|
|
|
2023-06-09 23:30:00 +03:00
|
|
|
if num_rpcs < self.min_synced_rpcs {
|
2023-05-24 00:40:34 +03:00
|
|
|
return Err(Web3ProxyError::NotEnoughRpcs {
|
|
|
|
num_known: num_rpcs,
|
2023-06-09 23:30:00 +03:00
|
|
|
min_head_rpcs: self.min_synced_rpcs,
|
2023-05-24 00:40:34 +03:00
|
|
|
});
|
|
|
|
}
|
|
|
|
|
2023-02-26 10:52:33 +03:00
|
|
|
Ok(())
|
|
|
|
}
|
|
|
|
|
2023-07-20 09:09:39 +03:00
|
|
|
pub fn get(&self, conn_name: &str) -> Option<Arc<Web3Rpc>> {
|
|
|
|
self.by_name.read().get(conn_name).cloned()
|
2022-08-26 20:26:17 +03:00
|
|
|
}
|
|
|
|
|
2023-07-20 09:09:39 +03:00
|
|
|
pub fn len(&self) -> usize {
|
|
|
|
self.by_name.read().len()
|
2023-03-02 02:21:09 +03:00
|
|
|
}
|
|
|
|
|
2023-07-20 09:09:39 +03:00
|
|
|
pub fn is_empty(&self) -> bool {
|
|
|
|
self.by_name.read().is_empty()
|
2023-03-02 02:21:09 +03:00
|
|
|
}
|
|
|
|
|
2023-07-10 21:05:07 +03:00
|
|
|
/// TODO: rename to be consistent between "head" and "synced"
|
2023-02-27 07:00:13 +03:00
|
|
|
pub fn min_head_rpcs(&self) -> usize {
|
2023-06-09 23:30:00 +03:00
|
|
|
self.min_synced_rpcs
|
2023-02-26 10:52:33 +03:00
|
|
|
}
|
|
|
|
|
2022-07-22 22:30:39 +03:00
|
|
|
/// subscribe to blocks and transactions from all the backend rpcs.
|
2023-02-06 20:55:27 +03:00
|
|
|
/// blocks are processed by all the `Web3Rpc`s and then sent to the `block_receiver`
|
|
|
|
/// transaction ids from all the `Web3Rpc`s are deduplicated and forwarded to `pending_tx_sender`
|
2022-06-14 07:04:14 +03:00
|
|
|
async fn subscribe(
|
|
|
|
self: Arc<Self>,
|
2023-07-11 09:08:06 +03:00
|
|
|
block_receiver: mpsc::UnboundedReceiver<BlockAndRpc>,
|
2023-05-24 00:40:34 +03:00
|
|
|
) -> Web3ProxyResult<()> {
|
2022-06-16 05:53:37 +03:00
|
|
|
let mut futures = vec![];
|
2022-05-18 19:35:06 +03:00
|
|
|
|
2023-07-13 20:58:22 +03:00
|
|
|
// // setup the transaction funnel
|
|
|
|
// // it skips any duplicates (unless they are being orphaned)
|
|
|
|
// // fetches new transactions from the notifying rpc
|
|
|
|
// // forwards new transacitons to pending_tx_receipt_sender
|
|
|
|
// if let Some(pending_tx_sender) = pending_tx_sender.clone() {
|
|
|
|
// let clone = self.clone();
|
|
|
|
// let handle = tokio::task::spawn(async move {
|
|
|
|
// // TODO: set up this future the same as the block funnel
|
|
|
|
// while let Some((pending_tx_id, rpc)) =
|
|
|
|
// clone.pending_tx_id_receiver.write().await.recv().await
|
|
|
|
// {
|
|
|
|
// let f = clone.clone().process_incoming_tx_id(
|
|
|
|
// rpc,
|
|
|
|
// pending_tx_id,
|
|
|
|
// pending_tx_sender.clone(),
|
|
|
|
// );
|
|
|
|
// tokio::spawn(f);
|
|
|
|
// }
|
|
|
|
|
|
|
|
// Ok(())
|
|
|
|
// });
|
|
|
|
|
|
|
|
// futures.push(flatten_handle(handle));
|
|
|
|
// }
|
2022-06-14 07:04:14 +03:00
|
|
|
|
2022-06-14 08:43:28 +03:00
|
|
|
// setup the block funnel
|
2023-06-27 22:36:41 +03:00
|
|
|
if self.watch_head_block.is_some() {
|
2022-06-14 07:04:14 +03:00
|
|
|
let connections = Arc::clone(&self);
|
2022-11-08 22:58:11 +03:00
|
|
|
|
2023-03-25 19:56:45 +03:00
|
|
|
let handle = tokio::task::Builder::default()
|
2022-08-26 20:26:17 +03:00
|
|
|
.name("process_incoming_blocks")
|
2023-07-13 20:58:22 +03:00
|
|
|
.spawn(async move { connections.process_incoming_blocks(block_receiver).await })?;
|
2022-05-18 19:35:06 +03:00
|
|
|
|
2022-06-16 05:53:37 +03:00
|
|
|
futures.push(flatten_handle(handle));
|
2022-05-16 01:02:14 +03:00
|
|
|
}
|
|
|
|
|
2022-06-14 08:43:28 +03:00
|
|
|
if futures.is_empty() {
|
|
|
|
// no transaction or block subscriptions.
|
2022-08-11 00:29:50 +03:00
|
|
|
|
2023-03-25 19:56:45 +03:00
|
|
|
let handle = tokio::task::Builder::default()
|
|
|
|
.name("noop")
|
|
|
|
.spawn(async move {
|
|
|
|
loop {
|
|
|
|
sleep(Duration::from_secs(600)).await;
|
2023-05-30 01:48:22 +03:00
|
|
|
// TODO: "every interval, do a health check or disconnect the rpc"
|
2023-03-25 19:56:45 +03:00
|
|
|
}
|
|
|
|
})?;
|
2022-08-11 00:29:50 +03:00
|
|
|
|
|
|
|
futures.push(flatten_handle(handle));
|
2022-06-14 08:43:28 +03:00
|
|
|
}
|
|
|
|
|
2022-06-16 05:53:37 +03:00
|
|
|
if let Err(e) = try_join_all(futures).await {
|
2023-07-15 04:30:01 +03:00
|
|
|
error!(?self, "subscriptions over");
|
2022-06-16 05:53:37 +03:00
|
|
|
return Err(e);
|
2022-06-14 07:04:14 +03:00
|
|
|
}
|
2022-05-18 19:35:06 +03:00
|
|
|
|
2023-07-15 04:30:01 +03:00
|
|
|
info!(?self, "subscriptions over");
|
2022-06-14 07:04:14 +03:00
|
|
|
Ok(())
|
2022-05-05 22:07:09 +03:00
|
|
|
}
|
|
|
|
|
2022-05-28 07:26:24 +03:00
|
|
|
/// Send the same request to all the handles. Returning the most common success or most common error.
|
2023-01-19 13:13:00 +03:00
|
|
|
/// TODO: option to return the fastest response and handles for all the others instead?
|
2023-05-31 02:32:34 +03:00
|
|
|
pub async fn try_send_parallel_requests<P: JsonRpcParams>(
|
2022-05-28 07:26:24 +03:00
|
|
|
&self,
|
2022-08-24 03:14:49 +03:00
|
|
|
active_request_handles: Vec<OpenRequestHandle>,
|
2022-05-28 07:26:24 +03:00
|
|
|
method: &str,
|
2023-05-31 02:32:34 +03:00
|
|
|
params: &P,
|
2022-09-23 01:14:24 +03:00
|
|
|
// TODO: remove this box once i figure out how to do the options
|
2023-05-31 02:32:34 +03:00
|
|
|
) -> Result<Box<RawValue>, ProviderError> {
|
2022-08-26 20:26:17 +03:00
|
|
|
// TODO: if only 1 active_request_handles, do self.try_send_request?
|
2022-05-05 22:07:09 +03:00
|
|
|
|
2023-05-31 02:32:34 +03:00
|
|
|
// TODO: iter stream
|
2022-05-28 07:26:24 +03:00
|
|
|
let responses = active_request_handles
|
|
|
|
.into_iter()
|
|
|
|
.map(|active_request_handle| async move {
|
2023-05-31 02:32:34 +03:00
|
|
|
let result: Result<Box<RawValue>, _> =
|
|
|
|
active_request_handle.request(method, &json!(¶ms)).await;
|
2022-05-28 07:26:24 +03:00
|
|
|
result
|
|
|
|
})
|
|
|
|
.collect::<FuturesUnordered<_>>()
|
|
|
|
.collect::<Vec<Result<Box<RawValue>, ProviderError>>>()
|
|
|
|
.await;
|
|
|
|
|
2022-05-28 21:45:45 +03:00
|
|
|
// TODO: Strings are not great keys, but we can't use RawValue or ProviderError as keys because they don't implement Hash or Eq
|
2022-12-24 04:32:58 +03:00
|
|
|
let mut count_map: HashMap<String, _> = HashMap::new();
|
2022-05-28 07:26:24 +03:00
|
|
|
let mut counts: Counter<String> = Counter::new();
|
2022-12-24 04:32:58 +03:00
|
|
|
let mut any_ok_with_json_result = false;
|
|
|
|
for partial_response in responses {
|
|
|
|
if partial_response.is_ok() {
|
|
|
|
any_ok_with_json_result = true;
|
|
|
|
}
|
|
|
|
|
2023-05-13 21:13:02 +03:00
|
|
|
// TODO: better key!
|
|
|
|
let s = format!("{:?}", partial_response);
|
2022-05-28 07:26:24 +03:00
|
|
|
|
|
|
|
if count_map.get(&s).is_none() {
|
2023-05-13 21:13:02 +03:00
|
|
|
count_map.insert(s.clone(), partial_response);
|
2022-05-28 07:26:24 +03:00
|
|
|
}
|
2022-05-05 22:07:09 +03:00
|
|
|
|
2023-07-20 23:19:50 +03:00
|
|
|
counts.update([s]);
|
2022-05-05 22:07:09 +03:00
|
|
|
}
|
|
|
|
|
2023-05-31 02:32:34 +03:00
|
|
|
// return the most_common success if any. otherwise return the most_common error
|
2022-05-28 07:26:24 +03:00
|
|
|
for (most_common, _) in counts.most_common_ordered() {
|
2022-12-24 04:32:58 +03:00
|
|
|
let most_common = count_map
|
|
|
|
.remove(&most_common)
|
|
|
|
.expect("most_common key must exist");
|
|
|
|
|
|
|
|
match most_common {
|
|
|
|
Ok(x) => {
|
|
|
|
// return the most common success
|
2023-05-31 02:32:34 +03:00
|
|
|
return Ok(x);
|
2022-12-24 04:32:58 +03:00
|
|
|
}
|
|
|
|
Err(err) => {
|
2023-05-13 21:13:02 +03:00
|
|
|
if any_ok_with_json_result {
|
2023-05-31 02:32:34 +03:00
|
|
|
// the most common is an error, but there is an Ok in here somewhere. continue the loop to find it
|
2022-12-24 04:32:58 +03:00
|
|
|
continue;
|
|
|
|
}
|
2023-05-13 21:13:02 +03:00
|
|
|
|
2023-05-31 02:32:34 +03:00
|
|
|
return Err(err);
|
2022-12-24 04:32:58 +03:00
|
|
|
}
|
2022-05-05 22:07:09 +03:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-05-28 07:26:24 +03:00
|
|
|
// TODO: what should we do if we get here? i don't think we will
|
2022-12-21 08:55:12 +03:00
|
|
|
unimplemented!("this shouldn't be possible")
|
2022-05-05 22:07:09 +03:00
|
|
|
}
|
|
|
|
|
2023-05-19 08:43:07 +03:00
|
|
|
async fn _best_available_rpc(
|
2022-07-02 04:20:28 +03:00
|
|
|
&self,
|
2023-01-19 14:05:39 +03:00
|
|
|
authorization: &Arc<Authorization>,
|
2023-05-31 02:32:34 +03:00
|
|
|
error_handler: Option<RequestErrorHandler>,
|
2023-05-19 08:43:07 +03:00
|
|
|
potential_rpcs: &[Arc<Web3Rpc>],
|
2023-05-18 23:51:28 +03:00
|
|
|
skip: &mut Vec<Arc<Web3Rpc>>,
|
2023-05-19 08:43:07 +03:00
|
|
|
) -> OpenRequestResult {
|
|
|
|
let mut earliest_retry_at = None;
|
2023-04-30 04:38:31 +03:00
|
|
|
|
2023-05-19 08:43:07 +03:00
|
|
|
for (rpc_a, rpc_b) in potential_rpcs.iter().circular_tuple_windows() {
|
|
|
|
trace!("{} vs {}", rpc_a, rpc_b);
|
2023-07-14 04:13:16 +03:00
|
|
|
// TODO: ties within X% to the server with the smallest block_data_limit
|
|
|
|
// faster rpc. backups always lose.
|
|
|
|
let faster_rpc = min_by_key(rpc_a, rpc_b, |x| (x.backup, x.weighted_peak_latency()));
|
2023-05-19 08:43:07 +03:00
|
|
|
trace!("winner: {}", faster_rpc);
|
2022-11-07 00:05:03 +03:00
|
|
|
|
2023-05-19 08:43:07 +03:00
|
|
|
// add to the skip list in case this one fails
|
|
|
|
skip.push(Arc::clone(faster_rpc));
|
2023-02-27 09:44:09 +03:00
|
|
|
|
2023-05-19 08:43:07 +03:00
|
|
|
// just because it has lower latency doesn't mean we are sure to get a connection. there might be rate limits
|
2023-05-31 02:32:34 +03:00
|
|
|
// TODO: what error_handler?
|
|
|
|
match faster_rpc
|
|
|
|
.try_request_handle(authorization, error_handler)
|
|
|
|
.await
|
|
|
|
{
|
2023-05-19 08:43:07 +03:00
|
|
|
Ok(OpenRequestResult::Handle(handle)) => {
|
|
|
|
trace!("opened handle: {}", faster_rpc);
|
|
|
|
return OpenRequestResult::Handle(handle);
|
|
|
|
}
|
|
|
|
Ok(OpenRequestResult::RetryAt(retry_at)) => {
|
|
|
|
trace!(
|
|
|
|
"retry on {} @ {}",
|
|
|
|
faster_rpc,
|
|
|
|
retry_at.duration_since(Instant::now()).as_secs_f32()
|
|
|
|
);
|
2023-01-03 19:33:49 +03:00
|
|
|
|
2023-05-19 08:43:07 +03:00
|
|
|
if earliest_retry_at.is_none() {
|
|
|
|
earliest_retry_at = Some(retry_at);
|
|
|
|
} else {
|
|
|
|
earliest_retry_at = earliest_retry_at.min(Some(retry_at));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
Ok(OpenRequestResult::NotReady) => {
|
|
|
|
// TODO: log a warning? emit a stat?
|
|
|
|
trace!("best_rpc not ready: {}", faster_rpc);
|
|
|
|
}
|
|
|
|
Err(err) => {
|
|
|
|
trace!("No request handle for {}. err={:?}", faster_rpc, err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2023-05-13 01:15:32 +03:00
|
|
|
|
2023-05-19 08:43:07 +03:00
|
|
|
if let Some(retry_at) = earliest_retry_at {
|
|
|
|
OpenRequestResult::RetryAt(retry_at)
|
|
|
|
} else {
|
|
|
|
OpenRequestResult::NotReady
|
|
|
|
}
|
|
|
|
}
|
2023-04-14 10:15:27 +03:00
|
|
|
|
2023-05-19 08:43:07 +03:00
|
|
|
pub async fn wait_for_best_rpc(
|
|
|
|
&self,
|
|
|
|
request_metadata: Option<&Arc<RequestMetadata>>,
|
|
|
|
skip_rpcs: &mut Vec<Arc<Web3Rpc>>,
|
|
|
|
min_block_needed: Option<&U64>,
|
|
|
|
max_block_needed: Option<&U64>,
|
2023-05-24 06:46:27 +03:00
|
|
|
max_wait: Option<Duration>,
|
2023-05-31 02:32:34 +03:00
|
|
|
error_handler: Option<RequestErrorHandler>,
|
2023-05-19 08:43:07 +03:00
|
|
|
) -> Web3ProxyResult<OpenRequestResult> {
|
2023-06-21 00:22:14 +03:00
|
|
|
let start = Instant::now();
|
|
|
|
|
2023-05-19 08:43:07 +03:00
|
|
|
let mut earliest_retry_at: Option<Instant> = None;
|
2023-01-04 09:37:51 +03:00
|
|
|
|
2023-05-31 02:32:34 +03:00
|
|
|
// TODO: pass db_conn to the "default" authorization for revert logging
|
|
|
|
let authorization = request_metadata
|
|
|
|
.and_then(|x| x.authorization.clone())
|
|
|
|
.unwrap_or_default();
|
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
let mut watch_ranked_rpcs = self.watch_ranked_rpcs.subscribe();
|
2023-02-16 08:05:41 +03:00
|
|
|
|
2023-07-10 21:05:07 +03:00
|
|
|
let mut potential_rpcs = Vec::new();
|
2023-04-30 04:38:31 +03:00
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
loop {
|
|
|
|
// TODO: need a change so that protected and 4337 rpcs set watch_consensus_rpcs on start
|
|
|
|
let ranked_rpcs: Option<Arc<RankedRpcs>> =
|
|
|
|
watch_ranked_rpcs.borrow_and_update().clone();
|
|
|
|
|
|
|
|
// first check everything that is synced
|
|
|
|
// even though we might be querying an old block that an unsynced server can handle,
|
|
|
|
// it is best to not send queries to a syncing server. that slows down sync and can bloat erigon's disk usage.
|
|
|
|
if let Some(ranked_rpcs) = ranked_rpcs {
|
|
|
|
potential_rpcs.extend(
|
|
|
|
ranked_rpcs
|
|
|
|
.all()
|
|
|
|
.iter()
|
|
|
|
.filter(|rpc| {
|
|
|
|
ranked_rpcs.rpc_will_work_now(
|
2023-05-31 02:32:34 +03:00
|
|
|
skip_rpcs,
|
2023-06-27 22:36:41 +03:00
|
|
|
min_block_needed,
|
|
|
|
max_block_needed,
|
|
|
|
rpc,
|
2023-05-31 02:32:34 +03:00
|
|
|
)
|
2023-06-27 22:36:41 +03:00
|
|
|
})
|
|
|
|
.cloned(),
|
|
|
|
);
|
|
|
|
|
|
|
|
if potential_rpcs.len() >= self.min_synced_rpcs {
|
|
|
|
// we have enough potential rpcs. try to load balance
|
|
|
|
potential_rpcs.sort_by_cached_key(|x| {
|
|
|
|
x.shuffle_for_load_balancing_on(max_block_needed.copied())
|
|
|
|
});
|
|
|
|
|
|
|
|
match self
|
|
|
|
._best_available_rpc(
|
|
|
|
&authorization,
|
|
|
|
error_handler,
|
|
|
|
&potential_rpcs,
|
|
|
|
skip_rpcs,
|
|
|
|
)
|
|
|
|
.await
|
|
|
|
{
|
|
|
|
OpenRequestResult::Handle(x) => return Ok(OpenRequestResult::Handle(x)),
|
|
|
|
OpenRequestResult::NotReady => {}
|
|
|
|
OpenRequestResult::RetryAt(retry_at) => {
|
|
|
|
if earliest_retry_at.is_none() {
|
|
|
|
earliest_retry_at = Some(retry_at);
|
|
|
|
} else {
|
|
|
|
earliest_retry_at = earliest_retry_at.min(Some(retry_at));
|
2023-05-19 08:43:07 +03:00
|
|
|
}
|
|
|
|
}
|
2022-12-08 09:54:38 +03:00
|
|
|
}
|
2023-06-27 22:36:41 +03:00
|
|
|
}
|
2023-05-13 21:13:02 +03:00
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
let waiting_for = min_block_needed.max(max_block_needed);
|
2023-05-19 08:43:07 +03:00
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
if let Some(max_wait) = max_wait {
|
|
|
|
match ranked_rpcs.should_wait_for_block(waiting_for, skip_rpcs) {
|
|
|
|
ShouldWaitForBlock::NeverReady => break,
|
|
|
|
ShouldWaitForBlock::Ready => {
|
|
|
|
if start.elapsed() > max_wait {
|
|
|
|
break;
|
2023-05-19 08:43:07 +03:00
|
|
|
}
|
|
|
|
}
|
2023-06-27 22:36:41 +03:00
|
|
|
ShouldWaitForBlock::Wait { .. } => select! {
|
|
|
|
_ = watch_ranked_rpcs.changed() => {
|
|
|
|
// no need to borrow_and_update because we do that at the top of the loop
|
2023-06-21 00:22:14 +03:00
|
|
|
},
|
2023-06-27 22:36:41 +03:00
|
|
|
_ = sleep_until(start + max_wait) => break,
|
2023-06-21 00:22:14 +03:00
|
|
|
},
|
2023-05-29 08:37:37 +03:00
|
|
|
}
|
2022-05-06 23:44:12 +03:00
|
|
|
}
|
2023-06-27 22:36:41 +03:00
|
|
|
} else if let Some(max_wait) = max_wait {
|
|
|
|
select! {
|
|
|
|
_ = watch_ranked_rpcs.changed() => {
|
|
|
|
// no need to borrow_and_update because we do that at the top of the loop
|
|
|
|
},
|
|
|
|
_ = sleep_until(start + max_wait) => break,
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
break;
|
2022-05-05 22:07:09 +03:00
|
|
|
}
|
2023-06-27 22:36:41 +03:00
|
|
|
|
|
|
|
// clear for the next loop
|
|
|
|
potential_rpcs.clear();
|
2022-05-05 22:07:09 +03:00
|
|
|
}
|
|
|
|
|
2022-12-08 09:54:38 +03:00
|
|
|
if let Some(request_metadata) = request_metadata {
|
2023-05-13 09:00:03 +03:00
|
|
|
request_metadata.no_servers.fetch_add(1, Ordering::AcqRel);
|
2022-12-08 09:54:38 +03:00
|
|
|
}
|
|
|
|
|
2023-05-19 08:43:07 +03:00
|
|
|
if let Some(retry_at) = earliest_retry_at {
|
|
|
|
// TODO: log the server that retry_at came from
|
|
|
|
warn!(
|
2023-06-29 07:30:00 +03:00
|
|
|
?skip_rpcs,
|
|
|
|
retry_in_s=?retry_at.duration_since(Instant::now()).as_secs_f32(),
|
|
|
|
"no servers in {} ready!",
|
2023-06-29 20:15:12 +03:00
|
|
|
self,
|
2023-05-19 08:43:07 +03:00
|
|
|
);
|
2022-10-12 00:31:34 +03:00
|
|
|
|
2023-05-19 08:43:07 +03:00
|
|
|
Ok(OpenRequestResult::RetryAt(retry_at))
|
|
|
|
} else {
|
2023-06-29 07:30:00 +03:00
|
|
|
warn!(?skip_rpcs, "no servers in {} ready!", self);
|
2022-10-12 00:31:34 +03:00
|
|
|
|
2023-05-19 08:43:07 +03:00
|
|
|
Ok(OpenRequestResult::NotReady)
|
2022-08-07 09:48:57 +03:00
|
|
|
}
|
2022-05-05 22:07:09 +03:00
|
|
|
}
|
|
|
|
|
|
|
|
/// get all rpc servers that are not rate limited
|
2023-01-24 12:58:31 +03:00
|
|
|
/// this prefers synced servers, but it will return servers even if they aren't fully in sync.
|
|
|
|
/// This is useful for broadcasting signed transactions.
|
2022-08-07 09:48:57 +03:00
|
|
|
// TODO: better type on this that can return an anyhow::Result
|
2023-03-02 00:35:50 +03:00
|
|
|
// TODO: this is broken
|
2023-01-24 12:58:31 +03:00
|
|
|
pub async fn all_connections(
|
2022-07-09 06:34:39 +03:00
|
|
|
&self,
|
2023-05-31 02:32:34 +03:00
|
|
|
request_metadata: Option<&Arc<RequestMetadata>>,
|
2023-02-11 07:45:57 +03:00
|
|
|
min_block_needed: Option<&U64>,
|
|
|
|
max_block_needed: Option<&U64>,
|
2023-01-12 01:51:01 +03:00
|
|
|
max_count: Option<usize>,
|
2023-05-31 02:32:34 +03:00
|
|
|
error_level: Option<RequestErrorHandler>,
|
2022-08-24 03:14:49 +03:00
|
|
|
) -> Result<Vec<OpenRequestHandle>, Option<Instant>> {
|
2022-08-07 09:48:57 +03:00
|
|
|
let mut earliest_retry_at = None;
|
2022-05-05 22:07:09 +03:00
|
|
|
|
2023-07-10 21:05:07 +03:00
|
|
|
// TODO: filter the rpcs with Ranked.will_work_now
|
2023-07-20 09:09:39 +03:00
|
|
|
let mut all_rpcs: Vec<_> = self.by_name.read().values().cloned().collect();
|
2023-07-10 21:05:07 +03:00
|
|
|
|
2023-01-24 12:58:31 +03:00
|
|
|
let mut max_count = if let Some(max_count) = max_count {
|
|
|
|
max_count
|
2023-01-12 01:51:01 +03:00
|
|
|
} else {
|
2023-07-10 21:05:07 +03:00
|
|
|
all_rpcs.len()
|
2023-01-12 01:51:01 +03:00
|
|
|
};
|
|
|
|
|
2023-02-27 09:44:09 +03:00
|
|
|
trace!("max_count: {}", max_count);
|
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
if max_count == 0 {
|
|
|
|
// TODO: return a future that resolves when we know a head block?
|
|
|
|
return Err(None);
|
|
|
|
}
|
|
|
|
|
2023-02-27 09:44:09 +03:00
|
|
|
let mut selected_rpcs = Vec::with_capacity(max_count);
|
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
// TODO: this sorts them all even though we probably won't need all of them. think about this more
|
2023-06-09 23:09:58 +03:00
|
|
|
all_rpcs.sort_by_cached_key(|x| x.sort_for_load_balancing_on(max_block_needed.copied()));
|
2023-02-27 09:44:09 +03:00
|
|
|
|
|
|
|
trace!("all_rpcs: {:#?}", all_rpcs);
|
2023-02-03 01:48:23 +03:00
|
|
|
|
2023-05-31 02:32:34 +03:00
|
|
|
let authorization = request_metadata
|
|
|
|
.and_then(|x| x.authorization.clone())
|
|
|
|
.unwrap_or_default();
|
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
for rpc in all_rpcs {
|
2023-02-27 09:44:09 +03:00
|
|
|
trace!("trying {}", rpc);
|
2023-01-24 12:58:31 +03:00
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
// TODO: use a helper function for these
|
2023-02-11 07:45:57 +03:00
|
|
|
if let Some(block_needed) = min_block_needed {
|
2023-02-27 09:44:09 +03:00
|
|
|
if !rpc.has_block_data(block_needed) {
|
2023-05-13 01:15:32 +03:00
|
|
|
trace!("{} is missing min_block_needed. skipping", rpc);
|
2023-02-11 07:45:57 +03:00
|
|
|
continue;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if let Some(block_needed) = max_block_needed {
|
2023-02-27 09:44:09 +03:00
|
|
|
if !rpc.has_block_data(block_needed) {
|
2023-05-13 01:15:32 +03:00
|
|
|
trace!("{} is missing max_block_needed. skipping", rpc);
|
2022-07-19 04:31:12 +03:00
|
|
|
continue;
|
|
|
|
}
|
2022-07-09 07:25:59 +03:00
|
|
|
}
|
|
|
|
|
2022-05-05 22:07:09 +03:00
|
|
|
// check rate limits and increment our connection counter
|
2023-05-31 02:32:34 +03:00
|
|
|
match rpc.try_request_handle(&authorization, error_level).await {
|
2022-08-24 03:14:49 +03:00
|
|
|
Ok(OpenRequestResult::RetryAt(retry_at)) => {
|
2022-05-06 07:29:25 +03:00
|
|
|
// this rpc is not available. skip it
|
2023-05-13 21:13:02 +03:00
|
|
|
trace!("{} is rate limited. skipping", rpc);
|
2022-08-07 09:48:57 +03:00
|
|
|
earliest_retry_at = earliest_retry_at.min(Some(retry_at));
|
|
|
|
}
|
2023-01-12 01:51:01 +03:00
|
|
|
Ok(OpenRequestResult::Handle(handle)) => {
|
2023-02-27 09:44:09 +03:00
|
|
|
trace!("{} is available", rpc);
|
2023-06-21 00:22:14 +03:00
|
|
|
selected_rpcs.push(handle);
|
|
|
|
|
2023-01-12 01:51:01 +03:00
|
|
|
max_count -= 1;
|
2023-06-21 00:22:14 +03:00
|
|
|
if max_count == 0 {
|
|
|
|
break;
|
|
|
|
}
|
2023-01-12 01:51:01 +03:00
|
|
|
}
|
2023-02-15 04:41:40 +03:00
|
|
|
Ok(OpenRequestResult::NotReady) => {
|
2023-02-27 09:44:09 +03:00
|
|
|
warn!("no request handle for {}", rpc)
|
2022-08-07 09:48:57 +03:00
|
|
|
}
|
|
|
|
Err(err) => {
|
2023-06-29 07:30:00 +03:00
|
|
|
warn!(?err, "error getting request handle for {}", rpc)
|
2022-05-06 07:29:25 +03:00
|
|
|
}
|
2022-05-05 22:07:09 +03:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if !selected_rpcs.is_empty() {
|
|
|
|
return Ok(selected_rpcs);
|
|
|
|
}
|
|
|
|
|
2022-05-22 02:34:05 +03:00
|
|
|
// return the earliest retry_after (if no rpcs are synced, this will be None)
|
2022-08-07 09:48:57 +03:00
|
|
|
Err(earliest_retry_at)
|
2022-05-05 22:07:09 +03:00
|
|
|
}
|
2022-05-28 07:26:24 +03:00
|
|
|
|
2023-05-31 02:32:34 +03:00
|
|
|
pub async fn internal_request<P: JsonRpcParams, R: JsonRpcResultData>(
|
2022-05-29 04:23:58 +03:00
|
|
|
&self,
|
2023-05-31 02:32:34 +03:00
|
|
|
method: &str,
|
|
|
|
params: &P,
|
2023-06-29 01:04:55 +03:00
|
|
|
max_tries: Option<usize>,
|
2023-06-21 00:22:14 +03:00
|
|
|
max_wait: Option<Duration>,
|
2023-05-31 02:32:34 +03:00
|
|
|
) -> Web3ProxyResult<R> {
|
|
|
|
// TODO: no request_metadata means we won't have stats on this internal request.
|
2023-06-29 04:36:17 +03:00
|
|
|
self.request_with_metadata_and_retries(
|
|
|
|
method, params, None, max_tries, max_wait, None, None,
|
|
|
|
)
|
|
|
|
.await
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Make a request with stat tracking.
|
|
|
|
#[allow(clippy::too_many_arguments)]
|
|
|
|
pub async fn request_with_metadata_and_retries<P: JsonRpcParams, R: JsonRpcResultData>(
|
|
|
|
&self,
|
|
|
|
method: &str,
|
|
|
|
params: &P,
|
|
|
|
request_metadata: Option<&Arc<RequestMetadata>>,
|
|
|
|
max_tries: Option<usize>,
|
|
|
|
max_wait: Option<Duration>,
|
|
|
|
min_block_needed: Option<&U64>,
|
|
|
|
max_block_needed: Option<&U64>,
|
|
|
|
) -> Web3ProxyResult<R> {
|
|
|
|
let mut tries = max_tries.unwrap_or(1);
|
|
|
|
|
|
|
|
let mut last_error = None;
|
|
|
|
|
|
|
|
while tries > 0 {
|
|
|
|
tries -= 1;
|
|
|
|
|
|
|
|
match self
|
|
|
|
.request_with_metadata(
|
|
|
|
method,
|
|
|
|
params,
|
|
|
|
request_metadata,
|
|
|
|
max_wait,
|
|
|
|
min_block_needed,
|
|
|
|
max_block_needed,
|
|
|
|
)
|
|
|
|
.await
|
|
|
|
{
|
|
|
|
Ok(x) => return Ok(x),
|
|
|
|
Err(Web3ProxyError::JsonRpcErrorData(err)) => {
|
|
|
|
// TODO: retry some of these? i think request_with_metadata is already smart enough though
|
|
|
|
return Err(err.into());
|
|
|
|
}
|
|
|
|
Err(err) => {
|
|
|
|
// TODO: only log params in dev
|
2023-06-29 20:15:12 +03:00
|
|
|
warn!(rpc=%self, %method, ?params, ?err, %tries, "retry-able error");
|
2023-06-29 04:36:17 +03:00
|
|
|
last_error = Some(err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if let Some(err) = last_error {
|
|
|
|
return Err(err);
|
|
|
|
}
|
|
|
|
|
|
|
|
Err(anyhow::anyhow!("no response, but no error either. this is a bug").into())
|
2023-05-31 02:32:34 +03:00
|
|
|
}
|
|
|
|
|
2023-06-27 07:11:43 +03:00
|
|
|
/// Make a request with stat tracking.
|
2023-05-31 02:32:34 +03:00
|
|
|
pub async fn request_with_metadata<P: JsonRpcParams, R: JsonRpcResultData>(
|
|
|
|
&self,
|
|
|
|
method: &str,
|
|
|
|
params: &P,
|
2022-10-11 22:58:25 +03:00
|
|
|
request_metadata: Option<&Arc<RequestMetadata>>,
|
2023-06-21 00:22:14 +03:00
|
|
|
max_wait: Option<Duration>,
|
2022-07-22 22:30:39 +03:00
|
|
|
min_block_needed: Option<&U64>,
|
2023-02-11 07:45:57 +03:00
|
|
|
max_block_needed: Option<&U64>,
|
2023-05-31 02:32:34 +03:00
|
|
|
) -> Web3ProxyResult<R> {
|
2022-07-02 04:20:28 +03:00
|
|
|
let mut skip_rpcs = vec![];
|
2023-01-24 20:36:07 +03:00
|
|
|
let mut method_not_available_response = None;
|
2022-07-02 04:20:28 +03:00
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
let mut watch_consensus_rpcs = self.watch_ranked_rpcs.subscribe();
|
2023-01-23 09:02:08 +03:00
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
let start = Instant::now();
|
2023-01-25 07:44:50 +03:00
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
// set error_handler to Save. this might be overridden depending on the request_metadata.authorization
|
2023-05-31 02:32:34 +03:00
|
|
|
let error_handler = Some(RequestErrorHandler::Save);
|
|
|
|
|
2023-06-27 09:04:56 +03:00
|
|
|
let mut last_provider_error = None;
|
|
|
|
|
2023-05-19 08:43:07 +03:00
|
|
|
// TODO: the loop here feels somewhat redundant with the loop in best_available_rpc
|
2023-06-21 00:22:14 +03:00
|
|
|
loop {
|
|
|
|
if let Some(max_wait) = max_wait {
|
|
|
|
if start.elapsed() > max_wait {
|
2023-06-29 01:04:55 +03:00
|
|
|
trace!("max_wait exceeded");
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
2023-06-21 00:22:14 +03:00
|
|
|
|
2022-07-19 04:31:12 +03:00
|
|
|
match self
|
2023-05-19 08:43:07 +03:00
|
|
|
.wait_for_best_rpc(
|
2022-10-12 00:31:34 +03:00
|
|
|
request_metadata,
|
2023-05-18 23:51:28 +03:00
|
|
|
&mut skip_rpcs,
|
2022-10-12 00:31:34 +03:00
|
|
|
min_block_needed,
|
2023-02-11 07:45:57 +03:00
|
|
|
max_block_needed,
|
2023-06-21 00:22:14 +03:00
|
|
|
max_wait,
|
2023-05-31 02:32:34 +03:00
|
|
|
error_handler,
|
2022-10-12 00:31:34 +03:00
|
|
|
)
|
2022-09-10 03:58:33 +03:00
|
|
|
.await?
|
2022-07-19 04:31:12 +03:00
|
|
|
{
|
2022-09-10 03:58:33 +03:00
|
|
|
OpenRequestResult::Handle(active_request_handle) => {
|
2022-07-02 04:20:28 +03:00
|
|
|
// save the rpc in case we get an error and want to retry on another server
|
2023-01-20 08:46:47 +03:00
|
|
|
// TODO: look at backend_requests instead
|
2023-01-26 08:24:09 +03:00
|
|
|
let rpc = active_request_handle.clone_connection();
|
2022-07-02 04:20:28 +03:00
|
|
|
|
2022-10-11 22:58:25 +03:00
|
|
|
if let Some(request_metadata) = request_metadata {
|
2023-05-13 01:15:32 +03:00
|
|
|
request_metadata.backend_requests.lock().push(rpc.clone());
|
2022-10-11 22:58:25 +03:00
|
|
|
}
|
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
let is_backup_response = rpc.backup;
|
|
|
|
|
2023-05-31 02:32:34 +03:00
|
|
|
match active_request_handle.request::<P, R>(method, params).await {
|
2022-06-04 00:45:44 +03:00
|
|
|
Ok(response) => {
|
2023-05-13 01:15:32 +03:00
|
|
|
// TODO: if there are multiple responses being aggregated, this will only use the last server's backup type
|
|
|
|
if let Some(request_metadata) = request_metadata {
|
|
|
|
request_metadata
|
|
|
|
.response_from_backup_rpc
|
|
|
|
.store(is_backup_response, Ordering::Release);
|
|
|
|
|
2023-07-06 04:18:10 +03:00
|
|
|
request_metadata
|
|
|
|
.user_error_response
|
|
|
|
.store(false, Ordering::Release);
|
|
|
|
|
2023-06-27 08:40:00 +03:00
|
|
|
request_metadata
|
|
|
|
.error_response
|
|
|
|
.store(false, Ordering::Release);
|
|
|
|
}
|
|
|
|
|
2023-05-31 02:32:34 +03:00
|
|
|
return Ok(response);
|
2023-05-13 21:13:02 +03:00
|
|
|
}
|
|
|
|
Err(error) => {
|
2023-06-27 08:40:00 +03:00
|
|
|
// TODO: if this is an error, do NOT return. continue to try on another server
|
|
|
|
let error = match JsonRpcErrorData::try_from(&error) {
|
2023-07-06 04:18:10 +03:00
|
|
|
Ok(x) => {
|
|
|
|
if let Some(request_metadata) = request_metadata {
|
|
|
|
request_metadata
|
|
|
|
.user_error_response
|
|
|
|
.store(true, Ordering::Release);
|
|
|
|
}
|
|
|
|
x
|
|
|
|
}
|
2023-06-27 08:40:00 +03:00
|
|
|
Err(err) => {
|
|
|
|
warn!(?err, "error from {}", rpc);
|
2023-06-27 09:04:56 +03:00
|
|
|
|
2023-07-06 04:18:10 +03:00
|
|
|
if let Some(request_metadata) = request_metadata {
|
|
|
|
request_metadata
|
|
|
|
.error_response
|
|
|
|
.store(true, Ordering::Release);
|
|
|
|
|
|
|
|
request_metadata
|
|
|
|
.user_error_response
|
|
|
|
.store(false, Ordering::Release);
|
|
|
|
}
|
|
|
|
|
2023-06-27 09:04:56 +03:00
|
|
|
last_provider_error = Some(error);
|
|
|
|
|
2023-06-27 08:40:00 +03:00
|
|
|
continue;
|
|
|
|
}
|
|
|
|
};
|
2023-01-19 13:13:00 +03:00
|
|
|
|
2023-05-13 21:13:02 +03:00
|
|
|
// some errors should be retried on other nodes
|
|
|
|
let error_msg = error.message.as_ref();
|
|
|
|
|
|
|
|
// different providers do different codes. check all of them
|
|
|
|
// TODO: there's probably more strings to add here
|
|
|
|
let rate_limit_substrings = ["limit", "exceeded", "quota usage"];
|
|
|
|
for rate_limit_substr in rate_limit_substrings {
|
|
|
|
if error_msg.contains(rate_limit_substr) {
|
2023-07-08 01:56:46 +03:00
|
|
|
if error_msg.contains("block size") {
|
|
|
|
// TODO: this message is likely wrong, but i can't find the actual one in my terminal now
|
|
|
|
// they hit an expected limit. return the error now
|
|
|
|
return Err(error.into());
|
|
|
|
} else if error_msg.contains("result on length") {
|
2023-06-13 04:58:51 +03:00
|
|
|
// this error contains "limit" but is not a rate limit error
|
|
|
|
// TODO: make the expected limit configurable
|
|
|
|
// TODO: parse the rate_limit_substr and only continue if it is < expected limit
|
2023-07-11 09:38:23 +03:00
|
|
|
if error_msg.contains("exceeding limit 2000000")
|
2023-07-11 23:18:19 +03:00
|
|
|
|| error_msg.ends_with(
|
2023-07-11 09:38:23 +03:00
|
|
|
"exceeding --rpc.returndata.limit 2000000",
|
|
|
|
)
|
|
|
|
{
|
2023-06-13 04:58:51 +03:00
|
|
|
// they hit our expected limit. return the error now
|
|
|
|
return Err(error.into());
|
|
|
|
} else {
|
2023-07-08 01:56:46 +03:00
|
|
|
// they hit a limit lower than what we expect. the server is misconfigured
|
|
|
|
error!(
|
2023-06-29 07:30:00 +03:00
|
|
|
%error_msg,
|
2023-07-08 01:56:46 +03:00
|
|
|
"unexpected result limit by {}",
|
2023-06-29 07:30:00 +03:00
|
|
|
skip_rpcs.last().unwrap(),
|
2023-06-13 04:58:51 +03:00
|
|
|
);
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
warn!(
|
2023-06-29 07:30:00 +03:00
|
|
|
%error_msg,
|
|
|
|
"rate limited by {}",
|
2023-06-13 04:58:51 +03:00
|
|
|
skip_rpcs.last().unwrap()
|
|
|
|
);
|
|
|
|
continue;
|
|
|
|
}
|
2023-01-19 13:13:00 +03:00
|
|
|
}
|
2023-05-13 21:13:02 +03:00
|
|
|
}
|
2023-01-19 13:13:00 +03:00
|
|
|
|
2023-05-13 21:13:02 +03:00
|
|
|
match error.code {
|
|
|
|
-32000 => {
|
|
|
|
// TODO: regex?
|
|
|
|
let retry_prefixes = [
|
|
|
|
"header not found",
|
|
|
|
"header for hash not found",
|
|
|
|
"missing trie node",
|
|
|
|
"node not started",
|
|
|
|
"RPC timeout",
|
|
|
|
];
|
|
|
|
for retry_prefix in retry_prefixes {
|
|
|
|
if error_msg.starts_with(retry_prefix) {
|
|
|
|
// TODO: too verbose
|
|
|
|
debug!("retrying on another server");
|
2023-01-24 20:36:07 +03:00
|
|
|
continue;
|
|
|
|
}
|
2023-05-13 21:13:02 +03:00
|
|
|
}
|
|
|
|
}
|
|
|
|
-32601 => {
|
|
|
|
let error_msg = error.message.as_ref();
|
|
|
|
|
|
|
|
// sometimes a provider does not support all rpc methods
|
|
|
|
// we check other connections rather than returning the error
|
|
|
|
// but sometimes the method is something that is actually unsupported,
|
|
|
|
// so we save the response here to return it later
|
|
|
|
|
|
|
|
// some providers look like this
|
|
|
|
if error_msg.starts_with("the method")
|
|
|
|
&& error_msg.ends_with("is not available")
|
|
|
|
{
|
|
|
|
method_not_available_response = Some(error);
|
|
|
|
continue;
|
|
|
|
}
|
2023-01-24 20:36:07 +03:00
|
|
|
|
2023-05-13 21:13:02 +03:00
|
|
|
// others look like this (this is the example in the official spec)
|
|
|
|
if error_msg == "Method not found" {
|
|
|
|
method_not_available_response = Some(error);
|
|
|
|
continue;
|
2022-07-09 07:25:59 +03:00
|
|
|
}
|
2022-07-02 04:20:28 +03:00
|
|
|
}
|
2023-05-13 21:13:02 +03:00
|
|
|
_ => {}
|
2022-06-04 00:45:44 +03:00
|
|
|
}
|
2022-05-29 04:23:58 +03:00
|
|
|
|
2023-05-13 21:13:02 +03:00
|
|
|
// let rpc = skip_rpcs
|
|
|
|
// .last()
|
|
|
|
// .expect("there must have been a provider if we got an error");
|
2022-09-10 03:58:33 +03:00
|
|
|
|
2022-12-17 07:05:01 +03:00
|
|
|
// TODO: emit a stat. if a server is getting skipped a lot, something is not right
|
|
|
|
|
2023-03-23 04:43:13 +03:00
|
|
|
// TODO: if we get a TrySendError, reconnect. wait why do we see a trysenderror on a dual provider? shouldn't it be using reqwest
|
|
|
|
|
2023-05-13 21:13:02 +03:00
|
|
|
// TODO! WRONG! ONLY SET RETRY_AT IF THIS IS A SERVER/CONNECTION ERROR. JSONRPC "error" is FINE
|
|
|
|
// trace!(
|
|
|
|
// "Backend server error on {}! Retrying {:?} on another. err={:?}",
|
|
|
|
// rpc,
|
|
|
|
// request,
|
|
|
|
// error,
|
|
|
|
// );
|
|
|
|
// if let Some(ref hard_limit_until) = rpc.hard_limit_until {
|
|
|
|
// let retry_at = Instant::now() + Duration::from_secs(1);
|
2022-05-29 04:23:58 +03:00
|
|
|
|
2023-05-13 21:13:02 +03:00
|
|
|
// hard_limit_until.send_replace(retry_at);
|
|
|
|
// }
|
2023-05-13 01:15:32 +03:00
|
|
|
|
2023-05-31 02:32:34 +03:00
|
|
|
return Err(error.into());
|
2022-06-04 00:45:44 +03:00
|
|
|
}
|
|
|
|
}
|
2022-05-29 04:23:58 +03:00
|
|
|
}
|
2022-09-10 03:58:33 +03:00
|
|
|
OpenRequestResult::RetryAt(retry_at) => {
|
2022-08-07 09:48:57 +03:00
|
|
|
// TODO: move this to a helper function
|
|
|
|
// sleep (TODO: with a lock?) until our rate limits should be available
|
|
|
|
// TODO: if a server catches up sync while we are waiting, we could stop waiting
|
2023-01-25 07:44:50 +03:00
|
|
|
warn!(
|
2023-05-13 21:13:02 +03:00
|
|
|
"All rate limits exceeded. waiting for change in synced servers or {:?}s",
|
|
|
|
retry_at.duration_since(Instant::now()).as_secs_f32()
|
2023-01-25 07:44:50 +03:00
|
|
|
);
|
2022-05-29 04:23:58 +03:00
|
|
|
|
2022-10-12 00:31:34 +03:00
|
|
|
// TODO: have a separate column for rate limited?
|
|
|
|
if let Some(request_metadata) = request_metadata {
|
2023-05-13 09:00:03 +03:00
|
|
|
request_metadata.no_servers.fetch_add(1, Ordering::AcqRel);
|
2022-10-12 00:31:34 +03:00
|
|
|
}
|
|
|
|
|
2023-01-25 07:44:50 +03:00
|
|
|
tokio::select! {
|
|
|
|
_ = sleep_until(retry_at) => {
|
2023-05-13 21:13:02 +03:00
|
|
|
trace!("slept!");
|
2023-01-25 07:44:50 +03:00
|
|
|
skip_rpcs.pop();
|
|
|
|
}
|
2023-05-16 23:26:39 +03:00
|
|
|
_ = watch_consensus_rpcs.changed() => {
|
|
|
|
watch_consensus_rpcs.borrow_and_update();
|
2023-01-23 09:02:08 +03:00
|
|
|
}
|
|
|
|
}
|
2022-05-29 04:23:58 +03:00
|
|
|
}
|
2023-02-15 04:41:40 +03:00
|
|
|
OpenRequestResult::NotReady => {
|
2023-07-06 04:18:10 +03:00
|
|
|
if let Some(request_metadata) = request_metadata {
|
|
|
|
request_metadata
|
|
|
|
.error_response
|
|
|
|
.store(true, Ordering::Release);
|
|
|
|
}
|
2023-05-19 08:43:07 +03:00
|
|
|
break;
|
2022-05-29 04:23:58 +03:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2022-07-02 04:20:28 +03:00
|
|
|
|
2023-05-13 21:13:02 +03:00
|
|
|
if let Some(err) = method_not_available_response {
|
2023-07-06 04:18:10 +03:00
|
|
|
if let Some(request_metadata) = request_metadata {
|
|
|
|
request_metadata
|
|
|
|
.error_response
|
|
|
|
.store(false, Ordering::Release);
|
|
|
|
|
|
|
|
request_metadata
|
|
|
|
.user_error_response
|
|
|
|
.store(true, Ordering::Release);
|
|
|
|
}
|
|
|
|
|
2023-05-31 02:32:34 +03:00
|
|
|
// this error response is likely the user's fault
|
|
|
|
// TODO: emit a stat for unsupported methods. then we can know what there is demand for or if we are missing a feature
|
|
|
|
return Err(err.into());
|
2023-02-16 08:05:41 +03:00
|
|
|
}
|
|
|
|
|
2023-06-27 09:04:56 +03:00
|
|
|
if let Some(err) = last_provider_error {
|
|
|
|
return Err(err.into());
|
|
|
|
}
|
|
|
|
|
2023-07-20 09:09:39 +03:00
|
|
|
let num_conns = self.len();
|
2023-01-25 09:45:20 +03:00
|
|
|
let num_skipped = skip_rpcs.len();
|
2022-11-28 09:52:16 +03:00
|
|
|
|
2023-04-11 08:28:31 +03:00
|
|
|
let needed = min_block_needed.max(max_block_needed);
|
|
|
|
|
2023-05-16 23:26:39 +03:00
|
|
|
let head_block_num = watch_consensus_rpcs
|
2023-04-11 02:05:53 +03:00
|
|
|
.borrow()
|
|
|
|
.as_ref()
|
|
|
|
.map(|x| *x.head_block.number());
|
2023-03-09 22:22:40 +03:00
|
|
|
|
2023-04-11 02:05:53 +03:00
|
|
|
// TODO: error? warn? debug? trace?
|
2023-04-11 08:28:31 +03:00
|
|
|
if head_block_num.is_none() {
|
2023-03-01 23:56:00 +03:00
|
|
|
error!(
|
2023-06-29 20:15:12 +03:00
|
|
|
min=?min_block_needed,
|
|
|
|
max=?max_block_needed,
|
|
|
|
head=?head_block_num,
|
|
|
|
known=num_conns,
|
|
|
|
%method,
|
|
|
|
?params,
|
|
|
|
"No servers synced",
|
2023-03-01 23:56:00 +03:00
|
|
|
);
|
2023-05-13 01:15:32 +03:00
|
|
|
} else if head_block_num.as_ref() > needed {
|
2023-04-11 08:28:31 +03:00
|
|
|
// we have synced past the needed block
|
2023-07-06 09:13:32 +03:00
|
|
|
// TODO: log ranked rpcs
|
2023-06-29 20:15:12 +03:00
|
|
|
// TODO: only log params in development
|
2023-04-11 08:28:31 +03:00
|
|
|
error!(
|
2023-06-29 20:15:12 +03:00
|
|
|
min=?min_block_needed,
|
|
|
|
max=?max_block_needed,
|
|
|
|
head=?head_block_num,
|
|
|
|
known=%num_conns,
|
|
|
|
%method,
|
|
|
|
?params,
|
|
|
|
"No archive servers synced",
|
2023-04-11 08:28:31 +03:00
|
|
|
);
|
2023-01-24 20:36:07 +03:00
|
|
|
} else {
|
2023-06-29 20:15:12 +03:00
|
|
|
// TODO: only log params in development
|
2023-07-06 09:13:32 +03:00
|
|
|
// TODO: log ranked rpcs
|
2023-03-23 01:19:09 +03:00
|
|
|
error!(
|
2023-06-29 20:15:12 +03:00
|
|
|
min=?min_block_needed,
|
|
|
|
max=?max_block_needed,
|
|
|
|
head=?head_block_num,
|
|
|
|
skipped=%num_skipped,
|
|
|
|
known=%num_conns,
|
|
|
|
%method,
|
|
|
|
?params,
|
|
|
|
"Requested data is not available",
|
2023-01-25 09:45:20 +03:00
|
|
|
);
|
2023-01-24 20:36:07 +03:00
|
|
|
}
|
2023-03-01 23:56:00 +03:00
|
|
|
|
|
|
|
// TODO: what error code?
|
|
|
|
// cloudflare gives {"jsonrpc":"2.0","error":{"code":-32043,"message":"Requested data cannot be older than 128 blocks."},"id":1}
|
2023-05-31 02:32:34 +03:00
|
|
|
Err(JsonRpcErrorData {
|
2023-06-10 04:31:47 +03:00
|
|
|
message: "Requested data is not available".into(),
|
2023-05-13 21:13:02 +03:00
|
|
|
code: -32043,
|
|
|
|
data: None,
|
|
|
|
}
|
|
|
|
.into())
|
2022-05-29 04:23:58 +03:00
|
|
|
}
|
|
|
|
|
2022-07-02 04:20:28 +03:00
|
|
|
/// be sure there is a timeout on this or it might loop forever
|
2023-02-27 09:44:09 +03:00
|
|
|
#[allow(clippy::too_many_arguments)]
|
2023-05-31 02:32:34 +03:00
|
|
|
pub async fn try_send_all_synced_connections<P: JsonRpcParams>(
|
2023-04-24 21:00:12 +03:00
|
|
|
self: &Arc<Self>,
|
2023-05-31 02:32:34 +03:00
|
|
|
method: &str,
|
|
|
|
params: &P,
|
|
|
|
request_metadata: Option<&Arc<RequestMetadata>>,
|
2023-02-11 07:45:57 +03:00
|
|
|
min_block_needed: Option<&U64>,
|
|
|
|
max_block_needed: Option<&U64>,
|
2023-06-21 00:22:14 +03:00
|
|
|
max_wait: Option<Duration>,
|
2023-05-31 02:32:34 +03:00
|
|
|
error_level: Option<RequestErrorHandler>,
|
|
|
|
max_sends: Option<usize>,
|
|
|
|
) -> Web3ProxyResult<Box<RawValue>> {
|
2023-06-27 22:36:41 +03:00
|
|
|
let mut watch_consensus_rpcs = self.watch_ranked_rpcs.subscribe();
|
2023-03-01 23:56:00 +03:00
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
let start = Instant::now();
|
2023-05-13 01:15:32 +03:00
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
loop {
|
|
|
|
if let Some(max_wait) = max_wait {
|
|
|
|
if start.elapsed() > max_wait {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
2023-05-13 01:15:32 +03:00
|
|
|
|
2022-11-23 01:45:22 +03:00
|
|
|
match self
|
2023-02-09 22:47:56 +03:00
|
|
|
.all_connections(
|
2023-05-31 02:32:34 +03:00
|
|
|
request_metadata,
|
2023-02-11 07:45:57 +03:00
|
|
|
min_block_needed,
|
|
|
|
max_block_needed,
|
2023-05-31 02:32:34 +03:00
|
|
|
max_sends,
|
|
|
|
error_level,
|
2023-02-09 22:47:56 +03:00
|
|
|
)
|
2022-11-23 01:45:22 +03:00
|
|
|
.await
|
|
|
|
{
|
2022-05-28 07:26:24 +03:00
|
|
|
Ok(active_request_handles) => {
|
2022-10-12 00:31:34 +03:00
|
|
|
if let Some(request_metadata) = request_metadata {
|
2023-05-13 01:15:32 +03:00
|
|
|
let mut only_backups_used = true;
|
2023-01-20 08:46:47 +03:00
|
|
|
|
|
|
|
request_metadata.backend_requests.lock().extend(
|
|
|
|
active_request_handles.iter().map(|x| {
|
|
|
|
let rpc = x.clone_connection();
|
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
if !rpc.backup {
|
2023-01-20 08:46:47 +03:00
|
|
|
// TODO: its possible we serve from a synced connection though. think about this more
|
2023-05-13 01:15:32 +03:00
|
|
|
only_backups_used = false;
|
2023-01-20 08:46:47 +03:00
|
|
|
}
|
|
|
|
|
2023-07-11 01:41:55 +03:00
|
|
|
rpc
|
2023-01-20 08:46:47 +03:00
|
|
|
}),
|
|
|
|
);
|
|
|
|
|
2022-10-12 00:31:34 +03:00
|
|
|
request_metadata
|
2023-01-20 08:46:47 +03:00
|
|
|
.response_from_backup_rpc
|
2023-05-13 01:15:32 +03:00
|
|
|
.store(only_backups_used, Ordering::Release);
|
2022-10-12 00:31:34 +03:00
|
|
|
}
|
|
|
|
|
2023-05-31 02:32:34 +03:00
|
|
|
let x = self
|
|
|
|
.try_send_parallel_requests(active_request_handles, method, params)
|
|
|
|
.await?;
|
|
|
|
|
|
|
|
return Ok(x);
|
2022-05-28 07:26:24 +03:00
|
|
|
}
|
|
|
|
Err(None) => {
|
2023-03-01 23:56:00 +03:00
|
|
|
warn!(
|
2023-07-11 01:31:10 +03:00
|
|
|
?self,
|
|
|
|
?min_block_needed,
|
|
|
|
?max_block_needed,
|
|
|
|
"No servers in sync on! Retrying",
|
2023-03-01 23:56:00 +03:00
|
|
|
);
|
2022-05-29 04:23:58 +03:00
|
|
|
|
2022-10-12 00:31:34 +03:00
|
|
|
if let Some(request_metadata) = &request_metadata {
|
2023-03-01 23:56:00 +03:00
|
|
|
// TODO: if this times out, i think we drop this
|
2023-05-13 09:00:03 +03:00
|
|
|
request_metadata.no_servers.fetch_add(1, Ordering::AcqRel);
|
2022-10-12 00:31:34 +03:00
|
|
|
}
|
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
let max_sleep = if let Some(max_wait) = max_wait {
|
|
|
|
start + max_wait
|
|
|
|
} else {
|
|
|
|
break;
|
|
|
|
};
|
|
|
|
|
2023-05-31 02:32:34 +03:00
|
|
|
tokio::select! {
|
2023-06-21 00:22:14 +03:00
|
|
|
_ = sleep_until(max_sleep) => {
|
|
|
|
// rpcs didn't change and we have waited too long. break to return an error
|
2023-07-11 01:41:55 +03:00
|
|
|
warn!(?self, "timeout waiting for try_send_all_synced_connections!");
|
2023-06-21 00:22:14 +03:00
|
|
|
break;
|
|
|
|
},
|
2023-05-31 02:32:34 +03:00
|
|
|
_ = watch_consensus_rpcs.changed() => {
|
2023-06-21 00:22:14 +03:00
|
|
|
// consensus rpcs changed!
|
2023-05-31 02:32:34 +03:00
|
|
|
watch_consensus_rpcs.borrow_and_update();
|
2023-06-21 00:22:14 +03:00
|
|
|
// continue to try again
|
|
|
|
continue;
|
2023-05-31 02:32:34 +03:00
|
|
|
}
|
|
|
|
}
|
2022-05-28 07:26:24 +03:00
|
|
|
}
|
2022-08-07 09:48:57 +03:00
|
|
|
Err(Some(retry_at)) => {
|
2022-10-12 00:31:34 +03:00
|
|
|
if let Some(request_metadata) = &request_metadata {
|
2023-05-13 09:00:03 +03:00
|
|
|
request_metadata.no_servers.fetch_add(1, Ordering::AcqRel);
|
2022-10-12 00:31:34 +03:00
|
|
|
}
|
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
if let Some(max_wait) = max_wait {
|
|
|
|
if start.elapsed() > max_wait {
|
2023-07-11 01:41:55 +03:00
|
|
|
warn!(
|
|
|
|
?self,
|
|
|
|
"All rate limits exceeded. And sleeping would take too long"
|
|
|
|
);
|
2023-06-21 00:22:14 +03:00
|
|
|
break;
|
|
|
|
}
|
2023-05-31 02:32:34 +03:00
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
warn!("All rate limits exceeded. Sleeping");
|
2023-05-31 02:32:34 +03:00
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
// TODO: only make one of these sleep_untils
|
|
|
|
|
|
|
|
tokio::select! {
|
|
|
|
_ = sleep_until(start + max_wait) => {break}
|
|
|
|
_ = sleep_until(retry_at) => {}
|
|
|
|
_ = watch_consensus_rpcs.changed() => {
|
|
|
|
watch_consensus_rpcs.borrow_and_update();
|
|
|
|
}
|
2023-03-01 23:56:00 +03:00
|
|
|
}
|
2022-05-28 07:26:24 +03:00
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
continue;
|
|
|
|
} else {
|
2023-07-11 01:41:55 +03:00
|
|
|
warn!(?self, "all rate limits exceeded");
|
2023-06-21 00:22:14 +03:00
|
|
|
break;
|
|
|
|
}
|
2022-05-28 07:26:24 +03:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2023-05-13 01:15:32 +03:00
|
|
|
|
|
|
|
Err(Web3ProxyError::NoServersSynced)
|
2022-05-28 07:26:24 +03:00
|
|
|
}
|
2023-01-17 09:54:40 +03:00
|
|
|
|
2023-06-29 01:04:55 +03:00
|
|
|
#[allow(clippy::too_many_arguments)]
|
2023-05-31 02:32:34 +03:00
|
|
|
pub async fn try_proxy_connection<P: JsonRpcParams, R: JsonRpcResultData>(
|
2023-01-17 09:54:40 +03:00
|
|
|
&self,
|
2023-05-31 02:32:34 +03:00
|
|
|
method: &str,
|
|
|
|
params: &P,
|
2023-01-17 09:54:40 +03:00
|
|
|
request_metadata: Option<&Arc<RequestMetadata>>,
|
2023-06-29 01:04:55 +03:00
|
|
|
max_tries: Option<usize>,
|
2023-06-21 00:22:14 +03:00
|
|
|
max_wait: Option<Duration>,
|
2023-01-17 09:54:40 +03:00
|
|
|
min_block_needed: Option<&U64>,
|
2023-02-11 07:45:57 +03:00
|
|
|
max_block_needed: Option<&U64>,
|
2023-05-31 02:32:34 +03:00
|
|
|
) -> Web3ProxyResult<R> {
|
|
|
|
let proxy_mode = request_metadata.map(|x| x.proxy_mode()).unwrap_or_default();
|
|
|
|
|
|
|
|
match proxy_mode {
|
2023-03-03 04:39:50 +03:00
|
|
|
ProxyMode::Debug | ProxyMode::Best => {
|
2023-06-29 04:36:17 +03:00
|
|
|
self.request_with_metadata_and_retries(
|
2023-05-31 02:32:34 +03:00
|
|
|
method,
|
|
|
|
params,
|
2023-01-17 09:54:40 +03:00
|
|
|
request_metadata,
|
2023-06-29 01:04:55 +03:00
|
|
|
max_tries,
|
2023-06-21 00:22:14 +03:00
|
|
|
max_wait,
|
2023-01-17 09:54:40 +03:00
|
|
|
min_block_needed,
|
2023-02-11 07:45:57 +03:00
|
|
|
max_block_needed,
|
2023-01-17 09:54:40 +03:00
|
|
|
)
|
|
|
|
.await
|
|
|
|
}
|
2023-03-17 05:38:11 +03:00
|
|
|
ProxyMode::Fastest(_x) => todo!("Fastest"),
|
2023-01-17 09:54:40 +03:00
|
|
|
ProxyMode::Versus => todo!("Versus"),
|
|
|
|
}
|
|
|
|
}
|
2022-05-05 22:07:09 +03:00
|
|
|
}
|
2022-07-16 07:13:02 +03:00
|
|
|
|
2023-05-17 02:04:17 +03:00
|
|
|
impl Display for Web3Rpcs {
|
|
|
|
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
|
|
|
f.write_str(&self.name)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-02-06 20:55:27 +03:00
|
|
|
impl fmt::Debug for Web3Rpcs {
|
2022-08-24 02:13:56 +03:00
|
|
|
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
|
|
|
// TODO: the default formatter takes forever to write. this is too quiet though
|
2023-07-11 01:31:10 +03:00
|
|
|
let consensus_rpcs = self.watch_ranked_rpcs.borrow().is_some();
|
|
|
|
|
2023-02-06 20:55:27 +03:00
|
|
|
f.debug_struct("Web3Rpcs")
|
2023-02-15 04:41:40 +03:00
|
|
|
.field("rpcs", &self.by_name)
|
2023-07-11 01:31:10 +03:00
|
|
|
.field("consensus_rpcs", &consensus_rpcs)
|
2022-08-24 02:13:56 +03:00
|
|
|
.finish_non_exhaustive()
|
|
|
|
}
|
|
|
|
}
|
2022-08-24 02:56:47 +03:00
|
|
|
|
2023-07-20 09:09:39 +03:00
|
|
|
impl Serialize for Web3Rpcs {
|
|
|
|
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
|
|
|
|
where
|
|
|
|
S: Serializer,
|
|
|
|
{
|
2023-07-11 09:08:06 +03:00
|
|
|
let mut state = serializer.serialize_struct("Web3Rpcs", 5)?;
|
2022-09-22 02:50:55 +03:00
|
|
|
|
2023-02-27 07:00:13 +03:00
|
|
|
{
|
2023-07-20 09:09:39 +03:00
|
|
|
let by_name = self.by_name.read();
|
2023-07-11 04:09:58 +03:00
|
|
|
let rpcs: Vec<&Arc<Web3Rpc>> = by_name.values().collect();
|
2023-02-27 07:00:13 +03:00
|
|
|
// TODO: coordinate with frontend team to rename "conns" to "rpcs"
|
|
|
|
state.serialize_field("conns", &rpcs)?;
|
|
|
|
}
|
2022-09-22 02:50:55 +03:00
|
|
|
|
2023-01-23 09:02:08 +03:00
|
|
|
{
|
2023-06-27 22:36:41 +03:00
|
|
|
let consensus_rpcs = self.watch_ranked_rpcs.borrow().clone();
|
2023-02-27 09:44:09 +03:00
|
|
|
// TODO: rename synced_connections to consensus_rpcs
|
|
|
|
|
|
|
|
if let Some(consensus_rpcs) = consensus_rpcs.as_ref() {
|
|
|
|
state.serialize_field("synced_connections", consensus_rpcs)?;
|
|
|
|
} else {
|
|
|
|
state.serialize_field("synced_connections", &None::<()>)?;
|
|
|
|
}
|
2023-01-23 09:02:08 +03:00
|
|
|
}
|
2022-09-22 02:50:55 +03:00
|
|
|
|
2023-06-08 03:26:38 +03:00
|
|
|
state.serialize_field(
|
2023-06-21 20:00:05 +03:00
|
|
|
"caches",
|
2023-06-21 20:01:51 +03:00
|
|
|
&(
|
2023-06-21 20:00:05 +03:00
|
|
|
MokaCacheSerializer(&self.blocks_by_hash),
|
|
|
|
MokaCacheSerializer(&self.blocks_by_number),
|
2023-06-21 20:01:51 +03:00
|
|
|
),
|
2023-06-08 03:26:38 +03:00
|
|
|
)?;
|
2023-06-07 23:05:15 +03:00
|
|
|
|
|
|
|
state.serialize_field(
|
|
|
|
"watch_consensus_rpcs_receivers",
|
2023-06-27 22:36:41 +03:00
|
|
|
&self.watch_ranked_rpcs.receiver_count(),
|
2023-06-07 23:05:15 +03:00
|
|
|
)?;
|
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
if let Some(ref x) = self.watch_head_block {
|
2023-06-07 23:05:15 +03:00
|
|
|
state.serialize_field("watch_consensus_head_receivers", &x.receiver_count())?;
|
|
|
|
} else {
|
|
|
|
state.serialize_field("watch_consensus_head_receivers", &None::<()>)?;
|
|
|
|
}
|
2023-06-07 22:04:39 +03:00
|
|
|
|
2022-08-24 02:56:47 +03:00
|
|
|
state.end()
|
|
|
|
}
|
|
|
|
}
|
2022-11-23 01:45:22 +03:00
|
|
|
|
|
|
|
mod tests {
|
2022-12-03 08:31:03 +03:00
|
|
|
#![allow(unused_imports)]
|
2023-01-26 08:24:09 +03:00
|
|
|
|
2022-11-23 01:45:22 +03:00
|
|
|
use super::*;
|
2023-05-23 01:32:15 +03:00
|
|
|
use crate::rpcs::blockchain::Web3ProxyBlock;
|
2023-02-14 23:14:50 +03:00
|
|
|
use crate::rpcs::consensus::ConsensusFinder;
|
2023-05-13 01:15:32 +03:00
|
|
|
use arc_swap::ArcSwap;
|
2023-05-18 23:34:22 +03:00
|
|
|
use ethers::types::H256;
|
2022-12-01 01:11:14 +03:00
|
|
|
use ethers::types::{Block, U256};
|
2023-05-11 23:09:15 +03:00
|
|
|
use latency::PeakEwmaLatency;
|
2023-07-13 20:58:22 +03:00
|
|
|
use moka::future::{Cache, CacheBuilder};
|
2023-06-25 05:36:21 +03:00
|
|
|
use std::time::{SystemTime, UNIX_EPOCH};
|
2023-06-24 02:28:45 +03:00
|
|
|
use tracing::trace;
|
2022-11-23 01:45:22 +03:00
|
|
|
|
2023-05-11 23:09:15 +03:00
|
|
|
#[cfg(test)]
|
|
|
|
fn new_peak_latency() -> PeakEwmaLatency {
|
2023-05-23 01:43:39 +03:00
|
|
|
PeakEwmaLatency::spawn(Duration::from_secs(1), 4, Duration::from_secs(1))
|
2023-05-11 23:09:15 +03:00
|
|
|
}
|
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
#[test_log::test(tokio::test)]
|
2023-02-03 01:48:23 +03:00
|
|
|
async fn test_sort_connections_by_sync_status() {
|
|
|
|
let block_0 = Block {
|
|
|
|
number: Some(0.into()),
|
|
|
|
hash: Some(H256::random()),
|
|
|
|
..Default::default()
|
|
|
|
};
|
|
|
|
let block_1 = Block {
|
|
|
|
number: Some(1.into()),
|
|
|
|
hash: Some(H256::random()),
|
|
|
|
parent_hash: block_0.hash.unwrap(),
|
|
|
|
..Default::default()
|
|
|
|
};
|
|
|
|
let block_2 = Block {
|
|
|
|
number: Some(2.into()),
|
|
|
|
hash: Some(H256::random()),
|
|
|
|
parent_hash: block_1.hash.unwrap(),
|
|
|
|
..Default::default()
|
|
|
|
};
|
|
|
|
|
|
|
|
let blocks: Vec<_> = [block_0, block_1, block_2]
|
|
|
|
.into_iter()
|
2023-02-15 04:41:40 +03:00
|
|
|
.map(|x| Web3ProxyBlock::try_new(Arc::new(x)).unwrap())
|
2023-02-03 01:48:23 +03:00
|
|
|
.collect();
|
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
let (tx_a, _) = watch::channel(None);
|
|
|
|
let (tx_b, _) = watch::channel(blocks.get(1).cloned());
|
|
|
|
let (tx_c, _) = watch::channel(blocks.get(2).cloned());
|
|
|
|
let (tx_d, _) = watch::channel(None);
|
|
|
|
let (tx_e, _) = watch::channel(blocks.get(1).cloned());
|
|
|
|
let (tx_f, _) = watch::channel(blocks.get(2).cloned());
|
|
|
|
|
2023-02-11 07:24:20 +03:00
|
|
|
let mut rpcs: Vec<_> = [
|
2023-02-06 20:55:27 +03:00
|
|
|
Web3Rpc {
|
2023-02-03 01:48:23 +03:00
|
|
|
name: "a".to_string(),
|
2023-06-09 23:09:58 +03:00
|
|
|
tier: 0.into(),
|
2023-06-13 20:00:08 +03:00
|
|
|
head_block: Some(tx_a),
|
2023-05-11 23:09:15 +03:00
|
|
|
peak_latency: Some(new_peak_latency()),
|
2023-02-03 01:48:23 +03:00
|
|
|
..Default::default()
|
|
|
|
},
|
2023-02-06 20:55:27 +03:00
|
|
|
Web3Rpc {
|
2023-02-03 01:48:23 +03:00
|
|
|
name: "b".to_string(),
|
2023-06-09 23:09:58 +03:00
|
|
|
tier: 0.into(),
|
2023-06-13 20:00:08 +03:00
|
|
|
head_block: Some(tx_b),
|
2023-05-11 23:09:15 +03:00
|
|
|
peak_latency: Some(new_peak_latency()),
|
2023-02-03 01:48:23 +03:00
|
|
|
..Default::default()
|
|
|
|
},
|
2023-02-06 20:55:27 +03:00
|
|
|
Web3Rpc {
|
2023-02-03 01:48:23 +03:00
|
|
|
name: "c".to_string(),
|
2023-06-09 23:09:58 +03:00
|
|
|
tier: 0.into(),
|
2023-06-13 20:00:08 +03:00
|
|
|
head_block: Some(tx_c),
|
2023-05-11 23:09:15 +03:00
|
|
|
peak_latency: Some(new_peak_latency()),
|
2023-02-03 01:48:23 +03:00
|
|
|
..Default::default()
|
|
|
|
},
|
2023-02-06 20:55:27 +03:00
|
|
|
Web3Rpc {
|
2023-02-03 01:48:23 +03:00
|
|
|
name: "d".to_string(),
|
2023-06-09 23:09:58 +03:00
|
|
|
tier: 1.into(),
|
2023-06-13 20:00:08 +03:00
|
|
|
head_block: Some(tx_d),
|
2023-05-11 23:09:15 +03:00
|
|
|
peak_latency: Some(new_peak_latency()),
|
2023-02-03 01:48:23 +03:00
|
|
|
..Default::default()
|
|
|
|
},
|
2023-02-06 20:55:27 +03:00
|
|
|
Web3Rpc {
|
2023-02-03 01:48:23 +03:00
|
|
|
name: "e".to_string(),
|
2023-06-09 23:09:58 +03:00
|
|
|
tier: 1.into(),
|
2023-06-13 20:00:08 +03:00
|
|
|
head_block: Some(tx_e),
|
2023-05-11 23:09:15 +03:00
|
|
|
peak_latency: Some(new_peak_latency()),
|
2023-02-03 01:48:23 +03:00
|
|
|
..Default::default()
|
|
|
|
},
|
2023-02-06 20:55:27 +03:00
|
|
|
Web3Rpc {
|
2023-02-03 01:48:23 +03:00
|
|
|
name: "f".to_string(),
|
2023-06-09 23:09:58 +03:00
|
|
|
tier: 1.into(),
|
2023-06-13 20:00:08 +03:00
|
|
|
head_block: Some(tx_f),
|
2023-05-11 23:09:15 +03:00
|
|
|
peak_latency: Some(new_peak_latency()),
|
2023-02-03 01:48:23 +03:00
|
|
|
..Default::default()
|
|
|
|
},
|
|
|
|
]
|
|
|
|
.into_iter()
|
|
|
|
.map(Arc::new)
|
|
|
|
.collect();
|
|
|
|
|
2023-06-09 23:09:58 +03:00
|
|
|
rpcs.sort_by_cached_key(|x| x.sort_for_load_balancing_on(None));
|
2023-02-03 01:48:23 +03:00
|
|
|
|
|
|
|
let names_in_sort_order: Vec<_> = rpcs.iter().map(|x| x.name.as_str()).collect();
|
|
|
|
|
2023-06-24 10:41:30 +03:00
|
|
|
// assert_eq!(names_in_sort_order, ["c", "b", "a", "f", "e", "d"]);
|
|
|
|
assert_eq!(names_in_sort_order, ["c", "f", "b", "e", "a", "d"]);
|
2023-02-03 01:48:23 +03:00
|
|
|
}
|
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
#[test_log::test(tokio::test)]
|
2022-11-25 10:41:53 +03:00
|
|
|
async fn test_server_selection_by_height() {
|
2023-01-26 08:24:09 +03:00
|
|
|
let now = chrono::Utc::now().timestamp().into();
|
2022-12-01 01:11:14 +03:00
|
|
|
|
2022-11-23 01:45:22 +03:00
|
|
|
let lagged_block = Block {
|
|
|
|
hash: Some(H256::random()),
|
|
|
|
number: Some(0.into()),
|
2022-12-01 01:11:14 +03:00
|
|
|
timestamp: now - 1,
|
2022-11-23 01:45:22 +03:00
|
|
|
..Default::default()
|
|
|
|
};
|
|
|
|
|
|
|
|
let head_block = Block {
|
|
|
|
hash: Some(H256::random()),
|
|
|
|
number: Some(1.into()),
|
|
|
|
parent_hash: lagged_block.hash.unwrap(),
|
2022-12-01 01:11:14 +03:00
|
|
|
timestamp: now,
|
2022-11-23 01:45:22 +03:00
|
|
|
..Default::default()
|
|
|
|
};
|
|
|
|
|
|
|
|
let lagged_block = Arc::new(lagged_block);
|
|
|
|
let head_block = Arc::new(head_block);
|
|
|
|
|
|
|
|
let block_data_limit = u64::MAX;
|
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
let (tx_synced, _) = watch::channel(None);
|
|
|
|
|
2023-02-06 20:55:27 +03:00
|
|
|
let head_rpc = Web3Rpc {
|
2022-11-23 01:45:22 +03:00
|
|
|
name: "synced".to_string(),
|
|
|
|
soft_limit: 1_000,
|
2023-02-06 20:55:27 +03:00
|
|
|
automatic_block_limit: false,
|
2023-01-19 13:13:00 +03:00
|
|
|
backup: false,
|
2022-11-23 01:45:22 +03:00
|
|
|
block_data_limit: block_data_limit.into(),
|
2023-06-13 20:00:08 +03:00
|
|
|
head_block: Some(tx_synced),
|
2023-05-11 23:09:15 +03:00
|
|
|
peak_latency: Some(new_peak_latency()),
|
2023-02-06 20:55:27 +03:00
|
|
|
..Default::default()
|
2022-11-23 01:45:22 +03:00
|
|
|
};
|
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
let (tx_lagged, _) = watch::channel(None);
|
|
|
|
|
2023-02-06 20:55:27 +03:00
|
|
|
let lagged_rpc = Web3Rpc {
|
2022-11-23 01:45:22 +03:00
|
|
|
name: "lagged".to_string(),
|
|
|
|
soft_limit: 1_000,
|
2022-12-06 00:13:36 +03:00
|
|
|
automatic_block_limit: false,
|
2023-01-19 13:13:00 +03:00
|
|
|
backup: false,
|
2022-11-23 01:45:22 +03:00
|
|
|
block_data_limit: block_data_limit.into(),
|
2023-06-13 20:00:08 +03:00
|
|
|
head_block: Some(tx_lagged),
|
2023-05-11 23:09:15 +03:00
|
|
|
peak_latency: Some(new_peak_latency()),
|
2023-02-06 20:55:27 +03:00
|
|
|
..Default::default()
|
2022-11-23 01:45:22 +03:00
|
|
|
};
|
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
assert!(!head_rpc.has_block_data(lagged_block.number.as_ref().unwrap()));
|
|
|
|
assert!(!head_rpc.has_block_data(head_block.number.as_ref().unwrap()));
|
2022-11-23 01:45:22 +03:00
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
assert!(!lagged_rpc.has_block_data(lagged_block.number.as_ref().unwrap()));
|
|
|
|
assert!(!lagged_rpc.has_block_data(head_block.number.as_ref().unwrap()));
|
2022-11-23 01:45:22 +03:00
|
|
|
|
|
|
|
let head_rpc = Arc::new(head_rpc);
|
|
|
|
let lagged_rpc = Arc::new(lagged_rpc);
|
|
|
|
|
2023-07-11 09:08:06 +03:00
|
|
|
let (block_sender, _block_receiver) = mpsc::unbounded_channel();
|
2023-06-27 22:36:41 +03:00
|
|
|
let (watch_ranked_rpcs, _watch_consensus_rpcs_receiver) = watch::channel(None);
|
2023-05-13 01:15:32 +03:00
|
|
|
let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None);
|
2023-01-23 09:02:08 +03:00
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
let chain_id = 1;
|
|
|
|
|
2023-07-11 04:09:58 +03:00
|
|
|
let mut by_name = HashMap::new();
|
|
|
|
by_name.insert(head_rpc.name.clone(), head_rpc.clone());
|
|
|
|
by_name.insert(lagged_rpc.name.clone(), lagged_rpc.clone());
|
2023-07-10 21:05:07 +03:00
|
|
|
|
2023-02-06 20:55:27 +03:00
|
|
|
// TODO: make a Web3Rpcs::new
|
2023-02-15 04:41:40 +03:00
|
|
|
let rpcs = Web3Rpcs {
|
2023-05-13 01:15:32 +03:00
|
|
|
block_sender: block_sender.clone(),
|
2023-07-11 04:09:58 +03:00
|
|
|
by_name: RwLock::new(by_name),
|
2023-06-21 00:22:14 +03:00
|
|
|
chain_id,
|
2023-07-15 04:35:40 +03:00
|
|
|
name: "test".into(),
|
2023-06-27 22:36:41 +03:00
|
|
|
watch_head_block: Some(watch_consensus_head_sender),
|
|
|
|
watch_ranked_rpcs,
|
2023-06-08 03:26:38 +03:00
|
|
|
blocks_by_hash: CacheBuilder::new(100)
|
|
|
|
.time_to_live(Duration::from_secs(60))
|
|
|
|
.build(),
|
|
|
|
blocks_by_number: CacheBuilder::new(100)
|
|
|
|
.time_to_live(Duration::from_secs(60))
|
|
|
|
.build(),
|
2023-06-17 19:19:05 +03:00
|
|
|
// TODO: test max_head_block_age?
|
2023-06-17 20:11:48 +03:00
|
|
|
max_head_block_age: Duration::from_secs(60),
|
|
|
|
// TODO: test max_head_block_lag?
|
|
|
|
max_head_block_lag: 5.into(),
|
2023-06-09 23:30:00 +03:00
|
|
|
min_synced_rpcs: 1,
|
2022-11-23 01:45:22 +03:00
|
|
|
min_sum_soft_limit: 1,
|
|
|
|
};
|
|
|
|
|
2023-02-27 07:00:13 +03:00
|
|
|
let mut consensus_finder = ConsensusFinder::new(None, None);
|
2022-11-23 01:45:22 +03:00
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
consensus_finder
|
2023-07-13 20:58:22 +03:00
|
|
|
.process_block_from_rpc(&rpcs, None, lagged_rpc.clone())
|
2023-06-21 00:22:14 +03:00
|
|
|
.await
|
|
|
|
.expect(
|
|
|
|
"its lagged, but it should still be seen as consensus if its the first to report",
|
|
|
|
);
|
|
|
|
consensus_finder
|
2023-07-13 20:58:22 +03:00
|
|
|
.process_block_from_rpc(&rpcs, None, head_rpc.clone())
|
2023-06-21 00:22:14 +03:00
|
|
|
.await
|
|
|
|
.unwrap();
|
2022-11-23 01:45:22 +03:00
|
|
|
|
|
|
|
// no head block because the rpcs haven't communicated through their channels
|
2023-02-15 04:41:40 +03:00
|
|
|
assert!(rpcs.head_block_hash().is_none());
|
2022-11-23 01:45:22 +03:00
|
|
|
|
2023-02-09 22:47:56 +03:00
|
|
|
// all_backend_connections gives all non-backup servers regardless of sync status
|
2022-11-23 01:45:22 +03:00
|
|
|
assert_eq!(
|
2023-06-27 22:36:41 +03:00
|
|
|
rpcs.all_connections(None, None, None, None, None)
|
2022-11-23 01:45:22 +03:00
|
|
|
.await
|
|
|
|
.unwrap()
|
|
|
|
.len(),
|
|
|
|
2
|
|
|
|
);
|
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
// best_synced_backend_connection which servers to be synced with the head block should not find any nodes
|
2023-02-15 04:41:40 +03:00
|
|
|
let x = rpcs
|
2023-05-19 08:43:07 +03:00
|
|
|
.wait_for_best_rpc(
|
2023-05-13 01:15:32 +03:00
|
|
|
None,
|
2023-05-18 23:51:28 +03:00
|
|
|
&mut vec![],
|
2023-05-13 01:15:32 +03:00
|
|
|
Some(head_block.number.as_ref().unwrap()),
|
|
|
|
None,
|
2023-05-24 06:46:27 +03:00
|
|
|
Some(Duration::from_secs(0)),
|
2023-05-31 02:32:34 +03:00
|
|
|
Some(RequestErrorHandler::DebugLevel),
|
2023-05-13 01:15:32 +03:00
|
|
|
)
|
2022-11-28 09:52:16 +03:00
|
|
|
.await
|
|
|
|
.unwrap();
|
|
|
|
|
2023-06-30 07:28:31 +03:00
|
|
|
info!(?x);
|
2022-11-28 09:52:16 +03:00
|
|
|
|
2023-02-15 04:41:40 +03:00
|
|
|
assert!(matches!(x, OpenRequestResult::NotReady));
|
2022-11-23 01:45:22 +03:00
|
|
|
|
2023-02-15 04:41:40 +03:00
|
|
|
// add lagged blocks to the rpcs. both servers should be allowed
|
2023-05-13 01:15:32 +03:00
|
|
|
lagged_rpc
|
|
|
|
.send_head_block_result(
|
|
|
|
Ok(Some(lagged_block.clone())),
|
|
|
|
&block_sender,
|
2023-05-24 00:40:34 +03:00
|
|
|
&rpcs.blocks_by_hash,
|
2023-05-13 01:15:32 +03:00
|
|
|
)
|
|
|
|
.await
|
|
|
|
.unwrap();
|
2022-11-23 01:45:22 +03:00
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
// TODO: calling process_block_from_rpc and send_head_block_result seperate seems very fragile
|
|
|
|
consensus_finder
|
|
|
|
.process_block_from_rpc(
|
|
|
|
&rpcs,
|
|
|
|
Some(lagged_block.clone().try_into().unwrap()),
|
|
|
|
lagged_rpc.clone(),
|
|
|
|
)
|
|
|
|
.await
|
|
|
|
.unwrap();
|
2023-05-13 01:15:32 +03:00
|
|
|
|
|
|
|
head_rpc
|
|
|
|
.send_head_block_result(
|
|
|
|
Ok(Some(lagged_block.clone())),
|
|
|
|
&block_sender,
|
2023-05-24 00:40:34 +03:00
|
|
|
&rpcs.blocks_by_hash,
|
2023-05-13 01:15:32 +03:00
|
|
|
)
|
|
|
|
.await
|
|
|
|
.unwrap();
|
|
|
|
|
|
|
|
// TODO: this is fragile
|
2023-06-21 00:22:14 +03:00
|
|
|
consensus_finder
|
|
|
|
.process_block_from_rpc(
|
|
|
|
&rpcs,
|
|
|
|
Some(lagged_block.clone().try_into().unwrap()),
|
|
|
|
head_rpc.clone(),
|
|
|
|
)
|
|
|
|
.await
|
|
|
|
.unwrap();
|
2022-11-23 01:45:22 +03:00
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
// TODO: how do we spawn this and wait for it to process things? subscribe and watch consensus connections?
|
2023-07-13 20:58:22 +03:00
|
|
|
// rpcs.process_incoming_blocks(block_receiver, pending_tx_sender)
|
2023-05-13 01:15:32 +03:00
|
|
|
|
|
|
|
assert!(head_rpc.has_block_data(lagged_block.number.as_ref().unwrap()));
|
|
|
|
assert!(!head_rpc.has_block_data(head_block.number.as_ref().unwrap()));
|
|
|
|
|
|
|
|
assert!(lagged_rpc.has_block_data(lagged_block.number.as_ref().unwrap()));
|
|
|
|
assert!(!lagged_rpc.has_block_data(head_block.number.as_ref().unwrap()));
|
|
|
|
|
2023-02-15 04:41:40 +03:00
|
|
|
assert_eq!(rpcs.num_synced_rpcs(), 2);
|
2022-11-23 01:45:22 +03:00
|
|
|
|
2023-02-15 04:41:40 +03:00
|
|
|
// add head block to the rpcs. lagged_rpc should not be available
|
2023-05-13 01:15:32 +03:00
|
|
|
head_rpc
|
|
|
|
.send_head_block_result(
|
|
|
|
Ok(Some(head_block.clone())),
|
|
|
|
&block_sender,
|
2023-05-24 00:40:34 +03:00
|
|
|
&rpcs.blocks_by_hash,
|
2023-05-13 01:15:32 +03:00
|
|
|
)
|
|
|
|
.await
|
|
|
|
.unwrap();
|
2022-11-23 01:45:22 +03:00
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
// TODO: this is fragile
|
2023-06-21 00:22:14 +03:00
|
|
|
consensus_finder
|
|
|
|
.process_block_from_rpc(
|
|
|
|
&rpcs,
|
|
|
|
Some(head_block.clone().try_into().unwrap()),
|
|
|
|
head_rpc.clone(),
|
|
|
|
)
|
|
|
|
.await
|
|
|
|
.unwrap();
|
2022-11-23 01:45:22 +03:00
|
|
|
|
2023-02-15 04:41:40 +03:00
|
|
|
assert_eq!(rpcs.num_synced_rpcs(), 1);
|
2022-11-23 01:45:22 +03:00
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
assert!(head_rpc.has_block_data(lagged_block.number.as_ref().unwrap()));
|
|
|
|
assert!(head_rpc.has_block_data(head_block.number.as_ref().unwrap()));
|
|
|
|
|
|
|
|
assert!(lagged_rpc.has_block_data(lagged_block.number.as_ref().unwrap()));
|
|
|
|
assert!(!lagged_rpc.has_block_data(head_block.number.as_ref().unwrap()));
|
|
|
|
|
|
|
|
// TODO: make sure the handle is for the expected rpc
|
2022-11-25 10:41:53 +03:00
|
|
|
assert!(matches!(
|
2023-05-24 06:46:27 +03:00
|
|
|
rpcs.wait_for_best_rpc(
|
|
|
|
None,
|
|
|
|
&mut vec![],
|
|
|
|
None,
|
|
|
|
None,
|
2023-05-31 02:32:34 +03:00
|
|
|
Some(Duration::from_secs(0)),
|
|
|
|
None,
|
2023-05-24 06:46:27 +03:00
|
|
|
)
|
|
|
|
.await,
|
2022-11-25 10:41:53 +03:00
|
|
|
Ok(OpenRequestResult::Handle(_))
|
|
|
|
));
|
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
// TODO: make sure the handle is for the expected rpc
|
2022-11-25 10:41:53 +03:00
|
|
|
assert!(matches!(
|
2023-05-24 06:46:27 +03:00
|
|
|
rpcs.wait_for_best_rpc(
|
|
|
|
None,
|
|
|
|
&mut vec![],
|
|
|
|
Some(&0.into()),
|
|
|
|
None,
|
|
|
|
Some(Duration::from_secs(0)),
|
2023-05-31 02:32:34 +03:00
|
|
|
None,
|
2023-05-24 06:46:27 +03:00
|
|
|
)
|
|
|
|
.await,
|
2022-11-25 10:41:53 +03:00
|
|
|
Ok(OpenRequestResult::Handle(_))
|
|
|
|
));
|
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
// TODO: make sure the handle is for the expected rpc
|
2022-11-25 10:41:53 +03:00
|
|
|
assert!(matches!(
|
2023-05-24 06:46:27 +03:00
|
|
|
rpcs.wait_for_best_rpc(
|
|
|
|
None,
|
|
|
|
&mut vec![],
|
|
|
|
Some(&1.into()),
|
|
|
|
None,
|
|
|
|
Some(Duration::from_secs(0)),
|
2023-05-31 02:32:34 +03:00
|
|
|
None,
|
2023-05-24 06:46:27 +03:00
|
|
|
)
|
|
|
|
.await,
|
2022-11-25 10:41:53 +03:00
|
|
|
Ok(OpenRequestResult::Handle(_))
|
|
|
|
));
|
|
|
|
|
|
|
|
// future block should not get a handle
|
2023-02-16 08:05:41 +03:00
|
|
|
let future_rpc = rpcs
|
2023-05-24 06:46:27 +03:00
|
|
|
.wait_for_best_rpc(
|
|
|
|
None,
|
|
|
|
&mut vec![],
|
|
|
|
Some(&2.into()),
|
|
|
|
None,
|
|
|
|
Some(Duration::from_secs(0)),
|
2023-05-31 02:32:34 +03:00
|
|
|
None,
|
2023-05-24 06:46:27 +03:00
|
|
|
)
|
2023-02-16 08:05:41 +03:00
|
|
|
.await;
|
|
|
|
assert!(matches!(future_rpc, Ok(OpenRequestResult::NotReady)));
|
2022-11-25 10:41:53 +03:00
|
|
|
}
|
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
#[test_log::test(tokio::test)]
|
2022-11-25 10:41:53 +03:00
|
|
|
async fn test_server_selection_by_archive() {
|
2023-01-26 08:24:09 +03:00
|
|
|
let now = chrono::Utc::now().timestamp().into();
|
2022-12-01 01:11:14 +03:00
|
|
|
|
2022-12-03 08:31:03 +03:00
|
|
|
let head_block = Block {
|
2022-11-25 10:41:53 +03:00
|
|
|
hash: Some(H256::random()),
|
|
|
|
number: Some(1_000_000.into()),
|
|
|
|
parent_hash: H256::random(),
|
2022-12-01 01:11:14 +03:00
|
|
|
timestamp: now,
|
2022-11-25 10:41:53 +03:00
|
|
|
..Default::default()
|
|
|
|
};
|
|
|
|
|
2023-02-15 04:41:40 +03:00
|
|
|
let head_block: Web3ProxyBlock = Arc::new(head_block).try_into().unwrap();
|
2022-11-25 10:41:53 +03:00
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
let (tx_pruned, _) = watch::channel(Some(head_block.clone()));
|
|
|
|
|
2023-02-06 20:55:27 +03:00
|
|
|
let pruned_rpc = Web3Rpc {
|
2022-11-25 10:41:53 +03:00
|
|
|
name: "pruned".to_string(),
|
|
|
|
soft_limit: 3_000,
|
2022-12-06 00:13:36 +03:00
|
|
|
automatic_block_limit: false,
|
2023-01-19 13:13:00 +03:00
|
|
|
backup: false,
|
2022-11-25 10:41:53 +03:00
|
|
|
block_data_limit: 64.into(),
|
2023-06-27 22:36:41 +03:00
|
|
|
tier: 1.into(),
|
2023-06-13 20:00:08 +03:00
|
|
|
head_block: Some(tx_pruned),
|
2023-02-06 20:55:27 +03:00
|
|
|
..Default::default()
|
2022-11-25 10:41:53 +03:00
|
|
|
};
|
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
let (tx_archive, _) = watch::channel(Some(head_block.clone()));
|
|
|
|
|
2023-02-06 20:55:27 +03:00
|
|
|
let archive_rpc = Web3Rpc {
|
2022-11-25 10:41:53 +03:00
|
|
|
name: "archive".to_string(),
|
|
|
|
soft_limit: 1_000,
|
2022-12-06 00:13:36 +03:00
|
|
|
automatic_block_limit: false,
|
2023-01-19 13:13:00 +03:00
|
|
|
backup: false,
|
2022-11-25 10:41:53 +03:00
|
|
|
block_data_limit: u64::MAX.into(),
|
2023-06-27 22:36:41 +03:00
|
|
|
tier: 2.into(),
|
2023-06-13 20:00:08 +03:00
|
|
|
head_block: Some(tx_archive),
|
2023-02-06 20:55:27 +03:00
|
|
|
..Default::default()
|
2022-11-25 10:41:53 +03:00
|
|
|
};
|
|
|
|
|
2023-02-27 09:44:09 +03:00
|
|
|
assert!(pruned_rpc.has_block_data(head_block.number()));
|
|
|
|
assert!(archive_rpc.has_block_data(head_block.number()));
|
2022-11-25 10:41:53 +03:00
|
|
|
assert!(!pruned_rpc.has_block_data(&1.into()));
|
|
|
|
assert!(archive_rpc.has_block_data(&1.into()));
|
|
|
|
|
|
|
|
let pruned_rpc = Arc::new(pruned_rpc);
|
|
|
|
let archive_rpc = Arc::new(archive_rpc);
|
|
|
|
|
2023-07-11 09:08:06 +03:00
|
|
|
let (block_sender, _) = mpsc::unbounded_channel();
|
2023-06-27 22:36:41 +03:00
|
|
|
let (watch_ranked_rpcs, _) = watch::channel(None);
|
2023-05-13 01:15:32 +03:00
|
|
|
let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None);
|
2023-01-23 09:02:08 +03:00
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
let chain_id = 1;
|
|
|
|
|
2023-07-11 04:09:58 +03:00
|
|
|
let mut by_name = HashMap::new();
|
|
|
|
by_name.insert(pruned_rpc.name.clone(), pruned_rpc.clone());
|
|
|
|
by_name.insert(archive_rpc.name.clone(), archive_rpc.clone());
|
2023-07-10 21:05:07 +03:00
|
|
|
|
2023-02-15 04:41:40 +03:00
|
|
|
let rpcs = Web3Rpcs {
|
2023-02-26 10:52:33 +03:00
|
|
|
block_sender,
|
2023-07-11 04:09:58 +03:00
|
|
|
by_name: RwLock::new(by_name),
|
2023-06-21 00:22:14 +03:00
|
|
|
chain_id,
|
2023-07-15 04:35:40 +03:00
|
|
|
name: "test".into(),
|
2023-06-27 22:36:41 +03:00
|
|
|
watch_head_block: Some(watch_consensus_head_sender),
|
|
|
|
watch_ranked_rpcs,
|
2023-06-08 03:26:38 +03:00
|
|
|
blocks_by_hash: CacheBuilder::new(100)
|
|
|
|
.time_to_live(Duration::from_secs(120))
|
|
|
|
.build(),
|
|
|
|
blocks_by_number: CacheBuilder::new(100)
|
|
|
|
.time_to_live(Duration::from_secs(120))
|
|
|
|
.build(),
|
2023-06-09 23:30:00 +03:00
|
|
|
min_synced_rpcs: 1,
|
2023-02-15 04:41:40 +03:00
|
|
|
min_sum_soft_limit: 4_000,
|
2023-06-17 20:11:48 +03:00
|
|
|
max_head_block_age: Duration::from_secs(60),
|
|
|
|
max_head_block_lag: 5.into(),
|
2022-11-25 10:41:53 +03:00
|
|
|
};
|
|
|
|
|
2023-02-27 07:00:13 +03:00
|
|
|
let mut connection_heads = ConsensusFinder::new(None, None);
|
2022-11-25 10:41:53 +03:00
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
// min sum soft limit will require 2 servers
|
|
|
|
let x = connection_heads
|
2023-07-13 20:58:22 +03:00
|
|
|
.process_block_from_rpc(&rpcs, Some(head_block.clone()), pruned_rpc.clone())
|
2023-06-21 00:22:14 +03:00
|
|
|
.await
|
2023-06-27 22:36:41 +03:00
|
|
|
.unwrap();
|
|
|
|
assert!(!x);
|
|
|
|
|
|
|
|
assert_eq!(rpcs.num_synced_rpcs(), 0);
|
2023-06-21 00:22:14 +03:00
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
let x = connection_heads
|
2023-07-13 20:58:22 +03:00
|
|
|
.process_block_from_rpc(&rpcs, Some(head_block.clone()), archive_rpc.clone())
|
2023-06-21 00:22:14 +03:00
|
|
|
.await
|
|
|
|
.unwrap();
|
2023-06-27 22:36:41 +03:00
|
|
|
assert!(x);
|
2022-11-25 10:41:53 +03:00
|
|
|
|
2023-02-15 04:41:40 +03:00
|
|
|
assert_eq!(rpcs.num_synced_rpcs(), 2);
|
2022-11-25 10:41:53 +03:00
|
|
|
|
|
|
|
// best_synced_backend_connection requires servers to be synced with the head block
|
2023-02-11 07:45:57 +03:00
|
|
|
// TODO: test with and without passing the head_block.number?
|
2023-02-16 08:05:41 +03:00
|
|
|
let best_available_server = rpcs
|
2023-05-19 08:43:07 +03:00
|
|
|
.wait_for_best_rpc(
|
2023-05-18 23:51:28 +03:00
|
|
|
None,
|
|
|
|
&mut vec![],
|
|
|
|
Some(head_block.number()),
|
|
|
|
None,
|
2023-05-24 06:46:27 +03:00
|
|
|
Some(Duration::from_secs(0)),
|
2023-05-31 02:32:34 +03:00
|
|
|
None,
|
2023-05-18 23:51:28 +03:00
|
|
|
)
|
2022-11-25 10:41:53 +03:00
|
|
|
.await;
|
|
|
|
|
2023-02-16 08:05:41 +03:00
|
|
|
debug!("best_available_server: {:#?}", best_available_server);
|
2023-02-15 04:41:40 +03:00
|
|
|
|
2022-11-25 10:41:53 +03:00
|
|
|
assert!(matches!(
|
2023-02-16 08:05:41 +03:00
|
|
|
best_available_server.unwrap(),
|
2022-11-25 10:41:53 +03:00
|
|
|
OpenRequestResult::Handle(_)
|
|
|
|
));
|
|
|
|
|
2023-03-20 01:50:25 +03:00
|
|
|
let _best_available_server_from_none = rpcs
|
2023-05-24 06:46:27 +03:00
|
|
|
.wait_for_best_rpc(
|
|
|
|
None,
|
|
|
|
&mut vec![],
|
|
|
|
None,
|
|
|
|
None,
|
|
|
|
Some(Duration::from_secs(0)),
|
2023-05-31 02:32:34 +03:00
|
|
|
None,
|
2023-05-24 06:46:27 +03:00
|
|
|
)
|
2023-02-16 08:05:41 +03:00
|
|
|
.await;
|
|
|
|
|
|
|
|
// assert_eq!(best_available_server, best_available_server_from_none);
|
|
|
|
|
2023-02-15 04:41:40 +03:00
|
|
|
let best_archive_server = rpcs
|
2023-05-24 06:46:27 +03:00
|
|
|
.wait_for_best_rpc(
|
|
|
|
None,
|
|
|
|
&mut vec![],
|
|
|
|
Some(&1.into()),
|
|
|
|
None,
|
|
|
|
Some(Duration::from_secs(0)),
|
2023-05-31 02:32:34 +03:00
|
|
|
None,
|
2023-05-24 06:46:27 +03:00
|
|
|
)
|
2022-11-25 10:41:53 +03:00
|
|
|
.await;
|
|
|
|
|
|
|
|
match best_archive_server {
|
|
|
|
Ok(OpenRequestResult::Handle(x)) => {
|
|
|
|
assert_eq!(x.clone_connection().name, "archive".to_string())
|
|
|
|
}
|
|
|
|
x => {
|
|
|
|
error!("unexpected result: {:?}", x);
|
|
|
|
}
|
|
|
|
}
|
2022-11-23 01:45:22 +03:00
|
|
|
}
|
2023-03-02 00:35:50 +03:00
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
#[test_log::test(tokio::test)]
|
2023-03-02 01:20:31 +03:00
|
|
|
async fn test_all_connections() {
|
|
|
|
// TODO: use chrono, not SystemTime
|
|
|
|
let now: U256 = SystemTime::now()
|
|
|
|
.duration_since(UNIX_EPOCH)
|
|
|
|
.unwrap()
|
|
|
|
.as_secs()
|
|
|
|
.into();
|
|
|
|
|
|
|
|
let block_1 = Block {
|
|
|
|
hash: Some(H256::random()),
|
|
|
|
number: Some(1_000_000.into()),
|
|
|
|
parent_hash: H256::random(),
|
|
|
|
timestamp: now,
|
|
|
|
..Default::default()
|
|
|
|
};
|
|
|
|
let block_2 = Block {
|
|
|
|
hash: Some(H256::random()),
|
|
|
|
number: Some(1_000_001.into()),
|
|
|
|
parent_hash: block_1.hash.unwrap(),
|
|
|
|
timestamp: now + 1,
|
|
|
|
..Default::default()
|
|
|
|
};
|
|
|
|
|
|
|
|
let block_1: Web3ProxyBlock = Arc::new(block_1).try_into().unwrap();
|
|
|
|
let block_2: Web3ProxyBlock = Arc::new(block_2).try_into().unwrap();
|
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
let (tx_mock_geth, _) = watch::channel(Some(block_1.clone()));
|
|
|
|
let (tx_mock_erigon_archive, _) = watch::channel(Some(block_2.clone()));
|
|
|
|
|
2023-03-02 01:20:31 +03:00
|
|
|
let mock_geth = Web3Rpc {
|
|
|
|
name: "mock_geth".to_string(),
|
|
|
|
soft_limit: 1_000,
|
|
|
|
automatic_block_limit: false,
|
|
|
|
backup: false,
|
|
|
|
block_data_limit: 64.into(),
|
2023-06-09 22:21:50 +03:00
|
|
|
// tier: 0,
|
2023-06-13 20:00:08 +03:00
|
|
|
head_block: Some(tx_mock_geth),
|
2023-05-11 23:09:15 +03:00
|
|
|
peak_latency: Some(new_peak_latency()),
|
2023-03-02 01:20:31 +03:00
|
|
|
..Default::default()
|
|
|
|
};
|
|
|
|
|
|
|
|
let mock_erigon_archive = Web3Rpc {
|
|
|
|
name: "mock_erigon_archive".to_string(),
|
|
|
|
soft_limit: 1_000,
|
|
|
|
automatic_block_limit: false,
|
|
|
|
backup: false,
|
|
|
|
block_data_limit: u64::MAX.into(),
|
2023-06-09 22:21:50 +03:00
|
|
|
// tier: 1,
|
2023-06-13 20:00:08 +03:00
|
|
|
head_block: Some(tx_mock_erigon_archive),
|
2023-05-11 23:09:15 +03:00
|
|
|
peak_latency: Some(new_peak_latency()),
|
2023-03-02 01:20:31 +03:00
|
|
|
..Default::default()
|
|
|
|
};
|
|
|
|
|
|
|
|
assert!(mock_geth.has_block_data(block_1.number()));
|
|
|
|
assert!(mock_erigon_archive.has_block_data(block_1.number()));
|
|
|
|
assert!(!mock_geth.has_block_data(block_2.number()));
|
|
|
|
assert!(mock_erigon_archive.has_block_data(block_2.number()));
|
|
|
|
|
|
|
|
let mock_geth = Arc::new(mock_geth);
|
|
|
|
let mock_erigon_archive = Arc::new(mock_erigon_archive);
|
|
|
|
|
2023-07-11 09:08:06 +03:00
|
|
|
let (block_sender, _) = mpsc::unbounded_channel();
|
2023-06-27 22:36:41 +03:00
|
|
|
let (watch_ranked_rpcs, _) = watch::channel(None);
|
2023-05-13 01:15:32 +03:00
|
|
|
let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None);
|
2023-03-02 01:20:31 +03:00
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
let chain_id = 1;
|
|
|
|
|
2023-07-11 04:09:58 +03:00
|
|
|
let mut by_name = HashMap::new();
|
|
|
|
by_name.insert(mock_geth.name.clone(), mock_geth.clone());
|
|
|
|
by_name.insert(
|
|
|
|
mock_erigon_archive.name.clone(),
|
|
|
|
mock_erigon_archive.clone(),
|
|
|
|
);
|
2023-07-10 21:05:07 +03:00
|
|
|
|
2023-03-02 01:20:31 +03:00
|
|
|
// TODO: make a Web3Rpcs::new
|
|
|
|
let rpcs = Web3Rpcs {
|
|
|
|
block_sender,
|
2023-07-11 04:09:58 +03:00
|
|
|
by_name: RwLock::new(by_name),
|
2023-06-21 00:22:14 +03:00
|
|
|
chain_id,
|
2023-07-15 04:35:40 +03:00
|
|
|
name: "test".into(),
|
2023-06-27 22:36:41 +03:00
|
|
|
watch_head_block: Some(watch_consensus_head_sender),
|
|
|
|
watch_ranked_rpcs,
|
2023-06-08 03:26:38 +03:00
|
|
|
blocks_by_hash: Cache::new(10_000),
|
|
|
|
blocks_by_number: Cache::new(10_000),
|
2023-06-09 23:30:00 +03:00
|
|
|
min_synced_rpcs: 1,
|
2023-03-02 01:20:31 +03:00
|
|
|
min_sum_soft_limit: 1_000,
|
2023-06-17 20:11:48 +03:00
|
|
|
max_head_block_age: Duration::from_secs(60),
|
|
|
|
max_head_block_lag: 5.into(),
|
2023-03-02 01:20:31 +03:00
|
|
|
};
|
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
let mut consensus_finder = ConsensusFinder::new(None, None);
|
|
|
|
|
|
|
|
consensus_finder
|
2023-07-13 20:58:22 +03:00
|
|
|
.process_block_from_rpc(&rpcs, Some(block_1.clone()), mock_geth.clone())
|
2023-06-21 00:22:14 +03:00
|
|
|
.await
|
|
|
|
.unwrap();
|
2023-03-02 01:20:31 +03:00
|
|
|
|
2023-06-21 00:22:14 +03:00
|
|
|
consensus_finder
|
2023-07-13 20:58:22 +03:00
|
|
|
.process_block_from_rpc(&rpcs, Some(block_2.clone()), mock_erigon_archive.clone())
|
2023-06-21 00:22:14 +03:00
|
|
|
.await
|
|
|
|
.unwrap();
|
2023-03-02 01:20:31 +03:00
|
|
|
|
|
|
|
assert_eq!(rpcs.num_synced_rpcs(), 1);
|
|
|
|
|
|
|
|
// best_synced_backend_connection requires servers to be synced with the head block
|
|
|
|
// TODO: test with and without passing the head_block.number?
|
|
|
|
let head_connections = rpcs
|
2023-06-27 22:36:41 +03:00
|
|
|
.all_connections(None, Some(block_2.number()), None, None, None)
|
2023-03-02 01:20:31 +03:00
|
|
|
.await;
|
|
|
|
|
|
|
|
debug!("head_connections: {:#?}", head_connections);
|
|
|
|
|
|
|
|
assert_eq!(
|
|
|
|
head_connections.unwrap().len(),
|
|
|
|
1,
|
|
|
|
"wrong number of connections"
|
|
|
|
);
|
|
|
|
|
|
|
|
let all_connections = rpcs
|
2023-06-27 22:36:41 +03:00
|
|
|
.all_connections(None, Some(block_1.number()), None, None, None)
|
2023-03-02 01:20:31 +03:00
|
|
|
.await;
|
|
|
|
|
|
|
|
debug!("all_connections: {:#?}", all_connections);
|
|
|
|
|
2023-03-02 01:50:44 +03:00
|
|
|
assert_eq!(
|
|
|
|
all_connections.unwrap().len(),
|
|
|
|
2,
|
|
|
|
"wrong number of connections"
|
|
|
|
);
|
|
|
|
|
2023-06-27 22:36:41 +03:00
|
|
|
let all_connections = rpcs.all_connections(None, None, None, None, None).await;
|
2023-03-02 01:50:44 +03:00
|
|
|
|
|
|
|
debug!("all_connections: {:#?}", all_connections);
|
|
|
|
|
2023-03-02 01:20:31 +03:00
|
|
|
assert_eq!(
|
|
|
|
all_connections.unwrap().len(),
|
|
|
|
2,
|
|
|
|
"wrong number of connections"
|
|
|
|
)
|
2023-03-02 00:35:50 +03:00
|
|
|
}
|
2022-11-23 01:45:22 +03:00
|
|
|
}
|
2023-04-11 08:28:31 +03:00
|
|
|
|
2023-05-13 01:15:32 +03:00
|
|
|
#[cfg(test)]
|
|
|
|
mod test {
|
|
|
|
use std::cmp::Reverse;
|
|
|
|
|
|
|
|
#[test]
|
|
|
|
fn test_block_num_sort() {
|
|
|
|
let test_vec = vec![
|
|
|
|
Reverse(Some(3)),
|
|
|
|
Reverse(Some(2)),
|
|
|
|
Reverse(Some(1)),
|
|
|
|
Reverse(None),
|
|
|
|
];
|
|
|
|
|
|
|
|
let mut sorted_vec = test_vec.clone();
|
|
|
|
sorted_vec.sort();
|
|
|
|
|
|
|
|
assert_eq!(test_vec, sorted_vec);
|
|
|
|
}
|
2023-07-14 04:13:16 +03:00
|
|
|
|
|
|
|
#[test]
|
|
|
|
fn test_bool_sort() {
|
|
|
|
let test_vec = vec![false, true];
|
|
|
|
|
|
|
|
let mut sorted_vec = test_vec.clone();
|
|
|
|
sorted_vec.sort();
|
|
|
|
|
|
|
|
assert_eq!(test_vec, sorted_vec);
|
|
|
|
}
|
2023-05-13 01:15:32 +03:00
|
|
|
}
|