web3-proxy/web3_proxy/src/rpcs/many.rs

1989 lines
76 KiB
Rust
Raw Normal View History

///! Load balanced communication with a group of web3 rpc providers
use super::blockchain::{BlocksByHashCache, Web3ProxyBlock};
use super::consensus::ConsensusWeb3Rpcs;
use super::one::Web3Rpc;
2023-04-06 01:34:28 +03:00
use super::request::{OpenRequestHandle, OpenRequestResult, RequestErrorHandler};
use crate::app::{flatten_handle, AnyhowJoinHandle, Web3ProxyApp};
use crate::config::{BlockAndRpc, TxHashAndRpc, Web3RpcConfig};
use crate::frontend::authorization::{Authorization, RequestMetadata};
use crate::frontend::errors::{Web3ProxyError, Web3ProxyResult};
2023-01-17 09:54:40 +03:00
use crate::frontend::rpc_proxy_ws::ProxyMode;
use crate::jsonrpc::{JsonRpcErrorData, JsonRpcRequest};
use crate::response_cache::JsonRpcResponseData;
use crate::rpcs::consensus::{RankedRpcMap, RpcRanking};
2022-08-24 03:59:05 +03:00
use crate::rpcs::transactions::TxStatus;
2023-02-27 10:52:37 +03:00
use anyhow::Context;
use arc_swap::ArcSwap;
2022-05-28 07:26:24 +03:00
use counter::Counter;
2022-05-05 22:07:09 +03:00
use derive_more::From;
use ethers::prelude::{ProviderError, TxHash, H256, 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-01-24 12:58:31 +03:00
use hashbrown::{HashMap, HashSet};
2023-02-15 04:41:40 +03:00
use itertools::Itertools;
2022-12-17 07:05:01 +03:00
use log::{debug, error, info, trace, warn, Level};
2022-11-14 21:24:52 +03:00
use migration::sea_orm::DatabaseConnection;
use moka::future::Cache;
2023-02-15 22:42:25 +03:00
use ordered_float::OrderedFloat;
2022-05-21 01:16:15 +03:00
use serde::ser::{SerializeStruct, Serializer};
use serde::Serialize;
use serde_json::json;
2022-05-05 22:07:09 +03:00
use serde_json::value::RawValue;
use std::borrow::Cow;
use std::cmp::{min_by_key, Reverse};
2022-12-08 09:54:38 +03:00
use std::collections::BTreeMap;
use std::fmt;
use std::sync::atomic::Ordering;
2022-05-05 22:07:09 +03:00
use std::sync::Arc;
2022-11-12 09:11:58 +03:00
use thread_fast_rng::rand::seq::SliceRandom;
2023-02-27 07:00:13 +03:00
use tokio::sync::{broadcast, watch};
2022-09-05 08:53:58 +03:00
use tokio::time::{interval, sleep, sleep_until, Duration, Instant, MissedTickBehavior};
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)]
pub struct Web3Rpcs {
/// if watch_consensus_head_sender is some, Web3Rpc inside self will send blocks here when they get them
pub(crate) block_sender: flume::Sender<(Option<Web3ProxyBlock>, Arc<Web3Rpc>)>,
2022-08-27 02:44:25 +03:00
/// any requests will be forwarded to one (or more) of these connections
pub(crate) by_name: ArcSwap<HashMap<String, Arc<Web3Rpc>>>,
/// notify all http providers to check their blocks at the same time
pub(crate) http_interval_sender: Option<Arc<broadcast::Sender<()>>>,
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`
/// 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.
pub(crate) watch_consensus_rpcs_sender: watch::Sender<Option<Arc<ConsensusWeb3Rpcs>>>,
2023-01-23 09:02:08 +03:00
/// this head receiver makes it easy to wait until there is a new block
pub(super) watch_consensus_head_sender: Option<watch::Sender<Option<Web3ProxyBlock>>>,
/// keep track of transactions that we have sent through subscriptions
pub(super) pending_transaction_cache:
Cache<TxHash, TxStatus, hashbrown::hash_map::DefaultHashBuilder>,
pub(super) pending_tx_id_receiver: flume::Receiver<TxHashAndRpc>,
pub(super) pending_tx_id_sender: flume::Sender<TxHashAndRpc>,
/// TODO: this map is going to grow forever unless we do some sort of pruning. maybe store pruned in redis?
/// all blocks, including orphans
pub(super) blocks_by_hash: BlocksByHashCache,
2022-09-01 08:58:55 +03:00
/// blocks on the heaviest chain
pub(super) blocks_by_number: Cache<U64, H256, hashbrown::hash_map::DefaultHashBuilder>,
/// the number of rpcs required to agree on consensus for the head block (thundering herd protection)
pub(super) min_head_rpcs: usize,
/// 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,
/// how far behind the highest known block height we can be before we stop serving requests
pub(super) max_block_lag: Option<U64>,
/// how old our consensus head block we can be before we stop serving requests
pub(super) max_block_age: Option<u64>,
2022-05-05 22:07:09 +03:00
}
impl Web3Rpcs {
/// Spawn durable connections to multiple Web3 providers.
2022-08-26 20:26:17 +03:00
#[allow(clippy::too_many_arguments)]
pub async fn spawn(
chain_id: u64,
db_conn: Option<DatabaseConnection>,
http_client: Option<reqwest::Client>,
max_block_age: Option<u64>,
max_block_lag: Option<U64>,
min_head_rpcs: usize,
min_sum_soft_limit: u32,
pending_transaction_cache: Cache<TxHash, TxStatus, hashbrown::hash_map::DefaultHashBuilder>,
pending_tx_sender: Option<broadcast::Sender<TxStatus>>,
2023-02-15 04:41:40 +03:00
watch_consensus_head_sender: Option<watch::Sender<Option<Web3ProxyBlock>>>,
) -> anyhow::Result<(
Arc<Self>,
AnyhowJoinHandle<()>,
watch::Receiver<Option<Arc<ConsensusWeb3Rpcs>>>,
// watch::Receiver<Arc<ConsensusWeb3Rpcs>>,
)> {
let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded();
let (block_sender, block_receiver) = flume::unbounded::<BlockAndRpc>();
2022-05-05 22:07:09 +03:00
2023-02-25 20:48:40 +03:00
// TODO: query the rpc to get the actual expected block time, or get from config? maybe have this be part of a health check?
2023-01-03 22:37:42 +03:00
let expected_block_time_ms = match chain_id {
// ethereum
1 => 12_000,
2023-04-14 10:36:46 +03:00
// ethereum-goerli
5 => 12_000,
2023-01-03 22:37:42 +03:00
// polygon
137 => 2_000,
// fantom
250 => 1_000,
// arbitrum
2023-01-03 22:38:06 +03:00
42161 => 500,
2023-01-03 22:37:42 +03:00
// anything else
_ => {
2023-04-14 10:36:46 +03:00
warn!(
"unexpected chain_id ({}). polling every {} seconds",
chain_id, 10
);
2023-01-03 22:37:42 +03:00
10_000
}
};
2022-06-29 22:15:05 +03:00
let http_interval_sender = if http_client.is_some() {
2023-02-07 02:20:36 +03:00
let (sender, _) = broadcast::channel(1);
2022-06-29 22:15:05 +03:00
// TODO: what interval? follow a websocket also? maybe by watching synced connections with a timeout. will need debounce
2023-02-07 00:48:50 +03:00
let mut interval = interval(Duration::from_millis(expected_block_time_ms / 2));
2022-06-29 22:15:05 +03:00
interval.set_missed_tick_behavior(MissedTickBehavior::Delay);
let sender = Arc::new(sender);
let f = {
let sender = sender.clone();
async move {
loop {
interval.tick().await;
2023-02-07 00:48:50 +03:00
// trace!("http interval ready");
2022-07-16 08:21:08 +03:00
if sender.send(()).is_err() {
2023-02-07 02:20:36 +03:00
// errors are okay. they mean that all receivers have been dropped, or the rpcs just haven't started yet
2023-03-01 23:56:00 +03:00
// TODO: i'm seeing this error a lot more than expected
2023-02-07 02:20:36 +03:00
trace!("no http receivers");
2023-02-07 01:13:15 +03:00
};
2022-06-29 22:15:05 +03:00
}
}
};
// TODO: do something with this handle?
tokio::spawn(f);
Some(sender)
} else {
None
};
// these blocks don't have full transactions, but they do have rather variable amounts of transaction hashes
// TODO: how can we do the weigher better? need to know actual allocated size
// TODO: time_to_idle instead?
// TODO: limits from config
let blocks_by_hash: BlocksByHashCache = Cache::builder()
.max_capacity(1024 * 1024 * 1024)
.weigher(|_k, v: &Web3ProxyBlock| {
1 + v.block.transactions.len().try_into().unwrap_or(u32::MAX)
})
.time_to_live(Duration::from_secs(30 * 60))
2022-11-11 21:40:52 +03:00
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default());
2022-09-17 05:30:06 +03:00
// all block numbers are the same size, so no need for weigher
// TODO: limits from config
// TODO: time_to_idle instead?
let blocks_by_number = Cache::builder()
.time_to_live(Duration::from_secs(30 * 60))
.max_capacity(10_000)
2022-11-11 21:40:52 +03:00
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default());
2022-09-05 08:53:58 +03:00
let (watch_consensus_rpcs_sender, consensus_connections_watcher) =
watch::channel(Default::default());
2023-01-23 09:02:08 +03:00
// by_name starts empty. self.apply_server_configs will add to it
let by_name = Default::default();
2023-01-23 09:02:08 +03:00
2022-05-16 01:02:14 +03:00
let connections = Arc::new(Self {
block_sender,
by_name,
http_interval_sender,
watch_consensus_rpcs_sender,
watch_consensus_head_sender,
pending_transaction_cache,
pending_tx_id_sender,
pending_tx_id_receiver,
blocks_by_hash,
blocks_by_number,
2022-08-27 06:11:58 +03:00
min_sum_soft_limit,
min_head_rpcs,
max_block_age,
max_block_lag,
2022-05-16 01:02:14 +03:00
});
let authorization = Arc::new(Authorization::internal(db_conn)?);
let handle = {
let connections = connections.clone();
tokio::spawn(connections.subscribe(authorization, block_receiver, pending_tx_sender))
};
Ok((connections, handle, consensus_connections_watcher))
2022-05-18 19:35:06 +03:00
}
/// update the rpcs in this group
pub async fn apply_server_configs(
&self,
app: &Web3ProxyApp,
rpc_configs: HashMap<String, Web3RpcConfig>,
) -> anyhow::Result<()> {
// 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!(
"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(());
}
// 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);
// TODO: require a buffer?
anyhow::ensure!(
sum_soft_limit >= self.min_sum_soft_limit,
"Only {}/{} soft limit! Add more rpcs, increase soft limits, or reduce min_sum_soft_limit.",
sum_soft_limit,
self.min_sum_soft_limit,
);
// turn configs into connections (in parallel)
// TODO: move this into a helper function. then we can use it when configs change (will need a remove function too)
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 db_conn = app.db_conn();
let http_client = app.http_client.clone();
let vredis_pool = app.vredis_pool.clone();
let block_sender = if self.watch_consensus_head_sender.is_some() {
Some(self.block_sender.clone())
} else {
None
};
let pending_tx_id_sender = Some(self.pending_tx_id_sender.clone());
let blocks_by_hash = self.blocks_by_hash.clone();
let http_interval_sender = self.http_interval_sender.clone();
let chain_id = app.config.chain_id;
debug!("spawning {}", server_name);
let handle = tokio::spawn(server_config.spawn(
server_name,
db_conn,
vredis_pool,
chain_id,
http_client,
http_interval_sender,
blocks_by_hash,
block_sender,
pending_tx_id_sender,
true,
));
Some(handle)
})
.collect();
while let Some(x) = spawn_handles.next().await {
match x {
2023-02-28 00:40:13 +03:00
Ok(Ok((rpc, _handle))) => {
// web3 connection worked
let mut new_by_name = (*self.by_name.load_full()).clone();
let old_rpc = new_by_name.insert(rpc.name.clone(), rpc.clone());
self.by_name.store(Arc::new(new_by_name));
if let Some(old_rpc) = old_rpc {
if old_rpc.head_block.as_ref().unwrap().borrow().is_some() {
let mut new_head_receiver =
rpc.head_block.as_ref().unwrap().subscribe();
2023-02-28 00:40:13 +03:00
debug!("waiting for new {} to sync", rpc);
2023-02-28 00:40:13 +03:00
// TODO: maximum wait time or this could block things for too long
while new_head_receiver.borrow_and_update().is_none() {
new_head_receiver.changed().await?;
2023-02-28 00:40:13 +03:00
}
}
2023-02-27 10:52:37 +03:00
old_rpc.disconnect().await.context("disconnect old rpc")?;
}
// TODO: what should we do with the new handle? make sure error logs aren't dropped
}
Ok(Err(err)) => {
// if we got an error here, the app can continue on
// TODO: include context about which connection failed
// TODO: will this retry automatically? i don't think so
error!("Unable to create connection. err={:?}", err);
}
Err(err) => {
// something actually bad happened. exit with an error
return Err(err.into());
}
}
}
Ok(())
}
2023-02-27 07:00:13 +03:00
pub fn get(&self, conn_name: &str) -> Option<Arc<Web3Rpc>> {
self.by_name.load().get(conn_name).cloned()
2022-08-26 20:26:17 +03:00
}
2023-03-02 02:21:09 +03:00
pub fn len(&self) -> usize {
self.by_name.load().len()
2023-03-02 02:21:09 +03:00
}
pub fn is_empty(&self) -> bool {
self.by_name.load().is_empty()
2023-03-02 02:21:09 +03:00
}
2023-02-27 07:00:13 +03:00
pub fn min_head_rpcs(&self) -> usize {
self.min_head_rpcs
}
2022-07-22 22:30:39 +03:00
/// subscribe to blocks and transactions from all the backend rpcs.
/// 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`
async fn subscribe(
self: Arc<Self>,
authorization: Arc<Authorization>,
block_receiver: flume::Receiver<BlockAndRpc>,
2022-08-24 03:59:05 +03:00
pending_tx_sender: Option<broadcast::Sender<TxStatus>>,
) -> anyhow::Result<()> {
2022-06-16 05:53:37 +03:00
let mut futures = vec![];
2022-05-18 19:35:06 +03:00
2022-06-14 08:43:28 +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() {
2022-06-14 09:42:52 +03:00
let clone = self.clone();
let authorization = authorization.clone();
let pending_tx_id_receiver = self.pending_tx_id_receiver.clone();
let handle = tokio::task::spawn(async move {
2022-08-26 20:26:17 +03:00
// TODO: set up this future the same as the block funnel
while let Ok((pending_tx_id, rpc)) = pending_tx_id_receiver.recv_async().await {
2022-08-26 20:26:17 +03:00
let f = clone.clone().process_incoming_tx_id(
authorization.clone(),
2022-06-16 05:53:37 +03:00
rpc,
pending_tx_id,
pending_tx_sender.clone(),
);
tokio::spawn(f);
}
Ok(())
});
2022-06-16 05:53:37 +03:00
futures.push(flatten_handle(handle));
}
2022-06-14 08:43:28 +03:00
// setup the block funnel
if self.watch_consensus_head_sender.is_some() {
let connections = Arc::clone(&self);
let pending_tx_sender = pending_tx_sender.clone();
let handle = tokio::task::Builder::default()
2022-08-26 20:26:17 +03:00
.name("process_incoming_blocks")
2022-05-17 19:23:27 +03:00
.spawn(async move {
connections
.process_incoming_blocks(&authorization, block_receiver, pending_tx_sender)
2022-05-17 19:23:27 +03:00
.await
2022-09-02 23:16:20 +03:00
})?;
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
let handle = tokio::task::Builder::default()
.name("noop")
.spawn(async move {
loop {
sleep(Duration::from_secs(600)).await;
// TODO: "every interval, check that the provider is still connected"
}
})?;
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 {
2022-06-16 20:51:49 +03:00
error!("subscriptions over: {:?}", self);
2022-06-16 05:53:37 +03:00
return Err(e);
}
2022-05-18 19:35:06 +03:00
2022-06-14 08:43:28 +03:00
info!("subscriptions over: {:?}", self);
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.
/// TODO: option to return the fastest response and handles for all the others instead?
2022-05-12 02:50:52 +03:00
pub async fn try_send_parallel_requests(
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,
params: Option<&serde_json::Value>,
error_level: Level,
2022-09-23 01:14:24 +03:00
// TODO: remove this box once i figure out how to do the options
) -> Web3ProxyResult<JsonRpcResponseData> {
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
2022-05-28 07:26:24 +03:00
let responses = active_request_handles
.into_iter()
.map(|active_request_handle| async move {
let result: Result<Box<RawValue>, _> = active_request_handle
.request(method, &json!(&params), error_level.into(), None)
.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;
}
// TODO: better key!
let s = format!("{:?}", partial_response);
2022-05-28 07:26:24 +03:00
if count_map.get(&s).is_none() {
count_map.insert(s.clone(), partial_response);
2022-05-28 07:26:24 +03:00
}
2022-05-05 22:07:09 +03:00
2022-05-28 07:26:24 +03:00
counts.update([s].into_iter());
2022-05-05 22:07:09 +03:00
}
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
return Ok(x.into());
2022-12-24 04:32:58 +03:00
}
Err(err) => {
if any_ok_with_json_result {
2022-12-24 04:32:58 +03:00
// the most common is an error, but there is an Ok in here somewhere. loop to find it
continue;
}
let err: JsonRpcErrorData = err.try_into()?;
return Ok(err.into());
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
unimplemented!("this shouldn't be possible")
2022-05-05 22:07:09 +03:00
}
2023-02-16 08:05:41 +03:00
pub 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>,
request_metadata: Option<&Arc<RequestMetadata>>,
skip: &[Arc<Web3Rpc>],
2023-01-19 14:05:39 +03:00
min_block_needed: Option<&U64>,
2023-02-11 07:45:57 +03:00
max_block_needed: Option<&U64>,
) -> Web3ProxyResult<OpenRequestResult> {
// TODO: use tracing and add this so logs are easy
let request_ulid = request_metadata.map(|x| &x.request_ulid);
2023-04-30 04:38:31 +03:00
let usable_rpcs_by_tier_and_head_number = {
let mut m: RankedRpcMap = BTreeMap::new();
2022-11-07 00:05:03 +03:00
if let Some(consensus_rpcs) = self.watch_consensus_rpcs_sender.borrow().as_ref() {
// first place is the blocks that are synced close to head. if those don't work. try all the rpcs. if those don't work, keep trying for a few seconds
let head_block = &consensus_rpcs.head_block;
2023-01-03 19:33:49 +03:00
let head_block_num = *head_block.number();
let best_key = RpcRanking::new(
consensus_rpcs.tier,
consensus_rpcs.backups_needed,
Some(head_block_num),
);
// todo: for now, build the map m here. once that works, do as much of it as possible while building ConsensusWeb3Rpcs
for x in consensus_rpcs.best_rpcs.iter().filter(|rpc| {
consensus_rpcs.filter(skip, min_block_needed, max_block_needed, rpc)
}) {
m.entry(best_key).or_insert_with(Vec::new).push(x.clone());
2023-02-15 04:41:40 +03:00
}
2023-01-04 09:37:51 +03:00
let tier_offset = consensus_rpcs.tier + 1;
for (k, v) in consensus_rpcs.other_rpcs.iter() {
let v: Vec<_> = v
.iter()
.filter(|rpc| {
consensus_rpcs.filter(skip, min_block_needed, max_block_needed, rpc)
})
.cloned()
.collect();
2023-02-15 04:41:40 +03:00
let offset_ranking = k.add_offset(tier_offset);
2023-02-15 04:41:40 +03:00
m.entry(offset_ranking).or_insert_with(Vec::new).extend(v);
}
} else if self.watch_consensus_head_sender.is_none() {
trace!("this Web3Rpcs is not tracking head blocks. pick any server");
for x in self.by_name.load().values() {
if skip.contains(x) {
trace!("{:?} - already skipped. {}", request_ulid, x);
continue;
2023-01-20 05:30:30 +03:00
}
2023-02-16 08:05:41 +03:00
let key = RpcRanking::default_with_backup(x.backup);
2023-02-16 08:05:41 +03:00
m.entry(key).or_insert_with(Vec::new).push(x.clone());
2023-01-20 05:30:30 +03:00
}
}
2023-04-30 04:38:31 +03:00
m
2023-01-04 23:12:44 +03:00
};
2022-07-02 04:20:28 +03:00
2023-02-16 08:05:41 +03:00
trace!(
"{:?} - usable_rpcs_by_tier_and_head_number: {:#?}",
request_ulid,
2023-02-16 08:05:41 +03:00
usable_rpcs_by_tier_and_head_number
);
2022-12-08 09:54:38 +03:00
let mut earliest_retry_at = None;
2022-11-12 09:11:58 +03:00
2023-02-15 04:41:40 +03:00
for mut usable_rpcs in usable_rpcs_by_tier_and_head_number.into_values() {
// sort the tier randomly
if usable_rpcs.len() == 1 {
// TODO: include an rpc from the next tier?
} else {
// we can't get the rng outside of this loop because it is not Send
// this function should be pretty fast anyway, so it shouldn't matter too much
let mut rng = thread_fast_rng::thread_fast_rng();
usable_rpcs.shuffle(&mut rng);
2022-12-08 09:54:38 +03:00
};
2023-02-15 04:41:40 +03:00
// now that the rpcs are shuffled, try to get an active request handle for one of them
// pick the first two and try the one with the lower rpc.latency.ewma
// TODO: chunks or tuple windows?
for (rpc_a, rpc_b) in usable_rpcs.into_iter().circular_tuple_windows() {
trace!("{:?} - {} vs {}", request_ulid, rpc_a, rpc_b);
2023-02-16 08:05:41 +03:00
// TODO: cached key to save a read lock
// TODO: ties to the server with the smallest block_data_limit
let best_rpc = min_by_key(rpc_a, rpc_b, |x| x.peak_ewma());
trace!("{:?} - winner: {}", request_ulid, best_rpc);
2023-02-15 04:41:40 +03:00
// just because it has lower latency doesn't mean we are sure to get a connection
match best_rpc.try_request_handle(authorization, None).await {
2022-12-08 09:54:38 +03:00
Ok(OpenRequestResult::Handle(handle)) => {
trace!("{:?} - opened handle: {}", request_ulid, best_rpc);
2022-12-08 09:54:38 +03:00
return Ok(OpenRequestResult::Handle(handle));
}
Ok(OpenRequestResult::RetryAt(retry_at)) => {
trace!(
"{:?} - retry on {} @ {}",
request_ulid,
best_rpc,
retry_at.duration_since(Instant::now()).as_secs_f32()
);
if earliest_retry_at.is_none() {
earliest_retry_at = Some(retry_at);
} else {
earliest_retry_at = earliest_retry_at.min(Some(retry_at));
}
2022-12-08 09:54:38 +03:00
}
2023-02-15 04:41:40 +03:00
Ok(OpenRequestResult::NotReady) => {
2022-12-20 00:53:38 +03:00
// TODO: log a warning? emit a stat?
trace!("{:?} - best_rpc not ready: {}", request_ulid, best_rpc);
2022-12-08 09:54:38 +03:00
}
Err(err) => {
trace!(
"{:?} - No request handle for {}. err={:?}",
request_ulid,
best_rpc,
err
)
2022-12-08 09:54:38 +03:00
}
2022-05-06 23:44:12 +03:00
}
2022-05-05 22:07:09 +03:00
}
}
2022-12-08 09:54:38 +03:00
if let Some(request_metadata) = request_metadata {
request_metadata.no_servers.fetch_add(1, Ordering::AcqRel);
2022-12-08 09:54:38 +03:00
}
2022-08-07 09:48:57 +03:00
match earliest_retry_at {
2022-09-28 20:01:11 +03:00
None => {
2022-11-25 10:41:53 +03:00
// none of the servers gave us a time to retry at
2023-05-16 23:26:39 +03:00
debug!("no servers on {:?} gave a retry time. {:?}", self, skip);
2022-10-12 00:31:34 +03:00
2023-01-19 14:05:39 +03:00
// TODO: bring this back? need to think about how to do this with `allow_backups`
2022-11-25 10:41:53 +03:00
// we could return an error here, but maybe waiting a second will fix the problem
// TODO: configurable max wait? the whole max request time, or just some portion?
2022-12-08 09:54:38 +03:00
// let handle = sorted_rpcs
// .get(0)
// .expect("at least 1 is available")
// .wait_for_request_handle(authorization, Duration::from_secs(3), false)
// .await?;
// Ok(OpenRequestResult::Handle(handle))
2023-02-15 04:41:40 +03:00
Ok(OpenRequestResult::NotReady)
2022-09-28 20:01:11 +03:00
}
Some(earliest_retry_at) => {
// TODO: log the server that retry_at came from
// TODO: `self` doesn't log well. get a pretty name for this group of servers
warn!(
"{:?} - no servers on {:?}! retry in {:?}s",
request_ulid,
self,
earliest_retry_at
.duration_since(Instant::now())
.as_secs_f32()
);
2022-10-12 00:31:34 +03:00
2022-09-28 20:01:11 +03:00
Ok(OpenRequestResult::RetryAt(earliest_retry_at))
}
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,
authorization: &Arc<Authorization>,
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-01-19 14:05:39 +03:00
allow_backups: bool,
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-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 {
self.by_name.load().len()
2023-01-12 01:51:01 +03:00
};
trace!("max_count: {}", max_count);
let mut selected_rpcs = Vec::with_capacity(max_count);
2023-01-24 12:58:31 +03:00
let mut tried = HashSet::new();
let mut synced_rpcs = {
let synced_rpcs = self.watch_consensus_rpcs_sender.borrow();
if let Some(synced_rpcs) = synced_rpcs.as_ref() {
synced_rpcs.best_rpcs.clone()
} else {
vec![]
}
};
// synced connections are all on the same block. sort them by tier with higher soft limits first
synced_rpcs.sort_by_cached_key(rpc_sync_status_sort_key);
trace!("synced_rpcs: {:#?}", synced_rpcs);
// if there aren't enough synced connections, include more connections
// TODO: only do this sorting if the synced_rpcs isn't enough
let mut all_rpcs: Vec<_> = self.by_name.load().values().cloned().collect();
all_rpcs.sort_by_cached_key(rpc_sync_status_sort_key);
trace!("all_rpcs: {:#?}", all_rpcs);
for rpc in itertools::chain(synced_rpcs, all_rpcs) {
2023-01-12 01:51:01 +03:00
if max_count == 0 {
break;
}
2023-03-02 02:03:25 +03:00
if tried.contains(&rpc) {
2023-01-24 12:58:31 +03:00
continue;
}
trace!("trying {}", rpc);
2023-01-24 12:58:31 +03:00
2023-03-02 02:03:25 +03:00
tried.insert(rpc.clone());
if !allow_backups && rpc.backup {
2023-03-02 02:03:25 +03:00
warn!("{} is a backup. skipping", rpc);
2023-01-19 14:05:39 +03:00
continue;
}
2023-02-11 07:45:57 +03:00
if let Some(block_needed) = min_block_needed {
if !rpc.has_block_data(block_needed) {
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 {
if !rpc.has_block_data(block_needed) {
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
match rpc.try_request_handle(authorization, None).await {
2022-08-24 03:14:49 +03:00
Ok(OpenRequestResult::RetryAt(retry_at)) => {
// this rpc is not available. skip it
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)) => {
trace!("{} is available", rpc);
2023-01-12 01:51:01 +03:00
max_count -= 1;
selected_rpcs.push(handle)
}
2023-02-15 04:41:40 +03:00
Ok(OpenRequestResult::NotReady) => {
warn!("no request handle for {}", rpc)
2022-08-07 09:48:57 +03:00
}
Err(err) => {
warn!("error getting request handle for {}. err={:?}", rpc, err)
}
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
2022-05-29 04:23:58 +03:00
/// be sure there is a timeout on this or it might loop forever
2023-01-23 09:02:08 +03:00
/// TODO: think more about wait_for_sync
2023-01-17 09:54:40 +03:00
pub async fn try_send_best_consensus_head_connection(
2022-05-29 04:23:58 +03:00
&self,
authorization: &Arc<Authorization>,
request: &JsonRpcRequest,
request_metadata: Option<&Arc<RequestMetadata>>,
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>,
) -> Web3ProxyResult<JsonRpcResponseData> {
2022-07-02 04:20:28 +03:00
let mut skip_rpcs = vec![];
let mut method_not_available_response = None;
2022-07-02 04:20:28 +03:00
2023-05-16 23:26:39 +03:00
let mut watch_consensus_rpcs = self.watch_consensus_rpcs_sender.subscribe();
2023-01-23 09:02:08 +03:00
let start = Instant::now();
// TODO: get from config
let max_wait = Duration::from_secs(10);
while start.elapsed() < max_wait {
2022-07-19 04:31:12 +03:00
match self
2023-02-16 08:05:41 +03:00
.best_available_rpc(
authorization,
2022-10-12 00:31:34 +03:00
request_metadata,
&skip_rpcs,
2022-10-12 00:31:34 +03:00
min_block_needed,
2023-02-11 07:45:57 +03:00
max_block_needed,
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
// TODO: look at backend_requests instead
let rpc = active_request_handle.clone_connection();
2022-07-02 04:20:28 +03:00
if let Some(request_metadata) = request_metadata {
request_metadata.backend_requests.lock().push(rpc.clone());
}
let is_backup_response = rpc.backup;
// TODO: instead of entirely skipping, maybe demote a tier?
skip_rpcs.push(rpc);
// TODO: get the log percent from the user data
let response_result: Result<Box<RawValue>, _> = active_request_handle
.request(
&request.method,
&json!(request.params),
2023-04-06 01:34:28 +03:00
RequestErrorHandler::Save,
None,
)
2022-05-29 04:23:58 +03:00
.await;
match response_result {
2022-06-04 00:45:44 +03:00
Ok(response) => {
// 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);
}
return Ok(response.into());
}
Err(error) => {
// trace!(?response, "rpc error");
2022-07-02 04:20:28 +03:00
// TODO: separate jsonrpc error and web3 proxy error!
if let Some(request_metadata) = request_metadata {
request_metadata
.error_response
.store(true, Ordering::Release);
}
let error: JsonRpcErrorData = error.try_into()?;
// 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) {
warn!("rate limited by {}", skip_rpcs.last().unwrap());
continue;
}
}
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");
continue;
}
}
}
-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;
}
// 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
}
_ => {}
2022-06-04 00:45:44 +03:00
}
2022-05-29 04:23:58 +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
// TODO: if we get a TrySendError, reconnect. wait why do we see a trysenderror on a dual provider? shouldn't it be using reqwest
// 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
// hard_limit_until.send_replace(retry_at);
// }
return Ok(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
warn!(
"All rate limits exceeded. waiting for change in synced servers or {:?}s",
retry_at.duration_since(Instant::now()).as_secs_f32()
);
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 {
request_metadata.no_servers.fetch_add(1, Ordering::AcqRel);
2022-10-12 00:31:34 +03:00
}
tokio::select! {
_ = sleep_until(retry_at) => {
trace!("slept!");
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 => {
2022-10-12 00:31:34 +03:00
if let Some(request_metadata) = request_metadata {
request_metadata.no_servers.fetch_add(1, Ordering::AcqRel);
2022-10-12 00:31:34 +03:00
}
let waiting_for = min_block_needed.max(max_block_needed);
2023-05-16 23:26:39 +03:00
if watch_for_block(waiting_for, &skip_rpcs, &mut watch_consensus_rpcs).await? {
// block found! continue so we can check for another rpc
} else {
// rate limits are likely keeping us from serving the head block
2023-05-16 23:26:39 +03:00
watch_consensus_rpcs.changed().await?;
watch_consensus_rpcs.borrow_and_update();
}
2022-05-29 04:23:58 +03:00
}
}
}
2022-07-02 04:20:28 +03:00
2023-02-16 08:05:41 +03:00
// TODO: do we need this here, or do we do it somewhere else? like, the code could change and a try operator in here would skip this increment
2022-10-12 00:31:34 +03:00
if let Some(request_metadata) = request_metadata {
request_metadata
.error_response
.store(true, Ordering::Release);
}
if let Some(err) = method_not_available_response {
// TODO: this error response is likely the user's fault. do we actually want it marked as an error? maybe keep user and server error bools?
2023-02-16 08:05:41 +03:00
// TODO: emit a stat for unsupported methods? it would be best to block them at the proxy instead of at the backend
// TODO: this is too verbose!
debug!("{}", serde_json::to_string(&err)?);
return Ok(err.into());
2023-02-16 08:05:41 +03:00
}
let num_conns = self.by_name.load().len();
let num_skipped = skip_rpcs.len();
2022-11-28 09:52:16 +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?
if head_block_num.is_none() {
2023-03-01 23:56:00 +03:00
error!(
"No servers synced (min {:?}, max {:?}, head {:?}) ({} known)",
2023-03-22 12:06:26 +03:00
min_block_needed, max_block_needed, head_block_num, num_conns
2023-03-01 23:56:00 +03:00
);
} else if head_block_num.as_ref() > needed {
// we have synced past the needed block
// TODO: this is likely caused by rate limits. make the error message better
error!(
"No archive servers synced (min {:?}, max {:?}, head {:?}) ({} known)",
min_block_needed, max_block_needed, head_block_num, num_conns
);
} else {
2023-03-23 01:19:09 +03:00
error!(
2023-04-11 02:05:53 +03:00
"Requested data is not available (min {:?}, max {:?}, head {:?}) ({} skipped, {} known)",
2023-03-23 01:19:09 +03:00
min_block_needed, max_block_needed, head_block_num, num_skipped, num_conns
);
// TODO: remove this, or move to trace level
// debug!("{}", serde_json::to_string(&request).unwrap());
}
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}
Ok(JsonRpcErrorData {
message: Cow::Borrowed("Requested data is not available"),
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
#[allow(clippy::too_many_arguments)]
2023-01-17 09:54:40 +03:00
pub async fn try_send_all_synced_connections(
self: &Arc<Self>,
authorization: &Arc<Authorization>,
2022-12-24 04:32:58 +03:00
request: &JsonRpcRequest,
request_metadata: Option<Arc<RequestMetadata>>,
2023-02-11 07:45:57 +03:00
min_block_needed: Option<&U64>,
max_block_needed: Option<&U64>,
error_level: Level,
2023-01-12 01:51:01 +03:00
max_count: Option<usize>,
always_include_backups: bool,
) -> Web3ProxyResult<JsonRpcResponseData> {
2023-03-01 23:56:00 +03:00
let mut watch_consensus_rpcs = self.watch_consensus_rpcs_sender.subscribe();
let start = Instant::now();
// TODO: get from config
let max_wait = Duration::from_secs(3);
while start.elapsed() < max_wait {
match self
.all_connections(
authorization,
2023-02-11 07:45:57 +03:00
min_block_needed,
max_block_needed,
max_count,
always_include_backups,
)
.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 {
let mut only_backups_used = true;
request_metadata.backend_requests.lock().extend(
active_request_handles.iter().map(|x| {
let rpc = x.clone_connection();
if !rpc.backup {
// TODO: its possible we serve from a synced connection though. think about this more
only_backups_used = false;
}
x.clone_connection()
}),
);
2022-10-12 00:31:34 +03:00
request_metadata
.response_from_backup_rpc
.store(only_backups_used, Ordering::Release);
2022-10-12 00:31:34 +03:00
}
2022-12-24 04:32:58 +03:00
return self
2022-05-28 07:26:24 +03:00
.try_send_parallel_requests(
active_request_handles,
request.method.as_ref(),
request.params.as_ref(),
error_level,
2022-05-28 07:26:24 +03:00
)
2022-12-24 04:32:58 +03:00
.await;
2022-05-28 07:26:24 +03:00
}
Err(None) => {
2023-03-01 23:56:00 +03:00
warn!(
"No servers in sync on {:?} (block {:?} - {:?})! Retrying",
self, min_block_needed, max_block_needed
);
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
request_metadata.no_servers.fetch_add(1, Ordering::AcqRel);
2022-10-12 00:31:34 +03:00
}
2023-03-01 23:56:00 +03:00
watch_consensus_rpcs.changed().await?;
watch_consensus_rpcs.borrow_and_update();
2022-06-03 00:47:43 +03:00
continue;
2022-05-28 07:26:24 +03:00
}
2022-08-07 09:48:57 +03:00
Err(Some(retry_at)) => {
2022-05-28 07:26:24 +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
2022-06-03 00:47:43 +03:00
warn!("All rate limits exceeded. Sleeping");
2022-10-12 00:31:34 +03:00
if let Some(request_metadata) = &request_metadata {
request_metadata.no_servers.fetch_add(1, Ordering::AcqRel);
2022-10-12 00:31:34 +03:00
}
2023-03-01 23:56:00 +03:00
tokio::select! {
_ = sleep_until(retry_at) => {}
_ = watch_consensus_rpcs.changed() => {
watch_consensus_rpcs.borrow_and_update();
}
}
2022-05-28 07:26:24 +03:00
2022-06-03 00:47:43 +03:00
continue;
2022-05-28 07:26:24 +03:00
}
}
}
Err(Web3ProxyError::NoServersSynced)
2022-05-28 07:26:24 +03:00
}
2023-01-17 09:54:40 +03:00
pub async fn try_proxy_connection(
&self,
authorization: &Arc<Authorization>,
request: &JsonRpcRequest,
2023-01-17 09:54:40 +03:00
request_metadata: Option<&Arc<RequestMetadata>>,
min_block_needed: Option<&U64>,
2023-02-11 07:45:57 +03:00
max_block_needed: Option<&U64>,
) -> Web3ProxyResult<JsonRpcResponseData> {
2023-03-03 04:39:50 +03:00
match authorization.checks.proxy_mode {
ProxyMode::Debug | ProxyMode::Best => {
2023-01-17 09:54:40 +03:00
self.try_send_best_consensus_head_connection(
authorization,
request,
request_metadata,
min_block_needed,
2023-02-11 07:45:57 +03:00
max_block_needed,
2023-01-17 09:54:40 +03:00
)
.await
}
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
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
f.debug_struct("Web3Rpcs")
2023-02-15 04:41:40 +03:00
.field("rpcs", &self.by_name)
2022-08-24 02:13:56 +03:00
.finish_non_exhaustive()
}
}
impl Serialize for Web3Rpcs {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
let mut state = serializer.serialize_struct("Web3Rpcs", 1)?;
2022-09-22 02:50:55 +03:00
2023-02-27 07:00:13 +03:00
{
let by_name = self.by_name.load();
2023-02-27 07:00:13 +03:00
let rpcs: Vec<&Web3Rpc> = by_name.values().map(|x| x.as_ref()).collect();
// 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
{
let consensus_rpcs = self.watch_consensus_rpcs_sender.borrow().clone();
// 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
// self.blocks_by_hash.sync();
// self.blocks_by_number.sync();
// state.serialize_field("block_hashes_count", &self.blocks_by_hash.entry_count())?;
// state.serialize_field("block_hashes_size", &self.blocks_by_hash.weighted_size())?;
// state.serialize_field("block_numbers_count", &self.blocks_by_number.entry_count())?;
// state.serialize_field("block_numbers_size", &self.blocks_by_number.weighted_size())?;
state.end()
}
}
/// sort by block number (descending) and tier (ascending)
2023-02-11 07:24:20 +03:00
/// TODO: should this be moved into a `impl Web3Rpc`?
2023-02-15 22:42:25 +03:00
/// TODO: i think we still have sorts scattered around the code that should use this
2023-02-11 07:24:20 +03:00
/// TODO: take AsRef or something like that? We don't need an Arc here
fn rpc_sync_status_sort_key(x: &Arc<Web3Rpc>) -> (Reverse<U64>, u64, bool, OrderedFloat<f64>) {
2023-03-31 14:43:41 +03:00
let head_block = x
.head_block
.as_ref()
.and_then(|x| x.borrow().as_ref().map(|x| *x.number()))
2023-03-31 14:43:41 +03:00
.unwrap_or_default();
2023-02-11 07:24:20 +03:00
let tier = x.tier;
let peak_ewma = x.peak_ewma();
2023-02-16 11:26:58 +03:00
2023-03-02 00:35:50 +03:00
let backup = x.backup;
(Reverse(head_block), tier, backup, peak_ewma)
}
mod tests {
2022-12-03 08:31:03 +03:00
#![allow(unused_imports)]
use std::time::{SystemTime, UNIX_EPOCH};
use super::*;
use crate::rpcs::consensus::ConsensusFinder;
use crate::rpcs::{blockchain::Web3ProxyBlock, provider::Web3Provider};
use arc_swap::ArcSwap;
2022-12-01 01:11:14 +03:00
use ethers::types::{Block, U256};
use latency::PeakEwmaLatency;
2022-11-25 10:41:53 +03:00
use log::{trace, LevelFilter};
use parking_lot::RwLock;
2022-12-03 08:31:03 +03:00
use tokio::sync::RwLock as AsyncRwLock;
#[cfg(test)]
fn new_peak_latency() -> PeakEwmaLatency {
const NANOS_PER_MILLI: f64 = 1_000_000.0;
PeakEwmaLatency::spawn(1_000.0 * NANOS_PER_MILLI, 4, Duration::from_secs(1))
}
#[tokio::test]
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())
.collect();
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<_> = [
Web3Rpc {
name: "a".to_string(),
tier: 0,
head_block: Some(tx_a),
peak_latency: Some(new_peak_latency()),
..Default::default()
},
Web3Rpc {
name: "b".to_string(),
tier: 0,
head_block: Some(tx_b),
peak_latency: Some(new_peak_latency()),
..Default::default()
},
Web3Rpc {
name: "c".to_string(),
tier: 0,
head_block: Some(tx_c),
peak_latency: Some(new_peak_latency()),
..Default::default()
},
Web3Rpc {
name: "d".to_string(),
tier: 1,
head_block: Some(tx_d),
peak_latency: Some(new_peak_latency()),
..Default::default()
},
Web3Rpc {
name: "e".to_string(),
tier: 1,
head_block: Some(tx_e),
peak_latency: Some(new_peak_latency()),
..Default::default()
},
Web3Rpc {
name: "f".to_string(),
tier: 1,
head_block: Some(tx_f),
peak_latency: Some(new_peak_latency()),
..Default::default()
},
]
.into_iter()
.map(Arc::new)
.collect();
2023-02-11 07:45:57 +03:00
rpcs.sort_by_cached_key(rpc_sync_status_sort_key);
let names_in_sort_order: Vec<_> = rpcs.iter().map(|x| x.name.as_str()).collect();
assert_eq!(names_in_sort_order, ["c", "f", "b", "e", "a", "d"]);
}
#[tokio::test]
2022-11-25 10:41:53 +03:00
async fn test_server_selection_by_height() {
// TODO: do this better. can test_env_logger and tokio test be stacked?
let _ = env_logger::builder()
.filter_level(LevelFilter::Error)
.filter_module("web3_proxy", LevelFilter::Trace)
.is_test(true)
.try_init();
let now = chrono::Utc::now().timestamp().into();
2022-12-01 01:11:14 +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,
..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,
..Default::default()
};
let lagged_block = Arc::new(lagged_block);
let head_block = Arc::new(head_block);
let block_data_limit = u64::MAX;
let (tx_synced, _) = watch::channel(None);
let head_rpc = Web3Rpc {
name: "synced".to_string(),
soft_limit: 1_000,
automatic_block_limit: false,
backup: false,
block_data_limit: block_data_limit.into(),
2023-01-04 09:37:51 +03:00
tier: 0,
head_block: Some(tx_synced),
provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))),
peak_latency: Some(new_peak_latency()),
..Default::default()
};
let (tx_lagged, _) = watch::channel(None);
let lagged_rpc = Web3Rpc {
name: "lagged".to_string(),
soft_limit: 1_000,
2022-12-06 00:13:36 +03:00
automatic_block_limit: false,
backup: false,
block_data_limit: block_data_limit.into(),
2023-01-04 09:37:51 +03:00
tier: 0,
head_block: Some(tx_lagged),
provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))),
peak_latency: Some(new_peak_latency()),
..Default::default()
};
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()));
let head_rpc = Arc::new(head_rpc);
let lagged_rpc = Arc::new(lagged_rpc);
2023-02-15 04:41:40 +03:00
let rpcs_by_name = HashMap::from([
(head_rpc.name.clone(), head_rpc.clone()),
(lagged_rpc.name.clone(), lagged_rpc.clone()),
]);
let (block_sender, _block_receiver) = flume::unbounded();
let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded();
let (watch_consensus_rpcs_sender, _watch_consensus_rpcs_receiver) = watch::channel(None);
let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None);
2023-01-23 09:02:08 +03:00
// TODO: make a Web3Rpcs::new
2023-02-15 04:41:40 +03:00
let rpcs = Web3Rpcs {
block_sender: block_sender.clone(),
by_name: ArcSwap::from_pointee(rpcs_by_name),
http_interval_sender: None,
watch_consensus_head_sender: Some(watch_consensus_head_sender),
watch_consensus_rpcs_sender,
pending_transaction_cache: Cache::builder()
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()),
pending_tx_id_receiver,
pending_tx_id_sender,
blocks_by_hash: Cache::builder()
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()),
blocks_by_number: Cache::builder()
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()),
// TODO: test max_block_age?
max_block_age: None,
// TODO: test max_block_lag?
max_block_lag: None,
min_head_rpcs: 1,
min_sum_soft_limit: 1,
};
let authorization = Arc::new(Authorization::internal(None).unwrap());
2023-02-27 07:00:13 +03:00
let mut consensus_finder = ConsensusFinder::new(None, None);
// process None so that
2023-02-15 04:41:40 +03:00
rpcs.process_block_from_rpc(
&authorization,
&mut consensus_finder,
None,
lagged_rpc.clone(),
&None,
)
.await
.expect("its lagged, but it should still be seen as consensus if its the first to report");
rpcs.process_block_from_rpc(
&authorization,
&mut consensus_finder,
None,
head_rpc.clone(),
&None,
)
.await
.unwrap();
// 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());
// all_backend_connections gives all non-backup servers regardless of sync status
assert_eq!(
2023-02-15 04:41:40 +03:00
rpcs.all_connections(&authorization, None, None, None, false)
.await
.unwrap()
.len(),
2
);
// 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
.best_available_rpc(
&authorization,
None,
&[],
Some(head_block.number.as_ref().unwrap()),
None,
)
2022-11-28 09:52:16 +03:00
.await
.unwrap();
dbg!(&x);
2023-02-15 04:41:40 +03:00
assert!(matches!(x, OpenRequestResult::NotReady));
2023-02-15 04:41:40 +03:00
// add lagged blocks to the rpcs. both servers should be allowed
lagged_rpc
.send_head_block_result(
Ok(Some(lagged_block.clone())),
&block_sender,
rpcs.blocks_by_hash.clone(),
)
.await
.unwrap();
// TODO: this is fragile
2023-02-15 04:41:40 +03:00
rpcs.process_block_from_rpc(
&authorization,
&mut consensus_finder,
Some(lagged_block.clone().try_into().unwrap()),
lagged_rpc.clone(),
2023-02-15 04:41:40 +03:00
&None,
)
.await
.unwrap();
head_rpc
.send_head_block_result(
Ok(Some(lagged_block.clone())),
&block_sender,
rpcs.blocks_by_hash.clone(),
)
.await
.unwrap();
// TODO: this is fragile
2023-02-15 04:41:40 +03:00
rpcs.process_block_from_rpc(
&authorization,
&mut consensus_finder,
Some(lagged_block.clone().try_into().unwrap()),
2023-02-15 04:41:40 +03:00
head_rpc.clone(),
&None,
)
.await
.unwrap();
// TODO: how do we spawn this and wait for it to process things? subscribe and watch consensus connections?
// rpcs.process_incoming_blocks(&authorization, block_receiver, pending_tx_sender)
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);
2023-02-15 04:41:40 +03:00
// add head block to the rpcs. lagged_rpc should not be available
head_rpc
.send_head_block_result(
Ok(Some(head_block.clone())),
&block_sender,
rpcs.blocks_by_hash.clone(),
)
.await
.unwrap();
// TODO: this is fragile
2023-02-15 04:41:40 +03:00
rpcs.process_block_from_rpc(
&authorization,
&mut consensus_finder,
Some(head_block.clone().try_into().unwrap()),
head_rpc.clone(),
2023-02-15 04:41:40 +03:00
&None,
)
.await
.unwrap();
2023-02-15 04:41:40 +03:00
assert_eq!(rpcs.num_synced_rpcs(), 1);
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-02-16 08:05:41 +03:00
rpcs.best_available_rpc(&authorization, None, &[], None, None)
2022-11-25 10:41:53 +03:00
.await,
Ok(OpenRequestResult::Handle(_))
));
// TODO: make sure the handle is for the expected rpc
2022-11-25 10:41:53 +03:00
assert!(matches!(
2023-02-16 08:05:41 +03:00
rpcs.best_available_rpc(&authorization, None, &[], Some(&0.into()), None)
2022-11-25 10:41:53 +03:00
.await,
Ok(OpenRequestResult::Handle(_))
));
// TODO: make sure the handle is for the expected rpc
2022-11-25 10:41:53 +03:00
assert!(matches!(
2023-02-16 08:05:41 +03:00
rpcs.best_available_rpc(&authorization, None, &[], Some(&1.into()), None)
2022-11-25 10:41:53 +03:00
.await,
Ok(OpenRequestResult::Handle(_))
));
// future block should not get a handle
2023-02-16 08:05:41 +03:00
let future_rpc = rpcs
.best_available_rpc(&authorization, None, &[], Some(&2.into()), None)
.await;
assert!(matches!(future_rpc, Ok(OpenRequestResult::NotReady)));
2022-11-25 10:41:53 +03:00
}
#[tokio::test]
async fn test_server_selection_by_archive() {
// TODO: do this better. can test_env_logger and tokio test be stacked?
let _ = env_logger::builder()
.filter_level(LevelFilter::Error)
.filter_module("web3_proxy", LevelFilter::Trace)
.is_test(true)
.try_init();
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
let (tx_pruned, _) = watch::channel(Some(head_block.clone()));
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,
backup: false,
2022-11-25 10:41:53 +03:00
block_data_limit: 64.into(),
2023-01-04 09:37:51 +03:00
tier: 1,
head_block: Some(tx_pruned),
2023-02-16 08:05:41 +03:00
provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))),
..Default::default()
2022-11-25 10:41:53 +03:00
};
let (tx_archive, _) = watch::channel(Some(head_block.clone()));
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,
backup: false,
2022-11-25 10:41:53 +03:00
block_data_limit: u64::MAX.into(),
2023-01-04 09:37:51 +03:00
tier: 2,
head_block: Some(tx_archive),
2023-02-16 08:05:41 +03:00
provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))),
..Default::default()
2022-11-25 10:41:53 +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-02-15 04:41:40 +03:00
let rpcs_by_name = HashMap::from([
2022-11-25 10:41:53 +03:00
(pruned_rpc.name.clone(), pruned_rpc.clone()),
(archive_rpc.name.clone(), archive_rpc.clone()),
]);
let (block_sender, _) = flume::unbounded();
let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded();
let (watch_consensus_rpcs_sender, _) = watch::channel(None);
let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None);
2023-01-23 09:02:08 +03:00
// TODO: make a Web3Rpcs::new
2023-02-15 04:41:40 +03:00
let rpcs = Web3Rpcs {
block_sender,
by_name: ArcSwap::from_pointee(rpcs_by_name),
http_interval_sender: None,
watch_consensus_head_sender: Some(watch_consensus_head_sender),
watch_consensus_rpcs_sender,
pending_transaction_cache: Cache::builder()
2022-11-25 10:41:53 +03:00
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()),
pending_tx_id_receiver,
pending_tx_id_sender,
blocks_by_hash: Cache::builder()
2022-11-25 10:41:53 +03:00
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()),
blocks_by_number: Cache::builder()
2022-11-25 10:41:53 +03:00
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()),
min_head_rpcs: 1,
2023-02-15 04:41:40 +03:00
min_sum_soft_limit: 4_000,
max_block_age: None,
max_block_lag: None,
2022-11-25 10:41:53 +03:00
};
let authorization = Arc::new(Authorization::internal(None).unwrap());
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-02-15 04:41:40 +03:00
// min sum soft limit will require tier 2
rpcs.process_block_from_rpc(
&authorization,
&mut connection_heads,
Some(head_block.clone()),
pruned_rpc.clone(),
&None,
)
.await
.unwrap_err();
rpcs.process_block_from_rpc(
&authorization,
&mut connection_heads,
Some(head_block.clone()),
archive_rpc.clone(),
&None,
)
.await
.unwrap();
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
.best_available_rpc(&authorization, None, &[], Some(head_block.number()), None)
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(_)
));
let _best_available_server_from_none = rpcs
2023-02-16 08:05:41 +03:00
.best_available_rpc(&authorization, None, &[], None, None)
.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-02-16 08:05:41 +03:00
.best_available_rpc(&authorization, None, &[], Some(&1.into()), None)
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);
}
}
}
2023-03-02 00:35:50 +03:00
#[tokio::test]
async fn test_all_connections() {
let _ = env_logger::builder()
.filter_level(LevelFilter::Error)
.filter_module("web3_proxy", LevelFilter::Trace)
.is_test(true)
.try_init();
// 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();
let (tx_mock_geth, _) = watch::channel(Some(block_1.clone()));
let (tx_mock_erigon_archive, _) = watch::channel(Some(block_2.clone()));
let mock_geth = Web3Rpc {
name: "mock_geth".to_string(),
soft_limit: 1_000,
automatic_block_limit: false,
backup: false,
block_data_limit: 64.into(),
tier: 0,
head_block: Some(tx_mock_geth),
provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))),
peak_latency: Some(new_peak_latency()),
..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(),
tier: 1,
head_block: Some(tx_mock_erigon_archive),
provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))),
peak_latency: Some(new_peak_latency()),
..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);
let rpcs_by_name = HashMap::from([
(mock_geth.name.clone(), mock_geth.clone()),
(
mock_erigon_archive.name.clone(),
mock_erigon_archive.clone(),
),
]);
let (block_sender, _) = flume::unbounded();
let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded();
let (watch_consensus_rpcs_sender, _) = watch::channel(None);
let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None);
// TODO: make a Web3Rpcs::new
let rpcs = Web3Rpcs {
block_sender,
by_name: ArcSwap::from_pointee(rpcs_by_name),
http_interval_sender: None,
watch_consensus_head_sender: Some(watch_consensus_head_sender),
watch_consensus_rpcs_sender,
pending_transaction_cache: Cache::builder()
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()),
pending_tx_id_receiver,
pending_tx_id_sender,
blocks_by_hash: Cache::builder()
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()),
blocks_by_number: Cache::builder()
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()),
min_head_rpcs: 1,
min_sum_soft_limit: 1_000,
max_block_age: None,
max_block_lag: None,
};
let authorization = Arc::new(Authorization::internal(None).unwrap());
let mut connection_heads = ConsensusFinder::new(None, None);
rpcs.process_block_from_rpc(
&authorization,
&mut connection_heads,
Some(block_1.clone()),
mock_geth.clone(),
&None,
)
.await
.unwrap();
rpcs.process_block_from_rpc(
&authorization,
&mut connection_heads,
Some(block_2.clone()),
mock_erigon_archive.clone(),
&None,
)
.await
.unwrap();
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
.all_connections(&authorization, Some(block_2.number()), None, None, false)
.await;
debug!("head_connections: {:#?}", head_connections);
assert_eq!(
head_connections.unwrap().len(),
1,
"wrong number of connections"
);
let all_connections = rpcs
.all_connections(&authorization, Some(block_1.number()), None, None, false)
.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"
);
let all_connections = rpcs
.all_connections(&authorization, None, None, None, false)
.await;
debug!("all_connections: {:#?}", all_connections);
assert_eq!(
all_connections.unwrap().len(),
2,
"wrong number of connections"
)
2023-03-02 00:35:50 +03:00
}
}
/// returns `true` when the desired block number is available
/// TODO: max wait time? max number of blocks to wait for? time is probably best
async fn watch_for_block(
2023-05-16 23:26:39 +03:00
needed_block_num: Option<&U64>,
skip_rpcs: &[Arc<Web3Rpc>],
watch_consensus_rpcs: &mut watch::Receiver<Option<Arc<ConsensusWeb3Rpcs>>>,
) -> Web3ProxyResult<bool> {
2023-05-16 23:26:39 +03:00
info!("waiting for {:?}", needed_block_num);
let mut best_block_num: Option<U64> = watch_consensus_rpcs
.borrow_and_update()
.as_ref()
2023-05-16 23:26:39 +03:00
.and_then(|x| x.best_block_num(skip_rpcs).copied());
2023-05-16 23:26:39 +03:00
match (needed_block_num, best_block_num.as_ref()) {
(Some(x), Some(best)) => {
if x <= best {
// the best block is past the needed block and no servers have the needed data
// this happens if the block is old and all archive servers are offline
// there is no chance we will get this block without adding an archive server to the config
2023-05-16 23:26:39 +03:00
// TODO: i think this can also happen if we are being rate limited! but then waiting might work. need skip_rpcs to be smarter
return Ok(false);
}
}
(None, None) => {
// i don't think this is possible
// maybe if we internally make a request for the latest block and all our servers are disconnected?
warn!("how'd this None/None happen?");
return Ok(false);
}
(Some(_), None) => {
// block requested but no servers synced. we will wait
2023-05-16 23:26:39 +03:00
// TODO: if the web3rpcs connected to this consensus isn't watching head blocks, exit with an erorr (waiting for blocks won't ever work)
}
(None, Some(head)) => {
// i don't think this is possible
// maybe if we internally make a request for the latest block and all our servers are disconnected?
warn!("how'd this None/Some({}) happen?", head);
return Ok(false);
}
};
// future block is requested
// wait for the block to arrive
2023-05-16 23:26:39 +03:00
while best_block_num.as_ref() < needed_block_num {
watch_consensus_rpcs.changed().await?;
2023-05-16 23:26:39 +03:00
let consensus_rpcs = watch_consensus_rpcs.borrow_and_update();
best_block_num = consensus_rpcs
.as_ref()
2023-05-16 23:26:39 +03:00
.and_then(|x| x.best_block_num(skip_rpcs).copied());
}
Ok(true)
}
#[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);
}
}