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

1576 lines
60 KiB
Rust
Raw Normal View History

2022-05-12 02:50:52 +03:00
///! Load balanced communication with a group of web3 providers
use super::blockchain::{ArcBlock, BlockHashesCache};
2022-08-24 03:11:49 +03:00
use super::connection::Web3Connection;
use super::request::{
2023-01-17 09:54:40 +03:00
OpenRequestHandle, OpenRequestHandleMetrics, OpenRequestResult, RequestRevertHandler,
};
use super::synced_connections::ConsensusConnections;
2022-08-24 03:59:05 +03:00
use crate::app::{flatten_handle, AnyhowJoinHandle};
2022-08-30 23:01:42 +03:00
use crate::config::{BlockAndRpc, TxHashAndRpc, Web3ConnectionConfig};
use crate::frontend::authorization::{Authorization, RequestMetadata};
2023-01-17 09:54:40 +03:00
use crate::frontend::rpc_proxy_ws::ProxyMode;
use crate::jsonrpc::{JsonRpcForwardedResponse, JsonRpcRequest};
2022-08-24 03:59:05 +03:00
use crate::rpcs::transactions::TxStatus;
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};
use futures::future::{join_all, 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};
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;
2022-09-22 02:50:55 +03:00
use moka::future::{Cache, ConcurrentCacheExt};
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;
2022-12-08 09:54:38 +03:00
use std::collections::BTreeMap;
use std::sync::atomic::Ordering;
2022-05-05 22:07:09 +03:00
use std::sync::Arc;
2023-01-20 05:30:30 +03:00
use std::{cmp, fmt};
2022-11-12 09:11:58 +03:00
use thread_fast_rng::rand::seq::SliceRandom;
2022-06-16 05:53:37 +03:00
use tokio::sync::{broadcast, watch};
2022-05-17 19:23:27 +03:00
use tokio::task;
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 Web3Connections {
2022-08-27 02:44:25 +03:00
/// any requests will be forwarded to one (or more) of these connections
2023-01-23 09:02:08 +03:00
pub(crate) conns: HashMap<String, Arc<Web3Connection>>,
/// all providers with the same consensus head block. won't update if there is no `self.watch_consensus_head_sender`
pub(super) watch_consensus_connections_sender: watch::Sender<Arc<ConsensusConnections>>,
/// this head receiver makes it easy to wait until there is a new block
pub(super) watch_consensus_head_receiver: Option<watch::Receiver<ArcBlock>>,
pub(super) pending_transactions:
Cache<TxHash, TxStatus, hashbrown::hash_map::DefaultHashBuilder>,
/// 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) block_hashes: BlockHashesCache,
2022-09-01 08:58:55 +03:00
/// blocks on the heaviest chain
pub(super) block_numbers: Cache<U64, H256, hashbrown::hash_map::DefaultHashBuilder>,
pub(super) min_head_rpcs: usize,
2022-08-27 06:11:58 +03:00
pub(super) min_sum_soft_limit: u32,
2022-05-05 22:07:09 +03:00
}
impl Web3Connections {
/// 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>,
2022-08-10 08:56:09 +03:00
server_configs: HashMap<String, Web3ConnectionConfig>,
http_client: Option<reqwest::Client>,
2022-09-15 20:57:24 +03:00
redis_pool: Option<redis_rate_limiter::RedisPool>,
block_map: BlockHashesCache,
2023-01-23 09:02:08 +03:00
watch_consensus_head_sender: Option<watch::Sender<ArcBlock>>,
2022-08-27 06:11:58 +03:00
min_sum_soft_limit: u32,
min_head_rpcs: usize,
2022-08-24 03:59:05 +03:00
pending_tx_sender: Option<broadcast::Sender<TxStatus>>,
pending_transactions: Cache<TxHash, TxStatus, hashbrown::hash_map::DefaultHashBuilder>,
2022-09-09 06:53:16 +03:00
open_request_handle_metrics: Arc<OpenRequestHandleMetrics>,
2022-06-14 08:43:28 +03:00
) -> anyhow::Result<(Arc<Self>, AnyhowJoinHandle<()>)> {
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-01-03 22:37:42 +03:00
// TODO: query the rpc to get the actual expected block time, or get from config?
let expected_block_time_ms = match chain_id {
// ethereum
1 => 12_000,
// 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
_ => {
warn!("unexpected chain_id. polling every {} seconds", 10);
10_000
}
};
2022-06-29 22:15:05 +03:00
let http_interval_sender = if http_client.is_some() {
let (sender, receiver) = broadcast::channel(1);
drop(receiver);
// TODO: what interval? follow a websocket also? maybe by watching synced connections with a timeout. will need debounce
2023-01-03 22:37:42 +03:00
let mut interval = interval(Duration::from_millis(expected_block_time_ms));
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 {
// TODO: every time a head_block arrives (with a small delay for known slow servers), or on the interval.
2022-06-29 22:15:05 +03:00
interval.tick().await;
2022-11-12 11:24:32 +03:00
// // trace!("http interval ready");
2022-07-16 08:21:08 +03:00
2022-06-29 22:15:05 +03:00
// errors are okay. they mean that all receivers have been dropped
let _ = sender.send(());
}
}
};
// TODO: do something with this handle?
tokio::spawn(f);
Some(sender)
} else {
None
};
// turn configs into connections (in parallel)
2022-08-12 22:07:14 +03:00
// TODO: move this into a helper function. then we can use it when configs change (will need a remove function too)
// TODO: futures unordered?
let spawn_handles: Vec<_> = server_configs
.into_iter()
2022-10-19 02:27:33 +03:00
.filter_map(|(server_name, server_config)| {
if server_config.disabled {
return None;
}
let db_conn = db_conn.clone();
let http_client = http_client.clone();
let redis_pool = redis_pool.clone();
let http_interval_sender = http_interval_sender.clone();
2023-01-23 09:02:08 +03:00
let block_sender = if watch_consensus_head_sender.is_some() {
Some(block_sender.clone())
} else {
None
};
let pending_tx_id_sender = Some(pending_tx_id_sender.clone());
2022-08-26 20:26:17 +03:00
let block_map = block_map.clone();
2022-09-09 06:53:16 +03:00
let open_request_handle_metrics = open_request_handle_metrics.clone();
2022-10-19 02:27:33 +03:00
let handle = tokio::spawn(async move {
server_config
.spawn(
2022-08-10 08:56:09 +03:00
server_name,
db_conn,
redis_pool,
chain_id,
http_client,
http_interval_sender,
2022-08-26 20:26:17 +03:00
block_map,
block_sender,
pending_tx_id_sender,
2022-09-09 06:53:16 +03:00
open_request_handle_metrics,
)
.await
2022-10-19 02:27:33 +03:00
});
Some(handle)
})
.collect();
// map of connection names to their connection
2022-08-26 20:26:17 +03:00
let mut connections = HashMap::new();
let mut handles = vec![];
// TODO: futures unordered?
for x in join_all(spawn_handles).await {
// TODO: how should we handle errors here? one rpc being down shouldn't cause the program to exit
match x {
Ok(Ok((connection, handle))) => {
connections.insert(connection.name.clone(), connection);
handles.push(handle);
}
Ok(Err(err)) => {
2022-09-14 06:00:35 +03:00
// if we got an error here, it is not retryable
// TODO: include context about which connection failed
2022-11-12 11:24:32 +03:00
error!("Unable to create connection. err={:?}", err);
}
Err(err) => {
return Err(err.into());
2022-06-14 08:43:28 +03:00
}
2022-05-06 00:38:15 +03:00
}
2022-05-05 22:07:09 +03:00
}
2022-09-17 05:30:06 +03:00
// TODO: max_capacity and time_to_idle from config
// all block hashes are the same size, so no need for weigher
let block_hashes = Cache::builder()
.time_to_idle(Duration::from_secs(600))
.max_capacity(10_000)
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
let block_numbers = Cache::builder()
.time_to_idle(Duration::from_secs(600))
.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
2023-01-23 09:02:08 +03:00
let (watch_consensus_connections_sender, _) = watch::channel(Default::default());
let watch_consensus_head_receiver =
watch_consensus_head_sender.as_ref().map(|x| x.subscribe());
2022-05-16 01:02:14 +03:00
let connections = Arc::new(Self {
conns: connections,
2023-01-23 09:02:08 +03:00
watch_consensus_connections_sender,
watch_consensus_head_receiver,
2022-06-16 20:51:49 +03:00
pending_transactions,
2022-09-05 08:53:58 +03:00
block_hashes,
block_numbers,
2022-08-27 06:11:58 +03:00
min_sum_soft_limit,
min_head_rpcs,
2022-05-16 01:02:14 +03:00
});
let authorization = Arc::new(Authorization::internal(db_conn.clone())?);
let handle = {
let connections = connections.clone();
tokio::spawn(async move {
2022-06-16 20:51:49 +03:00
// TODO: try_join_all with the other handles here
connections
2022-06-14 08:43:28 +03:00
.subscribe(
authorization,
2022-06-14 08:43:28 +03:00
pending_tx_id_receiver,
block_receiver,
2023-01-23 09:02:08 +03:00
watch_consensus_head_sender,
2022-06-14 08:43:28 +03:00
pending_tx_sender,
)
.await
})
};
2022-06-14 08:43:28 +03:00
Ok((connections, handle))
2022-05-18 19:35:06 +03:00
}
2022-08-26 20:26:17 +03:00
pub fn get(&self, conn_name: &str) -> Option<&Arc<Web3Connection>> {
self.conns.get(conn_name)
2022-08-26 20:26:17 +03:00
}
2022-07-22 22:30:39 +03:00
/// subscribe to blocks and transactions from all the backend rpcs.
/// blocks are processed by all the `Web3Connection`s and then sent to the `block_receiver`
/// transaction ids from all the `Web3Connection`s are deduplicated and forwarded to `pending_tx_sender`
async fn subscribe(
self: Arc<Self>,
authorization: Arc<Authorization>,
2022-08-30 23:01:42 +03:00
pending_tx_id_receiver: flume::Receiver<TxHashAndRpc>,
2022-08-26 20:26:17 +03:00
block_receiver: flume::Receiver<BlockAndRpc>,
2022-08-30 23:01:42 +03:00
head_block_sender: Option<watch::Sender<ArcBlock>>,
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 handle = task::spawn(async move {
2022-08-26 20:26:17 +03:00
// TODO: set up this future the same as the block funnel
2022-06-16 05:53:37 +03:00
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 let Some(head_block_sender) = head_block_sender {
let connections = Arc::clone(&self);
let pending_tx_sender = pending_tx_sender.clone();
2022-05-18 19:35:06 +03:00
let handle = 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
2022-08-26 20:26:17 +03:00
.process_incoming_blocks(
&authorization,
2022-08-26 20:26:17 +03:00
block_receiver,
head_block_sender,
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 = task::Builder::default().name("noop").spawn(async move {
loop {
sleep(Duration::from_secs(600)).await;
2022-08-26 20:26:17 +03:00
// TODO: "every interval, check that the provider is still connected"
2022-08-11 00:29:50 +03:00
}
2022-09-02 23:16:20 +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 {
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-12-24 04:32:58 +03:00
id: Box<RawValue>,
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
2022-12-24 04:32:58 +03:00
) -> anyhow::Result<JsonRpcForwardedResponse> {
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())
.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;
let mut any_ok_but_maybe_json_error = false;
for partial_response in responses {
if partial_response.is_ok() {
any_ok_with_json_result = true;
}
let response =
JsonRpcForwardedResponse::try_from_response_result(partial_response, id.clone());
// TODO: better key?
2022-05-28 07:26:24 +03:00
let s = format!("{:?}", response);
if count_map.get(&s).is_none() {
if response.is_ok() {
2022-12-24 04:32:58 +03:00
any_ok_but_maybe_json_error = true;
2022-05-28 07:26:24 +03:00
}
2022-05-17 19:23:27 +03:00
2022-05-28 07:26:24 +03:00
count_map.insert(s.clone(), response);
}
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) => {
if any_ok_with_json_result && x.error.is_some() {
// this one may be an "Ok", but the json has an error inside it
continue;
}
// return the most common success
return Ok(x);
}
Err(err) => {
if any_ok_but_maybe_json_error {
// the most common is an error, but there is an Ok in here somewhere. loop to find it
continue;
}
return Err(err);
}
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-01-17 09:54:40 +03:00
pub async fn best_consensus_head_connection(
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<Web3Connection>],
min_block_needed: Option<&U64>,
) -> anyhow::Result<OpenRequestResult> {
if let Ok(without_backups) = self
._best_consensus_head_connection(
false,
authorization,
request_metadata,
skip,
min_block_needed,
)
.await
{
// TODO: this might use backups too eagerly. but even when we allow backups, we still prioritize our own
if matches!(without_backups, OpenRequestResult::Handle(_)) {
return Ok(without_backups);
}
2023-01-19 14:05:39 +03:00
}
self._best_consensus_head_connection(
true,
authorization,
request_metadata,
skip,
min_block_needed,
)
.await
}
/// get the best available rpc server with the consensus head block. it might have blocks after the consensus head
async fn _best_consensus_head_connection(
&self,
allow_backups: bool,
authorization: &Arc<Authorization>,
2022-10-12 00:31:34 +03:00
request_metadata: Option<&Arc<RequestMetadata>>,
2022-07-02 04:20:28 +03:00
skip: &[Arc<Web3Connection>],
2022-07-22 22:30:39 +03:00
min_block_needed: Option<&U64>,
2022-08-24 03:14:49 +03:00
) -> anyhow::Result<OpenRequestResult> {
2023-01-04 23:12:44 +03:00
let usable_rpcs_by_head_num_and_weight: BTreeMap<
(Option<U64>, u64),
Vec<Arc<Web3Connection>>,
2023-01-20 05:30:30 +03:00
> = {
2023-01-23 09:02:08 +03:00
let synced_connections = self.watch_consensus_connections_sender.borrow().clone();
2022-11-07 00:05:03 +03:00
2023-01-20 05:30:30 +03:00
let head_block_num = if let Some(head_block) = synced_connections.head_block.as_ref() {
head_block.number()
} else {
2023-01-23 09:02:08 +03:00
// TODO: optionally wait for a head block >= min_block_needed
return Ok(OpenRequestResult::NotReady(allow_backups));
2023-01-20 05:30:30 +03:00
};
2023-01-03 19:33:49 +03:00
2023-01-20 05:30:30 +03:00
let min_block_needed = min_block_needed.unwrap_or(&head_block_num);
2023-01-04 09:37:51 +03:00
2023-01-20 05:30:30 +03:00
let mut m = BTreeMap::new();
2023-01-04 09:37:51 +03:00
2023-01-20 05:30:30 +03:00
match min_block_needed.cmp(&head_block_num) {
cmp::Ordering::Less => {
// need an old block. check all the rpcs. prefer the most synced
for x in self
.conns
.values()
.filter(|x| if allow_backups { true } else { !x.backup })
.filter(|x| !skip.contains(x))
.filter(|x| x.has_block_data(min_block_needed))
.cloned()
{
let x_head_block = x.head_block.read().clone();
match x_head_block {
None => continue,
Some(x_head) => {
let key = (Some(x_head.number()), u64::MAX - x.tier);
m.entry(key).or_insert_with(Vec::new).push(x);
}
}
}
}
cmp::Ordering::Equal => {
// need the consensus head block. filter the synced rpcs
for x in synced_connections
.conns
.iter()
.filter(|x| !skip.contains(x))
{
let key = (None, u64::MAX - x.tier);
m.entry(key).or_insert_with(Vec::new).push(x.clone());
}
}
cmp::Ordering::Greater => {
2023-01-23 09:02:08 +03:00
// TODO? if the blocks is close and wait_for_sync and allow_backups, wait for change on a watch_consensus_connections_receiver().subscribe()
return Ok(OpenRequestResult::NotReady(allow_backups));
2023-01-20 05:30:30 +03:00
}
2023-01-04 23:12:44 +03:00
}
2022-11-25 10:41:53 +03:00
2023-01-04 23:12:44 +03:00
m
};
2022-07-02 04:20:28 +03:00
2022-12-08 09:54:38 +03:00
let mut earliest_retry_at = None;
2022-11-12 09:11:58 +03:00
2023-01-04 09:37:51 +03:00
for usable_rpcs in usable_rpcs_by_head_num_and_weight.into_values().rev() {
// under heavy load, it is possible for even our best server to be negative
2022-12-08 09:54:38 +03:00
let mut minimum = f64::MAX;
2023-01-04 23:07:53 +03:00
let mut maximum = f64::MIN;
2022-05-06 23:44:12 +03:00
2022-12-08 09:54:38 +03:00
// we sort on a combination of values. cache them here so that we don't do this math multiple times.
let mut available_request_map: HashMap<_, f64> = usable_rpcs
.iter()
.map(|rpc| {
// TODO: are active requests what we want? do we want a counter for requests in the last second + any actives longer than that?
// TODO: get active requests out of redis (that's definitely too slow)
// TODO: do something with hard limit instead? (but that is hitting redis too much)
let active_requests = rpc.active_requests() as f64;
2023-01-04 09:37:51 +03:00
let soft_limit = rpc.soft_limit as f64;
2022-05-06 23:44:12 +03:00
2022-12-08 09:54:38 +03:00
let available_requests = soft_limit - active_requests;
2022-11-25 10:41:53 +03:00
// trace!("available requests on {}: {}", rpc, available_requests);
2023-01-04 23:07:53 +03:00
minimum = minimum.min(available_requests);
maximum = maximum.max(available_requests);
2022-05-05 22:07:09 +03:00
2022-12-08 09:54:38 +03:00
(rpc, available_requests)
})
.collect();
2022-11-25 10:41:53 +03:00
// trace!("minimum available requests: {}", minimum);
// trace!("maximum available requests: {}", maximum);
2023-01-04 23:07:53 +03:00
if maximum < 0.0 {
// TODO: if maximum < 0 and there are other tiers on the same block, we should include them now
warn!("soft limits overloaded: {} to {}", minimum, maximum)
}
2022-11-12 09:11:58 +03:00
2023-01-04 09:37:51 +03:00
// choose_multiple_weighted can't have negative numbers. shift up if any are negative
// TODO: is this a correct way to shift?
2022-12-08 09:54:38 +03:00
if minimum < 0.0 {
available_request_map = available_request_map
.into_iter()
2023-01-14 00:45:48 +03:00
.map(|(rpc, available_requests)| {
2022-12-08 09:54:38 +03:00
// TODO: is simple addition the right way to shift everyone?
// TODO: probably want something non-linear
2023-01-14 00:45:48 +03:00
// minimum is negative, so we subtract to make available requests bigger
let x = available_requests - minimum;
2022-05-06 23:44:12 +03:00
2022-12-08 09:54:38 +03:00
(rpc, x)
2022-11-25 10:41:53 +03:00
})
2022-12-08 09:54:38 +03:00
.collect()
2022-11-25 10:41:53 +03:00
}
2022-05-05 22:07:09 +03:00
2022-12-08 09:54:38 +03:00
let sorted_rpcs = {
if usable_rpcs.len() == 1 {
2023-01-04 23:07:53 +03:00
// TODO: return now instead? we shouldn't need another alloc
2022-12-08 09:54:38 +03:00
vec![usable_rpcs.get(0).expect("there should be 1")]
} else {
let mut rng = thread_fast_rng::thread_fast_rng();
usable_rpcs
.choose_multiple_weighted(&mut rng, usable_rpcs.len(), |rpc| {
*available_request_map
.get(rpc)
2023-01-14 00:45:48 +03:00
.expect("rpc should always be in available_request_map")
2022-12-08 09:54:38 +03:00
})
.unwrap()
.collect::<Vec<_>>()
2022-08-07 09:48:57 +03:00
}
2022-12-08 09:54:38 +03:00
};
// now that the rpcs are sorted, try to get an active request handle for one of them
for best_rpc in sorted_rpcs.into_iter() {
// increment our connection counter
2022-12-21 08:47:41 +03:00
match best_rpc
.try_request_handle(authorization, min_block_needed.is_none())
.await
{
2022-12-08 09:54:38 +03:00
Ok(OpenRequestResult::Handle(handle)) => {
// trace!("opened handle: {}", best_rpc);
2022-12-08 09:54:38 +03:00
return Ok(OpenRequestResult::Handle(handle));
}
Ok(OpenRequestResult::RetryAt(retry_at)) => {
earliest_retry_at = earliest_retry_at.min(Some(retry_at));
}
Ok(OpenRequestResult::NotReady(_)) => {
2022-12-20 00:53:38 +03:00
// TODO: log a warning? emit a stat?
2022-12-08 09:54:38 +03:00
}
Err(err) => {
warn!("No request handle for {}. err={:?}", best_rpc, err)
}
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::Release);
}
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
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))
2022-12-20 00:53:38 +03:00
// TODO: should we log here?
Ok(OpenRequestResult::NotReady(allow_backups))
2022-09-28 20:01:11 +03:00
}
Some(earliest_retry_at) => {
warn!("no servers on {:?}! {:?}", self, earliest_retry_at);
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-01-24 12:58:31 +03:00
pub async fn all_connections(
2022-07-09 06:34:39 +03:00
&self,
authorization: &Arc<Authorization>,
2022-09-07 06:54:16 +03:00
block_needed: Option<&U64>,
2023-01-12 01:51:01 +03:00
max_count: Option<usize>,
2023-01-19 14:05:39 +03:00
) -> Result<Vec<OpenRequestHandle>, Option<Instant>> {
if let Ok(without_backups) = self
2023-01-24 12:58:31 +03:00
._all_connections(false, authorization, block_needed, max_count)
2023-01-19 14:05:39 +03:00
.await
{
return Ok(without_backups);
}
2023-01-24 12:58:31 +03:00
self._all_connections(true, authorization, block_needed, max_count)
2023-01-19 14:05:39 +03:00
.await
}
2023-01-24 12:58:31 +03:00
async fn _all_connections(
2023-01-19 14:05:39 +03:00
&self,
allow_backups: bool,
authorization: &Arc<Authorization>,
block_needed: Option<&U64>,
max_count: Option<usize>,
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
// TODO: with capacity?
let mut selected_rpcs = vec![];
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-01-24 12:58:31 +03:00
self.conns.len()
2023-01-12 01:51:01 +03:00
};
2023-01-24 12:58:31 +03:00
let mut tried = HashSet::new();
let conns_to_try = itertools::chain(
// TODO: sort by tier
self.watch_consensus_connections_sender
.borrow()
.conns
.clone(),
// TODO: sort by tier
self.conns.values().cloned(),
);
for connection in conns_to_try {
2023-01-12 01:51:01 +03:00
if max_count == 0 {
break;
}
2023-01-24 12:58:31 +03:00
if tried.contains(&connection.name) {
continue;
}
tried.insert(connection.name.clone());
2023-01-19 14:05:39 +03:00
if !allow_backups && connection.backup {
continue;
}
2022-09-07 06:54:16 +03:00
if let Some(block_needed) = block_needed {
if !connection.has_block_data(block_needed) {
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
2022-12-21 08:47:41 +03:00
match connection
.try_request_handle(authorization, block_needed.is_none())
.await
{
2022-08-24 03:14:49 +03:00
Ok(OpenRequestResult::RetryAt(retry_at)) => {
// this rpc is not available. skip it
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)) => {
max_count -= 1;
selected_rpcs.push(handle)
}
Ok(OpenRequestResult::NotReady(_)) => {
2022-08-07 09:48:57 +03:00
warn!("no request handle for {}", connection)
}
Err(err) => {
2022-11-12 11:24:32 +03:00
warn!(
"error getting request handle for {}. err={:?}",
connection, 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>,
2022-05-29 04:23:58 +03:00
request: JsonRpcRequest,
request_metadata: Option<&Arc<RequestMetadata>>,
2022-07-22 22:30:39 +03:00
min_block_needed: Option<&U64>,
2022-05-29 04:23:58 +03:00
) -> anyhow::Result<JsonRpcForwardedResponse> {
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
let mut watch_consensus_connections = self.watch_consensus_connections_sender.subscribe();
2023-01-23 09:02:08 +03:00
// TODO: maximum retries? right now its the total number of servers
2022-05-29 04:23:58 +03:00
loop {
let num_skipped = skip_rpcs.len();
if num_skipped > 0 {
// trace!("skip_rpcs: {:?}", skip_rpcs);
// TODO: is self.conns still right now that we split main and backup servers?
if num_skipped == self.conns.len() {
break;
}
2022-07-02 04:20:28 +03:00
}
2022-07-19 04:31:12 +03:00
match self
2023-01-17 09:54:40 +03:00
.best_consensus_head_connection(
authorization,
2022-10-12 00:31:34 +03:00
request_metadata,
&skip_rpcs,
min_block_needed,
)
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
2022-07-02 04:20:28 +03:00
skip_rpcs.push(active_request_handle.clone_connection());
if let Some(request_metadata) = request_metadata {
let rpc = active_request_handle.clone_connection();
request_metadata
.response_from_backup_rpc
.store(rpc.backup, Ordering::Release);
request_metadata.backend_requests.lock().push(rpc);
}
// TODO: get the log percent from the user data
2022-05-29 04:23:58 +03:00
let response_result = active_request_handle
.request(
&request.method,
&json!(request.params),
2023-01-17 09:54:40 +03:00
RequestRevertHandler::Save,
)
2022-05-29 04:23:58 +03:00
.await;
2022-07-02 04:20:28 +03:00
match JsonRpcForwardedResponse::try_from_response_result(
2022-06-04 00:45:44 +03:00
response_result,
request.id.clone(),
) {
Ok(response) => {
if let Some(error) = &response.error.as_ref() {
2022-12-20 02:59:01 +03:00
// trace!(?response, "rpc error");
2022-07-02 04:20:28 +03:00
if let Some(request_metadata) = request_metadata {
request_metadata
.error_response
.store(true, Ordering::Release);
}
2022-07-09 07:25:59 +03:00
// some errors should be retried on other nodes
let error_msg = error.message.as_str();
// different providers do different codes. check all of them
// TODO: there's probably more strings to add here
let rate_limit_substrings = ["limit", "exceeded"];
for rate_limit_substr in rate_limit_substrings {
if error_msg.contains(rate_limit_substr) {
2023-01-20 04:57:36 +03:00
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) {
continue;
}
}
}
-32601 => {
let error_msg = error.message.as_str();
// 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(response);
continue;
}
// others look like this
if error_msg == "Method not found" {
method_not_available_response = Some(response);
2022-07-09 07:25:59 +03:00
continue;
}
}
_ => {}
2022-07-02 04:20:28 +03:00
}
2022-06-04 00:45:44 +03:00
} else {
2022-12-20 02:59:01 +03:00
// trace!(?response, "rpc success");
2022-06-04 00:45:44 +03:00
}
2022-05-29 04:23:58 +03:00
2022-06-04 00:45:44 +03:00
return Ok(response);
}
2022-11-12 11:24:32 +03:00
Err(err) => {
2022-09-10 03:58:33 +03:00
let rpc = skip_rpcs
.last()
.expect("there must have been a provider if we got an error");
2022-12-17 07:05:01 +03:00
// TODO: emit a stat. if a server is getting skipped a lot, something is not right
debug!(
2022-11-12 11:24:32 +03:00
"Backend server error on {}! Retrying on another. err={:?}",
rpc, err
);
2022-05-29 04:23:58 +03:00
2022-06-04 00:45:44 +03:00
continue;
}
}
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 {:?}",
retry_at
);
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::Release);
}
tokio::select! {
_ = sleep_until(retry_at) => {
skip_rpcs.pop();
}
_ = watch_consensus_connections.changed() => {
watch_consensus_connections.borrow_and_update();
2023-01-23 09:02:08 +03:00
}
}
2022-05-29 04:23:58 +03:00
}
OpenRequestResult::NotReady(backups_included) => {
2022-10-12 00:31:34 +03:00
if let Some(request_metadata) = request_metadata {
request_metadata.no_servers.fetch_add(1, Ordering::Release);
}
// todo!(
// "check if we are requesting an old block and no archive servers are synced"
// );
if let Some(min_block_needed) = min_block_needed {
let mut theres_a_chance = false;
for potential_conn in self.conns.values() {
if skip_rpcs.contains(potential_conn) {
continue;
}
// TODO: should we instead check if has_block_data but with the current head block?
if potential_conn.has_block_data(min_block_needed) {
trace!("chance for {} on {}", min_block_needed, potential_conn);
theres_a_chance = true;
break;
}
skip_rpcs.push(potential_conn.clone());
}
if !theres_a_chance {
debug!("no chance of finding data in block #{}", min_block_needed);
break;
}
}
if backups_included {
// if NotReady and we tried backups, there's no chance
warn!("No servers ready even after checking backups");
break;
}
debug!("No servers ready. Waiting up to 1 second for change in synced servers");
// TODO: exponential backoff?
tokio::select! {
_ = sleep(Duration::from_secs(1)) => {
// do NOT pop the last rpc off skip here
}
_ = watch_consensus_connections.changed() => {
watch_consensus_connections.borrow_and_update();
}
2023-01-23 09:02:08 +03:00
}
2022-05-29 04:23:58 +03:00
}
}
}
2022-07-02 04:20:28 +03:00
if let Some(r) = method_not_available_response {
// TODO: emit a stat for unsupported methods?
return Ok(r);
}
2022-10-12 00:31:34 +03:00
// TODO: do we need this here, or do we do it somewhere else?
if let Some(request_metadata) = request_metadata {
request_metadata
.error_response
.store(true, Ordering::Release);
}
2022-12-24 04:32:58 +03:00
let num_conns = self.conns.len();
let num_skipped = skip_rpcs.len();
2022-11-28 09:52:16 +03:00
if num_skipped == 0 {
error!("No servers synced ({} known)", num_conns);
2022-12-24 04:32:58 +03:00
return Ok(JsonRpcForwardedResponse::from_str(
"No servers synced",
Some(-32000),
Some(request.id),
));
} else {
// TODO: warn? debug? trace?
warn!(
"Requested data was not available on {}/{} servers",
num_skipped, num_conns
);
// TODO: what error code?
// cloudflare gives {"jsonrpc":"2.0","error":{"code":-32043,"message":"Requested data cannot be older than 128 blocks."},"id":1}
return Ok(JsonRpcForwardedResponse::from_str(
"Requested data is not available",
Some(-32043),
Some(request.id),
));
}
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-01-17 09:54:40 +03:00
pub async fn try_send_all_synced_connections(
2022-05-28 07:26:24 +03:00
&self,
authorization: &Arc<Authorization>,
2022-12-24 04:32:58 +03:00
request: &JsonRpcRequest,
request_metadata: Option<Arc<RequestMetadata>>,
2022-09-07 06:54:16 +03:00
block_needed: Option<&U64>,
error_level: Level,
2023-01-12 01:51:01 +03:00
max_count: Option<usize>,
2022-05-28 07:26:24 +03:00
) -> anyhow::Result<JsonRpcForwardedResponse> {
loop {
match self
2023-01-24 12:58:31 +03:00
.all_connections(authorization, block_needed, max_count)
.await
{
2022-05-28 07:26:24 +03:00
Ok(active_request_handles) => {
// TODO: benchmark this compared to waiting on unbounded futures
// TODO: do something with this handle?
// TODO: this is not working right. simplify
2022-10-12 00:31:34 +03:00
if let Some(request_metadata) = request_metadata {
let mut backup_used = false;
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
backup_used = true;
}
x.clone_connection()
}),
);
2022-10-12 00:31:34 +03:00
request_metadata
.response_from_backup_rpc
.store(true, 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,
2022-12-24 04:32:58 +03:00
request.id.clone(),
2022-05-28 07:26:24 +03:00
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) => {
2022-11-12 11:24:32 +03:00
warn!("No servers in sync on {:?}! Retrying", self);
2022-05-29 04:23:58 +03:00
2022-10-12 00:31:34 +03:00
if let Some(request_metadata) = &request_metadata {
request_metadata.no_servers.fetch_add(1, Ordering::Release);
}
2022-05-28 07:26:24 +03:00
// TODO: i don't think this will ever happen
2022-06-03 00:47:43 +03:00
// TODO: return a 502? if it does?
// return Err(anyhow::anyhow!("no available rpcs!"));
// TODO: sleep how long?
// TODO: subscribe to something in ConsensusConnections instead
2022-06-03 00:47:43 +03:00
sleep(Duration::from_millis(200)).await;
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::Release);
}
2022-08-07 09:48:57 +03:00
sleep_until(retry_at).await;
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
}
}
}
}
2023-01-17 09:54:40 +03:00
pub async fn try_proxy_connection(
&self,
proxy_mode: ProxyMode,
authorization: &Arc<Authorization>,
request: JsonRpcRequest,
request_metadata: Option<&Arc<RequestMetadata>>,
min_block_needed: Option<&U64>,
) -> anyhow::Result<JsonRpcForwardedResponse> {
match proxy_mode {
ProxyMode::Best => {
self.try_send_best_consensus_head_connection(
authorization,
request,
request_metadata,
min_block_needed,
)
.await
}
ProxyMode::Fastest(x) => todo!("Fastest"),
ProxyMode::Versus => todo!("Versus"),
}
}
2022-05-05 22:07:09 +03:00
}
2022-07-16 07:13:02 +03:00
impl fmt::Debug for Web3Connections {
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("Web3Connections")
.field("conns", &self.conns)
2022-08-24 02:13:56 +03:00
.finish_non_exhaustive()
}
}
impl Serialize for Web3Connections {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
let mut state = serializer.serialize_struct("Web3Connections", 6)?;
2022-09-22 02:50:55 +03:00
let conns: Vec<&Web3Connection> = self.conns.values().map(|x| x.as_ref()).collect();
state.serialize_field("conns", &conns)?;
2022-09-22 02:50:55 +03:00
2023-01-23 09:02:08 +03:00
{
let consensus_connections = self.watch_consensus_connections_sender.borrow().clone();
// TODO: rename synced_connections to consensus_connections?
state.serialize_field("synced_connections", &consensus_connections)?;
}
2022-09-22 02:50:55 +03:00
self.block_hashes.sync();
self.block_numbers.sync();
state.serialize_field("block_hashes_count", &self.block_hashes.entry_count())?;
state.serialize_field("block_hashes_size", &self.block_hashes.weighted_size())?;
state.serialize_field("block_numbers_count", &self.block_numbers.entry_count())?;
state.serialize_field("block_numbers_size", &self.block_numbers.weighted_size())?;
state.end()
}
}
mod tests {
2022-11-30 00:34:42 +03:00
// TODO: why is this allow needed? does tokio::test get in the way somehow?
2022-12-03 08:31:03 +03:00
#![allow(unused_imports)]
use super::*;
use crate::rpcs::{
blockchain::{ConsensusFinder, SavedBlock},
connection::ProviderState,
provider::Web3Provider,
};
2022-12-01 01:11:14 +03:00
use ethers::types::{Block, U256};
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 std::time::{SystemTime, UNIX_EPOCH};
use tokio::sync::RwLock as AsyncRwLock;
#[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();
2022-12-01 01:11:14 +03:00
let now: U256 = SystemTime::now()
.duration_since(UNIX_EPOCH)
.unwrap()
.as_secs()
.into();
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);
2022-12-03 08:31:03 +03:00
// TODO: write a impl From for Block -> BlockId?
let mut lagged_block: SavedBlock = lagged_block.into();
let mut head_block: SavedBlock = head_block.into();
2022-12-03 08:31:03 +03:00
let block_data_limit = u64::MAX;
let head_rpc = Web3Connection {
name: "synced".to_string(),
db_conn: None,
display_name: None,
url: "ws://example.com/synced".to_string(),
http_client: None,
active_requests: 0.into(),
frontend_requests: 0.into(),
internal_requests: 0.into(),
provider_state: AsyncRwLock::new(ProviderState::Connected(Arc::new(
Web3Provider::Mock,
))),
hard_limit: None,
hard_limit_until: None,
soft_limit: 1_000,
2022-12-06 00:13:36 +03:00
automatic_block_limit: true,
backup: false,
block_data_limit: block_data_limit.into(),
2023-01-04 09:37:51 +03:00
tier: 0,
2022-12-03 08:31:03 +03:00
head_block: RwLock::new(Some(head_block.clone())),
open_request_handle_metrics: Arc::new(Default::default()),
};
let lagged_rpc = Web3Connection {
name: "lagged".to_string(),
db_conn: None,
display_name: None,
url: "ws://example.com/lagged".to_string(),
http_client: None,
active_requests: 0.into(),
frontend_requests: 0.into(),
internal_requests: 0.into(),
provider_state: AsyncRwLock::new(ProviderState::Connected(Arc::new(
Web3Provider::Mock,
))),
hard_limit: None,
hard_limit_until: None,
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,
2022-12-03 08:31:03 +03:00
head_block: RwLock::new(Some(lagged_block.clone())),
open_request_handle_metrics: Arc::new(Default::default()),
};
2022-12-03 08:31:03 +03:00
assert!(head_rpc.has_block_data(&lagged_block.number()));
assert!(head_rpc.has_block_data(&head_block.number()));
2022-12-03 08:31:03 +03:00
assert!(lagged_rpc.has_block_data(&lagged_block.number()));
assert!(!lagged_rpc.has_block_data(&head_block.number()));
let head_rpc = Arc::new(head_rpc);
let lagged_rpc = Arc::new(lagged_rpc);
let conns = HashMap::from([
(head_rpc.name.clone(), head_rpc.clone()),
(lagged_rpc.name.clone(), lagged_rpc.clone()),
]);
2023-01-23 09:02:08 +03:00
let (watch_consensus_connections_sender, _) = watch::channel(Default::default());
// TODO: make a Web3Connections::new
let conns = Web3Connections {
conns,
2023-01-23 09:02:08 +03:00
watch_consensus_head_receiver: None,
watch_consensus_connections_sender,
pending_transactions: Cache::builder()
.max_capacity(10_000)
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()),
block_hashes: Cache::builder()
.max_capacity(10_000)
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()),
block_numbers: Cache::builder()
.max_capacity(10_000)
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()),
min_head_rpcs: 1,
min_sum_soft_limit: 1,
};
let authorization = Arc::new(Authorization::internal(None).unwrap());
let (head_block_sender, _head_block_receiver) =
watch::channel::<ArcBlock>(Default::default());
let mut connection_heads = ConsensusFinder::default();
// process None so that
conns
.process_block_from_rpc(
&authorization,
&mut connection_heads,
None,
lagged_rpc.clone(),
&head_block_sender,
&None,
)
.await
.unwrap();
conns
.process_block_from_rpc(
&authorization,
&mut connection_heads,
None,
head_rpc.clone(),
&head_block_sender,
&None,
)
.await
.unwrap();
// no head block because the rpcs haven't communicated through their channels
assert!(conns.head_block_hash().is_none());
// all_backend_connections gives everything regardless of sync status
assert_eq!(
conns
2023-01-24 12:58:31 +03:00
.all_connections(&authorization, None, None)
.await
.unwrap()
.len(),
2
);
// best_synced_backend_connection requires servers to be synced with the head block
2022-11-28 09:52:16 +03:00
let x = conns
2023-01-19 14:05:39 +03:00
.best_consensus_head_connection(&authorization, None, &[], None)
2022-11-28 09:52:16 +03:00
.await
.unwrap();
dbg!(&x);
assert!(matches!(x, OpenRequestResult::NotReady(true)));
// add lagged blocks to the conns. both servers should be allowed
lagged_block.block = conns.save_block(lagged_block.block, true).await.unwrap();
conns
.process_block_from_rpc(
&authorization,
&mut connection_heads,
Some(lagged_block.clone()),
lagged_rpc,
&head_block_sender,
&None,
)
.await
.unwrap();
conns
.process_block_from_rpc(
&authorization,
&mut connection_heads,
Some(lagged_block.clone()),
head_rpc.clone(),
&head_block_sender,
&None,
)
.await
.unwrap();
assert_eq!(conns.num_synced_rpcs(), 2);
// add head block to the conns. lagged_rpc should not be available
head_block.block = conns.save_block(head_block.block, true).await.unwrap();
conns
.process_block_from_rpc(
&authorization,
&mut connection_heads,
Some(head_block.clone()),
head_rpc,
&head_block_sender,
&None,
)
.await
.unwrap();
assert_eq!(conns.num_synced_rpcs(), 1);
2022-11-25 10:41:53 +03:00
assert!(matches!(
conns
2023-01-19 14:05:39 +03:00
.best_consensus_head_connection(&authorization, None, &[], None)
2022-11-25 10:41:53 +03:00
.await,
Ok(OpenRequestResult::Handle(_))
));
assert!(matches!(
conns
2023-01-19 14:05:39 +03:00
.best_consensus_head_connection(&authorization, None, &[], Some(&0.into()))
2022-11-25 10:41:53 +03:00
.await,
Ok(OpenRequestResult::Handle(_))
));
assert!(matches!(
conns
2023-01-19 14:05:39 +03:00
.best_consensus_head_connection(&authorization, None, &[], Some(&1.into()))
2022-11-25 10:41:53 +03:00
.await,
Ok(OpenRequestResult::Handle(_))
));
// future block should not get a handle
assert!(matches!(
conns
2023-01-19 14:05:39 +03:00
.best_consensus_head_connection(&authorization, None, &[], Some(&2.into()))
2022-11-25 10:41:53 +03:00
.await,
Ok(OpenRequestResult::NotReady(true))
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();
2022-12-01 01:11:14 +03:00
let now: U256 = SystemTime::now()
.duration_since(UNIX_EPOCH)
.unwrap()
.as_secs()
.into();
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()
};
2022-12-03 08:31:03 +03:00
let head_block: SavedBlock = Arc::new(head_block).into();
2022-11-25 10:41:53 +03:00
let pruned_rpc = Web3Connection {
name: "pruned".to_string(),
db_conn: None,
2022-11-25 10:41:53 +03:00
display_name: None,
url: "ws://example.com/pruned".to_string(),
http_client: None,
active_requests: 0.into(),
frontend_requests: 0.into(),
internal_requests: 0.into(),
provider_state: AsyncRwLock::new(ProviderState::Connected(Arc::new(
Web3Provider::Mock,
))),
2022-11-25 10:41:53 +03:00
hard_limit: None,
hard_limit_until: None,
2022-11-25 10:41:53 +03:00
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,
2022-12-03 08:31:03 +03:00
head_block: RwLock::new(Some(head_block.clone())),
2022-11-25 10:41:53 +03:00
open_request_handle_metrics: Arc::new(Default::default()),
};
let archive_rpc = Web3Connection {
name: "archive".to_string(),
db_conn: None,
2022-11-25 10:41:53 +03:00
display_name: None,
url: "ws://example.com/archive".to_string(),
http_client: None,
active_requests: 0.into(),
frontend_requests: 0.into(),
internal_requests: 0.into(),
provider_state: AsyncRwLock::new(ProviderState::Connected(Arc::new(
Web3Provider::Mock,
))),
2022-11-25 10:41:53 +03:00
hard_limit: None,
hard_limit_until: None,
2022-11-25 10:41:53 +03:00
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,
2022-12-03 08:31:03 +03:00
head_block: RwLock::new(Some(head_block.clone())),
2022-11-25 10:41:53 +03:00
open_request_handle_metrics: Arc::new(Default::default()),
};
2022-12-03 08:31:03 +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);
let conns = HashMap::from([
(pruned_rpc.name.clone(), pruned_rpc.clone()),
(archive_rpc.name.clone(), archive_rpc.clone()),
]);
2023-01-23 09:02:08 +03:00
let (watch_consensus_connections_sender, _) = watch::channel(Default::default());
// TODO: make a Web3Connections::new
2022-11-25 10:41:53 +03:00
let conns = Web3Connections {
conns,
2023-01-23 09:02:08 +03:00
watch_consensus_head_receiver: None,
watch_consensus_connections_sender,
2022-11-25 10:41:53 +03:00
pending_transactions: Cache::builder()
.max_capacity(10)
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()),
block_hashes: Cache::builder()
.max_capacity(10)
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()),
block_numbers: Cache::builder()
.max_capacity(10)
.build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()),
min_head_rpcs: 1,
min_sum_soft_limit: 3_000,
};
let authorization = Arc::new(Authorization::internal(None).unwrap());
let (head_block_sender, _head_block_receiver) =
watch::channel::<ArcBlock>(Default::default());
let mut connection_heads = ConsensusFinder::default();
2022-11-25 10:41:53 +03:00
conns
.process_block_from_rpc(
&authorization,
&mut connection_heads,
Some(head_block.clone()),
pruned_rpc.clone(),
&head_block_sender,
&None,
)
.await
2022-11-25 10:41:53 +03:00
.unwrap();
conns
.process_block_from_rpc(
&authorization,
&mut connection_heads,
Some(head_block.clone()),
archive_rpc.clone(),
&head_block_sender,
&None,
)
.await
2022-11-25 10:41:53 +03:00
.unwrap();
assert_eq!(conns.num_synced_rpcs(), 2);
// best_synced_backend_connection requires servers to be synced with the head block
let best_head_server = conns
2023-01-19 14:05:39 +03:00
.best_consensus_head_connection(&authorization, None, &[], Some(&head_block.number()))
2022-11-25 10:41:53 +03:00
.await;
assert!(matches!(
best_head_server.unwrap(),
OpenRequestResult::Handle(_)
));
let best_archive_server = conns
2023-01-19 14:05:39 +03:00
.best_consensus_head_connection(&authorization, None, &[], Some(&1.into()))
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);
}
}
}
}