web3-proxy/web3_proxy/src/app/mod.rs

1771 lines
72 KiB
Rust
Raw Normal View History

mod ws;
2022-08-10 05:37:34 +03:00
use crate::block_number::CacheMode;
use crate::caches::{RegisteredUserRateLimitKey, RpcSecretKeyCache, UserBalanceCache};
2022-08-24 03:59:05 +03:00
use crate::config::{AppConfig, TopConfig};
use crate::errors::{Web3ProxyError, Web3ProxyErrorContext, Web3ProxyResult};
use crate::frontend::authorization::{
Authorization, RequestMetadata, RequestOrMethod, ResponseOrBytes,
};
2023-01-17 09:54:40 +03:00
use crate::frontend::rpc_proxy_ws::ProxyMode;
use crate::globals::{global_db_conn, DatabaseError, DB_CONN, DB_REPLICA};
use crate::jsonrpc::{
JsonRpcErrorData, JsonRpcForwardedResponse, JsonRpcForwardedResponseEnum, JsonRpcId,
JsonRpcParams, JsonRpcRequest, JsonRpcRequestEnum, JsonRpcResultData,
};
use crate::relational_db::{connect_db, migrate_db};
use crate::response_cache::{
JsonRpcQueryCacheKey, JsonRpcResponseCache, JsonRpcResponseEnum, JsonRpcResponseWeigher,
};
use crate::rpcs::blockchain::Web3ProxyBlock;
2023-06-27 22:36:41 +03:00
use crate::rpcs::consensus::RankedRpcs;
use crate::rpcs::many::Web3Rpcs;
use crate::rpcs::one::Web3Rpc;
2023-05-31 08:31:35 +03:00
use crate::rpcs::provider::{connect_http, EthersHttpProvider};
More balance tests (#182) * fix popularity contest * more info in the Debug for Web3Rpc * add frontend_requests and cache_misses to the Balance query * add more to balance and stats flushing and improved test coverage * it compiles * deserializer for Ulid to Uuid I think a wrapper type on Ulid that implements sea_orm::Value is probably better * rename variable to match struct name * add deserializer for Address -> Vec<u8> * sql sum returns a Decimal. need to convert to u64 * assert more * one log and assert more * log more * use a helper to get the user's rpc provider * this should be 2 now that we have a public and authed call * this should be zero. the public has the cache miss * instrument cu calcs * trace the value we took, not the default that replaced it * move usd_per_chain into config * remove some extra logging * use Arc::into_inner to maybe avoid a race * off by 1 * pass paid credits used instead of returning it this lets us use it to write to our user balance cache first. importantly, this keeps us from holding a write lock while writing to mysql * no cache misses expected in this test * actually check the admin * put the balance checks back now that the rest of the test works * archive request is being set incorrectly * wow howd we manage flipping the greater than sign on archive depth * move latest_balance and premium_credits_used to before any stats are emitted * lint * and build undoes the linting. fun i didnt even want to lint them in the first place, so this is fine * missed incrementing total_spent when not incrementing total_spent_paid_credits * use the credits on self * use the credits on self (pt 2) * fix type for 10 cu query * convert the requestmetadata on the other side of the channel * logs * viewing stats is allowed even without a balance * move paid_credits_used to AuthorizationChecks * wip * test_sum_credits_used finally passes * UserBalanceCache::get_or_insert * re-enable rpc_secret_key_cache * move invalidate to a helper function and always call it **after** the db is commited * fix PartialEq and Eq on RpcSecretKey * cargo upgrade
2023-07-12 10:35:07 +03:00
use crate::stats::{AppStat, FlushedStats, StatBuffer};
2022-07-26 07:53:38 +03:00
use anyhow::Context;
use axum::http::StatusCode;
2022-12-28 09:11:18 +03:00
use chrono::Utc;
2022-09-15 20:57:24 +03:00
use deferred_rate_limiter::DeferredRateLimiter;
2023-01-03 04:06:36 +03:00
use entities::user;
2022-07-22 22:30:39 +03:00
use ethers::core::utils::keccak256;
use ethers::prelude::{Address, Bytes, Transaction, H256, U64};
2023-01-20 05:08:53 +03:00
use ethers::types::U256;
2022-12-24 04:32:58 +03:00
use ethers::utils::rlp::{Decodable, Rlp};
2023-02-27 10:52:37 +03:00
use futures::future::join_all;
use futures::stream::{FuturesUnordered, StreamExt};
use hashbrown::{HashMap, HashSet};
use migration::sea_orm::{EntityTrait, PaginatorTrait};
2023-06-08 03:26:38 +03:00
use moka::future::{Cache, CacheBuilder};
use once_cell::sync::OnceCell;
2022-12-29 09:21:09 +03:00
use redis_rate_limiter::redis::AsyncCommands;
use redis_rate_limiter::{redis, DeadpoolRuntime, RedisConfig, RedisPool, RedisRateLimiter};
2022-09-09 06:53:16 +03:00
use serde::Serialize;
2022-05-30 21:23:55 +03:00
use serde_json::json;
use serde_json::value::RawValue;
2023-02-27 10:52:37 +03:00
use std::fmt;
2022-09-15 20:57:24 +03:00
use std::net::IpAddr;
2023-06-08 03:26:38 +03:00
use std::num::NonZeroU64;
2022-07-14 02:25:01 +03:00
use std::str::FromStr;
use std::sync::atomic::{AtomicU16, Ordering};
use std::sync::{atomic, Arc};
2022-05-12 02:50:52 +03:00
use std::time::Duration;
use tokio::select;
More balance tests (#182) * fix popularity contest * more info in the Debug for Web3Rpc * add frontend_requests and cache_misses to the Balance query * add more to balance and stats flushing and improved test coverage * it compiles * deserializer for Ulid to Uuid I think a wrapper type on Ulid that implements sea_orm::Value is probably better * rename variable to match struct name * add deserializer for Address -> Vec<u8> * sql sum returns a Decimal. need to convert to u64 * assert more * one log and assert more * log more * use a helper to get the user's rpc provider * this should be 2 now that we have a public and authed call * this should be zero. the public has the cache miss * instrument cu calcs * trace the value we took, not the default that replaced it * move usd_per_chain into config * remove some extra logging * use Arc::into_inner to maybe avoid a race * off by 1 * pass paid credits used instead of returning it this lets us use it to write to our user balance cache first. importantly, this keeps us from holding a write lock while writing to mysql * no cache misses expected in this test * actually check the admin * put the balance checks back now that the rest of the test works * archive request is being set incorrectly * wow howd we manage flipping the greater than sign on archive depth * move latest_balance and premium_credits_used to before any stats are emitted * lint * and build undoes the linting. fun i didnt even want to lint them in the first place, so this is fine * missed incrementing total_spent when not incrementing total_spent_paid_credits * use the credits on self * use the credits on self (pt 2) * fix type for 10 cu query * convert the requestmetadata on the other side of the channel * logs * viewing stats is allowed even without a balance * move paid_credits_used to AuthorizationChecks * wip * test_sum_credits_used finally passes * UserBalanceCache::get_or_insert * re-enable rpc_secret_key_cache * move invalidate to a helper function and always call it **after** the db is commited * fix PartialEq and Eq on RpcSecretKey * cargo upgrade
2023-07-12 10:35:07 +03:00
use tokio::sync::{broadcast, mpsc, oneshot, watch, Semaphore};
2022-06-14 08:43:28 +03:00
use tokio::task::JoinHandle;
use tokio::time::{sleep, timeout};
use tracing::{error, info, trace, warn, Level};
2022-05-12 02:50:52 +03:00
// TODO: make this customizable?
2023-01-18 07:18:18 +03:00
// TODO: include GIT_REF in here. i had trouble getting https://docs.rs/vergen/latest/vergen/ to work with a workspace. also .git is in .dockerignore
pub static APP_USER_AGENT: &str = concat!(
2023-01-18 07:18:18 +03:00
"llamanodes_",
2022-05-12 02:50:52 +03:00
env!("CARGO_PKG_NAME"),
2023-01-18 07:18:18 +03:00
"/v",
env!("CARGO_PKG_VERSION")
2022-05-12 02:50:52 +03:00
);
/// aggregate across 1 week
pub const BILLING_PERIOD_SECONDS: i64 = 60 * 60 * 24 * 7;
2022-11-20 01:05:51 +03:00
/// Convenience type
2023-05-24 00:40:34 +03:00
pub type Web3ProxyJoinHandle<T> = JoinHandle<Web3ProxyResult<T>>;
2022-06-14 08:43:28 +03:00
2022-08-24 03:59:05 +03:00
/// The application
// TODO: i'm sure this is more arcs than necessary, but spawning futures makes references hard
pub struct Web3ProxyApp {
/// Send requests to the best server available
pub balanced_rpcs: Arc<Web3Rpcs>,
2023-04-14 10:04:35 +03:00
/// Send 4337 Abstraction Bundler requests to one of these servers
pub bundler_4337_rpcs: Option<Arc<Web3Rpcs>>,
/// application config
/// TODO: this will need a large refactor to handle reloads while running. maybe use a watch::Receiver?
pub config: AppConfig,
pub http_client: Option<reqwest::Client>,
/// track JSONRPC responses
pub jsonrpc_response_cache: JsonRpcResponseCache,
/// rpc clients that subscribe to newHeads use this channel
/// don't drop this or the sender will stop working
/// TODO: broadcast channel instead?
pub watch_consensus_head_receiver: watch::Receiver<Option<Web3ProxyBlock>>,
2023-03-23 01:36:40 +03:00
pub hostname: Option<String>,
pub frontend_port: Arc<AtomicU16>,
/// rate limit anonymous users
2022-09-15 20:57:24 +03:00
pub frontend_ip_rate_limiter: Option<DeferredRateLimiter<IpAddr>>,
/// rate limit authenticated users
More balance tests (#182) * fix popularity contest * more info in the Debug for Web3Rpc * add frontend_requests and cache_misses to the Balance query * add more to balance and stats flushing and improved test coverage * it compiles * deserializer for Ulid to Uuid I think a wrapper type on Ulid that implements sea_orm::Value is probably better * rename variable to match struct name * add deserializer for Address -> Vec<u8> * sql sum returns a Decimal. need to convert to u64 * assert more * one log and assert more * log more * use a helper to get the user's rpc provider * this should be 2 now that we have a public and authed call * this should be zero. the public has the cache miss * instrument cu calcs * trace the value we took, not the default that replaced it * move usd_per_chain into config * remove some extra logging * use Arc::into_inner to maybe avoid a race * off by 1 * pass paid credits used instead of returning it this lets us use it to write to our user balance cache first. importantly, this keeps us from holding a write lock while writing to mysql * no cache misses expected in this test * actually check the admin * put the balance checks back now that the rest of the test works * archive request is being set incorrectly * wow howd we manage flipping the greater than sign on archive depth * move latest_balance and premium_credits_used to before any stats are emitted * lint * and build undoes the linting. fun i didnt even want to lint them in the first place, so this is fine * missed incrementing total_spent when not incrementing total_spent_paid_credits * use the credits on self * use the credits on self (pt 2) * fix type for 10 cu query * convert the requestmetadata on the other side of the channel * logs * viewing stats is allowed even without a balance * move paid_credits_used to AuthorizationChecks * wip * test_sum_credits_used finally passes * UserBalanceCache::get_or_insert * re-enable rpc_secret_key_cache * move invalidate to a helper function and always call it **after** the db is commited * fix PartialEq and Eq on RpcSecretKey * cargo upgrade
2023-07-12 10:35:07 +03:00
pub frontend_registered_user_rate_limiter:
Option<DeferredRateLimiter<RegisteredUserRateLimitKey>>,
/// concurrent/parallel request limits for anonymous users
pub ip_semaphores: Cache<IpAddr, Arc<Semaphore>>,
pub kafka_producer: Option<rdkafka::producer::FutureProducer>,
/// rate limit the login endpoint
/// we do this because each pending login is a row in the database
2022-09-24 06:59:21 +03:00
pub login_rate_limiter: Option<RedisRateLimiter>,
/// Send private requests (like eth_sendRawTransaction) to all these servers
/// TODO: include another type so that we can use private miner relays that do not use JSONRPC requests
pub private_rpcs: Option<Arc<Web3Rpcs>>,
pub prometheus_port: Arc<AtomicU16>,
/// cache authenticated users so that we don't have to query the database on the hot path
// TODO: should the key be our RpcSecretKey class instead of Ulid?
pub rpc_secret_key_cache: RpcSecretKeyCache,
/// cache user balances so we don't have to check downgrade logic every single time
pub user_balance_cache: UserBalanceCache,
/// concurrent/parallel RPC request limits for authenticated users
pub user_semaphores: Cache<(NonZeroU64, IpAddr), Arc<Semaphore>>,
/// volatile cache used for rate limits
/// TODO: i think i might just delete this entirely. instead use local-only concurrency limits.
pub vredis_pool: Option<RedisPool>,
/// channel for sending stats in a background task
pub stat_sender: Option<mpsc::UnboundedSender<AppStat>>,
/// Optional time series database for making pretty graphs that load quickly
influxdb_client: Option<influxdb2::Client>,
/// Simple way to connect ethers Contracsts to the proxy
/// TODO: make this more efficient
internal_provider: OnceCell<Arc<EthersHttpProvider>>,
2022-08-24 03:59:05 +03:00
}
2022-07-16 03:08:22 +03:00
/// flatten a JoinError into an anyhow error
2022-08-10 05:37:34 +03:00
/// Useful when joining multiple futures.
2023-05-24 00:40:34 +03:00
pub async fn flatten_handle<T>(handle: Web3ProxyJoinHandle<T>) -> Web3ProxyResult<T> {
2022-06-14 08:43:28 +03:00
match handle.await {
Ok(Ok(result)) => Ok(result),
Ok(Err(err)) => Err(err),
Err(err) => Err(err.into()),
}
}
/// return the first error, or Ok if everything worked
pub async fn flatten_handles<T>(
2023-05-24 00:40:34 +03:00
mut handles: FuturesUnordered<Web3ProxyJoinHandle<T>>,
) -> Web3ProxyResult<()> {
2022-06-16 20:51:49 +03:00
while let Some(x) = handles.next().await {
match x {
Err(e) => return Err(e.into()),
Ok(Err(e)) => return Err(e),
2022-07-16 03:08:22 +03:00
Ok(Ok(_)) => continue,
2022-06-16 20:51:49 +03:00
}
}
Ok(())
}
/// starting an app creates many tasks
2022-10-31 23:05:58 +03:00
pub struct Web3ProxyAppSpawn {
/// the app. probably clone this to use in other groups of handles
pub app: Arc<Web3ProxyApp>,
/// handles for the balanced and private rpcs
2023-05-24 00:40:34 +03:00
pub app_handles: FuturesUnordered<Web3ProxyJoinHandle<()>>,
2022-10-31 23:05:58 +03:00
/// these are important and must be allowed to finish
2023-05-24 00:40:34 +03:00
pub background_handles: FuturesUnordered<Web3ProxyJoinHandle<()>>,
2023-02-27 10:52:37 +03:00
/// config changes are sent here
2023-07-14 00:56:17 +03:00
pub new_top_config: Arc<watch::Sender<TopConfig>>,
/// watch this to know when the app is ready to serve requests
2023-06-27 22:36:41 +03:00
pub ranked_rpcs: watch::Receiver<Option<Arc<RankedRpcs>>>,
2022-10-31 23:05:58 +03:00
}
2022-05-12 02:50:52 +03:00
impl Web3ProxyApp {
2022-10-03 21:08:01 +03:00
/// The main entrypoint.
pub async fn spawn(
frontend_port: Arc<AtomicU16>,
prometheus_port: Arc<AtomicU16>,
2022-08-12 22:07:14 +03:00
top_config: TopConfig,
2022-09-14 09:18:13 +03:00
num_workers: usize,
shutdown_sender: broadcast::Sender<()>,
More balance tests (#182) * fix popularity contest * more info in the Debug for Web3Rpc * add frontend_requests and cache_misses to the Balance query * add more to balance and stats flushing and improved test coverage * it compiles * deserializer for Ulid to Uuid I think a wrapper type on Ulid that implements sea_orm::Value is probably better * rename variable to match struct name * add deserializer for Address -> Vec<u8> * sql sum returns a Decimal. need to convert to u64 * assert more * one log and assert more * log more * use a helper to get the user's rpc provider * this should be 2 now that we have a public and authed call * this should be zero. the public has the cache miss * instrument cu calcs * trace the value we took, not the default that replaced it * move usd_per_chain into config * remove some extra logging * use Arc::into_inner to maybe avoid a race * off by 1 * pass paid credits used instead of returning it this lets us use it to write to our user balance cache first. importantly, this keeps us from holding a write lock while writing to mysql * no cache misses expected in this test * actually check the admin * put the balance checks back now that the rest of the test works * archive request is being set incorrectly * wow howd we manage flipping the greater than sign on archive depth * move latest_balance and premium_credits_used to before any stats are emitted * lint * and build undoes the linting. fun i didnt even want to lint them in the first place, so this is fine * missed incrementing total_spent when not incrementing total_spent_paid_credits * use the credits on self * use the credits on self (pt 2) * fix type for 10 cu query * convert the requestmetadata on the other side of the channel * logs * viewing stats is allowed even without a balance * move paid_credits_used to AuthorizationChecks * wip * test_sum_credits_used finally passes * UserBalanceCache::get_or_insert * re-enable rpc_secret_key_cache * move invalidate to a helper function and always call it **after** the db is commited * fix PartialEq and Eq on RpcSecretKey * cargo upgrade
2023-07-12 10:35:07 +03:00
flush_stat_buffer_sender: mpsc::Sender<oneshot::Sender<FlushedStats>>,
flush_stat_buffer_receiver: mpsc::Receiver<oneshot::Sender<FlushedStats>>,
2022-10-31 23:05:58 +03:00
) -> anyhow::Result<Web3ProxyAppSpawn> {
2023-03-31 14:43:41 +03:00
let stat_buffer_shutdown_receiver = shutdown_sender.subscribe();
let mut config_watcher_shutdown_receiver = shutdown_sender.subscribe();
let mut background_shutdown_receiver = shutdown_sender.subscribe();
let (new_top_config_sender, mut new_top_config_receiver) =
watch::channel(top_config.clone());
new_top_config_receiver.borrow_and_update();
2023-07-21 08:34:32 +03:00
// TODO: take this from config
// TODO: how should we handle hitting this max?
let max_users = 20_000;
// safety checks on the config
// while i would prefer this to be in a "apply_top_config" function, that is a larger refactor
// TODO: maybe don't spawn with a config at all. have all config updates come through an apply_top_config call
if let Some(redirect) = &top_config.app.redirect_rpc_key_url {
assert!(
redirect.contains("{{rpc_key_id}}"),
"redirect_rpc_key_url user url must contain \"{{rpc_key_id}}\""
);
}
if !top_config.extra.is_empty() {
warn!(
2023-06-29 07:30:00 +03:00
extra=?top_config.extra.keys(),
"unknown TopConfig fields!",
);
}
if !top_config.app.extra.is_empty() {
warn!(
2023-06-29 07:30:00 +03:00
extra=?top_config.app.extra.keys(),
"unknown Web3ProxyAppConfig fields!",
);
}
// these futures are key parts of the app. if they stop running, the app has encountered an irrecoverable error
2023-05-24 00:40:34 +03:00
// TODO: this is a small enough group, that a vec with try_join_all is probably fine
let app_handles: FuturesUnordered<Web3ProxyJoinHandle<()>> = FuturesUnordered::new();
2023-02-25 20:48:40 +03:00
// we must wait for these to end on their own (and they need to subscribe to shutdown_sender)
2023-05-24 00:40:34 +03:00
let important_background_handles: FuturesUnordered<Web3ProxyJoinHandle<()>> =
FuturesUnordered::new();
2022-12-28 19:36:22 +03:00
2023-03-03 04:39:50 +03:00
// connect to kafka for logging requests from the /debug/ urls
let mut kafka_producer: Option<rdkafka::producer::FutureProducer> = None;
if let Some(kafka_brokers) = top_config.app.kafka_urls.clone() {
2023-04-20 01:30:01 +03:00
info!("Connecting to kafka");
2023-04-20 01:22:27 +03:00
let security_protocol = &top_config.app.kafka_protocol;
2023-04-19 20:42:57 +03:00
2023-03-03 04:39:50 +03:00
match rdkafka::ClientConfig::new()
.set("bootstrap.servers", kafka_brokers)
.set("message.timeout.ms", "5000")
2023-04-19 20:42:57 +03:00
.set("security.protocol", security_protocol)
2023-03-03 04:39:50 +03:00
.create()
{
Ok(k) => {
// TODO: create our topic
kafka_producer = Some(k)
}
2023-03-03 04:39:50 +03:00
Err(err) => error!("Failed connecting to kafka. This will not retry. {:?}", err),
}
}
2023-02-25 20:48:40 +03:00
// TODO: do this during apply_config so that we can change redis url while running
2022-10-03 21:08:01 +03:00
// create a connection pool for redis
// a failure to connect does NOT block the application from starting
let vredis_pool = match top_config.app.volatile_redis_url.as_ref() {
2022-07-26 07:53:38 +03:00
Some(redis_url) => {
2022-09-14 04:43:09 +03:00
// TODO: scrub credentials and then include the redis_url in logs
info!("Connecting to vredis");
2022-05-22 21:39:06 +03:00
2022-09-14 09:18:13 +03:00
// TODO: what is a good default?
2022-09-02 23:16:20 +03:00
let redis_max_connections = top_config
.app
.volatile_redis_max_connections
2022-09-14 09:18:13 +03:00
.unwrap_or(num_workers * 2);
// TODO: what are reasonable timeouts?
2022-09-14 09:18:13 +03:00
let redis_pool = RedisConfig::from_url(redis_url)
.builder()?
.max_size(redis_max_connections)
2022-09-15 20:57:24 +03:00
.runtime(DeadpoolRuntime::Tokio1)
2022-09-14 09:18:13 +03:00
.build()?;
// test the redis pool
2022-09-17 04:19:11 +03:00
if let Err(err) = redis_pool.get().await {
error!(
2022-11-12 11:24:32 +03:00
"failed to connect to vredis. some features will be disabled. err={:?}",
err
);
2022-09-17 04:19:11 +03:00
};
2022-09-14 09:18:13 +03:00
Some(redis_pool)
2022-05-22 21:39:06 +03:00
}
None => {
warn!("no redis connection. some features will be disabled");
2022-05-22 21:39:06 +03:00
None
2022-05-22 02:34:05 +03:00
}
};
2022-05-12 02:50:52 +03:00
let influxdb_client = match top_config.app.influxdb_host.as_ref() {
Some(influxdb_host) => {
let influxdb_org = top_config
.app
.influxdb_org
.clone()
.expect("influxdb_org needed when influxdb_host is set");
let influxdb_token = top_config
.app
.influxdb_token
.clone()
.expect("influxdb_token needed when influxdb_host is set");
top_config
.app
.influxdb_bucket
2023-06-13 20:00:08 +03:00
.as_ref()
.expect("influxdb_bucket needed when influxdb_host is set");
let influxdb_client =
influxdb2::Client::new(influxdb_host, influxdb_org, influxdb_token);
2022-10-10 07:15:07 +03:00
// TODO: test the client now. having a stat for "started" can be useful on graphs to mark deploys
Some(influxdb_client)
}
None => None,
};
User Balance + Referral Logic (#44) * will implement balance topup endpoint * will quickly fix other PR reviews * merging from master * will finish up godmoe * will finish up login * added logic to top up balance (first iteration) * should implement additional columns soon (currency, amount, tx-hash), as well as a new table for spend * updated migrations, will account for spend next * get back to this later * will merge PR from stats-v2 * stats v2 rebased all my commits and squashed them down to one * cargo upgrade * added migrtation for spend in accounting table. will run test-deposit next * trying to get request from polygon * first iteration /user/balance/:tx_hash works, needs to add accepted tokens next * creating the referral code seems to work * will now check if spending enough credits will lead to both parties receiving credits * rpcstats takes care of accounting for spend data * removed track spend from table * Revert "removed track spend from table" This reverts commit a50802d6ae75f786864c5ec42d0ceb2cb27124ed. * Revert "rpcstats takes care of accounting for spend data" This reverts commit 1cec728bf241e4cfd24351134637ed81c1a5a10b. * removed rpc request table entity * updated referral code to use ulid s * credits used are aggregated * added a bunch of fields to referrer * added database logic whenever an aggregate stats is added. will have to iterate over this a couple times i think. go to (1) detecting accepted stables next, (2) fix influxdb bug and (3) start to write test * removed track spend as this will occur in the database * will first work on "balance", then referral. these should really be treated as two separate PRs (although already convoluted) * balance logic initial commit * breaking WIP, changing the RPC call logic functions * will start testing next * got rid of warnings & lint * will proceed with subtracting / adding to balance * added decimal points, balance tracking seems to work * will beautify code a bit * removed deprecated dependency, and added topic + deposit contract to app.yaml * brownie test suite does not rely on local contract files it pulls all from polygonscan * will continue with referral * should perhaps (in a future revision) recordhow much the referees got for free. marking referrals seems to work rn * user is upgraded to premium if they deposit more than 10$. we dont accept more than $10M in a single tx * will start PR, referral seems to be fine so far, perhaps up to some numbers that still may need tweaking * will start PR * removed rogue comments, cleaned up payments a bit * changes before PR * apply stats * added unique constraint * some refactoring such that the user file is not too bloated * compiling * progress with subusers, creating a table entry seems to work * good response type is there as well now, will work on getters from primary user and secondary user next * subuser logic also seems fine now * downgrade logic * fixed bug influxdb does not support different types in same query (which makes sense) * WIP temporary commit * merging with PR * Delete daemon.rs there are multiple daemons now, so this was moved to `proxyd` * will remove request clone to &mut * multiple request handles for payment * making requests still seem fine * removed redundant commented out bits * added deposit endpoint, added deposit amount and deposit user, untested yet * small bug with downgrade tier id * will add authorization so balance can be received for users * balance history should be set now too * will check balance over time again * subususer can see rpc key balance if admin or owner * stats also seems to work fine now with historical balance * things seem to be building and working * removed clone from OpenRequestHandle * removed influxdb from workspace members * changed config files * reran sea-orm generate entities, added a foreign key, should be proper now * removed contract from commit * made deposit contract optional * added topic in polygon dev * changed deposit contract to deposit factory contract * added selfrelation on user_tier * added payment required * changed chain id to u64 * add wss in polygon llamarpc * removed origin and method from the table * added onchain transactions naming (and forgot to add a migration before) * changed foreign key to be the referrer (id), not the code itself * forgot to add id as the target foreign key * WIP adding cache to update role * fixed merge conflicts --------- Co-authored-by: Bryan Stitt <bryan@llamanodes.com> Co-authored-by: Bryan Stitt <bryan@stitthappens.com>
2023-05-12 19:45:15 +03:00
// all the users are the same size, so no need for a weigher
// if there is no database of users, there will be no keys and so this will be empty
// TODO: max_capacity from config
// TODO: ttl from config
2023-07-21 08:34:32 +03:00
let rpc_secret_key_cache = CacheBuilder::new(max_users)
2023-06-08 03:55:34 +03:00
.name("rpc_secret_key")
2023-06-08 03:26:38 +03:00
.time_to_live(Duration::from_secs(600))
.build();
User Balance + Referral Logic (#44) * will implement balance topup endpoint * will quickly fix other PR reviews * merging from master * will finish up godmoe * will finish up login * added logic to top up balance (first iteration) * should implement additional columns soon (currency, amount, tx-hash), as well as a new table for spend * updated migrations, will account for spend next * get back to this later * will merge PR from stats-v2 * stats v2 rebased all my commits and squashed them down to one * cargo upgrade * added migrtation for spend in accounting table. will run test-deposit next * trying to get request from polygon * first iteration /user/balance/:tx_hash works, needs to add accepted tokens next * creating the referral code seems to work * will now check if spending enough credits will lead to both parties receiving credits * rpcstats takes care of accounting for spend data * removed track spend from table * Revert "removed track spend from table" This reverts commit a50802d6ae75f786864c5ec42d0ceb2cb27124ed. * Revert "rpcstats takes care of accounting for spend data" This reverts commit 1cec728bf241e4cfd24351134637ed81c1a5a10b. * removed rpc request table entity * updated referral code to use ulid s * credits used are aggregated * added a bunch of fields to referrer * added database logic whenever an aggregate stats is added. will have to iterate over this a couple times i think. go to (1) detecting accepted stables next, (2) fix influxdb bug and (3) start to write test * removed track spend as this will occur in the database * will first work on "balance", then referral. these should really be treated as two separate PRs (although already convoluted) * balance logic initial commit * breaking WIP, changing the RPC call logic functions * will start testing next * got rid of warnings & lint * will proceed with subtracting / adding to balance * added decimal points, balance tracking seems to work * will beautify code a bit * removed deprecated dependency, and added topic + deposit contract to app.yaml * brownie test suite does not rely on local contract files it pulls all from polygonscan * will continue with referral * should perhaps (in a future revision) recordhow much the referees got for free. marking referrals seems to work rn * user is upgraded to premium if they deposit more than 10$. we dont accept more than $10M in a single tx * will start PR, referral seems to be fine so far, perhaps up to some numbers that still may need tweaking * will start PR * removed rogue comments, cleaned up payments a bit * changes before PR * apply stats * added unique constraint * some refactoring such that the user file is not too bloated * compiling * progress with subusers, creating a table entry seems to work * good response type is there as well now, will work on getters from primary user and secondary user next * subuser logic also seems fine now * downgrade logic * fixed bug influxdb does not support different types in same query (which makes sense) * WIP temporary commit * merging with PR * Delete daemon.rs there are multiple daemons now, so this was moved to `proxyd` * will remove request clone to &mut * multiple request handles for payment * making requests still seem fine * removed redundant commented out bits * added deposit endpoint, added deposit amount and deposit user, untested yet * small bug with downgrade tier id * will add authorization so balance can be received for users * balance history should be set now too * will check balance over time again * subususer can see rpc key balance if admin or owner * stats also seems to work fine now with historical balance * things seem to be building and working * removed clone from OpenRequestHandle * removed influxdb from workspace members * changed config files * reran sea-orm generate entities, added a foreign key, should be proper now * removed contract from commit * made deposit contract optional * added topic in polygon dev * changed deposit contract to deposit factory contract * added selfrelation on user_tier * added payment required * changed chain id to u64 * add wss in polygon llamarpc * removed origin and method from the table * added onchain transactions naming (and forgot to add a migration before) * changed foreign key to be the referrer (id), not the code itself * forgot to add id as the target foreign key * WIP adding cache to update role * fixed merge conflicts --------- Co-authored-by: Bryan Stitt <bryan@llamanodes.com> Co-authored-by: Bryan Stitt <bryan@stitthappens.com>
2023-05-12 19:45:15 +03:00
// TODO: TTL left low, this could also be a solution instead of modifiying the cache, that may be disgusting across threads / slow anyways
2023-07-21 08:34:32 +03:00
let user_balance_cache: UserBalanceCache = CacheBuilder::new(max_users)
2023-06-08 03:55:34 +03:00
.name("user_balance")
2023-06-08 03:26:38 +03:00
.time_to_live(Duration::from_secs(600))
More balance tests (#182) * fix popularity contest * more info in the Debug for Web3Rpc * add frontend_requests and cache_misses to the Balance query * add more to balance and stats flushing and improved test coverage * it compiles * deserializer for Ulid to Uuid I think a wrapper type on Ulid that implements sea_orm::Value is probably better * rename variable to match struct name * add deserializer for Address -> Vec<u8> * sql sum returns a Decimal. need to convert to u64 * assert more * one log and assert more * log more * use a helper to get the user's rpc provider * this should be 2 now that we have a public and authed call * this should be zero. the public has the cache miss * instrument cu calcs * trace the value we took, not the default that replaced it * move usd_per_chain into config * remove some extra logging * use Arc::into_inner to maybe avoid a race * off by 1 * pass paid credits used instead of returning it this lets us use it to write to our user balance cache first. importantly, this keeps us from holding a write lock while writing to mysql * no cache misses expected in this test * actually check the admin * put the balance checks back now that the rest of the test works * archive request is being set incorrectly * wow howd we manage flipping the greater than sign on archive depth * move latest_balance and premium_credits_used to before any stats are emitted * lint * and build undoes the linting. fun i didnt even want to lint them in the first place, so this is fine * missed incrementing total_spent when not incrementing total_spent_paid_credits * use the credits on self * use the credits on self (pt 2) * fix type for 10 cu query * convert the requestmetadata on the other side of the channel * logs * viewing stats is allowed even without a balance * move paid_credits_used to AuthorizationChecks * wip * test_sum_credits_used finally passes * UserBalanceCache::get_or_insert * re-enable rpc_secret_key_cache * move invalidate to a helper function and always call it **after** the db is commited * fix PartialEq and Eq on RpcSecretKey * cargo upgrade
2023-07-12 10:35:07 +03:00
.build()
.into();
// create a channel for receiving stats
// we do this in a channel so we don't slow down our response to the users
// stats can be saved in mysql, influxdb, both, or none
let stat_sender = if let Some(spawned_stat_buffer) = StatBuffer::try_spawn(
BILLING_PERIOD_SECONDS,
top_config.app.chain_id,
2023-07-22 02:12:38 +03:00
30,
top_config.app.influxdb_bucket.clone(),
influxdb_client.clone(),
rpc_secret_key_cache.clone(),
user_balance_cache.clone(),
stat_buffer_shutdown_receiver,
2023-07-22 02:39:27 +03:00
10,
flush_stat_buffer_sender.clone(),
flush_stat_buffer_receiver,
top_config.app.influxdb_id,
)? {
// since the database entries are used for accounting, we want to be sure everything is saved before exiting
important_background_handles.push(spawned_stat_buffer.background_handle);
Some(spawned_stat_buffer.stat_sender)
} else {
2023-04-11 02:05:53 +03:00
info!("stats will not be collected");
None
};
2022-10-03 21:08:01 +03:00
2023-02-25 20:48:40 +03:00
// make a http shared client
// TODO: can we configure the connection pool? should we?
// TODO: timeouts from config. defaults are hopefully good
let http_client = Some(
reqwest::ClientBuilder::new()
.connect_timeout(Duration::from_secs(5))
.timeout(Duration::from_secs(5 * 60))
.user_agent(APP_USER_AGENT)
.build()?,
);
// create rate limiters
// these are optional. they require redis
let mut frontend_ip_rate_limiter = None;
let mut frontend_registered_user_rate_limiter = None;
let mut login_rate_limiter = None;
if let Some(ref redis_pool) = vredis_pool {
if let Some(public_requests_per_period) = top_config.app.public_requests_per_period {
// chain id is included in the app name so that rpc rate limits are per-chain
let rpc_rrl = RedisRateLimiter::new(
&format!("web3_proxy:{}", top_config.app.chain_id),
"frontend",
public_requests_per_period,
60.0,
redis_pool.clone(),
);
// these two rate limiters can share the base limiter
// these are deferred rate limiters because we don't want redis network requests on the hot path
// TODO: take cache_size from config
More balance tests (#182) * fix popularity contest * more info in the Debug for Web3Rpc * add frontend_requests and cache_misses to the Balance query * add more to balance and stats flushing and improved test coverage * it compiles * deserializer for Ulid to Uuid I think a wrapper type on Ulid that implements sea_orm::Value is probably better * rename variable to match struct name * add deserializer for Address -> Vec<u8> * sql sum returns a Decimal. need to convert to u64 * assert more * one log and assert more * log more * use a helper to get the user's rpc provider * this should be 2 now that we have a public and authed call * this should be zero. the public has the cache miss * instrument cu calcs * trace the value we took, not the default that replaced it * move usd_per_chain into config * remove some extra logging * use Arc::into_inner to maybe avoid a race * off by 1 * pass paid credits used instead of returning it this lets us use it to write to our user balance cache first. importantly, this keeps us from holding a write lock while writing to mysql * no cache misses expected in this test * actually check the admin * put the balance checks back now that the rest of the test works * archive request is being set incorrectly * wow howd we manage flipping the greater than sign on archive depth * move latest_balance and premium_credits_used to before any stats are emitted * lint * and build undoes the linting. fun i didnt even want to lint them in the first place, so this is fine * missed incrementing total_spent when not incrementing total_spent_paid_credits * use the credits on self * use the credits on self (pt 2) * fix type for 10 cu query * convert the requestmetadata on the other side of the channel * logs * viewing stats is allowed even without a balance * move paid_credits_used to AuthorizationChecks * wip * test_sum_credits_used finally passes * UserBalanceCache::get_or_insert * re-enable rpc_secret_key_cache * move invalidate to a helper function and always call it **after** the db is commited * fix PartialEq and Eq on RpcSecretKey * cargo upgrade
2023-07-12 10:35:07 +03:00
frontend_ip_rate_limiter =
Some(DeferredRateLimiter::new(20_000, "ip", rpc_rrl.clone(), None).await);
2023-05-18 10:04:30 +03:00
frontend_registered_user_rate_limiter =
Some(DeferredRateLimiter::new(20_000, "key", rpc_rrl, None).await);
}
// login rate limiter
login_rate_limiter = Some(RedisRateLimiter::new(
"web3_proxy",
"login",
top_config.app.login_rate_limit_per_period,
60.0,
redis_pool.clone(),
));
}
2023-02-15 04:41:40 +03:00
let (watch_consensus_head_sender, watch_consensus_head_receiver) = watch::channel(None);
2022-06-16 20:51:49 +03:00
2023-02-25 20:48:40 +03:00
// responses can be very different in sizes, so this is a cache with a max capacity and a weigher
// TODO: we should emit stats to calculate a more accurate expected cache size
2023-05-15 20:48:59 +03:00
// TODO: do we actually want a TTL on this?
// TODO: configurable max item weight insted of hard coding to .1% of the cache?
let jsonrpc_weigher =
JsonRpcResponseWeigher((top_config.app.response_cache_max_bytes / 1000) as u32);
2023-06-13 21:51:19 +03:00
let jsonrpc_response_cache: JsonRpcResponseCache =
2023-06-08 03:26:38 +03:00
CacheBuilder::new(top_config.app.response_cache_max_bytes)
2023-06-13 21:51:19 +03:00
.name("jsonrpc_response_cache")
.time_to_idle(Duration::from_secs(3600))
.weigher(move |k, v| jsonrpc_weigher.weigh(k, v))
2023-06-08 03:26:38 +03:00
.build();
2023-02-25 20:48:40 +03:00
// create semaphores for concurrent connection limits
// TODO: time-to-idle on these. need to make sure the arcs aren't anywhere though. so maybe arc isn't correct and it should be refs
let ip_semaphores = CacheBuilder::new(max_users).name("ip_semaphores").build();
let user_semaphores = CacheBuilder::new(max_users).name("user_semaphores").build();
2023-02-25 20:48:40 +03:00
2023-06-17 20:11:48 +03:00
let chain_id = top_config.app.chain_id;
// TODO: remove this. it should only be done by apply_top_config
let (balanced_rpcs, balanced_handle, consensus_connections_watcher) = Web3Rpcs::spawn(
2023-06-17 20:11:48 +03:00
chain_id,
top_config.app.max_head_block_lag,
2022-08-27 03:33:45 +03:00
top_config.app.min_synced_rpcs,
top_config.app.min_sum_soft_limit,
2023-07-15 04:35:40 +03:00
"balanced rpcs".into(),
Some(watch_consensus_head_sender),
2022-05-22 02:34:05 +03:00
)
2022-07-26 07:53:38 +03:00
.await
.web3_context("spawning balanced rpcs")?;
2022-05-18 19:35:06 +03:00
app_handles.push(balanced_handle);
2022-06-14 08:43:28 +03:00
// prepare a Web3Rpcs to hold all our private connections
2022-10-03 21:08:01 +03:00
// only some chains have this, so this is optional
// TODO: remove this. it should only be done by apply_top_config
let private_rpcs = if top_config.private_rpcs.is_none() {
2022-05-12 02:50:52 +03:00
warn!("No private relays configured. Any transactions will be broadcast to the public mempool!");
None
2022-05-12 02:50:52 +03:00
} else {
2023-02-25 20:48:40 +03:00
// TODO: do something with the spawn handle
let (private_rpcs, private_handle, _) = Web3Rpcs::spawn(
2023-06-17 20:11:48 +03:00
chain_id,
// private rpcs don't get subscriptions, so no need for max_head_block_lag
None,
0,
0,
2023-07-15 04:35:40 +03:00
"protected rpcs".into(),
2022-08-11 00:29:50 +03:00
// subscribing to new heads here won't work well. if they are fast, they might be ahead of balanced_rpcs
2023-01-23 09:02:08 +03:00
// they also often have low rate limits
// however, they are well connected to miners/validators. so maybe using them as a safety check would be good
// TODO: but maybe we could include privates in the "backup" tier
2022-08-11 00:29:50 +03:00
None,
2022-05-22 02:34:05 +03:00
)
2022-07-26 07:53:38 +03:00
.await
.web3_context("spawning private_rpcs")?;
2022-06-14 08:43:28 +03:00
app_handles.push(private_handle);
2023-02-25 20:48:40 +03:00
Some(private_rpcs)
2022-05-12 02:50:52 +03:00
};
2023-04-14 10:04:35 +03:00
// prepare a Web3Rpcs to hold all our 4337 Abstraction Bundler connections
// only some chains have this, so this is optional
// TODO: remove this. it should only be done by apply_top_config
2023-04-14 10:04:35 +03:00
let bundler_4337_rpcs = if top_config.bundler_4337_rpcs.is_none() {
warn!("No bundler_4337_rpcs configured");
None
} else {
// TODO: do something with the spawn handle
2023-04-19 06:14:12 +03:00
let (bundler_4337_rpcs, bundler_4337_rpcs_handle, _) = Web3Rpcs::spawn(
2023-06-17 20:11:48 +03:00
chain_id,
// bundler_4337_rpcs don't get subscriptions, so no need for max_head_block_lag
2023-04-14 10:04:35 +03:00
None,
0,
0,
2023-07-15 04:35:40 +03:00
"eip4337 rpcs".into(),
2023-04-14 10:04:35 +03:00
None,
)
.await
.web3_context("spawning bundler_4337_rpcs")?;
2023-04-14 10:04:35 +03:00
app_handles.push(bundler_4337_rpcs_handle);
Some(bundler_4337_rpcs)
};
2023-03-30 15:42:56 +03:00
let hostname = hostname::get()
.ok()
.and_then(|x| x.to_str().map(|x| x.to_string()));
2023-03-23 01:18:54 +03:00
2023-02-25 20:48:40 +03:00
let app = Self {
balanced_rpcs,
2023-04-14 10:04:35 +03:00
bundler_4337_rpcs,
2023-05-31 08:31:35 +03:00
config: top_config.app.clone(),
frontend_port: frontend_port.clone(),
2023-05-31 08:31:35 +03:00
frontend_ip_rate_limiter,
frontend_registered_user_rate_limiter,
2023-03-23 01:18:54 +03:00
hostname,
2023-05-31 08:31:35 +03:00
http_client,
influxdb_client,
internal_provider: Default::default(),
2023-02-25 20:48:40 +03:00
ip_semaphores,
2023-06-13 21:51:19 +03:00
jsonrpc_response_cache,
2023-05-31 08:31:35 +03:00
kafka_producer,
login_rate_limiter,
private_rpcs,
prometheus_port: prometheus_port.clone(),
rpc_secret_key_cache,
2023-02-25 20:48:40 +03:00
stat_sender,
user_balance_cache,
2023-05-31 08:31:35 +03:00
user_semaphores,
vredis_pool,
2023-05-31 08:31:35 +03:00
watch_consensus_head_receiver,
2023-02-25 20:48:40 +03:00
};
// TODO: do apply_top_config once we don't duplicate the db
if let Err(err) = app.apply_top_config_db(&top_config).await {
warn!(?err, "unable to fully apply config while starting!");
};
2023-02-25 20:48:40 +03:00
let app = Arc::new(app);
// watch for config changes
// TODO: move this to its own function/struct
2023-02-27 10:52:37 +03:00
{
let app = app.clone();
let config_handle = tokio::spawn(async move {
loop {
2023-02-27 10:52:37 +03:00
let new_top_config = new_top_config_receiver.borrow_and_update().to_owned();
// TODO: compare new and old here? the sender should be doing that already but maybe its better here
if let Err(err) = app.apply_top_config_rpcs(&new_top_config).await {
error!(?err, "unable to apply config! Retrying in 10 seconds (or if the config changes)");
select! {
_ = config_watcher_shutdown_receiver.recv() => {
break;
}
_ = sleep(Duration::from_secs(10)) => {}
_ = new_top_config_receiver.changed() => {}
}
} else {
select! {
_ = config_watcher_shutdown_receiver.recv() => {
break;
}
_ = new_top_config_receiver.changed() => {}
}
}
}
Ok(())
});
important_background_handles.push(config_handle);
}
2023-04-04 15:40:22 +03:00
if important_background_handles.is_empty() {
trace!("no important background handles");
2023-04-04 15:40:22 +03:00
let f = tokio::spawn(async move {
let _ = background_shutdown_receiver.recv().await;
Ok(())
});
important_background_handles.push(f);
}
2023-02-25 20:48:40 +03:00
Ok(Web3ProxyAppSpawn {
2023-02-27 10:52:37 +03:00
app,
app_handles,
background_handles: important_background_handles,
2023-07-14 00:56:17 +03:00
new_top_config: Arc::new(new_top_config_sender),
ranked_rpcs: consensus_connections_watcher,
})
2023-02-25 20:48:40 +03:00
}
pub async fn apply_top_config(&self, new_top_config: &TopConfig) -> Web3ProxyResult<()> {
// TODO: update self.config from new_top_config.app (or move it entirely to a global)
// connect to the db first
let db = self.apply_top_config_db(new_top_config).await;
// then refresh rpcs
let rpcs = self.apply_top_config_rpcs(new_top_config).await;
// error if either failed
// TODO: if both error, log both errors
db?;
rpcs?;
Ok(())
}
async fn apply_top_config_rpcs(&self, new_top_config: &TopConfig) -> Web3ProxyResult<()> {
2023-07-11 04:09:58 +03:00
info!("applying new config");
2023-02-25 20:48:40 +03:00
let balanced = self
.balanced_rpcs
.apply_server_configs(self, new_top_config.balanced_rpcs.clone())
2023-04-14 10:38:49 +03:00
.await
.web3_context("updating balanced rpcs");
2023-02-25 20:48:40 +03:00
let private = if let Some(private_rpc_configs) = new_top_config.private_rpcs.clone() {
if let Some(ref private_rpcs) = self.private_rpcs {
private_rpcs
.apply_server_configs(self, private_rpc_configs)
2023-04-14 10:38:49 +03:00
.await
.web3_context("updating private_rpcs")
} else {
// TODO: maybe we should have private_rpcs just be empty instead of being None
todo!("handle toggling private_rpcs")
2023-02-22 08:10:23 +03:00
}
} else {
Ok(())
};
2022-07-07 06:22:09 +03:00
let bundler_4337 =
if let Some(bundler_4337_rpc_configs) = new_top_config.bundler_4337_rpcs.clone() {
if let Some(ref bundler_4337_rpcs) = self.bundler_4337_rpcs {
bundler_4337_rpcs
.apply_server_configs(self, bundler_4337_rpc_configs.clone())
.await
.web3_context("updating bundler_4337_rpcs")
} else {
// TODO: maybe we should have bundler_4337_rpcs just be empty instead of being None
todo!("handle toggling bundler_4337_rpcs")
}
} else {
Ok(())
};
// TODO: log all the errors if there are multiple
balanced?;
private?;
bundler_4337?;
Ok(())
}
pub async fn apply_top_config_db(&self, new_top_config: &TopConfig) -> Web3ProxyResult<()> {
// TODO: get the actual value
let num_workers = 2;
// connect to the db
// THIS DOES NOT RUN MIGRATIONS!
if let Some(db_url) = new_top_config.app.db_url.clone() {
let db_min_connections = new_top_config
.app
.db_min_connections
.unwrap_or(num_workers as u32);
// TODO: what default multiple?
let db_max_connections = new_top_config
.app
.db_max_connections
.unwrap_or(db_min_connections * 2);
let db_conn = if let Ok(old_db_conn) = global_db_conn().await {
// TODO: compare old settings with new settings. don't always re-use!
Ok(old_db_conn)
} else {
connect_db(db_url.clone(), db_min_connections, db_max_connections)
2023-04-14 10:38:49 +03:00
.await
.map_err(|err| DatabaseError::Connect(Arc::new(err)))
};
let db_replica = if let Ok(db_conn) = db_conn.as_ref() {
let db_replica =
if let Some(db_replica_url) = new_top_config.app.db_replica_url.clone() {
if db_replica_url == db_url {
// url is the same. do not make a new connection or we might go past our max connections
Ok(db_conn.clone().into())
} else {
let db_replica_min_connections = new_top_config
.app
.db_replica_min_connections
.unwrap_or(db_min_connections);
let db_replica_max_connections = new_top_config
.app
.db_replica_max_connections
.unwrap_or(db_max_connections);
let db_replica = if let Ok(old_db_replica) = global_db_conn().await {
// TODO: compare old settings with new settings. don't always re-use!
Ok(old_db_replica)
} else {
connect_db(
db_replica_url,
db_replica_min_connections,
db_replica_max_connections,
)
.await
.map_err(|err| DatabaseError::Connect(Arc::new(err)))
};
// if db_replica is error, but db_conn is success. log error and just use db_conn
if let Err(err) = db_replica.as_ref() {
error!(?err, "db replica is down! using primary");
Ok(db_conn.clone().into())
} else {
db_replica.map(Into::into)
}
}
} else {
// just clone so that we don't need a bunch of checks all over our code
trace!("no db replica configured");
Ok(db_conn.clone().into())
};
// db and replica are connected. try to migrate
if let Err(err) = migrate_db(db_conn, false).await {
error!(?err, "unable to migrate!");
}
db_replica
2023-04-14 10:04:35 +03:00
} else {
db_conn.clone().map(Into::into)
};
let mut locked_conn = DB_CONN.write().await;
let mut locked_replica = DB_REPLICA.write().await;
*locked_conn = db_conn.clone();
*locked_replica = db_replica.clone();
2023-04-14 10:04:35 +03:00
db_conn?;
db_replica?;
info!("set global db connections");
} else if new_top_config.app.db_replica_url.is_some() {
return Err(anyhow::anyhow!("db_replica_url set, but no db_url set!").into());
} else {
warn!("no database. some features will be disabled");
};
2023-07-11 04:09:58 +03:00
2023-02-25 20:48:40 +03:00
Ok(())
2022-05-12 02:50:52 +03:00
}
2023-02-15 04:41:40 +03:00
pub fn head_block_receiver(&self) -> watch::Receiver<Option<Web3ProxyBlock>> {
2023-01-23 09:02:08 +03:00
self.watch_consensus_head_receiver.clone()
}
pub fn influxdb_client(&self) -> Web3ProxyResult<&influxdb2::Client> {
More balance tests (#182) * fix popularity contest * more info in the Debug for Web3Rpc * add frontend_requests and cache_misses to the Balance query * add more to balance and stats flushing and improved test coverage * it compiles * deserializer for Ulid to Uuid I think a wrapper type on Ulid that implements sea_orm::Value is probably better * rename variable to match struct name * add deserializer for Address -> Vec<u8> * sql sum returns a Decimal. need to convert to u64 * assert more * one log and assert more * log more * use a helper to get the user's rpc provider * this should be 2 now that we have a public and authed call * this should be zero. the public has the cache miss * instrument cu calcs * trace the value we took, not the default that replaced it * move usd_per_chain into config * remove some extra logging * use Arc::into_inner to maybe avoid a race * off by 1 * pass paid credits used instead of returning it this lets us use it to write to our user balance cache first. importantly, this keeps us from holding a write lock while writing to mysql * no cache misses expected in this test * actually check the admin * put the balance checks back now that the rest of the test works * archive request is being set incorrectly * wow howd we manage flipping the greater than sign on archive depth * move latest_balance and premium_credits_used to before any stats are emitted * lint * and build undoes the linting. fun i didnt even want to lint them in the first place, so this is fine * missed incrementing total_spent when not incrementing total_spent_paid_credits * use the credits on self * use the credits on self (pt 2) * fix type for 10 cu query * convert the requestmetadata on the other side of the channel * logs * viewing stats is allowed even without a balance * move paid_credits_used to AuthorizationChecks * wip * test_sum_credits_used finally passes * UserBalanceCache::get_or_insert * re-enable rpc_secret_key_cache * move invalidate to a helper function and always call it **after** the db is commited * fix PartialEq and Eq on RpcSecretKey * cargo upgrade
2023-07-12 10:35:07 +03:00
self.influxdb_client
.as_ref()
.ok_or(Web3ProxyError::NoDatabaseConfigured)
}
2023-05-31 09:17:05 +03:00
/// an ethers provider that you can use with ether's abigen.
/// this works for now, but I don't like it
/// TODO: I would much prefer we figure out the traits and `impl JsonRpcClient for Web3ProxyApp`
pub fn internal_provider(&self) -> &Arc<EthersHttpProvider> {
self.internal_provider.get_or_init(|| {
// TODO: i'm sure theres much better ways to do this, but i don't want to spend time fighting traits right now
// TODO: what interval? i don't think we use it
// i tried and failed to `impl JsonRpcClient for Web3ProxyApi`
// i tried and failed to set up ipc. http is already running, so lets just use that
let frontend_port = self.frontend_port.load(Ordering::Relaxed);
if frontend_port == 0 {
panic!("frontend is not running. cannot create provider yet");
}
let internal_provider = connect_http(
format!("http://127.0.0.1:{}", frontend_port)
.parse()
.unwrap(),
self.http_client.clone(),
Duration::from_secs(10),
)
.unwrap();
Arc::new(internal_provider)
})
2023-05-31 09:17:05 +03:00
}
2022-12-28 09:11:18 +03:00
pub async fn prometheus_metrics(&self) -> String {
2022-09-09 00:01:36 +03:00
let globals = HashMap::new();
// TODO: what globals? should this be the hostname or what?
// globals.insert("service", "web3_proxy");
// TODO: this needs a refactor to get HELP and TYPE into the serialized text
2023-01-03 04:06:36 +03:00
#[derive(Default, Serialize)]
struct UserCount(i64);
let user_count: UserCount = if let Ok(db) = global_db_conn().await {
match user::Entity::find().count(&db).await {
2023-01-03 04:06:36 +03:00
Ok(user_count) => UserCount(user_count as i64),
Err(err) => {
2023-06-29 07:30:00 +03:00
warn!(?err, "unable to count users");
2023-01-03 04:06:36 +03:00
UserCount(-1)
}
}
} else {
UserCount(-1)
};
2022-12-29 09:21:09 +03:00
#[derive(Default, Serialize)]
struct RecentCounts {
2023-01-03 03:34:32 +03:00
one_week: i64,
one_day: i64,
one_hour: i64,
one_minute: i64,
}
2022-12-28 09:11:18 +03:00
impl RecentCounts {
fn for_err() -> Self {
Self {
2023-01-03 03:34:32 +03:00
one_week: -1,
one_day: -1,
one_hour: -1,
one_minute: -1,
}
2022-12-28 09:11:18 +03:00
}
}
2022-12-28 09:11:18 +03:00
2022-12-29 09:21:09 +03:00
let (recent_ip_counts, recent_user_id_counts, recent_tx_counts): (
RecentCounts,
RecentCounts,
RecentCounts,
) = match self.redis_conn().await {
Ok(mut redis_conn) => {
2022-12-29 09:21:09 +03:00
// TODO: delete any hash entries where
const ONE_MINUTE: i64 = 60;
const ONE_HOUR: i64 = ONE_MINUTE * 60;
const ONE_DAY: i64 = ONE_HOUR * 24;
2023-01-03 03:34:32 +03:00
const ONE_WEEK: i64 = ONE_DAY * 7;
2022-12-29 09:21:09 +03:00
2023-01-03 03:34:32 +03:00
let one_week_ago = Utc::now().timestamp() - ONE_WEEK;
2022-12-29 09:21:09 +03:00
let one_day_ago = Utc::now().timestamp() - ONE_DAY;
let one_hour_ago = Utc::now().timestamp() - ONE_HOUR;
let one_minute_ago = Utc::now().timestamp() - ONE_MINUTE;
let recent_users_by_id = format!("recent_users:id:{}", self.config.chain_id);
let recent_users_by_ip = format!("recent_users:ip:{}", self.config.chain_id);
let recent_transactions =
format!("eth_sendRawTransaction:{}", self.config.chain_id);
match redis::pipe()
.atomic()
2023-01-03 03:34:32 +03:00
// delete any entries older than 1 week
.zrembyscore(&recent_users_by_id, i64::MIN, one_week_ago)
2022-12-29 09:21:09 +03:00
.ignore()
2023-01-03 03:34:32 +03:00
.zrembyscore(&recent_users_by_ip, i64::MIN, one_week_ago)
2022-12-29 09:21:09 +03:00
.ignore()
2023-01-03 03:34:32 +03:00
.zrembyscore(&recent_transactions, i64::MIN, one_week_ago)
2022-12-29 09:21:09 +03:00
.ignore()
2023-01-03 03:34:32 +03:00
// get counts for last week
.zcount(&recent_users_by_id, one_week_ago, i64::MAX)
.zcount(&recent_users_by_ip, one_week_ago, i64::MAX)
.zcount(&recent_transactions, one_week_ago, i64::MAX)
2022-12-29 09:21:09 +03:00
// get counts for last day
.zcount(&recent_users_by_id, one_day_ago, i64::MAX)
.zcount(&recent_users_by_ip, one_day_ago, i64::MAX)
.zcount(&recent_transactions, one_day_ago, i64::MAX)
// get counts for last hour
.zcount(&recent_users_by_id, one_hour_ago, i64::MAX)
.zcount(&recent_users_by_ip, one_hour_ago, i64::MAX)
.zcount(&recent_transactions, one_hour_ago, i64::MAX)
// get counts for last minute
.zcount(&recent_users_by_id, one_minute_ago, i64::MAX)
.zcount(&recent_users_by_ip, one_minute_ago, i64::MAX)
.zcount(&recent_transactions, one_minute_ago, i64::MAX)
.query_async(&mut redis_conn)
.await
{
Ok((
2023-01-03 03:34:32 +03:00
user_id_in_week,
ip_in_week,
txs_in_week,
2022-12-29 09:21:09 +03:00
user_id_in_day,
ip_in_day,
txs_in_day,
user_id_in_hour,
ip_in_hour,
txs_in_hour,
user_id_in_minute,
ip_in_minute,
txs_in_minute,
)) => {
let recent_user_id_counts = RecentCounts {
2023-01-03 03:34:32 +03:00
one_week: user_id_in_week,
2022-12-29 09:21:09 +03:00
one_day: user_id_in_day,
one_hour: user_id_in_hour,
one_minute: user_id_in_minute,
};
let recent_ip_counts = RecentCounts {
2023-01-03 03:34:32 +03:00
one_week: ip_in_week,
2022-12-29 09:21:09 +03:00
one_day: ip_in_day,
one_hour: ip_in_hour,
one_minute: ip_in_minute,
};
let recent_tx_counts = RecentCounts {
2023-01-03 03:34:32 +03:00
one_week: txs_in_week,
2022-12-29 09:21:09 +03:00
one_day: txs_in_day,
one_hour: txs_in_hour,
one_minute: txs_in_minute,
};
(recent_ip_counts, recent_user_id_counts, recent_tx_counts)
}
Err(err) => {
2023-06-29 07:30:00 +03:00
warn!(?err, "unable to count recent users");
2022-12-29 09:21:09 +03:00
(
RecentCounts::for_err(),
RecentCounts::for_err(),
RecentCounts::for_err(),
)
}
}
2022-12-29 09:21:09 +03:00
}
Err(err) => {
2023-06-29 07:30:00 +03:00
warn!(?err, "unable to connect to redis while counting users");
2022-12-29 09:21:09 +03:00
(
RecentCounts::for_err(),
RecentCounts::for_err(),
RecentCounts::for_err(),
)
}
};
2022-12-28 09:11:18 +03:00
2022-09-09 06:53:16 +03:00
#[derive(Serialize)]
struct CombinedMetrics {
recent_ip_counts: RecentCounts,
2022-12-29 09:21:09 +03:00
recent_user_id_counts: RecentCounts,
recent_tx_counts: RecentCounts,
2023-01-03 04:06:36 +03:00
user_count: UserCount,
2022-09-09 06:53:16 +03:00
}
let metrics = CombinedMetrics {
recent_ip_counts,
2022-12-29 09:21:09 +03:00
recent_user_id_counts,
recent_tx_counts,
2023-01-03 04:06:36 +03:00
user_count,
2022-09-09 06:53:16 +03:00
};
// TODO: i don't like this library. it doesn't include HELP or TYPE lines and so our prometheus server fails to parse it
2022-09-10 05:59:07 +03:00
serde_prometheus::to_string(&metrics, Some("web3_proxy"), globals)
.expect("prometheus metrics should always serialize")
}
/// make an internal request with stats and caching
pub async fn internal_request<P: JsonRpcParams, R: JsonRpcResultData>(
self: &Arc<Self>,
method: &str,
params: P,
) -> Web3ProxyResult<R> {
let authorization = Arc::new(Authorization::internal()?);
self.authorized_request(method, params, authorization).await
}
/// this is way more round-a-bout than we want, but it means stats are emitted and caches are used
pub async fn authorized_request<P: JsonRpcParams, R: JsonRpcResultData>(
self: &Arc<Self>,
method: &str,
params: P,
authorization: Arc<Authorization>,
) -> Web3ProxyResult<R> {
// TODO: proper ids
let request = JsonRpcRequest::new(JsonRpcId::Number(1), method.to_string(), json!(params))?;
let (_, response, _) = self.proxy_request(request, authorization, None).await;
if let Some(result) = response.result {
let result = serde_json::from_str(result.get())?;
Ok(result)
} else if let Some(error_data) = response.error {
// TODO: this might lose the http error code
Err(Web3ProxyError::JsonRpcErrorData(error_data))
} else {
unimplemented!();
}
}
2022-07-22 22:30:39 +03:00
/// send the request or batch of requests to the approriate RPCs
2022-05-12 02:50:52 +03:00
pub async fn proxy_web3_rpc(
2022-09-09 00:01:36 +03:00
self: &Arc<Self>,
authorization: Arc<Authorization>,
2022-05-12 02:50:52 +03:00
request: JsonRpcRequestEnum,
) -> Web3ProxyResult<(StatusCode, JsonRpcForwardedResponseEnum, Vec<Arc<Web3Rpc>>)> {
2023-01-17 09:54:40 +03:00
// trace!(?request, "proxy_web3_rpc");
2022-05-12 02:50:52 +03:00
let response = match request {
JsonRpcRequestEnum::Single(request) => {
2023-05-24 01:50:29 +03:00
let (status_code, response, rpcs) = self
.proxy_request(request, authorization.clone(), None)
2023-05-24 01:50:29 +03:00
.await;
(
status_code,
JsonRpcForwardedResponseEnum::Single(response),
rpcs,
)
}
JsonRpcRequestEnum::Batch(requests) => {
2023-05-24 01:50:29 +03:00
let (responses, rpcs) = self
.proxy_web3_rpc_requests(&authorization, requests)
.await?;
2022-05-12 02:50:52 +03:00
2023-06-01 02:05:41 +03:00
// TODO: real status code. if an error happens, i don't think we are following the spec here
(
StatusCode::OK,
JsonRpcForwardedResponseEnum::Batch(responses),
rpcs,
)
}
};
2022-05-17 03:56:56 +03:00
2022-05-20 08:27:18 +03:00
Ok(response)
2022-05-12 02:50:52 +03:00
}
2022-09-09 00:01:36 +03:00
/// cut up the request and send to potentually different servers
/// TODO: make sure this isn't a problem
2022-05-12 02:50:52 +03:00
async fn proxy_web3_rpc_requests(
2022-09-09 00:01:36 +03:00
self: &Arc<Self>,
authorization: &Arc<Authorization>,
requests: Vec<JsonRpcRequest>,
) -> Web3ProxyResult<(Vec<JsonRpcForwardedResponse>, Vec<Arc<Web3Rpc>>)> {
// TODO: we should probably change ethers-rs to support this directly. they pushed this off to v2 though
2022-05-12 02:50:52 +03:00
let num_requests = requests.len();
2022-10-10 07:15:07 +03:00
2023-06-13 07:54:33 +03:00
if num_requests == 0 {
return Ok((vec![], vec![]));
}
2023-02-15 04:41:40 +03:00
// get the head block now so that any requests that need it all use the same block
// TODO: this still has an edge condition if there is a reorg in the middle of the request!!!
let head_block: Web3ProxyBlock = self
2023-02-15 04:41:40 +03:00
.balanced_rpcs
.head_block()
.ok_or(Web3ProxyError::NoServersSynced)?
.clone();
2023-02-15 04:41:40 +03:00
// TODO: use streams and buffers so we don't overwhelm our server
2022-05-12 02:50:52 +03:00
let responses = join_all(
requests
.into_iter()
2023-02-15 04:41:40 +03:00
.map(|request| {
self.proxy_request(request, authorization.clone(), Some(&head_block))
2023-02-15 04:41:40 +03:00
})
2022-05-12 02:50:52 +03:00
.collect::<Vec<_>>(),
)
.await;
let mut collected: Vec<JsonRpcForwardedResponse> = Vec::with_capacity(num_requests);
let mut collected_rpc_names: HashSet<String> = HashSet::new();
let mut collected_rpcs: Vec<Arc<Web3Rpc>> = vec![];
2022-05-12 02:50:52 +03:00
for response in responses {
// TODO: any way to attach the tried rpcs to the error? it is likely helpful
let (_status_code, response, rpcs) = response;
collected.push(response);
collected_rpcs.extend(rpcs.into_iter().filter(|x| {
if collected_rpc_names.contains(&x.name) {
false
} else {
collected_rpc_names.insert(x.name.clone());
true
}
}));
// TODO: what should we do with the status code? check the jsonrpc spec
2022-05-12 02:50:52 +03:00
}
Ok((collected, collected_rpcs))
2022-05-12 02:50:52 +03:00
}
pub async fn redis_conn(&self) -> Web3ProxyResult<redis_rate_limiter::RedisConnection> {
match self.vredis_pool.as_ref() {
None => Err(Web3ProxyError::NoDatabaseConfigured),
Some(redis_pool) => {
// TODO: add a From for this
let redis_conn = redis_pool.get().await.context("redis pool error")?;
Ok(redis_conn)
}
}
}
/// try to send transactions to the best available rpcs with protected/private mempools
/// if no protected rpcs are configured, then some public rpcs are used instead
async fn try_send_protected<P: JsonRpcParams>(
self: &Arc<Self>,
method: &str,
params: &P,
request_metadata: &Arc<RequestMetadata>,
) -> Web3ProxyResult<Box<RawValue>> {
if let Some(protected_rpcs) = self.private_rpcs.as_ref() {
if !protected_rpcs.is_empty() {
let protected_response = protected_rpcs
.try_send_all_synced_connections(
method,
params,
Some(request_metadata),
None,
None,
Some(Duration::from_secs(30)),
Some(Level::TRACE.into()),
None,
)
.await;
return protected_response;
}
}
let num_public_rpcs = match request_metadata.proxy_mode() {
// TODO: how many balanced rpcs should we send to? configurable? percentage of total?
ProxyMode::Best | ProxyMode::Debug => Some(4),
ProxyMode::Fastest(0) => None,
// TODO: how many balanced rpcs should we send to? configurable? percentage of total?
// TODO: what if we do 2 per tier? we want to blast the third party rpcs
// TODO: maybe having the third party rpcs in their own Web3Rpcs would be good for this
ProxyMode::Fastest(x) => Some(x * 4),
ProxyMode::Versus => None,
};
// no private rpcs to send to. send to a few public rpcs
// try_send_all_upstream_servers puts the request id into the response. no need to do that ourselves here.
self.balanced_rpcs
.try_send_all_synced_connections(
method,
params,
Some(request_metadata),
None,
None,
Some(Duration::from_secs(30)),
Some(Level::TRACE.into()),
num_public_rpcs,
)
.await
}
2023-06-27 07:08:32 +03:00
/// proxy request with up to 3 tries.
async fn proxy_request(
2022-09-09 00:01:36 +03:00
self: &Arc<Self>,
2023-06-27 07:38:41 +03:00
mut request: JsonRpcRequest,
authorization: Arc<Authorization>,
head_block: Option<&Web3ProxyBlock>,
) -> (StatusCode, JsonRpcForwardedResponse, Vec<Arc<Web3Rpc>>) {
let request_metadata = RequestMetadata::new(
self,
authorization,
RequestOrMethod::Request(&request),
head_block,
)
.await;
2023-03-03 17:58:45 +03:00
let response_id = request.id;
2023-07-08 00:06:44 +03:00
// TODO: trace/kafka log request.params before we send them to _proxy_request_with_caching which might modify them
2023-06-29 01:04:55 +03:00
2023-07-08 00:06:44 +03:00
let (code, response_data) = match self
._proxy_request_with_caching(
&request.method,
&mut request.params,
head_block,
Some(2),
&request_metadata,
)
.await
{
Ok(response_data) => {
More balance tests (#182) * fix popularity contest * more info in the Debug for Web3Rpc * add frontend_requests and cache_misses to the Balance query * add more to balance and stats flushing and improved test coverage * it compiles * deserializer for Ulid to Uuid I think a wrapper type on Ulid that implements sea_orm::Value is probably better * rename variable to match struct name * add deserializer for Address -> Vec<u8> * sql sum returns a Decimal. need to convert to u64 * assert more * one log and assert more * log more * use a helper to get the user's rpc provider * this should be 2 now that we have a public and authed call * this should be zero. the public has the cache miss * instrument cu calcs * trace the value we took, not the default that replaced it * move usd_per_chain into config * remove some extra logging * use Arc::into_inner to maybe avoid a race * off by 1 * pass paid credits used instead of returning it this lets us use it to write to our user balance cache first. importantly, this keeps us from holding a write lock while writing to mysql * no cache misses expected in this test * actually check the admin * put the balance checks back now that the rest of the test works * archive request is being set incorrectly * wow howd we manage flipping the greater than sign on archive depth * move latest_balance and premium_credits_used to before any stats are emitted * lint * and build undoes the linting. fun i didnt even want to lint them in the first place, so this is fine * missed incrementing total_spent when not incrementing total_spent_paid_credits * use the credits on self * use the credits on self (pt 2) * fix type for 10 cu query * convert the requestmetadata on the other side of the channel * logs * viewing stats is allowed even without a balance * move paid_credits_used to AuthorizationChecks * wip * test_sum_credits_used finally passes * UserBalanceCache::get_or_insert * re-enable rpc_secret_key_cache * move invalidate to a helper function and always call it **after** the db is commited * fix PartialEq and Eq on RpcSecretKey * cargo upgrade
2023-07-12 10:35:07 +03:00
request_metadata
.error_response
.store(false, Ordering::Release);
2023-06-27 07:38:41 +03:00
2023-07-08 00:06:44 +03:00
(StatusCode::OK, response_data)
More balance tests (#182) * fix popularity contest * more info in the Debug for Web3Rpc * add frontend_requests and cache_misses to the Balance query * add more to balance and stats flushing and improved test coverage * it compiles * deserializer for Ulid to Uuid I think a wrapper type on Ulid that implements sea_orm::Value is probably better * rename variable to match struct name * add deserializer for Address -> Vec<u8> * sql sum returns a Decimal. need to convert to u64 * assert more * one log and assert more * log more * use a helper to get the user's rpc provider * this should be 2 now that we have a public and authed call * this should be zero. the public has the cache miss * instrument cu calcs * trace the value we took, not the default that replaced it * move usd_per_chain into config * remove some extra logging * use Arc::into_inner to maybe avoid a race * off by 1 * pass paid credits used instead of returning it this lets us use it to write to our user balance cache first. importantly, this keeps us from holding a write lock while writing to mysql * no cache misses expected in this test * actually check the admin * put the balance checks back now that the rest of the test works * archive request is being set incorrectly * wow howd we manage flipping the greater than sign on archive depth * move latest_balance and premium_credits_used to before any stats are emitted * lint * and build undoes the linting. fun i didnt even want to lint them in the first place, so this is fine * missed incrementing total_spent when not incrementing total_spent_paid_credits * use the credits on self * use the credits on self (pt 2) * fix type for 10 cu query * convert the requestmetadata on the other side of the channel * logs * viewing stats is allowed even without a balance * move paid_credits_used to AuthorizationChecks * wip * test_sum_credits_used finally passes * UserBalanceCache::get_or_insert * re-enable rpc_secret_key_cache * move invalidate to a helper function and always call it **after** the db is commited * fix PartialEq and Eq on RpcSecretKey * cargo upgrade
2023-07-12 10:35:07 +03:00
}
2023-07-08 00:06:44 +03:00
Err(err) => {
More balance tests (#182) * fix popularity contest * more info in the Debug for Web3Rpc * add frontend_requests and cache_misses to the Balance query * add more to balance and stats flushing and improved test coverage * it compiles * deserializer for Ulid to Uuid I think a wrapper type on Ulid that implements sea_orm::Value is probably better * rename variable to match struct name * add deserializer for Address -> Vec<u8> * sql sum returns a Decimal. need to convert to u64 * assert more * one log and assert more * log more * use a helper to get the user's rpc provider * this should be 2 now that we have a public and authed call * this should be zero. the public has the cache miss * instrument cu calcs * trace the value we took, not the default that replaced it * move usd_per_chain into config * remove some extra logging * use Arc::into_inner to maybe avoid a race * off by 1 * pass paid credits used instead of returning it this lets us use it to write to our user balance cache first. importantly, this keeps us from holding a write lock while writing to mysql * no cache misses expected in this test * actually check the admin * put the balance checks back now that the rest of the test works * archive request is being set incorrectly * wow howd we manage flipping the greater than sign on archive depth * move latest_balance and premium_credits_used to before any stats are emitted * lint * and build undoes the linting. fun i didnt even want to lint them in the first place, so this is fine * missed incrementing total_spent when not incrementing total_spent_paid_credits * use the credits on self * use the credits on self (pt 2) * fix type for 10 cu query * convert the requestmetadata on the other side of the channel * logs * viewing stats is allowed even without a balance * move paid_credits_used to AuthorizationChecks * wip * test_sum_credits_used finally passes * UserBalanceCache::get_or_insert * re-enable rpc_secret_key_cache * move invalidate to a helper function and always call it **after** the db is commited * fix PartialEq and Eq on RpcSecretKey * cargo upgrade
2023-07-12 10:35:07 +03:00
request_metadata
.error_response
.store(true, Ordering::Release);
2023-03-03 04:39:50 +03:00
2023-07-08 00:06:44 +03:00
err.as_response_parts()
More balance tests (#182) * fix popularity contest * more info in the Debug for Web3Rpc * add frontend_requests and cache_misses to the Balance query * add more to balance and stats flushing and improved test coverage * it compiles * deserializer for Ulid to Uuid I think a wrapper type on Ulid that implements sea_orm::Value is probably better * rename variable to match struct name * add deserializer for Address -> Vec<u8> * sql sum returns a Decimal. need to convert to u64 * assert more * one log and assert more * log more * use a helper to get the user's rpc provider * this should be 2 now that we have a public and authed call * this should be zero. the public has the cache miss * instrument cu calcs * trace the value we took, not the default that replaced it * move usd_per_chain into config * remove some extra logging * use Arc::into_inner to maybe avoid a race * off by 1 * pass paid credits used instead of returning it this lets us use it to write to our user balance cache first. importantly, this keeps us from holding a write lock while writing to mysql * no cache misses expected in this test * actually check the admin * put the balance checks back now that the rest of the test works * archive request is being set incorrectly * wow howd we manage flipping the greater than sign on archive depth * move latest_balance and premium_credits_used to before any stats are emitted * lint * and build undoes the linting. fun i didnt even want to lint them in the first place, so this is fine * missed incrementing total_spent when not incrementing total_spent_paid_credits * use the credits on self * use the credits on self (pt 2) * fix type for 10 cu query * convert the requestmetadata on the other side of the channel * logs * viewing stats is allowed even without a balance * move paid_credits_used to AuthorizationChecks * wip * test_sum_credits_used finally passes * UserBalanceCache::get_or_insert * re-enable rpc_secret_key_cache * move invalidate to a helper function and always call it **after** the db is commited * fix PartialEq and Eq on RpcSecretKey * cargo upgrade
2023-07-12 10:35:07 +03:00
}
2023-07-08 00:06:44 +03:00
};
2023-06-27 07:38:41 +03:00
2023-07-08 00:06:44 +03:00
let response = JsonRpcForwardedResponse::from_response_data(response_data, response_id);
// TODO: this serializes twice :/
request_metadata.add_response(ResponseOrBytes::Response(&response));
2023-03-03 04:39:50 +03:00
let rpcs = request_metadata.backend_rpcs_used();
2023-03-03 04:39:50 +03:00
More balance tests (#182) * fix popularity contest * more info in the Debug for Web3Rpc * add frontend_requests and cache_misses to the Balance query * add more to balance and stats flushing and improved test coverage * it compiles * deserializer for Ulid to Uuid I think a wrapper type on Ulid that implements sea_orm::Value is probably better * rename variable to match struct name * add deserializer for Address -> Vec<u8> * sql sum returns a Decimal. need to convert to u64 * assert more * one log and assert more * log more * use a helper to get the user's rpc provider * this should be 2 now that we have a public and authed call * this should be zero. the public has the cache miss * instrument cu calcs * trace the value we took, not the default that replaced it * move usd_per_chain into config * remove some extra logging * use Arc::into_inner to maybe avoid a race * off by 1 * pass paid credits used instead of returning it this lets us use it to write to our user balance cache first. importantly, this keeps us from holding a write lock while writing to mysql * no cache misses expected in this test * actually check the admin * put the balance checks back now that the rest of the test works * archive request is being set incorrectly * wow howd we manage flipping the greater than sign on archive depth * move latest_balance and premium_credits_used to before any stats are emitted * lint * and build undoes the linting. fun i didnt even want to lint them in the first place, so this is fine * missed incrementing total_spent when not incrementing total_spent_paid_credits * use the credits on self * use the credits on self (pt 2) * fix type for 10 cu query * convert the requestmetadata on the other side of the channel * logs * viewing stats is allowed even without a balance * move paid_credits_used to AuthorizationChecks * wip * test_sum_credits_used finally passes * UserBalanceCache::get_or_insert * re-enable rpc_secret_key_cache * move invalidate to a helper function and always call it **after** the db is commited * fix PartialEq and Eq on RpcSecretKey * cargo upgrade
2023-07-12 10:35:07 +03:00
// there might be clones in the background, so this isn't a sure thing
let _ = request_metadata.try_send_arc_stat();
(code, response, rpcs)
}
2023-03-03 04:39:50 +03:00
/// main logic for proxy_cached_request but in a dedicated function so the try operator is easy to use
/// TODO: how can we make this generic?
async fn _proxy_request_with_caching(
self: &Arc<Self>,
method: &str,
2023-06-27 07:38:41 +03:00
params: &mut serde_json::Value,
head_block: Option<&Web3ProxyBlock>,
2023-06-29 01:04:55 +03:00
max_tries: Option<usize>,
request_metadata: &Arc<RequestMetadata>,
2023-06-07 23:57:38 +03:00
) -> Web3ProxyResult<JsonRpcResponseEnum<Arc<RawValue>>> {
// TODO: don't clone into a new string?
let request_method = method.to_string();
// TODO: serve net_version without querying the backend
// TODO: don't force RawValue
2023-06-07 23:57:38 +03:00
let response_data: JsonRpcResponseEnum<Arc<RawValue>> = match request_method.as_ref() {
// lots of commands are blocked
2023-02-03 21:56:05 +03:00
method @ ("db_getHex"
| "db_getString"
| "db_putHex"
| "db_putString"
| "debug_accountRange"
| "debug_backtraceAt"
| "debug_blockProfile"
| "debug_bundler_clearState"
| "debug_bundler_dumpMempool"
| "debug_bundler_sendBundleNow"
2022-06-14 09:54:19 +03:00
| "debug_chaindbCompact"
| "debug_chaindbProperty"
| "debug_cpuProfile"
| "debug_freeOSMemory"
2022-06-14 09:54:19 +03:00
| "debug_freezeClient"
| "debug_gcStats"
2022-06-14 09:54:19 +03:00
| "debug_goTrace"
| "debug_memStats"
2022-06-14 09:54:19 +03:00
| "debug_mutexProfile"
| "debug_setBlockProfileRate"
| "debug_setGCPercent"
| "debug_setHead"
| "debug_setMutexProfileFraction"
| "debug_standardTraceBadBlockToFile"
| "debug_standardTraceBlockToFile"
2022-06-14 09:54:19 +03:00
| "debug_startCPUProfile"
| "debug_startGoTrace"
| "debug_stopCPUProfile"
| "debug_stopGoTrace"
| "debug_writeBlockProfile"
| "debug_writeMemProfile"
| "debug_writeMutexProfile"
| "erigon_cacheCheck"
| "eth_compileLLL"
| "eth_compileSerpent"
| "eth_compileSolidity"
| "eth_getCompilers"
| "eth_sendTransaction"
| "eth_sign"
| "eth_signTransaction"
| "eth_submitHashrate"
| "eth_submitWork"
2022-06-14 09:54:19 +03:00
| "les_addBalance"
| "les_setClientParams"
| "les_setDefaultParams"
| "miner_setEtherbase"
2022-06-14 09:54:19 +03:00
| "miner_setExtra"
| "miner_setGasLimit"
2022-06-14 09:54:19 +03:00
| "miner_setGasPrice"
| "miner_start"
| "miner_stop"
| "personal_ecRecover"
2022-06-14 09:54:19 +03:00
| "personal_importRawKey"
| "personal_listAccounts"
| "personal_lockAccount"
| "personal_newAccount"
| "personal_sendTransaction"
| "personal_sign"
| "personal_unlockAccount"
| "shh_addToGroup"
| "shh_getFilterChanges"
| "shh_getMessages"
| "shh_hasIdentity"
| "shh_newFilter"
| "shh_newGroup"
| "shh_newIdentity"
| "shh_post"
| "shh_uninstallFilter"
2022-09-09 00:01:36 +03:00
| "shh_version") => {
// i don't think we will ever support these methods. maybe do Forbidden?
2023-01-24 19:07:10 +03:00
// TODO: what error code?
JsonRpcErrorData::from(format!(
"the method {} does not exist/is not available",
method
)).into()
}
// TODO: implement these commands
2022-09-09 00:01:36 +03:00
method @ ("eth_getFilterChanges"
| "eth_getFilterLogs"
| "eth_newBlockFilter"
| "eth_newFilter"
| "eth_newPendingTransactionFilter"
2023-02-02 00:56:43 +03:00
| "eth_pollSubscriptions"
2022-09-09 00:01:36 +03:00
| "eth_uninstallFilter") => {
// TODO: unsupported command stat. use the count to prioritize new features
2023-01-24 19:07:10 +03:00
// TODO: what error code?
JsonRpcErrorData::from(format!(
"the method {} is not yet implemented. contact us if you need this",
method
))
.into()
2023-04-14 10:04:35 +03:00
}
method @ ("eth_sendUserOperation"
2023-04-14 10:04:35 +03:00
| "eth_estimateUserOperationGas"
| "eth_getUserOperationByHash"
| "eth_getUserOperationReceipt"
| "eth_supportedEntryPoints") => match self.bundler_4337_rpcs.as_ref() {
Some(bundler_4337_rpcs) => {
// TODO: timeout
let x = bundler_4337_rpcs
.try_proxy_connection::<_, Box<RawValue>>(
method,
2023-06-27 07:38:41 +03:00
params,
Some(request_metadata),
2023-06-29 01:04:55 +03:00
max_tries,
Some(Duration::from_secs(30)),
2023-04-14 10:04:35 +03:00
None,
None,
)
.await?;
x.into()
2023-04-14 10:04:35 +03:00
}
None => {
// TODO: stats even when we error!
// TODO: dedicated error for no 4337 bundlers
return Err(Web3ProxyError::NoServersSynced);
2023-04-14 10:04:35 +03:00
}
},
"eth_accounts" => JsonRpcResponseEnum::from(serde_json::Value::Array(vec![])),
"eth_blockNumber" => {
match head_block.cloned().or(self.balanced_rpcs.head_block()) {
2023-06-29 08:59:46 +03:00
Some(head_block) => JsonRpcResponseEnum::from(json!(head_block.number())),
2022-09-01 08:58:55 +03:00
None => {
return Err(Web3ProxyError::NoServersSynced);
2022-09-01 08:58:55 +03:00
}
}
}
"eth_chainId" => JsonRpcResponseEnum::from(json!(U64::from(self.config.chain_id))),
// TODO: eth_callBundle (https://docs.flashbots.net/flashbots-auction/searchers/advanced/rpc-endpoint#eth_callbundle)
// TODO: eth_cancelPrivateTransaction (https://docs.flashbots.net/flashbots-auction/searchers/advanced/rpc-endpoint#eth_cancelprivatetransaction, but maybe just reject)
// TODO: eth_sendPrivateTransaction (https://docs.flashbots.net/flashbots-auction/searchers/advanced/rpc-endpoint#eth_sendprivatetransaction)
"eth_coinbase" => {
2022-07-22 22:30:39 +03:00
// no need for serving coinbase
JsonRpcResponseEnum::from(json!(Address::zero()))
}
"eth_estimateGas" => {
2023-05-24 01:50:29 +03:00
// TODO: timeout
let mut gas_estimate = self
.balanced_rpcs
.try_proxy_connection::<_, U256>(
method,
2023-06-27 07:38:41 +03:00
params,
Some(request_metadata),
2023-06-29 01:04:55 +03:00
max_tries,
Some(Duration::from_secs(30)),
None,
2023-02-11 07:45:57 +03:00
None,
)
.await?;
let gas_increase = if let Some(gas_increase_percent) =
self.config.gas_increase_percent
{
let gas_increase = gas_estimate * gas_increase_percent / U256::from(100);
2023-01-20 05:08:53 +03:00
let min_gas_increase = self.config.gas_increase_min.unwrap_or_default();
2023-01-20 05:08:53 +03:00
gas_increase.max(min_gas_increase)
} else {
self.config.gas_increase_min.unwrap_or_default()
};
2023-01-20 05:08:53 +03:00
gas_estimate += gas_increase;
// TODO: from_serializable?
JsonRpcResponseEnum::from(json!(gas_estimate))
}
"eth_getTransactionReceipt" | "eth_getTransactionByHash" => {
// try to get the transaction without specifying a min_block_height
2023-05-24 01:50:29 +03:00
// TODO: timeout
let mut response_data = self
.balanced_rpcs
.try_proxy_connection::<_, Box<RawValue>>(
method,
2023-06-27 07:38:41 +03:00
params,
Some(request_metadata),
2023-06-29 01:04:55 +03:00
max_tries,
Some(Duration::from_secs(30)),
None,
None,
)
.await;
// if we got "null", it is probably because the tx is old. retry on nodes with old block data
let try_archive = if let Ok(value) = &response_data {
value.get() == "null"
} else {
true
};
if try_archive {
request_metadata
.archive_request
.store(true, atomic::Ordering::Release);
response_data = self
.balanced_rpcs
.try_proxy_connection::<_, Box<RawValue>>(
method,
2023-06-27 07:38:41 +03:00
params,
Some(request_metadata),
2023-06-29 01:04:55 +03:00
max_tries,
Some(Duration::from_secs(30)),
2023-06-29 01:04:55 +03:00
// TODO: should this be block 0 instead?
Some(&U64::one()),
None,
)
.await;
}
response_data.try_into()?
}
// TODO: eth_gasPrice that does awesome magic to predict the future
"eth_hashrate" => JsonRpcResponseEnum::from(json!(U64::zero())),
"eth_mining" => JsonRpcResponseEnum::from(serde_json::Value::Bool(false)),
// TODO: eth_sendBundle (flashbots/eden command)
// broadcast transactions to all private rpcs at once
2022-08-09 19:54:05 +03:00
"eth_sendRawTransaction" => {
// TODO: decode the transaction
2023-01-17 09:54:40 +03:00
// TODO: error if the chain_id is incorrect
let response = timeout(
Duration::from_secs(30),
self
.try_send_protected(
method,
2023-06-27 07:38:41 +03:00
params,
request_metadata,
)
)
.await?;
let mut response = response.try_into()?;
2022-12-24 04:32:58 +03:00
// sometimes we get an error that the transaction is already known by our nodes,
// that's not really an error. Return the hash like a successful response would.
// TODO: move this to a helper function
if let JsonRpcResponseEnum::RpcError{ error_data, ..} = &response {
if error_data.code == -32000
&& (error_data.message == "ALREADY_EXISTS: already known"
|| error_data.message == "INTERNAL_ERROR: existing tx with same hash")
2022-12-24 04:32:58 +03:00
{
let params = params
.as_array()
.ok_or_else(|| {
Web3ProxyError::BadRequest(
2023-05-31 09:17:05 +03:00
"Unable to get array from params".into(),
)
})?
2022-12-24 04:32:58 +03:00
.get(0)
.ok_or_else(|| {
Web3ProxyError::BadRequest(
2023-05-31 09:17:05 +03:00
"Unable to get item 0 from params".into(),
)
})?
2022-12-24 04:32:58 +03:00
.as_str()
.ok_or_else(|| {
Web3ProxyError::BadRequest(
2023-05-31 09:17:05 +03:00
"Unable to get string from params item 0".into(),
)
})?;
2022-12-24 04:32:58 +03:00
let params = Bytes::from_str(params)
.expect("there must be Bytes if we got this far");
let rlp = Rlp::new(params.as_ref());
if let Ok(tx) = Transaction::decode(&rlp) {
// TODO: decode earlier and confirm that tx.chain_id (if set) matches self.config.chain_id
2022-12-24 04:32:58 +03:00
let tx_hash = json!(tx.hash());
trace!("tx_hash: {:#?}", tx_hash);
2022-12-24 04:32:58 +03:00
response = JsonRpcResponseEnum::from(tx_hash);
2022-12-24 04:32:58 +03:00
}
}
}
// emit transaction count stats
// TODO: use this cache to avoid sending duplicate transactions?
// TODO: different salt for ips and transactions?
if let Some(ref salt) = self.config.public_recent_ips_salt {
if let JsonRpcResponseEnum::Result { value, .. } = &response {
2022-12-29 09:21:09 +03:00
let now = Utc::now().timestamp();
let app = self.clone();
let salted_tx_hash = format!("{}:{}", salt, value.get());
2022-12-29 09:21:09 +03:00
let f = async move {
match app.redis_conn().await {
Ok(mut redis_conn) => {
2022-12-29 09:21:09 +03:00
let hashed_tx_hash =
Bytes::from(keccak256(salted_tx_hash.as_bytes()));
let recent_tx_hash_key =
format!("eth_sendRawTransaction:{}", app.config.chain_id);
redis_conn
.zadd(recent_tx_hash_key, hashed_tx_hash.to_string(), now)
.await?;
}
Err(Web3ProxyError::NoDatabaseConfigured) => {},
2022-12-29 09:21:09 +03:00
Err(err) => {
warn!(
More balance tests (#182) * fix popularity contest * more info in the Debug for Web3Rpc * add frontend_requests and cache_misses to the Balance query * add more to balance and stats flushing and improved test coverage * it compiles * deserializer for Ulid to Uuid I think a wrapper type on Ulid that implements sea_orm::Value is probably better * rename variable to match struct name * add deserializer for Address -> Vec<u8> * sql sum returns a Decimal. need to convert to u64 * assert more * one log and assert more * log more * use a helper to get the user's rpc provider * this should be 2 now that we have a public and authed call * this should be zero. the public has the cache miss * instrument cu calcs * trace the value we took, not the default that replaced it * move usd_per_chain into config * remove some extra logging * use Arc::into_inner to maybe avoid a race * off by 1 * pass paid credits used instead of returning it this lets us use it to write to our user balance cache first. importantly, this keeps us from holding a write lock while writing to mysql * no cache misses expected in this test * actually check the admin * put the balance checks back now that the rest of the test works * archive request is being set incorrectly * wow howd we manage flipping the greater than sign on archive depth * move latest_balance and premium_credits_used to before any stats are emitted * lint * and build undoes the linting. fun i didnt even want to lint them in the first place, so this is fine * missed incrementing total_spent when not incrementing total_spent_paid_credits * use the credits on self * use the credits on self (pt 2) * fix type for 10 cu query * convert the requestmetadata on the other side of the channel * logs * viewing stats is allowed even without a balance * move paid_credits_used to AuthorizationChecks * wip * test_sum_credits_used finally passes * UserBalanceCache::get_or_insert * re-enable rpc_secret_key_cache * move invalidate to a helper function and always call it **after** the db is commited * fix PartialEq and Eq on RpcSecretKey * cargo upgrade
2023-07-12 10:35:07 +03:00
?err,
2023-06-29 07:30:00 +03:00
"unable to save stats for eth_sendRawTransaction",
2022-12-29 09:21:09 +03:00
)
}
}
Ok::<_, anyhow::Error>(())
};
tokio::spawn(f);
}
}
response
2022-08-09 19:54:05 +03:00
}
"eth_syncing" => {
// no stats on this. its cheap
// TODO: return a real response if all backends are syncing or if no servers in sync
// TODO: const
JsonRpcResponseEnum::from(serde_json::Value::Bool(false))
}
"eth_subscribe" => JsonRpcErrorData {
message: "notifications not supported. eth_subscribe is only available over a websocket".into(),
code: -32601,
data: None,
}
.into(),
"eth_unsubscribe" => JsonRpcErrorData {
message: "notifications not supported. eth_unsubscribe is only available over a websocket".into(),
code: -32601,
data: None,
}.into(),
"net_listening" => {
// TODO: only true if there are some backends on balanced_rpcs?
// TODO: const
JsonRpcResponseEnum::from(serde_json::Value::Bool(true))
}
"net_peerCount" =>
JsonRpcResponseEnum::from(json!(U64::from(self.balanced_rpcs.num_synced_rpcs())))
,
"web3_clientVersion" =>
JsonRpcResponseEnum::from(serde_json::Value::String(APP_USER_AGENT.to_string()))
,
2022-07-22 22:30:39 +03:00
"web3_sha3" => {
// returns Keccak-256 (not the standardized SHA3-256) of the given data.
2023-05-24 01:50:29 +03:00
// TODO: timeout
match &params {
serde_json::Value::Array(params) => {
2022-08-10 05:37:34 +03:00
// TODO: make a struct and use serde conversion to clean this up
2023-03-09 20:32:30 +03:00
if params.len() != 1
|| !params.get(0).map(|x| x.is_string()).unwrap_or(false)
{
2023-01-24 19:07:10 +03:00
// TODO: what error code?
// TODO: use Web3ProxyError::BadRequest
JsonRpcErrorData {
message: "Invalid request".into(),
code: -32600,
data: None
}.into()
} else {
// TODO: BadRequest instead of web3_context
let param = Bytes::from_str(
params[0]
.as_str()
.ok_or(Web3ProxyError::ParseBytesError(None))
.web3_context("parsing params 0 into str then bytes")?,
)
.map_err(|x| {
trace!("bad request: {:?}", x);
Web3ProxyError::BadRequest(
2023-05-31 09:17:05 +03:00
"param 0 could not be read as H256".into(),
)
})?;
let hash = H256::from(keccak256(param));
JsonRpcResponseEnum::from(json!(hash))
}
2022-07-22 22:30:39 +03:00
}
2022-09-10 03:58:33 +03:00
_ => {
// TODO: this needs the correct error code in the response
// TODO: Web3ProxyError::BadRequest instead?
JsonRpcErrorData {
message: "invalid request".into(),
code: StatusCode::BAD_REQUEST.as_u16().into(),
data: None,
}.into()
2022-09-10 03:58:33 +03:00
}
2022-07-22 22:30:39 +03:00
}
}
"test" => JsonRpcErrorData {
message: "The method test does not exist/is not available.".into(),
code: -32601,
data: None,
}.into(),
2022-07-22 22:30:39 +03:00
// anything else gets sent to backend rpcs and cached
2022-06-30 03:52:04 +03:00
method => {
2023-02-03 21:56:05 +03:00
if method.starts_with("admin_") {
// TODO: emit a stat? will probably just be noise
return Err(Web3ProxyError::AccessDenied("admin methods are not allowed".into()));
2023-02-03 21:56:05 +03:00
}
2023-01-23 09:02:08 +03:00
// TODO: if no servers synced, wait for them to be synced? probably better to error and let haproxy retry another server
let head_block: Web3ProxyBlock = head_block
.cloned()
.or_else(|| self.balanced_rpcs.head_block())
.ok_or(Web3ProxyError::NoServersSynced)?;
2022-07-16 07:13:02 +03:00
// we do this check before checking caches because it might modify the request params
2022-07-16 08:48:02 +03:00
// TODO: add a stat for archive vs full since they should probably cost different
2022-12-17 07:05:01 +03:00
// TODO: this cache key can be rather large. is that okay?
let cache_key: Option<JsonRpcQueryCacheKey> = match CacheMode::new(
2022-09-22 02:50:55 +03:00
method,
2023-06-27 07:38:41 +03:00
params,
&head_block,
2022-09-22 02:50:55 +03:00
&self.balanced_rpcs,
)
.await
2022-09-05 09:13:36 +03:00
{
CacheMode::CacheSuccessForever => Some(JsonRpcQueryCacheKey::new(
None,
None,
method,
2023-06-27 07:38:41 +03:00
params,
false,
)),
CacheMode::CacheNever => None,
CacheMode::Cache {
block,
2022-12-17 07:05:01 +03:00
cache_errors,
} => {
2023-06-29 05:19:36 +03:00
let block_depth = (head_block.number().saturating_sub(*block.num())).as_u64();
2022-12-17 07:05:01 +03:00
More balance tests (#182) * fix popularity contest * more info in the Debug for Web3Rpc * add frontend_requests and cache_misses to the Balance query * add more to balance and stats flushing and improved test coverage * it compiles * deserializer for Ulid to Uuid I think a wrapper type on Ulid that implements sea_orm::Value is probably better * rename variable to match struct name * add deserializer for Address -> Vec<u8> * sql sum returns a Decimal. need to convert to u64 * assert more * one log and assert more * log more * use a helper to get the user's rpc provider * this should be 2 now that we have a public and authed call * this should be zero. the public has the cache miss * instrument cu calcs * trace the value we took, not the default that replaced it * move usd_per_chain into config * remove some extra logging * use Arc::into_inner to maybe avoid a race * off by 1 * pass paid credits used instead of returning it this lets us use it to write to our user balance cache first. importantly, this keeps us from holding a write lock while writing to mysql * no cache misses expected in this test * actually check the admin * put the balance checks back now that the rest of the test works * archive request is being set incorrectly * wow howd we manage flipping the greater than sign on archive depth * move latest_balance and premium_credits_used to before any stats are emitted * lint * and build undoes the linting. fun i didnt even want to lint them in the first place, so this is fine * missed incrementing total_spent when not incrementing total_spent_paid_credits * use the credits on self * use the credits on self (pt 2) * fix type for 10 cu query * convert the requestmetadata on the other side of the channel * logs * viewing stats is allowed even without a balance * move paid_credits_used to AuthorizationChecks * wip * test_sum_credits_used finally passes * UserBalanceCache::get_or_insert * re-enable rpc_secret_key_cache * move invalidate to a helper function and always call it **after** the db is commited * fix PartialEq and Eq on RpcSecretKey * cargo upgrade
2023-07-12 10:35:07 +03:00
if block_depth > self.config.archive_depth {
trace!(%block_depth, archive_depth=%self.config.archive_depth);
2022-12-17 07:05:01 +03:00
request_metadata
.archive_request
.store(true, atomic::Ordering::Release);
2022-12-17 07:05:01 +03:00
}
2022-12-03 08:31:03 +03:00
Some(JsonRpcQueryCacheKey::new(
Some(block),
None,
method,
2023-06-27 07:38:41 +03:00
params,
cache_errors,
))
}
CacheMode::CacheRange {
from_block,
to_block,
cache_errors,
} => {
2023-06-29 05:19:36 +03:00
let block_depth = (head_block.number().saturating_sub(*from_block.num())).as_u64();
More balance tests (#182) * fix popularity contest * more info in the Debug for Web3Rpc * add frontend_requests and cache_misses to the Balance query * add more to balance and stats flushing and improved test coverage * it compiles * deserializer for Ulid to Uuid I think a wrapper type on Ulid that implements sea_orm::Value is probably better * rename variable to match struct name * add deserializer for Address -> Vec<u8> * sql sum returns a Decimal. need to convert to u64 * assert more * one log and assert more * log more * use a helper to get the user's rpc provider * this should be 2 now that we have a public and authed call * this should be zero. the public has the cache miss * instrument cu calcs * trace the value we took, not the default that replaced it * move usd_per_chain into config * remove some extra logging * use Arc::into_inner to maybe avoid a race * off by 1 * pass paid credits used instead of returning it this lets us use it to write to our user balance cache first. importantly, this keeps us from holding a write lock while writing to mysql * no cache misses expected in this test * actually check the admin * put the balance checks back now that the rest of the test works * archive request is being set incorrectly * wow howd we manage flipping the greater than sign on archive depth * move latest_balance and premium_credits_used to before any stats are emitted * lint * and build undoes the linting. fun i didnt even want to lint them in the first place, so this is fine * missed incrementing total_spent when not incrementing total_spent_paid_credits * use the credits on self * use the credits on self (pt 2) * fix type for 10 cu query * convert the requestmetadata on the other side of the channel * logs * viewing stats is allowed even without a balance * move paid_credits_used to AuthorizationChecks * wip * test_sum_credits_used finally passes * UserBalanceCache::get_or_insert * re-enable rpc_secret_key_cache * move invalidate to a helper function and always call it **after** the db is commited * fix PartialEq and Eq on RpcSecretKey * cargo upgrade
2023-07-12 10:35:07 +03:00
if block_depth > self.config.archive_depth {
trace!(%block_depth, archive_depth=%self.config.archive_depth);
request_metadata
.archive_request
.store(true, atomic::Ordering::Release);
}
Some(JsonRpcQueryCacheKey::new(
Some(from_block),
Some(to_block),
method,
2023-06-27 07:38:41 +03:00
params,
2022-12-17 07:05:01 +03:00
cache_errors,
))
2022-12-17 07:05:01 +03:00
}
};
2022-12-17 07:05:01 +03:00
// TODO: different timeouts for different user tiers. get the duration out of the request_metadata
let backend_request_timetout = Duration::from_secs(240);
2023-05-24 01:50:29 +03:00
if let Some(cache_key) = cache_key {
let from_block_num = cache_key.from_block_num().copied();
let to_block_num = cache_key.to_block_num().copied();
let cache_jsonrpc_errors = cache_key.cache_errors();
// TODO: try to fetch out of s3
2023-06-08 03:26:38 +03:00
self
.jsonrpc_response_cache
.try_get_with::<_, Web3ProxyError>(cache_key.hash(), async {
2023-05-24 01:50:29 +03:00
let response_data = timeout(
backend_request_timetout + Duration::from_millis(100),
2023-05-24 01:50:29 +03:00
self.balanced_rpcs
2023-06-07 23:57:38 +03:00
.try_proxy_connection::<_, Arc<RawValue>>(
method,
2023-06-27 07:38:41 +03:00
params,
2023-05-24 01:50:29 +03:00
Some(request_metadata),
2023-06-29 01:04:55 +03:00
max_tries,
Some(backend_request_timetout),
2023-05-24 01:50:29 +03:00
from_block_num.as_ref(),
to_block_num.as_ref(),
))
.await?;
2023-06-08 03:26:38 +03:00
if !cache_jsonrpc_errors && let Err(err) = response_data {
// if we are not supposed to cache jsonrpc errors,
// then we must not convert Provider errors into a JsonRpcResponseEnum
// return all the errors now. moka will not cache Err results
Err(err)
2023-06-08 03:26:38 +03:00
} else {
let response_data: JsonRpcResponseEnum<Arc<RawValue>> = response_data.try_into()?;
// TODO: response data should maybe be Arc<JsonRpcResponseEnum<Box<RawValue>>>, but that's more work
Ok(response_data)
}
}).await?
} else {
let x = timeout(
backend_request_timetout + Duration::from_millis(100),
2023-05-24 01:50:29 +03:00
self.balanced_rpcs
2023-06-07 23:57:38 +03:00
.try_proxy_connection::<_, Arc<RawValue>>(
method,
2023-06-27 07:38:41 +03:00
params,
2023-05-24 01:50:29 +03:00
Some(request_metadata),
2023-06-29 01:04:55 +03:00
max_tries,
Some(backend_request_timetout),
2023-05-24 01:50:29 +03:00
None,
None,
)
)
.await??;
x.into()
}
}
2022-07-22 22:30:39 +03:00
};
Ok(response_data)
2022-05-12 02:50:52 +03:00
}
}
2022-08-24 03:59:05 +03:00
impl fmt::Debug for Web3ProxyApp {
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("Web3ProxyApp").finish_non_exhaustive()
}
}