From 0531e2f8dd353d5c382a95671314d97ccd071a65 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Mon, 8 May 2023 11:25:27 -0600 Subject: [PATCH 01/66] add 'quota usage' rate limit substring --- web3_proxy/src/rpcs/many.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index dbead8a8..8fce98df 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -929,7 +929,7 @@ impl Web3Rpcs { // different providers do different codes. check all of them // TODO: there's probably more strings to add here - let rate_limit_substrings = ["limit", "exceeded"]; + let rate_limit_substrings = ["limit", "exceeded", "quota usage"]; for rate_limit_substr in rate_limit_substrings { if error_msg.contains(rate_limit_substr) { warn!("rate limited by {}", skip_rpcs.last().unwrap()); From ec11e210eee76e3f1fbef7c576f290add93f3bf8 Mon Sep 17 00:00:00 2001 From: Rory Trent Date: Thu, 11 May 2023 13:09:15 -0700 Subject: [PATCH 02/66] Peak ewma (#63) * use peak-ewma instead of head for latency calculation * Implement some suggested changes from PR * move latency to new package in workspace root * fix unit tests which now require peak_latency on Web3Rpc * Switch to atomics for peak-ewma This change is to avoid locking from tokio::sync::watch. * add decay calculation to latency reads in peak-ewma * Add some tests for peak-ewma * Sensible latency defaults and not blocking on full * Cleanup and a couple additional comments --- Cargo.lock | 12 +- Cargo.toml | 1 + latency/Cargo.toml | 15 +++ latency/src/ewma.rs | 67 ++++++++++ latency/src/lib.rs | 5 + latency/src/peak_ewma/mod.rs | 149 +++++++++++++++++++++++ latency/src/peak_ewma/rtt_estimate.rs | 168 ++++++++++++++++++++++++++ latency/src/util/atomic_f32_pair.rs | 88 ++++++++++++++ latency/src/util/mod.rs | 2 + latency/src/util/nanos.rs | 30 +++++ web3_proxy/Cargo.toml | 2 +- web3_proxy/src/rpcs/many.rs | 31 ++++- web3_proxy/src/rpcs/one.rs | 86 +++---------- web3_proxy/src/rpcs/request.rs | 12 +- 14 files changed, 586 insertions(+), 82 deletions(-) create mode 100644 latency/Cargo.toml create mode 100644 latency/src/ewma.rs create mode 100644 latency/src/lib.rs create mode 100644 latency/src/peak_ewma/mod.rs create mode 100644 latency/src/peak_ewma/rtt_estimate.rs create mode 100644 latency/src/util/atomic_f32_pair.rs create mode 100644 latency/src/util/mod.rs create mode 100644 latency/src/util/nanos.rs diff --git a/Cargo.lock b/Cargo.lock index ae05a0fb..7a56dc18 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3020,6 +3020,16 @@ dependencies = [ "regex", ] +[[package]] +name = "latency" +version = "0.1.0" +dependencies = [ + "ewma", + "log", + "serde", + "tokio", +] + [[package]] name = "lazy_static" version = "1.4.0" @@ -6657,7 +6667,6 @@ dependencies = [ "entities", "env_logger", "ethers", - "ewma", "fdlimit", "flume", "fstrings", @@ -6673,6 +6682,7 @@ dependencies = [ "influxdb2-structmap", "ipnet", "itertools", + "latency", "log", "migration", "moka", diff --git a/Cargo.toml b/Cargo.toml index 9c7db66d..3c0a9c22 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -2,6 +2,7 @@ members = [ "deferred-rate-limiter", "entities", + "latency", "migration", "rate-counter", "redis-rate-limiter", diff --git a/latency/Cargo.toml b/latency/Cargo.toml new file mode 100644 index 00000000..daf303e3 --- /dev/null +++ b/latency/Cargo.toml @@ -0,0 +1,15 @@ +[package] +name = "latency" +version = "0.1.0" +edition = "2021" + +# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html + +[dependencies] +ewma = "0.1.1" +log = "0.4.17" +serde = { version = "1.0.159", features = [] } +tokio = { version = "1.27.0", features = ["full"] } + +[dev-dependencies] +tokio = { version = "1.27.0", features = ["full", "test-util"] } diff --git a/latency/src/ewma.rs b/latency/src/ewma.rs new file mode 100644 index 00000000..073dad54 --- /dev/null +++ b/latency/src/ewma.rs @@ -0,0 +1,67 @@ +use serde::ser::Serializer; +use serde::Serialize; +use tokio::time::Duration; + +pub struct EwmaLatency { + /// exponentially weighted moving average of how many milliseconds behind the fastest node we are + ewma: ewma::EWMA, +} + +impl Serialize for EwmaLatency { + fn serialize(&self, serializer: S) -> Result + where + S: Serializer, + { + serializer.serialize_f64(self.ewma.value()) + } +} + +impl EwmaLatency { + #[inline(always)] + pub fn record(&mut self, duration: Duration) { + self.record_ms(duration.as_secs_f64() * 1000.0); + } + + #[inline(always)] + pub fn record_ms(&mut self, milliseconds: f64) { + self.ewma.add(milliseconds); + } + + /// Current EWMA value in milliseconds + #[inline(always)] + pub fn value(&self) -> f64 { + self.ewma.value() + } +} + +impl Default for EwmaLatency { + fn default() -> Self { + // TODO: what should the default span be? 25 requests? + let span = 25.0; + + let start = 1000.0; + + Self::new(span, start) + } +} + +impl EwmaLatency { + // depending on the span, start might not be perfect + pub fn new(span: f64, start_ms: f64) -> Self { + let alpha = Self::span_to_alpha(span); + + let mut ewma = ewma::EWMA::new(alpha); + + if start_ms > 0.0 { + for _ in 0..(span as u64) { + ewma.add(start_ms); + } + } + + Self { ewma } + } + + fn span_to_alpha(span: f64) -> f64 { + 2.0 / (span + 1.0) + } +} diff --git a/latency/src/lib.rs b/latency/src/lib.rs new file mode 100644 index 00000000..4f305e85 --- /dev/null +++ b/latency/src/lib.rs @@ -0,0 +1,5 @@ +mod ewma; +mod peak_ewma; +mod util; + +pub use self::{ewma::EwmaLatency, peak_ewma::PeakEwmaLatency}; diff --git a/latency/src/peak_ewma/mod.rs b/latency/src/peak_ewma/mod.rs new file mode 100644 index 00000000..9b6f2f8b --- /dev/null +++ b/latency/src/peak_ewma/mod.rs @@ -0,0 +1,149 @@ +mod rtt_estimate; + +use std::sync::Arc; + +use log::error; +use tokio::sync::mpsc; +use tokio::sync::mpsc::error::TrySendError; +use tokio::task::JoinHandle; +use tokio::time::{Duration, Instant}; + +use self::rtt_estimate::AtomicRttEstimate; +use crate::util::nanos::nanos; + +/// Latency calculation using Peak EWMA algorithm +/// +/// Updates are done in a separate task to avoid locking or race +/// conditions. Reads may happen on any thread. +#[derive(Debug)] +pub struct PeakEwmaLatency { + /// Join handle for the latency calculation task + pub join_handle: JoinHandle<()>, + /// Send to update with each request duration + request_tx: mpsc::Sender, + /// Latency average and last update time + rtt_estimate: Arc, + /// Decay time + decay_ns: f64, +} + +impl PeakEwmaLatency { + /// Spawn the task for calculating peak request latency + /// + /// Returns a handle that can also be used to read the current + /// average latency. + pub fn spawn(decay_ns: f64, buf_size: usize, start_latency: Duration) -> Self { + debug_assert!(decay_ns > 0.0, "decay_ns must be positive"); + let (request_tx, request_rx) = mpsc::channel(buf_size); + let rtt_estimate = Arc::new(AtomicRttEstimate::new(start_latency)); + let task = PeakEwmaLatencyTask { + request_rx, + rtt_estimate: rtt_estimate.clone(), + update_at: Instant::now(), + decay_ns, + }; + let join_handle = tokio::spawn(task.run()); + Self { + join_handle, + request_tx, + rtt_estimate, + decay_ns, + } + } + + /// Get the current peak-ewma latency estimate + pub fn latency(&self) -> Duration { + let mut estimate = self.rtt_estimate.load(); + + let now = Instant::now(); + debug_assert!( + estimate.update_at <= now, + "update_at={:?} in the future", + estimate.update_at, + ); + + // Update the RTT estimate to account for decay since the last update. + estimate.update(0.0, self.decay_ns, now) + } + + /// Report latency from a single request + /// + /// Should only be called from the Web3Rpc that owns it. + pub fn report(&self, duration: Duration) { + match self.request_tx.try_send(duration) { + Ok(()) => {} + Err(TrySendError::Full(_)) => { + // We don't want to block if the channel is full, just + // report the error + error!("Latency report channel full"); + // TODO: could we spawn a new tokio task to report tthis later? + } + Err(TrySendError::Closed(_)) => { + unreachable!("Owner should keep channel open"); + } + }; + //.expect("Owner should keep channel open"); + } +} + +/// Task to be spawned per-Web3Rpc for calculating the peak request latency +#[derive(Debug)] +struct PeakEwmaLatencyTask { + /// Receive new request timings for update + request_rx: mpsc::Receiver, + /// Current estimate and update time + rtt_estimate: Arc, + /// Last update time, used for decay calculation + update_at: Instant, + /// Decay time + decay_ns: f64, +} + +impl PeakEwmaLatencyTask { + /// Run the loop for updating latency + async fn run(mut self) { + while let Some(rtt) = self.request_rx.recv().await { + self.update(rtt); + } + } + + /// Update the estimate object atomically. + fn update(&mut self, rtt: Duration) { + let rtt = nanos(rtt); + + let now = Instant::now(); + debug_assert!( + self.update_at <= now, + "update_at={:?} in the future", + self.update_at, + ); + + self.rtt_estimate + .fetch_update(|mut rtt_estimate| rtt_estimate.update(rtt, self.decay_ns, now)); + } +} + +#[cfg(test)] +mod tests { + use tokio::time::{self, Duration}; + + use crate::util::nanos::NANOS_PER_MILLI; + + /// The default RTT estimate decays, so that new nodes are considered if the + /// default RTT is too high. + #[tokio::test(start_paused = true)] + async fn default_decay() { + let estimate = + super::PeakEwmaLatency::spawn(NANOS_PER_MILLI * 1_000.0, 8, Duration::from_millis(10)); + let load = estimate.latency(); + assert_eq!(load, Duration::from_millis(10)); + + time::advance(Duration::from_millis(100)).await; + let load = estimate.latency(); + assert!(Duration::from_millis(9) < load && load < Duration::from_millis(10)); + + time::advance(Duration::from_millis(100)).await; + let load = estimate.latency(); + assert!(Duration::from_millis(8) < load && load < Duration::from_millis(9)); + } +} diff --git a/latency/src/peak_ewma/rtt_estimate.rs b/latency/src/peak_ewma/rtt_estimate.rs new file mode 100644 index 00000000..be56fe9c --- /dev/null +++ b/latency/src/peak_ewma/rtt_estimate.rs @@ -0,0 +1,168 @@ +use std::sync::atomic::Ordering; + +use log::trace; +use tokio::time::{Duration, Instant}; + +use crate::util::atomic_f32_pair::AtomicF32Pair; +use crate::util::nanos::{nanos, NANOS_PER_MILLI}; + +/// Holds the current RTT estimate and the last time this value was updated. +#[derive(Debug)] +pub struct RttEstimate { + pub update_at: Instant, + pub rtt: Duration, +} + +impl RttEstimate { + /// Update the estimate with a new rtt value. Use rtt=0.0 for simply + /// decaying the current value. + pub fn update(&mut self, rtt: f64, decay_ns: f64, now: Instant) -> Duration { + let ewma = nanos(self.rtt); + self.rtt = if ewma < rtt { + // For Peak-EWMA, always use the worst-case (peak) value as the estimate for + // subsequent requests. + trace!( + "update peak rtt={}ms prior={}ms", + rtt / NANOS_PER_MILLI, + ewma / NANOS_PER_MILLI, + ); + Duration::from_nanos(rtt as u64) + } else { + // When a latency is observed that is less than the estimated latency, we decay the + // prior estimate according to how much time has elapsed since the last + // update. The inverse of the decay is used to scale the estimate towards the + // observed latency value. + let elapsed = nanos(now.saturating_duration_since(self.update_at)); + let decay = (-elapsed / decay_ns).exp(); + let recency = 1.0 - decay; + let next_estimate = (ewma * decay) + (rtt * recency); + trace!( + "update duration={:03.0}ms decay={:06.0}ns; next={:03.0}ms", + rtt / NANOS_PER_MILLI, + ewma - next_estimate, + next_estimate / NANOS_PER_MILLI, + ); + Duration::from_nanos(next_estimate as u64) + }; + self.rtt + } + + /// Build a new estimate object using current time. + fn new(start_duration: Duration) -> Self { + Self { + update_at: Instant::now(), + rtt: start_duration, + } + } + + /// Convert to pair of f32 + fn as_pair(&self, start_time: Instant) -> [f32; 2] { + let update_at = self + .update_at + .saturating_duration_since(start_time) + .as_secs_f32(); + let rtt = self.rtt.as_secs_f32(); + [update_at, rtt] + } + + /// Build from pair of f32 + fn from_pair(pair: [f32; 2], start_time: Instant) -> Self { + let update_at = start_time + Duration::from_secs_f32(pair[0]); + let rtt = Duration::from_secs_f32(pair[1]); + Self { update_at, rtt } + } +} + +/// Atomic storage of RttEstimate using AtomicF32Pair +/// +/// Start time is needed to (de-)serialize the update_at instance. +#[derive(Debug)] +pub struct AtomicRttEstimate { + pair: AtomicF32Pair, + start_time: Instant, +} + +impl AtomicRttEstimate { + /// Creates a new atomic rtt estimate. + pub fn new(start_duration: Duration) -> Self { + let estimate = RttEstimate::new(start_duration); + Self { + pair: AtomicF32Pair::new(estimate.as_pair(estimate.update_at)), + start_time: estimate.update_at, + } + } + + /// Loads a value from the atomic rtt estimate. + /// + /// This method omits the ordering argument since loads may use + /// slightly stale data to avoid adding additional latency. + pub fn load(&self) -> RttEstimate { + RttEstimate::from_pair(self.pair.load(Ordering::Relaxed), self.start_time) + } + + /// Fetches the value, and applies a function to it that returns an + /// new rtt. Retrns the new RttEstimate with new update_at. + /// + /// Automatically updates the update_at with Instant::now(). This + /// method omits ordering arguments, defaulting to Relaxed since + /// all writes are serial and any reads may rely on slightly stale + /// data. + pub fn fetch_update(&self, mut f: F) -> RttEstimate + where + F: FnMut(RttEstimate) -> Duration, + { + let mut update_at = Instant::now(); + let mut rtt = Duration::ZERO; + self.pair + .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |pair| { + rtt = f(RttEstimate::from_pair(pair, self.start_time)); + // Save the new update_at inside the function in case it + // is run multiple times + update_at = Instant::now(); + Some(RttEstimate { rtt, update_at }.as_pair(self.start_time)) + }) + .expect("Should never Err"); + RttEstimate { update_at, rtt } + } +} + +#[cfg(test)] +mod tests { + use tokio::time::{self, Duration, Instant}; + + use super::{AtomicRttEstimate, RttEstimate}; + + #[test] + fn test_rtt_estimate_f32_conversions() { + let rtt = Duration::from_secs(1); + let expected = RttEstimate::new(rtt); + let actual = + RttEstimate::from_pair(expected.as_pair(expected.update_at), expected.update_at); + assert_eq!(expected.update_at, actual.update_at); + assert_eq!(expected.rtt, actual.rtt); + } + + #[tokio::test(start_paused = true)] + async fn test_atomic_rtt_estimate_load() { + let rtt = Duration::from_secs(1); + let estimate = AtomicRttEstimate::new(rtt); + let actual = estimate.load(); + assert_eq!(Instant::now(), actual.update_at); + assert_eq!(rtt, actual.rtt); + } + + #[tokio::test(start_paused = true)] + async fn test_atomic_rtt_estimate_fetch_update() { + let start_time = Instant::now(); + let rtt = Duration::from_secs(1); + let estimate = AtomicRttEstimate::new(rtt); + time::advance(Duration::from_secs(1)).await; + let rv = estimate.fetch_update(|value| { + assert_eq!(start_time, value.update_at); + assert_eq!(rtt, value.rtt); + Duration::from_secs(2) + }); + assert_eq!(start_time + Duration::from_secs(1), rv.update_at); + assert_eq!(Duration::from_secs(2), rv.rtt); + } +} diff --git a/latency/src/util/atomic_f32_pair.rs b/latency/src/util/atomic_f32_pair.rs new file mode 100644 index 00000000..08ee3953 --- /dev/null +++ b/latency/src/util/atomic_f32_pair.rs @@ -0,0 +1,88 @@ +use std::sync::atomic::{AtomicU64, Ordering}; + +/// Implements an atomic pair of f32s +/// +/// This uses an AtomicU64 internally. +#[derive(Debug)] +pub struct AtomicF32Pair(AtomicU64); + +impl AtomicF32Pair { + /// Creates a new atomic pair. + pub fn new(pair: [f32; 2]) -> Self { + Self(AtomicU64::new(to_bits(pair))) + } + + /// Loads a value from the atomic pair. + pub fn load(&self, ordering: Ordering) -> [f32; 2] { + from_bits(self.0.load(ordering)) + } + + /// Fetches the value, and applies a function to it that returns an + /// optional new value. Returns a Result of Ok(previous_value) if + /// the function returned Some(_), else Err(previous_value). + pub fn fetch_update( + &self, + set_order: Ordering, + fetch_order: Ordering, + mut f: F, + ) -> Result<[f32; 2], [f32; 2]> + where + F: FnMut([f32; 2]) -> Option<[f32; 2]>, + { + self.0 + .fetch_update(set_order, fetch_order, |bits| { + f(from_bits(bits)).map(to_bits) + }) + .map(from_bits) + .map_err(from_bits) + } +} + +/// Convert a f32 pair to its bit-representation as u64 +fn to_bits(pair: [f32; 2]) -> u64 { + let f1 = pair[0].to_bits() as u64; + let f2 = pair[1].to_bits() as u64; + (f1 << 32) | f2 +} + +/// Build a f32 pair from its bit-representation as u64 +fn from_bits(bits: u64) -> [f32; 2] { + let f1 = f32::from_bits((bits >> 32) as u32); + let f2 = f32::from_bits(bits as u32); + [f1, f2] +} + +#[cfg(test)] +mod tests { + use std::sync::atomic::Ordering; + + use super::{from_bits, to_bits, AtomicF32Pair}; + + #[test] + fn test_f32_pair_bit_conversions() { + let pair = [3.14159, 2.71828]; + assert_eq!(pair, from_bits(to_bits(pair))); + } + + #[test] + fn test_atomic_f32_pair_load() { + let pair = [3.14159, 2.71828]; + let atomic = AtomicF32Pair::new(pair); + assert_eq!(pair, atomic.load(Ordering::Relaxed)); + } + + #[test] + fn test_atomic_f32_pair_fetch_update() { + let pair = [3.14159, 2.71828]; + let atomic = AtomicF32Pair::new(pair); + atomic + .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |[f1, f2]| { + Some([f1 + 1.0, f2 + 1.0]) + }) + .unwrap(); + assert_eq!( + [pair[0] + 1.0, pair[1] + 1.0], + atomic.load(Ordering::Relaxed) + ); + } +} diff --git a/latency/src/util/mod.rs b/latency/src/util/mod.rs new file mode 100644 index 00000000..531c7f56 --- /dev/null +++ b/latency/src/util/mod.rs @@ -0,0 +1,2 @@ +pub(crate) mod atomic_f32_pair; +pub(crate) mod nanos; diff --git a/latency/src/util/nanos.rs b/latency/src/util/nanos.rs new file mode 100644 index 00000000..d6947a70 --- /dev/null +++ b/latency/src/util/nanos.rs @@ -0,0 +1,30 @@ +use tokio::time::Duration; + +pub const NANOS_PER_MILLI: f64 = 1_000_000.0; + +/// Utility that converts durations to nanos in f64. +/// +/// Due to a lossy transformation, the maximum value that can be represented is ~585 years, +/// which, I hope, is more than enough to represent request latencies. +pub fn nanos(d: Duration) -> f64 { + const NANOS_PER_SEC: u64 = 1_000_000_000; + let n = f64::from(d.subsec_nanos()); + let s = d.as_secs().saturating_mul(NANOS_PER_SEC) as f64; + n + s +} + +#[cfg(test)] +mod tests { + use tokio::time::Duration; + + #[test] + fn nanos() { + assert_eq!(super::nanos(Duration::new(0, 0)), 0.0); + assert_eq!(super::nanos(Duration::new(0, 123)), 123.0); + assert_eq!(super::nanos(Duration::new(1, 23)), 1_000_000_023.0); + assert_eq!( + super::nanos(Duration::new(::std::u64::MAX, 999_999_999)), + 18446744074709553000.0 + ); + } +} diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index 1136ad26..0bd41831 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -15,6 +15,7 @@ rdkafka-src = ["rdkafka/cmake-build", "rdkafka/libz", "rdkafka/ssl", "rdkafka/zs [dependencies] deferred-rate-limiter = { path = "../deferred-rate-limiter" } entities = { path = "../entities" } +latency = { path = "../latency" } migration = { path = "../migration" } redis-rate-limiter = { path = "../redis-rate-limiter" } thread-fast-rng = { path = "../thread-fast-rng" } @@ -37,7 +38,6 @@ derive_more = "0.99.17" dotenv = "0.15.0" env_logger = "0.10.0" ethers = { version = "2.0.3", default-features = false, features = ["rustls", "ws"] } -ewma = "0.1.1" fdlimit = "0.2.1" flume = "0.10.14" fstrings = "0.2" diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index 8fce98df..3ee83fa3 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -1282,13 +1282,16 @@ fn rpc_sync_status_sort_key(x: &Arc) -> (Reverse, u64, bool, Order // TODO: use request latency instead of head latency // TODO: have the latency decay automatically - let head_ewma = x.head_latency.read().value(); + let peak_latency = x + .peak_latency + .as_ref() + .expect("peak_latency uniniialized") + .latency(); let active_requests = x.active_requests.load(atomic::Ordering::Relaxed) as f64; // TODO: i'm not sure head * active is exactly right. but we'll see - // TODO: i don't think this actually counts as peak. investigate with atomics.rs and peak_ewma.rs - let peak_ewma = OrderedFloat(head_ewma * active_requests); + let peak_ewma = OrderedFloat(peak_latency.as_millis() as f64 * (active_requests + 1.0)); let backup = x.backup; @@ -1296,18 +1299,26 @@ fn rpc_sync_status_sort_key(x: &Arc) -> (Reverse, u64, bool, Order } mod tests { - // TODO: why is this allow needed? does tokio::test get in the way somehow? #![allow(unused_imports)] + use std::time::{SystemTime, UNIX_EPOCH}; + use super::*; use crate::rpcs::consensus::ConsensusFinder; use crate::rpcs::{blockchain::Web3ProxyBlock, provider::Web3Provider}; + use ethers::types::{Block, U256}; + use latency::PeakEwmaLatency; use log::{trace, LevelFilter}; use parking_lot::RwLock; - use std::time::{SystemTime, UNIX_EPOCH}; use tokio::sync::RwLock as AsyncRwLock; + #[cfg(test)] + fn new_peak_latency() -> PeakEwmaLatency { + const NANOS_PER_MILLI: f64 = 1_000_000.0; + PeakEwmaLatency::spawn(1_000.0 * NANOS_PER_MILLI, 4, Duration::from_secs(1)) + } + #[tokio::test] async fn test_sort_connections_by_sync_status() { let block_0 = Block { @@ -1338,36 +1349,42 @@ mod tests { name: "a".to_string(), tier: 0, head_block: RwLock::new(None), + peak_latency: Some(new_peak_latency()), ..Default::default() }, Web3Rpc { name: "b".to_string(), tier: 0, head_block: RwLock::new(blocks.get(1).cloned()), + peak_latency: Some(new_peak_latency()), ..Default::default() }, Web3Rpc { name: "c".to_string(), tier: 0, head_block: RwLock::new(blocks.get(2).cloned()), + peak_latency: Some(new_peak_latency()), ..Default::default() }, Web3Rpc { name: "d".to_string(), tier: 1, head_block: RwLock::new(None), + peak_latency: Some(new_peak_latency()), ..Default::default() }, Web3Rpc { name: "e".to_string(), tier: 1, head_block: RwLock::new(blocks.get(1).cloned()), + peak_latency: Some(new_peak_latency()), ..Default::default() }, Web3Rpc { name: "f".to_string(), tier: 1, head_block: RwLock::new(blocks.get(2).cloned()), + peak_latency: Some(new_peak_latency()), ..Default::default() }, ] @@ -1425,6 +1442,7 @@ mod tests { tier: 0, head_block: RwLock::new(Some(head_block.clone())), provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), + peak_latency: Some(new_peak_latency()), ..Default::default() }; @@ -1437,6 +1455,7 @@ mod tests { tier: 0, head_block: RwLock::new(Some(lagged_block.clone())), provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), + peak_latency: Some(new_peak_latency()), ..Default::default() }; @@ -1781,6 +1800,7 @@ mod tests { tier: 0, head_block: RwLock::new(Some(block_1.clone())), provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), + peak_latency: Some(new_peak_latency()), ..Default::default() }; @@ -1793,6 +1813,7 @@ mod tests { tier: 1, head_block: RwLock::new(Some(block_2.clone())), provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), + peak_latency: Some(new_peak_latency()), ..Default::default() }; diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index 1b7246e2..bd92293d 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -12,6 +12,7 @@ use ethers::prelude::{Bytes, Middleware, ProviderError, TxHash, H256, U64}; use ethers::types::{Address, Transaction, U256}; use futures::future::try_join_all; use futures::StreamExt; +use latency::{EwmaLatency, PeakEwmaLatency}; use log::{debug, error, info, trace, warn, Level}; use migration::sea_orm::DatabaseConnection; use ordered_float::OrderedFloat; @@ -30,69 +31,6 @@ use thread_fast_rng::thread_fast_rng; use tokio::sync::{broadcast, oneshot, watch, RwLock as AsyncRwLock}; use tokio::time::{sleep, sleep_until, timeout, Duration, Instant}; -pub struct Latency { - /// exponentially weighted moving average of how many milliseconds behind the fastest node we are - ewma: ewma::EWMA, -} - -impl Serialize for Latency { - fn serialize(&self, serializer: S) -> Result - where - S: Serializer, - { - serializer.serialize_f64(self.ewma.value()) - } -} - -impl Latency { - #[inline(always)] - pub fn record(&mut self, duration: Duration) { - self.record_ms(duration.as_secs_f64() * 1000.0); - } - - #[inline(always)] - pub fn record_ms(&mut self, milliseconds: f64) { - self.ewma.add(milliseconds); - } - - #[inline(always)] - pub fn value(&self) -> f64 { - self.ewma.value() - } -} - -impl Default for Latency { - fn default() -> Self { - // TODO: what should the default span be? 25 requests? have a "new" - let span = 25.0; - - let start = 1000.0; - - Self::new(span, start) - } -} - -impl Latency { - // depending on the span, start might not be perfect - pub fn new(span: f64, start: f64) -> Self { - let alpha = Self::span_to_alpha(span); - - let mut ewma = ewma::EWMA::new(alpha); - - if start > 0.0 { - for _ in 0..(span as u64) { - ewma.add(start); - } - } - - Self { ewma } - } - - fn span_to_alpha(span: f64) -> f64 { - 2.0 / (span + 1.0) - } -} - /// An active connection to a Web3 RPC server like geth or erigon. #[derive(Default)] pub struct Web3Rpc { @@ -127,10 +65,11 @@ pub struct Web3Rpc { /// TODO: change this to a watch channel so that http providers can subscribe and take action on change. pub(super) head_block: RwLock>, /// Track head block latency - pub(super) head_latency: RwLock, - // /// Track request latency - // /// TODO: refactor this. this lock kills perf. for now just use head_latency - // pub(super) request_latency: RwLock, + pub(super) head_latency: RwLock, + /// Track peak request latency + /// + /// This is only inside an Option so that the "Default" derive works. it will always be set. + pub(super) peak_latency: Option, /// Track total requests served /// TODO: maybe move this to graphana pub(super) total_requests: AtomicUsize, @@ -227,6 +166,18 @@ impl Web3Rpc { let (disconnect_sender, disconnect_receiver) = watch::channel(false); let reconnect = reconnect.into(); + // Spawn the task for calculting average peak latency + // TODO Should these defaults be in config + let peak_latency = PeakEwmaLatency::spawn( + // Decay over 15s + Duration::from_secs(15).as_millis() as f64, + // Peak requests so far around 5k, we will use an order of magnitude + // more to be safe. Should only use about 50mb RAM + 50_000, + // Start latency at 1 second + Duration::from_secs(1), + ); + let new_connection = Self { name, db_conn: db_conn.clone(), @@ -245,6 +196,7 @@ impl Web3Rpc { disconnect_watch: Some(disconnect_sender), created_at: Some(created_at), head_block: RwLock::new(Default::default()), + peak_latency: Some(peak_latency), ..Default::default() }; diff --git a/web3_proxy/src/rpcs/request.rs b/web3_proxy/src/rpcs/request.rs index 4c6b2dbf..7620716d 100644 --- a/web3_proxy/src/rpcs/request.rs +++ b/web3_proxy/src/rpcs/request.rs @@ -194,7 +194,7 @@ impl OpenRequestHandle { .active_requests .fetch_add(1, std::sync::atomic::Ordering::Relaxed); - // let latency = Instant::now(); + let start = Instant::now(); // TODO: replace ethers-rs providers with our own that supports streaming the responses let response = match provider.as_ref() { @@ -366,14 +366,10 @@ impl OpenRequestHandle { tokio::spawn(f); } } + } else if let Some(peak_latency) = &self.rpc.peak_latency { + peak_latency.report(start.elapsed()); } else { - // TODO: record request latency - // let latency_ms = start.elapsed().as_secs_f64() * 1000.0; - - // TODO: is this lock here a problem? should this be done through a channel? i started to code it, but it didn't seem to matter - // let mut latency_recording = self.rpc.request_latency.write(); - - // latency_recording.record(latency_ms); + unreachable!("peak_latency not initialized"); } response From 6e8536d47757e15217f790b4f0874979e0016580 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Thu, 11 May 2023 13:07:31 -0700 Subject: [PATCH 03/66] add support for http basic auth --- TODO.md | 18 +++++------ web3_proxy/src/rpcs/one.rs | 28 ++++++++++++----- web3_proxy/src/rpcs/provider.rs | 53 ++++++++++++++++++++++++++------- 3 files changed, 72 insertions(+), 27 deletions(-) diff --git a/TODO.md b/TODO.md index 5417d951..b405b251 100644 --- a/TODO.md +++ b/TODO.md @@ -410,9 +410,9 @@ These are not yet ordered. There might be duplicates. We might not actually need - [-] add configurable size limits to all the Caches - instead of configuring each cache with MB sizes, have one value for total memory footprint and then percentages for each cache - https://github.com/moka-rs/moka/issues/201 -- [ ] all anyhow::Results need to be replaced with FrontendErrorResponse. - - [ ] rename FrontendErrorResponse to Web3ProxyError - - [ ] almost all the anyhows should be Web3ProxyError::BadRequest +- [x] all anyhow::Results need to be replaced with FrontendErrorResponse. + - [x] rename FrontendErrorResponse to Web3ProxyError + - [x] almost all the anyhows should be Web3ProxyError::BadRequest - as is, these errors are seen as 500 errors and so haproxy keeps retrying them - change premium concurrency limit to be against ip+rpckey - then sites like curve.fi don't have to worry about their user count @@ -422,18 +422,18 @@ These are not yet ordered. There might be duplicates. We might not actually need - all nodes have all blocks - most nodes have all receipts - only archives have old state -- [ ] don't use new_head_provider anywhere except new head subscription -- [ ] enable mev protected transactions with either a /protect/ url (instead of /private/) or the database (when on /rpc/) -- [-] have private transactions be enabled by a url setting rather than a setting on the key +- [x] don't use new_head_provider anywhere except new head subscription +- [x] add support for http basic auth +- [-] enable mev protected transactions with either a /protect/ url (instead of /private/) or the database (when on /rpc/) +- [ ] a **lot** got done that wasn't included in this todo list. go through commits and update this - [ ] eth_sendRawTransaction should only forward if the chain_id matches what we are running - [ ] cli for adding rpc keys to an existing user - [ ] rename "private" to "mev protected" to avoid confusion about private transactions being public once they are mined - [ ] allow restricting an rpc key to specific chains -- [ ] writes to request_latency should be handled by a background task so they don't slow down the request - - maybe we can use https://docs.rs/hdrhistogram/latest/hdrhistogram/sync/struct.SyncHistogram.html +- [-] writes to request_latency should be handled by a background task so they don't slow down the request - [ ] keep re-broadcasting transactions until they are confirmed - [ ] if mev protection is disabled, we should send to *both* balanced_rpcs *and* private_rps -- [ ] if mev protection is enabled, we should sent to *only* private_rpcs +- [x] if mev protection is enabled, we should sent to *only* private_rpcs - [ ] rate limiting/throttling on query_user_stats - [ ] web3rpc configs should have a max_concurrent_requests - will probably want a tool for calculating a safe value for this. too low and we could kill our performance diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index bd92293d..eed4884f 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -21,6 +21,7 @@ use redis_rate_limiter::{RedisPool, RedisRateLimitResult, RedisRateLimiter}; use serde::ser::{SerializeStruct, Serializer}; use serde::Serialize; use serde_json::json; +use std::borrow::Cow; use std::cmp::min; use std::fmt; use std::hash::{Hash, Hasher}; @@ -30,6 +31,7 @@ use thread_fast_rng::rand::Rng; use thread_fast_rng::thread_fast_rng; use tokio::sync::{broadcast, oneshot, watch, RwLock as AsyncRwLock}; use tokio::time::{sleep, sleep_until, timeout, Duration, Instant}; +use url::Url; /// An active connection to a Web3 RPC server like geth or erigon. #[derive(Default)] @@ -37,8 +39,8 @@ pub struct Web3Rpc { pub name: String, pub display_name: Option, pub db_conn: Option, - pub(super) ws_url: Option, - pub(super) http_url: Option, + pub(super) ws_url: Option, + pub(super) http_url: Option, /// Some connections use an http_client. we keep a clone for reconnecting pub(super) http_client: Option, /// provider is in a RwLock so that we can replace it if re-connecting @@ -178,13 +180,25 @@ impl Web3Rpc { Duration::from_secs(1), ); + let http_url = if let Some(http_url) = config.http_url { + Some(http_url.parse()?) + } else { + None + }; + + let ws_url = if let Some(ws_url) = config.ws_url { + Some(ws_url.parse()?) + } else { + None + }; + let new_connection = Self { name, db_conn: db_conn.clone(), display_name: config.display_name, http_client, - ws_url: config.ws_url, - http_url: config.http_url, + ws_url, + http_url, hard_limit, hard_limit_until, soft_limit: config.soft_limit, @@ -478,7 +492,7 @@ impl Web3Rpc { } } - let p = Web3Provider::from_str(ws_url.as_str(), None) + let p = Web3Provider::new(Cow::Borrowed(ws_url), None) .await .context(format!("failed connecting to {}", ws_url))?; @@ -488,7 +502,7 @@ impl Web3Rpc { } else { // http client if let Some(url) = &self.http_url { - let p = Web3Provider::from_str(url, self.http_client.clone()) + let p = Web3Provider::new(Cow::Borrowed(url), self.http_client.clone()) .await .context(format!("failed connecting to {}", url))?; @@ -1433,7 +1447,7 @@ mod tests { let x = Web3Rpc { name: "name".to_string(), - ws_url: Some("ws://example.com".to_string()), + ws_url: Some("ws://example.com".parse::().unwrap()), soft_limit: 1_000, automatic_block_limit: false, backup: false, diff --git a/web3_proxy/src/rpcs/provider.rs b/web3_proxy/src/rpcs/provider.rs index d5ab318e..a70bc88f 100644 --- a/web3_proxy/src/rpcs/provider.rs +++ b/web3_proxy/src/rpcs/provider.rs @@ -1,6 +1,8 @@ -use anyhow::Context; +use anyhow::anyhow; use derive_more::From; -use std::time::Duration; +use ethers::providers::{Authorization, ConnectionDetails}; +use std::{borrow::Cow, time::Duration}; +use url::Url; // TODO: our own structs for these that handle streaming large responses type EthersHttpProvider = ethers::providers::Provider; @@ -34,24 +36,53 @@ impl Web3Provider { } } - pub async fn from_str( - url_str: &str, + /// Note, if the http url has an authority the http_client param is ignored and a dedicated http_client will be used + /// TODO: take a reqwest::Client or a reqwest::ClientBuilder. that way we can do things like set compression even when auth is set + pub async fn new( + mut url: Cow<'_, Url>, http_client: Option, ) -> anyhow::Result { - let provider = if url_str.starts_with("http") { - let url: url::Url = url_str.parse()?; + let auth = if let Some(pass) = url.password().map(|x| x.to_string()) { + // to_string is needed because we are going to remove these items from the url + let user = url.username().to_string(); - let http_client = http_client.context("no http_client")?; + // clear username and password from the url + let mut_url = url.to_mut(); - let provider = ethers::providers::Http::new_with_client(url, http_client); + mut_url + .set_username("") + .map_err(|_| anyhow!("unable to clear username on websocket"))?; + mut_url + .set_password(None) + .map_err(|_| anyhow!("unable to clear password on websocket"))?; + + // keep them + Some(Authorization::basic(user, pass)) + } else { + None + }; + + let provider = if url.scheme().starts_with("http") { + let provider = if let Some(auth) = auth { + ethers::providers::Http::new_with_auth(url.into_owned(), auth)? + } else if let Some(http_client) = http_client { + ethers::providers::Http::new_with_client(url.into_owned(), http_client) + } else { + ethers::providers::Http::new(url.into_owned()) + }; - // TODO: dry this up (needs https://github.com/gakonst/ethers-rs/issues/592) // TODO: i don't think this interval matters for our uses, but we should probably set it to like `block time / 2` ethers::providers::Provider::new(provider) .interval(Duration::from_secs(12)) .into() - } else if url_str.starts_with("ws") { - let provider = ethers::providers::Ws::connect(url_str).await?; + } else if url.scheme().starts_with("ws") { + let provider = if auth.is_some() { + let connection_details = ConnectionDetails::new(url.as_str(), auth); + + ethers::providers::Ws::connect(connection_details).await? + } else { + ethers::providers::Ws::connect(url.as_str()).await? + }; // TODO: dry this up (needs https://github.com/gakonst/ethers-rs/issues/592) // TODO: i don't think this interval matters From d67d6232165b81e58b9bde5ef24e96c43da8ead8 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Thu, 11 May 2023 13:23:22 -0700 Subject: [PATCH 04/66] rustdocflags need to be the same as rustflags --- .cargo/config.toml | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/.cargo/config.toml b/.cargo/config.toml index f4ad2dbf..41beade0 100644 --- a/.cargo/config.toml +++ b/.cargo/config.toml @@ -5,4 +5,9 @@ rustflags = [ # tokio unstable is needed for tokio-console "--cfg", "tokio_unstable" ] -rustdocflags = ["--cfg", "tokio_unstable"] +rustdocflags = [ + # potentially faster. https://nnethercote.github.io/perf-book/build-configuration.html + "-C", "target-cpu=native", + # tokio unstable is needed for tokio-console + "--cfg", "tokio_unstable" +] From 2a242fe37fda45d923dbdc00597131d5a691b028 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Thu, 11 May 2023 13:25:55 -0700 Subject: [PATCH 05/66] cargo upgrade --incompatible --- Cargo.lock | 534 +++++++++++++++---------------- deferred-rate-limiter/Cargo.toml | 6 +- entities/Cargo.toml | 8 +- latency/Cargo.toml | 6 +- migration/Cargo.toml | 4 +- rate-counter/Cargo.toml | 2 +- redis-rate-limiter/Cargo.toml | 6 +- web3_proxy/Cargo.toml | 28 +- 8 files changed, 286 insertions(+), 308 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7a56dc18..5dea22fa 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -46,7 +46,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "433cfd6710c9986c576a25ca913c39d66a6474107b406f34f91d4a8923395241" dependencies = [ "cfg-if", - "cipher 0.4.3", + "cipher 0.4.4", "cpufeatures", ] @@ -74,9 +74,9 @@ dependencies = [ [[package]] name = "aho-corasick" -version = "0.7.20" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc936419f96fa211c1b9166887b38e5e40b19958e5b895be7c1f93adec7071ac" +checksum = "67fc08ce920c31afb70f013dcce1bfc3a3195de6a228474e45e1f145b36f8d04" dependencies = [ "memchr", ] @@ -107,9 +107,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.70" +version = "1.0.71" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7de8ce5e0f9f8d88245311066a578d72b7af3e7088f32783804676302df237e4" +checksum = "9c7d0618f0e0b7e8ff11427422b64564d5fb0be1940354bfe2e0529b18a9d9b8" dependencies = [ "backtrace", ] @@ -262,9 +262,9 @@ dependencies = [ [[package]] name = "auto_impl" -version = "1.0.1" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a8c1df849285fbacd587de7818cc7d13be6cd2cbcd47a04fb1801b0e2706e33" +checksum = "fee3da8ef1276b0bee5dd1c7258010d8fffd31801447323115a25560e1327b89" dependencies = [ "proc-macro-error", "proc-macro2", @@ -280,9 +280,9 @@ checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" [[package]] name = "axum" -version = "0.6.16" +version = "0.6.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "113713495a32dd0ab52baf5c10044725aa3aec00b31beda84218e469029b72a3" +checksum = "f8175979259124331c1d7bf6586ee7e0da434155e4b2d48ec2c8386281d8df39" dependencies = [ "async-trait", "axum-core", @@ -316,9 +316,9 @@ dependencies = [ [[package]] name = "axum-client-ip" -version = "0.4.0" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d719fabd6813392bbc10e1fe67f2977fad52791a836e51236f7e02f2482e017" +checksum = "df8e81eacc93f36480825da5f46a33b5fb2246ed024eacc9e8933425b80c5807" dependencies = [ "axum", "forwarded-header-value", @@ -408,9 +408,9 @@ checksum = "a4a4ddaa51a5bc52a6948f74c06d20aaaddb71924eab79b8c97a8c556e942d6a" [[package]] name = "base64ct" -version = "1.5.3" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b645a089122eccb6111b4f81cbc1a49f5900ac4666bb93ac027feaecf15607bf" +checksum = "8c3c1a368f70d6cf7302d78f8f7093da241fb8e8807c05cc9e51a125895a6d5b" [[package]] name = "bech32" @@ -492,9 +492,9 @@ dependencies = [ [[package]] name = "block-buffer" -version = "0.10.3" +version = "0.10.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69cce20737498f97b993470a6e536b8523f0af7892a4f928cceb1ac5e52ebe7e" +checksum = "3078c7629b62d3f0439517fa394996acacc5cbc91c5a20d8c658e77abd503a71" dependencies = [ "generic-array", ] @@ -555,9 +555,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.12.0" +version = "3.12.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d261e256854913907f67ed06efbc3338dfe6179796deefc1ff763fc1aee5535" +checksum = "3c6ed94e98ecff0c12dd1b04c15ec0d7d9458ca8fe806cea6f12954efe74c63b" [[package]] name = "byte-slice-cast" @@ -567,19 +567,20 @@ checksum = "c3ac9f8b63eca6fd385229b3675f6cc0dc5c8a5c8a54a59d4f52ffd670d87b0c" [[package]] name = "bytecheck" -version = "0.6.9" +version = "0.6.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d11cac2c12b5adc6570dad2ee1b87eff4955dac476fe12d81e5fdd352e52406f" +checksum = "13fe11640a23eb24562225322cd3e452b93a3d4091d62fab69c70542fcd17d1f" dependencies = [ "bytecheck_derive", "ptr_meta", + "simdutf8", ] [[package]] name = "bytecheck_derive" -version = "0.6.9" +version = "0.6.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13e576ebe98e605500b3c8041bb888e966653577172df6dd97398714eb30b9bf" +checksum = "e31225543cb46f81a7e224762764f4a6a0f097b1db0b175f69e8065efaa42de5" dependencies = [ "proc-macro2", "quote", @@ -630,9 +631,9 @@ dependencies = [ [[package]] name = "camino" -version = "1.1.2" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c77df041dc383319cc661b428b6961a005db4d6808d5e12536931b1ca9556055" +checksum = "c530edf18f37068ac2d977409ed5cd50d53d73bc653c7647b48eb78976ac9ae2" dependencies = [ "serde", ] @@ -721,9 +722,9 @@ dependencies = [ [[package]] name = "cipher" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d1873270f8f7942c191139cb8a40fd228da6c3fd2fc376d7e92d47aa14aeb59e" +checksum = "773f3b9af64447d2ce9850330c473515014aa235e6a783b02db81ff39e4a3dad" dependencies = [ "crypto-common", "inout", @@ -817,21 +818,19 @@ dependencies = [ [[package]] name = "coins-bip39" -version = "0.8.5" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad2a68a46b9d8cc90484f0689adc0e4c890eb215bf698ae52e5235bb88f40be7" +checksum = "84f4d04ee18e58356accd644896aeb2094ddeafb6a713e056cef0c0a8e468c15" dependencies = [ "bitvec 0.17.4", "coins-bip32", "getrandom", - "hex", "hmac", "once_cell", "pbkdf2 0.12.1", "rand", "sha2 0.10.6", "thiserror", - "tracing", ] [[package]] @@ -965,7 +964,7 @@ dependencies = [ "tonic", "tracing", "tracing-core", - "tracing-subscriber 0.3.16", + "tracing-subscriber 0.3.17", ] [[package]] @@ -1019,9 +1018,9 @@ dependencies = [ [[package]] name = "cpufeatures" -version = "0.2.5" +version = "0.2.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28d997bd5e24a5928dd43e46dc529867e207907fe0b239c3477d924f7f2ca320" +checksum = "3e4c1eaa2012c47becbbad2ab175484c2a84d1185b566fb2cc5b8707343dfe58" dependencies = [ "libc", ] @@ -1043,9 +1042,9 @@ checksum = "6548a0ad5d2549e111e1f6a11a6c2e2d00ce6a3dafe22948d67c2b443f775e52" [[package]] name = "crossbeam-channel" -version = "0.5.6" +version = "0.5.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2dd04ddaf88237dc3b8d8f9a3c1004b506b54b3313403944054d23c0870c521" +checksum = "a33c2bf77f2df06183c3aa30d1e96c0695a313d4f9c453cc3762a6db39f99200" dependencies = [ "cfg-if", "crossbeam-utils", @@ -1053,9 +1052,9 @@ dependencies = [ [[package]] name = "crossbeam-deque" -version = "0.8.2" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "715e8152b692bba2d374b53d4875445368fdf21a94751410af607a5ac677d1fc" +checksum = "ce6fd6f855243022dcecf8702fef0c297d4338e226845fe067f6341ad9fa0cef" dependencies = [ "cfg-if", "crossbeam-epoch", @@ -1064,9 +1063,9 @@ dependencies = [ [[package]] name = "crossbeam-epoch" -version = "0.9.13" +version = "0.9.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01a9af1f4c2ef74bb8aa1f7e19706bc72d03598c8a570bb5de72243c7a9d9d5a" +checksum = "46bd5f3f85273295a9d14aedfb86f6aadbff6d8f5295c4a9edb08e819dcf5695" dependencies = [ "autocfg", "cfg-if", @@ -1087,9 +1086,9 @@ dependencies = [ [[package]] name = "crossbeam-utils" -version = "0.8.14" +version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fb766fa798726286dbbb842f174001dab8abc7b627a1dd86e0b7222a95d929f" +checksum = "3c063cd8cc95f5c377ed0d4b49a4b21f632396ff690e8470c29b3359b346984b" dependencies = [ "cfg-if", ] @@ -1150,9 +1149,9 @@ dependencies = [ [[package]] name = "crypto-bigint" -version = "0.5.1" +version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c2538c4e68e52548bacb3e83ac549f903d44f011ac9d5abb5e132e67d0808f7" +checksum = "cf4c2f4e1afd912bc40bfd6fed5d9dc1f288e0ba01bfcc835cc5bc3eb13efe15" dependencies = [ "generic-array", "rand_core", @@ -1197,7 +1196,7 @@ version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0369ee1ad671834580515889b80f2ea915f23b8be8d0daa4bbaf2ac5c7590835" dependencies = [ - "cipher 0.4.3", + "cipher 0.4.4", ] [[package]] @@ -1270,9 +1269,9 @@ dependencies = [ [[package]] name = "deadpool-redis" -version = "0.11.1" +version = "0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b8bde44cbfdf17ae5baa45c9f43073b320f1a19955389315629304a23909ad2" +checksum = "5f1760f60ffc6653b4afd924c5792098d8c00d9a3deb6b3d989eac17949dc422" dependencies = [ "deadpool", "redis", @@ -1295,7 +1294,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bef552e6f588e446098f6ba40d89ac146c8c7b64aade83c051ee00bb5d2bc18d" dependencies = [ "serde", - "uuid 1.3.1", + "uuid 1.3.2", ] [[package]] @@ -1333,9 +1332,9 @@ dependencies = [ [[package]] name = "der" -version = "0.7.3" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "82b10af9f9f9f2134a42d3f8aa74658660f2e0234b0eb81bd171df8aa32779ed" +checksum = "05e58dffcdcc8ee7b22f0c1f71a69243d7c2d9ad87b5a14361f2424a1565c219" dependencies = [ "const-oid 0.9.2", "zeroize", @@ -1387,7 +1386,7 @@ version = "0.10.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8168378f4e5023e7218c89c891c0fd8ecdb5e5e4f18cb78f38cf245dd021e76f" dependencies = [ - "block-buffer 0.10.3", + "block-buffer 0.10.4", "const-oid 0.9.2", "crypto-common", "subtle", @@ -1454,9 +1453,9 @@ checksum = "03d8c417d7a8cb362e0c37e5d815f5eb7c37f79ff93707329d5a194e42e54ca0" [[package]] name = "dunce" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0bd4b30a6560bbd9b4620f4de34c3f14f60848e58a9b7216801afcb4c7b31c3c" +checksum = "56ce8c6da7551ec6c462cbaf3bfbc75131ebbfa1c944aeaa9dab51ca1c5f0c3b" [[package]] name = "ecdsa" @@ -1472,15 +1471,16 @@ dependencies = [ [[package]] name = "ecdsa" -version = "0.16.6" +version = "0.16.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a48e5d537b8a30c0b023116d981b16334be1485af7ca68db3a2b7024cbc957fd" +checksum = "0997c976637b606099b9985693efa3581e84e41f5c11ba5255f88711058ad428" dependencies = [ - "der 0.7.3", + "der 0.7.5", "digest 0.10.6", "elliptic-curve 0.13.4", "rfc6979 0.4.0", "signature 2.1.0", + "spki 0.7.2", ] [[package]] @@ -1515,23 +1515,23 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "75c71eaa367f2e5d556414a8eea812bc62985c879748d6403edabd9cb03f16e7" dependencies = [ "base16ct 0.2.0", - "crypto-bigint 0.5.1", + "crypto-bigint 0.5.2", "digest 0.10.6", "ff 0.13.0", "generic-array", "group 0.13.0", "pkcs8 0.10.2", "rand_core", - "sec1 0.7.1", + "sec1 0.7.2", "subtle", "zeroize", ] [[package]] name = "ena" -version = "0.14.0" +version = "0.14.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7402b94a93c24e742487327a7cd839dc9d36fec9de9fb25b09f2dae459f36c3" +checksum = "c533630cf40e9caa44bd91aadc88a75d75a4c3a12b4cfde353cbed41daa1e1f1" dependencies = [ "log", ] @@ -1559,9 +1559,9 @@ dependencies = [ [[package]] name = "enr" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb4d5fbf6f56acecd38f5988eb2e4ae412008a2a30268c748c701ec6322f39d4" +checksum = "cf56acd72bb22d2824e66ae8e9e5ada4d0de17a69c7fd35569dde2ada8ec9116" dependencies = [ "base64 0.13.1", "bytes", @@ -1583,7 +1583,7 @@ dependencies = [ "sea-orm", "serde", "ulid", - "uuid 1.3.1", + "uuid 1.3.2", ] [[package]] @@ -1712,9 +1712,9 @@ dependencies = [ [[package]] name = "ethers" -version = "2.0.3" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "356eaf2e3947efa975c3f953e33f73193e5e21b9d8bab26c3ca532676931696f" +checksum = "8d5486fdc149826f38c388f26a7df72534ee3f20d3a3f72539376fa7b3bbc43d" dependencies = [ "ethers-addressbook", "ethers-contract", @@ -1728,9 +1728,9 @@ dependencies = [ [[package]] name = "ethers-addressbook" -version = "2.0.3" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d56f1c92cb7c406794c8324158da4caf9e54018e28b776df8155085e1d06db75" +checksum = "1c66a426b824a0f6d1361ad74b6b01adfd26c44ee1e14c3662dcf28406763ec5" dependencies = [ "ethers-core", "once_cell", @@ -1740,9 +1740,9 @@ dependencies = [ [[package]] name = "ethers-contract" -version = "2.0.3" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a328cb42014ac0ac577a8dac32eb658ee0f32b5a9a5317a0329ac1d4201f1c6" +checksum = "dfa43e2e69632492d7b38e59465d125a0066cf4c477390ece00d3acbd11b338b" dependencies = [ "ethers-contract-abigen", "ethers-contract-derive", @@ -1759,9 +1759,9 @@ dependencies = [ [[package]] name = "ethers-contract-abigen" -version = "2.0.3" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d96813e4b58b6c6b817367380db900dddbd67bfe27610ec89fd3263778d5a4aa" +checksum = "2edb8fdbf77459819a443234b461171a024476bfc12f1853b889a62c6e1185ff" dependencies = [ "Inflector", "dunce", @@ -1786,9 +1786,9 @@ dependencies = [ [[package]] name = "ethers-contract-derive" -version = "2.0.3" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "373068bb24b4dea8fe0d1758aadab2dd4ec9de1d2c28316439cadcda3ed48eae" +checksum = "939b0c37746929f869285ee37d270b7c998d80cc7404c2e20dda8efe93e3b295" dependencies = [ "Inflector", "ethers-contract-abigen", @@ -1802,9 +1802,9 @@ dependencies = [ [[package]] name = "ethers-core" -version = "2.0.3" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3a5f8f85ba96698eab9a4782ed2215d0979b1981b99f1be0726c200ffdac22f5" +checksum = "198ea9efa8480fa69f73d31d41b1601dace13d053c6fe4be6f5878d9dfcf0108" dependencies = [ "arrayvec", "bytes", @@ -1833,9 +1833,9 @@ dependencies = [ [[package]] name = "ethers-etherscan" -version = "2.0.3" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7cfc3d36be9e16bac241e1d59ec9ace00e8e4241c09f604a0f65158eb37d4878" +checksum = "196a21d6939ab78b7a1e4c45c2b33b0c2dd821a2e1af7c896f06721e1ba2a0c7" dependencies = [ "ethers-core", "ethers-solc", @@ -1850,9 +1850,9 @@ dependencies = [ [[package]] name = "ethers-middleware" -version = "2.0.3" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e6f3543b4f6679b2558901c4b323cecb06b19239439d2588fa8b489bac9675d" +checksum = "75594cc450992fc7de701c9145de612325fd8a18be765b8ae78767ba2b74876f" dependencies = [ "async-trait", "auto_impl", @@ -1877,9 +1877,9 @@ dependencies = [ [[package]] name = "ethers-providers" -version = "2.0.3" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c9d2cbed43cf618004dbe339e389e10dae46ea8e55872ab63a25fad25a6082a" +checksum = "1009041f40476b972b5d79346cc512e97c662b1a0a2f78285eabe9a122909783" dependencies = [ "async-trait", "auto_impl", @@ -1915,9 +1915,9 @@ dependencies = [ [[package]] name = "ethers-signers" -version = "2.0.3" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d56871f70b97cc5cced5c39c88a0196c206d8f7fb8e1e0952fbf1fb73c033219" +checksum = "c3bd11ad6929f01f01be74bb00d02bbd6552f22de030865c898b340a3a592db1" dependencies = [ "async-trait", "coins-bip32", @@ -1934,9 +1934,9 @@ dependencies = [ [[package]] name = "ethers-solc" -version = "2.0.3" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e967e60bca8fc83d640a7aebd1492d625c43e86b88a5a5bb08b1019472d5d6b" +checksum = "2284784306de73d8ad1bc792ecc1b87da0268185683698d60fd096d23d168c99" dependencies = [ "cfg-if", "dunce", @@ -2401,9 +2401,9 @@ dependencies = [ [[package]] name = "handlebars" -version = "4.3.6" +version = "4.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "035ef95d03713f2c347a72547b7cd38cbc9af7cd51e6099fb62d586d4a6dee3a" +checksum = "83c3372087601b532857d332f5957cbae686da52bb7810bf038c3e3c3cc2fa0d" dependencies = [ "log", "pest", @@ -2570,11 +2570,11 @@ dependencies = [ [[package]] name = "home" -version = "0.5.4" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "747309b4b440c06d57b0b25f2aee03ee9b5e5397d288c60e21fc709bb98a7408" +checksum = "5444c27eef6923071f7ebcc33e3444508466a76f7a2b93da00ed6e19f30c1ddb" dependencies = [ - "winapi", + "windows-sys 0.48.0", ] [[package]] @@ -2945,9 +2945,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.61" +version = "0.3.62" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "445dde2150c55e483f3d8416706b97ec8e8237c307e5b7b4b8dd15e6af2a0730" +checksum = "68c16e1bfd491478ab155fd8b4896b86f9ede344949b641e61501e07c2b8b4d5" dependencies = [ "wasm-bindgen", ] @@ -2972,7 +2972,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cadb76004ed8e97623117f3df85b17aaa6626ab0b0831e6573f104df16cd1bcc" dependencies = [ "cfg-if", - "ecdsa 0.16.6", + "ecdsa 0.16.7", "elliptic-curve 0.13.4", "once_cell", "sha2 0.10.6", @@ -2981,30 +2981,29 @@ dependencies = [ [[package]] name = "keccak" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3afef3b6eff9ce9d8ff9b3601125eec7f0c8cbac7abd14f355d053fa56c98768" +checksum = "8f6d5ed8676d904364de097082f4e7d240b571b67989ced0240f08b7f966f940" dependencies = [ "cpufeatures", ] [[package]] name = "lalrpop" -version = "0.19.8" +version = "0.19.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b30455341b0e18f276fa64540aff54deafb54c589de6aca68659c63dd2d5d823" +checksum = "0a1cbf952127589f2851ab2046af368fd20645491bb4b376f04b7f94d7a9837b" dependencies = [ "ascii-canvas", - "atty", "bit-set", "diff", "ena", + "is-terminal", "itertools", "lalrpop-util", "petgraph", - "pico-args", "regex", - "regex-syntax", + "regex-syntax 0.6.29", "string_cache", "term", "tiny-keccak", @@ -3013,9 +3012,9 @@ dependencies = [ [[package]] name = "lalrpop-util" -version = "0.19.8" +version = "0.19.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bcf796c978e9b4d983414f4caedc9273aa33ee214c5b887bd55fde84c85d2dc4" +checksum = "d3c48237b9604c5a4702de6b824e02006c3214327564636aef27c1028a8fa0ed" dependencies = [ "regex", ] @@ -3041,9 +3040,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.141" +version = "0.2.144" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3304a64d199bb964be99741b7a14d26972741915b3649639149b2479bb46f4b5" +checksum = "2b00cc1c228a6782d0f076e7b232802e0c5689d41bb5df366f2a6b6621cfdfe1" [[package]] name = "libm" @@ -3080,9 +3079,9 @@ checksum = "f051f77a7c8e6957c0696eac88f26b0117e54f52d3fc682ab19397a8812846a4" [[package]] name = "linux-raw-sys" -version = "0.3.3" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b085a4f2cde5781fc4b1717f2e86c62f5cda49de7ba99a7c2eae02b61c9064c" +checksum = "ece97ea872ece730aed82664c424eb4c8291e1ff2480247ccf7409044bc6479f" [[package]] name = "lock_api" @@ -3150,9 +3149,9 @@ checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" [[package]] name = "memoffset" -version = "0.7.1" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5de893c32cde5f383baa4c04c5d6dbdd735cfd4a794b0debdb2bb1b421da5ff4" +checksum = "d61c719bcfbcf5d62b3a09efa6088de8c54bc0bfcd3ea7ae39fcc186108b8de1" dependencies = [ "autocfg", ] @@ -3222,9 +3221,9 @@ dependencies = [ [[package]] name = "moka" -version = "0.10.2" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0d3b8e76a2e4b17de765db9432e377a171c42fbe0512b8bc860ff1bfe2e273b" +checksum = "934030d03f6191edbb4ba16835ccdb80d560788ac686570a8e2986a0fb59ded8" dependencies = [ "async-io", "async-lock", @@ -3242,7 +3241,7 @@ dependencies = [ "tagptr", "thiserror", "triomphe", - "uuid 1.3.1", + "uuid 1.3.2", ] [[package]] @@ -3288,15 +3287,6 @@ dependencies = [ "minimal-lexical", ] -[[package]] -name = "nom8" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae01545c9c7fc4486ab7debaf2aad7003ac19431791868fb2e8066df97fad2f8" -dependencies = [ - "memchr", -] - [[package]] name = "nonempty" version = "0.7.0" @@ -3450,7 +3440,7 @@ version = "0.5.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dcbff9bc912032c62bf65ef1d5aea88983b420f4f839db1e9b0c281a25c9c799" dependencies = [ - "proc-macro-crate 1.3.0", + "proc-macro-crate 1.3.1", "proc-macro2", "quote", "syn 1.0.109", @@ -3462,7 +3452,7 @@ version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "96667db765a921f7b295ffee8b60472b686a51d4f21c2ee4ffdb94c7013b65a6" dependencies = [ - "proc-macro-crate 1.3.0", + "proc-macro-crate 1.3.1", "proc-macro2", "quote", "syn 2.0.15", @@ -3522,9 +3512,9 @@ dependencies = [ [[package]] name = "openssl" -version = "0.10.45" +version = "0.10.52" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b102428fd03bc5edf97f62620f7298614c45cedf287c271e7ed450bbaf83f2e1" +checksum = "01b8574602df80f7b85fdfc5392fa884a4e3b3f4f35402c070ab34c3d3f78d56" dependencies = [ "bitflags", "cfg-if", @@ -3537,13 +3527,13 @@ dependencies = [ [[package]] name = "openssl-macros" -version = "0.1.0" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b501e44f11665960c7e7fcf062c7d96a14ade4aa98116c004b2e37b5be7d736c" +checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" dependencies = [ "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.15", ] [[package]] @@ -3554,9 +3544,9 @@ checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" [[package]] name = "openssl-sys" -version = "0.9.85" +version = "0.9.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d3d193fb1488ad46ffe3aaabc912cc931d02ee8518fe2959aea8ef52718b0c0" +checksum = "8e17f59264b2809d77ae94f0e1ebabc434773f370d6ca667bd223ea10e06cc7e" dependencies = [ "cc", "libc", @@ -3586,9 +3576,9 @@ dependencies = [ [[package]] name = "ordered-float" -version = "3.6.0" +version = "3.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13a384337e997e6860ffbaa83708b2ef329fd8c54cb67a5f64d421e0f943254f" +checksum = "2fc2dbde8f8a79f2102cc474ceb0ad68e3b80b85289ea62389b60e66777e4213" dependencies = [ "num-traits", ] @@ -3606,9 +3596,9 @@ dependencies = [ [[package]] name = "os_str_bytes" -version = "6.4.1" +version = "6.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b7820b9daea5457c9f21c69448905d723fbd21136ccf521748f23fd49e723ee" +checksum = "ceedf44fb00f2d1984b0bc98102627ce622e083e49a5bacdb3e514fa4238e267" [[package]] name = "ouroboros" @@ -3653,15 +3643,15 @@ checksum = "bd10bab2b6df910bbe6c4987d76aa4221235103d9a9c000cfabcee6a6abc8f7a" dependencies = [ "reqwest", "serde", - "time 0.3.20", + "time 0.3.21", "url", ] [[package]] name = "parity-scale-codec" -version = "3.4.0" +version = "3.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "637935964ff85a605d114591d4d2c13c5d1ba2806dae97cea6bf180238a749ac" +checksum = "5ddb756ca205bd108aee3c62c6d3c994e1df84a59b9d6d4a5ea42ee1fd5a9a28" dependencies = [ "arrayvec", "bitvec 1.0.1", @@ -3677,7 +3667,7 @@ version = "3.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "86b26a931f824dd4eca30b3e43bb4f31cd5f0d3a403c5f5ff27106b805bfde7b" dependencies = [ - "proc-macro-crate 1.3.0", + "proc-macro-crate 1.3.1", "proc-macro2", "quote", "syn 1.0.109", @@ -3915,12 +3905,6 @@ dependencies = [ "siphasher", ] -[[package]] -name = "pico-args" -version = "0.4.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db8bcd96cb740d03149cbad5518db9fd87126a10ab519c011893b1754134c468" - [[package]] name = "pin-project" version = "1.0.12" @@ -3991,8 +3975,8 @@ version = "0.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f950b2377845cebe5cf8b5165cb3cc1a5e0fa5cfa3e1f7f55707d8fd82e0a7b7" dependencies = [ - "der 0.7.3", - "spki 0.7.1", + "der 0.7.5", + "spki 0.7.2", ] [[package]] @@ -4076,12 +4060,12 @@ dependencies = [ [[package]] name = "proc-macro-crate" -version = "1.3.0" +version = "1.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "66618389e4ec1c7afe67d51a9bf34ff9236480f8d51e7489b7d5ab0303c13f34" +checksum = "7f4c021e1093a56626774e81216a4ce732a735e5bad4868a03f3ed65ca0c3919" dependencies = [ "once_cell", - "toml_edit 0.18.1", + "toml_edit", ] [[package]] @@ -4199,9 +4183,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.26" +version = "1.0.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4424af4bf778aae2051a77b60283332f386554255d722233d09fbfc7e30da2fc" +checksum = "8f4f29d145265ec1c483c7c654450edde0bfe043d3938d6972630663356d9500" dependencies = [ "proc-macro2", ] @@ -4322,9 +4306,9 @@ dependencies = [ [[package]] name = "redis" -version = "0.22.3" +version = "0.23.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aa8455fa3621f6b41c514946de66ea0531f57ca017b2e6c7cc368035ea5b46df" +checksum = "3ea8c51b5dc1d8e5fd3350ec8167f464ec0995e79f2e90a075b63371500d557f" dependencies = [ "async-trait", "bytes", @@ -4380,13 +4364,13 @@ dependencies = [ [[package]] name = "regex" -version = "1.7.3" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b1f693b24f6ac912f4893ef08244d70b6067480d2f1a46e950c9691e6749d1d" +checksum = "af83e617f331cc6ae2da5443c602dfa5af81e517212d9d611a5b3ba1777b5370" dependencies = [ "aho-corasick", "memchr", - "regex-syntax", + "regex-syntax 0.7.1", ] [[package]] @@ -4395,7 +4379,7 @@ version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6c230d73fb8d8c1b9c0b3135c5142a8acee3a0558fb8db5cf1cb65f8d7862132" dependencies = [ - "regex-syntax", + "regex-syntax 0.6.29", ] [[package]] @@ -4404,6 +4388,12 @@ version = "0.6.29" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f162c6dd7b008981e4d40210aca20b4bd0f9b60ca9271061b07f78537722f2e1" +[[package]] +name = "regex-syntax" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a5996294f19bd3aae0453a862ad728f60e6600695733dd5df01da90c54363a3c" + [[package]] name = "rend" version = "0.4.0" @@ -4415,9 +4405,9 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.11.16" +version = "0.11.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "27b71749df584b7f4cac2c426c127a7c785a5106cc98f7a8feb044115f0fa254" +checksum = "13293b639a097af28fc8a90f22add145a9c954e49d77da06263d58cf44d5fb91" dependencies = [ "base64 0.21.0", "bytes", @@ -4658,15 +4648,15 @@ dependencies = [ [[package]] name = "rustix" -version = "0.37.12" +version = "0.37.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "722529a737f5a942fdbac3a46cee213053196737c5eaa3386d52e85b786f2659" +checksum = "acf8729d8542766f1b2cf77eb034d52f40d375bb8b615d0b147089946e16613d" dependencies = [ "bitflags", "errno 0.3.1", "io-lifetimes", "libc", - "linux-raw-sys 0.3.3", + "linux-raw-sys 0.3.7", "windows-sys 0.48.0", ] @@ -4693,9 +4683,9 @@ dependencies = [ [[package]] name = "rustversion" -version = "1.0.11" +version = "1.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5583e89e108996506031660fe09baa5011b9dd0341b89029313006d1fb508d70" +checksum = "4f3208ce4d8448b3f3e7d168a73f5e0c43a61e32930de3bceeccedb388b6bf06" [[package]] name = "ryu" @@ -4709,7 +4699,7 @@ version = "0.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "97a22f5af31f73a954c10289c93e8a50cc23d971e80ee446f1f6f7137a088213" dependencies = [ - "cipher 0.4.3", + "cipher 0.4.4", ] [[package]] @@ -4723,9 +4713,9 @@ dependencies = [ [[package]] name = "scale-info" -version = "2.3.1" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "001cf62ece89779fd16105b5f515ad0e5cedcd5440d3dd806bb067978e7c3608" +checksum = "dfdef77228a4c05dc94211441595746732131ad7f6530c6c18f045da7b7ab937" dependencies = [ "cfg-if", "derive_more", @@ -4735,11 +4725,11 @@ dependencies = [ [[package]] name = "scale-info-derive" -version = "2.3.1" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "303959cf613a6f6efd19ed4b4ad5bf79966a13352716299ad532cfb115f4205c" +checksum = "53012eae69e5aa5c14671942a5dd47de59d4cdcff8532a6dd0e081faf1119482" dependencies = [ - "proc-macro-crate 1.3.0", + "proc-macro-crate 1.3.1", "proc-macro2", "quote", "syn 1.0.109", @@ -4805,9 +4795,9 @@ dependencies = [ [[package]] name = "sea-orm" -version = "0.11.2" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5d875e2fcd965320e50066028ac0b4877ff07edbb734a6ddfeff48a87dbab38" +checksum = "fade86e8d41fd1a4721f84cb834f4ca2783f973cc30e6212b7fafc134f169214" dependencies = [ "async-stream", "async-trait", @@ -4825,17 +4815,17 @@ dependencies = [ "serde_json", "sqlx", "thiserror", - "time 0.3.20", + "time 0.3.21", "tracing", "url", - "uuid 1.3.1", + "uuid 1.3.2", ] [[package]] name = "sea-orm-cli" -version = "0.11.2" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ead9f7dac975f10447f17d08edbb2046daa087b5e0b50bbf8211f303459078c" +checksum = "efbf34a2caf70c2e3be9bb1e674e9540f6dfd7c8f40f6f05daf3b9740e476005" dependencies = [ "chrono", "clap", @@ -4843,15 +4833,15 @@ dependencies = [ "regex", "sea-schema", "tracing", - "tracing-subscriber 0.3.16", + "tracing-subscriber 0.3.17", "url", ] [[package]] name = "sea-orm-macros" -version = "0.11.2" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9b593e9c0cdbb18cafd4da7b92e67a9c2d9892934f3a2d8bbac73d5ba4a98a1" +checksum = "28936f26d62234ff0be16f80115dbdeb3237fe9c25cf18fbcd1e3b3592360f20" dependencies = [ "bae", "heck 0.3.3", @@ -4862,9 +4852,9 @@ dependencies = [ [[package]] name = "sea-orm-migration" -version = "0.11.2" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edba7a6123c1035b0530deb713820688f0234431ab6c1893b14dce493ade76af" +checksum = "278d3adfd0832b6ffc17d3cfbc574d3695a5c1b38814e0bc8ac238d33f3d87cf" dependencies = [ "async-trait", "clap", @@ -4874,7 +4864,7 @@ dependencies = [ "sea-orm-cli", "sea-schema", "tracing", - "tracing-subscriber 0.3.16", + "tracing-subscriber 0.3.17", ] [[package]] @@ -4888,8 +4878,8 @@ dependencies = [ "rust_decimal", "sea-query-derive", "serde_json", - "time 0.3.20", - "uuid 1.3.1", + "time 0.3.21", + "uuid 1.3.2", ] [[package]] @@ -4904,8 +4894,8 @@ dependencies = [ "sea-query", "serde_json", "sqlx", - "time 0.3.20", - "uuid 1.3.1", + "time 0.3.21", + "uuid 1.3.2", ] [[package]] @@ -4988,12 +4978,12 @@ dependencies = [ [[package]] name = "sec1" -version = "0.7.1" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48518a2b5775ba8ca5b46596aae011caa431e6ce7e4a67ead66d92f08884220e" +checksum = "f0aec48e813d6b90b15f0b8948af3c63483992dee44c03e9930b3eebdabe046e" dependencies = [ "base16ct 0.2.0", - "der 0.7.3", + "der 0.7.5", "generic-array", "pkcs8 0.10.2", "subtle", @@ -5146,25 +5136,25 @@ dependencies = [ "serde", "serde_json", "thiserror", - "time 0.3.20", + "time 0.3.21", "url", - "uuid 1.3.1", + "uuid 1.3.2", ] [[package]] name = "serde" -version = "1.0.160" +version = "1.0.163" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb2f3770c8bce3bcda7e149193a069a0f4365bda1fa5cd88e03bca26afc1216c" +checksum = "2113ab51b87a539ae008b5c6c02dc020ffa39afd2d83cffcb3f4eb2722cebec2" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.160" +version = "1.0.163" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "291a097c63d8497e00160b166a967a4a79c64f3facdd01cbd7502231688d77df" +checksum = "8c805777e3930c8883389c602315a24224bcc738b63905ef87cd1420353ea93e" dependencies = [ "proc-macro2", "quote", @@ -5184,9 +5174,9 @@ dependencies = [ [[package]] name = "serde_path_to_error" -version = "0.1.9" +version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26b04f22b563c91331a10074bda3dd5492e3cc39d56bd557e91c0af42b6c7341" +checksum = "f7f05c1d5476066defcdfacce1f52fc3cae3af1d3089727100c02ae92e5abbe0" dependencies = [ "serde", ] @@ -5282,9 +5272,9 @@ dependencies = [ [[package]] name = "sha3" -version = "0.10.6" +version = "0.10.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bdf0c33fae925bdc080598b84bc15c55e7b9a4a43b3c704da051f977469691c9" +checksum = "75872d278a8f37ef87fa0ddbda7802605cb18344497949862c0d4dcb291eba60" dependencies = [ "digest 0.10.6", "keccak", @@ -5349,6 +5339,12 @@ dependencies = [ "rand_core", ] +[[package]] +name = "simdutf8" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f27f6278552951f1f2b8cf9da965d10969b2efdea95a6ec47987ab46edfe263a" + [[package]] name = "siphasher" version = "0.3.10" @@ -5368,7 +5364,7 @@ dependencies = [ "rand", "sha3", "thiserror", - "time 0.3.20", + "time 0.3.21", ] [[package]] @@ -5483,12 +5479,12 @@ dependencies = [ [[package]] name = "spki" -version = "0.7.1" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37a5be806ab6f127c3da44b7378837ebf01dadca8510a0e572460216b228bd0e" +checksum = "9d1e996ef02c474957d681f1b05213dfb0abab947b446a62d37770b23500184a" dependencies = [ "base64ct", - "der 0.7.3", + "der 0.7.5", ] [[package]] @@ -5560,10 +5556,10 @@ dependencies = [ "sqlx-rt", "stringprep", "thiserror", - "time 0.3.20", + "time 0.3.21", "tokio-stream", "url", - "uuid 1.3.1", + "uuid 1.3.2", "webpki-roots", ] @@ -5611,9 +5607,9 @@ checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" [[package]] name = "string_cache" -version = "0.8.4" +version = "0.8.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "213494b7a2b503146286049378ce02b482200519accc31872ee8be91fa820a08" +checksum = "f91138e76242f575eb1d3b38b4f1362f10d3a43f47d182a5b359af488a02293b" dependencies = [ "new_debug_unreachable", "once_cell", @@ -5668,9 +5664,9 @@ checksum = "6bdef32e8150c2a081110b42772ffe7d7c9032b606bc226c8260fd97e0976601" [[package]] name = "svm-rs" -version = "0.2.20" +version = "0.2.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1b8e811a6443e8d93665a5e532efa8429ea8e2052a234a82e2cd69478913310" +checksum = "01afefe60c02f4a2271fb15d1965c37856712cebb338330b06649d12afec42df" dependencies = [ "anyhow", "cfg-if", @@ -5746,7 +5742,7 @@ dependencies = [ "cfg-if", "fastrand", "redox_syscall 0.3.5", - "rustix 0.37.12", + "rustix 0.37.19", "windows-sys 0.45.0", ] @@ -5848,9 +5844,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.20" +version = "0.3.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd0cbfecb4d19b5ea75bb31ad904eb5b9fa13f21079c3b92017ebdf4999a5890" +checksum = "8f3403384eaacbca9923fa06940178ac13e4edb725486d70e8e15881d0c836cc" dependencies = [ "itoa", "serde", @@ -5860,15 +5856,15 @@ dependencies = [ [[package]] name = "time-core" -version = "0.1.0" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e153e1f1acaef8acc537e68b44906d2db6436e2b35ac2c6b42640fff91f00fd" +checksum = "7300fbefb4dadc1af235a9cef3737cea692a9d97e1b9cbcd4ebdae6f8868e6fb" [[package]] name = "time-macros" -version = "0.2.8" +version = "0.2.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd80a657e71da814b8e5d60d3374fc6d35045062245d80224748ae522dd76f36" +checksum = "372950940a5f07bf38dbe211d7283c9e6d7327df53794992d293e534c733d09b" dependencies = [ "time-core", ] @@ -5899,9 +5895,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.27.0" +version = "1.28.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0de47a4eecbe11f498978a9b29d792f0d2692d1dd003650c24c76510e3bc001" +checksum = "0aa32867d44e6f2ce3385e89dceb990188b8bb0fb25b0cf576647a6f98ac5105" dependencies = [ "autocfg", "bytes", @@ -5914,7 +5910,7 @@ dependencies = [ "socket2", "tokio-macros", "tracing", - "windows-sys 0.45.0", + "windows-sys 0.48.0", ] [[package]] @@ -5942,7 +5938,7 @@ dependencies = [ "toml 0.5.11", "tonic", "tracing", - "tracing-subscriber 0.3.16", + "tracing-subscriber 0.3.17", "tui", ] @@ -5958,9 +5954,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61a573bdc87985e9d6ddeed1b3d864e8a302c847e40d647746df2f1de209d1ce" +checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" dependencies = [ "proc-macro2", "quote", @@ -5990,9 +5986,9 @@ dependencies = [ [[package]] name = "tokio-stream" -version = "0.1.12" +version = "0.1.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8fb52b74f05dbf495a8fba459fdc331812b96aa086d9eb78101fa0d4569c3313" +checksum = "397c988d37662c7dda6d2208364a706264bf3d6138b11d436cbac0ad38832842" dependencies = [ "futures-core", "pin-project-lite", @@ -6032,9 +6028,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.7" +version = "0.7.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5427d89453009325de0d8f342c9490009f76e999cb7672d77e46267448f7e6b2" +checksum = "806fe8c2c87eccc8b3267cbae29ed3ab2d0bd37fca70ab622e46aaa9375ddb7d" dependencies = [ "bytes", "futures-core", @@ -6061,16 +6057,10 @@ checksum = "b403acf6f2bb0859c93c7f0d967cb4a75a7ac552100f9322faf64dc047669b21" dependencies = [ "serde", "serde_spanned", - "toml_datetime 0.6.1", - "toml_edit 0.19.8", + "toml_datetime", + "toml_edit", ] -[[package]] -name = "toml_datetime" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4553f467ac8e3d374bc9a177a26801e5d0f9b211aa1673fb137a403afd1c9cf5" - [[package]] name = "toml_datetime" version = "0.6.1" @@ -6080,17 +6070,6 @@ dependencies = [ "serde", ] -[[package]] -name = "toml_edit" -version = "0.18.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56c59d8dd7d0dcbc6428bf7aa2f0e823e26e43b3c9aca15bbc9475d23e5fa12b" -dependencies = [ - "indexmap", - "nom8", - "toml_datetime 0.5.1", -] - [[package]] name = "toml_edit" version = "0.19.8" @@ -6100,7 +6079,7 @@ dependencies = [ "indexmap", "serde", "serde_spanned", - "toml_datetime 0.6.1", + "toml_datetime", "winnow", ] @@ -6201,13 +6180,13 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.23" +version = "0.1.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4017f8f45139870ca7e672686113917c71c7a6e02d4924eda67186083c03081a" +checksum = "0f57e3ca2a01450b1a921183a9c9cbfda207fd822cef4ccb00a65402cbba7a74" dependencies = [ "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.15", ] [[package]] @@ -6285,9 +6264,9 @@ dependencies = [ [[package]] name = "tracing-subscriber" -version = "0.3.16" +version = "0.3.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a6176eae26dd70d0c919749377897b54a9276bd7061339665dd68777926b5a70" +checksum = "30a651bc37f915e81f087d86e62a18eec5f79550c7faff886f7090b4ea757c77" dependencies = [ "matchers 0.1.0", "nu-ansi-term", @@ -6495,9 +6474,9 @@ dependencies = [ [[package]] name = "uuid" -version = "1.3.1" +version = "1.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b55a3fef2a1e3b3a00ce878640918820d3c51081576ac657d23af9fc7928fdb" +checksum = "4dad5567ad0cf5b760e5665964bec1b47dfd077ba8a2544b513f3556d3d239a2" dependencies = [ "getrandom", "serde", @@ -6529,12 +6508,11 @@ checksum = "9d5b2c62b4012a3e1eca5a7e077d13b3bf498c4073e33ccd58626607748ceeca" [[package]] name = "walkdir" -version = "2.3.2" +version = "2.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "808cf2735cd4b6866113f648b791c6adc5714537bc222d9347bb203386ffda56" +checksum = "36df944cda56c7d8d8b7496af378e6b16de9284591917d307c9b4d313c44e698" dependencies = [ "same-file", - "winapi", "winapi-util", ] @@ -6562,9 +6540,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.84" +version = "0.2.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31f8dcbc21f30d9b8f2ea926ecb58f6b91192c17e9d33594b3df58b2007ca53b" +checksum = "5b6cb788c4e39112fbe1822277ef6fb3c55cd86b95cb3d3c4c1c9597e4ac74b4" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -6572,24 +6550,24 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.84" +version = "0.2.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95ce90fd5bcc06af55a641a86428ee4229e44e07033963a2290a8e241607ccb9" +checksum = "35e522ed4105a9d626d885b35d62501b30d9666283a5c8be12c14a8bdafe7822" dependencies = [ "bumpalo", "log", "once_cell", "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.15", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-futures" -version = "0.4.34" +version = "0.4.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f219e0d211ba40266969f6dbdd90636da12f75bee4fc9d6c23d1260dadb51454" +checksum = "083abe15c5d88556b77bdf7aef403625be9e327ad37c62c4e4129af740168163" dependencies = [ "cfg-if", "js-sys", @@ -6599,9 +6577,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.84" +version = "0.2.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c21f77c0bedc37fd5dc21f897894a5ca01e7bb159884559461862ae90c0b4c5" +checksum = "358a79a0cb89d21db8120cbfb91392335913e4890665b1a7981d9e956903b434" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -6609,22 +6587,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.84" +version = "0.2.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2aff81306fcac3c7515ad4e177f521b5c9a15f2b08f4e32d823066102f35a5f6" +checksum = "4783ce29f09b9d93134d41297aded3a712b7b979e9c6f28c32cb88c973a94869" dependencies = [ "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.15", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.84" +version = "0.2.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0046fef7e28c3804e5e38bfa31ea2a0f73905319b677e57ebe37e49358989b5d" +checksum = "a901d592cafaa4d711bc324edfaff879ac700b19c3dfd60058d2b445be2691eb" [[package]] name = "wasm-streams" @@ -6641,9 +6619,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.61" +version = "0.3.62" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e33b99f4b23ba3eec1a53ac264e35a755f00e966e0065077d6027c0f575b0b97" +checksum = "16b5f940c7edfdc6d12126d98c9ef4d1b3d470011c47c76a6581df47ad9ba721" dependencies = [ "js-sys", "wasm-bindgen", @@ -6706,7 +6684,7 @@ dependencies = [ "serde_prometheus", "siwe", "thread-fast-rng", - "time 0.3.20", + "time 0.3.21", "tokio", "tokio-console", "tokio-stream", @@ -6716,7 +6694,7 @@ dependencies = [ "tower-http", "ulid", "url", - "uuid 1.3.1", + "uuid 1.3.2", ] [[package]] @@ -6936,9 +6914,9 @@ checksum = "1a515f5799fe4961cb532f983ce2b23082366b898e52ffbce459c86f67c8378a" [[package]] name = "winnow" -version = "0.4.1" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae8970b36c66498d8ff1d66685dc86b91b29db0c7739899012f63a63814b4b28" +checksum = "61de7bac303dc551fe038e2b3cef0f571087a47571ea6e79a87692ac99b99699" dependencies = [ "memchr", ] @@ -6988,9 +6966,9 @@ checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" [[package]] name = "zeroize" -version = "1.5.7" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c394b5bd0c6f669e7275d9c20aa90ae064cb22e75a1cad54e1b34088034b149f" +checksum = "2a0956f1ba7c7909bfb66c2e9e4124ab6f6482560f6628b5aaeba39207c9aad9" [[package]] name = "zip" @@ -7008,7 +6986,7 @@ dependencies = [ "hmac", "pbkdf2 0.11.0", "sha1", - "time 0.3.20", + "time 0.3.21", "zstd", ] @@ -7033,9 +7011,9 @@ dependencies = [ [[package]] name = "zstd-sys" -version = "2.0.7+zstd.1.5.4" +version = "2.0.8+zstd.1.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94509c3ba2fe55294d752b79842c530ccfab760192521df74a081a78d2b3c7f5" +checksum = "5556e6ee25d32df2586c098bbfa278803692a20d0ab9565e049480d52707ec8c" dependencies = [ "cc", "libc", diff --git a/deferred-rate-limiter/Cargo.toml b/deferred-rate-limiter/Cargo.toml index 752e2d8d..699c9927 100644 --- a/deferred-rate-limiter/Cargo.toml +++ b/deferred-rate-limiter/Cargo.toml @@ -7,8 +7,8 @@ edition = "2021" [dependencies] redis-rate-limiter = { path = "../redis-rate-limiter" } -anyhow = "1.0.70" +anyhow = "1.0.71" hashbrown = "0.13.2" log = "0.4.17" -moka = { version = "0.10.2", default-features = false, features = ["future"] } -tokio = "1.27.0" +moka = { version = "0.11.0", default-features = false, features = ["future"] } +tokio = "1.28.1" diff --git a/entities/Cargo.toml b/entities/Cargo.toml index 8fe7206e..2e94c960 100644 --- a/entities/Cargo.toml +++ b/entities/Cargo.toml @@ -10,8 +10,8 @@ path = "src/mod.rs" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] -sea-orm = "0.11.2" -serde = "1.0.160" -uuid = "1.3.1" -ethers = "2.0.3" +sea-orm = "0.11.3" +serde = "1.0.163" +uuid = "1.3.2" +ethers = "2.0.4" ulid = "1.0.0" diff --git a/latency/Cargo.toml b/latency/Cargo.toml index daf303e3..eb51eba9 100644 --- a/latency/Cargo.toml +++ b/latency/Cargo.toml @@ -8,8 +8,8 @@ edition = "2021" [dependencies] ewma = "0.1.1" log = "0.4.17" -serde = { version = "1.0.159", features = [] } -tokio = { version = "1.27.0", features = ["full"] } +serde = { version = "1.0.163", features = [] } +tokio = { version = "1.28.1", features = ["full"] } [dev-dependencies] -tokio = { version = "1.27.0", features = ["full", "test-util"] } +tokio = { version = "1.28.1", features = ["full", "test-util"] } diff --git a/migration/Cargo.toml b/migration/Cargo.toml index 22d5c752..d8c07ad9 100644 --- a/migration/Cargo.toml +++ b/migration/Cargo.toml @@ -9,10 +9,10 @@ name = "migration" path = "src/lib.rs" [dependencies] -tokio = { version = "1.27.0", features = ["full", "tracing"] } +tokio = { version = "1.28.1", features = ["full", "tracing"] } [dependencies.sea-orm-migration] -version = "0.11.2" +version = "0.11.3" features = [ # Enable at least one `ASYNC_RUNTIME` and `DATABASE_DRIVER` feature if you want to run migration via CLI. # View the list of supported features at https://www.sea-ql.org/SeaORM/docs/install-and-config/database-and-async-runtime. diff --git a/rate-counter/Cargo.toml b/rate-counter/Cargo.toml index 06dbf0b8..7bf027e4 100644 --- a/rate-counter/Cargo.toml +++ b/rate-counter/Cargo.toml @@ -6,4 +6,4 @@ edition = "2021" [dependencies] flume = "0.10.14" -tokio = { version = "1.27.0", features = ["time"] } +tokio = { version = "1.28.1", features = ["time"] } diff --git a/redis-rate-limiter/Cargo.toml b/redis-rate-limiter/Cargo.toml index 5f0480e8..cdb16c7c 100644 --- a/redis-rate-limiter/Cargo.toml +++ b/redis-rate-limiter/Cargo.toml @@ -5,7 +5,7 @@ authors = ["Bryan Stitt "] edition = "2021" [dependencies] -anyhow = "1.0.70" +anyhow = "1.0.71" chrono = "0.4.24" -deadpool-redis = { version = "0.11.1", features = ["rt_tokio_1", "serde"] } -tokio = "1.27.0" +deadpool-redis = { version = "0.12.0", features = ["rt_tokio_1", "serde"] } +tokio = "1.28.1" diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index 0bd41831..293b3ee9 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -26,10 +26,10 @@ thread-fast-rng = { path = "../thread-fast-rng" } # TODO: import chrono from sea-orm so we always have the same version # TODO: make sure this time version matches siwe. PR to put this in their prelude -anyhow = { version = "1.0.70", features = ["backtrace"] } +anyhow = { version = "1.0.71", features = ["backtrace"] } argh = "0.1.10" -axum = { version = "0.6.16", features = ["headers", "ws"] } -axum-client-ip = "0.4.0" +axum = { version = "0.6.18", features = ["headers", "ws"] } +axum-client-ip = "0.4.1" axum-macros = "0.3.7" chrono = "0.4.24" console-subscriber = { version = "*", optional = true } @@ -37,14 +37,14 @@ counter = "0.5.7" derive_more = "0.99.17" dotenv = "0.15.0" env_logger = "0.10.0" -ethers = { version = "2.0.3", default-features = false, features = ["rustls", "ws"] } +ethers = { version = "2.0.4", default-features = false, features = ["rustls", "ws"] } fdlimit = "0.2.1" flume = "0.10.14" fstrings = "0.2" futures = { version = "0.3.28", features = ["thread-pool"] } gethostname = "0.4.2" glob = "0.3.1" -handlebars = "4.3.6" +handlebars = "4.3.7" hashbrown = { version = "0.13.2", features = ["serde"] } hdrhistogram = "7.5.2" http = "0.2.9" @@ -54,33 +54,33 @@ hostname = "0.3.1" ipnet = "2.7.2" itertools = "0.10.5" log = "0.4.17" -moka = { version = "0.10.2", default-features = false, features = ["future"] } +moka = { version = "0.11.0", default-features = false, features = ["future"] } num = "0.4.0" num-traits = "0.2.15" once_cell = { version = "1.17.1" } -ordered-float = "3.6.0" +ordered-float = "3.7.0" pagerduty-rs = { version = "0.1.6", default-features = false, features = ["async", "rustls", "sync"] } parking_lot = { version = "0.12.1", features = ["arc_lock"] } prettytable = "*" proctitle = "0.1.1" rdkafka = { version = "0.29.0" } -regex = "1.7.3" -reqwest = { version = "0.11.16", default-features = false, features = ["json", "tokio-rustls"] } +regex = "1.8.1" +reqwest = { version = "0.11.17", default-features = false, features = ["json", "tokio-rustls"] } rmp-serde = "1.1.1" rustc-hash = "1.1.0" sentry = { version = "0.31.0", default-features = false, features = ["backtrace", "contexts", "panic", "anyhow", "reqwest", "rustls", "log", "sentry-log"] } -serde = { version = "1.0.160", features = [] } +serde = { version = "1.0.163", features = [] } serde_json = { version = "1.0.96", default-features = false, features = ["alloc", "raw_value"] } serde_prometheus = "0.2.2" siwe = "0.5.0" -time = "0.3.20" -tokio = { version = "1.27.0", features = ["full"] } +time = "0.3.21" +tokio = { version = "1.28.1", features = ["full"] } tokio-console = { version = "*", optional = true } -tokio-stream = { version = "0.1.12", features = ["sync"] } +tokio-stream = { version = "0.1.14", features = ["sync"] } tokio-uring = { version = "0.4.0", optional = true } toml = "0.7.3" tower = "0.4.13" tower-http = { version = "0.4.0", features = ["cors", "sensitive-headers"] } ulid = { version = "1.0.0", features = ["serde"] } url = "2.3.1" -uuid = "1.3.1" +uuid = "1.3.2" From fc8ca4ba4f3ed6ed085e605dd9569033dd4644dd Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Thu, 11 May 2023 17:04:33 -0700 Subject: [PATCH 06/66] less max lag --- README.md | 2 +- web3_proxy/src/rpcs/blockchain.rs | 2 ++ web3_proxy/src/rpcs/consensus.rs | 9 +++++++-- 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 0c1931e5..8f70d866 100644 --- a/README.md +++ b/README.md @@ -131,7 +131,7 @@ Flame graphs make a developer's join of finding slow code painless: 4 $ echo -1 | sudo tee /proc/sys/kernel/perf_event_paranoid -1 - $ CARGO_PROFILE_RELEASE_DEBUG=true cargo flamegraph --bin web3_proxy --no-inline + $ CARGO_PROFILE_RELEASE_DEBUG=true cargo flamegraph --bin web3_proxy_cli --no-inline -- proxyd Be sure to use `--no-inline` or perf will be VERY slow diff --git a/web3_proxy/src/rpcs/blockchain.rs b/web3_proxy/src/rpcs/blockchain.rs index 34a09e7a..356a8fa8 100644 --- a/web3_proxy/src/rpcs/blockchain.rs +++ b/web3_proxy/src/rpcs/blockchain.rs @@ -436,6 +436,8 @@ impl Web3Rpcs { Ok(Some(x)) => x, }; + trace!("new_synced_connections: {:?}", new_synced_connections); + let watch_consensus_head_sender = self.watch_consensus_head_sender.as_ref().unwrap(); let consensus_tier = new_synced_connections.tier; // TODO: think more about this unwrap diff --git a/web3_proxy/src/rpcs/consensus.rs b/web3_proxy/src/rpcs/consensus.rs index a352c7f9..a20f55f9 100644 --- a/web3_proxy/src/rpcs/consensus.rs +++ b/web3_proxy/src/rpcs/consensus.rs @@ -218,8 +218,8 @@ impl ConsensusFinder { trace!("lowest_block_number: {}", lowest_block.number()); - let max_lag_block_number = highest_block_number - .saturating_sub(self.max_block_lag.unwrap_or_else(|| U64::from(10))); + let max_lag_block_number = + highest_block_number.saturating_sub(self.max_block_lag.unwrap_or_else(|| U64::from(5))); trace!("max_lag_block_number: {}", max_lag_block_number); @@ -231,6 +231,7 @@ impl ConsensusFinder { if num_known < web3_rpcs.min_head_rpcs { // this keeps us from serving requests when the proxy first starts + trace!("not enough servers known"); return Ok(None); } @@ -250,18 +251,22 @@ impl ConsensusFinder { .0 .tier; + trace!("first_tier: {}", current_tier); + // loop over all the rpc heads (grouped by tier) and their parents to find consensus // TODO: i'm sure theres a lot of shortcuts that could be taken, but this is simplest to implement for (rpc, rpc_head) in self.rpc_heads.iter() { if current_tier != rpc.tier { // we finished processing a tier. check for primary results if let Some(consensus) = self.count_votes(&primary_votes, web3_rpcs) { + trace!("found enough votes on tier {}", current_tier); return Ok(Some(consensus)); } // only set backup consensus once. we don't want it to keep checking on worse tiers if it already found consensus if backup_consensus.is_none() { if let Some(consensus) = self.count_votes(&backup_votes, web3_rpcs) { + trace!("found backup votes on tier {}", current_tier); backup_consensus = Some(consensus) } } From 34ed450fab40016495c1e30622a6a3e47bbc6bb4 Mon Sep 17 00:00:00 2001 From: David Date: Fri, 12 May 2023 12:45:15 -0400 Subject: [PATCH 07/66] 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 Co-authored-by: Bryan Stitt --- Cargo.lock | 19 +- config/development_polygon.toml | 212 +++++ config/example.toml | 5 +- docs/curl login.md | 10 - docs/faster perf.txt | 8 - docs/http routes.txt | 144 --- docs/tracing notes.txt | 15 - entities/src/balance.rs | 37 + .../src/increase_on_chain_balance_receipt.rs | 37 + entities/src/mod.rs | 4 + entities/src/pending_login.rs | 17 +- entities/src/prelude.rs | 4 + entities/src/referee.rs | 51 ++ entities/src/referrer.rs | 42 + entities/src/rpc_accounting_v2.rs | 4 +- entities/src/rpc_key.rs | 8 + entities/src/secondary_user.rs | 15 + entities/src/user_tier.rs | 9 + migration/src/lib.rs | 14 + migration/src/m20230125_204810_stats_v2.rs | 12 + .../src/m20230205_130035_create_balance.rs | 72 ++ .../src/m20230205_133755_create_referrals.rs | 133 +++ ...14_134254_increase_balance_transactions.rs | 97 ++ migration/src/m20230221_230953_track_spend.rs | 42 + ..._modify_secondary_user_add_primary_user.rs | 58 ++ ...20230422_172555_premium_downgrade_logic.rs | 129 +++ ...emove_columns_statsv2_origin_and_method.rs | 50 ++ scripts/brownie-tests/.gitattributes | 2 + scripts/brownie-tests/.gitignore | 6 + scripts/brownie-tests/brownie-config.yaml | 1 + scripts/brownie-tests/scripts/make_payment.py | 34 + scripts/get-stats-aggregated.sh | 5 +- .../12-subusers-premium-account.sh | 110 +++ scripts/manual-tests/16-change-user-tier.sh | 8 +- .../24-simple-referral-program.sh | 111 +++ scripts/manual-tests/42-simple-balance.sh | 86 ++ scripts/manual-tests/48-balance-downgrade.sh | 88 ++ .../manual-tests/52-simple-get-deposits.sh | 5 + scripts/requirements.txt | 4 + web3_proxy/Cargo.toml | 7 +- web3_proxy/src/app/mod.rs | 21 +- .../bin/web3_proxy_cli/migrate_stats_to_v2.rs | 1 + web3_proxy/src/config.rs | 8 +- web3_proxy/src/frontend/authorization.rs | 10 +- web3_proxy/src/frontend/errors.rs | 24 + web3_proxy/src/frontend/mod.rs | 54 +- web3_proxy/src/frontend/users.rs | 838 ------------------ .../src/frontend/users/authentication.rs | 473 ++++++++++ web3_proxy/src/frontend/users/mod.rs | 83 ++ web3_proxy/src/frontend/users/payment.rs | 499 +++++++++++ web3_proxy/src/frontend/users/referral.rs | 87 ++ web3_proxy/src/frontend/users/rpc_keys.rs | 259 ++++++ web3_proxy/src/frontend/users/stats.rs | 123 +++ web3_proxy/src/frontend/users/subuser.rs | 426 +++++++++ web3_proxy/src/http_params.rs | 15 +- web3_proxy/src/lib.rs | 2 + web3_proxy/src/referral_code.rs | 24 + web3_proxy/src/rpcs/many.rs | 1 - web3_proxy/src/stats/db_queries.rs | 7 +- web3_proxy/src/stats/influxdb_queries.rs | 664 ++++++++------ web3_proxy/src/stats/mod.rs | 310 ++++++- 61 files changed, 4282 insertions(+), 1362 deletions(-) create mode 100644 config/development_polygon.toml delete mode 100644 docs/curl login.md delete mode 100644 docs/faster perf.txt delete mode 100644 docs/http routes.txt delete mode 100644 docs/tracing notes.txt create mode 100644 entities/src/balance.rs create mode 100644 entities/src/increase_on_chain_balance_receipt.rs create mode 100644 entities/src/referee.rs create mode 100644 entities/src/referrer.rs create mode 100644 migration/src/m20230205_130035_create_balance.rs create mode 100644 migration/src/m20230205_133755_create_referrals.rs create mode 100644 migration/src/m20230214_134254_increase_balance_transactions.rs create mode 100644 migration/src/m20230221_230953_track_spend.rs create mode 100644 migration/src/m20230412_171916_modify_secondary_user_add_primary_user.rs create mode 100644 migration/src/m20230422_172555_premium_downgrade_logic.rs create mode 100644 migration/src/m20230511_161214_remove_columns_statsv2_origin_and_method.rs create mode 100644 scripts/brownie-tests/.gitattributes create mode 100644 scripts/brownie-tests/.gitignore create mode 100644 scripts/brownie-tests/brownie-config.yaml create mode 100644 scripts/brownie-tests/scripts/make_payment.py create mode 100644 scripts/manual-tests/12-subusers-premium-account.sh create mode 100644 scripts/manual-tests/24-simple-referral-program.sh create mode 100644 scripts/manual-tests/42-simple-balance.sh create mode 100644 scripts/manual-tests/48-balance-downgrade.sh create mode 100644 scripts/manual-tests/52-simple-get-deposits.sh create mode 100644 scripts/requirements.txt delete mode 100644 web3_proxy/src/frontend/users.rs create mode 100644 web3_proxy/src/frontend/users/authentication.rs create mode 100644 web3_proxy/src/frontend/users/mod.rs create mode 100644 web3_proxy/src/frontend/users/payment.rs create mode 100644 web3_proxy/src/frontend/users/referral.rs create mode 100644 web3_proxy/src/frontend/users/rpc_keys.rs create mode 100644 web3_proxy/src/frontend/users/stats.rs create mode 100644 web3_proxy/src/frontend/users/subuser.rs create mode 100644 web3_proxy/src/referral_code.rs diff --git a/Cargo.lock b/Cargo.lock index 0f3b7617..3fe8f6f0 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2547,6 +2547,12 @@ version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" +[[package]] +name = "hex_fmt" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b07f60793ff0a4d9cef0f18e63b5357e06209987153a64648c972c1e5aff336f" + [[package]] name = "hmac" version = "0.12.1" @@ -2786,9 +2792,8 @@ dependencies = [ [[package]] name = "influxdb2" -version = "0.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "320c502ec0cf39e9b9fc36afc57435944fdfb6f15e8e8b0ecbc9a871d398cf63" +version = "0.4.0" +source = "git+https://github.com/llamanodes/influxdb2#9c2e50bee6f00fff99688ac2a39f702bb6a0b5bb" dependencies = [ "base64 0.13.1", "bytes", @@ -2819,8 +2824,7 @@ dependencies = [ [[package]] name = "influxdb2-derive" version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "990f899841aa30130fc06f7938e3cc2cbc3d5b92c03fd4b5d79a965045abcf16" +source = "git+https://github.com/llamanodes/influxdb2#9c2e50bee6f00fff99688ac2a39f702bb6a0b5bb" dependencies = [ "itertools", "proc-macro2", @@ -2832,8 +2836,7 @@ dependencies = [ [[package]] name = "influxdb2-structmap" version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1408e712051787357e99ff732e44e8833e79cea0fabc9361018abfbff72b6265" +source = "git+https://github.com/llamanodes/influxdb2#9c2e50bee6f00fff99688ac2a39f702bb6a0b5bb" dependencies = [ "chrono", "num-traits", @@ -6334,6 +6337,7 @@ checksum = "13a3aaa69b04e5b66cc27309710a569ea23593612387d67daaf102e73aa974fd" dependencies = [ "rand", "serde", + "uuid 1.3.2", ] [[package]] @@ -6631,6 +6635,7 @@ dependencies = [ "handlebars", "hashbrown 0.13.2", "hdrhistogram", + "hex_fmt", "hostname", "http", "influxdb2", diff --git a/config/development_polygon.toml b/config/development_polygon.toml new file mode 100644 index 00000000..f6eb5743 --- /dev/null +++ b/config/development_polygon.toml @@ -0,0 +1,212 @@ +[app] +chain_id = 137 + +# a database is optional. it is used for user authentication and accounting +# TODO: how do we find the optimal db_max_connections? too high actually ends up being slower +db_max_connections = 20 +# development runs cargo commands on the host and so uses "mysql://root:dev_web3_proxy@127.0.0.1:13306/dev_web3_proxy" for db_url +# production runs inside docker and so uses "mysql://root:web3_proxy@db:3306/web3_proxy" for db_url +db_url = "mysql://root:dev_web3_proxy@127.0.0.1:13306/dev_web3_proxy" + +deposit_factory_contract = "0x4e3bc2054788de923a04936c6addb99a05b0ea36" +deposit_topic = "0x45fdc265dc29885b9a485766b03e70978440d38c7c328ee0a14fa40c76c6af54" + +# a timeseries database is optional. it is used for making pretty graphs +influxdb_host = "http://127.0.0.1:18086" +influxdb_org = "dev_org" +influxdb_token = "dev_web3_proxy_auth_token" +influxdb_bucket = "dev_web3_proxy" + +# thundering herd protection +# only mark a block as the head block if the sum of their soft limits is greater than or equal to min_sum_soft_limit +min_sum_soft_limit = 1_000 +# only mark a block as the head block if the number of servers with it is great than or equal to min_synced_rpcs +min_synced_rpcs = 1 + +# redis is optional. it is used for rate limits set by `hard_limit` +# TODO: how do we find the optimal redis_max_connections? too high actually ends up being slower +volatile_redis_max_connections = 20 +# development runs cargo commands on the host and so uses "redis://127.0.0.1:16379/" for volatile_redis_url +# production runs inside docker and so uses "redis://redis:6379/" for volatile_redis_url +volatile_redis_url = "redis://127.0.0.1:16379/" + +# redirect_public_url is optional +redirect_public_url = "https://llamanodes.com/public-rpc" +# redirect_rpc_key_url is optional +# it only does something if db_url is set +redirect_rpc_key_url = "https://llamanodes.com/dashboard/keys?key={{rpc_key_id}}" + +# sentry is optional. it is used for browsing error logs +# sentry_url = "https://SENTRY_KEY_A.ingest.sentry.io/SENTRY_KEY_B" + +# public limits are when no key is used. these are instead grouped by ip +# 0 = block all public requests +# Not defined = allow all requests +#public_max_concurrent_requests = +# 0 = block all public requests +# Not defined = allow all requests +#public_requests_per_period = + +public_recent_ips_salt = "" + +login_domain = "llamanodes.com" + +# 1GB of cache +response_cache_max_bytes = 1_000_000_000 + +# allowed_origin_requests_per_period changes the min_sum_soft_limit for requests with the specified (AND SPOOFABLE) Origin header +# origins not in the list for requests without an rpc_key will use public_requests_per_period instead +[app.allowed_origin_requests_per_period] +"https://chainlist.org" = 1_000 + +[balanced_rpcs] + +[balanced_rpcs.llama_public] +disabled = false +display_name = "LlamaNodes" +http_url = "https://polygon.llamarpc.com" +ws_url = "wss://polygon.llamarpc.com" +soft_limit = 1_000 +tier = 0 + +[balanced_rpcs.quicknode] +disabled = false +display_name = "Quicknode" +http_url = "https://rpc-mainnet.matic.quiknode.pro" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.maticvigil] +disabled = false +display_name = "Maticvigil" +http_url = "https://rpc-mainnet.maticvigil.com" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.matic-network] +disabled = false +display_name = "Matic Network" +http_url = "https://rpc-mainnet.matic.network" +soft_limit = 10 +tier = 1 + +[balanced_rpcs.chainstack] +disabled = false +http_url = "https://matic-mainnet.chainstacklabs.com" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.bware] +disabled = false +display_name = "Bware Labs" +http_url = "https://matic-mainnet-full-rpc.bwarelabs.com" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.bware_archive] +disabled = false +display_name = "Bware Labs Archive" +http_url = "https://matic-mainnet-archive-rpc.bwarelabs.com" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.polygonapi] +disabled = false +display_name = "Polygon API" +http_url = "https://polygonapi.terminet.io/rpc" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.one-rpc] +disabled = false +display_name = "1RPC" +http_url = "https://1rpc.io/matic" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.fastrpc] +disabled = false +display_name = "FastRPC" +http_url = "https://polygon-mainnet.rpcfast.com?api_key=xbhWBI1Wkguk8SNMu1bvvLurPGLXmgwYeC4S6g2H7WdwFigZSmPWVZRxrskEQwIf" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.unifra] +disabled = false +display_name = "Unifra" +http_url = "https://polygon-mainnet-public.unifra.io" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.onfinality] +disabled = false +display_name = "Onfinality" +http_url = "https://polygon.api.onfinality.io/public" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.alchemy] +disabled = false +display_name = "Alchemy" +heept_url = "https://polygon-mainnet.g.alchemy.com/v2/demo" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.blockpi] +disabled = false +display_name = "Blockpi" +http_url = "https://polygon.blockpi.network/v1/rpc/public" +soft_limit = 100 +tier = 2 + +[balanced_rpcs.polygon] +backup = true +disabled = false +display_name = "Polygon" +http_url = "https://polygon-rpc.com" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.pokt] +disabled = false +display_name = "Pokt" +http_url = "https://poly-rpc.gateway.pokt.network" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.ankr] +backup = true +disabled = false +display_name = "Ankr" +http_url = "https://rpc.ankr.com/polygon" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.blastapi] +backup = true +disabled = true +display_name = "Blast" +http_url = "https://polygon-mainnet.public.blastapi.io" +hard_limit = 10 +soft_limit = 10 +tier = 2 + +[balanced_rpcs.omnia] +disabled = true +display_name = "Omnia" +http_url = "https://endpoints.omniatech.io/v1/matic/mainnet/public" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.bor] +disabled = true +http_url = "https://polygon-bor.publicnode.com" +soft_limit = 10 +tier = 2 + +[balanced_rpcs.blxr] +disabled = false +http_url = "https://polygon.rpc.blxrbdn.com" +soft_limit = 10 +tier = 2 + diff --git a/config/example.toml b/config/example.toml index d393b405..ad6ac303 100644 --- a/config/example.toml +++ b/config/example.toml @@ -11,6 +11,9 @@ db_url = "mysql://root:dev_web3_proxy@127.0.0.1:13306/dev_web3_proxy" # read-only replica useful when running the proxy in multiple regions db_replica_url = "mysql://root:dev_web3_proxy@127.0.0.1:13306/dev_web3_proxy" +deposit_factory_contract = "0x4e3bc2054788de923a04936c6addb99a05b0ea36" +deposit_topic = "0x45fdc265dc29885b9a485766b03e70978440d38c7c328ee0a14fa40c76c6af54" + kafka_urls = "127.0.0.1:19092" kafka_protocol = "plaintext" @@ -18,7 +21,7 @@ kafka_protocol = "plaintext" influxdb_host = "http://127.0.0.1:18086" influxdb_org = "dev_org" influxdb_token = "dev_web3_proxy_auth_token" -influxdb_bucketname = "web3_proxy" +influxdb_bucketname = "dev_web3_proxy" # thundering herd protection # only mark a block as the head block if the sum of their soft limits is greater than or equal to min_sum_soft_limit diff --git a/docs/curl login.md b/docs/curl login.md deleted file mode 100644 index 16ec43b7..00000000 --- a/docs/curl login.md +++ /dev/null @@ -1,10 +0,0 @@ -# log in with curl - -1. curl http://127.0.0.1:8544/user/login/$ADDRESS -2. Sign the text with a site like https://www.myetherwallet.com/wallet/sign -3. POST the signed data: - - curl -X POST http://127.0.0.1:8544/user/login -H 'Content-Type: application/json' -d - '{ "address": "0x9eb9e3dc2543dc9ff4058e2a2da43a855403f1fd", "msg": "0x6c6c616d616e6f6465732e636f6d2077616e747320796f7520746f207369676e20696e207769746820796f757220457468657265756d206163636f756e743a0a3078396562396533646332353433646339464634303538653241324441343341383535343033463166440a0af09fa699f09fa699f09fa699f09fa699f09fa6990a0a5552493a2068747470733a2f2f6c6c616d616e6f6465732e636f6d2f0a56657273696f6e3a20310a436861696e2049443a20310a4e6f6e63653a203031474d37373330375344324448333854454d3957545156454a0a4973737565642041743a20323032322d31322d31345430323a32333a31372e3735333736335a0a45787069726174696f6e2054696d653a20323032322d31322d31345430323a34333a31372e3735333736335a", "sig": "16bac055345279723193737c6c67cf995e821fd7c038d31fd6f671102088c7b85ab4b13069fd2ed02da186cf549530e315d8d042d721bf81289b3ffdbe8cf9ce1c", "version": "3", "signer": "MEW" }' - -4. The response will include a bearer token. Use it with curl ... -H 'Authorization: Bearer $TOKEN' diff --git a/docs/faster perf.txt b/docs/faster perf.txt deleted file mode 100644 index 4a6f2073..00000000 --- a/docs/faster perf.txt +++ /dev/null @@ -1,8 +0,0 @@ -sudo apt install bison flex -wget https://eighty-twenty.org/files/0001-tools-perf-Use-long-running-addr2line-per-dso.patch -git clone https://github.com/torvalds/linux.git -cd linux -git checkout v5.15 -git apply ../0001-tools-perf-Use-long-running-addr2line-per-dso.patch -cd tools/perf -make prefix=$HOME/.local VERSION=5.15 install-bin diff --git a/docs/http routes.txt b/docs/http routes.txt deleted file mode 100644 index b8798224..00000000 --- a/docs/http routes.txt +++ /dev/null @@ -1,144 +0,0 @@ - -GET / - This entrypoint handles two things. - If connecting with a browser, it redirects to the public stat page on llamanodes.com. - If connecting with a websocket, it is rate limited by IP and routes to the Web3 RPC. - -POST / - This entrypoint handles two things. - If connecting with a browser, it redirects to the public stat page on llamanodes.com. - If connecting with a websocket, it is rate limited by IP and routes to the Web3 RPC. - -GET /rpc/:rpc_key - This entrypoint handles two things. - If connecting with a browser, it redirects to the key's stat page on llamanodes.com. - If connecting with a websocket, it is rate limited by key and routes to the Web3 RPC. - -POST /rpc/:rpc_key - This entrypoint handles two things. - If connecting with a browser, it redirects to the key's stat page on llamanodes.com. - If connecting with a websocket, it is rate limited by key and routes to the Web3 RPC. - -GET /health - If servers are synced, this gives a 200 "OK". - If no servers are synced, it gives a 502 ":(" - -GET /user/login/:user_address - Displays a "Sign in With Ethereum" message to be signed by the address's private key. - Once signed, continue to `POST /user/login` - -GET /user/login/:user_address/:message_eip - Similar to `GET /user/login/:user_address` but gives the message in different formats depending on the eip. - Wallets have varying support. This shouldn't be needed by most users. - The message_eip should be hidden behind a small gear icon near the login button. - Once signed, continue to `POST /user/login` - - Supported: - EIP191 as bytes - EIP191 as a hash - EIP4361 (the default) - - Support coming soon: - EIP1271 for contract signing - -POST /user/login?invite_code=SOMETHING_SECRET - Verifies the user's signed message. - - The post should have JSON data containing "sig" (the signature) and "msg" (the original message). - - Optionally requires an invite_code. - The invite code is only needed for new users. Once registered, it is not necessary. - - If the invite code and signature are valid, this returns JSON data containing "rpc_keys", "bearer_token" and the "user". - - "rpc_keys" contains the key and settings for all of the user's keys. - If the user is new, an "rpc_key" will be created for them. - - The "bearer_token" is required by some endpoints. Include it in the "AUTHORIZATION" header in this format: "bearer :bearer_token". - The token is good for 4 weeks and the 4 week time will reset whenever the token is used. - - The "user" just has an address at first, but you can prompt them to add an email address. See `POST /user` - -GET /user - Checks the "AUTHORIZATION" header for a valid bearer token. - If valid, display's the user's data as JSON. - - - -POST /user - POST the data in the same format that `GET /user` gives it. - If you do not want to update a field, do not include it in the POSTed JSON. - If you want to delete a field, include the data's key and set the value to an empty string. - - Checks the "AUTHORIZATION" header for a valid bearer token. - If valid, updates the user's data and returns the updated data as JSON. - -GET /user/balance - Not yet implemented. - - Checks the "AUTHORIZATION" header for a valid bearer token. - If valid, displays data about the user's balance and payments as JSON. - -POST /user/balance/:txid - Not yet implemented. Rate limited by IP. - - Checks the ":txid" for a transaction that updates a user's balance. - The backend will be watching for these transactions, so this should not be needed in the common case. - However, log susbcriptions are not perfect and so it might sometimes be needed. - -GET /user/keys - Checks the "AUTHORIZATION" header for a valid bearer token. - If valid, displays data about the user's keys as JSON. - -POST or PUT /user/keys - Checks the "AUTHORIZATION" header for a valid bearer token. - If valid, allows the user to create a new key or change options on their keys. - - The POSTed JSON can have these fields: - key_id: Option, - description: Option, - private_txs: Option, - active: Option, - allowed_ips: Option, - allowed_origins: Option, - allowed_referers: Option, - allowed_user_agents: Option, - - The PUTed JSON has the same fields as the POSTed JSON, except for there is no `key_id` - - If you do not want to update a field, do not include it in the POSTed JSON. - If you want to delete a string field, include the data's key and set the value to an empty string. - - `allowed_ips`, `allowed_origins`, `allowed_referers`, and `allowed_user_agents` can have multiple values by separating them with commas. - `allowed_ips` must be in CIDR Notation (ex: "10.1.1.0/24" for a network, "10.1.1.10/32" for a single address). - The spec technically allows for bytes in `allowed_origins` or `allowed_referers`, but our code currently only supports strings. If a customer needs bytes, then we can code support for them. - - `private_txs` are not currently recommended. If high gas is not supplied then they will likely never be included. Improvements to this are in the works - - Soon, the POST data will also have a `log_revert_trace: Option`. This will by the percent chance to log any calls that "revert" to the database. Large dapps probably want this to be a small percent, but development keys will probably want 100%. This will not be enabled until automatic pruning is coded. - -GET `/user/revert_logs` - Checks the "AUTHORIZATION" header for a valid bearer token. - If valid, fetches paginated revert logs for the user. - More documentation will be written here once revert logging is enabled. - -GET /user/stats/aggregate - Checks the "AUTHORIZATION" header for a valid bearer token. - If valid, fetches paginated aggregated stats for the user. - Pages are limited to 200 entries. The backend config can change this page size if necessary. - Can be filtered by: - `chain_id` - set to 0 for all. 0 is the default. - `query_start` - The start date in unix epoch time. - `query_window_seconds` - How many seconds to aggregate the stats over. - `page` - The page to request. Defaults to 0. - -GET /user/stats/detailed - Checks the "AUTHORIZATION" header for a valid bearer token. - If valid, fetches paginated stats for the user with more detail. The request method is included. For user privacy, we intentionally do not include the request's calldata. - Can be filtered the same as `GET /user/stats/aggregate` - Soon will also be filterable by "method" - -POST /user/logout - Checks the "AUTHORIZATION" header for a valid bearer token. - If valid, deletes the bearer token from the proxy. - The user will need to `POST /user/login` to get a new bearer token. diff --git a/docs/tracing notes.txt b/docs/tracing notes.txt deleted file mode 100644 index 67418909..00000000 --- a/docs/tracing notes.txt +++ /dev/null @@ -1,15 +0,0 @@ -Hello, I'm pretty new to tracing so my vocabulary might be wrong. I've got my app using tracing to log to stdout. I have a bunch of fields including user_id and ip_addr that make telling where logs are from nice and easy. - -Now there is one part of my code where I want to save a log to a database. I'm not sure of the best/correct way to do this. I can get the current span with tracing::Span::current(), but AFAICT that doesn't have a way to get to the values. I think I need to write my own Subscriber or Visitor (or both) and then tell tracing to use it only in this one part of the code. Am I on the right track? Is there a place in the docs that explains something similar? - -https://burgers.io/custom-logging-in-rust-using-tracing - -if you are doing it learn how to write a subscriber then you should write a custom layer. If you are simply trying to work on your main project there are several subscribers that already do this work for you. - -look at opentelemetry_otlp .. this will let you connect opentelemetry collector to your tracing using tracing_opentelemetry - -I'd suggest using the Registry subscriber because it can take multiple layers ... and use a filtered_layer to filter out the messages (look at env_filter, it can take the filtering params from an environment variable or a config string) and then have your collector be the second layer. e.... Registery can take in a vector of layers that are also-in-turn multi-layered. -let me see if i can pull up an example -On the https://docs.rs/tracing-subscriber/latest/tracing_subscriber/layer/ page about half-way down there is an example of boxed layers - -you basically end up composing different layers that output to different trace stores and also configure each using per-layer filtering (see https://docs.rs/tracing-subscriber/latest/tracing_subscriber/layer/#per-layer-filtering) diff --git a/entities/src/balance.rs b/entities/src/balance.rs new file mode 100644 index 00000000..9125442c --- /dev/null +++ b/entities/src/balance.rs @@ -0,0 +1,37 @@ +//! `SeaORM` Entity. Generated by sea-orm-codegen 0.10.6 + +use sea_orm::entity::prelude::*; +use serde::{Deserialize, Serialize}; + +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq, Serialize, Deserialize)] +#[sea_orm(table_name = "balance")] +pub struct Model { + #[sea_orm(primary_key)] + pub id: i32, + #[sea_orm(column_type = "Decimal(Some((20, 10)))")] + pub available_balance: Decimal, + #[sea_orm(column_type = "Decimal(Some((20, 10)))")] + pub used_balance: Decimal, + #[sea_orm(unique)] + pub user_id: u64, +} + +#[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] +pub enum Relation { + #[sea_orm( + belongs_to = "super::user::Entity", + from = "Column::UserId", + to = "super::user::Column::Id", + on_update = "NoAction", + on_delete = "NoAction" + )] + User, +} + +impl Related for Entity { + fn to() -> RelationDef { + Relation::User.def() + } +} + +impl ActiveModelBehavior for ActiveModel {} diff --git a/entities/src/increase_on_chain_balance_receipt.rs b/entities/src/increase_on_chain_balance_receipt.rs new file mode 100644 index 00000000..24acfd43 --- /dev/null +++ b/entities/src/increase_on_chain_balance_receipt.rs @@ -0,0 +1,37 @@ +//! `SeaORM` Entity. Generated by sea-orm-codegen 0.10.6 + +use sea_orm::entity::prelude::*; +use serde::{Deserialize, Serialize}; + +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq, Serialize, Deserialize)] +#[sea_orm(table_name = "increase_on_chain_balance_receipt")] +pub struct Model { + #[sea_orm(primary_key)] + pub id: i32, + #[sea_orm(unique)] + pub tx_hash: String, + pub chain_id: u64, + #[sea_orm(column_type = "Decimal(Some((20, 10)))")] + pub amount: Decimal, + pub deposit_to_user_id: u64, +} + +#[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] +pub enum Relation { + #[sea_orm( + belongs_to = "super::user::Entity", + from = "Column::DepositToUserId", + to = "super::user::Column::Id", + on_update = "NoAction", + on_delete = "NoAction" + )] + User, +} + +impl Related for Entity { + fn to() -> RelationDef { + Relation::User.def() + } +} + +impl ActiveModelBehavior for ActiveModel {} diff --git a/entities/src/mod.rs b/entities/src/mod.rs index fccd7e86..91a8a460 100644 --- a/entities/src/mod.rs +++ b/entities/src/mod.rs @@ -4,8 +4,12 @@ pub mod prelude; pub mod admin; pub mod admin_trail; +pub mod balance; +pub mod increase_on_chain_balance_receipt; pub mod login; pub mod pending_login; +pub mod referee; +pub mod referrer; pub mod revert_log; pub mod rpc_accounting; pub mod rpc_accounting_v2; diff --git a/entities/src/pending_login.rs b/entities/src/pending_login.rs index c162aaa9..6c701fca 100644 --- a/entities/src/pending_login.rs +++ b/entities/src/pending_login.rs @@ -19,6 +19,21 @@ pub struct Model { } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] -pub enum Relation {} +pub enum Relation { + #[sea_orm( + belongs_to = "super::user::Entity", + from = "Column::ImitatingUser", + to = "super::user::Column::Id", + on_update = "NoAction", + on_delete = "NoAction" + )] + User, +} + +impl Related for Entity { + fn to() -> RelationDef { + Relation::User.def() + } +} impl ActiveModelBehavior for ActiveModel {} diff --git a/entities/src/prelude.rs b/entities/src/prelude.rs index a4dda8b1..9d5f4cc0 100644 --- a/entities/src/prelude.rs +++ b/entities/src/prelude.rs @@ -2,8 +2,12 @@ pub use super::admin::Entity as Admin; pub use super::admin_trail::Entity as AdminTrail; +pub use super::balance::Entity as Balance; +pub use super::increase_on_chain_balance_receipt::Entity as IncreaseOnChainBalanceReceipt; pub use super::login::Entity as Login; pub use super::pending_login::Entity as PendingLogin; +pub use super::referee::Entity as Referee; +pub use super::referrer::Entity as Referrer; pub use super::revert_log::Entity as RevertLog; pub use super::rpc_accounting::Entity as RpcAccounting; pub use super::rpc_accounting_v2::Entity as RpcAccountingV2; diff --git a/entities/src/referee.rs b/entities/src/referee.rs new file mode 100644 index 00000000..5fad66a4 --- /dev/null +++ b/entities/src/referee.rs @@ -0,0 +1,51 @@ +//! `SeaORM` Entity. Generated by sea-orm-codegen 0.10.6 + +use sea_orm::entity::prelude::*; + +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] +#[sea_orm(table_name = "referee")] +pub struct Model { + #[sea_orm(primary_key)] + pub id: i32, + pub credits_applied_for_referee: bool, + #[sea_orm(column_type = "Decimal(Some((20, 10)))")] + pub credits_applied_for_referrer: Decimal, + pub referral_start_date: DateTime, + pub used_referral_code: i32, + #[sea_orm(unique)] + pub user_id: u64, +} + +#[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] +pub enum Relation { + #[sea_orm( + belongs_to = "super::referrer::Entity", + from = "Column::UsedReferralCode", + to = "super::referrer::Column::Id", + on_update = "NoAction", + on_delete = "NoAction" + )] + Referrer, + #[sea_orm( + belongs_to = "super::user::Entity", + from = "Column::UserId", + to = "super::user::Column::Id", + on_update = "NoAction", + on_delete = "NoAction" + )] + User, +} + +impl Related for Entity { + fn to() -> RelationDef { + Relation::Referrer.def() + } +} + +impl Related for Entity { + fn to() -> RelationDef { + Relation::User.def() + } +} + +impl ActiveModelBehavior for ActiveModel {} diff --git a/entities/src/referrer.rs b/entities/src/referrer.rs new file mode 100644 index 00000000..c2069776 --- /dev/null +++ b/entities/src/referrer.rs @@ -0,0 +1,42 @@ +//! `SeaORM` Entity. Generated by sea-orm-codegen 0.10.6 + +use sea_orm::entity::prelude::*; + +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] +#[sea_orm(table_name = "referrer")] +pub struct Model { + #[sea_orm(primary_key)] + pub id: i32, + #[sea_orm(unique)] + pub referral_code: String, + #[sea_orm(unique)] + pub user_id: u64, +} + +#[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] +pub enum Relation { + #[sea_orm(has_many = "super::referee::Entity")] + Referee, + #[sea_orm( + belongs_to = "super::user::Entity", + from = "Column::UserId", + to = "super::user::Column::Id", + on_update = "NoAction", + on_delete = "NoAction" + )] + User, +} + +impl Related for Entity { + fn to() -> RelationDef { + Relation::Referee.def() + } +} + +impl Related for Entity { + fn to() -> RelationDef { + Relation::User.def() + } +} + +impl ActiveModelBehavior for ActiveModel {} diff --git a/entities/src/rpc_accounting_v2.rs b/entities/src/rpc_accounting_v2.rs index 9f94c7e7..49121125 100644 --- a/entities/src/rpc_accounting_v2.rs +++ b/entities/src/rpc_accounting_v2.rs @@ -11,8 +11,6 @@ pub struct Model { pub rpc_key_id: u64, pub chain_id: u64, pub period_datetime: DateTimeUtc, - pub method: String, - pub origin: String, pub archive_needed: bool, pub error_response: bool, pub frontend_requests: u64, @@ -24,6 +22,8 @@ pub struct Model { pub sum_request_bytes: u64, pub sum_response_millis: u64, pub sum_response_bytes: u64, + #[sea_orm(column_type = "Decimal(Some((20, 10)))")] + pub sum_credits_used: Decimal, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] diff --git a/entities/src/rpc_key.rs b/entities/src/rpc_key.rs index 54102209..547aabdf 100644 --- a/entities/src/rpc_key.rs +++ b/entities/src/rpc_key.rs @@ -38,6 +38,8 @@ pub enum Relation { RpcAccounting, #[sea_orm(has_many = "super::rpc_accounting_v2::Entity")] RpcAccountingV2, + #[sea_orm(has_many = "super::secondary_user::Entity")] + SecondaryUser, #[sea_orm( belongs_to = "super::user::Entity", from = "Column::UserId", @@ -66,6 +68,12 @@ impl Related for Entity { } } +impl Related for Entity { + fn to() -> RelationDef { + Relation::SecondaryUser.def() + } +} + impl Related for Entity { fn to() -> RelationDef { Relation::User.def() diff --git a/entities/src/secondary_user.rs b/entities/src/secondary_user.rs index 69b62220..11ce8c2d 100644 --- a/entities/src/secondary_user.rs +++ b/entities/src/secondary_user.rs @@ -11,6 +11,7 @@ pub struct Model { pub id: u64, pub user_id: u64, pub description: Option, + pub rpc_secret_key_id: u64, pub role: Role, } @@ -24,6 +25,14 @@ pub enum Relation { on_delete = "NoAction" )] User, + #[sea_orm( + belongs_to = "super::rpc_key::Entity", + from = "Column::RpcSecretKeyId", + to = "super::rpc_key::Column::Id", + on_update = "NoAction", + on_delete = "NoAction" + )] + RpcKey, } impl Related for Entity { @@ -32,4 +41,10 @@ impl Related for Entity { } } +impl Related for Entity { + fn to() -> RelationDef { + Relation::RpcKey.def() + } +} + impl ActiveModelBehavior for ActiveModel {} diff --git a/entities/src/user_tier.rs b/entities/src/user_tier.rs index a025bc96..d0409f23 100644 --- a/entities/src/user_tier.rs +++ b/entities/src/user_tier.rs @@ -11,12 +11,21 @@ pub struct Model { pub title: String, pub max_requests_per_period: Option, pub max_concurrent_requests: Option, + pub downgrade_tier_id: Option, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] pub enum Relation { #[sea_orm(has_many = "super::user::Entity")] User, + #[sea_orm( + belongs_to = "Entity", + from = "Column::DowngradeTierId", + to = "Column::Id", + on_update = "NoAction", + on_delete = "NoAction" + )] + SelfRef, } impl Related for Entity { diff --git a/migration/src/lib.rs b/migration/src/lib.rs index cc031348..ddd8160d 100644 --- a/migration/src/lib.rs +++ b/migration/src/lib.rs @@ -19,6 +19,13 @@ mod m20230130_124740_read_only_login_logic; mod m20230130_165144_prepare_admin_imitation_pre_login; mod m20230215_152254_admin_trail; mod m20230307_002623_migrate_rpc_accounting_to_rpc_accounting_v2; +mod m20230205_130035_create_balance; +mod m20230205_133755_create_referrals; +mod m20230214_134254_increase_balance_transactions; +mod m20230221_230953_track_spend; +mod m20230412_171916_modify_secondary_user_add_primary_user; +mod m20230422_172555_premium_downgrade_logic; +mod m20230511_161214_remove_columns_statsv2_origin_and_method; pub struct Migrator; @@ -45,6 +52,13 @@ impl MigratorTrait for Migrator { Box::new(m20230130_165144_prepare_admin_imitation_pre_login::Migration), Box::new(m20230215_152254_admin_trail::Migration), Box::new(m20230307_002623_migrate_rpc_accounting_to_rpc_accounting_v2::Migration), + Box::new(m20230205_130035_create_balance::Migration), + Box::new(m20230205_133755_create_referrals::Migration), + Box::new(m20230214_134254_increase_balance_transactions::Migration), + Box::new(m20230221_230953_track_spend::Migration), + Box::new(m20230412_171916_modify_secondary_user_add_primary_user::Migration), + Box::new(m20230422_172555_premium_downgrade_logic::Migration), + Box::new(m20230511_161214_remove_columns_statsv2_origin_and_method::Migration), ] } } diff --git a/migration/src/m20230125_204810_stats_v2.rs b/migration/src/m20230125_204810_stats_v2.rs index 7082fec0..dc61dede 100644 --- a/migration/src/m20230125_204810_stats_v2.rs +++ b/migration/src/m20230125_204810_stats_v2.rs @@ -23,6 +23,12 @@ impl MigrationTrait for Migration { .not_null() .default(0), ) + .foreign_key( + ForeignKeyCreateStatement::new() + .from_col(RpcAccountingV2::RpcKeyId) + .to_tbl(RpcKey::Table) + .to_col(RpcKey::Id), + ) .col( ColumnDef::new(RpcAccountingV2::ChainId) .big_unsigned() @@ -136,6 +142,12 @@ impl MigrationTrait for Migration { } } +#[derive(Iden)] +enum RpcKey { + Table, + Id, +} + #[derive(Iden)] enum RpcAccountingV2 { Table, diff --git a/migration/src/m20230205_130035_create_balance.rs b/migration/src/m20230205_130035_create_balance.rs new file mode 100644 index 00000000..11076fce --- /dev/null +++ b/migration/src/m20230205_130035_create_balance.rs @@ -0,0 +1,72 @@ +use sea_orm_migration::prelude::*; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + // Replace the sample below with your own migration scripts + manager + .create_table( + Table::create() + .table(Balance::Table) + .if_not_exists() + .col( + ColumnDef::new(Balance::Id) + .integer() + .not_null() + .auto_increment() + .primary_key(), + ) + .col( + ColumnDef::new(Balance::AvailableBalance) + .decimal_len(20, 10) + .not_null() + .default(0.0), + ) + .col( + ColumnDef::new(Balance::UsedBalance) + .decimal_len(20, 10) + .not_null() + .default(0.0), + ) + .col( + ColumnDef::new(Balance::UserId) + .big_unsigned() + .unique_key() + .not_null(), + ) + .foreign_key( + sea_query::ForeignKey::create() + .from(Balance::Table, Balance::UserId) + .to(User::Table, User::Id), + ) + .to_owned(), + ) + .await + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + // Replace the sample below with your own migration scripts + manager + .drop_table(Table::drop().table(Balance::Table).to_owned()) + .await + } +} + +/// Learn more at https://docs.rs/sea-query#iden +#[derive(Iden)] +enum User { + Table, + Id, +} + +#[derive(Iden)] +enum Balance { + Table, + Id, + UserId, + AvailableBalance, + UsedBalance, +} diff --git a/migration/src/m20230205_133755_create_referrals.rs b/migration/src/m20230205_133755_create_referrals.rs new file mode 100644 index 00000000..c7a5d52c --- /dev/null +++ b/migration/src/m20230205_133755_create_referrals.rs @@ -0,0 +1,133 @@ +use sea_orm_migration::prelude::*; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + // Create one table for the referrer + manager + .create_table( + Table::create() + .table(Referrer::Table) + .if_not_exists() + .col( + ColumnDef::new(Referrer::Id) + .integer() + .not_null() + .auto_increment() + .primary_key(), + ) + .col( + ColumnDef::new(Referrer::ReferralCode) + .string() + .unique_key() + .not_null(), + ) + .col( + ColumnDef::new(Referrer::UserId) + .big_unsigned() + .unique_key() + .not_null(), + ) + .foreign_key( + sea_query::ForeignKey::create() + .from(Referrer::Table, Referrer::UserId) + .to(User::Table, User::Id), + ) + .to_owned(), + ) + .await?; + + // Create one table for the referrer + manager + .create_table( + Table::create() + .table(Referee::Table) + .if_not_exists() + .col( + ColumnDef::new(Referee::Id) + .integer() + .not_null() + .auto_increment() + .primary_key(), + ) + .col( + ColumnDef::new(Referee::CreditsAppliedForReferee) + .boolean() + .not_null(), + ) + .col( + ColumnDef::new(Referee::CreditsAppliedForReferrer) + .decimal_len(20, 10) + .not_null() + .default(0), + ) + .col( + ColumnDef::new(Referee::ReferralStartDate) + .date_time() + .not_null() + .extra("DEFAULT CURRENT_TIMESTAMP".to_string()), + ) + .col( + ColumnDef::new(Referee::UsedReferralCode) + .integer() + .not_null(), + ) + .foreign_key( + sea_query::ForeignKey::create() + .from(Referee::Table, Referee::UsedReferralCode) + .to(Referrer::Table, Referrer::Id), + ) + .col( + ColumnDef::new(Referee::UserId) + .big_unsigned() + .unique_key() + .not_null(), + ) + .foreign_key( + sea_query::ForeignKey::create() + .from(Referee::Table, Referee::UserId) + .to(User::Table, User::Id), + ) + .to_owned(), + ) + .await + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .drop_table(Table::drop().table(Referrer::Table).to_owned()) + .await?; + manager + .drop_table(Table::drop().table(Referee::Table).to_owned()) + .await + } +} + +/// Learn more at https://docs.rs/sea-query#iden +#[derive(Iden)] +enum Referrer { + Table, + Id, + UserId, + ReferralCode, +} + +#[derive(Iden)] +enum Referee { + Table, + Id, + UserId, + UsedReferralCode, + CreditsAppliedForReferrer, + CreditsAppliedForReferee, + ReferralStartDate, +} + +#[derive(Iden)] +enum User { + Table, + Id, +} diff --git a/migration/src/m20230214_134254_increase_balance_transactions.rs b/migration/src/m20230214_134254_increase_balance_transactions.rs new file mode 100644 index 00000000..72ea4d60 --- /dev/null +++ b/migration/src/m20230214_134254_increase_balance_transactions.rs @@ -0,0 +1,97 @@ +use sea_orm_migration::prelude::*; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + // Adds a table which keeps track of which transactions were already added (basically to prevent double spending) + manager + .create_table( + Table::create() + .table(IncreaseOnChainBalanceReceipt::Table) + .if_not_exists() + .col( + ColumnDef::new(IncreaseOnChainBalanceReceipt::Id) + .integer() + .not_null() + .auto_increment() + .primary_key(), + ) + .col( + ColumnDef::new(IncreaseOnChainBalanceReceipt::TxHash) + .string() + .not_null(), + ) + .col( + ColumnDef::new(IncreaseOnChainBalanceReceipt::ChainId) + .big_integer() + .not_null(), + ) + .col( + ColumnDef::new(IncreaseOnChainBalanceReceipt::Amount) + .decimal_len(20, 10) + .not_null(), + ) + .col( + ColumnDef::new(IncreaseOnChainBalanceReceipt::DepositToUserId) + .big_unsigned() + .unique_key() + .not_null(), + ) + .foreign_key( + ForeignKey::create() + .name("fk-deposit_to_user_id") + .from( + IncreaseOnChainBalanceReceipt::Table, + IncreaseOnChainBalanceReceipt::DepositToUserId, + ) + .to(User::Table, User::Id), + ) + .to_owned(), + ) + .await?; + + // Add a unique-constraint on chain-id and tx-hash + manager + .create_index( + Index::create() + .name("idx-increase_on_chain_balance_receipt-unique-chain_id-tx_hash") + .table(IncreaseOnChainBalanceReceipt::Table) + .col(IncreaseOnChainBalanceReceipt::ChainId) + .col(IncreaseOnChainBalanceReceipt::TxHash) + .unique() + .to_owned(), + ) + .await + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + // Replace the sample below with your own migration scripts + manager + .drop_table( + Table::drop() + .table(IncreaseOnChainBalanceReceipt::Table) + .to_owned(), + ) + .await + } +} + +/// Learn more at https://docs.rs/sea-query#iden +#[derive(Iden)] +enum IncreaseOnChainBalanceReceipt { + Table, + Id, + TxHash, + ChainId, + Amount, + DepositToUserId, +} + +#[derive(Iden)] +enum User { + Table, + Id, +} diff --git a/migration/src/m20230221_230953_track_spend.rs b/migration/src/m20230221_230953_track_spend.rs new file mode 100644 index 00000000..d6a62d32 --- /dev/null +++ b/migration/src/m20230221_230953_track_spend.rs @@ -0,0 +1,42 @@ +use sea_orm_migration::prelude::*; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + // Track spend inside the RPC accounting v2 table + manager + .alter_table( + Table::alter() + .table(RpcAccountingV2::Table) + .add_column( + ColumnDef::new(RpcAccountingV2::SumCreditsUsed) + .decimal_len(20, 10) + .not_null(), + ) + .to_owned(), + ) + .await + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + // Replace the sample below with your own migration scripts + manager + .alter_table( + sea_query::Table::alter() + .table(RpcAccountingV2::Table) + .drop_column(RpcAccountingV2::SumCreditsUsed) + .to_owned(), + ) + .await + } +} + +/// Learn more at https://docs.rs/sea-query#iden +#[derive(Iden)] +enum RpcAccountingV2 { + Table, + SumCreditsUsed, +} diff --git a/migration/src/m20230412_171916_modify_secondary_user_add_primary_user.rs b/migration/src/m20230412_171916_modify_secondary_user_add_primary_user.rs new file mode 100644 index 00000000..4a09040b --- /dev/null +++ b/migration/src/m20230412_171916_modify_secondary_user_add_primary_user.rs @@ -0,0 +1,58 @@ +use sea_orm_migration::prelude::*; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + Table::alter() + .table(SecondaryUser::Table) + .add_column( + ColumnDef::new(SecondaryUser::RpcSecretKeyId) + .big_unsigned() + .not_null(), // add foreign key to user table ..., + ) + .add_foreign_key( + TableForeignKey::new() + .name("FK_secondary_user-rpc_key") + .from_tbl(SecondaryUser::Table) + .from_col(SecondaryUser::RpcSecretKeyId) + .to_tbl(RpcKey::Table) + .to_col(RpcKey::Id) + .on_delete(ForeignKeyAction::NoAction) + .on_update(ForeignKeyAction::NoAction), + ) + .to_owned(), + ) + .await + + // TODO: Add a unique index on RpcKey + Subuser + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + sea_query::Table::alter() + .table(SecondaryUser::Table) + .drop_column(SecondaryUser::RpcSecretKeyId) + .to_owned(), + ) + .await + } +} + +/// Learn more at https://docs.rs/sea-query#iden +#[derive(Iden)] +enum SecondaryUser { + Table, + RpcSecretKeyId, +} + +#[derive(Iden)] +enum RpcKey { + Table, + Id, +} diff --git a/migration/src/m20230422_172555_premium_downgrade_logic.rs b/migration/src/m20230422_172555_premium_downgrade_logic.rs new file mode 100644 index 00000000..e474a785 --- /dev/null +++ b/migration/src/m20230422_172555_premium_downgrade_logic.rs @@ -0,0 +1,129 @@ +use crate::sea_orm::ConnectionTrait; +use sea_orm_migration::prelude::*; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + // Replace the sample below with your own migration scripts + + // Add a column "downgrade_tier_id" + // It is a "foreign key" that references other items in this table + manager + .alter_table( + Table::alter() + .table(UserTier::Table) + .add_column(ColumnDef::new(UserTier::DowngradeTierId).big_unsigned()) + .add_foreign_key( + TableForeignKey::new() + .to_tbl(UserTier::Table) + .to_tbl(UserTier::Table) + .from_col(UserTier::DowngradeTierId) + .to_col(UserTier::Id), + ) + .to_owned(), + ) + .await?; + + // Insert Premium, and PremiumOutOfFunds + let premium_out_of_funds_tier = Query::insert() + .into_table(UserTier::Table) + .columns([ + UserTier::Title, + UserTier::MaxRequestsPerPeriod, + UserTier::MaxConcurrentRequests, + UserTier::DowngradeTierId, + ]) + .values_panic([ + "Premium Out Of Funds".into(), + Some("6000").into(), + Some("5").into(), + None::.into(), + ]) + .to_owned(); + + manager.exec_stmt(premium_out_of_funds_tier).await?; + + // Insert Premium Out Of Funds + // get the premium tier ... + let db_conn = manager.get_connection(); + let db_backend = manager.get_database_backend(); + + let select_premium_out_of_funds_tier_id = Query::select() + .column(UserTier::Id) + .from(UserTier::Table) + .cond_where(Expr::col(UserTier::Title).eq("Premium Out Of Funds")) + .to_owned(); + let premium_out_of_funds_tier_id: u64 = db_conn + .query_one(db_backend.build(&select_premium_out_of_funds_tier_id)) + .await? + .expect("we just created Premium Out Of Funds") + .try_get("", &UserTier::Id.to_string())?; + + // Add two tiers for premium: premium, and premium-out-of-funds + let premium_tier = Query::insert() + .into_table(UserTier::Table) + .columns([ + UserTier::Title, + UserTier::MaxRequestsPerPeriod, + UserTier::MaxConcurrentRequests, + UserTier::DowngradeTierId, + ]) + .values_panic([ + "Premium".into(), + None::<&str>.into(), + Some("100").into(), + Some(premium_out_of_funds_tier_id).into(), + ]) + .to_owned(); + + manager.exec_stmt(premium_tier).await + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + // Replace the sample below with your own migration scripts + + // Remove the two tiers that you just added + // And remove the column you just added + let db_conn = manager.get_connection(); + let db_backend = manager.get_database_backend(); + + let delete_premium = Query::delete() + .from_table(UserTier::Table) + .cond_where(Expr::col(UserTier::Title).eq("Premium")) + .to_owned(); + + db_conn.execute(db_backend.build(&delete_premium)).await?; + + let delete_premium_out_of_funds = Query::delete() + .from_table(UserTier::Table) + .cond_where(Expr::col(UserTier::Title).eq("Premium Out Of Funds")) + .to_owned(); + + db_conn + .execute(db_backend.build(&delete_premium_out_of_funds)) + .await?; + + // Finally drop the downgrade column + manager + .alter_table( + Table::alter() + .table(UserTier::Table) + .drop_column(UserTier::DowngradeTierId) + .to_owned(), + ) + .await + } +} + +#[derive(Iden)] +enum UserTier { + Table, + Id, + Title, + MaxRequestsPerPeriod, + MaxConcurrentRequests, + DowngradeTierId, +} diff --git a/migration/src/m20230511_161214_remove_columns_statsv2_origin_and_method.rs b/migration/src/m20230511_161214_remove_columns_statsv2_origin_and_method.rs new file mode 100644 index 00000000..0dc736c1 --- /dev/null +++ b/migration/src/m20230511_161214_remove_columns_statsv2_origin_and_method.rs @@ -0,0 +1,50 @@ +use sea_orm_migration::prelude::*; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + Table::alter() + .table(RpcAccountingV2::Table) + .drop_column(RpcAccountingV2::Origin) + .drop_column(RpcAccountingV2::Method) + .to_owned(), + ) + .await + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + Table::alter() + .table(RpcAccountingV2::Table) + .add_column( + ColumnDef::new(RpcAccountingV2::Method) + .string() + .not_null() + .default(""), + ) + .add_column( + ColumnDef::new(RpcAccountingV2::Origin) + .string() + .not_null() + .default(""), + ) + .to_owned(), + ) + .await + } +} + +/// Learn more at https://docs.rs/sea-query#iden +#[derive(Iden)] +enum RpcAccountingV2 { + Table, + Id, + Origin, + Method, +} diff --git a/scripts/brownie-tests/.gitattributes b/scripts/brownie-tests/.gitattributes new file mode 100644 index 00000000..adb20fe2 --- /dev/null +++ b/scripts/brownie-tests/.gitattributes @@ -0,0 +1,2 @@ +*.sol linguist-language=Solidity +*.vy linguist-language=Python diff --git a/scripts/brownie-tests/.gitignore b/scripts/brownie-tests/.gitignore new file mode 100644 index 00000000..898188a7 --- /dev/null +++ b/scripts/brownie-tests/.gitignore @@ -0,0 +1,6 @@ +__pycache__ +.env +.history +.hypothesis/ +build/ +reports/ diff --git a/scripts/brownie-tests/brownie-config.yaml b/scripts/brownie-tests/brownie-config.yaml new file mode 100644 index 00000000..a3e30288 --- /dev/null +++ b/scripts/brownie-tests/brownie-config.yaml @@ -0,0 +1 @@ +dotenv: .env \ No newline at end of file diff --git a/scripts/brownie-tests/scripts/make_payment.py b/scripts/brownie-tests/scripts/make_payment.py new file mode 100644 index 00000000..97bd49f5 --- /dev/null +++ b/scripts/brownie-tests/scripts/make_payment.py @@ -0,0 +1,34 @@ +from brownie import Contract, Sweeper, accounts +from brownie.network import priority_fee + +def main(): + print("Hello") + + + print("accounts are") + token = Contract.from_explorer("0xC9fCFA7e28fF320C49967f4522EBc709aa1fDE7c") + factory = Contract.from_explorer("0x4e3bc2054788de923a04936c6addb99a05b0ea36") + user = accounts.load("david") + # user = accounts.load("david-main") + + print("Llama token") + print(token) + + print("Factory token") + print(factory) + + print("User addr") + print(user) + + # Sweeper and Proxy are deployed by us, as the user, by calling factory + # Already been called before ... + # factory.create_payment_address({'from': user}) + sweeper = Sweeper.at(factory.account_to_payment_address(user)) + print("Sweeper is at") + print(sweeper) + + priority_fee("auto") + token._mint_for_testing(user, (10_000)*(10**18), {'from': user}) + # token.approve(sweeper, 2**256-1, {'from': user}) + sweeper.send_token(token, (5_000)*(10**18), {'from': user}) + # sweeper.send_token(token, (47)*(10**13), {'from': user}) diff --git a/scripts/get-stats-aggregated.sh b/scripts/get-stats-aggregated.sh index c1811988..3fcddb16 100644 --- a/scripts/get-stats-aggregated.sh +++ b/scripts/get-stats-aggregated.sh @@ -6,5 +6,6 @@ curl -X GET \ "http://localhost:8544/user/stats/aggregate?query_start=1678780033&query_window_seconds=1000" -#curl -X GET \ -#"http://localhost:8544/user/stats/detailed?query_start=1678780033&query_window_seconds=1000" +curl -X GET \ +-H "Authorization: Bearer 01GZK8MHHGQWK4VPGF97HS91MB" \ +"http://localhost:8544/user/stats/detailed?query_start=1678780033&query_window_seconds=1000" diff --git a/scripts/manual-tests/12-subusers-premium-account.sh b/scripts/manual-tests/12-subusers-premium-account.sh new file mode 100644 index 00000000..9980a3bd --- /dev/null +++ b/scripts/manual-tests/12-subusers-premium-account.sh @@ -0,0 +1,110 @@ +### Tests subuser premium account endpoints +################## +# Run the server +################## +# Run the proxyd instance +cargo run --release -- proxyd + +# Check if the instance is running +curl -X POST -H "Content-Type: application/json" --data '{"jsonrpc":"2.0","method":"web3_clientVersion","id":1}' 127.0.0.1:8544 + + +################## +# Create the premium / primary user & log in (Wallet 0xeB3E928A2E54BE013EF8241d4C9EaF4DfAE94D5a) +################## +cargo run create_user --address 0xeB3E928A2E54BE013EF8241d4C9EaF4DfAE94D5a + +# Make user premium, so he can create subusers +cargo run change_user_tier_by_address 0xeB3E928A2E54BE013EF8241d4C9EaF4DfAE94D5a "Unlimited" +# could also use CLI to change user role +# ULID 01GXRAGS5F9VJFQRVMZGE1Q85T +# UUID 018770a8-64af-4ee4-fbe3-74fc1c1ba0ba + +# Open this website to get the nonce to log in, sign the message, and paste the payload in the endpoint that follows it +http://127.0.0.1:8544/user/login/0xeB3E928A2E54BE013EF8241d4C9EaF4DfAE94D5a +https://www.myetherwallet.com/wallet/sign + +http://127.0.0.1:8544/user/login/0xeB3E928A2E54BE013EF8241d4C9EaF4DfAE94D5a +https://www.myetherwallet.com/wallet/sign + +# Use this site to sign a message +curl -X POST http://127.0.0.1:8544/user/login \ + -H 'Content-Type: application/json' \ + -d '{ + "address": "0x762390ae7a3c4d987062a398c1ea8767029ab08e", + "msg": "0x6c6c616d616e6f6465732e636f6d2077616e747320796f7520746f207369676e20696e207769746820796f757220457468657265756d206163636f756e743a0a3078373632333930616537613363344439383730363261333938433165413837363730323941423038450a0af09fa699f09fa699f09fa699f09fa699f09fa6990a0a5552493a2068747470733a2f2f6c6c616d616e6f6465732e636f6d2f0a56657273696f6e3a20310a436861696e2049443a20310a4e6f6e63653a203031475a484e4350315a57345134305a384b4e4e304454564a320a4973737565642041743a20323032332d30352d30335432303a33383a31392e3435363231345a0a45787069726174696f6e2054696d653a20323032332d30352d30335432303a35383a31392e3435363231345a", + "sig": "82d2ee89fb6075bdc57fa66db4e0b2b84ad0b6515e1b3d71bb1dd4e6f1711b2f0f6b5f5e40116fd51e609bc8b4c0642f4cdaaf96a6c48e66093fe153d4e2873f1c", + "version": "3", + "signer": "MEW" + }' + +# Bearer token is: 01GZHMCXHXHPGAABAQQTXKMSM3 +# RPC secret key is: 01GZHMCXGXT5Z4M8SCKCMKDAZ6 + +# 01GZHND8E5BYRVPXXMKPQ75RJ1 +# 01GZHND83W8VAHCZWEPP1AA24M + +# Top up the balance of the account +curl \ +-H "Authorization: Bearer 01GZHMCXHXHPGAABAQQTXKMSM3" \ +-X GET "127.0.0.1:8544/user/balance/0x749788a5766577431a0a4fc8721fd7cb981f55222e073ed17976f0aba5e8818a" + + +# Make an example RPC request to check if the tokens work +curl \ + -X POST "127.0.0.1:8544/rpc/01GZHMCXGXT5Z4M8SCKCMKDAZ6" \ + -H "Content-Type: application/json" \ + --data '{"method":"eth_blockNumber","params":[],"id":1,"jsonrpc":"2.0"}' + +################## +# Now act as the subuser (Wallet 0x762390ae7a3c4D987062a398C1eA8767029AB08E) +# We first login the subuser +################## +# Login using the referral link. This should create the user, and also mark him as being referred +# http://127.0.0.1:8544/user/login/0x762390ae7a3c4D987062a398C1eA8767029AB08E +# https://www.myetherwallet.com/wallet/sign +curl -X POST http://127.0.0.1:8544/user/login \ + -H 'Content-Type: application/json' \ + -d '{ + "address": "0x762390ae7a3c4d987062a398c1ea8767029ab08e", + "msg": "0x6c6c616d616e6f6465732e636f6d2077616e747320796f7520746f207369676e20696e207769746820796f757220457468657265756d206163636f756e743a0a3078373632333930616537613363344439383730363261333938433165413837363730323941423038450a0af09fa699f09fa699f09fa699f09fa699f09fa6990a0a5552493a2068747470733a2f2f6c6c616d616e6f6465732e636f6d2f0a56657273696f6e3a20310a436861696e2049443a20310a4e6f6e63653a20303147585246454b5654334d584531334b5956443159323853460a4973737565642041743a20323032332d30342d31315431353a33373a34382e3636373438315a0a45787069726174696f6e2054696d653a20323032332d30342d31315431353a35373a34382e3636373438315a", + "sig": "1784c968fdc244248a4c0b8d52158ff773e044646d6e5ce61d457679d740566b66fd16ad24777f09c971e2c3dfa74966ffb8c083a9bef2a527e49bc3770713431c", + "version": "3", + "signer": "MEW", + "referral_code": "llamanodes-01GXRB6RVM00MACTKABYVF8MJR" + }' + +# Bearer token 01GXRFKFQXDV0MQ2RT52BCPZ23 +# RPC key 01GXRFKFPY5DDRCRVB3B3HVDYK + +################## +# Now the primary user adds the secondary user as a subuser +################## +# Get first users RPC keys +curl \ +-H "Authorization: Bearer 01GXRB6AHZSXFDX2S1QJPJ8X51" \ +-X GET "127.0.0.1:8544/user/keys" + +# Secret key +curl \ + -X GET "127.0.0.1:8544/user/subuser?subuser_address=0x762390ae7a3c4D987062a398C1eA8767029AB08E&rpc_key=01GZHMCXGXT5Z4M8SCKCMKDAZ6&new_status=upsert&new_role=admin" \ + -H "Content-Type: application/json" \ + -H "Authorization: Bearer 01GZHMCXHXHPGAABAQQTXKMSM3" + +# The primary user can check what subusers he gave access to +curl \ + -X GET "127.0.0.1:8544/user/subusers?rpc_key=01GZHMCXGXT5Z4M8SCKCMKDAZ6" \ + -H "Content-Type: application/json" \ + -H "Authorization: Bearer 01GZHMCXHXHPGAABAQQTXKMSM3" + +# The secondary user can see all the projects that he is associated with +curl \ + -X GET "127.0.0.1:8544/subuser/rpc_keys" \ + -H "Content-Type: application/json" \ + -H "Authorization: Bearer 01GXRFKFQXDV0MQ2RT52BCPZ23" + +# Secret key +curl \ + -X GET "127.0.0.1:8544/user/subuser?subuser_address=0x762390ae7a3c4D987062a398C1eA8767029AB08E&rpc_key=01GXRFKFPY5DDRCRVB3B3HVDYK&new_status=remove&new_role=collaborator" \ + -H "Content-Type: application/json" \ + -H "Authorization: Bearer 01GXRFKFQXDV0MQ2RT52BCPZ23" \ No newline at end of file diff --git a/scripts/manual-tests/16-change-user-tier.sh b/scripts/manual-tests/16-change-user-tier.sh index 8012cb9b..64484d53 100644 --- a/scripts/manual-tests/16-change-user-tier.sh +++ b/scripts/manual-tests/16-change-user-tier.sh @@ -3,14 +3,14 @@ # sea-orm-cli migrate up # Use CLI to create the admin that will call the endpoint -RUSTFLAGS="--cfg tokio_unstable" cargo run create_user --address 0xeB3E928A2E54BE013EF8241d4C9EaF4DfAE94D5a -RUSTFLAGS="--cfg tokio_unstable" cargo run change_admin_status 0xeB3E928A2E54BE013EF8241d4C9EaF4DfAE94D5a true +cargo run create_user --address 0xeB3E928A2E54BE013EF8241d4C9EaF4DfAE94D5a +cargo run change_admin_status 0xeB3E928A2E54BE013EF8241d4C9EaF4DfAE94D5a true # Use CLI to create the user whose role will be changed via the endpoint -RUSTFLAGS="--cfg tokio_unstable" cargo run create_user --address 0x077e43dcca20da9859daa3fd78b5998b81f794f7 +cargo run create_user --address 0x077e43dcca20da9859daa3fd78b5998b81f794f7 # Run the proxyd instance -RUSTFLAGS="--cfg tokio_unstable" cargo run --release -- proxyd +cargo run --release -- proxyd # Check if the instance is running curl --verbose -X POST -H "Content-Type: application/json" --data '{"jsonrpc":"2.0","method":"web3_clientVersion","id":1}' 127.0.0.1:8544 diff --git a/scripts/manual-tests/24-simple-referral-program.sh b/scripts/manual-tests/24-simple-referral-program.sh new file mode 100644 index 00000000..cec54ad6 --- /dev/null +++ b/scripts/manual-tests/24-simple-referral-program.sh @@ -0,0 +1,111 @@ +################## +# Run the server +################## + +# Keep the proxyd instance running the background (and test that it works) +cargo run --release -- proxyd + +# Check if the instance is running +curl -X POST -H "Content-Type: application/json" --data '{"jsonrpc":"2.0","method":"web3_clientVersion","id":1}' 127.0.0.1:8544 + +################## +# Create the referring user & log in (Wallet 0xeB3E928A2E54BE013EF8241d4C9EaF4DfAE94D5a) +################## +cargo run create_user --address 0xeB3E928A2E54BE013EF8241d4C9EaF4DfAE94D5a + +# Make user premium, so he can create referral keys +cargo run change_user_tier_by_address 0xeB3E928A2E54BE013EF8241d4C9EaF4DfAE94D5a "Unlimited" +# could also use CLI to change user role +# ULID 01GXRAGS5F9VJFQRVMZGE1Q85T +# UUID 018770a8-64af-4ee4-fbe3-74fc1c1ba0ba + +# Open this website to get the nonce to log in, sign the message, and paste the payload in the endpoint that follows it +http://127.0.0.1:8544/user/login/0xeB3E928A2E54BE013EF8241d4C9EaF4DfAE94D5a +https://www.myetherwallet.com/wallet/sign + +# Use this site to sign a message +curl -X POST http://127.0.0.1:8544/user/login \ + -H 'Content-Type: application/json' \ + -d '{ + "address": "0xeb3e928a2e54be013ef8241d4c9eaf4dfae94d5a", + "msg": "0x6c6c616d616e6f6465732e636f6d2077616e747320796f7520746f207369676e20696e207769746820796f757220457468657265756d206163636f756e743a0a3078654233453932384132453534424530313345463832343164344339456146344466414539344435610a0af09fa699f09fa699f09fa699f09fa699f09fa6990a0a5552493a2068747470733a2f2f6c6c616d616e6f6465732e636f6d2f0a56657273696f6e3a20310a436861696e2049443a20310a4e6f6e63653a2030314758524235424a584b47535845454b5a314438424857565a0a4973737565642041743a20323032332d30342d31315431343a32323a35302e3937333930365a0a45787069726174696f6e2054696d653a20323032332d30342d31315431343a34323a35302e3937333930365a", + "sig": "be1f9fed3f6f206c15677b7da488071b936b68daf560715b75cf9232afe4b9923c2c5d00a558847131f0f04200b4b123011f62521b7b97bab2c8b794c82b29621b", + "version": "3", + "signer": "MEW" + }' + +# Bearer token is: 01GXRB6AHZSXFDX2S1QJPJ8X51 +# RPC secret key is: 01GXRAGS5F9VJFQRVMZGE1Q85T + +# Make an example RPC request to check if the tokens work +curl \ + -X POST "127.0.0.1:8544/rpc/01GXRAGS5F9VJFQRVMZGE1Q85T" \ + -H "Content-Type: application/json" \ + --data '{"method":"eth_blockNumber","params":[],"id":1,"jsonrpc":"2.0"}' + +# Now retrieve the referral link +curl \ +-H "Authorization: Bearer 01GXRB6AHZSXFDX2S1QJPJ8X51" \ +-X GET "127.0.0.1:8544/user/referral" + +# This is the referral code which will be used by the redeemer +# "llamanodes-01GXRB6RVM00MACTKABYVF8MJR" + +################## +# Now act as the referrer (Wallet 0x762390ae7a3c4D987062a398C1eA8767029AB08E) +# We first login the referrer +# Using the referrer code creates an entry in the table +################## +# Login using the referral link. This should create the user, and also mark him as being referred +# http://127.0.0.1:8544/user/login/0x762390ae7a3c4D987062a398C1eA8767029AB08E +# https://www.myetherwallet.com/wallet/sign +curl -X POST http://127.0.0.1:8544/user/login \ + -H 'Content-Type: application/json' \ + -d '{ + "address": "0x762390ae7a3c4d987062a398c1ea8767029ab08e", + "msg": "0x6c6c616d616e6f6465732e636f6d2077616e747320796f7520746f207369676e20696e207769746820796f757220457468657265756d206163636f756e743a0a3078373632333930616537613363344439383730363261333938433165413837363730323941423038450a0af09fa699f09fa699f09fa699f09fa699f09fa6990a0a5552493a2068747470733a2f2f6c6c616d616e6f6465732e636f6d2f0a56657273696f6e3a20310a436861696e2049443a20310a4e6f6e63653a20303147585246454b5654334d584531334b5956443159323853460a4973737565642041743a20323032332d30342d31315431353a33373a34382e3636373438315a0a45787069726174696f6e2054696d653a20323032332d30342d31315431353a35373a34382e3636373438315a", + "sig": "1784c968fdc244248a4c0b8d52158ff773e044646d6e5ce61d457679d740566b66fd16ad24777f09c971e2c3dfa74966ffb8c083a9bef2a527e49bc3770713431c", + "version": "3", + "signer": "MEW", + "referral_code": "llamanodes-01GXRB6RVM00MACTKABYVF8MJR" + }' + +# Bearer token 01GXRFKFQXDV0MQ2RT52BCPZ23 +# RPC key 01GXRFKFPY5DDRCRVB3B3HVDYK + +# Make some requests, the referrer should not receive any credits for this (balance table is not created for free-tier users ...) This works fine +for i in {1..1000} +do + curl \ + -X POST "127.0.0.1:8544/rpc/01GXRFKFPY5DDRCRVB3B3HVDYK" \ + -H "Content-Type: application/json" \ + --data '{"method":"eth_blockNumber","params":[],"id":1,"jsonrpc":"2.0"}' +done + +########################################### +# Now the referred user deposits some tokens +# They then send it to the endpoint +########################################### +curl \ +-H "Authorization: Bearer 01GXRFKFQXDV0MQ2RT52BCPZ23" \ +-X GET "127.0.0.1:8544/user/balance/0xda41f748106d2d1f1bf395e65d07bd9fc507c1eb4fd50c87d8ca1f34cfd536b0" + +curl \ +-H "Authorization: Bearer 01GXRFKFQXDV0MQ2RT52BCPZ23" \ +-X GET "127.0.0.1:8544/user/balance/0xd56dee328dfa3bea26c3762834081881e5eff62e77a2b45e72d98016daaeffba" + + +########################################### +# Now the referred user starts spending the money. Let's make requests worth $100 and see what happens ... +# At all times, the referrer should receive 10% of the spent tokens +########################################### +for i in {1..10000000} +do + curl \ + -X POST "127.0.0.1:8544/rpc/01GXRFKFPY5DDRCRVB3B3HVDYK" \ + -H "Content-Type: application/json" \ + --data '{"method":"eth_blockNumber","params":[],"id":1,"jsonrpc":"2.0"}' +done + +# Check that the new user was indeed logged in, and that a referral table entry was created (in the database) +# Check that the 10% referral rate works diff --git a/scripts/manual-tests/42-simple-balance.sh b/scripts/manual-tests/42-simple-balance.sh new file mode 100644 index 00000000..ce6e32da --- /dev/null +++ b/scripts/manual-tests/42-simple-balance.sh @@ -0,0 +1,86 @@ +################## +# Run the server +################## +# Run the proxyd instance +cargo run --release -- proxyd + +# Check if the instance is running +curl -X POST -H "Content-Type: application/json" --data '{"jsonrpc":"2.0","method":"web3_clientVersion","id":1}' 127.0.0.1:8544 + +########################## +# Create a User & Log in +########################## +cargo run create_user --address 0x762390ae7a3c4D987062a398C1eA8767029AB08E +# ULID: 01GXEDC66Z9RZE6AE22JE7FRAW +# UUID: 01875cd6-18df-4e3e-e329-c2149c77e15c + +# Log in as the user so we can check the balance +# Open this website to get the nonce to log in +curl -X GET "http://127.0.0.1:8544/user/login/0xeb3e928a2e54be013ef8241d4c9eaf4dfae94d5a" + +# Use this site to sign a message +# https://www.myetherwallet.com/wallet/sign (whatever is output with the above code) +curl -X POST http://127.0.0.1:8544/user/login \ + -H 'Content-Type: application/json' \ + -d '{ + "address": "0xeb3e928a2e54be013ef8241d4c9eaf4dfae94d5a", + "msg": "0x6c6c616d616e6f6465732e636f6d2077616e747320796f7520746f207369676e20696e207769746820796f757220457468657265756d206163636f756e743a0a3078654233453932384132453534424530313345463832343164344339456146344466414539344435610a0af09fa699f09fa699f09fa699f09fa699f09fa6990a0a5552493a2068747470733a2f2f6c6c616d616e6f6465732e636f6d2f0a56657273696f6e3a20310a436861696e2049443a20310a4e6f6e63653a203031475a4b384b4847305259474737514e5132475037464444470a4973737565642041743a20323032332d30352d30345431313a33333a32312e3533363734355a0a45787069726174696f6e2054696d653a20323032332d30352d30345431313a35333a32312e3533363734355a", + "sig": "cebd9effff15f4517e53522dbe91798d59dc0df0299faaec25d3f6443fa121f847e4311d5ca7386e75b87d6d45df92b8ced58c822117519c666ab1a6b2fc7bd21b", + "version": "3", + "signer": "MEW" + }' + +# bearer token is: 01GZK8MHHGQWK4VPGF97HS91MB +# scret key is: 01GZK65YNV0P0WN2SCXYTW3R9S + +# 01GZH2PS89EJJY6V8JFCVTQ4BX +# 01GZH2PS7CTHA3TAZ4HXCTX6KQ + +########################################### +# Initially check balance, it should be 0 +########################################### +# Check the balance of the user +# Balance seems to be returning properly (0, in this test case) +curl \ +-H "Authorization: Bearer 01GZK8MHHGQWK4VPGF97HS91MB" \ +-X GET "127.0.0.1:8544/user/balance" + + +########################################### +# The user submits a transaction on the matic network +# and submits it on the endpoint +########################################### +curl \ +-H "Authorization: Bearer 01GZK65YRW69KZECCGPSQH2XYK" \ +-X GET "127.0.0.1:8544/user/balance/0x749788a5766577431a0a4fc8721fd7cb981f55222e073ed17976f0aba5e8818a" + +########################################### +# Check the balance again, it should have increased according to how much USDC was spent +########################################### +# Check the balance of the user +# Balance seems to be returning properly (0, in this test case) +curl \ +-H "Authorization: Bearer 01GZGGDBMV0GM6MFBBHPDE78BW" \ +-X GET "127.0.0.1:8544/user/balance" + +# TODO: Now start using the RPC, balance should decrease + +# Get the RPC key +curl \ + -X GET "127.0.0.1:8544/user/keys" \ + -H "Authorization: Bearer 01GZGGDBMV0GM6MFBBHPDE78BW" \ + --data '{"method":"eth_blockNumber","params":[],"id":1,"jsonrpc":"2.0"}' + +## Check if calling an RPC endpoint logs the stats +## This one does already even it seems +for i in {1..100} +do + curl \ + -X POST "127.0.0.1:8544/rpc/01GZK65YNV0P0WN2SCXYTW3R9S" \ + -H "Content-Type: application/json" \ + --data '{"method":"eth_blockNumber","params":[],"id":1,"jsonrpc":"2.0"}' +done + + +# TODO: Now implement and test withdrawal + diff --git a/scripts/manual-tests/48-balance-downgrade.sh b/scripts/manual-tests/48-balance-downgrade.sh new file mode 100644 index 00000000..80cc4cfd --- /dev/null +++ b/scripts/manual-tests/48-balance-downgrade.sh @@ -0,0 +1,88 @@ +################## +# Run the server +################## +# Run the proxyd instance +cargo run --release -- proxyd + +# Check if the instance is running +curl -X POST -H "Content-Type: application/json" --data '{"jsonrpc":"2.0","method":"web3_clientVersion","id":1}' 127.0.0.1:8544 + +########################## +# Create a User & Log in +########################## +#cargo run create_user --address 0x762390ae7a3c4D987062a398C1eA8767029AB08E +# ULID: 01GXEDC66Z9RZE6AE22JE7FRAW +# UUID: 01875cd6-18df-4e3e-e329-c2149c77e15c + +# Log in as the user so we can check the balance +# Open this website to get the nonce to log in +curl -X GET "http://127.0.0.1:8544/user/login/0xeB3E928A2E54BE013EF8241d4C9EaF4DfAE94D5a" + +# Use this site to sign a message +# https://www.myetherwallet.com/wallet/sign (whatever is output with the above code) +curl -X POST http://127.0.0.1:8544/user/login \ + -H 'Content-Type: application/json' \ + -d '{ + "address": "0xeb3e928a2e54be013ef8241d4c9eaf4dfae94d5a", + "msg": "0x6c6c616d616e6f6465732e636f6d2077616e747320796f7520746f207369676e20696e207769746820796f757220457468657265756d206163636f756e743a0a3078654233453932384132453534424530313345463832343164344339456146344466414539344435610a0af09fa699f09fa699f09fa699f09fa699f09fa6990a0a5552493a2068747470733a2f2f6c6c616d616e6f6465732e636f6d2f0a56657273696f6e3a20310a436861696e2049443a20310a4e6f6e63653a2030314759513445564731474b34314b42364130324a344b45384b0a4973737565642041743a20323032332d30342d32335431333a32323a30392e3533373932365a0a45787069726174696f6e2054696d653a20323032332d30342d32335431333a34323a30392e3533373932365a", + "sig": "52071cc59afb427eb554126f4f9f2a445c2a539783ba45079ccc0911197062f135d6d347cf0c38fa078dc2369c32b5131b86811fc0916786d1e48252163f58131c", + "version": "3", + "signer": "MEW" + }' + +# bearer token is: 01GYQ4FMRKKWJEA2YBST3B89MJ +# scret key is: 01GYQ4FMNX9EMFBT43XEFGZV1K + +########################################### +# Initially check balance, it should be 0 +########################################### +# Check the balance of the user +# Balance seems to be returning properly (0, in this test case) +curl \ +-H "Authorization: Bearer 01GYQ4FMRKKWJEA2YBST3B89MJ" \ +-X GET "127.0.0.1:8544/user/balance" + + +########################################### +# The user submits a transaction on the matic network +# and submits it on the endpoint +########################################### +curl \ +-H "Authorization: Bearer 01GYQ4FMRKKWJEA2YBST3B89MJ" \ +-X GET "127.0.0.1:8544/user/balance/0x749788a5766577431a0a4fc8721fd7cb981f55222e073ed17976f0aba5e8818a" + +########################################### +# Check the balance again, it should have increased according to how much USDC was spent +########################################### +# Check the balance of the user +# Balance seems to be returning properly (0, in this test case) +curl \ +-H "Authorization: Bearer 01GYQ4FMRKKWJEA2YBST3B89MJ" \ +-X GET "127.0.0.1:8544/user/balance" + +# Get the RPC key +curl \ + -X GET "127.0.0.1:8544/user/keys" \ + -H "Authorization: Bearer 01GYQ4FMRKKWJEA2YBST3B89MJ" + +## Check if calling an RPC endpoint logs the stats +## This one does already even it seems +for i in {1..100000} +do + curl \ + -X POST "127.0.0.1:8544/rpc/01GZHMCXGXT5Z4M8SCKCMKDAZ6" \ + -H "Content-Type: application/json" \ + --data '{"method":"eth_blockNumber","params":[],"id":1,"jsonrpc":"2.0"}' +done + +for i in {1..100} +do + curl \ + -X POST "127.0.0.1:8544/" \ + -H "Content-Type: application/json" \ + --data '{"method":"eth_blockNumber","params":[],"id":1,"jsonrpc":"2.0"}' +done + + +# TODO: Now implement and test withdrawal + diff --git a/scripts/manual-tests/52-simple-get-deposits.sh b/scripts/manual-tests/52-simple-get-deposits.sh new file mode 100644 index 00000000..9cd7432e --- /dev/null +++ b/scripts/manual-tests/52-simple-get-deposits.sh @@ -0,0 +1,5 @@ +# Check the balance of the user +# Balance seems to be returning properly (0, in this test case) +curl \ +-H "Authorization: Bearer 01GZHMCXHXHPGAABAQQTXKMSM3" \ +-X GET "127.0.0.1:8544/user/deposits" diff --git a/scripts/requirements.txt b/scripts/requirements.txt new file mode 100644 index 00000000..3e6653f0 --- /dev/null +++ b/scripts/requirements.txt @@ -0,0 +1,4 @@ +python-dotenv +eth-brownie +ensurepath +brownie-token-tester \ No newline at end of file diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index 0f8d811b..e8d65868 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -47,11 +47,12 @@ gethostname = "0.4.2" glob = "0.3.1" handlebars = "4.3.7" hashbrown = { version = "0.13.2", features = ["serde"] } +hex_fmt = "0.3.0" hdrhistogram = "7.5.2" http = "0.2.9" +influxdb2 = { git = "https://github.com/llamanodes/influxdb2", features = ["rustls"] } +influxdb2-structmap = { git = "https://github.com/llamanodes/influxdb2/"} hostname = "0.3.1" -influxdb2 = { version = "0.4", features = ["rustls"] } -influxdb2-structmap = "0.2.0" ipnet = "2.7.2" itertools = "0.10.5" log = "0.4.17" @@ -82,6 +83,6 @@ tokio-uring = { version = "0.4.0", optional = true } toml = "0.7.3" tower = "0.4.13" tower-http = { version = "0.4.0", features = ["cors", "sensitive-headers"] } -ulid = { version = "1.0.0", features = ["serde"] } +ulid = { version = "1.0.0", features = ["uuid", "serde"] } url = "2.3.1" uuid = "1.3.2" diff --git a/web3_proxy/src/app/mod.rs b/web3_proxy/src/app/mod.rs index 03c7df72..46072197 100644 --- a/web3_proxy/src/app/mod.rs +++ b/web3_proxy/src/app/mod.rs @@ -33,6 +33,7 @@ use futures::stream::{FuturesUnordered, StreamExt}; use hashbrown::{HashMap, HashSet}; use ipnet::IpNet; use log::{debug, error, info, trace, warn, Level}; +use migration::sea_orm::prelude::Decimal; use migration::sea_orm::{ self, ConnectionTrait, Database, DatabaseConnection, EntityTrait, PaginatorTrait, }; @@ -189,6 +190,7 @@ pub struct AuthorizationChecks { /// IMPORTANT! Once confirmed by a miner, they will be public on the blockchain! pub private_txs: bool, pub proxy_mode: ProxyMode, + pub balance: Option, } /// Simple wrapper so that we can keep track of read only connections. @@ -579,6 +581,15 @@ impl Web3ProxyApp { None => None, }; + // 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 + let rpc_secret_key_cache = Cache::builder() + .max_capacity(10_000) + .time_to_live(Duration::from_secs(600)) + .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); + // 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 @@ -589,6 +600,7 @@ impl Web3ProxyApp { influxdb_bucket, db_conn.clone(), influxdb_client.clone(), + Some(rpc_secret_key_cache.clone()), 60, 1, BILLING_PERIOD_SECONDS, @@ -699,15 +711,6 @@ impl Web3ProxyApp { .time_to_live(Duration::from_secs(600)) .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); - // 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 - let rpc_secret_key_cache = Cache::builder() - .max_capacity(10_000) - .time_to_live(Duration::from_secs(600)) - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); - // create semaphores for concurrent connection limits // TODO: what should tti be for semaphores? let bearer_token_semaphores = Cache::builder() diff --git a/web3_proxy/src/bin/web3_proxy_cli/migrate_stats_to_v2.rs b/web3_proxy/src/bin/web3_proxy_cli/migrate_stats_to_v2.rs index bdd8350f..17ad9370 100644 --- a/web3_proxy/src/bin/web3_proxy_cli/migrate_stats_to_v2.rs +++ b/web3_proxy/src/bin/web3_proxy_cli/migrate_stats_to_v2.rs @@ -76,6 +76,7 @@ impl MigrateStatsToV2 { .context("No influxdb bucket was provided")?, Some(db_conn.clone()), influxdb_client.clone(), + None, 30, 1, BILLING_PERIOD_SECONDS, diff --git a/web3_proxy/src/config.rs b/web3_proxy/src/config.rs index 68bed7a0..05a947d5 100644 --- a/web3_proxy/src/config.rs +++ b/web3_proxy/src/config.rs @@ -2,7 +2,7 @@ use crate::app::AnyhowJoinHandle; use crate::rpcs::blockchain::{BlocksByHashCache, Web3ProxyBlock}; use crate::rpcs::one::Web3Rpc; use argh::FromArgs; -use ethers::prelude::TxHash; +use ethers::prelude::{Address, TxHash, H256}; use ethers::types::{U256, U64}; use hashbrown::HashMap; use log::warn; @@ -94,6 +94,12 @@ pub struct AppConfig { /// None = allow all requests pub default_user_max_requests_per_period: Option, + /// Default ERC address for out deposit contract + pub deposit_factory_contract: Option
, + + /// Default ERC address for out deposit contract + pub deposit_topic: Option, + /// minimum amount to increase eth_estimateGas results pub gas_increase_min: Option, diff --git a/web3_proxy/src/frontend/authorization.rs b/web3_proxy/src/frontend/authorization.rs index 39e0c9c1..97672da5 100644 --- a/web3_proxy/src/frontend/authorization.rs +++ b/web3_proxy/src/frontend/authorization.rs @@ -10,7 +10,7 @@ use axum::headers::{Header, Origin, Referer, UserAgent}; use chrono::Utc; use deferred_rate_limiter::DeferredRateLimitResult; use entities::sea_orm_active_enums::TrackingLevel; -use entities::{login, rpc_key, user, user_tier}; +use entities::{balance, login, rpc_key, user, user_tier}; use ethers::types::Bytes; use ethers::utils::keccak256; use futures::TryFutureExt; @@ -689,6 +689,13 @@ impl Web3ProxyApp { .await? .expect("related user"); + let balance = balance::Entity::find() + .filter(balance::Column::UserId.eq(user_model.id)) + .one(db_replica.conn()) + .await? + .expect("related balance") + .available_balance; + let user_tier_model = user_tier::Entity::find_by_id(user_model.user_tier_id) .one(db_replica.conn()) @@ -771,6 +778,7 @@ impl Web3ProxyApp { max_requests_per_period: user_tier_model.max_requests_per_period, private_txs: rpc_key_model.private_txs, proxy_mode, + balance: Some(balance), }) } None => Ok(AuthorizationChecks::default()), diff --git a/web3_proxy/src/frontend/errors.rs b/web3_proxy/src/frontend/errors.rs index 8911e8ea..1785975d 100644 --- a/web3_proxy/src/frontend/errors.rs +++ b/web3_proxy/src/frontend/errors.rs @@ -56,6 +56,7 @@ pub enum Web3ProxyError { InvalidHeaderValue(InvalidHeaderValue), InvalidEip, InvalidInviteCode, + InvalidReferralCode, InvalidReferer, InvalidSignatureLength, InvalidUserAgent, @@ -118,6 +119,7 @@ pub enum Web3ProxyError { #[error(ignore)] UserAgentNotAllowed(headers::UserAgent), UserIdZero, + PaymentRequired, VerificationError(siwe::VerificationError), WatchRecvError(tokio::sync::watch::error::RecvError), WatchSendError, @@ -353,6 +355,17 @@ impl Web3ProxyError { ), ) } + Self::InvalidReferralCode => { + warn!("InvalidReferralCode"); + ( + StatusCode::UNAUTHORIZED, + JsonRpcForwardedResponse::from_str( + "invalid referral code", + Some(StatusCode::UNAUTHORIZED.as_u16().into()), + None, + ), + ) + } Self::InvalidReferer => { warn!("InvalidReferer"); ( @@ -574,6 +587,17 @@ impl Web3ProxyError { ), ) } + Self::PaymentRequired => { + trace!("PaymentRequiredError"); + ( + StatusCode::PAYMENT_REQUIRED, + JsonRpcForwardedResponse::from_str( + "Payment is required and user is not premium.", + Some(StatusCode::PAYMENT_REQUIRED.as_u16().into()), + None, + ), + ) + } // TODO: this should actually by the id of the key. multiple users might control one key Self::RateLimited(authorization, retry_at) => { // TODO: emit a stat diff --git a/web3_proxy/src/frontend/mod.rs b/web3_proxy/src/frontend/mod.rs index 88388efa..9715c111 100644 --- a/web3_proxy/src/frontend/mod.rs +++ b/web3_proxy/src/frontend/mod.rs @@ -168,30 +168,58 @@ pub async fn serve( // // User stuff // - .route("/user/login/:user_address", get(users::user_login_get)) + .route( + "/user/login/:user_address", + get(users::authentication::user_login_get), + ) .route( "/user/login/:user_address/:message_eip", - get(users::user_login_get), + get(users::authentication::user_login_get), + ) + .route("/user/login", post(users::authentication::user_login_post)) + .route( + // /:rpc_key/:subuser_address/:new_status/:new_role + "/user/subuser", + get(users::subuser::modify_subuser), + ) + .route("/user/subusers", get(users::subuser::get_subusers)) + .route( + "/subuser/rpc_keys", + get(users::subuser::get_keys_as_subuser), ) - .route("/user/login", post(users::user_login_post)) .route("/user", get(users::user_get)) .route("/user", post(users::user_post)) - .route("/user/balance", get(users::user_balance_get)) - .route("/user/balance/:txid", post(users::user_balance_post)) - .route("/user/keys", get(users::rpc_keys_get)) - .route("/user/keys", post(users::rpc_keys_management)) - .route("/user/keys", put(users::rpc_keys_management)) - .route("/user/revert_logs", get(users::user_revert_logs_get)) + .route("/user/balance", get(users::payment::user_balance_get)) + .route("/user/deposits", get(users::payment::user_deposits_get)) + .route( + "/user/balance/:tx_hash", + get(users::payment::user_balance_post), + ) + .route("/user/keys", get(users::rpc_keys::rpc_keys_get)) + .route("/user/keys", post(users::rpc_keys::rpc_keys_management)) + .route("/user/keys", put(users::rpc_keys::rpc_keys_management)) + // .route("/user/referral/:referral_link", get(users::user_referral_link_get)) + .route( + "/user/referral", + get(users::referral::user_referral_link_get), + ) + .route("/user/revert_logs", get(users::stats::user_revert_logs_get)) .route( "/user/stats/aggregate", - get(users::user_stats_aggregated_get), + get(users::stats::user_stats_aggregated_get), ) .route( "/user/stats/aggregated", - get(users::user_stats_aggregated_get), + get(users::stats::user_stats_aggregated_get), + ) + .route( + "/user/stats/detailed", + get(users::stats::user_stats_detailed_get), + ) + .route( + "/user/logout", + post(users::authentication::user_logout_post), ) - .route("/user/stats/detailed", get(users::user_stats_detailed_get)) - .route("/user/logout", post(users::user_logout_post)) .route("/admin/modify_role", get(admin::admin_change_user_roles)) .route( "/admin/imitate-login/:admin_address/:user_address", diff --git a/web3_proxy/src/frontend/users.rs b/web3_proxy/src/frontend/users.rs deleted file mode 100644 index c405b075..00000000 --- a/web3_proxy/src/frontend/users.rs +++ /dev/null @@ -1,838 +0,0 @@ -//! Handle registration, logins, and managing account data. -use super::authorization::{login_is_authorized, RpcSecretKey}; -use super::errors::{Web3ProxyError, Web3ProxyErrorContext, Web3ProxyResponse}; -use crate::app::Web3ProxyApp; -use crate::http_params::{ - get_chain_id_from_params, get_page_from_params, get_query_start_from_params, -}; -use crate::stats::influxdb_queries::query_user_stats; -use crate::stats::StatType; -use crate::user_token::UserBearerToken; -use crate::{PostLogin, PostLoginQuery}; -use axum::headers::{Header, Origin, Referer, UserAgent}; -use axum::{ - extract::{Path, Query}, - headers::{authorization::Bearer, Authorization}, - response::IntoResponse, - Extension, Json, TypedHeader, -}; -use axum_client_ip::InsecureClientIp; -use axum_macros::debug_handler; -use chrono::{TimeZone, Utc}; -use entities::sea_orm_active_enums::TrackingLevel; -use entities::{login, pending_login, revert_log, rpc_key, user}; -use ethers::{prelude::Address, types::Bytes}; -use hashbrown::HashMap; -use http::{HeaderValue, StatusCode}; -use ipnet::IpNet; -use itertools::Itertools; -use log::{debug, warn}; -use migration::sea_orm::prelude::Uuid; -use migration::sea_orm::{ - self, ActiveModelTrait, ColumnTrait, EntityTrait, IntoActiveModel, PaginatorTrait, QueryFilter, - QueryOrder, TransactionTrait, TryIntoModel, -}; -use serde::Deserialize; -use serde_json::json; -use siwe::{Message, VerificationOpts}; -use std::ops::Add; -use std::str::FromStr; -use std::sync::Arc; -use time::{Duration, OffsetDateTime}; -use ulid::Ulid; - -/// `GET /user/login/:user_address` or `GET /user/login/:user_address/:message_eip` -- Start the "Sign In with Ethereum" (siwe) login flow. -/// -/// `message_eip`s accepted: -/// - eip191_bytes -/// - eip191_hash -/// - eip4361 (default) -/// -/// Coming soon: eip1271 -/// -/// This is the initial entrypoint for logging in. Take the response from this endpoint and give it to your user's wallet for singing. POST the response to `/user/login`. -/// -/// Rate limited by IP address. -/// -/// At first i thought about checking that user_address is in our db, -/// But theres no need to separate the registration and login flows. -/// It is a better UX to just click "login with ethereum" and have the account created if it doesn't exist. -/// We can prompt for an email and and payment after they log in. -#[debug_handler] -pub async fn user_login_get( - Extension(app): Extension>, - InsecureClientIp(ip): InsecureClientIp, - // TODO: what does axum's error handling look like if the path fails to parse? - Path(mut params): Path>, -) -> Web3ProxyResponse { - login_is_authorized(&app, ip).await?; - - // create a message and save it in redis - // TODO: how many seconds? get from config? - let expire_seconds: usize = 20 * 60; - - let nonce = Ulid::new(); - - let issued_at = OffsetDateTime::now_utc(); - - let expiration_time = issued_at.add(Duration::new(expire_seconds as i64, 0)); - - // TODO: allow ENS names here? - let user_address: Address = params - .remove("user_address") - .ok_or(Web3ProxyError::BadRouting)? - .parse() - .or(Err(Web3ProxyError::ParseAddressError))?; - - let login_domain = app - .config - .login_domain - .clone() - .unwrap_or_else(|| "llamanodes.com".to_string()); - - // TODO: get most of these from the app config - let message = Message { - // TODO: don't unwrap - // TODO: accept a login_domain from the request? - domain: login_domain.parse().unwrap(), - address: user_address.to_fixed_bytes(), - // TODO: config for statement - statement: Some("🦙🦙🦙🦙🦙".to_string()), - // TODO: don't unwrap - uri: format!("https://{}/", login_domain).parse().unwrap(), - version: siwe::Version::V1, - chain_id: 1, - expiration_time: Some(expiration_time.into()), - issued_at: issued_at.into(), - nonce: nonce.to_string(), - not_before: None, - request_id: None, - resources: vec![], - }; - - let db_conn = app.db_conn().web3_context("login requires a database")?; - - // massage types to fit in the database. sea-orm does not make this very elegant - let uuid = Uuid::from_u128(nonce.into()); - // we add 1 to expire_seconds just to be sure the database has the key for the full expiration_time - let expires_at = Utc - .timestamp_opt(expiration_time.unix_timestamp() + 1, 0) - .unwrap(); - - // we do not store a maximum number of attempted logins. anyone can request so we don't want to allow DOS attacks - // add a row to the database for this user - let user_pending_login = pending_login::ActiveModel { - id: sea_orm::NotSet, - nonce: sea_orm::Set(uuid), - message: sea_orm::Set(message.to_string()), - expires_at: sea_orm::Set(expires_at), - imitating_user: sea_orm::Set(None), - }; - - user_pending_login - .save(&db_conn) - .await - .web3_context("saving user's pending_login")?; - - // there are multiple ways to sign messages and not all wallets support them - // TODO: default message eip from config? - let message_eip = params - .remove("message_eip") - .unwrap_or_else(|| "eip4361".to_string()); - - let message: String = match message_eip.as_str() { - "eip191_bytes" => Bytes::from(message.eip191_bytes().unwrap()).to_string(), - "eip191_hash" => Bytes::from(&message.eip191_hash().unwrap()).to_string(), - "eip4361" => message.to_string(), - _ => { - return Err(Web3ProxyError::InvalidEip); - } - }; - - Ok(message.into_response()) -} - -/// `POST /user/login` - Register or login by posting a signed "siwe" message. -/// It is recommended to save the returned bearer token in a cookie. -/// The bearer token can be used to authenticate other requests, such as getting the user's stats or modifying the user's profile. -#[debug_handler] -pub async fn user_login_post( - Extension(app): Extension>, - InsecureClientIp(ip): InsecureClientIp, - Query(query): Query, - Json(payload): Json, -) -> Web3ProxyResponse { - login_is_authorized(&app, ip).await?; - - // TODO: this seems too verbose. how can we simply convert a String into a [u8; 65] - let their_sig_bytes = Bytes::from_str(&payload.sig).web3_context("parsing sig")?; - if their_sig_bytes.len() != 65 { - return Err(Web3ProxyError::InvalidSignatureLength); - } - let mut their_sig: [u8; 65] = [0; 65]; - for x in 0..65 { - their_sig[x] = their_sig_bytes[x] - } - - // we can't trust that they didn't tamper with the message in some way. like some clients return it hex encoded - // TODO: checking 0x seems fragile, but I think it will be fine. siwe message text shouldn't ever start with 0x - let their_msg: Message = if payload.msg.starts_with("0x") { - let their_msg_bytes = - Bytes::from_str(&payload.msg).web3_context("parsing payload message")?; - - // TODO: lossy or no? - String::from_utf8_lossy(their_msg_bytes.as_ref()) - .parse::() - .web3_context("parsing hex string message")? - } else { - payload - .msg - .parse::() - .web3_context("parsing string message")? - }; - - // the only part of the message we will trust is their nonce - // TODO: this is fragile. have a helper function/struct for redis keys - let login_nonce = UserBearerToken::from_str(&their_msg.nonce)?; - - // fetch the message we gave them from our database - let db_replica = app - .db_replica() - .web3_context("Getting database connection")?; - - // massage type for the db - let login_nonce_uuid: Uuid = login_nonce.clone().into(); - - let user_pending_login = pending_login::Entity::find() - .filter(pending_login::Column::Nonce.eq(login_nonce_uuid)) - .one(db_replica.conn()) - .await - .web3_context("database error while finding pending_login")? - .web3_context("login nonce not found")?; - - let our_msg: siwe::Message = user_pending_login - .message - .parse() - .web3_context("parsing siwe message")?; - - // default options are fine. the message includes timestamp and domain and nonce - let verify_config = VerificationOpts::default(); - - // Check with both verify and verify_eip191 - if let Err(err_1) = our_msg - .verify(&their_sig, &verify_config) - .await - .web3_context("verifying signature against our local message") - { - // verification method 1 failed. try eip191 - if let Err(err_191) = our_msg - .verify_eip191(&their_sig) - .web3_context("verifying eip191 signature against our local message") - { - let db_conn = app - .db_conn() - .web3_context("deleting expired pending logins requires a db")?; - - // delete ALL expired rows. - let now = Utc::now(); - let delete_result = pending_login::Entity::delete_many() - .filter(pending_login::Column::ExpiresAt.lte(now)) - .exec(&db_conn) - .await?; - - // TODO: emit a stat? if this is high something weird might be happening - debug!("cleared expired pending_logins: {:?}", delete_result); - - return Err(Web3ProxyError::EipVerificationFailed( - Box::new(err_1), - Box::new(err_191), - )); - } - } - - // TODO: limit columns or load whole user? - let u = user::Entity::find() - .filter(user::Column::Address.eq(our_msg.address.as_ref())) - .one(db_replica.conn()) - .await - .unwrap(); - - let db_conn = app.db_conn().web3_context("login requires a db")?; - - let (u, uks, status_code) = match u { - None => { - // user does not exist yet - - // check the invite code - // TODO: more advanced invite codes that set different request/minute and concurrency limits - if let Some(invite_code) = &app.config.invite_code { - if query.invite_code.as_ref() != Some(invite_code) { - return Err(Web3ProxyError::InvalidInviteCode); - } - } - - let txn = db_conn.begin().await?; - - // the only thing we need from them is an address - // everything else is optional - // TODO: different invite codes should allow different levels - // TODO: maybe decrement a count on the invite code? - let u = user::ActiveModel { - address: sea_orm::Set(our_msg.address.into()), - ..Default::default() - }; - - let u = u.insert(&txn).await?; - - // create the user's first api key - let rpc_secret_key = RpcSecretKey::new(); - - let uk = rpc_key::ActiveModel { - user_id: sea_orm::Set(u.id), - secret_key: sea_orm::Set(rpc_secret_key.into()), - description: sea_orm::Set(None), - ..Default::default() - }; - - let uk = uk - .insert(&txn) - .await - .web3_context("Failed saving new user key")?; - - let uks = vec![uk]; - - // save the user and key to the database - txn.commit().await?; - - (u, uks, StatusCode::CREATED) - } - Some(u) => { - // the user is already registered - let uks = rpc_key::Entity::find() - .filter(rpc_key::Column::UserId.eq(u.id)) - .all(db_replica.conn()) - .await - .web3_context("failed loading user's key")?; - - (u, uks, StatusCode::OK) - } - }; - - // create a bearer token for the user. - let user_bearer_token = UserBearerToken::default(); - - // json response with everything in it - // we could return just the bearer token, but I think they will always request api keys and the user profile - let response_json = json!({ - "rpc_keys": uks - .into_iter() - .map(|uk| (uk.id, uk)) - .collect::>(), - "bearer_token": user_bearer_token, - "user": u, - }); - - let response = (status_code, Json(response_json)).into_response(); - - // add bearer to the database - - // expire in 4 weeks - let expires_at = Utc::now() - .checked_add_signed(chrono::Duration::weeks(4)) - .unwrap(); - - let user_login = login::ActiveModel { - id: sea_orm::NotSet, - bearer_token: sea_orm::Set(user_bearer_token.uuid()), - user_id: sea_orm::Set(u.id), - expires_at: sea_orm::Set(expires_at), - read_only: sea_orm::Set(false), - }; - - user_login - .save(&db_conn) - .await - .web3_context("saving user login")?; - - if let Err(err) = user_pending_login - .into_active_model() - .delete(&db_conn) - .await - { - warn!("Failed to delete nonce:{}: {}", login_nonce.0, err); - } - - Ok(response) -} - -/// `POST /user/logout` - Forget the bearer token in the `Authentication` header. -#[debug_handler] -pub async fn user_logout_post( - Extension(app): Extension>, - TypedHeader(Authorization(bearer)): TypedHeader>, -) -> Web3ProxyResponse { - let user_bearer = UserBearerToken::try_from(bearer)?; - - let db_conn = app - .db_conn() - .web3_context("database needed for user logout")?; - - if let Err(err) = login::Entity::delete_many() - .filter(login::Column::BearerToken.eq(user_bearer.uuid())) - .exec(&db_conn) - .await - { - debug!("Failed to delete {}: {}", user_bearer.redis_key(), err); - } - - let now = Utc::now(); - - // also delete any expired logins - let delete_result = login::Entity::delete_many() - .filter(login::Column::ExpiresAt.lte(now)) - .exec(&db_conn) - .await; - - debug!("Deleted expired logins: {:?}", delete_result); - - // also delete any expired pending logins - let delete_result = login::Entity::delete_many() - .filter(login::Column::ExpiresAt.lte(now)) - .exec(&db_conn) - .await; - - debug!("Deleted expired pending logins: {:?}", delete_result); - - // TODO: what should the response be? probably json something - Ok("goodbye".into_response()) -} - -/// `GET /user` -- Use a bearer token to get the user's profile. -/// -/// - the email address of a user if they opted in to get contacted via email -/// -/// TODO: this will change as we add better support for secondary users. -#[debug_handler] -pub async fn user_get( - Extension(app): Extension>, - TypedHeader(Authorization(bearer_token)): TypedHeader>, -) -> Web3ProxyResponse { - let (user, _semaphore) = app.bearer_is_authorized(bearer_token).await?; - - Ok(Json(user).into_response()) -} - -/// the JSON input to the `post_user` handler. -#[derive(Debug, Deserialize)] -pub struct UserPost { - email: Option, -} - -/// `POST /user` -- modify the account connected to the bearer token in the `Authentication` header. -#[debug_handler] -pub async fn user_post( - Extension(app): Extension>, - TypedHeader(Authorization(bearer_token)): TypedHeader>, - Json(payload): Json, -) -> Web3ProxyResponse { - let (user, _semaphore) = app.bearer_is_authorized(bearer_token).await?; - - let mut user: user::ActiveModel = user.into(); - - // update the email address - if let Some(x) = payload.email { - // TODO: only Set if no change - if x.is_empty() { - user.email = sea_orm::Set(None); - } else { - // TODO: do some basic validation - // TODO: don't set immediatly, send a confirmation email first - // TODO: compare first? or is sea orm smart enough to do that for us? - user.email = sea_orm::Set(Some(x)); - } - } - - // TODO: what else can we update here? password hash? subscription to newsletter? - - let user = if user.is_changed() { - let db_conn = app.db_conn().web3_context("Getting database connection")?; - - user.save(&db_conn).await? - } else { - // no changes. no need to touch the database - user - }; - - let user: user::Model = user.try_into().web3_context("Returning updated user")?; - - Ok(Json(user).into_response()) -} - -/// `GET /user/balance` -- Use a bearer token to get the user's balance and spend. -/// -/// - show balance in USD -/// - show deposits history (currency, amounts, transaction id) -/// -/// TODO: one key per request? maybe /user/balance/:rpc_key? -/// TODO: this will change as we add better support for secondary users. -#[debug_handler] -pub async fn user_balance_get( - Extension(app): Extension>, - TypedHeader(Authorization(bearer)): TypedHeader>, -) -> Web3ProxyResponse { - let (_user, _semaphore) = app.bearer_is_authorized(bearer).await?; - - todo!("user_balance_get"); -} - -/// `POST /user/balance/:txhash` -- Manually process a confirmed txid to update a user's balance. -/// -/// We will subscribe to events to watch for any user deposits, but sometimes events can be missed. -/// -/// TODO: change this. just have a /tx/:txhash that is open to anyone. rate limit like we rate limit /login -#[debug_handler] -pub async fn user_balance_post( - Extension(app): Extension>, - TypedHeader(Authorization(bearer)): TypedHeader>, -) -> Web3ProxyResponse { - let (_user, _semaphore) = app.bearer_is_authorized(bearer).await?; - - todo!("user_balance_post"); -} - -/// `GET /user/keys` -- Use a bearer token to get the user's api keys and their settings. -#[debug_handler] -pub async fn rpc_keys_get( - Extension(app): Extension>, - TypedHeader(Authorization(bearer)): TypedHeader>, -) -> Web3ProxyResponse { - let (user, _semaphore) = app.bearer_is_authorized(bearer).await?; - - let db_replica = app - .db_replica() - .web3_context("db_replica is required to fetch a user's keys")?; - - let uks = rpc_key::Entity::find() - .filter(rpc_key::Column::UserId.eq(user.id)) - .all(db_replica.conn()) - .await - .web3_context("failed loading user's key")?; - - let response_json = json!({ - "user_id": user.id, - "user_rpc_keys": uks - .into_iter() - .map(|uk| (uk.id, uk)) - .collect::>(), - }); - - Ok(Json(response_json).into_response()) -} - -/// `DELETE /user/keys` -- Use a bearer token to delete an existing key. -#[debug_handler] -pub async fn rpc_keys_delete( - Extension(app): Extension>, - TypedHeader(Authorization(bearer)): TypedHeader>, -) -> Web3ProxyResponse { - let (_user, _semaphore) = app.bearer_is_authorized(bearer).await?; - - // TODO: think about how cascading deletes and billing should work - Err(Web3ProxyError::NotImplemented) -} - -/// the JSON input to the `rpc_keys_management` handler. -/// If `key_id` is set, it updates an existing key. -/// If `key_id` is not set, it creates a new key. -/// `log_request_method` cannot be change once the key is created -/// `user_tier` cannot be changed here -#[derive(Debug, Deserialize)] -pub struct UserKeyManagement { - key_id: Option, - active: Option, - allowed_ips: Option, - allowed_origins: Option, - allowed_referers: Option, - allowed_user_agents: Option, - description: Option, - log_level: Option, - // TODO: enable log_revert_trace: Option, - private_txs: Option, -} - -/// `POST /user/keys` or `PUT /user/keys` -- Use a bearer token to create or update an existing key. -#[debug_handler] -pub async fn rpc_keys_management( - Extension(app): Extension>, - TypedHeader(Authorization(bearer)): TypedHeader>, - Json(payload): Json, -) -> Web3ProxyResponse { - // TODO: is there a way we can know if this is a PUT or POST? right now we can modify or create keys with either. though that probably doesn't matter - - let (user, _semaphore) = app.bearer_is_authorized(bearer).await?; - - let db_replica = app - .db_replica() - .web3_context("getting db for user's keys")?; - - let mut uk = if let Some(existing_key_id) = payload.key_id { - // get the key and make sure it belongs to the user - rpc_key::Entity::find() - .filter(rpc_key::Column::UserId.eq(user.id)) - .filter(rpc_key::Column::Id.eq(existing_key_id)) - .one(db_replica.conn()) - .await - .web3_context("failed loading user's key")? - .web3_context("key does not exist or is not controlled by this bearer token")? - .into_active_model() - } else { - // make a new key - // TODO: limit to 10 keys? - let secret_key = RpcSecretKey::new(); - - let log_level = payload - .log_level - .web3_context("log level must be 'none', 'detailed', or 'aggregated'")?; - - rpc_key::ActiveModel { - user_id: sea_orm::Set(user.id), - secret_key: sea_orm::Set(secret_key.into()), - log_level: sea_orm::Set(log_level), - ..Default::default() - } - }; - - // TODO: do we need null descriptions? default to empty string should be fine, right? - if let Some(description) = payload.description { - if description.is_empty() { - uk.description = sea_orm::Set(None); - } else { - uk.description = sea_orm::Set(Some(description)); - } - } - - if let Some(private_txs) = payload.private_txs { - uk.private_txs = sea_orm::Set(private_txs); - } - - if let Some(active) = payload.active { - uk.active = sea_orm::Set(active); - } - - if let Some(allowed_ips) = payload.allowed_ips { - if allowed_ips.is_empty() { - uk.allowed_ips = sea_orm::Set(None); - } else { - // split allowed ips on ',' and try to parse them all. error on invalid input - let allowed_ips = allowed_ips - .split(',') - .map(|x| x.trim().parse::()) - .collect::, _>>()? - // parse worked. convert back to Strings - .into_iter() - .map(|x| x.to_string()); - - // and join them back together - let allowed_ips: String = - Itertools::intersperse(allowed_ips, ", ".to_string()).collect(); - - uk.allowed_ips = sea_orm::Set(Some(allowed_ips)); - } - } - - // TODO: this should actually be bytes - if let Some(allowed_origins) = payload.allowed_origins { - if allowed_origins.is_empty() { - uk.allowed_origins = sea_orm::Set(None); - } else { - // split allowed_origins on ',' and try to parse them all. error on invalid input - let allowed_origins = allowed_origins - .split(',') - .map(|x| HeaderValue::from_str(x.trim())) - .collect::, _>>()? - .into_iter() - .map(|x| Origin::decode(&mut [x].iter())) - .collect::, _>>()? - // parse worked. convert back to String and join them back together - .into_iter() - .map(|x| x.to_string()); - - let allowed_origins: String = - Itertools::intersperse(allowed_origins, ", ".to_string()).collect(); - - uk.allowed_origins = sea_orm::Set(Some(allowed_origins)); - } - } - - // TODO: this should actually be bytes - if let Some(allowed_referers) = payload.allowed_referers { - if allowed_referers.is_empty() { - uk.allowed_referers = sea_orm::Set(None); - } else { - // split allowed ips on ',' and try to parse them all. error on invalid input - let allowed_referers = allowed_referers - .split(',') - .map(|x| HeaderValue::from_str(x.trim())) - .collect::, _>>()? - .into_iter() - .map(|x| Referer::decode(&mut [x].iter())) - .collect::, _>>()?; - - // parse worked. now we can put it back together. - // but we can't go directly to String. - // so we convert to HeaderValues first - let mut header_map = vec![]; - for x in allowed_referers { - x.encode(&mut header_map); - } - - // convert HeaderValues to Strings - // since we got these from strings, this should always work (unless we figure out using bytes) - let allowed_referers = header_map - .into_iter() - .map(|x| x.to_str().map(|x| x.to_string())) - .collect::, _>>()?; - - // join strings together with commas - let allowed_referers: String = - Itertools::intersperse(allowed_referers.into_iter(), ", ".to_string()).collect(); - - uk.allowed_referers = sea_orm::Set(Some(allowed_referers)); - } - } - - if let Some(allowed_user_agents) = payload.allowed_user_agents { - if allowed_user_agents.is_empty() { - uk.allowed_user_agents = sea_orm::Set(None); - } else { - // split allowed_user_agents on ',' and try to parse them all. error on invalid input - let allowed_user_agents = allowed_user_agents - .split(',') - .filter_map(|x| x.trim().parse::().ok()) - // parse worked. convert back to String - .map(|x| x.to_string()); - - // join the strings together - let allowed_user_agents: String = - Itertools::intersperse(allowed_user_agents, ", ".to_string()).collect(); - - uk.allowed_user_agents = sea_orm::Set(Some(allowed_user_agents)); - } - } - - let uk = if uk.is_changed() { - let db_conn = app.db_conn().web3_context("login requires a db")?; - - uk.save(&db_conn) - .await - .web3_context("Failed saving user key")? - } else { - uk - }; - - let uk = uk.try_into_model()?; - - Ok(Json(uk).into_response()) -} - -/// `GET /user/revert_logs` -- Use a bearer token to get the user's revert logs. -#[debug_handler] -pub async fn user_revert_logs_get( - Extension(app): Extension>, - TypedHeader(Authorization(bearer)): TypedHeader>, - Query(params): Query>, -) -> Web3ProxyResponse { - let (user, _semaphore) = app.bearer_is_authorized(bearer).await?; - - let chain_id = get_chain_id_from_params(app.as_ref(), ¶ms)?; - let query_start = get_query_start_from_params(¶ms)?; - let page = get_page_from_params(¶ms)?; - - // TODO: page size from config - let page_size = 1_000; - - let mut response = HashMap::new(); - - response.insert("page", json!(page)); - response.insert("page_size", json!(page_size)); - response.insert("chain_id", json!(chain_id)); - response.insert("query_start", json!(query_start.timestamp() as u64)); - - let db_replica = app - .db_replica() - .web3_context("getting replica db for user's revert logs")?; - - let uks = rpc_key::Entity::find() - .filter(rpc_key::Column::UserId.eq(user.id)) - .all(db_replica.conn()) - .await - .web3_context("failed loading user's key")?; - - // TODO: only select the ids - let uks: Vec<_> = uks.into_iter().map(|x| x.id).collect(); - - // get revert logs - let mut q = revert_log::Entity::find() - .filter(revert_log::Column::Timestamp.gte(query_start)) - .filter(revert_log::Column::RpcKeyId.is_in(uks)) - .order_by_asc(revert_log::Column::Timestamp); - - if chain_id == 0 { - // don't do anything - } else { - // filter on chain id - q = q.filter(revert_log::Column::ChainId.eq(chain_id)) - } - - // query the database for number of items and pages - let pages_result = q - .clone() - .paginate(db_replica.conn(), page_size) - .num_items_and_pages() - .await?; - - response.insert("num_items", pages_result.number_of_items.into()); - response.insert("num_pages", pages_result.number_of_pages.into()); - - // query the database for the revert logs - let revert_logs = q - .paginate(db_replica.conn(), page_size) - .fetch_page(page) - .await?; - - response.insert("revert_logs", json!(revert_logs)); - - Ok(Json(response).into_response()) -} - -/// `GET /user/stats/aggregate` -- Public endpoint for aggregate stats such as bandwidth used and methods requested. -#[debug_handler] -pub async fn user_stats_aggregated_get( - Extension(app): Extension>, - bearer: Option>>, - Query(params): Query>, -) -> Web3ProxyResponse { - let response = query_user_stats(&app, bearer, ¶ms, StatType::Aggregated).await?; - - Ok(response) -} - -/// `GET /user/stats/detailed` -- Use a bearer token to get the user's key stats such as bandwidth used and methods requested. -/// -/// If no bearer is provided, detailed stats for all users will be shown. -/// View a single user with `?user_id=$x`. -/// View a single chain with `?chain_id=$x`. -/// -/// Set `$x` to zero to see all. -/// -/// TODO: this will change as we add better support for secondary users. -#[debug_handler] -pub async fn user_stats_detailed_get( - Extension(app): Extension>, - bearer: Option>>, - Query(params): Query>, -) -> Web3ProxyResponse { - let response = query_user_stats(&app, bearer, ¶ms, StatType::Detailed).await?; - - Ok(response) -} diff --git a/web3_proxy/src/frontend/users/authentication.rs b/web3_proxy/src/frontend/users/authentication.rs new file mode 100644 index 00000000..f70b63e9 --- /dev/null +++ b/web3_proxy/src/frontend/users/authentication.rs @@ -0,0 +1,473 @@ +//! Handle registration, logins, and managing account data. +use crate::app::Web3ProxyApp; +use crate::frontend::authorization::{login_is_authorized, RpcSecretKey}; +use crate::frontend::errors::{Web3ProxyError, Web3ProxyErrorContext, Web3ProxyResponse}; +use crate::user_token::UserBearerToken; +use crate::{PostLogin, PostLoginQuery}; +use axum::{ + extract::{Path, Query}, + headers::{authorization::Bearer, Authorization}, + response::IntoResponse, + Extension, Json, TypedHeader, +}; +use axum_client_ip::InsecureClientIp; +use axum_macros::debug_handler; +use chrono::{TimeZone, Utc}; +use entities; +use entities::{balance, login, pending_login, referee, referrer, rpc_key, user}; +use ethers::{prelude::Address, types::Bytes}; +use hashbrown::HashMap; +use http::StatusCode; +use log::{debug, warn}; +use migration::sea_orm::prelude::{Decimal, Uuid}; +use migration::sea_orm::{ + self, ActiveModelTrait, ColumnTrait, EntityTrait, IntoActiveModel, QueryFilter, + TransactionTrait, +}; +use serde_json::json; +use siwe::{Message, VerificationOpts}; +use std::ops::Add; +use std::str::FromStr; +use std::sync::Arc; +use time::{Duration, OffsetDateTime}; +use ulid::Ulid; + +/// `GET /user/login/:user_address` or `GET /user/login/:user_address/:message_eip` -- Start the "Sign In with Ethereum" (siwe) login flow. +/// +/// `message_eip`s accepted: +/// - eip191_bytes +/// - eip191_hash +/// - eip4361 (default) +/// +/// Coming soon: eip1271 +/// +/// This is the initial entrypoint for logging in. Take the response from this endpoint and give it to your user's wallet for singing. POST the response to `/user/login`. +/// +/// Rate limited by IP address. +/// +/// At first i thought about checking that user_address is in our db, +/// But theres no need to separate the registration and login flows. +/// It is a better UX to just click "login with ethereum" and have the account created if it doesn't exist. +/// We can prompt for an email and and payment after they log in. +#[debug_handler] +pub async fn user_login_get( + Extension(app): Extension>, + InsecureClientIp(ip): InsecureClientIp, + // TODO: what does axum's error handling look like if the path fails to parse? + Path(mut params): Path>, +) -> Web3ProxyResponse { + login_is_authorized(&app, ip).await?; + + // create a message and save it in redis + // TODO: how many seconds? get from config? + let expire_seconds: usize = 20 * 60; + + let nonce = Ulid::new(); + + let issued_at = OffsetDateTime::now_utc(); + + let expiration_time = issued_at.add(Duration::new(expire_seconds as i64, 0)); + + // TODO: allow ENS names here? + let user_address: Address = params + .remove("user_address") + .ok_or(Web3ProxyError::BadRouting)? + .parse() + .or(Err(Web3ProxyError::ParseAddressError))?; + + let login_domain = app + .config + .login_domain + .clone() + .unwrap_or_else(|| "llamanodes.com".to_string()); + + // TODO: get most of these from the app config + let message = Message { + // TODO: don't unwrap + // TODO: accept a login_domain from the request? + domain: login_domain.parse().unwrap(), + address: user_address.to_fixed_bytes(), + // TODO: config for statement + statement: Some("🦙🦙🦙🦙🦙".to_string()), + // TODO: don't unwrap + uri: format!("https://{}/", login_domain).parse().unwrap(), + version: siwe::Version::V1, + chain_id: 1, + expiration_time: Some(expiration_time.into()), + issued_at: issued_at.into(), + nonce: nonce.to_string(), + not_before: None, + request_id: None, + resources: vec![], + }; + + let db_conn = app.db_conn().web3_context("login requires a database")?; + + // massage types to fit in the database. sea-orm does not make this very elegant + let uuid = Uuid::from_u128(nonce.into()); + // we add 1 to expire_seconds just to be sure the database has the key for the full expiration_time + let expires_at = Utc + .timestamp_opt(expiration_time.unix_timestamp() + 1, 0) + .unwrap(); + + // we do not store a maximum number of attempted logins. anyone can request so we don't want to allow DOS attacks + // add a row to the database for this user + let user_pending_login = pending_login::ActiveModel { + id: sea_orm::NotSet, + nonce: sea_orm::Set(uuid), + message: sea_orm::Set(message.to_string()), + expires_at: sea_orm::Set(expires_at), + imitating_user: sea_orm::Set(None), + }; + + user_pending_login + .save(&db_conn) + .await + .web3_context("saving user's pending_login")?; + + // there are multiple ways to sign messages and not all wallets support them + // TODO: default message eip from config? + let message_eip = params + .remove("message_eip") + .unwrap_or_else(|| "eip4361".to_string()); + + let message: String = match message_eip.as_str() { + "eip191_bytes" => Bytes::from(message.eip191_bytes().unwrap()).to_string(), + "eip191_hash" => Bytes::from(&message.eip191_hash().unwrap()).to_string(), + "eip4361" => message.to_string(), + _ => { + return Err(Web3ProxyError::InvalidEip); + } + }; + + Ok(message.into_response()) +} + +/// `POST /user/login` - Register or login by posting a signed "siwe" message. +/// It is recommended to save the returned bearer token in a cookie. +/// The bearer token can be used to authenticate other requests, such as getting the user's stats or modifying the user's profile. +#[debug_handler] +pub async fn user_login_post( + Extension(app): Extension>, + InsecureClientIp(ip): InsecureClientIp, + Query(query): Query, + Json(payload): Json, +) -> Web3ProxyResponse { + login_is_authorized(&app, ip).await?; + + // TODO: this seems too verbose. how can we simply convert a String into a [u8; 65] + let their_sig_bytes = Bytes::from_str(&payload.sig).web3_context("parsing sig")?; + if their_sig_bytes.len() != 65 { + return Err(Web3ProxyError::InvalidSignatureLength); + } + let mut their_sig: [u8; 65] = [0; 65]; + for x in 0..65 { + their_sig[x] = their_sig_bytes[x] + } + + // we can't trust that they didn't tamper with the message in some way. like some clients return it hex encoded + // TODO: checking 0x seems fragile, but I think it will be fine. siwe message text shouldn't ever start with 0x + let their_msg: Message = if payload.msg.starts_with("0x") { + let their_msg_bytes = + Bytes::from_str(&payload.msg).web3_context("parsing payload message")?; + + // TODO: lossy or no? + String::from_utf8_lossy(their_msg_bytes.as_ref()) + .parse::() + .web3_context("parsing hex string message")? + } else { + payload + .msg + .parse::() + .web3_context("parsing string message")? + }; + + // the only part of the message we will trust is their nonce + // TODO: this is fragile. have a helper function/struct for redis keys + let login_nonce = UserBearerToken::from_str(&their_msg.nonce)?; + + // fetch the message we gave them from our database + let db_replica = app + .db_replica() + .web3_context("Getting database connection")?; + + // massage type for the db + let login_nonce_uuid: Uuid = login_nonce.clone().into(); + + let user_pending_login = pending_login::Entity::find() + .filter(pending_login::Column::Nonce.eq(login_nonce_uuid)) + .one(db_replica.conn()) + .await + .web3_context("database error while finding pending_login")? + .web3_context("login nonce not found")?; + + let our_msg: siwe::Message = user_pending_login + .message + .parse() + .web3_context("parsing siwe message")?; + + // default options are fine. the message includes timestamp and domain and nonce + let verify_config = VerificationOpts::default(); + + // Check with both verify and verify_eip191 + if let Err(err_1) = our_msg + .verify(&their_sig, &verify_config) + .await + .web3_context("verifying signature against our local message") + { + // verification method 1 failed. try eip191 + if let Err(err_191) = our_msg + .verify_eip191(&their_sig) + .web3_context("verifying eip191 signature against our local message") + { + let db_conn = app + .db_conn() + .web3_context("deleting expired pending logins requires a db")?; + + // delete ALL expired rows. + let now = Utc::now(); + let delete_result = pending_login::Entity::delete_many() + .filter(pending_login::Column::ExpiresAt.lte(now)) + .exec(&db_conn) + .await?; + + // TODO: emit a stat? if this is high something weird might be happening + debug!("cleared expired pending_logins: {:?}", delete_result); + + return Err(Web3ProxyError::EipVerificationFailed( + Box::new(err_1), + Box::new(err_191), + )); + } + } + + // TODO: limit columns or load whole user? + let caller = user::Entity::find() + .filter(user::Column::Address.eq(our_msg.address.as_ref())) + .one(db_replica.conn()) + .await?; + + let db_conn = app.db_conn().web3_context("login requires a db")?; + + let (caller, user_rpc_keys, status_code) = match caller { + None => { + // user does not exist yet + + // check the invite code + // TODO: more advanced invite codes that set different request/minute and concurrency limits + // Do nothing if app config is none (then there is basically no authentication invitation, and the user can process with a free tier ... + + // Prematurely return if there is a wrong invite code + if let Some(invite_code) = &app.config.invite_code { + if query.invite_code.as_ref() != Some(invite_code) { + return Err(Web3ProxyError::InvalidInviteCode); + } + } + + let txn = db_conn.begin().await?; + + // First add a user + + // the only thing we need from them is an address + // everything else is optional + // TODO: different invite codes should allow different levels + // TODO: maybe decrement a count on the invite code? + // TODO: There will be two different transactions. The first one inserts the user, the second one marks the user as being referred + let caller = user::ActiveModel { + address: sea_orm::Set(our_msg.address.into()), + ..Default::default() + }; + + let caller = caller.insert(&txn).await?; + + // create the user's first api key + let rpc_secret_key = RpcSecretKey::new(); + + let user_rpc_key = rpc_key::ActiveModel { + user_id: sea_orm::Set(caller.id.clone()), + secret_key: sea_orm::Set(rpc_secret_key.into()), + description: sea_orm::Set(None), + ..Default::default() + }; + + let user_rpc_key = user_rpc_key + .insert(&txn) + .await + .web3_context("Failed saving new user key")?; + + // We should also create the balance entry ... + let user_balance = balance::ActiveModel { + user_id: sea_orm::Set(caller.id.clone()), + available_balance: sea_orm::Set(Decimal::new(0, 0)), + used_balance: sea_orm::Set(Decimal::new(0, 0)), + ..Default::default() + }; + user_balance.insert(&txn).await?; + + let user_rpc_keys = vec![user_rpc_key]; + + // Also add a part for the invite code, i.e. who invited this guy + + // save the user and key to the database + txn.commit().await?; + + let txn = db_conn.begin().await?; + // First, optionally catch a referral code from the parameters if there is any + debug!("Refferal code is: {:?}", payload.referral_code); + if let Some(referral_code) = payload.referral_code.as_ref() { + // If it is not inside, also check in the database + warn!("Using register referral code: {:?}", referral_code); + let user_referrer = referrer::Entity::find() + .filter(referrer::Column::ReferralCode.eq(referral_code)) + .one(db_replica.conn()) + .await? + .ok_or(Web3ProxyError::InvalidReferralCode)?; + + // Create a new item in the database, + // marking this guy as the referrer (and ignoring a duplicate insert, if there is any...) + // First person to make the referral gets all credits + // Generate a random referral code ... + let used_referral = referee::ActiveModel { + used_referral_code: sea_orm::Set(user_referrer.id), + user_id: sea_orm::Set(caller.id), + credits_applied_for_referee: sea_orm::Set(false), + credits_applied_for_referrer: sea_orm::Set(Decimal::new(0, 10)), + ..Default::default() + }; + used_referral.insert(&txn).await?; + } + txn.commit().await?; + + (caller, user_rpc_keys, StatusCode::CREATED) + } + Some(caller) => { + // Let's say that a user that exists can actually also redeem a key in retrospect... + let txn = db_conn.begin().await?; + // TODO: Move this into a common variable outside ... + // First, optionally catch a referral code from the parameters if there is any + if let Some(referral_code) = payload.referral_code.as_ref() { + // If it is not inside, also check in the database + warn!("Using referral code: {:?}", referral_code); + let user_referrer = referrer::Entity::find() + .filter(referrer::Column::ReferralCode.eq(referral_code)) + .one(db_replica.conn()) + .await? + .ok_or(Web3ProxyError::BadRequest(format!( + "The referral_link you provided does not exist {}", + referral_code + )))?; + + // Create a new item in the database, + // marking this guy as the referrer (and ignoring a duplicate insert, if there is any...) + // First person to make the referral gets all credits + // Generate a random referral code ... + let used_referral = referee::ActiveModel { + used_referral_code: sea_orm::Set(user_referrer.id), + user_id: sea_orm::Set(caller.id), + credits_applied_for_referee: sea_orm::Set(false), + credits_applied_for_referrer: sea_orm::Set(Decimal::new(0, 10)), + ..Default::default() + }; + used_referral.insert(&txn).await?; + } + txn.commit().await?; + + // the user is already registered + let user_rpc_keys = rpc_key::Entity::find() + .filter(rpc_key::Column::UserId.eq(caller.id)) + .all(db_replica.conn()) + .await + .web3_context("failed loading user's key")?; + + (caller, user_rpc_keys, StatusCode::OK) + } + }; + + // create a bearer token for the user. + let user_bearer_token = UserBearerToken::default(); + + // json response with everything in it + // we could return just the bearer token, but I think they will always request api keys and the user profile + let response_json = json!({ + "rpc_keys": user_rpc_keys + .into_iter() + .map(|user_rpc_key| (user_rpc_key.id, user_rpc_key)) + .collect::>(), + "bearer_token": user_bearer_token, + "user": caller, + }); + + let response = (status_code, Json(response_json)).into_response(); + + // add bearer to the database + + // expire in 4 weeks + let expires_at = Utc::now() + .checked_add_signed(chrono::Duration::weeks(4)) + .unwrap(); + + let user_login = login::ActiveModel { + id: sea_orm::NotSet, + bearer_token: sea_orm::Set(user_bearer_token.uuid()), + user_id: sea_orm::Set(caller.id), + expires_at: sea_orm::Set(expires_at), + read_only: sea_orm::Set(false), + }; + + user_login + .save(&db_conn) + .await + .web3_context("saving user login")?; + + if let Err(err) = user_pending_login + .into_active_model() + .delete(&db_conn) + .await + { + warn!("Failed to delete nonce:{}: {}", login_nonce.0, err); + } + + Ok(response) +} + +/// `POST /user/logout` - Forget the bearer token in the `Authentication` header. +#[debug_handler] +pub async fn user_logout_post( + Extension(app): Extension>, + TypedHeader(Authorization(bearer)): TypedHeader>, +) -> Web3ProxyResponse { + let user_bearer = UserBearerToken::try_from(bearer)?; + + let db_conn = app + .db_conn() + .web3_context("database needed for user logout")?; + + if let Err(err) = login::Entity::delete_many() + .filter(login::Column::BearerToken.eq(user_bearer.uuid())) + .exec(&db_conn) + .await + { + debug!("Failed to delete {}: {}", user_bearer.redis_key(), err); + } + + let now = Utc::now(); + + // also delete any expired logins + let delete_result = login::Entity::delete_many() + .filter(login::Column::ExpiresAt.lte(now)) + .exec(&db_conn) + .await; + + debug!("Deleted expired logins: {:?}", delete_result); + + // also delete any expired pending logins + let delete_result = login::Entity::delete_many() + .filter(login::Column::ExpiresAt.lte(now)) + .exec(&db_conn) + .await; + + debug!("Deleted expired pending logins: {:?}", delete_result); + + // TODO: what should the response be? probably json something + Ok("goodbye".into_response()) +} diff --git a/web3_proxy/src/frontend/users/mod.rs b/web3_proxy/src/frontend/users/mod.rs new file mode 100644 index 00000000..06269c7b --- /dev/null +++ b/web3_proxy/src/frontend/users/mod.rs @@ -0,0 +1,83 @@ +//! Handle registration, logins, and managing account data. +pub mod authentication; +pub mod payment; +pub mod referral; +pub mod rpc_keys; +pub mod stats; +pub mod subuser; + +use super::errors::{Web3ProxyErrorContext, Web3ProxyResponse}; +use crate::app::Web3ProxyApp; + +use axum::{ + headers::{authorization::Bearer, Authorization}, + response::IntoResponse, + Extension, Json, TypedHeader, +}; +use axum_macros::debug_handler; +use entities; +use entities::user; +use migration::sea_orm::{self, ActiveModelTrait}; +use serde::Deserialize; +use std::sync::Arc; + +/// `GET /user` -- Use a bearer token to get the user's profile. +/// +/// - the email address of a user if they opted in to get contacted via email +/// +/// TODO: this will change as we add better support for secondary users. +#[debug_handler] +pub async fn user_get( + Extension(app): Extension>, + TypedHeader(Authorization(bearer_token)): TypedHeader>, +) -> Web3ProxyResponse { + let (user, _semaphore) = app.bearer_is_authorized(bearer_token).await?; + + Ok(Json(user).into_response()) +} + +/// the JSON input to the `post_user` handler. +#[derive(Debug, Deserialize)] +pub struct UserPost { + email: Option, +} + +/// `POST /user` -- modify the account connected to the bearer token in the `Authentication` header. +#[debug_handler] +pub async fn user_post( + Extension(app): Extension>, + TypedHeader(Authorization(bearer_token)): TypedHeader>, + Json(payload): Json, +) -> Web3ProxyResponse { + let (user, _semaphore) = app.bearer_is_authorized(bearer_token).await?; + + let mut user: user::ActiveModel = user.into(); + + // update the email address + if let Some(x) = payload.email { + // TODO: only Set if no change + if x.is_empty() { + user.email = sea_orm::Set(None); + } else { + // TODO: do some basic validation + // TODO: don't set immediatly, send a confirmation email first + // TODO: compare first? or is sea orm smart enough to do that for us? + user.email = sea_orm::Set(Some(x)); + } + } + + // TODO: what else can we update here? password hash? subscription to newsletter? + + let user = if user.is_changed() { + let db_conn = app.db_conn().web3_context("Getting database connection")?; + + user.save(&db_conn).await? + } else { + // no changes. no need to touch the database + user + }; + + let user: user::Model = user.try_into().web3_context("Returning updated user")?; + + Ok(Json(user).into_response()) +} diff --git a/web3_proxy/src/frontend/users/payment.rs b/web3_proxy/src/frontend/users/payment.rs new file mode 100644 index 00000000..9fbf7daa --- /dev/null +++ b/web3_proxy/src/frontend/users/payment.rs @@ -0,0 +1,499 @@ +use crate::app::Web3ProxyApp; +use crate::frontend::authorization::Authorization as InternalAuthorization; +use crate::frontend::errors::{Web3ProxyError, Web3ProxyResponse}; +use crate::rpcs::request::OpenRequestResult; +use anyhow::{anyhow, Context}; +use axum::{ + extract::Path, + headers::{authorization::Bearer, Authorization}, + response::IntoResponse, + Extension, Json, TypedHeader, +}; +use axum_macros::debug_handler; +use entities::{balance, increase_on_chain_balance_receipt, user, user_tier}; +use ethers::abi::{AbiEncode, ParamType}; +use ethers::types::{Address, TransactionReceipt, H256, U256}; +use ethers::utils::{hex, keccak256}; +use hashbrown::HashMap; +use hex_fmt::HexFmt; +use http::StatusCode; +use log::{debug, info, warn, Level}; +use migration::sea_orm; +use migration::sea_orm::prelude::Decimal; +use migration::sea_orm::ActiveModelTrait; +use migration::sea_orm::ColumnTrait; +use migration::sea_orm::EntityTrait; +use migration::sea_orm::IntoActiveModel; +use migration::sea_orm::QueryFilter; +use migration::sea_orm::TransactionTrait; +use serde_json::json; +use std::sync::Arc; + +/// Implements any logic related to payments +/// Removed this mainly from "user" as this was getting clogged +/// +/// `GET /user/balance` -- Use a bearer token to get the user's balance and spend. +/// +/// - show balance in USD +/// - show deposits history (currency, amounts, transaction id) +#[debug_handler] +pub async fn user_balance_get( + Extension(app): Extension>, + TypedHeader(Authorization(bearer)): TypedHeader>, +) -> Web3ProxyResponse { + let (_user, _semaphore) = app.bearer_is_authorized(bearer).await?; + + let db_replica = app.db_replica().context("Getting database connection")?; + + // Just return the balance for the user + let user_balance = match balance::Entity::find() + .filter(balance::Column::UserId.eq(_user.id)) + .one(db_replica.conn()) + .await? + { + Some(x) => x.available_balance, + None => Decimal::from(0), // That means the user has no balance as of yet + // (user exists, but balance entry does not exist) + // In that case add this guy here + // Err(FrontendErrorResponse::BadRequest("User not found!")) + }; + + let mut response = HashMap::new(); + response.insert("balance", json!(user_balance)); + + // TODO: Gotta create a new table for the spend part + Ok(Json(response).into_response()) +} + +/// `GET /user/deposits` -- Use a bearer token to get the user's balance and spend. +/// +/// - shows a list of all deposits, including their chain-id, amount and tx-hash +#[debug_handler] +pub async fn user_deposits_get( + Extension(app): Extension>, + TypedHeader(Authorization(bearer)): TypedHeader>, +) -> Web3ProxyResponse { + let (user, _semaphore) = app.bearer_is_authorized(bearer).await?; + + let db_replica = app.db_replica().context("Getting database connection")?; + + // Filter by user ... + let receipts = increase_on_chain_balance_receipt::Entity::find() + .filter(increase_on_chain_balance_receipt::Column::DepositToUserId.eq(user.id)) + .all(db_replica.conn()) + .await?; + + // Return the response, all except the user ... + let mut response = HashMap::new(); + let receipts = receipts + .into_iter() + .map(|x| { + let mut out = HashMap::new(); + out.insert("amount", serde_json::Value::String(x.amount.to_string())); + out.insert("chain_id", serde_json::Value::Number(x.chain_id.into())); + out.insert("tx_hash", serde_json::Value::String(x.tx_hash)); + out + }) + .collect::>(); + response.insert( + "user", + json!(format!("{:?}", Address::from_slice(&user.address))), + ); + response.insert("deposits", json!(receipts)); + + Ok(Json(response).into_response()) +} + +/// `POST /user/balance/:tx_hash` -- Manually process a confirmed txid to update a user's balance. +/// +/// We will subscribe to events to watch for any user deposits, but sometimes events can be missed. +/// TODO: change this. just have a /tx/:txhash that is open to anyone. rate limit like we rate limit /login +#[debug_handler] +pub async fn user_balance_post( + Extension(app): Extension>, + TypedHeader(Authorization(bearer)): TypedHeader>, + Path(mut params): Path>, +) -> Web3ProxyResponse { + // I suppose this is ok / good, so people don't spam this endpoint as it is not "cheap" + // Check that the user is logged-in and authorized. We don't need a semaphore here btw + let (_, _semaphore) = app.bearer_is_authorized(bearer).await?; + + // Get the transaction hash, and the amount that the user wants to top up by. + // Let's say that for now, 1 credit is equivalent to 1 dollar (assuming any stablecoin has a 1:1 peg) + let tx_hash: H256 = params + .remove("tx_hash") + // TODO: map_err so this becomes a 500. routing must be bad + .ok_or(Web3ProxyError::BadRequest( + "You have not provided the tx_hash in which you paid in".to_string(), + ))? + .parse() + .context("unable to parse tx_hash")?; + + let db_conn = app.db_conn().context("query_user_stats needs a db")?; + let db_replica = app + .db_replica() + .context("query_user_stats needs a db replica")?; + + // Return straight false if the tx was already added ... + let receipt = increase_on_chain_balance_receipt::Entity::find() + .filter(increase_on_chain_balance_receipt::Column::TxHash.eq(hex::encode(tx_hash))) + .one(&db_conn) + .await?; + if receipt.is_some() { + return Err(Web3ProxyError::BadRequest( + "The transaction you provided has already been accounted for!".to_string(), + )); + } + debug!("Receipt: {:?}", receipt); + + // Iterate through all logs, and add them to the transaction list if there is any + // Address will be hardcoded in the config + let authorization = Arc::new(InternalAuthorization::internal(None).unwrap()); + + // Just make an rpc request, idk if i need to call this super extensive code + let transaction_receipt: TransactionReceipt = match app + .balanced_rpcs + .best_available_rpc(&authorization, None, &[], None, None) + .await + { + Ok(OpenRequestResult::Handle(handle)) => { + debug!( + "Params are: {:?}", + &vec![format!("0x{}", hex::encode(tx_hash))] + ); + handle + .request( + "eth_getTransactionReceipt", + &vec![format!("0x{}", hex::encode(tx_hash))], + Level::Trace.into(), + None, + ) + .await + // TODO: What kind of error would be here + .map_err(|err| Web3ProxyError::Anyhow(err.into())) + } + Ok(_) => { + // TODO: @Brllan Is this the right error message? + Err(Web3ProxyError::NoHandleReady) + } + Err(err) => { + log::trace!( + "cancelled funneling transaction {} from: {:?}", + tx_hash, + err, + ); + Err(err) + } + }?; + debug!("Transaction receipt is: {:?}", transaction_receipt); + let accepted_token: Address = match app + .balanced_rpcs + .best_available_rpc(&authorization, None, &[], None, None) + .await + { + Ok(OpenRequestResult::Handle(handle)) => { + let mut accepted_tokens_request_object: serde_json::Map = + serde_json::Map::new(); + // We want to send a request to the contract + accepted_tokens_request_object.insert( + "to".to_owned(), + serde_json::Value::String(format!( + "{:?}", + app.config.deposit_factory_contract.clone() + )), + ); + // We then want to include the function that we want to call + accepted_tokens_request_object.insert( + "data".to_owned(), + serde_json::Value::String(format!( + "0x{}", + HexFmt(keccak256("get_approved_tokens()".to_owned().into_bytes())) + )), + // hex::encode( + ); + let params = serde_json::Value::Array(vec![ + serde_json::Value::Object(accepted_tokens_request_object), + serde_json::Value::String("latest".to_owned()), + ]); + debug!("Params are: {:?}", ¶ms); + let accepted_token: String = handle + .request("eth_call", ¶ms, Level::Trace.into(), None) + .await + // TODO: What kind of error would be here + .map_err(|err| Web3ProxyError::Anyhow(err.into()))?; + // Read the last + debug!("Accepted token response is: {:?}", accepted_token); + accepted_token[accepted_token.len() - 40..] + .parse::
() + .map_err(|err| Web3ProxyError::Anyhow(err.into())) + } + Ok(_) => { + // TODO: @Brllan Is this the right error message? + Err(Web3ProxyError::NoHandleReady) + } + Err(err) => { + log::trace!( + "cancelled funneling transaction {} from: {:?}", + tx_hash, + err, + ); + Err(err) + } + }?; + debug!("Accepted token is: {:?}", accepted_token); + let decimals: u32 = match app + .balanced_rpcs + .best_available_rpc(&authorization, None, &[], None, None) + .await + { + Ok(OpenRequestResult::Handle(handle)) => { + // Now get decimals points of the stablecoin + let mut token_decimals_request_object: serde_json::Map = + serde_json::Map::new(); + token_decimals_request_object.insert( + "to".to_owned(), + serde_json::Value::String(format!("0x{}", HexFmt(accepted_token))), + ); + token_decimals_request_object.insert( + "data".to_owned(), + serde_json::Value::String(format!( + "0x{}", + HexFmt(keccak256("decimals()".to_owned().into_bytes())) + )), + ); + let params = serde_json::Value::Array(vec![ + serde_json::Value::Object(token_decimals_request_object), + serde_json::Value::String("latest".to_owned()), + ]); + debug!("ERC20 Decimal request params are: {:?}", ¶ms); + let decimals: String = handle + .request("eth_call", ¶ms, Level::Trace.into(), None) + .await + .map_err(|err| Web3ProxyError::Anyhow(err.into()))?; + debug!("Decimals response is: {:?}", decimals); + u32::from_str_radix(&decimals[2..], 16) + .map_err(|err| Web3ProxyError::Anyhow(err.into())) + } + Ok(_) => { + // TODO: @Brllan Is this the right error message? + Err(Web3ProxyError::NoHandleReady) + } + Err(err) => { + log::trace!( + "cancelled funneling transaction {} from: {:?}", + tx_hash, + err, + ); + Err(err) + } + }?; + debug!("Decimals are: {:?}", decimals); + debug!("Tx receipt: {:?}", transaction_receipt); + + // Go through all logs, this should prob capture it, + // At least according to this SE logs are just concatenations of the underlying types (like a struct..) + // https://ethereum.stackexchange.com/questions/87653/how-to-decode-log-event-of-my-transaction-log + + let deposit_contract = match app.config.deposit_factory_contract { + Some(x) => Ok(x), + None => Err(Web3ProxyError::Anyhow(anyhow!( + "A deposit_contract must be provided in the config to parse payments" + ))), + }?; + let deposit_topic = match app.config.deposit_topic { + Some(x) => Ok(x), + None => Err(Web3ProxyError::Anyhow(anyhow!( + "A deposit_topic must be provided in the config to parse payments" + ))), + }?; + + // Make sure there is only a single log within that transaction ... + // I don't know how to best cover the case that there might be multiple logs inside + + for log in transaction_receipt.logs { + if log.address != deposit_contract { + debug!( + "Out: Log is not relevant, as it is not directed to the deposit contract {:?} {:?}", + format!("{:?}", log.address), + deposit_contract + ); + continue; + } + + // Get the topics out + let topic: H256 = H256::from(log.topics.get(0).unwrap().to_owned()); + if topic != deposit_topic { + debug!( + "Out: Topic is not relevant: {:?} {:?}", + topic, deposit_topic + ); + continue; + } + + // TODO: Will this work? Depends how logs are encoded + let (recipient_account, token, amount): (Address, Address, U256) = match ethers::abi::decode( + &[ + ParamType::Address, + ParamType::Address, + ParamType::Uint(256usize), + ], + &log.data, + ) { + Ok(tpl) => ( + tpl.get(0) + .unwrap() + .clone() + .into_address() + .context("Could not decode recipient")?, + tpl.get(1) + .unwrap() + .clone() + .into_address() + .context("Could not decode token")?, + tpl.get(2) + .unwrap() + .clone() + .into_uint() + .context("Could not decode amount")?, + ), + Err(err) => { + warn!("Out: Could not decode! {:?}", err); + continue; + } + }; + + // return early if amount is 0 + if amount == U256::from(0) { + warn!( + "Out: Found log has amount = 0 {:?}. This should never be the case according to the smart contract", + amount + ); + continue; + } + + // Skip if no accepted token. Right now we only accept a single stablecoin as input + if token != accepted_token { + warn!( + "Out: Token is not accepted: {:?} != {:?}", + token, accepted_token + ); + continue; + } + + info!( + "Found deposit transaction for: {:?} {:?} {:?}", + recipient_account, token, amount + ); + + // Encoding is inefficient, revisit later + let recipient = match user::Entity::find() + .filter(user::Column::Address.eq(&recipient_account.encode()[12..])) + .one(db_replica.conn()) + .await? + { + Some(x) => Ok(x), + None => Err(Web3ProxyError::BadRequest( + "The user must have signed up first. They are currently not signed up!".to_string(), + )), + }?; + + // For now we only accept stablecoins + // And we hardcode the peg (later we would have to depeg this, for example + // 1$ = Decimal(1) for any stablecoin + // TODO: Let's assume that people don't buy too much at _once_, we do support >$1M which should be fine for now + debug!("Arithmetic is: {:?} {:?}", amount, decimals); + debug!( + "Decimals arithmetic is: {:?} {:?}", + Decimal::from(amount.as_u128()), + Decimal::from(10_u64.pow(decimals)) + ); + let mut amount = Decimal::from(amount.as_u128()); + let _ = amount.set_scale(decimals); + debug!("Amount is: {:?}", amount); + + // Check if the item is in the database. If it is not, then add it into the database + let user_balance = balance::Entity::find() + .filter(balance::Column::UserId.eq(recipient.id)) + .one(&db_conn) + .await?; + + // Get the premium user-tier + let premium_user_tier = user_tier::Entity::find() + .filter(user_tier::Column::Title.eq("Premium")) + .one(&db_conn) + .await? + .context("Could not find 'Premium' Tier in user-database")?; + + let txn = db_conn.begin().await?; + match user_balance { + Some(user_balance) => { + let balance_plus_amount = user_balance.available_balance + amount; + info!("New user balance is: {:?}", balance_plus_amount); + // Update the entry, adding the balance + let mut active_user_balance = user_balance.into_active_model(); + active_user_balance.available_balance = sea_orm::Set(balance_plus_amount); + + if balance_plus_amount >= Decimal::new(10, 0) { + // Also make the user premium at this point ... + let mut active_recipient = recipient.clone().into_active_model(); + // Make the recipient premium "Effectively Unlimited" + active_recipient.user_tier_id = sea_orm::Set(premium_user_tier.id); + active_recipient.save(&txn).await?; + } + + debug!("New user balance model is: {:?}", active_user_balance); + active_user_balance.save(&txn).await?; + // txn.commit().await?; + // user_balance + } + None => { + // Create the entry with the respective balance + let active_user_balance = balance::ActiveModel { + available_balance: sea_orm::ActiveValue::Set(amount), + user_id: sea_orm::ActiveValue::Set(recipient.id), + ..Default::default() + }; + + if amount >= Decimal::new(10, 0) { + // Also make the user premium at this point ... + let mut active_recipient = recipient.clone().into_active_model(); + // Make the recipient premium "Effectively Unlimited" + active_recipient.user_tier_id = sea_orm::Set(premium_user_tier.id); + active_recipient.save(&txn).await?; + } + + info!("New user balance model is: {:?}", active_user_balance); + active_user_balance.save(&txn).await?; + // txn.commit().await?; + // user_balance // .try_into_model().unwrap() + } + }; + debug!("Setting tx_hash: {:?}", tx_hash); + let receipt = increase_on_chain_balance_receipt::ActiveModel { + tx_hash: sea_orm::ActiveValue::Set(hex::encode(tx_hash)), + chain_id: sea_orm::ActiveValue::Set(app.config.chain_id), + amount: sea_orm::ActiveValue::Set(amount), + deposit_to_user_id: sea_orm::ActiveValue::Set(recipient.id), + ..Default::default() + }; + + receipt.save(&txn).await?; + txn.commit().await?; + debug!("Saved to db"); + + let response = ( + StatusCode::CREATED, + Json(json!({ + "tx_hash": tx_hash, + "amount": amount + })), + ) + .into_response(); + // Return early if the log was added, assume there is at most one valid log per transaction + return Ok(response.into()); + } + + Err(Web3ProxyError::BadRequest( + "No such transaction was found, or token is not supported!".to_string(), + )) +} diff --git a/web3_proxy/src/frontend/users/referral.rs b/web3_proxy/src/frontend/users/referral.rs new file mode 100644 index 00000000..ac4649d0 --- /dev/null +++ b/web3_proxy/src/frontend/users/referral.rs @@ -0,0 +1,87 @@ +//! Handle registration, logins, and managing account data. +use crate::app::Web3ProxyApp; +use crate::frontend::errors::{Web3ProxyError, Web3ProxyResponse}; +use crate::referral_code::ReferralCode; +use anyhow::Context; +use axum::{ + extract::Query, + headers::{authorization::Bearer, Authorization}, + response::IntoResponse, + Extension, Json, TypedHeader, +}; +use axum_macros::debug_handler; +use entities::{referrer, user_tier}; +use hashbrown::HashMap; +use http::StatusCode; +use log::warn; +use migration::sea_orm; +use migration::sea_orm::ActiveModelTrait; +use migration::sea_orm::ColumnTrait; +use migration::sea_orm::EntityTrait; +use migration::sea_orm::QueryFilter; +use migration::sea_orm::TransactionTrait; +use serde_json::json; +use std::sync::Arc; + +/// Create or get the existing referral link. +/// This is the link that the user can share to third parties, and get credits. +/// Applies to premium users only +#[debug_handler] +pub async fn user_referral_link_get( + Extension(app): Extension>, + TypedHeader(Authorization(bearer)): TypedHeader>, + Query(_params): Query>, +) -> Web3ProxyResponse { + // First get the bearer token and check if the user is logged in + let (user, _semaphore) = app.bearer_is_authorized(bearer).await?; + + let db_replica = app + .db_replica() + .context("getting replica db for user's revert logs")?; + + // Second, check if the user is a premium user + let user_tier = user_tier::Entity::find() + .filter(user_tier::Column::Id.eq(user.user_tier_id)) + .one(db_replica.conn()) + .await? + .ok_or(Web3ProxyError::UnknownKey)?; + + warn!("User tier is: {:?}", user_tier); + // TODO: This shouldn't be hardcoded. Also, it should be an enum, not sth like this ... + if user_tier.id != 6 { + return Err(Web3ProxyError::PaymentRequired.into()); + } + + // Then get the referral token + let user_referrer = referrer::Entity::find() + .filter(referrer::Column::UserId.eq(user.id)) + .one(db_replica.conn()) + .await?; + + let (referral_code, status_code) = match user_referrer { + Some(x) => (x.referral_code, StatusCode::OK), + None => { + // Connect to the database for mutable write + let db_conn = app.db_conn().context("getting db_conn")?; + + let referral_code = ReferralCode::default().0; + // Log that this guy was referred by another guy + // Do not automatically create a new + let referrer_entry = referrer::ActiveModel { + user_id: sea_orm::ActiveValue::Set(user.id), + referral_code: sea_orm::ActiveValue::Set(referral_code.clone()), + ..Default::default() + }; + referrer_entry.save(&db_conn).await?; + (referral_code, StatusCode::CREATED) + } + }; + + let response_json = json!({ + "referral_code": referral_code, + "user": user, + }); + + let response = (status_code, Json(response_json)).into_response(); + Ok(response) +} diff --git a/web3_proxy/src/frontend/users/rpc_keys.rs b/web3_proxy/src/frontend/users/rpc_keys.rs new file mode 100644 index 00000000..10f6118d --- /dev/null +++ b/web3_proxy/src/frontend/users/rpc_keys.rs @@ -0,0 +1,259 @@ +//! Handle registration, logins, and managing account data. +use super::super::authorization::RpcSecretKey; +use super::super::errors::{Web3ProxyError, Web3ProxyErrorContext, Web3ProxyResponse}; +use crate::app::Web3ProxyApp; +use axum::headers::{Header, Origin, Referer, UserAgent}; +use axum::{ + headers::{authorization::Bearer, Authorization}, + response::IntoResponse, + Extension, Json, TypedHeader, +}; +use axum_macros::debug_handler; +use entities; +use entities::rpc_key; +use entities::sea_orm_active_enums::TrackingLevel; +use hashbrown::HashMap; +use http::HeaderValue; +use ipnet::IpNet; +use itertools::Itertools; +use migration::sea_orm::{ + self, ActiveModelTrait, ColumnTrait, EntityTrait, IntoActiveModel, QueryFilter, TryIntoModel, +}; +use serde::Deserialize; +use serde_json::json; +use std::sync::Arc; + +/// `GET /user/keys` -- Use a bearer token to get the user's api keys and their settings. +#[debug_handler] +pub async fn rpc_keys_get( + Extension(app): Extension>, + TypedHeader(Authorization(bearer)): TypedHeader>, +) -> Web3ProxyResponse { + let (user, _semaphore) = app.bearer_is_authorized(bearer).await?; + + let db_replica = app + .db_replica() + .web3_context("db_replica is required to fetch a user's keys")?; + + let uks = rpc_key::Entity::find() + .filter(rpc_key::Column::UserId.eq(user.id)) + .all(db_replica.conn()) + .await + .web3_context("failed loading user's key")?; + + let response_json = json!({ + "user_id": user.id, + "user_rpc_keys": uks + .into_iter() + .map(|uk| (uk.id, uk)) + .collect::>(), + }); + + Ok(Json(response_json).into_response()) +} + +/// `DELETE /user/keys` -- Use a bearer token to delete an existing key. +#[debug_handler] +pub async fn rpc_keys_delete( + Extension(app): Extension>, + TypedHeader(Authorization(bearer)): TypedHeader>, +) -> Web3ProxyResponse { + let (_user, _semaphore) = app.bearer_is_authorized(bearer).await?; + + // TODO: think about how cascading deletes and billing should work + Err(Web3ProxyError::NotImplemented) +} + +/// the JSON input to the `rpc_keys_management` handler. +/// If `key_id` is set, it updates an existing key. +/// If `key_id` is not set, it creates a new key. +/// `log_request_method` cannot be change once the key is created +/// `user_tier` cannot be changed here +#[derive(Debug, Deserialize)] +pub struct UserKeyManagement { + key_id: Option, + active: Option, + allowed_ips: Option, + allowed_origins: Option, + allowed_referers: Option, + allowed_user_agents: Option, + description: Option, + log_level: Option, + // TODO: enable log_revert_trace: Option, + private_txs: Option, +} + +/// `POST /user/keys` or `PUT /user/keys` -- Use a bearer token to create or update an existing key. +#[debug_handler] +pub async fn rpc_keys_management( + Extension(app): Extension>, + TypedHeader(Authorization(bearer)): TypedHeader>, + Json(payload): Json, +) -> Web3ProxyResponse { + // TODO: is there a way we can know if this is a PUT or POST? right now we can modify or create keys with either. though that probably doesn't matter + + let (user, _semaphore) = app.bearer_is_authorized(bearer).await?; + + let db_replica = app + .db_replica() + .web3_context("getting db for user's keys")?; + + let mut uk = if let Some(existing_key_id) = payload.key_id { + // get the key and make sure it belongs to the user + rpc_key::Entity::find() + .filter(rpc_key::Column::UserId.eq(user.id)) + .filter(rpc_key::Column::Id.eq(existing_key_id)) + .one(db_replica.conn()) + .await + .web3_context("failed loading user's key")? + .web3_context("key does not exist or is not controlled by this bearer token")? + .into_active_model() + } else { + // make a new key + // TODO: limit to 10 keys? + let secret_key = RpcSecretKey::new(); + + let log_level = payload + .log_level + .web3_context("log level must be 'none', 'detailed', or 'aggregated'")?; + + rpc_key::ActiveModel { + user_id: sea_orm::Set(user.id), + secret_key: sea_orm::Set(secret_key.into()), + log_level: sea_orm::Set(log_level), + ..Default::default() + } + }; + + // TODO: do we need null descriptions? default to empty string should be fine, right? + if let Some(description) = payload.description { + if description.is_empty() { + uk.description = sea_orm::Set(None); + } else { + uk.description = sea_orm::Set(Some(description)); + } + } + + if let Some(private_txs) = payload.private_txs { + uk.private_txs = sea_orm::Set(private_txs); + } + + if let Some(active) = payload.active { + uk.active = sea_orm::Set(active); + } + + if let Some(allowed_ips) = payload.allowed_ips { + if allowed_ips.is_empty() { + uk.allowed_ips = sea_orm::Set(None); + } else { + // split allowed ips on ',' and try to parse them all. error on invalid input + let allowed_ips = allowed_ips + .split(',') + .map(|x| x.trim().parse::()) + .collect::, _>>()? + // parse worked. convert back to Strings + .into_iter() + .map(|x| x.to_string()); + + // and join them back together + let allowed_ips: String = + Itertools::intersperse(allowed_ips, ", ".to_string()).collect(); + + uk.allowed_ips = sea_orm::Set(Some(allowed_ips)); + } + } + + // TODO: this should actually be bytes + if let Some(allowed_origins) = payload.allowed_origins { + if allowed_origins.is_empty() { + uk.allowed_origins = sea_orm::Set(None); + } else { + // split allowed_origins on ',' and try to parse them all. error on invalid input + let allowed_origins = allowed_origins + .split(',') + .map(|x| HeaderValue::from_str(x.trim())) + .collect::, _>>()? + .into_iter() + .map(|x| Origin::decode(&mut [x].iter())) + .collect::, _>>()? + // parse worked. convert back to String and join them back together + .into_iter() + .map(|x| x.to_string()); + + let allowed_origins: String = + Itertools::intersperse(allowed_origins, ", ".to_string()).collect(); + + uk.allowed_origins = sea_orm::Set(Some(allowed_origins)); + } + } + + // TODO: this should actually be bytes + if let Some(allowed_referers) = payload.allowed_referers { + if allowed_referers.is_empty() { + uk.allowed_referers = sea_orm::Set(None); + } else { + // split allowed ips on ',' and try to parse them all. error on invalid input + let allowed_referers = allowed_referers + .split(',') + .map(|x| HeaderValue::from_str(x.trim())) + .collect::, _>>()? + .into_iter() + .map(|x| Referer::decode(&mut [x].iter())) + .collect::, _>>()?; + + // parse worked. now we can put it back together. + // but we can't go directly to String. + // so we convert to HeaderValues first + let mut header_map = vec![]; + for x in allowed_referers { + x.encode(&mut header_map); + } + + // convert HeaderValues to Strings + // since we got these from strings, this should always work (unless we figure out using bytes) + let allowed_referers = header_map + .into_iter() + .map(|x| x.to_str().map(|x| x.to_string())) + .collect::, _>>()?; + + // join strings together with commas + let allowed_referers: String = + Itertools::intersperse(allowed_referers.into_iter(), ", ".to_string()).collect(); + + uk.allowed_referers = sea_orm::Set(Some(allowed_referers)); + } + } + + if let Some(allowed_user_agents) = payload.allowed_user_agents { + if allowed_user_agents.is_empty() { + uk.allowed_user_agents = sea_orm::Set(None); + } else { + // split allowed_user_agents on ',' and try to parse them all. error on invalid input + let allowed_user_agents = allowed_user_agents + .split(',') + .filter_map(|x| x.trim().parse::().ok()) + // parse worked. convert back to String + .map(|x| x.to_string()); + + // join the strings together + let allowed_user_agents: String = + Itertools::intersperse(allowed_user_agents, ", ".to_string()).collect(); + + uk.allowed_user_agents = sea_orm::Set(Some(allowed_user_agents)); + } + } + + let uk = if uk.is_changed() { + let db_conn = app.db_conn().web3_context("login requires a db")?; + + uk.save(&db_conn) + .await + .web3_context("Failed saving user key")? + } else { + uk + }; + + let uk = uk.try_into_model()?; + + Ok(Json(uk).into_response()) +} diff --git a/web3_proxy/src/frontend/users/stats.rs b/web3_proxy/src/frontend/users/stats.rs new file mode 100644 index 00000000..56a2f137 --- /dev/null +++ b/web3_proxy/src/frontend/users/stats.rs @@ -0,0 +1,123 @@ +//! Handle registration, logins, and managing account data. +use crate::app::Web3ProxyApp; +use crate::frontend::errors::{Web3ProxyErrorContext, Web3ProxyResponse}; +use crate::http_params::{ + get_chain_id_from_params, get_page_from_params, get_query_start_from_params, +}; +use crate::stats::influxdb_queries::query_user_stats; +use crate::stats::StatType; +use axum::{ + extract::Query, + headers::{authorization::Bearer, Authorization}, + response::IntoResponse, + Extension, Json, TypedHeader, +}; +use axum_macros::debug_handler; +use entities; +use entities::{revert_log, rpc_key}; +use hashbrown::HashMap; +use migration::sea_orm::{ColumnTrait, EntityTrait, PaginatorTrait, QueryFilter, QueryOrder}; +use serde_json::json; +use std::sync::Arc; + +/// `GET /user/revert_logs` -- Use a bearer token to get the user's revert logs. +#[debug_handler] +pub async fn user_revert_logs_get( + Extension(app): Extension>, + TypedHeader(Authorization(bearer)): TypedHeader>, + Query(params): Query>, +) -> Web3ProxyResponse { + let (user, _semaphore) = app.bearer_is_authorized(bearer).await?; + + let chain_id = get_chain_id_from_params(app.as_ref(), ¶ms)?; + let query_start = get_query_start_from_params(¶ms)?; + let page = get_page_from_params(¶ms)?; + + // TODO: page size from config + let page_size = 1_000; + + let mut response = HashMap::new(); + + response.insert("page", json!(page)); + response.insert("page_size", json!(page_size)); + response.insert("chain_id", json!(chain_id)); + response.insert("query_start", json!(query_start.timestamp() as u64)); + + let db_replica = app + .db_replica() + .web3_context("getting replica db for user's revert logs")?; + + let uks = rpc_key::Entity::find() + .filter(rpc_key::Column::UserId.eq(user.id)) + .all(db_replica.conn()) + .await + .web3_context("failed loading user's key")?; + + // TODO: only select the ids + let uks: Vec<_> = uks.into_iter().map(|x| x.id).collect(); + + // get revert logs + let mut q = revert_log::Entity::find() + .filter(revert_log::Column::Timestamp.gte(query_start)) + .filter(revert_log::Column::RpcKeyId.is_in(uks)) + .order_by_asc(revert_log::Column::Timestamp); + + if chain_id == 0 { + // don't do anything + } else { + // filter on chain id + q = q.filter(revert_log::Column::ChainId.eq(chain_id)) + } + + // query the database for number of items and pages + let pages_result = q + .clone() + .paginate(db_replica.conn(), page_size) + .num_items_and_pages() + .await?; + + response.insert("num_items", pages_result.number_of_items.into()); + response.insert("num_pages", pages_result.number_of_pages.into()); + + // query the database for the revert logs + let revert_logs = q + .paginate(db_replica.conn(), page_size) + .fetch_page(page) + .await?; + + response.insert("revert_logs", json!(revert_logs)); + + Ok(Json(response).into_response()) +} + +/// `GET /user/stats/aggregate` -- Public endpoint for aggregate stats such as bandwidth used and methods requested. +#[debug_handler] +pub async fn user_stats_aggregated_get( + Extension(app): Extension>, + bearer: Option>>, + Query(params): Query>, +) -> Web3ProxyResponse { + let response = query_user_stats(&app, bearer, ¶ms, StatType::Aggregated).await?; + + Ok(response) +} + +/// `GET /user/stats/detailed` -- Use a bearer token to get the user's key stats such as bandwidth used and methods requested. +/// +/// If no bearer is provided, detailed stats for all users will be shown. +/// View a single user with `?user_id=$x`. +/// View a single chain with `?chain_id=$x`. +/// +/// Set `$x` to zero to see all. +/// +/// TODO: this will change as we add better support for secondary users. +#[debug_handler] +pub async fn user_stats_detailed_get( + Extension(app): Extension>, + bearer: Option>>, + Query(params): Query>, +) -> Web3ProxyResponse { + let response = query_user_stats(&app, bearer, ¶ms, StatType::Detailed).await?; + + Ok(response) +} diff --git a/web3_proxy/src/frontend/users/subuser.rs b/web3_proxy/src/frontend/users/subuser.rs new file mode 100644 index 00000000..c1d4eb3b --- /dev/null +++ b/web3_proxy/src/frontend/users/subuser.rs @@ -0,0 +1,426 @@ +//! Handle subusers, viewing subusers, and viewing accessible rpc-keys +use crate::app::Web3ProxyApp; +use crate::frontend::authorization::RpcSecretKey; +use crate::frontend::errors::{Web3ProxyError, Web3ProxyErrorContext, Web3ProxyResponse}; +use anyhow::Context; +use axum::{ + extract::Query, + headers::{authorization::Bearer, Authorization}, + response::IntoResponse, + Extension, Json, TypedHeader, +}; +use axum_macros::debug_handler; +use entities::sea_orm_active_enums::Role; +use entities::{balance, rpc_key, secondary_user, user, user_tier}; +use ethers::types::Address; +use hashbrown::HashMap; +use http::StatusCode; +use log::{debug, warn}; +use migration::sea_orm; +use migration::sea_orm::prelude::Decimal; +use migration::sea_orm::ActiveModelTrait; +use migration::sea_orm::ColumnTrait; +use migration::sea_orm::EntityTrait; +use migration::sea_orm::IntoActiveModel; +use migration::sea_orm::QueryFilter; +use migration::sea_orm::TransactionTrait; +use serde_json::json; +use std::sync::Arc; +use ulid::{self, Ulid}; +use uuid::Uuid; + +pub async fn get_keys_as_subuser( + Extension(app): Extension>, + TypedHeader(Authorization(bearer)): TypedHeader>, + Query(_params): Query>, +) -> Web3ProxyResponse { + // First, authenticate + let (subuser, _semaphore) = app.bearer_is_authorized(bearer).await?; + + let db_replica = app + .db_replica() + .context("getting replica db for user's revert logs")?; + + // TODO: JOIN over RPC_KEY, SUBUSER, PRIMARY_USER and return these items + + // Get all secondary users that have access to this rpc key + let secondary_user_entities = secondary_user::Entity::find() + .filter(secondary_user::Column::UserId.eq(subuser.id)) + .all(db_replica.conn()) + .await? + .into_iter() + .map(|x| (x.rpc_secret_key_id.clone(), x)) + .collect::>(); + + // Now return a list of all subusers (their wallets) + let rpc_key_entities: Vec<(rpc_key::Model, Option)> = rpc_key::Entity::find() + .filter( + rpc_key::Column::Id.is_in( + secondary_user_entities + .iter() + .map(|(x, _)| *x) + .collect::>(), + ), + ) + .find_also_related(user::Entity) + .all(db_replica.conn()) + .await?; + + // TODO: Merge rpc-key with respective user (join is probably easiest ...) + + // Now return the list + let response_json = json!({ + "subuser": format!("{:?}", Address::from_slice(&subuser.address)), + "rpc_keys": rpc_key_entities + .into_iter() + .flat_map(|(rpc_key, rpc_owner)| { + match rpc_owner { + Some(inner_rpc_owner) => { + let mut tmp = HashMap::new(); + tmp.insert("rpc-key", serde_json::Value::String(Ulid::from(rpc_key.secret_key).to_string())); + tmp.insert("rpc-owner", serde_json::Value::String(format!("{:?}", Address::from_slice(&inner_rpc_owner.address)))); + tmp.insert("role", serde_json::Value::String(format!("{:?}", secondary_user_entities.get(&rpc_key.id).unwrap().role))); // .to_string() returns ugly "'...'" + Some(tmp) + }, + None => { + // error!("Found RPC secret key with no user!".to_owned()); + None + } + } + }) + .collect::>(), + }); + + Ok(Json(response_json).into_response()) +} + +pub async fn get_subusers( + Extension(app): Extension>, + TypedHeader(Authorization(bearer)): TypedHeader>, + Query(mut params): Query>, +) -> Web3ProxyResponse { + // First, authenticate + let (user, _semaphore) = app.bearer_is_authorized(bearer).await?; + + let db_replica = app + .db_replica() + .context("getting replica db for user's revert logs")?; + + // Second, check if the user is a premium user + let user_tier = user_tier::Entity::find() + .filter(user_tier::Column::Id.eq(user.user_tier_id)) + .one(db_replica.conn()) + .await? + .ok_or(Web3ProxyError::BadRequest( + "Could not find user in db although bearer token is there!".to_string(), + ))?; + + debug!("User tier is: {:?}", user_tier); + // TODO: This shouldn't be hardcoded. Also, it should be an enum, not sth like this ... + if user_tier.id != 6 { + return Err( + anyhow::anyhow!("User is not premium. Must be premium to create referrals.").into(), + ); + } + + let rpc_key: Ulid = params + .remove("rpc_key") + // TODO: map_err so this becomes a 500. routing must be bad + .ok_or(Web3ProxyError::BadRequest( + "You have not provided the 'rpc_key' whose access to modify".to_string(), + ))? + .parse() + .context(format!("unable to parse rpc_key {:?}", params))?; + + // Get the rpc key id + let rpc_key = rpc_key::Entity::find() + .filter(rpc_key::Column::SecretKey.eq(Uuid::from(rpc_key))) + .one(db_replica.conn()) + .await? + .ok_or(Web3ProxyError::BadRequest( + "The provided RPC key cannot be found".to_string(), + ))?; + + // Get all secondary users that have access to this rpc key + let secondary_user_entities = secondary_user::Entity::find() + .filter(secondary_user::Column::RpcSecretKeyId.eq(rpc_key.id)) + .all(db_replica.conn()) + .await? + .into_iter() + .map(|x| (x.user_id.clone(), x)) + .collect::>(); + + // Now return a list of all subusers (their wallets) + let subusers = user::Entity::find() + .filter( + user::Column::Id.is_in( + secondary_user_entities + .iter() + .map(|(x, _)| *x) + .collect::>(), + ), + ) + .all(db_replica.conn()) + .await?; + + warn!("Subusers are: {:?}", subusers); + + // Now return the list + let response_json = json!({ + "caller": format!("{:?}", Address::from_slice(&user.address)), + "rpc_key": rpc_key, + "subusers": subusers + .into_iter() + .map(|subuser| { + let mut tmp = HashMap::new(); + // .encode_hex() + tmp.insert("address", serde_json::Value::String(format!("{:?}", Address::from_slice(&subuser.address)))); + tmp.insert("role", serde_json::Value::String(format!("{:?}", secondary_user_entities.get(&subuser.id).unwrap().role))); + json!(tmp) + }) + .collect::>(), + }); + + Ok(Json(response_json).into_response()) +} + +#[debug_handler] +pub async fn modify_subuser( + Extension(app): Extension>, + TypedHeader(Authorization(bearer)): TypedHeader>, + Query(mut params): Query>, +) -> Web3ProxyResponse { + // First, authenticate + let (user, _semaphore) = app.bearer_is_authorized(bearer).await?; + + let db_replica = app + .db_replica() + .context("getting replica db for user's revert logs")?; + + // Second, check if the user is a premium user + let user_tier = user_tier::Entity::find() + .filter(user_tier::Column::Id.eq(user.user_tier_id)) + .one(db_replica.conn()) + .await? + .ok_or(Web3ProxyError::BadRequest( + "Could not find user in db although bearer token is there!".to_string(), + ))?; + + debug!("User tier is: {:?}", user_tier); + // TODO: This shouldn't be hardcoded. Also, it should be an enum, not sth like this ... + if user_tier.id != 6 { + return Err( + anyhow::anyhow!("User is not premium. Must be premium to create referrals.").into(), + ); + } + + warn!("Parameters are: {:?}", params); + + // Then, distinguish the endpoint to modify + let rpc_key_to_modify: Ulid = params + .remove("rpc_key") + // TODO: map_err so this becomes a 500. routing must be bad + .ok_or(Web3ProxyError::BadRequest( + "You have not provided the 'rpc_key' whose access to modify".to_string(), + ))? + .parse::() + .context(format!("unable to parse rpc_key {:?}", params))?; + // let rpc_key_to_modify: Uuid = ulid::serde::ulid_as_uuid::deserialize(rpc_key_to_modify)?; + + let subuser_address: Address = params + .remove("subuser_address") + // TODO: map_err so this becomes a 500. routing must be bad + .ok_or(Web3ProxyError::BadRequest( + "You have not provided the 'user_address' whose access to modify".to_string(), + ))? + .parse() + .context(format!("unable to parse subuser_address {:?}", params))?; + + // TODO: Check subuser address for eip55 checksum + + let keep_subuser: bool = match params + .remove("new_status") + // TODO: map_err so this becomes a 500. routing must be bad + .ok_or(Web3ProxyError::BadRequest( + "You have not provided the new_stats key in the request".to_string(), + ))? + .as_str() + { + "upsert" => Ok(true), + "remove" => Ok(false), + _ => Err(Web3ProxyError::BadRequest( + "'new_status' must be one of 'upsert' or 'remove'".to_string(), + )), + }?; + + let new_role: Role = match params + .remove("new_role") + // TODO: map_err so this becomes a 500. routing must be bad + .ok_or(Web3ProxyError::BadRequest( + "You have not provided the new_stats key in the request".to_string(), + ))? + .as_str() + { + // TODO: Technically, if this is the new owner, we should transpose the full table. + // For now, let's just not allow the primary owner to just delete his account + // (if there is even such a functionality) + "owner" => Ok(Role::Owner), + "admin" => Ok(Role::Admin), + "collaborator" => Ok(Role::Collaborator), + _ => Err(Web3ProxyError::BadRequest( + "'new_role' must be one of 'owner', 'admin', 'collaborator'".to_string(), + )), + }?; + + // --------------------------- + // First, check if the user exists as a user. If not, add them + // (and also create a balance, and rpc_key, same procedure as logging in for first time) + // --------------------------- + let subuser = user::Entity::find() + .filter(user::Column::Address.eq(subuser_address.as_ref())) + .one(db_replica.conn()) + .await?; + + let rpc_key_entity = rpc_key::Entity::find() + .filter(rpc_key::Column::SecretKey.eq(Uuid::from(rpc_key_to_modify))) + .one(db_replica.conn()) + .await? + .ok_or(Web3ProxyError::BadRequest( + "Provided RPC key does not exist!".to_owned(), + ))?; + + // Make sure that the user owns the rpc_key_entity + if rpc_key_entity.user_id != user.id { + return Err(Web3ProxyError::BadRequest( + "you must own the RPC for which you are giving permissions out".to_string(), + )); + } + + // TODO: There is a good chunk of duplicate logic as login-post. Consider refactoring ... + let db_conn = app.db_conn().web3_context("login requires a db")?; + let (subuser, _subuser_rpc_keys, _status_code) = match subuser { + None => { + let txn = db_conn.begin().await?; + // First add a user; the only thing we need from them is an address + // everything else is optional + let subuser = user::ActiveModel { + address: sea_orm::Set(subuser_address.to_fixed_bytes().into()), // Address::from_slice( + ..Default::default() + }; + + let subuser = subuser.insert(&txn).await?; + + // create the user's first api key + let rpc_secret_key = RpcSecretKey::new(); + + let subuser_rpc_key = rpc_key::ActiveModel { + user_id: sea_orm::Set(subuser.id.clone()), + secret_key: sea_orm::Set(rpc_secret_key.into()), + description: sea_orm::Set(None), + ..Default::default() + }; + + let subuser_rpc_keys = vec![subuser_rpc_key + .insert(&txn) + .await + .web3_context("Failed saving new user key")?]; + + // We should also create the balance entry ... + let subuser_balance = balance::ActiveModel { + user_id: sea_orm::Set(subuser.id.clone()), + available_balance: sea_orm::Set(Decimal::new(0, 0)), + used_balance: sea_orm::Set(Decimal::new(0, 0)), + ..Default::default() + }; + subuser_balance.insert(&txn).await?; + // save the user and key to the database + txn.commit().await?; + + (subuser, subuser_rpc_keys, StatusCode::CREATED) + } + Some(subuser) => { + if subuser.id == user.id { + return Err(Web3ProxyError::BadRequest( + "you cannot make a subuser out of yourself".to_string(), + )); + } + + // Let's say that a user that exists can actually also redeem a key in retrospect... + // the user is already registered + let subuser_rpc_keys = rpc_key::Entity::find() + .filter(rpc_key::Column::UserId.eq(subuser.id)) + .all(db_replica.conn()) + .await + .web3_context("failed loading user's key")?; + + (subuser, subuser_rpc_keys, StatusCode::OK) + } + }; + + // -------------------------------- + // Now apply the operation + // Either add the subuser + // Or revoke his subuser status + // -------------------------------- + + // Search for subuser first of all + // There should be a unique-constraint on user-id + rpc_key + let subuser_entry_secondary_user = secondary_user::Entity::find() + .filter(secondary_user::Column::UserId.eq(subuser.id)) + .filter(secondary_user::Column::RpcSecretKeyId.eq(rpc_key_entity.id)) + .one(db_replica.conn()) + .await + .web3_context("failed using the db to check for a subuser")?; + + let txn = db_conn.begin().await?; + let mut action = "no action"; + let _ = match subuser_entry_secondary_user { + Some(secondary_user) => { + // In this case, remove the subuser + let mut active_subuser_entry_secondary_user = secondary_user.into_active_model(); + if !keep_subuser { + // Remove the user + active_subuser_entry_secondary_user.delete(&db_conn).await?; + action = "removed"; + } else { + // Just change the role + active_subuser_entry_secondary_user.role = sea_orm::Set(new_role.clone()); + active_subuser_entry_secondary_user.save(&db_conn).await?; + action = "role modified"; + } + } + None if keep_subuser => { + let active_subuser_entry_secondary_user = secondary_user::ActiveModel { + user_id: sea_orm::Set(subuser.id), + rpc_secret_key_id: sea_orm::Set(rpc_key_entity.id), + role: sea_orm::Set(new_role.clone()), + ..Default::default() + }; + active_subuser_entry_secondary_user.insert(&txn).await?; + action = "added"; + } + _ => { + // Return if the user should be removed and if there is no entry; + // in this case, the user is not entered + + // Return if the user should be added and there is already an entry; + // in this case, they were already added, so we can skip this + // Do nothing in this case + } + }; + txn.commit().await?; + + let response = ( + StatusCode::OK, + Json(json!({ + "rpc_key": rpc_key_to_modify, + "subuser_address": subuser_address, + "keep_user": keep_subuser, + "new_role": new_role, + "action": action + })), + ) + .into_response(); + // Return early if the log was added, assume there is at most one valid log per transaction + Ok(response.into()) +} diff --git a/web3_proxy/src/http_params.rs b/web3_proxy/src/http_params.rs index 1b31f1c2..c2909e01 100644 --- a/web3_proxy/src/http_params.rs +++ b/web3_proxy/src/http_params.rs @@ -232,20 +232,23 @@ pub fn get_query_window_seconds_from_params( pub fn get_stats_column_from_params(params: &HashMap) -> Web3ProxyResult<&str> { params.get("query_stats_column").map_or_else( - || Ok("frontend_requests"), + || Ok(""), |query_stats_column: &String| { // Must be one of: Otherwise respond with an error ... match query_stats_column.as_str() { - "frontend_requests" + "" + | "frontend_requests" | "backend_requests" | "cache_hits" | "cache_misses" | "no_servers" | "sum_request_bytes" | "sum_response_bytes" - | "sum_response_millis" => Ok(query_stats_column), + | "sum_response_millis" + | "sum_credits_used" + | "balance" => Ok(query_stats_column), _ => Err(Web3ProxyError::BadRequest( - "Unable to parse query_stats_column. It must be one of: \ + "Unable to parse query_stats_column. It must be empty, or one of: \ frontend_requests, \ backend_requests, \ cache_hits, \ @@ -253,7 +256,9 @@ pub fn get_stats_column_from_params(params: &HashMap) -> Web3Pro no_servers, \ sum_request_bytes, \ sum_response_bytes, \ - sum_response_millis" + sum_response_millis, \ + sum_credits_used, \ + balance" .to_string(), )), } diff --git a/web3_proxy/src/lib.rs b/web3_proxy/src/lib.rs index c57695b9..1f29beb3 100644 --- a/web3_proxy/src/lib.rs +++ b/web3_proxy/src/lib.rs @@ -7,6 +7,7 @@ pub mod http_params; pub mod jsonrpc; pub mod pagerduty; pub mod prometheus; +pub mod referral_code; pub mod rpcs; pub mod stats; pub mod user_token; @@ -30,4 +31,5 @@ pub struct PostLoginQuery { pub struct PostLogin { sig: String, msg: String, + pub referral_code: Option, } diff --git a/web3_proxy/src/referral_code.rs b/web3_proxy/src/referral_code.rs new file mode 100644 index 00000000..d5343e84 --- /dev/null +++ b/web3_proxy/src/referral_code.rs @@ -0,0 +1,24 @@ +use anyhow::{self, Result}; +use ulid::Ulid; + +pub struct ReferralCode(pub String); + +impl Default for ReferralCode { + fn default() -> Self { + let out = Ulid::new(); + Self(format!("llamanodes-{}", out)) + } +} + +impl TryFrom for ReferralCode { + type Error = anyhow::Error; + + fn try_from(x: String) -> Result { + if !x.starts_with("llamanodes-") { + return Err(anyhow::anyhow!( + "Referral Code does not have the right format" + )); + } + Ok(Self(x)) + } +} diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index b6efd970..dd9c3e1c 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -4,7 +4,6 @@ use super::consensus::ConsensusWeb3Rpcs; use super::one::Web3Rpc; use super::request::{OpenRequestHandle, OpenRequestResult, RequestErrorHandler}; use crate::app::{flatten_handle, AnyhowJoinHandle, Web3ProxyApp}; -///! Load balanced communication with a group of web3 providers use crate::config::{BlockAndRpc, TxHashAndRpc, Web3RpcConfig}; use crate::frontend::authorization::{Authorization, RequestMetadata}; use crate::frontend::errors::{Web3ProxyError, Web3ProxyResult}; diff --git a/web3_proxy/src/stats/db_queries.rs b/web3_proxy/src/stats/db_queries.rs index eab9495b..ccc9404a 100644 --- a/web3_proxy/src/stats/db_queries.rs +++ b/web3_proxy/src/stats/db_queries.rs @@ -14,7 +14,6 @@ use axum::{ }; use entities::{rpc_accounting, rpc_key}; use hashbrown::HashMap; -use http::StatusCode; use log::warn; use migration::sea_orm::{ ColumnTrait, EntityTrait, PaginatorTrait, QueryFilter, QueryOrder, QuerySelect, Select, @@ -209,11 +208,7 @@ pub async fn query_user_stats<'a>( // TODO: move getting the param and checking the bearer token into a helper function if let Some(rpc_key_id) = params.get("rpc_key_id") { let rpc_key_id = rpc_key_id.parse::().map_err(|e| { - Web3ProxyError::StatusCode( - StatusCode::BAD_REQUEST, - "Unable to parse rpc_key_id".to_string(), - Some(e.into()), - ) + Web3ProxyError::BadRequest(format!("Unable to parse rpc_key_id. {:?}", e)) })?; response_body.insert("rpc_key_id", serde_json::Value::Number(rpc_key_id.into())); diff --git a/web3_proxy/src/stats/influxdb_queries.rs b/web3_proxy/src/stats/influxdb_queries.rs index 209f2d1c..d946dc2d 100644 --- a/web3_proxy/src/stats/influxdb_queries.rs +++ b/web3_proxy/src/stats/influxdb_queries.rs @@ -1,11 +1,11 @@ use super::StatType; -use crate::http_params::get_stats_column_from_params; +use crate::frontend::errors::Web3ProxyErrorContext; use crate::{ app::Web3ProxyApp, frontend::errors::{Web3ProxyError, Web3ProxyResponse}, http_params::{ get_chain_id_from_params, get_query_start_from_params, get_query_stop_from_params, - get_query_window_seconds_from_params, get_user_id_from_params, + get_query_window_seconds_from_params, }, }; use anyhow::Context; @@ -14,38 +14,18 @@ use axum::{ response::IntoResponse, Json, TypedHeader, }; -use chrono::{DateTime, FixedOffset}; +use entities::sea_orm_active_enums::Role; +use entities::{rpc_key, secondary_user}; use fstrings::{f, format_args_f}; use hashbrown::HashMap; +use influxdb2::api::query::FluxRecord; use influxdb2::models::Query; -use influxdb2::FromDataPoint; -use itertools::Itertools; -use log::trace; -use serde::Serialize; -use serde_json::{json, Number, Value}; - -// This type-API is extremely brittle! Make sure that the types conform 1-to-1 as defined here -// https://docs.rs/influxdb2-structmap/0.2.0/src/influxdb2_structmap/value.rs.html#1-98 -#[derive(Debug, Default, FromDataPoint, Serialize)] -pub struct AggregatedRpcAccounting { - chain_id: String, - _field: String, - _value: i64, - _time: DateTime, - error_response: String, - archive_needed: String, -} - -#[derive(Debug, Default, FromDataPoint, Serialize)] -pub struct DetailedRpcAccounting { - chain_id: String, - _field: String, - _value: i64, - _time: DateTime, - error_response: String, - archive_needed: String, - method: String, -} +use log::{error, info, warn}; +use migration::sea_orm::ColumnTrait; +use migration::sea_orm::EntityTrait; +use migration::sea_orm::QueryFilter; +use serde_json::json; +use ulid::Ulid; pub async fn query_user_stats<'a>( app: &'a Web3ProxyApp, @@ -53,15 +33,17 @@ pub async fn query_user_stats<'a>( params: &'a HashMap, stat_response_type: StatType, ) -> Web3ProxyResponse { - let db_conn = app.db_conn().context("query_user_stats needs a db")?; + let user_id = match bearer { + Some(inner_bearer) => { + let (user, _semaphore) = app.bearer_is_authorized(inner_bearer.0 .0).await?; + user.id + } + None => 0, + }; + let db_replica = app .db_replica() .context("query_user_stats needs a db replica")?; - let mut redis_conn = app - .redis_conn() - .await - .context("query_user_stats had a redis connection error")? - .context("query_user_stats needs a redis")?; // TODO: have a getter for this. do we need a connection pool on it? let influxdb_client = app @@ -69,22 +51,15 @@ pub async fn query_user_stats<'a>( .as_ref() .context("query_user_stats needs an influxdb client")?; - // get the user id first. if it is 0, we should use a cache on the app - let user_id = - get_user_id_from_params(&mut redis_conn, &db_conn, &db_replica, bearer, params).await?; - let query_window_seconds = get_query_window_seconds_from_params(params)?; let query_start = get_query_start_from_params(params)?.timestamp(); let query_stop = get_query_stop_from_params(params)?.timestamp(); let chain_id = get_chain_id_from_params(app, params)?; - let stats_column = get_stats_column_from_params(params)?; - - // query_window_seconds must be provided, and should be not 1s (?) by default .. // Return a bad request if query_start == query_stop, because then the query is empty basically if query_start == query_stop { return Err(Web3ProxyError::BadRequest( - "query_start and query_stop date cannot be equal. Please specify a different range" + "Start and Stop date cannot be equal. Please specify a (different) start date." .to_owned(), )); } @@ -95,273 +70,400 @@ pub async fn query_user_stats<'a>( "opt_in_proxy" }; + let mut join_candidates: Vec = vec![ + "_time".to_string(), + "_measurement".to_string(), + "chain_id".to_string(), + ]; + + // Include a hashmap to go from rpc_secret_key_id to the rpc_secret_key + let mut rpc_key_id_to_key = HashMap::new(); + + let rpc_key_filter = if user_id == 0 { + "".to_string() + } else { + // Fetch all rpc_secret_key_ids, and filter for these + let mut user_rpc_keys = rpc_key::Entity::find() + .filter(rpc_key::Column::UserId.eq(user_id)) + .all(db_replica.conn()) + .await + .web3_context("failed loading user's key")? + .into_iter() + .map(|x| { + let key = x.id.to_string(); + let val = Ulid::from(x.secret_key); + rpc_key_id_to_key.insert(key.clone(), val); + key + }) + .collect::>(); + + // Fetch all rpc_keys where we are the subuser + let mut subuser_rpc_keys = secondary_user::Entity::find() + .filter(secondary_user::Column::UserId.eq(user_id)) + .find_also_related(rpc_key::Entity) + .all(db_replica.conn()) + // TODO: Do a join with rpc-keys + .await + .web3_context("failed loading subuser keys")? + .into_iter() + .flat_map( + |(subuser, wrapped_shared_rpc_key)| match wrapped_shared_rpc_key { + Some(shared_rpc_key) => { + if subuser.role == Role::Admin || subuser.role == Role::Owner { + let key = shared_rpc_key.id.to_string(); + let val = Ulid::from(shared_rpc_key.secret_key); + rpc_key_id_to_key.insert(key.clone(), val); + Some(key) + } else { + None + } + } + None => None, + }, + ) + .collect::>(); + + user_rpc_keys.append(&mut subuser_rpc_keys); + + if user_rpc_keys.len() == 0 { + return Err(Web3ProxyError::BadRequest( + "User has no secret RPC keys yet".to_string(), + )); + } + + // Make the tables join on the rpc_key_id as well: + join_candidates.push("rpc_secret_key_id".to_string()); + + // Iterate, pop and add to string + f!( + r#"|> filter(fn: (r) => contains(value: r["rpc_secret_key_id"], set: {:?}))"#, + user_rpc_keys + ) + }; + + // TODO: Turn into a 500 error if bucket is not found .. + // Or just unwrap or so let bucket = &app .config .influxdb_bucket .clone() - .context("No influxdb bucket was provided")?; - trace!("Bucket is {:?}", bucket); + .context("No influxdb bucket was provided")?; // "web3_proxy"; - let mut group_columns = vec![ - "chain_id", - "_measurement", - "_field", - "_measurement", - "error_response", - "archive_needed", - ]; + info!("Bucket is {:?}", bucket); let mut filter_chain_id = "".to_string(); - - // Add to group columns the method, if we want the detailed view as well - if let StatType::Detailed = stat_response_type { - group_columns.push("method"); - } - - if chain_id == 0 { - group_columns.push("chain_id"); - } else { + if chain_id != 0 { filter_chain_id = f!(r#"|> filter(fn: (r) => r["chain_id"] == "{chain_id}")"#); } - let group_columns = serde_json::to_string(&json!(group_columns)).unwrap(); + // Fetch and request for balance - let group = match stat_response_type { - StatType::Aggregated => f!(r#"|> group(columns: {group_columns})"#), - StatType::Detailed => "".to_string(), - }; + info!( + "Query start and stop are: {:?} {:?}", + query_start, query_stop + ); + // info!("Query column parameters are: {:?}", stats_column); + info!("Query measurement is: {:?}", measurement); + info!("Filters are: {:?}", filter_chain_id); // filter_field + info!("window seconds are: {:?}", query_window_seconds); - let filter_field = match stat_response_type { - StatType::Aggregated => { - f!(r#"|> filter(fn: (r) => r["_field"] == "{stats_column}")"#) + let drop_method = match stat_response_type { + StatType::Aggregated => f!(r#"|> drop(columns: ["method"])"#), + StatType::Detailed => { + // Make the tables join on the method column as well + join_candidates.push("method".to_string()); + "".to_string() } - // TODO: Detailed should still filter it, but just "group-by" method (call it once per each method ... - // Or maybe it shouldn't filter it ... - StatType::Detailed => "".to_string(), }; - - trace!("query time range: {:?} - {:?}", query_start, query_stop); - trace!("stats_column: {:?}", stats_column); - trace!("measurement: {:?}", measurement); - trace!("filters: {:?} {:?}", filter_field, filter_chain_id); - trace!("group: {:?}", group); - trace!("query_window_seconds: {:?}", query_window_seconds); + let join_candidates = f!(r#"{:?}"#, join_candidates); let query = f!(r#" - from(bucket: "{bucket}") - |> range(start: {query_start}, stop: {query_stop}) - |> filter(fn: (r) => r["_measurement"] == "{measurement}") - {filter_field} - {filter_chain_id} - {group} - |> aggregateWindow(every: {query_window_seconds}s, fn: sum, createEmpty: false) - |> group() + base = from(bucket: "{bucket}") + |> range(start: {query_start}, stop: {query_stop}) + {rpc_key_filter} + |> filter(fn: (r) => r["_measurement"] == "{measurement}") + {filter_chain_id} + {drop_method} + + cumsum = base + |> aggregateWindow(every: {query_window_seconds}s, fn: sum, createEmpty: false) + |> pivot(rowKey: ["_time"], columnKey: ["_field"], valueColumn: "_value") + |> drop(columns: ["balance"]) + |> map(fn: (r) => ({{ r with "archive_needed": if r.archive_needed == "true" then r.frontend_requests else 0}})) + |> map(fn: (r) => ({{ r with "error_response": if r.error_response == "true" then r.frontend_requests else 0}})) + |> group(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"]) + |> sort(columns: ["frontend_requests"]) + |> map(fn:(r) => ({{ r with "sum_credits_used": float(v: r["sum_credits_used"]) }})) + |> cumulativeSum(columns: ["archive_needed", "error_response", "backend_requests", "cache_hits", "cache_misses", "frontend_requests", "sum_credits_used", "sum_request_bytes", "sum_response_bytes", "sum_response_millis"]) + |> sort(columns: ["frontend_requests"], desc: true) + |> limit(n: 1) + |> group() + |> sort(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"], desc: true) + + balance = base + |> toFloat() + |> aggregateWindow(every: {query_window_seconds}s, fn: mean, createEmpty: false) + |> pivot(rowKey: ["_time"], columnKey: ["_field"], valueColumn: "_value") + |> group(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"]) + |> mean(column: "balance") + |> group() + |> sort(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"], desc: true) + + join( + tables: {{cumsum, balance}}, + on: {join_candidates} + ) "#); - trace!("Raw query to db is: {:?}", query); + info!("Raw query to db is: {:?}", query); let query = Query::new(query.to_string()); - trace!("Query to db is: {:?}", query); + info!("Query to db is: {:?}", query); - // Return a different result based on the query - let datapoints = match stat_response_type { - StatType::Aggregated => { - let influx_responses: Vec = influxdb_client - .query::(Some(query)) - .await?; - trace!("Influx responses are {:?}", &influx_responses); - for res in &influx_responses { - trace!("Resp is: {:?}", res); - } + // Make the query and collect all data + let raw_influx_responses: Vec = + influxdb_client.query_raw(Some(query.clone())).await?; - influx_responses - .into_iter() - .map(|x| (x._time, x)) - .into_group_map() - .into_iter() - .map(|(group, grouped_items)| { - trace!("Group is: {:?}", group); - - // Now put all the fields next to each other - // (there will be exactly one field per timestamp, but we want to arrive at a new object) - let mut out = HashMap::new(); - // Could also add a timestamp - - let mut archive_requests = 0; - let mut error_responses = 0; - - out.insert("method".to_owned(), json!("null")); - - for x in grouped_items { - trace!("Iterating over grouped item {:?}", x); - - let key = format!("total_{}", x._field).to_string(); - trace!("Looking at {:?}: {:?}", key, x._value); - - // Insert it once, and then fix it - match out.get_mut(&key) { - Some(existing) => { - match existing { - Value::Number(old_value) => { - trace!("Old value is {:?}", old_value); - // unwrap will error when someone has too many credits .. - let old_value = old_value.as_i64().unwrap(); - *existing = serde_json::Value::Number(Number::from( - old_value + x._value, - )); - trace!("New value is {:?}", existing); - } - _ => { - panic!("Should be nothing but a number") - } - }; + // Basically rename all items to be "total", + // calculate number of "archive_needed" and "error_responses" through their boolean representations ... + // HashMap + // let mut datapoints = HashMap::new(); + // TODO: I must be able to probably zip the balance query... + let datapoints = raw_influx_responses + .into_iter() + // .into_values() + .map(|x| x.values) + .map(|value_map| { + // Unwrap all relevant numbers + // BTreeMap + let mut out: HashMap = HashMap::new(); + value_map.into_iter().for_each(|(key, value)| { + if key == "_measurement" { + match value { + influxdb2_structmap::value::Value::String(inner) => { + if inner == "opt_in_proxy" { + out.insert( + "collection".to_owned(), + serde_json::Value::String("opt-in".to_owned()), + ); + } else if inner == "global_proxy" { + out.insert( + "collection".to_owned(), + serde_json::Value::String("global".to_owned()), + ); + } else { + warn!("Some datapoints are not part of any _measurement!"); + out.insert( + "collection".to_owned(), + serde_json::Value::String("unknown".to_owned()), + ); } - None => { - trace!("Does not exist yet! Insert new!"); - out.insert(key, serde_json::Value::Number(Number::from(x._value))); - } - }; - - if !out.contains_key("query_window_timestamp") { - out.insert( - "query_window_timestamp".to_owned(), - // serde_json::Value::Number(x.time.timestamp().into()) - json!(x._time.timestamp()), - ); } - - // Interpret archive needed as a boolean - let archive_needed = match x.archive_needed.as_str() { - "true" => true, - "false" => false, - _ => { - panic!("This should never be!") - } - }; - let error_response = match x.error_response.as_str() { - "true" => true, - "false" => false, - _ => { - panic!("This should never be!") - } - }; - - // Add up to archive requests and error responses - // TODO: Gotta double check if errors & archive is based on frontend requests, or other metrics - if x._field == "frontend_requests" && archive_needed { - archive_requests += x._value as u64 // This is the number of requests - } - if x._field == "frontend_requests" && error_response { - error_responses += x._value as u64 + _ => { + error!("_measurement should always be a String!"); } } - - out.insert("archive_request".to_owned(), json!(archive_requests)); - out.insert("error_response".to_owned(), json!(error_responses)); - - json!(out) - }) - .collect::>() - } - StatType::Detailed => { - let influx_responses: Vec = influxdb_client - .query::(Some(query)) - .await?; - trace!("Influx responses are {:?}", &influx_responses); - for res in &influx_responses { - trace!("Resp is: {:?}", res); - } - - // Group by all fields together .. - influx_responses - .into_iter() - .map(|x| ((x._time, x.method.clone()), x)) - .into_group_map() - .into_iter() - .map(|(group, grouped_items)| { - // Now put all the fields next to each other - // (there will be exactly one field per timestamp, but we want to arrive at a new object) - let mut out = HashMap::new(); - // Could also add a timestamp - - let mut archive_requests = 0; - let mut error_responses = 0; - - // Should probably move this outside ... (?) - let method = group.1; - out.insert("method".to_owned(), json!(method)); - - for x in grouped_items { - trace!("Iterating over grouped item {:?}", x); - - let key = format!("total_{}", x._field).to_string(); - trace!("Looking at {:?}: {:?}", key, x._value); - - // Insert it once, and then fix it - match out.get_mut(&key) { - Some(existing) => { - match existing { - Value::Number(old_value) => { - trace!("Old value is {:?}", old_value); - - // unwrap will error when someone has too many credits .. - let old_value = old_value.as_i64().unwrap(); - *existing = serde_json::Value::Number(Number::from( - old_value + x._value, - )); - - trace!("New value is {:?}", existing.as_i64()); - } - _ => { - panic!("Should be nothing but a number") - } - }; - } - None => { - trace!("Does not exist yet! Insert new!"); - out.insert(key, serde_json::Value::Number(Number::from(x._value))); - } - }; - - if !out.contains_key("query_window_timestamp") { + } else if key == "_stop" { + match value { + influxdb2_structmap::value::Value::TimeRFC(inner) => { out.insert( - "query_window_timestamp".to_owned(), - json!(x._time.timestamp()), + "stop_time".to_owned(), + serde_json::Value::String(inner.to_string()), ); } - - // Interpret archive needed as a boolean - let archive_needed = match x.archive_needed.as_str() { - "true" => true, - "false" => false, - _ => { - panic!("This should never be!") - } - }; - let error_response = match x.error_response.as_str() { - "true" => true, - "false" => false, - _ => { - panic!("This should never be!") - } - }; - - // Add up to archive requests and error responses - // TODO: Gotta double check if errors & archive is based on frontend requests, or other metrics - if x._field == "frontend_requests" && archive_needed { - archive_requests += x._value as i32 // This is the number of requests + _ => { + error!("_stop should always be a TimeRFC!"); } - if x._field == "frontend_requests" && error_response { - error_responses += x._value as i32 + }; + } else if key == "_time" { + match value { + influxdb2_structmap::value::Value::TimeRFC(inner) => { + out.insert( + "time".to_owned(), + serde_json::Value::String(inner.to_string()), + ); + } + _ => { + error!("_stop should always be a TimeRFC!"); } } + } else if key == "backend_requests" { + match value { + influxdb2_structmap::value::Value::Long(inner) => { + out.insert( + "total_backend_requests".to_owned(), + serde_json::Value::Number(inner.into()), + ); + } + _ => { + error!("backend_requests should always be a Long!"); + } + } + } else if key == "balance" { + match value { + influxdb2_structmap::value::Value::Double(inner) => { + out.insert("balance".to_owned(), json!(f64::from(inner))); + } + _ => { + error!("balance should always be a Double!"); + } + } + } else if key == "cache_hits" { + match value { + influxdb2_structmap::value::Value::Long(inner) => { + out.insert( + "total_cache_hits".to_owned(), + serde_json::Value::Number(inner.into()), + ); + } + _ => { + error!("cache_hits should always be a Long!"); + } + } + } else if key == "cache_misses" { + match value { + influxdb2_structmap::value::Value::Long(inner) => { + out.insert( + "total_cache_misses".to_owned(), + serde_json::Value::Number(inner.into()), + ); + } + _ => { + error!("cache_misses should always be a Long!"); + } + } + } else if key == "frontend_requests" { + match value { + influxdb2_structmap::value::Value::Long(inner) => { + out.insert( + "total_frontend_requests".to_owned(), + serde_json::Value::Number(inner.into()), + ); + } + _ => { + error!("frontend_requests should always be a Long!"); + } + } + } else if key == "no_servers" { + match value { + influxdb2_structmap::value::Value::Long(inner) => { + out.insert( + "no_servers".to_owned(), + serde_json::Value::Number(inner.into()), + ); + } + _ => { + error!("no_servers should always be a Long!"); + } + } + } else if key == "sum_credits_used" { + match value { + influxdb2_structmap::value::Value::Double(inner) => { + out.insert("total_credits_used".to_owned(), json!(f64::from(inner))); + } + _ => { + error!("sum_credits_used should always be a Double!"); + } + } + } else if key == "sum_request_bytes" { + match value { + influxdb2_structmap::value::Value::Long(inner) => { + out.insert( + "total_request_bytes".to_owned(), + serde_json::Value::Number(inner.into()), + ); + } + _ => { + error!("sum_request_bytes should always be a Long!"); + } + } + } else if key == "sum_response_bytes" { + match value { + influxdb2_structmap::value::Value::Long(inner) => { + out.insert( + "total_response_bytes".to_owned(), + serde_json::Value::Number(inner.into()), + ); + } + _ => { + error!("sum_response_bytes should always be a Long!"); + } + } + } else if key == "rpc_secret_key_id" { + match value { + influxdb2_structmap::value::Value::String(inner) => { + out.insert( + "rpc_key".to_owned(), + serde_json::Value::String( + rpc_key_id_to_key.get(&inner).unwrap().to_string(), + ), + ); + } + _ => { + error!("rpc_secret_key_id should always be a String!"); + } + } + } else if key == "sum_response_millis" { + match value { + influxdb2_structmap::value::Value::Long(inner) => { + out.insert( + "total_response_millis".to_owned(), + serde_json::Value::Number(inner.into()), + ); + } + _ => { + error!("sum_response_millis should always be a Long!"); + } + } + } + // Make this if detailed ... + else if stat_response_type == StatType::Detailed && key == "method" { + match value { + influxdb2_structmap::value::Value::String(inner) => { + out.insert("method".to_owned(), serde_json::Value::String(inner)); + } + _ => { + error!("method should always be a String!"); + } + } + } else if key == "chain_id" { + match value { + influxdb2_structmap::value::Value::String(inner) => { + out.insert("chain_id".to_owned(), serde_json::Value::String(inner)); + } + _ => { + error!("chain_id should always be a String!"); + } + } + } else if key == "archive_needed" { + match value { + influxdb2_structmap::value::Value::Long(inner) => { + out.insert( + "archive_needed".to_owned(), + serde_json::Value::Number(inner.into()), + ); + } + _ => { + error!("archive_needed should always be a Long!"); + } + } + } else if key == "error_response" { + match value { + influxdb2_structmap::value::Value::Long(inner) => { + out.insert( + "error_response".to_owned(), + serde_json::Value::Number(inner.into()), + ); + } + _ => { + error!("error_response should always be a Long!"); + } + } + } + }); - out.insert("archive_request".to_owned(), json!(archive_requests)); - out.insert("error_response".to_owned(), json!(error_responses)); - - json!(out) - }) - .collect::>() - } - }; + // datapoints.insert(out.get("time"), out); + json!(out) + }) + .collect::>(); // I suppose archive requests could be either gathered by default (then summed up), or retrieved on a second go. // Same with error responses .. diff --git a/web3_proxy/src/stats/mod.rs b/web3_proxy/src/stats/mod.rs index 0b9a7411..af148e7d 100644 --- a/web3_proxy/src/stats/mod.rs +++ b/web3_proxy/src/stats/mod.rs @@ -1,22 +1,29 @@ //! Store "stats" in a database for billing and a different database for graphing -//! //! TODO: move some of these structs/functions into their own file? pub mod db_queries; pub mod influxdb_queries; - +use crate::app::AuthorizationChecks; use crate::frontend::authorization::{Authorization, RequestMetadata}; +use anyhow::Context; use axum::headers::Origin; -use chrono::{TimeZone, Utc}; +use chrono::{DateTime, Months, TimeZone, Utc}; use derive_more::From; -use entities::rpc_accounting_v2; use entities::sea_orm_active_enums::TrackingLevel; +use entities::{balance, referee, referrer, rpc_accounting_v2, rpc_key, user, user_tier}; use futures::stream; use hashbrown::HashMap; use influxdb2::api::write::TimestampPrecision; use influxdb2::models::DataPoint; -use log::{error, info, trace}; -use migration::sea_orm::{self, DatabaseConnection, EntityTrait}; +use log::{error, info, trace, warn}; +use migration::sea_orm::prelude::Decimal; +use migration::sea_orm::ActiveModelTrait; +use migration::sea_orm::ColumnTrait; +use migration::sea_orm::IntoActiveModel; +use migration::sea_orm::{self, DatabaseConnection, EntityTrait, QueryFilter}; use migration::{Expr, OnConflict}; +use moka::future::Cache; +use num_traits::ToPrimitive; +use std::cmp::max; use std::num::NonZeroU64; use std::sync::atomic::Ordering; use std::sync::Arc; @@ -24,7 +31,9 @@ use std::time::Duration; use tokio::sync::broadcast; use tokio::task::JoinHandle; use tokio::time::interval; +use ulid::Ulid; +#[derive(Debug, PartialEq, Eq)] pub enum StatType { Aggregated, Detailed, @@ -45,6 +54,8 @@ pub struct RpcQueryStats { pub response_bytes: u64, pub response_millis: u64, pub response_timestamp: i64, + /// Credits used signifies how how much money was used up + pub credits_used: Decimal, } #[derive(Clone, From, Hash, PartialEq, Eq)] @@ -104,6 +115,8 @@ impl RpcQueryStats { } }; + // Depending on method, add some arithmetic around calculating credits_used + // I think balance should not go here, this looks more like a key thingy RpcQueryKey { response_timestamp, archive_needed: self.archive_request, @@ -179,6 +192,9 @@ pub struct BufferedRpcQueryStats { pub sum_request_bytes: u64, pub sum_response_bytes: u64, pub sum_response_millis: u64, + pub sum_credits_used: Decimal, + /// Balance tells us the user's balance at this point in time + pub latest_balance: Decimal, } /// A stat that we aggregate and then store in a database. @@ -200,6 +216,8 @@ pub struct StatBuffer { db_conn: Option, influxdb_client: Option, tsdb_save_interval_seconds: u32, + rpc_secret_key_cache: + Option>, db_save_interval_seconds: u32, billing_period_seconds: i64, global_timeseries_buffer: HashMap, @@ -227,6 +245,14 @@ impl BufferedRpcQueryStats { self.sum_request_bytes += stat.request_bytes; self.sum_response_bytes += stat.response_bytes; self.sum_response_millis += stat.response_millis; + self.sum_credits_used += stat.credits_used; + + // Also record the latest balance for this user .. + self.latest_balance = stat + .authorization + .checks + .balance + .unwrap_or(Decimal::from(0)); } // TODO: take a db transaction instead so that we can batch? @@ -242,10 +268,8 @@ impl BufferedRpcQueryStats { let accounting_entry = rpc_accounting_v2::ActiveModel { id: sea_orm::NotSet, rpc_key_id: sea_orm::Set(key.rpc_secret_key_id.map(Into::into).unwrap_or_default()), - origin: sea_orm::Set(key.origin.map(|x| x.to_string()).unwrap_or_default()), chain_id: sea_orm::Set(chain_id), period_datetime: sea_orm::Set(period_datetime), - method: sea_orm::Set(key.method.unwrap_or_default()), archive_needed: sea_orm::Set(key.archive_needed), error_response: sea_orm::Set(key.error_response), frontend_requests: sea_orm::Set(self.frontend_requests), @@ -257,6 +281,7 @@ impl BufferedRpcQueryStats { sum_request_bytes: sea_orm::Set(self.sum_request_bytes), sum_response_millis: sea_orm::Set(self.sum_response_millis), sum_response_bytes: sea_orm::Set(self.sum_response_bytes), + sum_credits_used: sea_orm::Set(self.sum_credits_used), }; rpc_accounting_v2::Entity::insert(accounting_entry) @@ -306,12 +331,215 @@ impl BufferedRpcQueryStats { Expr::col(rpc_accounting_v2::Column::SumResponseBytes) .add(self.sum_response_bytes), ), + ( + rpc_accounting_v2::Column::SumCreditsUsed, + Expr::col(rpc_accounting_v2::Column::SumCreditsUsed) + .add(self.sum_credits_used), + ), ]) .to_owned(), ) .exec(db_conn) .await?; + // TODO: Refactor this function a bit more just so it looks and feels nicer + // TODO: Figure out how to go around unmatching, it shouldn't return an error, but this is disgusting + + // All the referral & balance arithmetic takes place here + let rpc_secret_key_id: u64 = match key.rpc_secret_key_id { + Some(x) => x.into(), + // Return early if the RPC key is not found, because then it is an anonymous user + None => return Ok(()), + }; + + // (1) Get the user with that RPC key. This is the referee + let sender_rpc_key = rpc_key::Entity::find() + .filter(rpc_key::Column::Id.eq(rpc_secret_key_id)) + .one(db_conn) + .await?; + + // Technicall there should always be a user ... still let's return "Ok(())" for now + let sender_user_id: u64 = match sender_rpc_key { + Some(x) => x.user_id.into(), + // Return early if the User is not found, because then it is an anonymous user + // Let's also issue a warning because obviously the RPC key should correspond to a user + None => { + warn!( + "No user was found for the following rpc key: {:?}", + rpc_secret_key_id + ); + return Ok(()); + } + }; + + // (1) Do some general bookkeeping on the user + let sender_balance = match balance::Entity::find() + .filter(balance::Column::UserId.eq(sender_user_id)) + .one(db_conn) + .await? + { + Some(x) => x, + None => { + warn!("This user id has no balance entry! {:?}", sender_user_id); + return Ok(()); + } + }; + + let mut active_sender_balance = sender_balance.clone().into_active_model(); + + // Still subtract from the user in any case, + // Modify the balance of the sender completely (in mysql, next to the stats) + // In any case, add this to "spent" + active_sender_balance.used_balance = + sea_orm::Set(sender_balance.used_balance + Decimal::from(self.sum_credits_used)); + + // Also update the available balance + let new_available_balance = max( + sender_balance.available_balance - Decimal::from(self.sum_credits_used), + Decimal::from(0), + ); + active_sender_balance.available_balance = sea_orm::Set(new_available_balance); + + active_sender_balance.save(db_conn).await?; + + let downgrade_user = match user::Entity::find() + .filter(user::Column::Id.eq(sender_user_id)) + .one(db_conn) + .await? + { + Some(x) => x, + None => { + warn!("No user was found with this sender id!"); + return Ok(()); + } + }; + + let downgrade_user_role = user_tier::Entity::find() + .filter(user_tier::Column::Id.eq(downgrade_user.user_tier_id)) + .one(db_conn) + .await? + .context(format!( + "The foreign key for the user's user_tier_id was not found! {:?}", + downgrade_user.user_tier_id + ))?; + + // Downgrade a user to premium - out of funds if there's less than 10$ in the account, and if the user was premium before + if new_available_balance < Decimal::from(10u64) && downgrade_user_role.title == "Premium" { + // Only downgrade the user in local process memory, not elsewhere + // app.rpc_secret_key_cache- + + // let mut active_downgrade_user = downgrade_user.into_active_model(); + // active_downgrade_user.user_tier_id = sea_orm::Set(downgrade_user_role.id); + // active_downgrade_user.save(db_conn).await?; + } + + // Get the referee, and the referrer + // (2) Look up the code that this user used. This is the referee table + let referee_object = match referee::Entity::find() + .filter(referee::Column::UserId.eq(sender_user_id)) + .one(db_conn) + .await? + { + Some(x) => x, + None => { + warn!( + "No referral code was found for this user: {:?}", + sender_user_id + ); + return Ok(()); + } + }; + + // (3) Look up the matching referrer in the referrer table + // Referral table -> Get the referee id + let user_with_that_referral_code = match referrer::Entity::find() + .filter(referrer::Column::ReferralCode.eq(referee_object.used_referral_code)) + .one(db_conn) + .await? + { + Some(x) => x, + None => { + warn!( + "No referrer with that referral code was found {:?}", + referee_object + ); + return Ok(()); + } + }; + + // Ok, now we add the credits to both users if applicable... + // (4 onwards) Add balance to the referrer, + + // (5) Check if referee has used up $100.00 USD in total (Have a config item that says how many credits account to 1$) + // Get balance for the referrer (optionally make it into an active model ...) + let sender_balance = match balance::Entity::find() + .filter(balance::Column::UserId.eq(referee_object.user_id)) + .one(db_conn) + .await? + { + Some(x) => x, + None => { + warn!( + "This user id has no balance entry! {:?}", + referee_object.user_id + ); + return Ok(()); + } + }; + + let mut active_sender_balance = sender_balance.clone().into_active_model(); + let referrer_balance = match balance::Entity::find() + .filter(balance::Column::UserId.eq(user_with_that_referral_code.user_id)) + .one(db_conn) + .await? + { + Some(x) => x, + None => { + warn!( + "This user id has no balance entry! {:?}", + referee_object.user_id + ); + return Ok(()); + } + }; + + // I could try to circumvene the clone here, but let's skip that for now + let mut active_referee = referee_object.clone().into_active_model(); + + // (5.1) If not, go to (7). If yes, go to (6) + // Hardcode this parameter also in config, so it's easier to tune + if !referee_object.credits_applied_for_referee + && (sender_balance.used_balance + self.sum_credits_used) >= Decimal::from(100) + { + // (6) If the credits have not yet been applied to the referee, apply 10M credits / $100.00 USD worth of credits. + // Make it into an active model, and add credits + active_sender_balance.available_balance = + sea_orm::Set(sender_balance.available_balance + Decimal::from(100)); + // Also mark referral as "credits_applied_for_referee" + active_referee.credits_applied_for_referee = sea_orm::Set(true); + } + + // (7) If the referral-start-date has not been passed, apply 10% of the credits to the referrer. + let now = Utc::now(); + let valid_until = DateTime::::from_utc(referee_object.referral_start_date, Utc) + .checked_add_months(Months::new(12)) + .unwrap(); + if now <= valid_until { + let mut active_referrer_balance = referrer_balance.clone().into_active_model(); + // Add 10% referral fees ... + active_referrer_balance.available_balance = sea_orm::Set( + referrer_balance.available_balance + + Decimal::from(self.sum_credits_used / Decimal::from(10)), + ); + // Also record how much the current referrer has "provided" / "gifted" away + active_referee.credits_applied_for_referrer = + sea_orm::Set(referee_object.credits_applied_for_referrer + self.sum_credits_used); + active_referrer_balance.save(db_conn).await?; + } + + active_sender_balance.save(db_conn).await?; + active_referee.save(db_conn).await?; + Ok(()) } @@ -343,7 +571,24 @@ impl BufferedRpcQueryStats { .field("cache_hits", self.cache_hits as i64) .field("sum_request_bytes", self.sum_request_bytes as i64) .field("sum_response_millis", self.sum_response_millis as i64) - .field("sum_response_bytes", self.sum_response_bytes as i64); + .field("sum_response_bytes", self.sum_response_bytes as i64) + // TODO: will this be enough of a range + // I guess Decimal can be a f64 + // TODO: This should prob be a float, i should change the query if we want float-precision for this (which would be important...) + .field( + "sum_credits_used", + self.sum_credits_used + .to_f64() + .expect("number is really (too) large"), + ) + .field( + "balance", + self.latest_balance + .to_f64() + .expect("number is really (too) large"), + ); + + // .round() as i64 builder = builder.timestamp(key.response_timestamp); @@ -370,6 +615,18 @@ impl RpcQueryStats { let response_millis = metadata.start_instant.elapsed().as_millis() as u64; let response_bytes = response_bytes as u64; + // TODO: Gotta make the arithmetic here + + // TODO: Depending on the method, metadata and response bytes, pick a different number of credits used + // This can be a slightly more complex function as we ll + // TODO: Here, let's implement the formula + let credits_used = Self::compute_cost( + request_bytes, + response_bytes, + backend_requests == 0, + &method, + ); + let response_timestamp = Utc::now().timestamp(); Self { @@ -382,6 +639,36 @@ impl RpcQueryStats { response_bytes, response_millis, response_timestamp, + credits_used, + } + } + + /// Compute cost per request + /// All methods cost the same + /// The number of bytes are based on input, and output bytes + pub fn compute_cost( + request_bytes: u64, + response_bytes: u64, + cache_hit: bool, + _method: &Option, + ) -> Decimal { + // TODO: Should make these lazy_static const? + // pays at least $0.000018 / credits per request + let cost_minimum = Decimal::new(18, 6); + // 1kb is included on each call + let cost_free_bytes = 1024; + // after that, we add cost per bytes, $0.000000006 / credits per byte + let cost_per_byte = Decimal::new(6, 9); + + let total_bytes = request_bytes + response_bytes; + let total_chargable_bytes = + Decimal::from(max(0, total_bytes as i64 - cost_free_bytes as i64)); + + let out = cost_minimum + cost_per_byte * total_chargable_bytes; + if cache_hit { + out * Decimal::new(5, 1) + } else { + out } } @@ -405,6 +692,9 @@ impl StatBuffer { bucket: String, db_conn: Option, influxdb_client: Option, + rpc_secret_key_cache: Option< + Cache, + >, db_save_interval_seconds: u32, tsdb_save_interval_seconds: u32, billing_period_seconds: i64, @@ -423,6 +713,7 @@ impl StatBuffer { influxdb_client, db_save_interval_seconds, tsdb_save_interval_seconds, + rpc_secret_key_cache, billing_period_seconds, global_timeseries_buffer: Default::default(), opt_in_timeseries_buffer: Default::default(), @@ -452,7 +743,6 @@ impl StatBuffer { // TODO: Somewhere here we should probably be updating the balance of the user // And also update the credits used etc. for the referred user - loop { tokio::select! { stat = stat_receiver.recv_async() => { From 8a097dabbe2f97a39df485731b99ad208dbb8f40 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Fri, 12 May 2023 15:15:32 -0700 Subject: [PATCH 08/66] Bryan devel 2023-05-12 (#67) * add minor todo * BadRequest instead of web3_context * more bad request error codes * use tokio-uring for the tcp listener * clear block instead of panic * clone earlier * more watch channels instead of rwlocks * drop uring for now (its single threaded) and combine get/post/put routes * clean up iter vs into_iter and unnecessary collect * arcswap instead of rwlock for Web3Rpcs.by_name * cargo upgrade * uuid fast-rng and alphabetize * if protected rpcs, only use protected rpcs * listenfd * make connectinfo optional * try_get_with_by_ref instead of try_get_with * anyhow ensure. and try_get_with_as_ref isn't actually needed * fix feature flags * more refs and less clone * automatic retry for eth_getTransactionReceipt and eth_getTransactionByHash thanks for the report Lefteris @ Rotki * ArcSwap for provider * set archive_request to true on transaction retrying * merge durable stats * Revert "ArcSwap for provider" This reverts commit 166d77f204cde9fa7722c0cefecbb27008749d47. * comments * less clones * more refs * fix test * add optional mimalloc feature * remove stale dependency * sort * cargo upgrade * lint constants * add todo * another todo * lint * anyhow::ensure instead of panic * allow rpc_accounting_v2 entries for requests without an rpc key --- Cargo.lock | 52 +- Dockerfile | 6 +- README.md | 4 +- deferred-rate-limiter/src/lib.rs | 4 +- entities/Cargo.toml | 2 +- entities/src/rpc_accounting_v2.rs | 2 +- latency/src/util/atomic_f32_pair.rs | 7 +- migration/Cargo.toml | 2 +- migration/src/lib.rs | 2 + migration/src/m20221031_211916_clean_up.rs | 1 - migration/src/m20230117_191358_admin_table.rs | 2 - .../src/m20230205_130035_create_balance.rs | 2 - ...14_134254_increase_balance_transactions.rs | 1 - migration/src/m20230215_152254_admin_trail.rs | 1 - migration/src/m20230221_230953_track_spend.rs | 1 - ...20230422_172555_premium_downgrade_logic.rs | 5 - ...emove_columns_statsv2_origin_and_method.rs | 1 - ...20213_allow_null_rpc_key_id_in_stats_v2.rs | 48 ++ web3_proxy/Cargo.toml | 14 +- web3_proxy/src/app/mod.rs | 368 +++++------ web3_proxy/src/app/ws.rs | 168 ++--- web3_proxy/src/bin/wait_for_sync.rs | 26 +- web3_proxy/src/bin/web3_proxy_cli/main.rs | 17 +- .../bin/web3_proxy_cli/migrate_stats_to_v2.rs | 131 ++-- web3_proxy/src/bin/web3_proxy_cli/proxyd.rs | 21 +- .../src/bin/web3_proxy_cli/rpc_accounting.rs | 6 +- .../src/bin/web3_proxy_cli/sentryd/compare.rs | 13 +- .../src/bin/web3_proxy_cli/sentryd/mod.rs | 2 +- .../src/bin/web3_proxy_cli/user_import.rs | 16 +- web3_proxy/src/frontend/admin.rs | 4 +- web3_proxy/src/frontend/authorization.rs | 466 +++++++++++++- web3_proxy/src/frontend/errors.rs | 21 + web3_proxy/src/frontend/mod.rs | 136 ++--- web3_proxy/src/frontend/rpc_proxy_http.rs | 14 +- web3_proxy/src/frontend/rpc_proxy_ws.rs | 189 +++--- web3_proxy/src/frontend/status.rs | 8 +- .../src/frontend/users/authentication.rs | 4 +- web3_proxy/src/frontend/users/payment.rs | 5 +- web3_proxy/src/frontend/users/referral.rs | 3 +- web3_proxy/src/frontend/users/subuser.rs | 14 +- web3_proxy/src/jsonrpc.rs | 7 +- web3_proxy/src/rpcs/blockchain.rs | 11 +- web3_proxy/src/rpcs/consensus.rs | 168 ++++- web3_proxy/src/rpcs/many.rs | 575 +++++++++--------- web3_proxy/src/rpcs/one.rs | 101 +-- web3_proxy/src/stats/db_queries.rs | 1 + web3_proxy/src/stats/influxdb_queries.rs | 2 +- web3_proxy/src/stats/mod.rs | 461 ++++---------- web3_proxy/src/stats/stat_buffer.rs | 269 ++++++++ 49 files changed, 2001 insertions(+), 1383 deletions(-) create mode 100644 migration/src/m20230512_220213_allow_null_rpc_key_id_in_stats_v2.rs create mode 100644 web3_proxy/src/stats/stat_buffer.rs diff --git a/Cargo.lock b/Cargo.lock index 3fe8f6f0..5832affc 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -114,6 +114,12 @@ dependencies = [ "backtrace", ] +[[package]] +name = "arc-swap" +version = "1.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bddcadddf5e9015d310179a59bb28c4d4b9920ad0f11e8e14dbadf654890c9a6" + [[package]] name = "argh" version = "0.1.10" @@ -1576,7 +1582,7 @@ dependencies = [ [[package]] name = "entities" -version = "0.17.0" +version = "0.27.0" dependencies = [ "ethers", "sea-orm", @@ -3041,6 +3047,16 @@ version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "348108ab3fba42ec82ff6e9564fc4ca0247bdccdc68dd8af9764bbc79c3c8ffb" +[[package]] +name = "libmimalloc-sys" +version = "0.1.33" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f4ac0e912c8ef1b735e92369695618dc5b1819f5a7bf3f167301a3ba1cea515e" +dependencies = [ + "cc", + "libc", +] + [[package]] name = "libz-sys" version = "1.1.9" @@ -3068,6 +3084,17 @@ version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ece97ea872ece730aed82664c424eb4c8291e1ff2480247ccf7409044bc6479f" +[[package]] +name = "listenfd" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e0500463acd96259d219abb05dc57e5a076ef04b2db9a2112846929b5f174c96" +dependencies = [ + "libc", + "uuid 1.3.2", + "winapi", +] + [[package]] name = "lock_api" version = "0.4.9" @@ -3143,12 +3170,21 @@ dependencies = [ [[package]] name = "migration" -version = "0.19.0" +version = "0.27.0" dependencies = [ "sea-orm-migration", "tokio", ] +[[package]] +name = "mimalloc" +version = "0.1.37" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4e2894987a3459f3ffb755608bd82188f8ed00d0ae077f1edea29c068d639d98" +dependencies = [ + "libmimalloc-sys", +] + [[package]] name = "mime" version = "0.3.17" @@ -4607,12 +4643,6 @@ version = "0.1.23" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" -[[package]] -name = "rustc-hash" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" - [[package]] name = "rustc-hex" version = "2.1.0" @@ -6609,9 +6639,10 @@ dependencies = [ [[package]] name = "web3_proxy" -version = "0.17.0" +version = "0.27.0" dependencies = [ "anyhow", + "arc-swap", "argh", "axum", "axum-client-ip", @@ -6643,8 +6674,10 @@ dependencies = [ "ipnet", "itertools", "latency", + "listenfd", "log", "migration", + "mimalloc", "moka", "num", "num-traits", @@ -6659,7 +6692,6 @@ dependencies = [ "regex", "reqwest", "rmp-serde", - "rustc-hash", "sentry", "serde", "serde_json", diff --git a/Dockerfile b/Dockerfile index d9717123..5e6c3019 100644 --- a/Dockerfile +++ b/Dockerfile @@ -32,17 +32,19 @@ RUN apt-get update && \ # copy the application COPY . . +ENV WEB3_PROXY_FEATURES "rdkafka-src" + # test the application with cargo-nextest RUN --mount=type=cache,target=/usr/local/cargo/registry \ --mount=type=cache,target=/app/target \ - cargo nextest run --features "rdkafka-src tokio-uring" --no-default-features + cargo nextest run --features "$WEB3_PROXY_FEATURES" --no-default-features # build the application # using a "release" profile (which install does) is **very** important RUN --mount=type=cache,target=/usr/local/cargo/registry \ --mount=type=cache,target=/app/target \ cargo install \ - --features "rdkafka-src tokio-uring" \ + --features "$WEB3_PROXY_FEATURES" \ --locked \ --no-default-features \ --path ./web3_proxy \ diff --git a/README.md b/README.md index 8f70d866..af5dabf1 100644 --- a/README.md +++ b/README.md @@ -79,7 +79,9 @@ web3_proxy_cli health_compass https://eth.llamarpc.com https://eth-ski.llamarpc. ### Run migrations -This is only really useful during development. The migrations run on application start. +Generally it is simplest to just run the app to run migrations. It runs migrations on start. + +But if you want to run them manually (generally only useful in development): ``` cd migration diff --git a/deferred-rate-limiter/src/lib.rs b/deferred-rate-limiter/src/lib.rs index f14fd4e5..8f055ce3 100644 --- a/deferred-rate-limiter/src/lib.rs +++ b/deferred-rate-limiter/src/lib.rs @@ -89,7 +89,7 @@ where // set arc_deferred_rate_limit_result and return the coun self.local_cache - .get_with(key, async move { + .get_with_by_ref(&key, async move { // we do not use the try operator here because we want to be okay with redis errors let redis_count = match rrl .throttle_label(&redis_key, Some(max_requests_per_period), count) @@ -110,7 +110,7 @@ where count } Ok(RedisRateLimitResult::RetryNever) => { - panic!("RetryNever shouldn't happen") + unreachable!(); } Err(err) => { let _ = deferred_rate_limit_result diff --git a/entities/Cargo.toml b/entities/Cargo.toml index 2e94c960..c88d7668 100644 --- a/entities/Cargo.toml +++ b/entities/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "entities" -version = "0.17.0" +version = "0.27.0" edition = "2021" [lib] diff --git a/entities/src/rpc_accounting_v2.rs b/entities/src/rpc_accounting_v2.rs index 49121125..d3cc8cb6 100644 --- a/entities/src/rpc_accounting_v2.rs +++ b/entities/src/rpc_accounting_v2.rs @@ -8,7 +8,7 @@ use serde::{Deserialize, Serialize}; pub struct Model { #[sea_orm(primary_key)] pub id: u64, - pub rpc_key_id: u64, + pub rpc_key_id: Option, pub chain_id: u64, pub period_datetime: DateTimeUtc, pub archive_needed: bool, diff --git a/latency/src/util/atomic_f32_pair.rs b/latency/src/util/atomic_f32_pair.rs index 08ee3953..fa74fa0b 100644 --- a/latency/src/util/atomic_f32_pair.rs +++ b/latency/src/util/atomic_f32_pair.rs @@ -54,26 +54,27 @@ fn from_bits(bits: u64) -> [f32; 2] { #[cfg(test)] mod tests { + use std::f32; use std::sync::atomic::Ordering; use super::{from_bits, to_bits, AtomicF32Pair}; #[test] fn test_f32_pair_bit_conversions() { - let pair = [3.14159, 2.71828]; + let pair = [f32::consts::PI, f32::consts::E]; assert_eq!(pair, from_bits(to_bits(pair))); } #[test] fn test_atomic_f32_pair_load() { - let pair = [3.14159, 2.71828]; + let pair = [f32::consts::PI, f32::consts::E]; let atomic = AtomicF32Pair::new(pair); assert_eq!(pair, atomic.load(Ordering::Relaxed)); } #[test] fn test_atomic_f32_pair_fetch_update() { - let pair = [3.14159, 2.71828]; + let pair = [f32::consts::PI, f32::consts::E]; let atomic = AtomicF32Pair::new(pair); atomic .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |[f1, f2]| { diff --git a/migration/Cargo.toml b/migration/Cargo.toml index d8c07ad9..97f07b5e 100644 --- a/migration/Cargo.toml +++ b/migration/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "migration" -version = "0.19.0" +version = "0.27.0" edition = "2021" publish = false diff --git a/migration/src/lib.rs b/migration/src/lib.rs index ddd8160d..182f5a1e 100644 --- a/migration/src/lib.rs +++ b/migration/src/lib.rs @@ -26,6 +26,7 @@ mod m20230221_230953_track_spend; mod m20230412_171916_modify_secondary_user_add_primary_user; mod m20230422_172555_premium_downgrade_logic; mod m20230511_161214_remove_columns_statsv2_origin_and_method; +mod m20230512_220213_allow_null_rpc_key_id_in_stats_v2; pub struct Migrator; @@ -59,6 +60,7 @@ impl MigratorTrait for Migrator { Box::new(m20230412_171916_modify_secondary_user_add_primary_user::Migration), Box::new(m20230422_172555_premium_downgrade_logic::Migration), Box::new(m20230511_161214_remove_columns_statsv2_origin_and_method::Migration), + Box::new(m20230512_220213_allow_null_rpc_key_id_in_stats_v2::Migration), ] } } diff --git a/migration/src/m20221031_211916_clean_up.rs b/migration/src/m20221031_211916_clean_up.rs index 82a1df7a..7cceebf9 100644 --- a/migration/src/m20221031_211916_clean_up.rs +++ b/migration/src/m20221031_211916_clean_up.rs @@ -92,7 +92,6 @@ impl MigrationTrait for Migration { ) .await?; - // rename column rpc_key to rpc_secret_key Ok(()) } diff --git a/migration/src/m20230117_191358_admin_table.rs b/migration/src/m20230117_191358_admin_table.rs index 1d11ebf3..5315a276 100644 --- a/migration/src/m20230117_191358_admin_table.rs +++ b/migration/src/m20230117_191358_admin_table.rs @@ -6,7 +6,6 @@ pub struct Migration; #[async_trait::async_trait] impl MigrationTrait for Migration { async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { - // Replace the sample below with your own migration scripts manager .create_table( Table::create() @@ -36,7 +35,6 @@ impl MigrationTrait for Migration { } async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { - // Replace the sample below with your own migration scripts manager .drop_table(Table::drop().table(Admin::Table).to_owned()) .await diff --git a/migration/src/m20230205_130035_create_balance.rs b/migration/src/m20230205_130035_create_balance.rs index 11076fce..0036b151 100644 --- a/migration/src/m20230205_130035_create_balance.rs +++ b/migration/src/m20230205_130035_create_balance.rs @@ -6,7 +6,6 @@ pub struct Migration; #[async_trait::async_trait] impl MigrationTrait for Migration { async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { - // Replace the sample below with your own migration scripts manager .create_table( Table::create() @@ -48,7 +47,6 @@ impl MigrationTrait for Migration { } async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { - // Replace the sample below with your own migration scripts manager .drop_table(Table::drop().table(Balance::Table).to_owned()) .await diff --git a/migration/src/m20230214_134254_increase_balance_transactions.rs b/migration/src/m20230214_134254_increase_balance_transactions.rs index 72ea4d60..2de3db9a 100644 --- a/migration/src/m20230214_134254_increase_balance_transactions.rs +++ b/migration/src/m20230214_134254_increase_balance_transactions.rs @@ -68,7 +68,6 @@ impl MigrationTrait for Migration { } async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { - // Replace the sample below with your own migration scripts manager .drop_table( Table::drop() diff --git a/migration/src/m20230215_152254_admin_trail.rs b/migration/src/m20230215_152254_admin_trail.rs index c4c4d21b..0a758796 100644 --- a/migration/src/m20230215_152254_admin_trail.rs +++ b/migration/src/m20230215_152254_admin_trail.rs @@ -49,7 +49,6 @@ impl MigrationTrait for Migration { } async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { - // Replace the sample below with your own migration scripts manager .drop_table(Table::drop().table(AdminTrail::Table).to_owned()) .await diff --git a/migration/src/m20230221_230953_track_spend.rs b/migration/src/m20230221_230953_track_spend.rs index d6a62d32..30fec499 100644 --- a/migration/src/m20230221_230953_track_spend.rs +++ b/migration/src/m20230221_230953_track_spend.rs @@ -22,7 +22,6 @@ impl MigrationTrait for Migration { } async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { - // Replace the sample below with your own migration scripts manager .alter_table( sea_query::Table::alter() diff --git a/migration/src/m20230422_172555_premium_downgrade_logic.rs b/migration/src/m20230422_172555_premium_downgrade_logic.rs index e474a785..a4ef2496 100644 --- a/migration/src/m20230422_172555_premium_downgrade_logic.rs +++ b/migration/src/m20230422_172555_premium_downgrade_logic.rs @@ -7,8 +7,6 @@ pub struct Migration; #[async_trait::async_trait] impl MigrationTrait for Migration { async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { - // Replace the sample below with your own migration scripts - // Add a column "downgrade_tier_id" // It is a "foreign key" that references other items in this table manager @@ -18,7 +16,6 @@ impl MigrationTrait for Migration { .add_column(ColumnDef::new(UserTier::DowngradeTierId).big_unsigned()) .add_foreign_key( TableForeignKey::new() - .to_tbl(UserTier::Table) .to_tbl(UserTier::Table) .from_col(UserTier::DowngradeTierId) .to_col(UserTier::Id), @@ -83,8 +80,6 @@ impl MigrationTrait for Migration { } async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { - // Replace the sample below with your own migration scripts - // Remove the two tiers that you just added // And remove the column you just added let db_conn = manager.get_connection(); diff --git a/migration/src/m20230511_161214_remove_columns_statsv2_origin_and_method.rs b/migration/src/m20230511_161214_remove_columns_statsv2_origin_and_method.rs index 0dc736c1..a5463c69 100644 --- a/migration/src/m20230511_161214_remove_columns_statsv2_origin_and_method.rs +++ b/migration/src/m20230511_161214_remove_columns_statsv2_origin_and_method.rs @@ -44,7 +44,6 @@ impl MigrationTrait for Migration { #[derive(Iden)] enum RpcAccountingV2 { Table, - Id, Origin, Method, } diff --git a/migration/src/m20230512_220213_allow_null_rpc_key_id_in_stats_v2.rs b/migration/src/m20230512_220213_allow_null_rpc_key_id_in_stats_v2.rs new file mode 100644 index 00000000..b9d35ae9 --- /dev/null +++ b/migration/src/m20230512_220213_allow_null_rpc_key_id_in_stats_v2.rs @@ -0,0 +1,48 @@ +use sea_orm_migration::prelude::*; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + sea_query::Table::alter() + .table(RpcAccountingV2::Table) + .to_owned() + // allow rpc_key_id to be null. Needed for public rpc stat tracking + .modify_column( + ColumnDef::new(RpcAccountingV2::RpcKeyId) + .big_unsigned() + .null(), + ) + .to_owned(), + ) + .await + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + sea_query::Table::alter() + .table(RpcAccountingV2::Table) + .to_owned() + .modify_column( + ColumnDef::new(RpcAccountingV2::RpcKeyId) + .big_unsigned() + .not_null() + .default(0), + ) + .to_owned(), + ) + .await + } +} + +/// Learn more at https://docs.rs/sea-query#iden +#[derive(Iden)] +enum RpcAccountingV2 { + Table, + RpcKeyId, +} diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index e8d65868..4883f9a2 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -1,16 +1,18 @@ [package] name = "web3_proxy" -version = "0.17.0" +version = "0.27.0" edition = "2021" default-run = "web3_proxy_cli" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [features] -default = ["deadlock_detection"] +default = ["connectinfo", "deadlock_detection"] deadlock_detection = ["parking_lot/deadlock_detection"] +mimalloc = ["dep:mimalloc"] tokio-console = ["dep:tokio-console", "dep:console-subscriber"] rdkafka-src = ["rdkafka/cmake-build", "rdkafka/libz", "rdkafka/ssl", "rdkafka/zstd-pkg-config"] +connectinfo = [] [dependencies] deferred-rate-limiter = { path = "../deferred-rate-limiter" } @@ -27,6 +29,7 @@ thread-fast-rng = { path = "../thread-fast-rng" } # TODO: make sure this time version matches siwe. PR to put this in their prelude anyhow = { version = "1.0.71", features = ["backtrace"] } +arc-swap = "1.6.0" argh = "0.1.10" axum = { version = "0.6.18", features = ["headers", "ws"] } axum-client-ip = "0.4.1" @@ -47,15 +50,17 @@ gethostname = "0.4.2" glob = "0.3.1" handlebars = "4.3.7" hashbrown = { version = "0.13.2", features = ["serde"] } -hex_fmt = "0.3.0" hdrhistogram = "7.5.2" +hex_fmt = "0.3.0" +hostname = "0.3.1" http = "0.2.9" influxdb2 = { git = "https://github.com/llamanodes/influxdb2", features = ["rustls"] } influxdb2-structmap = { git = "https://github.com/llamanodes/influxdb2/"} -hostname = "0.3.1" ipnet = "2.7.2" itertools = "0.10.5" +listenfd = "1.0.1" log = "0.4.17" +mimalloc = { version = "0.1.37", optional = true} moka = { version = "0.11.0", default-features = false, features = ["future"] } num = "0.4.0" num-traits = "0.2.15" @@ -69,7 +74,6 @@ rdkafka = { version = "0.29.0" } regex = "1.8.1" reqwest = { version = "0.11.17", default-features = false, features = ["json", "tokio-rustls"] } rmp-serde = "1.1.1" -rustc-hash = "1.1.0" sentry = { version = "0.31.0", default-features = false, features = ["backtrace", "contexts", "panic", "anyhow", "reqwest", "rustls", "log", "sentry-log"] } serde = { version = "1.0.163", features = [] } serde_json = { version = "1.0.96", default-features = false, features = ["alloc", "raw_value"] } diff --git a/web3_proxy/src/app/mod.rs b/web3_proxy/src/app/mod.rs index 46072197..63c3f9f5 100644 --- a/web3_proxy/src/app/mod.rs +++ b/web3_proxy/src/app/mod.rs @@ -3,7 +3,9 @@ mod ws; use crate::block_number::{block_needed, BlockNeeded}; use crate::config::{AppConfig, TopConfig}; -use crate::frontend::authorization::{Authorization, RequestMetadata, RpcSecretKey}; +use crate::frontend::authorization::{ + Authorization, RequestMetadata, RequestOrMethod, RpcSecretKey, +}; use crate::frontend::errors::{Web3ProxyError, Web3ProxyErrorContext, Web3ProxyResult}; use crate::frontend::rpc_proxy_ws::ProxyMode; use crate::jsonrpc::{ @@ -14,7 +16,7 @@ use crate::rpcs::consensus::ConsensusWeb3Rpcs; use crate::rpcs::many::Web3Rpcs; use crate::rpcs::one::Web3Rpc; use crate::rpcs::transactions::TxStatus; -use crate::stats::{AppStat, RpcQueryStats, StatBuffer}; +use crate::stats::{AppStat, StatBuffer}; use crate::user_token::UserBearerToken; use anyhow::Context; use axum::headers::{Origin, Referer, UserAgent}; @@ -40,8 +42,6 @@ use migration::sea_orm::{ use migration::sea_query::table::ColumnDef; use migration::{Alias, DbErr, Migrator, MigratorTrait, Table}; use moka::future::Cache; -use rdkafka::message::{Header, OwnedHeaders}; -use rdkafka::producer::FutureRecord; use redis_rate_limiter::redis::AsyncCommands; use redis_rate_limiter::{redis, DeadpoolRuntime, RedisConfig, RedisPool, RedisRateLimiter}; use serde::Serialize; @@ -78,7 +78,6 @@ struct ResponseCacheKey { // to_block is only set when ranges of blocks are requested (like with eth_getLogs) to_block: Option, method: String, - // TODO: better type for this params: Option, cache_errors: bool, } @@ -87,7 +86,7 @@ impl ResponseCacheKey { fn weight(&self) -> usize { let mut w = self.method.len(); - if let Some(p) = self.params.as_ref() { + if let Some(ref p) = self.params { w += p.to_string().len(); } @@ -206,6 +205,10 @@ impl DatabaseReplica { } } +// TODO: this should be a the secret key id, not the key itself! +pub type RpcSecretKeyCache = + Cache; + /// The application // TODO: i'm sure this is more arcs than necessary, but spawning futures makes references hard pub struct Web3ProxyApp { @@ -251,8 +254,7 @@ pub struct Web3ProxyApp { pub vredis_pool: Option, /// 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: - Cache, + pub rpc_secret_key_cache: RpcSecretKeyCache, /// concurrent/parallel RPC request limits for authenticated users pub registered_user_semaphores: Cache, hashbrown::hash_map::DefaultHashBuilder>, @@ -276,8 +278,7 @@ pub async fn flatten_handle(handle: AnyhowJoinHandle) -> anyhow::Result } } -/// return the first error or okay if everything worked - +/// return the first error, or Ok if everything worked pub async fn flatten_handles( mut handles: FuturesUnordered>, ) -> anyhow::Result<()> { @@ -493,11 +494,10 @@ impl Web3ProxyApp { db_conn.clone().map(DatabaseReplica) }; } else { - if top_config.app.db_replica_url.is_some() { - return Err(anyhow::anyhow!( - "if there is a db_replica_url, there must be a db_url" - )); - } + anyhow::ensure!( + top_config.app.db_replica_url.is_none(), + "if there is a db_replica_url, there must be a db_url" + ); warn!("no database. some features will be disabled"); }; @@ -516,7 +516,10 @@ impl Web3ProxyApp { .set("security.protocol", security_protocol) .create() { - Ok(k) => kafka_producer = Some(k), + Ok(k) => { + // TODO: create our topic + kafka_producer = Some(k) + } Err(err) => error!("Failed connecting to kafka. This will not retry. {:?}", err), } } @@ -596,15 +599,15 @@ impl Web3ProxyApp { let mut stat_sender = None; if let Some(influxdb_bucket) = top_config.app.influxdb_bucket.clone() { if let Some(spawned_stat_buffer) = StatBuffer::try_spawn( - top_config.app.chain_id, + BILLING_PERIOD_SECONDS, influxdb_bucket, + top_config.app.chain_id, db_conn.clone(), + 60, influxdb_client.clone(), Some(rpc_secret_key_cache.clone()), - 60, - 1, - BILLING_PERIOD_SECONDS, stat_buffer_shutdown_receiver, + 1, )? { // 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); @@ -634,7 +637,7 @@ impl Web3ProxyApp { let mut frontend_registered_user_rate_limiter = None; let mut login_rate_limiter = None; - if let Some(redis_pool) = vredis_pool.as_ref() { + 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( @@ -669,7 +672,6 @@ impl Web3ProxyApp { )); } - // TODO: i don't like doing Block::default here! Change this to "None"? let (watch_consensus_head_sender, watch_consensus_head_receiver) = watch::channel(None); // TODO: will one receiver lagging be okay? how big should this be? let (pending_tx_sender, pending_tx_receiver) = broadcast::channel(256); @@ -894,7 +896,7 @@ impl Web3ProxyApp { .context("updating balanced rpcs")?; if let Some(private_rpc_configs) = new_top_config.private_rpcs { - if let Some(private_rpcs) = self.private_rpcs.as_ref() { + if let Some(ref private_rpcs) = self.private_rpcs { private_rpcs .apply_server_configs(self, private_rpc_configs) .await @@ -906,7 +908,7 @@ impl Web3ProxyApp { } if let Some(bundler_4337_rpc_configs) = new_top_config.bundler_4337_rpcs { - if let Some(bundler_4337_rpcs) = self.bundler_4337_rpcs.as_ref() { + if let Some(ref bundler_4337_rpcs) = self.bundler_4337_rpcs { bundler_4337_rpcs .apply_server_configs(self, bundler_4337_rpc_configs) .await @@ -1106,23 +1108,29 @@ impl Web3ProxyApp { self: &Arc, authorization: Arc, request: JsonRpcRequestEnum, - ) -> Web3ProxyResult<(JsonRpcForwardedResponseEnum, Vec>)> { + ) -> Web3ProxyResult<(StatusCode, JsonRpcForwardedResponseEnum, Vec>)> { // trace!(?request, "proxy_web3_rpc"); // even though we have timeouts on the requests to our backend providers, // we need a timeout for the incoming request so that retries don't run forever - // TODO: take this as an optional argument. per user max? expiration time instead of duration? - let max_time = Duration::from_secs(120); + // TODO: take this as an optional argument. check for a different max from the user_tier? + // TODO: how much time was spent on this request alredy? + let max_time = Duration::from_secs(240); + // TODO: use streams and buffers so we don't overwhelm our server let response = match request { - JsonRpcRequestEnum::Single(request) => { - let (response, rpcs) = timeout( + JsonRpcRequestEnum::Single(mut request) => { + let (status_code, response, rpcs) = timeout( max_time, - self.proxy_cached_request(&authorization, request, None), + self.proxy_cached_request(&authorization, &mut request, None), ) - .await??; + .await?; - (JsonRpcForwardedResponseEnum::Single(response), rpcs) + ( + status_code, + JsonRpcForwardedResponseEnum::Single(response), + rpcs, + ) } JsonRpcRequestEnum::Batch(requests) => { let (responses, rpcs) = timeout( @@ -1131,7 +1139,12 @@ impl Web3ProxyApp { ) .await??; - (JsonRpcForwardedResponseEnum::Batch(responses), rpcs) + // TODO: real status code + ( + StatusCode::OK, + JsonRpcForwardedResponseEnum::Batch(responses), + rpcs, + ) } }; @@ -1143,14 +1156,11 @@ impl Web3ProxyApp { async fn proxy_web3_rpc_requests( self: &Arc, authorization: &Arc, - requests: Vec, + mut requests: Vec, ) -> Web3ProxyResult<(Vec, Vec>)> { // TODO: we should probably change ethers-rs to support this directly. they pushed this off to v2 though let num_requests = requests.len(); - // TODO: spawn so the requests go in parallel? need to think about rate limiting more if we do that - // TODO: improve flattening - // 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_num = self @@ -1160,7 +1170,7 @@ impl Web3ProxyApp { let responses = join_all( requests - .into_iter() + .iter_mut() .map(|request| { self.proxy_cached_request(authorization, request, Some(head_block_num)) }) @@ -1168,14 +1178,12 @@ impl Web3ProxyApp { ) .await; - // TODO: i'm sure this could be done better with iterators - // TODO: stream the response? let mut collected: Vec = Vec::with_capacity(num_requests); let mut collected_rpc_names: HashSet = HashSet::new(); let mut collected_rpcs: Vec> = vec![]; for response in responses { // TODO: any way to attach the tried rpcs to the error? it is likely helpful - let (response, rpcs) = response?; + let (status_code, response, rpcs) = response; collected.push(response); collected_rpcs.extend(rpcs.into_iter().filter(|x| { @@ -1186,6 +1194,8 @@ impl Web3ProxyApp { true } })); + + // TODO: what should we do with the status code? check the jsonrpc spec } Ok((collected, collected_rpcs)) @@ -1212,8 +1222,8 @@ impl Web3ProxyApp { } } - /// try to send transactions to the best available rpcs with private mempools - /// if no private rpcs are configured, then some public rpcs are used instead + /// 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( self: &Arc, authorization: &Arc, @@ -1261,82 +1271,50 @@ impl Web3ProxyApp { async fn proxy_cached_request( self: &Arc, authorization: &Arc, - mut request: JsonRpcRequest, + request: &mut JsonRpcRequest, head_block_num: Option, - ) -> Web3ProxyResult<(JsonRpcForwardedResponse, Vec>)> { + ) -> (StatusCode, JsonRpcForwardedResponse, Vec>) { // TODO: move this code to another module so that its easy to turn this trace logging on in dev trace!("Received request: {:?}", request); - let request_metadata = Arc::new(RequestMetadata::new(request.num_bytes())); + let request_metadata = RequestMetadata::new( + self, + authorization.clone(), + RequestOrMethod::Request(request), + head_block_num.as_ref(), + ) + .await; - let mut kafka_stuff = None; + let (status_code, response) = match self + ._proxy_cached_request(authorization, request, head_block_num, &request_metadata) + .await + { + Ok(x) => (StatusCode::OK, x), + Err(err) => err.into_response_parts(), + }; - if matches!(authorization.checks.proxy_mode, ProxyMode::Debug) { - if let Some(kafka_producer) = self.kafka_producer.clone() { - let kafka_topic = "proxy_cached_request".to_string(); + request_metadata.add_response(&response); - let rpc_secret_key_id = authorization - .checks - .rpc_secret_key_id - .map(|x| x.get()) - .unwrap_or_default(); + // TODO: with parallel request sending, I think there could be a race on this + let rpcs = request_metadata.backend_rpcs_used(); - let kafka_key = rmp_serde::to_vec(&rpc_secret_key_id)?; - - let request_bytes = rmp_serde::to_vec(&request)?; - - let request_hash = Some(keccak256(&request_bytes)); - - let chain_id = self.config.chain_id; - - // another item is added with the response, so initial_capacity is +1 what is needed here - let kafka_headers = OwnedHeaders::new_with_capacity(4) - .insert(Header { - key: "request_hash", - value: request_hash.as_ref(), - }) - .insert(Header { - key: "head_block_num", - value: head_block_num.map(|x| x.to_string()).as_ref(), - }) - .insert(Header { - key: "chain_id", - value: Some(&chain_id.to_le_bytes()), - }); - - // save the key and headers for when we log the response - kafka_stuff = Some(( - kafka_topic.clone(), - kafka_key.clone(), - kafka_headers.clone(), - )); - - let f = async move { - let produce_future = kafka_producer.send( - FutureRecord::to(&kafka_topic) - .key(&kafka_key) - .payload(&request_bytes) - .headers(kafka_headers), - Duration::from_secs(0), - ); - - if let Err((err, _)) = produce_future.await { - error!("produce kafka request log: {}", err); - // TODO: re-queue the msg? - } - }; - - tokio::spawn(f); - } - } + (status_code, response, rpcs) + } + /// main logic for proxy_cached_request but in a dedicated function so the try operator is easy to use + async fn _proxy_cached_request( + self: &Arc, + authorization: &Arc, + request: &mut JsonRpcRequest, + head_block_num: Option, + request_metadata: &Arc, + ) -> Web3ProxyResult { // save the id so we can attach it to the response - // TODO: instead of cloning, take the id out? let request_id = request.id.clone(); + // TODO: don't clone let request_method = request.method.clone(); - // TODO: if eth_chainId or net_version, serve those without querying the backend - // TODO: don't clone? + // TODO: serve net_version without querying the backend let response: JsonRpcForwardedResponse = match request_method.as_ref() { // lots of commands are blocked method @ ("db_getHex" @@ -1449,15 +1427,15 @@ impl Web3ProxyApp { .try_proxy_connection( authorization, request, - Some(&request_metadata), + Some(request_metadata), None, None, ) .await? } None => { - // TODO: stats! - // TODO: not synced error? + // TODO: stats even when we error! + // TODO: use Web3ProxyError? dedicated error for no 4337 bundlers return Err(anyhow::anyhow!("no bundler_4337_rpcs available").into()); } }, @@ -1493,26 +1471,19 @@ impl Web3ProxyApp { .try_proxy_connection( authorization, request, - Some(&request_metadata), + Some(request_metadata), None, None, ) .await?; - let mut gas_estimate: U256 = if let Some(gas_estimate) = response.result.take() { - serde_json::from_str(gas_estimate.get()) - .or(Err(Web3ProxyError::GasEstimateNotU256))? - } else { - // i think this is always an error response - let rpcs = request_metadata.backend_requests.lock().clone(); + if let Some(gas_estimate) = response.result.take() { + let mut gas_estimate: U256 = serde_json::from_str(gas_estimate.get()) + .or(Err(Web3ProxyError::GasEstimateNotU256))?; - // TODO! save stats - - return Ok((response, rpcs)); - }; - - let gas_increase = - if let Some(gas_increase_percent) = self.config.gas_increase_percent { + 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); let min_gas_increase = self.config.gas_increase_min.unwrap_or_default(); @@ -1522,9 +1493,12 @@ impl Web3ProxyApp { self.config.gas_increase_min.unwrap_or_default() }; - gas_estimate += gas_increase; + gas_estimate += gas_increase; - JsonRpcForwardedResponse::from_value(json!(gas_estimate), request_id) + JsonRpcForwardedResponse::from_value(json!(gas_estimate), request_id) + } else { + response + } } "eth_getTransactionReceipt" | "eth_getTransactionByHash" => { // try to get the transaction without specifying a min_block_height @@ -1532,8 +1506,8 @@ impl Web3ProxyApp { .balanced_rpcs .try_proxy_connection( authorization, - request.clone(), - Some(&request_metadata), + request, + Some(request_metadata), None, None, ) @@ -1551,7 +1525,7 @@ impl Web3ProxyApp { .try_proxy_connection( authorization, request, - Some(&request_metadata), + Some(request_metadata), Some(&U64::one()), None, ) @@ -1583,7 +1557,7 @@ impl Web3ProxyApp { let mut response = self .try_send_protected( authorization, - &request, + request, request_metadata.clone(), num_public_rpcs, ) @@ -1592,24 +1566,36 @@ impl Web3ProxyApp { // 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 Some(response_error) = response.error.as_ref() { + if let Some(ref response_error) = response.error { if response_error.code == -32000 && (response_error.message == "ALREADY_EXISTS: already known" || response_error.message == "INTERNAL_ERROR: existing tx with same hash") { - // TODO: expect instead of web3_context? let params = request .params + .as_mut() .web3_context("there must be params if we got this far")?; let params = params .as_array() - .web3_context("there must be an array if we got this far")? + .ok_or_else(|| { + Web3ProxyError::BadRequest( + "Unable to get array from params".to_string(), + ) + })? .get(0) - .web3_context("there must be an item if we got this far")? + .ok_or_else(|| { + Web3ProxyError::BadRequest( + "Unable to get item 0 from params".to_string(), + ) + })? .as_str() - .web3_context("there must be a string if we got this far")?; + .ok_or_else(|| { + Web3ProxyError::BadRequest( + "Unable to get string from params item 0".to_string(), + ) + })?; let params = Bytes::from_str(params) .expect("there must be Bytes if we got this far"); @@ -1617,6 +1603,7 @@ impl Web3ProxyApp { 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 let tx_hash = json!(tx.hash()); trace!("tx_hash: {:#?}", tx_hash); @@ -1630,17 +1617,16 @@ impl Web3ProxyApp { } // emit transaction count stats - if let Some(salt) = self.config.public_recent_ips_salt.as_ref() { - if let Some(tx_hash) = response.result.clone() { + if let Some(ref salt) = self.config.public_recent_ips_salt { + if let Some(ref tx_hash) = response.result { let now = Utc::now().timestamp(); - let salt = salt.clone(); let app = self.clone(); + let salted_tx_hash = format!("{}:{}", salt, tx_hash); + let f = async move { match app.redis_conn().await { Ok(Some(mut redis_conn)) => { - let salted_tx_hash = format!("{}:{}", salt, tx_hash); - let hashed_tx_hash = Bytes::from(keccak256(salted_tx_hash.as_bytes())); @@ -1685,7 +1671,7 @@ impl Web3ProxyApp { Some(request_id), ), "net_listening" => { - // TODO: only if there are some backends on balanced_rpcs? + // TODO: only true if there are some backends on balanced_rpcs? JsonRpcForwardedResponse::from_value(serde_json::Value::Bool(true), request_id) } "net_peerCount" => JsonRpcForwardedResponse::from_value( @@ -1705,35 +1691,35 @@ impl Web3ProxyApp { || !params.get(0).map(|x| x.is_string()).unwrap_or(false) { // TODO: what error code? - return Ok(( - JsonRpcForwardedResponse::from_str( - "Invalid request", - Some(-32600), - Some(request_id), - ), - vec![], - )); - } - - 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( - "param 0 could not be read as H256".to_string(), + // TODO: use Web3ProxyError::BadRequest + JsonRpcForwardedResponse::from_str( + "Invalid request", + Some(-32600), + Some(request_id), ) - })?; + } 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( + "param 0 could not be read as H256".to_string(), + ) + })?; - let hash = H256::from(keccak256(param)); + let hash = H256::from(keccak256(param)); - JsonRpcForwardedResponse::from_value(json!(hash), request_id) + JsonRpcForwardedResponse::from_value(json!(hash), request_id) + } } _ => { // TODO: this needs the correct error code in the response + // TODO: Web3ProxyError::BadRequest instead? JsonRpcForwardedResponse::from_str( "invalid request", Some(StatusCode::BAD_REQUEST.as_u16().into()), @@ -1759,6 +1745,9 @@ impl Web3ProxyApp { .or(self.balanced_rpcs.head_block_num()) .ok_or(Web3ProxyError::NoServersSynced)?; + // TODO: don't clone. this happens way too much. maybe &mut? + let mut request = request.clone(); + // we do this check before checking caches because it might modify the request params // TODO: add a stat for archive vs full since they should probably cost different // TODO: this cache key can be rather large. is that okay? @@ -1867,7 +1856,7 @@ impl Web3ProxyApp { .balanced_rpcs .try_proxy_connection( &authorization, - request, + &request, Some(&request_metadata), from_block_num.as_ref(), to_block_num.as_ref(), @@ -1888,7 +1877,7 @@ impl Web3ProxyApp { self.balanced_rpcs .try_proxy_connection( &authorization, - request, + &request, Some(&request_metadata), None, None, @@ -1897,7 +1886,7 @@ impl Web3ProxyApp { } }; - // since this data came likely out of a cache, the id is not going to match + // since this data likely came out of a cache, the response.id is not going to match the request.id // replace the id with our request's id. response.id = request_id; @@ -1905,52 +1894,7 @@ impl Web3ProxyApp { } }; - // save the rpcs so they can be included in a response header - let rpcs = request_metadata.backend_requests.lock().clone(); - - // send stats used for accounting and graphs - if let Some(stat_sender) = self.stat_sender.as_ref() { - let response_stat = RpcQueryStats::new( - Some(request_method), - authorization.clone(), - request_metadata, - response.num_bytes(), - ); - - stat_sender - .send_async(response_stat.into()) - .await - .map_err(Web3ProxyError::SendAppStatError)?; - } - - // send debug info as a kafka log - if let Some((kafka_topic, kafka_key, kafka_headers)) = kafka_stuff { - let kafka_producer = self - .kafka_producer - .clone() - .expect("if headers are set, producer must exist"); - - let response_bytes = - rmp_serde::to_vec(&response).web3_context("failed msgpack serialize response")?; - - let f = async move { - let produce_future = kafka_producer.send( - FutureRecord::to(&kafka_topic) - .key(&kafka_key) - .payload(&response_bytes) - .headers(kafka_headers), - Duration::from_secs(0), - ); - - if let Err((err, _)) = produce_future.await { - error!("produce kafka response log: {}", err); - } - }; - - tokio::spawn(f); - } - - Ok((response, rpcs)) + Ok(response) } } diff --git a/web3_proxy/src/app/ws.rs b/web3_proxy/src/app/ws.rs index 4b57c8a8..db348577 100644 --- a/web3_proxy/src/app/ws.rs +++ b/web3_proxy/src/app/ws.rs @@ -1,56 +1,55 @@ //! Websocket-specific functions for the Web3ProxyApp use super::Web3ProxyApp; -use crate::frontend::authorization::{Authorization, RequestMetadata}; -use crate::frontend::errors::{Web3ProxyError, Web3ProxyErrorContext, Web3ProxyResult}; +use crate::frontend::authorization::{Authorization, RequestMetadata, RequestOrMethod}; +use crate::frontend::errors::{Web3ProxyError, Web3ProxyResult}; use crate::jsonrpc::JsonRpcForwardedResponse; use crate::jsonrpc::JsonRpcRequest; use crate::rpcs::transactions::TxStatus; -use crate::stats::RpcQueryStats; use axum::extract::ws::Message; use ethers::prelude::U64; use futures::future::AbortHandle; use futures::future::Abortable; use futures::stream::StreamExt; -use log::{trace, warn}; +use log::trace; use serde_json::json; use std::sync::atomic::{self, AtomicUsize}; use std::sync::Arc; use tokio_stream::wrappers::{BroadcastStream, WatchStream}; impl Web3ProxyApp { - // TODO: #[measure([ErrorCount, HitCount, ResponseTime, Throughput])] pub async fn eth_subscribe<'a>( self: &'a Arc, authorization: Arc, - request_json: JsonRpcRequest, + jsonrpc_request: JsonRpcRequest, subscription_count: &'a AtomicUsize, // TODO: taking a sender for Message instead of the exact json we are planning to send feels wrong, but its easier for now response_sender: flume::Sender, ) -> Web3ProxyResult<(AbortHandle, JsonRpcForwardedResponse)> { - // TODO: this is not efficient - let request_bytes = serde_json::to_string(&request_json) - .web3_context("finding request size")? - .len(); - - let request_metadata = Arc::new(RequestMetadata::new(request_bytes)); + let request_metadata = RequestMetadata::new( + self, + authorization.clone(), + RequestOrMethod::Request(&jsonrpc_request), + None, + ) + .await; let (subscription_abort_handle, subscription_registration) = AbortHandle::new_pair(); // TODO: this only needs to be unique per connection. we don't need it globably unique + // TODO: have a max number of subscriptions per key/ip. have a global max number of subscriptions? how should this be calculated? let subscription_id = subscription_count.fetch_add(1, atomic::Ordering::SeqCst); let subscription_id = U64::from(subscription_id); // save the id so we can use it in the response - let id = request_json.id.clone(); + let id = jsonrpc_request.id.clone(); // TODO: calling json! on every request is probably not fast. but we can only match against // TODO: i think we need a stricter EthSubscribeRequest type that JsonRpcRequest can turn into - match request_json.params.as_ref() { + match jsonrpc_request.params.as_ref() { Some(x) if x == &json!(["newHeads"]) => { - let authorization = authorization.clone(); let head_block_receiver = self.watch_consensus_head_receiver.clone(); - let stat_sender = self.stat_sender.clone(); + let app = self.clone(); trace!("newHeads subscription {:?}", subscription_id); tokio::spawn(async move { @@ -66,8 +65,13 @@ impl Web3ProxyApp { continue; }; - // TODO: what should the payload for RequestMetadata be? - let request_metadata = Arc::new(RequestMetadata::new(0)); + let subscription_request_metadata = RequestMetadata::new( + &app, + authorization.clone(), + RequestOrMethod::Method("eth_subscribe(newHeads)", 0), + Some(new_head.number()), + ) + .await; // TODO: make a struct for this? using our JsonRpcForwardedResponse won't work because it needs an id let response_json = json!({ @@ -83,33 +87,20 @@ impl Web3ProxyApp { let response_str = serde_json::to_string(&response_json) .expect("this should always be valid json"); - // we could use response.num_bytes() here, but since we already have the string, this is easier + // we could use JsonRpcForwardedResponseEnum::num_bytes() here, but since we already have the string, this is easier let response_bytes = response_str.len(); // TODO: do clients support binary messages? + // TODO: can we check a content type header? let response_msg = Message::Text(response_str); if response_sender.send_async(response_msg).await.is_err() { + // TODO: increment error_response? i don't think so. i think this will happen once every time a client disconnects. // TODO: cancel this subscription earlier? select on head_block_receiver.next() and an abort handle? break; }; - if let Some(stat_sender) = stat_sender.as_ref() { - let response_stat = RpcQueryStats::new( - Some("eth_subscription(newHeads)".to_string()), - authorization.clone(), - request_metadata.clone(), - response_bytes, - ); - - if let Err(err) = stat_sender.send_async(response_stat.into()).await { - // TODO: what should we do? - warn!( - "stat_sender failed inside newPendingTransactions: {:?}", - err - ); - } - } + subscription_request_metadata.add_response(response_bytes); } trace!("closed newHeads subscription {:?}", subscription_id); @@ -117,8 +108,7 @@ impl Web3ProxyApp { } Some(x) if x == &json!(["newPendingTransactions"]) => { let pending_tx_receiver = self.pending_tx_sender.subscribe(); - let stat_sender = self.stat_sender.clone(); - let authorization = authorization.clone(); + let app = self.clone(); let mut pending_tx_receiver = Abortable::new( BroadcastStream::new(pending_tx_receiver), @@ -133,7 +123,13 @@ impl Web3ProxyApp { // TODO: do something with this handle? tokio::spawn(async move { while let Some(Ok(new_tx_state)) = pending_tx_receiver.next().await { - let request_metadata = Arc::new(RequestMetadata::new(0)); + let subscription_request_metadata = RequestMetadata::new( + &app, + authorization.clone(), + RequestOrMethod::Method("eth_subscribe(newPendingTransactions)", 0), + None, + ) + .await; let new_tx = match new_tx_state { TxStatus::Pending(tx) => tx, @@ -154,9 +150,11 @@ impl Web3ProxyApp { let response_str = serde_json::to_string(&response_json) .expect("this should always be valid json"); - // we could use response.num_bytes() here, but since we already have the string, this is easier + // TODO: test that this len is the same as JsonRpcForwardedResponseEnum.num_bytes() let response_bytes = response_str.len(); + subscription_request_metadata.add_response(response_bytes); + // TODO: do clients support binary messages? let response_msg = Message::Text(response_str); @@ -164,23 +162,6 @@ impl Web3ProxyApp { // TODO: cancel this subscription earlier? select on head_block_receiver.next() and an abort handle? break; }; - - if let Some(stat_sender) = stat_sender.as_ref() { - let response_stat = RpcQueryStats::new( - Some("eth_subscription(newPendingTransactions)".to_string()), - authorization.clone(), - request_metadata.clone(), - response_bytes, - ); - - if let Err(err) = stat_sender.send_async(response_stat.into()).await { - // TODO: what should we do? - warn!( - "stat_sender failed inside newPendingTransactions: {:?}", - err - ); - } - } } trace!( @@ -191,9 +172,8 @@ impl Web3ProxyApp { } Some(x) if x == &json!(["newPendingFullTransactions"]) => { // TODO: too much copy/pasta with newPendingTransactions - let authorization = authorization.clone(); let pending_tx_receiver = self.pending_tx_sender.subscribe(); - let stat_sender = self.stat_sender.clone(); + let app = self.clone(); let mut pending_tx_receiver = Abortable::new( BroadcastStream::new(pending_tx_receiver), @@ -208,7 +188,13 @@ impl Web3ProxyApp { // TODO: do something with this handle? tokio::spawn(async move { while let Some(Ok(new_tx_state)) = pending_tx_receiver.next().await { - let request_metadata = Arc::new(RequestMetadata::new(0)); + let subscription_request_metadata = RequestMetadata::new( + &app, + authorization.clone(), + RequestOrMethod::Method("eth_subscribe(newPendingFullTransactions)", 0), + None, + ) + .await; let new_tx = match new_tx_state { TxStatus::Pending(tx) => tx, @@ -227,12 +213,11 @@ impl Web3ProxyApp { }, }); + subscription_request_metadata.add_response(&response_json); + let response_str = serde_json::to_string(&response_json) .expect("this should always be valid json"); - // we could use response.num_bytes() here, but since we already have the string, this is easier - let response_bytes = response_str.len(); - // TODO: do clients support binary messages? let response_msg = Message::Text(response_str); @@ -240,23 +225,6 @@ impl Web3ProxyApp { // TODO: cancel this subscription earlier? select on head_block_receiver.next() and an abort handle? break; }; - - if let Some(stat_sender) = stat_sender.as_ref() { - let response_stat = RpcQueryStats::new( - Some("eth_subscription(newPendingFullTransactions)".to_string()), - authorization.clone(), - request_metadata.clone(), - response_bytes, - ); - - if let Err(err) = stat_sender.send_async(response_stat.into()).await { - // TODO: what should we do? - warn!( - "stat_sender failed inside newPendingFullTransactions: {:?}", - err - ); - } - } } trace!( @@ -267,9 +235,8 @@ impl Web3ProxyApp { } Some(x) if x == &json!(["newPendingRawTransactions"]) => { // TODO: too much copy/pasta with newPendingTransactions - let authorization = authorization.clone(); let pending_tx_receiver = self.pending_tx_sender.subscribe(); - let stat_sender = self.stat_sender.clone(); + let app = self.clone(); let mut pending_tx_receiver = Abortable::new( BroadcastStream::new(pending_tx_receiver), @@ -284,7 +251,13 @@ impl Web3ProxyApp { // TODO: do something with this handle? tokio::spawn(async move { while let Some(Ok(new_tx_state)) = pending_tx_receiver.next().await { - let request_metadata = Arc::new(RequestMetadata::new(0)); + let subscription_request_metadata = RequestMetadata::new( + &app, + authorization.clone(), + "eth_subscribe(newPendingRawTransactions)", + None, + ) + .await; let new_tx = match new_tx_state { TxStatus::Pending(tx) => tx, @@ -317,22 +290,7 @@ impl Web3ProxyApp { break; }; - if let Some(stat_sender) = stat_sender.as_ref() { - let response_stat = RpcQueryStats::new( - Some("eth_subscription(newPendingRawTransactions)".to_string()), - authorization.clone(), - request_metadata.clone(), - response_bytes, - ); - - if let Err(err) = stat_sender.send_async(response_stat.into()).await { - // TODO: what should we do? - warn!( - "stat_sender failed inside newPendingRawTransactions: {:?}", - err - ); - } - } + subscription_request_metadata.add_response(response_bytes); } trace!( @@ -348,19 +306,7 @@ impl Web3ProxyApp { let response = JsonRpcForwardedResponse::from_value(json!(subscription_id), id); - if let Some(stat_sender) = self.stat_sender.as_ref() { - let response_stat = RpcQueryStats::new( - Some(request_json.method.clone()), - authorization.clone(), - request_metadata, - response.num_bytes(), - ); - - if let Err(err) = stat_sender.send_async(response_stat.into()).await { - // TODO: what should we do? - warn!("stat_sender failed inside websocket: {:?}", err); - } - } + request_metadata.add_response(&response); // TODO: make a `SubscriptonHandle(AbortHandle, JoinHandle)` struct? Ok((subscription_abort_handle, response)) diff --git a/web3_proxy/src/bin/wait_for_sync.rs b/web3_proxy/src/bin/wait_for_sync.rs index a44a377c..0252f26a 100644 --- a/web3_proxy/src/bin/wait_for_sync.rs +++ b/web3_proxy/src/bin/wait_for_sync.rs @@ -60,13 +60,12 @@ async fn main() -> anyhow::Result<()> { .context("unknown chain id for check_url")?; if let Some(chain_id) = cli_config.chain_id { - if chain_id != check_id { - return Err(anyhow::anyhow!( - "chain_id of check_url is wrong! Need {}. Found {}", - chain_id, - check_id, - )); - } + anyhow::ensure!( + chain_id == check_id, + "chain_id of check_url is wrong! Need {}. Found {}", + chain_id, + check_id, + ); } let compare_url: String = match cli_config.compare_url { @@ -93,13 +92,12 @@ async fn main() -> anyhow::Result<()> { .await .context("unknown chain id for compare_url")?; - if check_id != compare_id { - return Err(anyhow::anyhow!( - "chain_id does not match! Need {}. Found {}", - check_id, - compare_id, - )); - } + anyhow::ensure!( + check_id == compare_id, + "chain_id does not match! Need {}. Found {}", + check_id, + compare_id, + ); // start ids at 2 because id 1 was checking the chain id let counter = AtomicU32::new(2); diff --git a/web3_proxy/src/bin/web3_proxy_cli/main.rs b/web3_proxy/src/bin/web3_proxy_cli/main.rs index d3124a43..53ba221d 100644 --- a/web3_proxy/src/bin/web3_proxy_cli/main.rs +++ b/web3_proxy/src/bin/web3_proxy_cli/main.rs @@ -38,6 +38,13 @@ use web3_proxy::{ config::TopConfig, }; +#[cfg(feature = "mimalloc")] +use mimalloc::MiMalloc; + +#[cfg(feature = "mimalloc")] +#[global_allocator] +static GLOBAL: MiMalloc = MiMalloc; + #[cfg(feature = "deadlock")] use {parking_lot::deadlock, std::thread, tokio::time::Duration}; @@ -120,10 +127,10 @@ fn main() -> anyhow::Result<()> { // if RUST_LOG isn't set, configure a default // TODO: is there a better way to do this? - #[cfg(tokio_console)] + #[cfg(feature = "tokio_console")] console_subscriber::init(); - #[cfg(not(tokio_console))] + #[cfg(not(feature = "tokio_console"))] let rust_log = match std::env::var("RUST_LOG") { Ok(x) => x, Err(_) => match std::env::var("WEB3_PROXY_TRACE").map(|x| x == "true") { @@ -202,7 +209,6 @@ fn main() -> anyhow::Result<()> { (None, None) }; - #[cfg(not(tokio_console))] { let logger = env_logger::builder().parse_filters(&rust_log).build(); @@ -267,9 +273,6 @@ fn main() -> anyhow::Result<()> { } // set up tokio's async runtime - #[cfg(tokio_uring)] - let mut rt_builder = tokio_uring::Builder::new_multi_thread(); - #[cfg(not(tokio_uring))] let mut rt_builder = runtime::Builder::new_multi_thread(); rt_builder.enable_all(); @@ -278,7 +281,7 @@ fn main() -> anyhow::Result<()> { rt_builder.worker_threads(cli_config.workers); } - if let Some(top_config) = top_config.as_ref() { + if let Some(ref top_config) = top_config { let chain_id = top_config.app.chain_id; rt_builder.thread_name_fn(move || { diff --git a/web3_proxy/src/bin/web3_proxy_cli/migrate_stats_to_v2.rs b/web3_proxy/src/bin/web3_proxy_cli/migrate_stats_to_v2.rs index 17ad9370..a933a86d 100644 --- a/web3_proxy/src/bin/web3_proxy_cli/migrate_stats_to_v2.rs +++ b/web3_proxy/src/bin/web3_proxy_cli/migrate_stats_to_v2.rs @@ -1,4 +1,4 @@ -use anyhow::Context; +use anyhow::{anyhow, Context}; use argh::FromArgs; use entities::{rpc_accounting, rpc_key}; use futures::stream::FuturesUnordered; @@ -9,17 +9,17 @@ use migration::sea_orm::{ ColumnTrait, DatabaseConnection, EntityTrait, QueryFilter, QuerySelect, UpdateResult, }; use migration::{Expr, Value}; -use std::net::{IpAddr, Ipv4Addr}; +use parking_lot::Mutex; use std::num::NonZeroU64; use std::sync::Arc; use tokio::sync::broadcast; use tokio::time::Instant; -use web3_proxy::app::{AuthorizationChecks, BILLING_PERIOD_SECONDS}; +use ulid::Ulid; +use web3_proxy::app::BILLING_PERIOD_SECONDS; use web3_proxy::config::TopConfig; -use web3_proxy::frontend::authorization::{ - Authorization, AuthorizationType, RequestMetadata, RpcSecretKey, -}; -use web3_proxy::stats::{RpcQueryStats, StatBuffer}; +use web3_proxy::frontend::authorization::{Authorization, RequestMetadata, RpcSecretKey}; +use web3_proxy::rpcs::one::Web3Rpc; +use web3_proxy::stats::StatBuffer; #[derive(FromArgs, PartialEq, Eq, Debug)] /// Migrate towards influxdb and rpc_accounting_v2 from rpc_accounting @@ -67,28 +67,28 @@ impl MigrateStatsToV2 { }; // Spawn the stat-sender - let stat_sender = if let Some(emitter_spawn) = StatBuffer::try_spawn( - top_config.app.chain_id, + let emitter_spawn = StatBuffer::try_spawn( + BILLING_PERIOD_SECONDS, top_config .app .influxdb_bucket .clone() .context("No influxdb bucket was provided")?, + top_config.app.chain_id, Some(db_conn.clone()), + 30, influxdb_client.clone(), None, - 30, - 1, - BILLING_PERIOD_SECONDS, rpc_account_shutdown_recevier, - )? { - // since the database entries are used for accounting, we want to be sure everything is saved before exiting - important_background_handles.push(emitter_spawn.background_handle); + 1, + ) + .context("Error spawning stat buffer")? + .context("No stat buffer spawned. Maybe missing influx or db credentials?")?; - Some(emitter_spawn.stat_sender) - } else { - None - }; + // since the database entries are used for accounting, we want to be sure everything is saved before exiting + important_background_handles.push(emitter_spawn.background_handle); + + let stat_sender = emitter_spawn.stat_sender; let migration_timestamp = chrono::offset::Utc::now(); @@ -110,7 +110,10 @@ impl MigrateStatsToV2 { // (2) Create request metadata objects to match the old data // Iterate through all old rows, and put them into the above objects. for x in old_records.iter() { - let authorization_checks = match x.rpc_key_id { + let mut authorization = Authorization::internal(None) + .context("failed creating internal authorization")?; + + match x.rpc_key_id { Some(rpc_key_id) => { let rpc_key_obj = rpc_key::Entity::find() .filter(rpc_key::Column::Id.eq(rpc_key_id)) @@ -118,34 +121,16 @@ impl MigrateStatsToV2 { .await? .context("Could not find rpc_key_obj for the given rpc_key_id")?; - // TODO: Create authrization - // We can probably also randomly generate this, as we don't care about the user (?) - AuthorizationChecks { - user_id: rpc_key_obj.user_id, - rpc_secret_key: Some(RpcSecretKey::Uuid(rpc_key_obj.secret_key)), - rpc_secret_key_id: Some( - NonZeroU64::new(rpc_key_id) - .context("Could not use rpc_key_id to create a u64")?, - ), - ..Default::default() - } + authorization.checks.user_id = rpc_key_obj.user_id; + authorization.checks.rpc_secret_key = + Some(RpcSecretKey::Uuid(rpc_key_obj.secret_key)); + authorization.checks.rpc_secret_key_id = + NonZeroU64::try_from(rpc_key_id).ok(); } None => Default::default(), }; - let authorization_type = AuthorizationType::Internal; - let authorization = Arc::new( - Authorization::try_new( - authorization_checks, - None, - IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), - None, - None, - None, - authorization_type, - ) - .context("Initializing Authorization Struct was not successful")?, - ); + let authorization = Arc::new(authorization); // It will be like a fork basically (to simulate getting multiple single requests ...) // Iterate through all frontend requests @@ -178,46 +163,38 @@ impl MigrateStatsToV2 { // Add module at the last step to include for any remained that we missed ... (?) - // TODO: Create RequestMetadata + let backend_rpcs: Vec<_> = (0..int_backend_requests) + .map(|_| Arc::new(Web3Rpc::default())) + .collect(); + + let request_ulid = Ulid::new(); + + // Create RequestMetadata let request_metadata = RequestMetadata { - start_instant: Instant::now(), // This is overwritten later on - request_bytes: int_request_bytes, // Get the mean of all the request bytes archive_request: x.archive_request.into(), - backend_requests: Default::default(), // This is not used, instead we modify the field later - no_servers: 0.into(), // This is not relevant in the new version + authorization: Some(authorization.clone()), + backend_requests: Mutex::new(backend_rpcs), error_response: x.error_response.into(), + // debug data is in kafka, not mysql or influx + kafka_debug_logger: None, + method: x.method.clone(), + // This is not relevant in the new version + no_servers: 0.into(), + // Get the mean of all the request bytes + request_bytes: int_request_bytes as usize, response_bytes: int_response_bytes.into(), + // We did not initially record this data + response_from_backup_rpc: false.into(), + response_timestamp: x.period_datetime.timestamp().into(), response_millis: int_response_millis.into(), - // We just don't have this data - response_from_backup_rpc: false.into(), // I think we did not record this back then // Default::default() + // This is overwritten later on + start_instant: Instant::now(), + stat_sender: Some(stat_sender.clone()), + request_ulid, }; - // (3) Send through a channel to a stat emitter - // Send it to the stats sender - if let Some(stat_sender_ref) = stat_sender.as_ref() { - // info!("Method is: {:?}", x.clone().method); - let mut response_stat = RpcQueryStats::new( - x.clone().method, - authorization.clone(), - Arc::new(request_metadata), - (int_response_bytes) - .try_into() - .context("sum bytes average is not calculated properly")?, - ); - // Modify the timestamps .. - response_stat.modify_struct( - int_response_millis, - x.period_datetime.timestamp(), - int_backend_requests, - ); - // info!("Sending stats: {:?}", response_stat); - stat_sender_ref - // .send(response_stat.into()) - .send_async(response_stat.into()) - .await - .context("stat_sender sending response_stat")?; - } else { - panic!("Stat sender was not spawned!"); + if let Some(x) = request_metadata.try_send_stat()? { + return Err(anyhow!("failed saving stat! {:?}", x)); } } } diff --git a/web3_proxy/src/bin/web3_proxy_cli/proxyd.rs b/web3_proxy/src/bin/web3_proxy_cli/proxyd.rs index c70d1816..37edcafb 100644 --- a/web3_proxy/src/bin/web3_proxy_cli/proxyd.rs +++ b/web3_proxy/src/bin/web3_proxy_cli/proxyd.rs @@ -195,7 +195,7 @@ async fn run( // start the frontend port let frontend_handle = tokio::spawn(frontend::serve( app_frontend_port, - spawned_app.app.clone(), + spawned_app.app, frontend_shutdown_receiver, frontend_shutdown_complete_sender, )); @@ -417,17 +417,14 @@ mod tests { let prometheus_port = 0; let shutdown_sender = shutdown_sender.clone(); - tokio::spawn(async move { - run( - top_config, - None, - frontend_port, - prometheus_port, - 2, - shutdown_sender, - ) - .await - }) + tokio::spawn(run( + top_config, + None, + frontend_port, + prometheus_port, + 2, + shutdown_sender, + )) }; // TODO: do something to the node. query latest block, mine another block, query again diff --git a/web3_proxy/src/bin/web3_proxy_cli/rpc_accounting.rs b/web3_proxy/src/bin/web3_proxy_cli/rpc_accounting.rs index 6b73238b..2fb84848 100644 --- a/web3_proxy/src/bin/web3_proxy_cli/rpc_accounting.rs +++ b/web3_proxy/src/bin/web3_proxy_cli/rpc_accounting.rs @@ -108,11 +108,9 @@ impl RpcAccountingSubCommand { .all(db_conn) .await?; - if u_keys.is_empty() { - return Err(anyhow::anyhow!("no user keys")); - } + anyhow::ensure!(!u_keys.is_empty(), "no user keys"); - let u_key_ids: Vec<_> = u_keys.iter().map(|x| x.id).collect(); + let u_key_ids: Vec<_> = u_keys.into_iter().map(|x| x.id).collect(); condition = condition.add(rpc_accounting::Column::RpcKeyId.is_in(u_key_ids)); } diff --git a/web3_proxy/src/bin/web3_proxy_cli/sentryd/compare.rs b/web3_proxy/src/bin/web3_proxy_cli/sentryd/compare.rs index e5225fbc..d8c257dd 100644 --- a/web3_proxy/src/bin/web3_proxy_cli/sentryd/compare.rs +++ b/web3_proxy/src/bin/web3_proxy_cli/sentryd/compare.rs @@ -217,13 +217,12 @@ async fn check_rpc( .await .context(format!("awaiting response from {}", rpc))?; - if !response.status().is_success() { - return Err(anyhow::anyhow!( - "bad response from {}: {}", - rpc, - response.status(), - )); - } + anyhow::ensure!( + response.status().is_success(), + "bad response from {}: {}", + rpc, + response.status(), + ); let body = response .text() diff --git a/web3_proxy/src/bin/web3_proxy_cli/sentryd/mod.rs b/web3_proxy/src/bin/web3_proxy_cli/sentryd/mod.rs index ed8274b3..5e0af642 100644 --- a/web3_proxy/src/bin/web3_proxy_cli/sentryd/mod.rs +++ b/web3_proxy/src/bin/web3_proxy_cli/sentryd/mod.rs @@ -141,7 +141,7 @@ impl SentrydSubCommand { None, ); - if let Some(pagerduty_async) = pagerduty_async.as_ref() { + if let Some(ref pagerduty_async) = pagerduty_async { info!( "sending to pagerduty: {:#}", serde_json::to_string_pretty(&alert)? diff --git a/web3_proxy/src/bin/web3_proxy_cli/user_import.rs b/web3_proxy/src/bin/web3_proxy_cli/user_import.rs index 9e6545a3..a9875b3d 100644 --- a/web3_proxy/src/bin/web3_proxy_cli/user_import.rs +++ b/web3_proxy/src/bin/web3_proxy_cli/user_import.rs @@ -32,12 +32,11 @@ impl UserImportSubCommand { pub async fn main(self, db_conn: &DatabaseConnection) -> anyhow::Result<()> { let import_dir = Path::new(&self.input_dir); - if !import_dir.exists() { - return Err(anyhow::anyhow!( - "import dir ({}) does not exist!", - import_dir.to_string_lossy() - )); - } + anyhow::ensure!( + import_dir.exists(), + "import dir ({}) does not exist!", + import_dir.to_string_lossy() + ); let user_glob_path = import_dir.join(format!("{}-users-*.json", self.export_timestamp)); @@ -180,10 +179,7 @@ impl UserImportSubCommand { .await? { // make sure it belongs to the mapped user - if existing_rk.user_id != mapped_id { - // TODO: error or import the rest? - return Err(anyhow::anyhow!("unexpected user id")); - } + anyhow::ensure!(existing_rk.user_id == mapped_id, "unexpected user id"); // the key exists under the expected user. we are good to continue } else { diff --git a/web3_proxy/src/frontend/admin.rs b/web3_proxy/src/frontend/admin.rs index 72a49b67..c3ddf453 100644 --- a/web3_proxy/src/frontend/admin.rs +++ b/web3_proxy/src/frontend/admin.rs @@ -108,8 +108,8 @@ pub async fn admin_login_get( let login_domain = app .config .login_domain - .clone() - .unwrap_or_else(|| "llamanodes.com".to_string()); + .as_deref() + .unwrap_or("llamanodes.com"); // Also there must basically be a token, that says that one admin logins _as a user_. // I'm not yet fully sure how to handle with that logic specifically ... diff --git a/web3_proxy/src/frontend/authorization.rs b/web3_proxy/src/frontend/authorization.rs index 97672da5..5d951956 100644 --- a/web3_proxy/src/frontend/authorization.rs +++ b/web3_proxy/src/frontend/authorization.rs @@ -3,29 +3,38 @@ use super::errors::{Web3ProxyError, Web3ProxyErrorContext, Web3ProxyResult}; use super::rpc_proxy_ws::ProxyMode; use crate::app::{AuthorizationChecks, Web3ProxyApp, APP_USER_AGENT}; +use crate::jsonrpc::{JsonRpcForwardedResponse, JsonRpcRequest}; use crate::rpcs::one::Web3Rpc; +use crate::stats::{AppStat, BackendRequests, RpcQueryStats}; use crate::user_token::UserBearerToken; use axum::headers::authorization::Bearer; use axum::headers::{Header, Origin, Referer, UserAgent}; use chrono::Utc; +use core::fmt; use deferred_rate_limiter::DeferredRateLimitResult; +use derive_more::From; use entities::sea_orm_active_enums::TrackingLevel; use entities::{balance, login, rpc_key, user, user_tier}; -use ethers::types::Bytes; +use ethers::types::{Bytes, U64}; use ethers::utils::keccak256; use futures::TryFutureExt; use hashbrown::HashMap; use http::HeaderValue; use ipnet::IpNet; -use log::{error, warn}; +use log::{error, trace, warn}; use migration::sea_orm::{ColumnTrait, DatabaseConnection, EntityTrait, QueryFilter}; -use parking_lot::Mutex; +use rdkafka::message::{Header as KafkaHeader, OwnedHeaders as KafkaOwnedHeaders, OwnedMessage}; +use rdkafka::producer::{FutureProducer, FutureRecord}; +use rdkafka::util::Timeout as KafkaTimeout; use redis_rate_limiter::redis::AsyncCommands; use redis_rate_limiter::RedisRateLimitResult; use std::fmt::Display; -use std::sync::atomic::{AtomicBool, AtomicU64}; +use std::mem; +use std::sync::atomic::{self, AtomicBool, AtomicI64, AtomicU64, AtomicUsize}; +use std::time::Duration; use std::{net::IpAddr, str::FromStr, sync::Arc}; use tokio::sync::{OwnedSemaphorePermit, Semaphore}; +use tokio::task::JoinHandle; use tokio::time::Instant; use ulid::Ulid; use uuid::Uuid; @@ -70,37 +79,448 @@ pub struct Authorization { pub authorization_type: AuthorizationType, } +pub struct KafkaDebugLogger { + topic: String, + key: Vec, + headers: KafkaOwnedHeaders, + producer: FutureProducer, + num_requests: AtomicUsize, + num_responses: AtomicUsize, +} + +impl fmt::Debug for KafkaDebugLogger { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("KafkaDebugLogger") + .field("topic", &self.topic) + .finish_non_exhaustive() + } +} + +type KafkaLogResult = Result<(i32, i64), (rdkafka::error::KafkaError, OwnedMessage)>; + +impl KafkaDebugLogger { + fn try_new( + app: &Web3ProxyApp, + authorization: Arc, + head_block_num: Option<&U64>, + kafka_topic: &str, + request_ulid: Ulid, + ) -> Option> { + let kafka_producer = app.kafka_producer.clone()?; + + let kafka_topic = kafka_topic.to_string(); + + let rpc_secret_key_id = authorization + .checks + .rpc_secret_key_id + .map(|x| x.get()) + .unwrap_or_default(); + + let kafka_key = + rmp_serde::to_vec(&rpc_secret_key_id).expect("ids should always serialize with rmp"); + + let chain_id = app.config.chain_id; + + let head_block_num = head_block_num + .copied() + .or_else(|| app.balanced_rpcs.head_block_num()); + + // TODO: would be nice to have the block hash too + + // another item is added with the response, so initial_capacity is +1 what is needed here + let kafka_headers = KafkaOwnedHeaders::new_with_capacity(6) + .insert(KafkaHeader { + key: "rpc_secret_key_id", + value: authorization + .checks + .rpc_secret_key_id + .map(|x| x.to_string()) + .as_ref(), + }) + .insert(KafkaHeader { + key: "ip", + value: Some(&authorization.ip.to_string()), + }) + .insert(KafkaHeader { + key: "request_ulid", + value: Some(&request_ulid.to_string()), + }) + .insert(KafkaHeader { + key: "head_block_num", + value: head_block_num.map(|x| x.to_string()).as_ref(), + }) + .insert(KafkaHeader { + key: "chain_id", + value: Some(&chain_id.to_le_bytes()), + }); + + // save the key and headers for when we log the response + let x = Self { + topic: kafka_topic, + key: kafka_key, + headers: kafka_headers, + producer: kafka_producer, + num_requests: 0.into(), + num_responses: 0.into(), + }; + + let x = Arc::new(x); + + Some(x) + } + + fn background_log(&self, payload: Vec) -> JoinHandle { + let topic = self.topic.clone(); + let key = self.key.clone(); + let producer = self.producer.clone(); + let headers = self.headers.clone(); + + let f = async move { + let record = FutureRecord::to(&topic) + .key(&key) + .payload(&payload) + .headers(headers); + + let produce_future = + producer.send(record, KafkaTimeout::After(Duration::from_secs(5 * 60))); + + let kafka_response = produce_future.await; + + if let Err((err, msg)) = kafka_response.as_ref() { + error!("produce kafka request: {} - {:?}", err, msg); + // TODO: re-queue the msg? log somewhere else like a file on disk? + // TODO: this is bad and should probably trigger an alarm + }; + + kafka_response + }; + + tokio::spawn(f) + } + + /// for opt-in debug usage, log the request to kafka + /// TODO: generic type for request + pub fn log_debug_request(&self, request: &JsonRpcRequest) -> JoinHandle { + // TODO: is rust message pack a good choice? try rkyv instead + let payload = + rmp_serde::to_vec(&request).expect("requests should always serialize with rmp"); + + self.num_requests.fetch_add(1, atomic::Ordering::SeqCst); + + self.background_log(payload) + } + + pub fn log_debug_response(&self, response: &R) -> JoinHandle + where + R: serde::Serialize, + { + let payload = + rmp_serde::to_vec(&response).expect("requests should always serialize with rmp"); + + self.num_responses.fetch_add(1, atomic::Ordering::SeqCst); + + self.background_log(payload) + } +} + #[derive(Debug)] pub struct RequestMetadata { - pub start_instant: tokio::time::Instant, - pub request_bytes: u64, - // TODO: do we need atomics? seems like we should be able to pass a &mut around - // TODO: "archive" isn't really a boolean. + /// TODO: set archive_request during the new instead of after + /// TODO: this is more complex than "requires a block older than X height". different types of data can be pruned differently pub archive_request: AtomicBool, + + pub authorization: Option>, + + pub request_ulid: Ulid, + + /// Size of the JSON request. Does not include headers or things like that. + pub request_bytes: usize, + + /// users can opt out of method tracking for their personal dashboads + /// but we still have to store the method at least temporarily for cost calculations + pub method: Option, + + /// Instant that the request was received (or at least close to it) + /// We use Instant and not timestamps to avoid problems with leap seconds and similar issues + pub start_instant: tokio::time::Instant, /// if this is empty, there was a cache_hit - pub backend_requests: Mutex>>, + /// otherwise, it is populated with any rpc servers that were used by this request + pub backend_requests: BackendRequests, + /// The number of times the request got stuck waiting because no servers were synced pub no_servers: AtomicU64, + /// If handling the request hit an application error + /// This does not count things like a transcation reverting or a malformed request pub error_response: AtomicBool, + /// Size in bytes of the JSON response. Does not include headers or things like that. pub response_bytes: AtomicU64, + /// How many milliseconds it took to respond to the request pub response_millis: AtomicU64, + /// What time the (first) response was proxied. + /// TODO: think about how to store response times for ProxyMode::Versus + pub response_timestamp: AtomicI64, + /// True if the response required querying a backup RPC + /// RPC aggregators that query multiple providers to compare response may use this header to ignore our response. pub response_from_backup_rpc: AtomicBool, + + /// ProxyMode::Debug logs requests and responses with Kafka + /// TODO: maybe this shouldn't be determined by ProxyMode. A request param should probably enable this + pub kafka_debug_logger: Option>, + + /// Channel to send stats to + pub stat_sender: Option>, +} + +impl Default for RequestMetadata { + fn default() -> Self { + Self { + archive_request: Default::default(), + authorization: Default::default(), + backend_requests: Default::default(), + error_response: Default::default(), + kafka_debug_logger: Default::default(), + method: Default::default(), + no_servers: Default::default(), + request_bytes: Default::default(), + request_ulid: Default::default(), + response_bytes: Default::default(), + response_from_backup_rpc: Default::default(), + response_millis: Default::default(), + response_timestamp: Default::default(), + start_instant: Instant::now(), + stat_sender: Default::default(), + } + } +} + +#[derive(From)] +pub enum RequestOrMethod<'a> { + Request(&'a JsonRpcRequest), + /// jsonrpc method (or similar label) and the size that the request should count as (sometimes 0) + Method(&'a str, usize), + RequestSize(usize), +} + +impl<'a> RequestOrMethod<'a> { + fn method(&self) -> Option<&str> { + match self { + Self::Request(x) => Some(&x.method), + Self::Method(x, _) => Some(x), + _ => None, + } + } + + fn jsonrpc_request(&self) -> Option<&JsonRpcRequest> { + match self { + Self::Request(x) => Some(x), + _ => None, + } + } + + fn num_bytes(&self) -> usize { + match self { + RequestOrMethod::Method(_, num_bytes) => *num_bytes, + RequestOrMethod::Request(x) => x.num_bytes(), + RequestOrMethod::RequestSize(num_bytes) => *num_bytes, + } + } +} + +impl<'a> From<&'a str> for RequestOrMethod<'a> { + fn from(value: &'a str) -> Self { + if value.is_empty() { + Self::RequestSize(0) + } else { + Self::Method(value, 0) + } + } +} + +// TODO: i think a trait is actually the right thing to use here +#[derive(From)] +pub enum ResponseOrBytes<'a> { + Json(&'a serde_json::Value), + Response(&'a JsonRpcForwardedResponse), + Bytes(usize), +} + +impl<'a> From for ResponseOrBytes<'a> { + fn from(value: u64) -> Self { + Self::Bytes(value as usize) + } +} + +impl ResponseOrBytes<'_> { + pub fn num_bytes(&self) -> usize { + match self { + Self::Json(x) => serde_json::to_string(x) + .expect("this should always serialize") + .len(), + Self::Response(x) => x.num_bytes(), + Self::Bytes(num_bytes) => *num_bytes, + } + } } impl RequestMetadata { - pub fn new(request_bytes: usize) -> Self { - // TODO: how can we do this without turning it into a string first. this is going to slow us down! - let request_bytes = request_bytes as u64; + pub async fn new<'a, R: Into>>( + app: &Web3ProxyApp, + authorization: Arc, + request: R, + head_block_num: Option<&U64>, + ) -> Arc { + let request = request.into(); - Self { - start_instant: Instant::now(), - request_bytes, + let method = request.method().map(|x| x.to_string()); + + let request_bytes = request.num_bytes(); + + // TODO: modify the request here? I don't really like that very much. but its a sure way to get archive_request set correctly + + // TODO: add the Ulid at the haproxy or amazon load balancer level? investigate OpenTelemetry + let request_ulid = Ulid::new(); + + let kafka_debug_logger = if matches!(authorization.checks.proxy_mode, ProxyMode::Debug) { + KafkaDebugLogger::try_new( + app, + authorization.clone(), + head_block_num, + "web3_proxy:rpc", + request_ulid, + ) + } else { + None + }; + + if let Some(ref kafka_debug_logger) = kafka_debug_logger { + if let Some(request) = request.jsonrpc_request() { + // TODO: channels might be more ergonomic than spawned futures + // spawned things run in parallel easier but generally need more Arcs + kafka_debug_logger.log_debug_request(request); + } else { + // there probably isn't a new request attached to this metadata. + // this happens with websocket subscriptions + } + } + + let x = Self { archive_request: false.into(), backend_requests: Default::default(), - no_servers: 0.into(), error_response: false.into(), + kafka_debug_logger, + no_servers: 0.into(), + authorization: Some(authorization), + request_bytes, + method, response_bytes: 0.into(), - response_millis: 0.into(), response_from_backup_rpc: false.into(), + response_millis: 0.into(), + request_ulid, + response_timestamp: 0.into(), + start_instant: Instant::now(), + stat_sender: app.stat_sender.clone(), + }; + + Arc::new(x) + } + + pub fn backend_rpcs_used(&self) -> Vec> { + self.backend_requests.lock().clone() + } + + pub fn tracking_level(&self) -> TrackingLevel { + if let Some(authorization) = self.authorization.as_ref() { + authorization.checks.tracking_level.clone() + } else { + TrackingLevel::None + } + } + + pub fn opt_in_method(&self) -> Option { + match self.tracking_level() { + TrackingLevel::None | TrackingLevel::Aggregated => None, + TrackingLevel::Detailed => self.method.clone(), + } + } + + pub fn take_opt_in_method(&mut self) -> Option { + match self.tracking_level() { + TrackingLevel::None | TrackingLevel::Aggregated => None, + TrackingLevel::Detailed => self.method.take(), + } + } + + pub fn try_send_stat(mut self) -> Web3ProxyResult> { + if let Some(stat_sender) = self.stat_sender.take() { + trace!("sending stat! {:?}", self); + + let stat: RpcQueryStats = self.try_into()?; + + let stat: AppStat = stat.into(); + + if let Err(err) = stat_sender.send(stat) { + error!("failed sending stats for {:?}: {:?}", err.0, err); + // TODO: return it? that seems like it might cause an infinite loop + }; + + Ok(None) + } else { + Ok(Some(self)) + } + } + + pub fn add_response<'a, R: Into>>(&'a self, response: R) { + // TODO: fetch? set? should it be None in a Mutex? or a OnceCell? + let response = response.into(); + + let num_bytes = response.num_bytes() as u64; + + self.response_bytes + .fetch_add(num_bytes, atomic::Ordering::AcqRel); + + self.response_millis.fetch_add( + self.start_instant.elapsed().as_millis() as u64, + atomic::Ordering::AcqRel, + ); + + // TODO: record first or last timestamp? really, we need multiple + self.response_timestamp + .store(Utc::now().timestamp(), atomic::Ordering::Release); + + if let Some(kafka_debug_logger) = self.kafka_debug_logger.as_ref() { + if let ResponseOrBytes::Response(response) = response { + kafka_debug_logger.log_debug_response(response); + } + } + } + + pub fn try_send_arc_stat(self: Arc) -> anyhow::Result>> { + match Arc::try_unwrap(self) { + Ok(x) => { + let not_sent = x.try_send_stat()?.map(Arc::new); + Ok(not_sent) + } + Err(not_sent) => { + trace!( + "could not send stat while {} arcs are active", + Arc::strong_count(¬_sent) + ); + Ok(Some(not_sent)) + } + } + } + + // TODO: helper function to duplicate? needs to clear request_bytes, and all the atomics tho... +} + +// TODO: is this where the panic comes from? +impl Drop for RequestMetadata { + fn drop(&mut self) { + if self.stat_sender.is_some() { + // turn `&mut self` into `self` + let x = mem::take(self); + + // warn!("request metadata dropped without stat send! {:?}", self); + let _ = x.try_send_stat(); } } } @@ -445,11 +865,11 @@ pub async fn key_is_authorized( impl Web3ProxyApp { /// Limit the number of concurrent requests from the given ip address. - pub async fn ip_semaphore(&self, ip: IpAddr) -> Web3ProxyResult> { + pub async fn ip_semaphore(&self, ip: &IpAddr) -> Web3ProxyResult> { if let Some(max_concurrent_requests) = self.config.public_max_concurrent_requests { let semaphore = self .ip_semaphores - .get_with(ip, async move { + .get_with_by_ref(ip, async move { // TODO: set max_concurrent_requests dynamically based on load? let s = Semaphore::new(max_concurrent_requests); Arc::new(s) @@ -516,7 +936,7 @@ impl Web3ProxyApp { // limit concurrent requests let semaphore = self .bearer_token_semaphores - .get_with(user_bearer_token.clone(), async move { + .get_with_by_ref(&user_bearer_token, async move { let s = Semaphore::new(self.config.bearer_token_max_concurrent_requests as usize); Arc::new(s) }) @@ -623,7 +1043,7 @@ impl Web3ProxyApp { { Ok(DeferredRateLimitResult::Allowed) => { // rate limit allowed us. check concurrent request limits - let semaphore = self.ip_semaphore(ip).await?; + let semaphore = self.ip_semaphore(&ip).await?; Ok(RateLimitResult::Allowed(authorization, semaphore)) } @@ -643,14 +1063,14 @@ impl Web3ProxyApp { error!("rate limiter is unhappy. allowing ip. err={:?}", err); // at least we can still check the semaphore - let semaphore = self.ip_semaphore(ip).await?; + let semaphore = self.ip_semaphore(&ip).await?; Ok(RateLimitResult::Allowed(authorization, semaphore)) } } } else { // no redis, but we can still check the ip semaphore - let semaphore = self.ip_semaphore(ip).await?; + let semaphore = self.ip_semaphore(&ip).await?; // TODO: if no redis, rate limit with a local cache? "warn!" probably isn't right Ok(RateLimitResult::Allowed(authorization, semaphore)) diff --git a/web3_proxy/src/frontend/errors.rs b/web3_proxy/src/frontend/errors.rs index 1785975d..b7d1a669 100644 --- a/web3_proxy/src/frontend/errors.rs +++ b/web3_proxy/src/frontend/errors.rs @@ -25,6 +25,12 @@ pub type Web3ProxyResult = Result; // TODO: take "IntoResponse" instead of Response? pub type Web3ProxyResponse = Web3ProxyResult; +impl From for Web3ProxyResult<()> { + fn from(value: Web3ProxyError) -> Self { + Err(value) + } +} + // TODO: #[derive(Debug, Display, Error, From)] pub enum Web3ProxyError { @@ -35,6 +41,9 @@ pub enum Web3ProxyError { #[error(ignore)] #[from(ignore)] BadRequest(String), + #[error(ignore)] + #[from(ignore)] + BadResponse(String), BadRouting, Database(DbErr), #[display(fmt = "{:#?}, {:#?}", _0, _1)] @@ -168,6 +177,18 @@ impl Web3ProxyError { ), ) } + Self::BadResponse(err) => { + // TODO: think about this one more. ankr gives us this because ethers fails to parse responses without an id + debug!("BAD_RESPONSE: {}", err); + ( + StatusCode::INTERNAL_SERVER_ERROR, + JsonRpcForwardedResponse::from_str( + &format!("bad response: {}", err), + Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), + None, + ), + ) + } Self::BadRouting => { error!("BadRouting"); ( diff --git a/web3_proxy/src/frontend/mod.rs b/web3_proxy/src/frontend/mod.rs index 9715c111..728978da 100644 --- a/web3_proxy/src/frontend/mod.rs +++ b/web3_proxy/src/frontend/mod.rs @@ -17,6 +17,7 @@ use axum::{ Extension, Router, }; use http::header::AUTHORIZATION; +use listenfd::ListenFd; use log::info; use moka::future::Cache; use std::net::SocketAddr; @@ -45,6 +46,7 @@ pub async fn serve( ) -> anyhow::Result<()> { // setup caches for whatever the frontend needs // no need for max items since it is limited by the enum key + // TODO: latest moka allows for different ttls for different let json_response_cache: FrontendJsonResponseCache = Cache::builder() .time_to_live(Duration::from_secs(2)) .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); @@ -62,102 +64,77 @@ pub async fn serve( // // HTTP RPC (POST) // + // Websocket RPC (GET) + // If not an RPC, GET will redirect to urls in the config + // // public - .route("/", post(rpc_proxy_http::proxy_web3_rpc)) + .route( + "/", + post(rpc_proxy_http::proxy_web3_rpc).get(rpc_proxy_ws::websocket_handler), + ) // authenticated with and without trailing slash .route( "/rpc/:rpc_key/", - post(rpc_proxy_http::proxy_web3_rpc_with_key), + post(rpc_proxy_http::proxy_web3_rpc_with_key) + .get(rpc_proxy_ws::websocket_handler_with_key), ) .route( "/rpc/:rpc_key", - post(rpc_proxy_http::proxy_web3_rpc_with_key), + post(rpc_proxy_http::proxy_web3_rpc_with_key) + .get(rpc_proxy_ws::websocket_handler_with_key), ) // authenticated debug route with and without trailing slash .route( "/debug/:rpc_key/", - post(rpc_proxy_http::debug_proxy_web3_rpc_with_key), + post(rpc_proxy_http::debug_proxy_web3_rpc_with_key) + .get(rpc_proxy_ws::debug_websocket_handler_with_key), ) .route( "/debug/:rpc_key", - post(rpc_proxy_http::debug_proxy_web3_rpc_with_key), + post(rpc_proxy_http::debug_proxy_web3_rpc_with_key) + .get(rpc_proxy_ws::debug_websocket_handler_with_key), ) // public fastest with and without trailing slash - .route("/fastest/", post(rpc_proxy_http::fastest_proxy_web3_rpc)) - .route("/fastest", post(rpc_proxy_http::fastest_proxy_web3_rpc)) + .route( + "/fastest/", + post(rpc_proxy_http::fastest_proxy_web3_rpc) + .get(rpc_proxy_ws::fastest_websocket_handler), + ) + .route( + "/fastest", + post(rpc_proxy_http::fastest_proxy_web3_rpc) + .get(rpc_proxy_ws::fastest_websocket_handler), + ) // authenticated fastest with and without trailing slash .route( "/fastest/:rpc_key/", - post(rpc_proxy_http::fastest_proxy_web3_rpc_with_key), + post(rpc_proxy_http::fastest_proxy_web3_rpc_with_key) + .get(rpc_proxy_ws::fastest_websocket_handler_with_key), ) .route( "/fastest/:rpc_key", - post(rpc_proxy_http::fastest_proxy_web3_rpc_with_key), - ) - // public versus - .route("/versus/", post(rpc_proxy_http::versus_proxy_web3_rpc)) - .route("/versus", post(rpc_proxy_http::versus_proxy_web3_rpc)) - // authenticated versus with and without trailing slash - .route( - "/versus/:rpc_key/", - post(rpc_proxy_http::versus_proxy_web3_rpc_with_key), - ) - .route( - "/versus/:rpc_key", - post(rpc_proxy_http::versus_proxy_web3_rpc_with_key), - ) - // - // Websocket RPC (GET) - // If not an RPC, this will redirect to configurable urls - // - // public - .route("/", get(rpc_proxy_ws::websocket_handler)) - // authenticated with and without trailing slash - .route( - "/rpc/:rpc_key/", - get(rpc_proxy_ws::websocket_handler_with_key), - ) - .route( - "/rpc/:rpc_key", - get(rpc_proxy_ws::websocket_handler_with_key), - ) - // debug with and without trailing slash - .route( - "/debug/:rpc_key/", - get(rpc_proxy_ws::websocket_handler_with_key), - ) - .route( - "/debug/:rpc_key", - get(rpc_proxy_ws::websocket_handler_with_key), - ) // public fastest with and without trailing slash - .route("/fastest/", get(rpc_proxy_ws::fastest_websocket_handler)) - .route("/fastest", get(rpc_proxy_ws::fastest_websocket_handler)) - // authenticated fastest with and without trailing slash - .route( - "/fastest/:rpc_key/", - get(rpc_proxy_ws::fastest_websocket_handler_with_key), - ) - .route( - "/fastest/:rpc_key", - get(rpc_proxy_ws::fastest_websocket_handler_with_key), + post(rpc_proxy_http::fastest_proxy_web3_rpc_with_key) + .get(rpc_proxy_ws::fastest_websocket_handler_with_key), ) // public versus .route( "/versus/", - get(rpc_proxy_ws::versus_websocket_handler_with_key), + post(rpc_proxy_http::versus_proxy_web3_rpc).get(rpc_proxy_ws::versus_websocket_handler), ) .route( "/versus", - get(rpc_proxy_ws::versus_websocket_handler_with_key), + post(rpc_proxy_http::versus_proxy_web3_rpc).get(rpc_proxy_ws::versus_websocket_handler), ) // authenticated versus with and without trailing slash .route( "/versus/:rpc_key/", - get(rpc_proxy_ws::versus_websocket_handler_with_key), + post(rpc_proxy_http::versus_proxy_web3_rpc_with_key) + .get(rpc_proxy_ws::versus_websocket_handler_with_key), ) .route( "/versus/:rpc_key", - get(rpc_proxy_ws::versus_websocket_handler_with_key), + post(rpc_proxy_http::versus_proxy_web3_rpc_with_key) + .get(rpc_proxy_ws::versus_websocket_handler_with_key), ) // // System things @@ -241,19 +218,29 @@ pub async fn serve( // handle cors .layer(CorsLayer::very_permissive()) // application state - .layer(Extension(proxy_app.clone())) + .layer(Extension(proxy_app)) // frontend caches .layer(Extension(json_response_cache)) .layer(Extension(health_cache)) // 404 for any unknown routes .fallback(errors::handler_404); - // run our app with hyper - // TODO: allow only listening on localhost? top_config.app.host.parse()? - let addr = SocketAddr::from(([0, 0, 0, 0], port)); - info!("listening on port {}", port); + let server_builder = if let Some(listener) = ListenFd::from_env().take_tcp_listener(0)? { + // use systemd socket magic for no downtime deploys + let addr = listener.local_addr()?; - // TODO: into_make_service is enough if we always run behind a proxy. make into_make_service_with_connect_info optional? + info!("listening with fd at {}", addr); + + axum::Server::from_tcp(listener)? + } else { + info!("listening on port {}", port); + // TODO: allow only listening on localhost? top_config.app.host.parse()? + let addr = SocketAddr::from(([0, 0, 0, 0], port)); + + axum::Server::try_bind(&addr)? + }; + + // into_make_service is enough if we always run behind a proxy /* It sequentially looks for an IP in: - x-forwarded-for header (de-facto standard) @@ -261,12 +248,21 @@ pub async fn serve( - forwarded header (new standard) - axum::extract::ConnectInfo (if not behind proxy) */ - let service = app.into_make_service_with_connect_info::(); + #[cfg(feature = "connectinfo")] + let make_service = { + info!("connectinfo feature enabled"); + app.into_make_service_with_connect_info::() + }; - // `axum::Server` is a re-export of `hyper::Server` - let server = axum::Server::bind(&addr) + #[cfg(not(feature = "connectinfo"))] + let make_service = { + info!("connectinfo feature disabled"); + app.into_make_service() + }; + + let server = server_builder + .serve(make_service) // TODO: option to use with_connect_info. we want it in dev, but not when running behind a proxy, but not - .serve(service) .with_graceful_shutdown(async move { let _ = shutdown_receiver.recv().await; }) diff --git a/web3_proxy/src/frontend/rpc_proxy_http.rs b/web3_proxy/src/frontend/rpc_proxy_http.rs index 2d938adb..06b55603 100644 --- a/web3_proxy/src/frontend/rpc_proxy_http.rs +++ b/web3_proxy/src/frontend/rpc_proxy_http.rs @@ -63,12 +63,12 @@ async fn _proxy_web3_rpc( let authorization = Arc::new(authorization); - let (response, rpcs, _semaphore) = app + let (status_code, response, rpcs, _semaphore) = app .proxy_web3_rpc(authorization, payload) .await - .map(|(x, y)| (x, y, semaphore))?; + .map(|(s, x, y)| (s, x, y, semaphore))?; - let mut response = Json(&response).into_response(); + let mut response = (status_code, Json(response)).into_response(); let headers = response.headers_mut(); @@ -129,6 +129,8 @@ pub async fn proxy_web3_rpc_with_key( .await } +// TODO: if a /debug/ request gets rejected by an invalid request, there won't be any kafka log +// TODO: #[debug_handler] pub async fn debug_proxy_web3_rpc_with_key( Extension(app): Extension>, @@ -228,12 +230,12 @@ async fn _proxy_web3_rpc_with_key( let rpc_secret_key_id = authorization.checks.rpc_secret_key_id; - let (response, rpcs, _semaphore) = app + let (status_code, response, rpcs, _semaphore) = app .proxy_web3_rpc(authorization, payload) .await - .map(|(x, y)| (x, y, semaphore))?; + .map(|(s, x, y)| (s, x, y, semaphore))?; - let mut response = Json(&response).into_response(); + let mut response = (status_code, Json(response)).into_response(); let headers = response.headers_mut(); diff --git a/web3_proxy/src/frontend/rpc_proxy_ws.rs b/web3_proxy/src/frontend/rpc_proxy_ws.rs index dd04f0dc..e0522b00 100644 --- a/web3_proxy/src/frontend/rpc_proxy_ws.rs +++ b/web3_proxy/src/frontend/rpc_proxy_ws.rs @@ -5,12 +5,12 @@ use super::authorization::{ip_is_authorized, key_is_authorized, Authorization, RequestMetadata}; use super::errors::{Web3ProxyError, Web3ProxyResponse}; use crate::jsonrpc::JsonRpcId; -use crate::stats::RpcQueryStats; use crate::{ app::Web3ProxyApp, frontend::errors::Web3ProxyResult, jsonrpc::{JsonRpcForwardedResponse, JsonRpcForwardedResponseEnum, JsonRpcRequest}, }; +use anyhow::Context; use axum::headers::{Origin, Referer, UserAgent}; use axum::{ extract::ws::{Message, WebSocket, WebSocketUpgrade}, @@ -20,6 +20,8 @@ use axum::{ }; use axum_client_ip::InsecureClientIp; use axum_macros::debug_handler; +use ethers::types::Bytes; +use fstrings::{f, format_args_f}; use futures::SinkExt; use futures::{ future::AbortHandle, @@ -28,12 +30,13 @@ use futures::{ use handlebars::Handlebars; use hashbrown::HashMap; use http::StatusCode; -use log::{info, trace, warn}; +use log::{info, trace}; use serde_json::json; use std::sync::Arc; use std::{str::from_utf8_mut, sync::atomic::AtomicUsize}; use tokio::sync::{broadcast, OwnedSemaphorePermit, RwLock}; +/// How to select backend servers for a request #[derive(Copy, Clone, Debug)] pub enum ProxyMode { /// send to the "best" synced server @@ -43,6 +46,7 @@ pub enum ProxyMode { /// send to all servers for benchmarking. return the fastest non-error response Versus, /// send all requests and responses to kafka + /// TODO: should this be seperate from best/fastest/versus? Debug, } @@ -314,14 +318,15 @@ async fn proxy_web3_socket( } /// websockets support a few more methods than http clients +/// TODO: i think this subscriptions hashmap grows unbounded async fn handle_socket_payload( app: Arc, authorization: &Arc, payload: &str, response_sender: &flume::Sender, subscription_count: &AtomicUsize, - subscriptions: Arc>>, -) -> (Message, Option) { + subscriptions: Arc>>, +) -> Web3ProxyResult<(Message, Option)> { let (authorization, semaphore) = match authorization.check_again(&app).await { Ok((a, s)) => (a, s), Err(err) => { @@ -329,7 +334,7 @@ async fn handle_socket_payload( let err = serde_json::to_string(&err).expect("to_string should always work here"); - return (Message::Text(err), None); + return Ok((Message::Text(err), None)); } }; @@ -338,84 +343,90 @@ async fn handle_socket_payload( Ok(json_request) => { let id = json_request.id.clone(); - let response: Web3ProxyResult = match &json_request.method - [..] - { - "eth_subscribe" => { - // TODO: how can we subscribe with proxy_mode? - match app - .eth_subscribe( - authorization.clone(), - json_request, - subscription_count, - response_sender.clone(), - ) + // TODO: move this to a seperate function so we can use the try operator + let response: Web3ProxyResult = + match &json_request.method[..] { + "eth_subscribe" => { + // TODO: how can we subscribe with proxy_mode? + match app + .eth_subscribe( + authorization.clone(), + json_request, + subscription_count, + response_sender.clone(), + ) + .await + { + Ok((handle, response)) => { + { + let mut x = subscriptions.write().await; + + let result: &serde_json::value::RawValue = response + .result + .as_ref() + .context("there should be a result here")?; + + // TODO: there must be a better way to do this + let k: Bytes = serde_json::from_str(result.get()) + .context("subscription ids must be bytes")?; + + x.insert(k, handle); + }; + + Ok(response.into()) + } + Err(err) => Err(err), + } + } + "eth_unsubscribe" => { + let request_metadata = + RequestMetadata::new(&app, authorization.clone(), &json_request, None) + .await; + + #[derive(serde::Deserialize)] + struct EthUnsubscribeParams([Bytes; 1]); + + if let Some(params) = json_request.params { + match serde_json::from_value(params) { + Ok::(params) => { + let subscription_id = ¶ms.0[0]; + + // TODO: is this the right response? + let partial_response = { + let mut x = subscriptions.write().await; + match x.remove(subscription_id) { + None => false, + Some(handle) => { + handle.abort(); + true + } + } + }; + + let response = JsonRpcForwardedResponse::from_value( + json!(partial_response), + id.clone(), + ); + + request_metadata.add_response(&response); + + Ok(response.into()) + } + Err(err) => Err(Web3ProxyError::BadRequest(f!( + "incorrect params given for eth_unsubscribe. {err:?}" + ))), + } + } else { + Err(Web3ProxyError::BadRequest( + "no params given for eth_unsubscribe".to_string(), + )) + } + } + _ => app + .proxy_web3_rpc(authorization.clone(), json_request.into()) .await - { - Ok((handle, response)) => { - // TODO: better key - let mut x = subscriptions.write().await; - - x.insert( - response - .result - .as_ref() - // TODO: what if there is an error? - .expect("response should always have a result, not an error") - .to_string(), - handle, - ); - - Ok(response.into()) - } - Err(err) => Err(err), - } - } - "eth_unsubscribe" => { - // TODO: move this logic into the app? - let request_bytes = json_request.num_bytes(); - - let request_metadata = Arc::new(RequestMetadata::new(request_bytes)); - - let subscription_id = json_request.params.unwrap().to_string(); - - let mut x = subscriptions.write().await; - - // TODO: is this the right response? - let partial_response = match x.remove(&subscription_id) { - None => false, - Some(handle) => { - handle.abort(); - true - } - }; - - drop(x); - - let response = - JsonRpcForwardedResponse::from_value(json!(partial_response), id.clone()); - - if let Some(stat_sender) = app.stat_sender.as_ref() { - let response_stat = RpcQueryStats::new( - Some(json_request.method.clone()), - authorization.clone(), - request_metadata, - response.num_bytes(), - ); - - if let Err(err) = stat_sender.send_async(response_stat.into()).await { - // TODO: what should we do? - warn!("stat_sender failed during eth_unsubscribe: {:?}", err); - } - } - - Ok(response.into()) - } - _ => app - .proxy_web3_rpc(authorization.clone(), json_request.into()) - .await - .map(|(response, _)| response), - }; + .map(|(status_code, response, _)| response), + }; (id, response) } @@ -434,7 +445,7 @@ async fn handle_socket_payload( } }; - (Message::Text(response_str), semaphore) + Ok((Message::Text(response_str), semaphore)) } async fn read_web3_socket( @@ -443,7 +454,7 @@ async fn read_web3_socket( mut ws_rx: SplitStream, response_sender: flume::Sender, ) { - // TODO: need a concurrent hashmap + // RwLock should be fine here. a user isn't going to be opening tons of subscriptions let subscriptions = Arc::new(RwLock::new(HashMap::new())); let subscription_count = Arc::new(AtomicUsize::new(1)); @@ -467,16 +478,17 @@ async fn read_web3_socket( // new message from our client. forward to a backend and then send it through response_tx let response_msg = match msg { - Message::Text(payload) => { + Message::Text(ref payload) => { + // TODO: do not unwrap! let (msg, s) = handle_socket_payload( app.clone(), &authorization, - &payload, + payload, &response_sender, &subscription_count, subscriptions, ) - .await; + .await.unwrap(); _semaphore = s; @@ -499,6 +511,7 @@ async fn read_web3_socket( Message::Binary(mut payload) => { let payload = from_utf8_mut(&mut payload).unwrap(); + // TODO: do not unwrap! let (msg, s) = handle_socket_payload( app.clone(), &authorization, @@ -507,7 +520,7 @@ async fn read_web3_socket( &subscription_count, subscriptions, ) - .await; + .await.unwrap(); _semaphore = s; diff --git a/web3_proxy/src/frontend/status.rs b/web3_proxy/src/frontend/status.rs index 970ad551..0e46c21a 100644 --- a/web3_proxy/src/frontend/status.rs +++ b/web3_proxy/src/frontend/status.rs @@ -30,9 +30,13 @@ pub async fn health( /// Easy alerting if backup servers are in use. pub async fn backups_needed(Extension(app): Extension>) -> impl IntoResponse { let code = { - let consensus_rpcs = app.balanced_rpcs.watch_consensus_rpcs_sender.borrow(); + let consensus_rpcs = app + .balanced_rpcs + .watch_consensus_rpcs_sender + .borrow() + .clone(); - if let Some(consensus_rpcs) = consensus_rpcs.as_ref() { + if let Some(ref consensus_rpcs) = consensus_rpcs { if consensus_rpcs.backups_needed { StatusCode::INTERNAL_SERVER_ERROR } else { diff --git a/web3_proxy/src/frontend/users/authentication.rs b/web3_proxy/src/frontend/users/authentication.rs index f70b63e9..e681ea41 100644 --- a/web3_proxy/src/frontend/users/authentication.rs +++ b/web3_proxy/src/frontend/users/authentication.rs @@ -284,7 +284,7 @@ pub async fn user_login_post( let rpc_secret_key = RpcSecretKey::new(); let user_rpc_key = rpc_key::ActiveModel { - user_id: sea_orm::Set(caller.id.clone()), + user_id: sea_orm::Set(caller.id), secret_key: sea_orm::Set(rpc_secret_key.into()), description: sea_orm::Set(None), ..Default::default() @@ -297,7 +297,7 @@ pub async fn user_login_post( // We should also create the balance entry ... let user_balance = balance::ActiveModel { - user_id: sea_orm::Set(caller.id.clone()), + user_id: sea_orm::Set(caller.id), available_balance: sea_orm::Set(Decimal::new(0, 0)), used_balance: sea_orm::Set(Decimal::new(0, 0)), ..Default::default() diff --git a/web3_proxy/src/frontend/users/payment.rs b/web3_proxy/src/frontend/users/payment.rs index 9fbf7daa..b49c3705 100644 --- a/web3_proxy/src/frontend/users/payment.rs +++ b/web3_proxy/src/frontend/users/payment.rs @@ -321,7 +321,7 @@ pub async fn user_balance_post( } // Get the topics out - let topic: H256 = H256::from(log.topics.get(0).unwrap().to_owned()); + let topic: H256 = log.topics.get(0).unwrap().to_owned(); if topic != deposit_topic { debug!( "Out: Topic is not relevant: {:?} {:?}", @@ -489,8 +489,9 @@ pub async fn user_balance_post( })), ) .into_response(); + // Return early if the log was added, assume there is at most one valid log per transaction - return Ok(response.into()); + return Ok(response); } Err(Web3ProxyError::BadRequest( diff --git a/web3_proxy/src/frontend/users/referral.rs b/web3_proxy/src/frontend/users/referral.rs index ac4649d0..c4f613ac 100644 --- a/web3_proxy/src/frontend/users/referral.rs +++ b/web3_proxy/src/frontend/users/referral.rs @@ -19,7 +19,6 @@ use migration::sea_orm::ActiveModelTrait; use migration::sea_orm::ColumnTrait; use migration::sea_orm::EntityTrait; use migration::sea_orm::QueryFilter; -use migration::sea_orm::TransactionTrait; use serde_json::json; use std::sync::Arc; @@ -49,7 +48,7 @@ pub async fn user_referral_link_get( warn!("User tier is: {:?}", user_tier); // TODO: This shouldn't be hardcoded. Also, it should be an enum, not sth like this ... if user_tier.id != 6 { - return Err(Web3ProxyError::PaymentRequired.into()); + return Err(Web3ProxyError::PaymentRequired); } // Then get the referral token diff --git a/web3_proxy/src/frontend/users/subuser.rs b/web3_proxy/src/frontend/users/subuser.rs index c1d4eb3b..e382da73 100644 --- a/web3_proxy/src/frontend/users/subuser.rs +++ b/web3_proxy/src/frontend/users/subuser.rs @@ -49,7 +49,7 @@ pub async fn get_keys_as_subuser( .all(db_replica.conn()) .await? .into_iter() - .map(|x| (x.rpc_secret_key_id.clone(), x)) + .map(|x| (x.rpc_secret_key_id, x)) .collect::>(); // Now return a list of all subusers (their wallets) @@ -147,7 +147,7 @@ pub async fn get_subusers( .all(db_replica.conn()) .await? .into_iter() - .map(|x| (x.user_id.clone(), x)) + .map(|x| (x.user_id, x)) .collect::>(); // Now return a list of all subusers (their wallets) @@ -314,7 +314,7 @@ pub async fn modify_subuser( let rpc_secret_key = RpcSecretKey::new(); let subuser_rpc_key = rpc_key::ActiveModel { - user_id: sea_orm::Set(subuser.id.clone()), + user_id: sea_orm::Set(subuser.id), secret_key: sea_orm::Set(rpc_secret_key.into()), description: sea_orm::Set(None), ..Default::default() @@ -327,7 +327,7 @@ pub async fn modify_subuser( // We should also create the balance entry ... let subuser_balance = balance::ActiveModel { - user_id: sea_orm::Set(subuser.id.clone()), + user_id: sea_orm::Set(subuser.id), available_balance: sea_orm::Set(Decimal::new(0, 0)), used_balance: sea_orm::Set(Decimal::new(0, 0)), ..Default::default() @@ -374,7 +374,8 @@ pub async fn modify_subuser( let txn = db_conn.begin().await?; let mut action = "no action"; - let _ = match subuser_entry_secondary_user { + + match subuser_entry_secondary_user { Some(secondary_user) => { // In this case, remove the subuser let mut active_subuser_entry_secondary_user = secondary_user.into_active_model(); @@ -421,6 +422,7 @@ pub async fn modify_subuser( })), ) .into_response(); + // Return early if the log was added, assume there is at most one valid log per transaction - Ok(response.into()) + Ok(response) } diff --git a/web3_proxy/src/jsonrpc.rs b/web3_proxy/src/jsonrpc.rs index 78c76b0e..eabd3d0b 100644 --- a/web3_proxy/src/jsonrpc.rs +++ b/web3_proxy/src/jsonrpc.rs @@ -290,9 +290,12 @@ impl JsonRpcForwardedResponse { data = err.data.clone(); } else if let Some(err) = err.as_serde_error() { // this is not an rpc error. keep it as an error - return Err(Web3ProxyError::BadRequest(format!("bad request: {}", err))); + return Err(Web3ProxyError::BadResponse(format!( + "bad response: {}", + err + ))); } else { - return Err(anyhow::anyhow!("unexpected ethers error!").into()); + return Err(anyhow::anyhow!("unexpected ethers error! {:?}", err).into()); } } e => return Err(e.into()), diff --git a/web3_proxy/src/rpcs/blockchain.rs b/web3_proxy/src/rpcs/blockchain.rs index c7f641e3..3d6ac3dd 100644 --- a/web3_proxy/src/rpcs/blockchain.rs +++ b/web3_proxy/src/rpcs/blockchain.rs @@ -169,7 +169,6 @@ impl Web3Rpcs { // TODO: i think we can rearrange this function to make it faster on the hot path let block_hash = block.hash(); - // skip Block::default() if block_hash.is_zero() { debug!("Skipping block without hash!"); return Ok(block); @@ -189,7 +188,7 @@ impl Web3Rpcs { // TODO: use their get_with let block = self .blocks_by_hash - .get_with(*block_hash, async move { block.clone() }) + .get_with(*block_hash, async move { block }) .await; Ok(block) @@ -236,7 +235,7 @@ impl Web3Rpcs { None => { // TODO: helper for method+params => JsonRpcRequest // TODO: does this id matter? - let request = json!({ "id": "1", "method": "eth_getBlockByHash", "params": get_block_params }); + let request = json!({ "jsonrpc": "2.0", "id": "1", "method": "eth_getBlockByHash", "params": get_block_params }); let request: JsonRpcRequest = serde_json::from_value(request)?; // TODO: request_metadata? maybe we should put it in the authorization? @@ -244,7 +243,7 @@ impl Web3Rpcs { let response = self .try_send_best_consensus_head_connection( authorization, - request, + &request, None, None, None, @@ -344,7 +343,7 @@ impl Web3Rpcs { let request: JsonRpcRequest = serde_json::from_value(request)?; let response = self - .try_send_best_consensus_head_connection(authorization, request, None, Some(num), None) + .try_send_best_consensus_head_connection(authorization, &request, None, Some(num), None) .await?; if response.error.is_some() { @@ -446,7 +445,7 @@ impl Web3Rpcs { let consensus_head_block = new_synced_connections.head_block.clone(); let num_consensus_rpcs = new_synced_connections.num_conns(); let num_active_rpcs = consensus_finder.len(); - let total_rpcs = self.by_name.read().len(); + let total_rpcs = self.by_name.load().len(); let old_consensus_head_connections = self .watch_consensus_rpcs_sender diff --git a/web3_proxy/src/rpcs/consensus.rs b/web3_proxy/src/rpcs/consensus.rs index a20f55f9..e6261412 100644 --- a/web3_proxy/src/rpcs/consensus.rs +++ b/web3_proxy/src/rpcs/consensus.rs @@ -3,6 +3,7 @@ use super::many::Web3Rpcs; use super::one::Web3Rpc; use crate::frontend::authorization::Authorization; use crate::frontend::errors::{Web3ProxyErrorContext, Web3ProxyResult}; +use derive_more::Constructor; use ethers::prelude::{H256, U64}; use hashbrown::{HashMap, HashSet}; use itertools::{Itertools, MinMaxResult}; @@ -10,28 +11,146 @@ use log::{trace, warn}; use moka::future::Cache; use serde::Serialize; use std::cmp::Reverse; +use std::collections::BTreeMap; use std::fmt; use std::sync::Arc; use tokio::time::Instant; +#[derive(Clone, Serialize)] +struct RpcData { + head_block_num: U64, + // TODO: this is too simple. erigon has 4 prune levels (hrct) + oldest_block_num: U64, +} + +impl RpcData { + fn new(rpc: &Web3Rpc, head: &Web3ProxyBlock) -> Self { + let head_block_num = *head.number(); + + let block_data_limit = rpc.block_data_limit(); + + let oldest_block_num = head_block_num.saturating_sub(block_data_limit); + + Self { + head_block_num, + oldest_block_num, + } + } + + // TODO: take an enum for the type of data (hrtc) + fn data_available(&self, block_num: &U64) -> bool { + *block_num >= self.oldest_block_num && *block_num <= self.head_block_num + } +} + +#[derive(Constructor, Clone, Copy, Debug, Default, Eq, PartialEq, Serialize)] +pub struct RpcRanking { + tier: u64, + backup: bool, + head_num: Option, +} + +impl RpcRanking { + pub fn add_offset(&self, offset: u64) -> Self { + Self { + tier: self.tier + offset, + backup: self.backup, + head_num: self.head_num, + } + } + + pub fn default_with_backup(backup: bool) -> Self { + Self { + backup, + ..Default::default() + } + } + + fn sort_key(&self) -> (u64, bool, Reverse>) { + // TODO: add soft_limit here? add peak_ewma here? + (self.tier, !self.backup, Reverse(self.head_num)) + } +} + +impl Ord for RpcRanking { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + self.sort_key().cmp(&other.sort_key()) + } +} + +impl PartialOrd for RpcRanking { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +pub type RankedRpcMap = BTreeMap>>; + /// A collection of Web3Rpcs that are on the same block. /// Serialize is so we can print it on our debug endpoint #[derive(Clone, Serialize)] pub struct ConsensusWeb3Rpcs { pub(crate) tier: u64, + pub(crate) backups_needed: bool, pub(crate) head_block: Web3ProxyBlock, pub(crate) best_rpcs: Vec>, - // TODO: functions like "compare_backup_vote()" - // pub(super) backups_voted: Option, - pub(crate) backups_needed: bool, + pub(crate) other_rpcs: RankedRpcMap, + + rpc_data: HashMap, RpcData>, } impl ConsensusWeb3Rpcs { - #[inline(always)] + #[inline] pub fn num_conns(&self) -> usize { self.best_rpcs.len() } + pub fn has_block_data(&self, rpc: &Web3Rpc, block_num: &U64) -> bool { + self.rpc_data + .get(rpc) + .map(|x| x.data_available(block_num)) + .unwrap_or(false) + } + + pub fn filter( + &self, + skip: &[Arc], + min_block_needed: Option<&U64>, + max_block_needed: Option<&U64>, + rpc: &Arc, + ) -> bool { + if skip.contains(rpc) { + trace!("skipping {}", rpc); + return false; + } + + if let Some(min_block_needed) = min_block_needed { + if !self.has_block_data(rpc, min_block_needed) { + trace!( + "{} is missing min_block_needed ({}). skipping", + rpc, + min_block_needed, + ); + return false; + } + } + + if let Some(max_block_needed) = max_block_needed { + if !self.has_block_data(rpc, max_block_needed) { + trace!( + "{} is missing max_block_needed ({}). skipping", + rpc, + max_block_needed, + ); + return false; + } + } + + // we could check hard rate limits here, but i think it is faster to do later + + true + } + // TODO: sum_hard_limit? } @@ -46,6 +165,7 @@ impl fmt::Debug for ConsensusWeb3Rpcs { } } +// TODO: refs for all of these. borrow on a Sender is cheap enough impl Web3Rpcs { // TODO: return a ref? pub fn head_block(&self) -> Option { @@ -93,7 +213,6 @@ pub struct ConsensusFinder { /// `tiers[0] = only tier 0` /// `tiers[1] = tier 0 and tier 1` /// `tiers[n] = tier 0..=n` - /// This is a BTreeMap and not a Vec because sometimes a tier is empty rpc_heads: HashMap, Web3ProxyBlock>, /// never serve blocks that are too old max_block_age: Option, @@ -137,7 +256,7 @@ impl ConsensusFinder { async fn insert(&mut self, rpc: Arc, block: Web3ProxyBlock) -> Option { let first_seen = self .first_seen - .get_with(*block.hash(), async move { Instant::now() }) + .get_with_by_ref(block.hash(), async move { Instant::now() }) .await; // TODO: this should be 0 if we are first seen, but i think it will be slightly non-zero. @@ -166,13 +285,6 @@ impl ConsensusFinder { .await .web3_context("failed caching block")?; - // if let Some(max_block_lag) = max_block_lag { - // if rpc_head_block.number() < ??? { - // trace!("rpc_head_block from {} is too far behind! {}", rpc, rpc_head_block); - // return Ok(self.remove(&rpc).is_some()); - // } - // } - if let Some(max_age) = self.max_block_age { if rpc_head_block.age() > max_age { trace!("rpc_head_block from {} is too old! {}", rpc, rpc_head_block); @@ -324,7 +436,7 @@ impl ConsensusFinder { ) -> Option { // sort the primary votes ascending by tier and descending by block num let mut votes: Vec<_> = votes - .iter() + .into_iter() .map(|(block, (rpc_names, sum_soft_limit))| (block, sum_soft_limit, rpc_names)) .collect(); votes.sort_by_cached_key(|(block, sum_soft_limit, rpc_names)| { @@ -366,11 +478,39 @@ impl ConsensusFinder { let backups_needed = consensus_rpcs.iter().any(|x| x.backup); + let mut other_rpcs = BTreeMap::new(); + + for (x, x_head) in self + .rpc_heads + .iter() + .filter(|(k, _)| !consensus_rpcs.contains(k)) + { + let x_head_num = *x_head.number(); + + let key: RpcRanking = RpcRanking::new(x.tier, x.backup, Some(x_head_num)); + + other_rpcs + .entry(key) + .or_insert_with(Vec::new) + .push(x.clone()); + } + + // TODO: how should we populate this? + let mut rpc_data = HashMap::with_capacity(self.rpc_heads.len()); + + for (x, x_head) in self.rpc_heads.iter() { + let y = RpcData::new(x, x_head); + + rpc_data.insert(x.clone(), y); + } + let consensus = ConsensusWeb3Rpcs { tier, head_block: maybe_head_block.clone(), best_rpcs: consensus_rpcs, + other_rpcs, backups_needed, + rpc_data, }; return Some(consensus); diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index dd9c3e1c..9d27ad43 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -9,8 +9,10 @@ use crate::frontend::authorization::{Authorization, RequestMetadata}; use crate::frontend::errors::{Web3ProxyError, Web3ProxyResult}; use crate::frontend::rpc_proxy_ws::ProxyMode; use crate::jsonrpc::{JsonRpcForwardedResponse, JsonRpcRequest}; +use crate::rpcs::consensus::{RankedRpcMap, RpcRanking}; use crate::rpcs::transactions::TxStatus; use anyhow::Context; +use arc_swap::ArcSwap; use counter::Counter; use derive_more::From; use ethers::prelude::{ProviderError, TxHash, H256, U64}; @@ -23,16 +25,15 @@ use log::{debug, error, info, trace, warn, Level}; use migration::sea_orm::DatabaseConnection; use moka::future::{Cache, ConcurrentCacheExt}; use ordered_float::OrderedFloat; -use parking_lot::RwLock; use serde::ser::{SerializeStruct, Serializer}; use serde::Serialize; use serde_json::json; use serde_json::value::RawValue; use std::cmp::{min_by_key, Reverse}; use std::collections::BTreeMap; -use std::sync::atomic::{self, Ordering}; +use std::fmt; +use std::sync::atomic::Ordering; use std::sync::Arc; -use std::{cmp, fmt}; use thread_fast_rng::rand::seq::SliceRandom; use tokio; use tokio::sync::{broadcast, watch}; @@ -44,9 +45,8 @@ pub struct Web3Rpcs { /// if watch_consensus_head_sender is some, Web3Rpc inside self will send blocks here when they get them pub(crate) block_sender: flume::Sender<(Option, Arc)>, /// any requests will be forwarded to one (or more) of these connections - /// TODO: i tried to make this an AsyncRwLock, but then we have trouble serializing it - /// TODO: maybe an ArcSwap would be better. writes are rare - pub(crate) by_name: RwLock>>, + pub(crate) by_name: ArcSwap>>, + /// notify all http providers to check their blocks at the same time pub(crate) http_interval_sender: Option>>, /// all providers with the same consensus head block. won't update if there is no `self.watch_consensus_head_sender` /// TODO: document that this is a watch sender and not a broadcast! if things get busy, blocks might get missed @@ -202,11 +202,7 @@ impl Web3Rpcs { let handle = { let connections = connections.clone(); - tokio::spawn(async move { - connections - .subscribe(authorization, block_receiver, pending_tx_sender) - .await - }) + tokio::spawn(connections.subscribe(authorization, block_receiver, pending_tx_sender)) }; Ok((connections, handle, consensus_connections_watcher)) @@ -234,14 +230,13 @@ impl Web3Rpcs { // TODO: will need to think about this more once sum_soft_limit is dynamic let sum_soft_limit = rpc_configs.values().fold(0, |acc, x| acc + x.soft_limit); - // TODO: < is a bit dangerous, we should require a buffer - if sum_soft_limit < self.min_sum_soft_limit { - return Err(anyhow::anyhow!( - "Only {}/{} soft limit! Add more rpcs, increase soft limits, or reduce min_sum_soft_limit.", - sum_soft_limit, - self.min_sum_soft_limit - )); - } + // TODO: require a buffer? + anyhow::ensure!( + sum_soft_limit >= self.min_sum_soft_limit, + "Only {}/{} soft limit! Add more rpcs, increase soft limits, or reduce min_sum_soft_limit.", + sum_soft_limit, + self.min_sum_soft_limit, + ); // turn configs into connections (in parallel) // TODO: move this into a helper function. then we can use it when configs change (will need a remove function too) @@ -270,22 +265,18 @@ impl Web3Rpcs { debug!("spawning {}", server_name); - let handle = tokio::spawn(async move { - server_config - .spawn( - server_name, - db_conn, - vredis_pool, - chain_id, - http_client, - http_interval_sender, - blocks_by_hash, - block_sender, - pending_tx_id_sender, - true, - ) - .await - }); + let handle = tokio::spawn(server_config.spawn( + server_name, + db_conn, + vredis_pool, + chain_id, + http_client, + http_interval_sender, + blocks_by_hash, + block_sender, + pending_tx_id_sender, + true, + )); Some(handle) }) @@ -295,15 +286,21 @@ impl Web3Rpcs { match x { Ok(Ok((rpc, _handle))) => { // web3 connection worked - let old_rpc = self.by_name.write().insert(rpc.name.clone(), rpc.clone()); + let mut new_by_name = (*self.by_name.load_full()).clone(); + + let old_rpc = new_by_name.insert(rpc.name.clone(), rpc.clone()); + + self.by_name.store(Arc::new(new_by_name)); if let Some(old_rpc) = old_rpc { - if old_rpc.head_block.read().is_some() { + if old_rpc.head_block.as_ref().unwrap().borrow().is_some() { + let mut new_head_receiver = + rpc.head_block.as_ref().unwrap().subscribe(); debug!("waiting for new {} to sync", rpc); - // TODO: wait for connection to have a block by watching a channel instead of looping + // TODO: maximum wait time or this could block things for too long - while rpc.head_block.read().is_none() { - sleep(Duration::from_millis(100)).await; + while new_head_receiver.borrow_and_update().is_none() { + new_head_receiver.changed().await?; } } @@ -329,15 +326,15 @@ impl Web3Rpcs { } pub fn get(&self, conn_name: &str) -> Option> { - self.by_name.read().get(conn_name).cloned() + self.by_name.load().get(conn_name).cloned() } pub fn len(&self) -> usize { - self.by_name.read().len() + self.by_name.load().len() } pub fn is_empty(&self) -> bool { - self.by_name.read().is_empty() + self.by_name.load().is_empty() } pub fn min_head_rpcs(&self) -> usize { @@ -506,169 +503,71 @@ impl Web3Rpcs { authorization: &Arc, request_metadata: Option<&Arc>, skip: &[Arc], - // TODO: if we are checking for the consensus head, i don' think we need min_block_needed/max_block_needed min_block_needed: Option<&U64>, max_block_needed: Option<&U64>, ) -> Web3ProxyResult { - let usable_rpcs_by_tier_and_head_number: BTreeMap< - (u64, Reverse>), - Vec>, - > = { - let mut m = BTreeMap::new(); + // TODO: use tracing and add this so logs are easy + let request_ulid = request_metadata.map(|x| &x.request_ulid); - if self.watch_consensus_head_sender.is_none() { - // pick any server + let usable_rpcs_by_tier_and_head_number = { + let mut m: RankedRpcMap = BTreeMap::new(); - let key = (0, Reverse(None)); + if let Some(consensus_rpcs) = self.watch_consensus_rpcs_sender.borrow().as_ref() { + // first place is the blocks that are synced close to head. if those don't work. try all the rpcs. if those don't work, keep trying for a few seconds - for x in self.by_name.read().values() { + let head_block = &consensus_rpcs.head_block; + + let head_block_num = *head_block.number(); + + let best_key = RpcRanking::new( + consensus_rpcs.tier, + consensus_rpcs.backups_needed, + Some(head_block_num), + ); + + // todo: for now, build the map m here. once that works, do as much of it as possible while building ConsensusWeb3Rpcs + for x in consensus_rpcs.best_rpcs.iter().filter(|rpc| { + consensus_rpcs.filter(skip, min_block_needed, max_block_needed, rpc) + }) { + m.entry(best_key).or_insert_with(Vec::new).push(x.clone()); + } + + let tier_offset = consensus_rpcs.tier + 1; + + for (k, v) in consensus_rpcs.other_rpcs.iter() { + let v: Vec<_> = v + .iter() + .filter(|rpc| { + consensus_rpcs.filter(skip, min_block_needed, max_block_needed, rpc) + }) + .cloned() + .collect(); + + let offset_ranking = k.add_offset(tier_offset); + + m.entry(offset_ranking).or_insert_with(Vec::new).extend(v); + } + } else if self.watch_consensus_head_sender.is_none() { + trace!("this Web3Rpcs is not tracking head blocks. pick any server"); + + for x in self.by_name.load().values() { if skip.contains(x) { - trace!("skipping: {}", x); + trace!("{:?} - already skipped. {}", request_ulid, x); continue; } - trace!("not skipped!"); + + let key = RpcRanking::default_with_backup(x.backup); m.entry(key).or_insert_with(Vec::new).push(x.clone()); } - } else { - let synced_connections = self.watch_consensus_rpcs_sender.borrow().clone(); - - if synced_connections.is_none() { - return Ok(OpenRequestResult::NotReady); - } - let synced_connections = - synced_connections.expect("synced_connections can't be None here"); - - let head_block_num = synced_connections.head_block.number(); - let head_block_age = synced_connections.head_block.age(); - - // TODO: double check the logic on this. especially if only min is set - let needed_blocks_comparison = match (min_block_needed, max_block_needed) { - (None, None) => { - // no required block given. treat this like they requested the consensus head block - cmp::Ordering::Equal - } - (None, Some(max_block_needed)) => max_block_needed.cmp(head_block_num), - (Some(min_block_needed), None) => min_block_needed.cmp(head_block_num), - (Some(min_block_needed), Some(max_block_needed)) => { - match min_block_needed.cmp(max_block_needed) { - cmp::Ordering::Less | cmp::Ordering::Equal => { - min_block_needed.cmp(head_block_num) - } - cmp::Ordering::Greater => { - // TODO: force a debug log of the original request to see if our logic is wrong? - // TODO: attach the rpc_key_id so we can find the user to ask if they need help - return Err(Web3ProxyError::InvalidBlockBounds { - min: min_block_needed.as_u64(), - max: max_block_needed.as_u64(), - }); - } - } - } - }; - - trace!("needed_blocks_comparison: {:?}", needed_blocks_comparison); - - // collect "usable_rpcs_by_head_num_and_weight" - match needed_blocks_comparison { - cmp::Ordering::Less => { - // need an old block. check all the rpcs. ignore rpcs that are still syncing - trace!("old block needed"); - - let min_block_age = - self.max_block_age.map(|x| head_block_age.saturating_sub(x)); - let min_sync_num = - self.max_block_lag.map(|x| head_block_num.saturating_sub(x)); - - // TODO: cache this somehow? - // TODO: maybe have a helper on synced_connections? that way sum_soft_limits/min_synced_rpcs will be DRY - for x in self - .by_name - .read() - .values() - .filter(|x| { - // TODO: move a bunch of this onto a rpc.is_synced function - #[allow(clippy::if_same_then_else)] - if skip.contains(x) { - // we've already tried this server or have some other reason to skip it - false - } else if max_block_needed - .map(|max_block_needed| !x.has_block_data(max_block_needed)) - .unwrap_or(false) - { - // server does not have the max block - trace!( - "{} does not have the max block ({:?})", - x, - max_block_needed - ); - false - } else { - !min_block_needed - .map(|min_block_needed| !x.has_block_data(min_block_needed)) - .unwrap_or(false) - } - }) - .cloned() - { - let x_head_block = x.head_block.read().clone(); - - if let Some(x_head) = x_head_block { - // TODO: should nodes that are ahead of the consensus block have priority? seems better to spread the load - let x_head_num = x_head.number().min(head_block_num); - - // TODO: do we really need to check head_num and age? - if let Some(min_sync_num) = min_sync_num.as_ref() { - if x_head_num < min_sync_num { - trace!("rpc is still syncing"); - continue; - } - } - if let Some(min_block_age) = min_block_age { - if x_head.age() > min_block_age { - // rpc is still syncing - trace!("server's block is too old"); - continue; - } - } - - let key = (x.tier, Reverse(Some(*x_head_num))); - - m.entry(key).or_insert_with(Vec::new).push(x); - } - } - - // TODO: check min_synced_rpcs and min_sum_soft_limits? or maybe better to just try to serve the request? - } - cmp::Ordering::Equal => { - // using the consensus head block. filter the synced rpcs - - // the key doesn't matter if we are checking synced connections - // they are all at the same block and it is already sized to what we need - let key = (0, Reverse(None)); - - for x in synced_connections.best_rpcs.iter() { - if skip.contains(x) { - trace!("skipping: {}", x); - continue; - } - trace!("not skipped!"); - - m.entry(key).or_insert_with(Vec::new).push(x.clone()); - } - } - cmp::Ordering::Greater => { - // TODO? if the blocks is close, maybe we could wait for change on a watch_consensus_connections_receiver().subscribe() - return Ok(OpenRequestResult::NotReady); - } - } } m }; trace!( - "usable_rpcs_by_tier_and_head_number: {:#?}", + "{:?} - usable_rpcs_by_tier_and_head_number: {:#?}", + request_ulid, usable_rpcs_by_tier_and_head_number ); @@ -689,16 +588,16 @@ impl Web3Rpcs { // pick the first two and try the one with the lower rpc.latency.ewma // TODO: chunks or tuple windows? for (rpc_a, rpc_b) in usable_rpcs.into_iter().circular_tuple_windows() { - trace!("{} vs {}", rpc_a, rpc_b); + trace!("{:?} - {} vs {}", request_ulid, rpc_a, rpc_b); // TODO: cached key to save a read lock // TODO: ties to the server with the smallest block_data_limit let best_rpc = min_by_key(rpc_a, rpc_b, |x| x.peak_ewma()); - trace!("winner: {}", best_rpc); + trace!("{:?} - winner: {}", request_ulid, best_rpc); // just because it has lower latency doesn't mean we are sure to get a connection match best_rpc.try_request_handle(authorization, None).await { Ok(OpenRequestResult::Handle(handle)) => { - // trace!("opened handle: {}", best_rpc); + trace!("{:?} - opened handle: {}", request_ulid, best_rpc); return Ok(OpenRequestResult::Handle(handle)); } Ok(OpenRequestResult::RetryAt(retry_at)) => { @@ -706,10 +605,15 @@ impl Web3Rpcs { } Ok(OpenRequestResult::NotReady) => { // TODO: log a warning? emit a stat? - trace!("best_rpc not ready: {}", best_rpc); + trace!("{:?} - best_rpc not ready: {}", request_ulid, best_rpc); } Err(err) => { - warn!("No request handle for {}. err={:?}", best_rpc, err) + trace!( + "{:?} - No request handle for {}. err={:?}", + request_ulid, + best_rpc, + err + ) } } } @@ -722,6 +626,7 @@ impl Web3Rpcs { match earliest_retry_at { None => { // none of the servers gave us a time to retry at + debug!("no servers on {:?} gave a retry time", self); // TODO: bring this back? need to think about how to do this with `allow_backups` // we could return an error here, but maybe waiting a second will fix the problem @@ -733,8 +638,6 @@ impl Web3Rpcs { // .await?; // Ok(OpenRequestResult::Handle(handle)) - // TODO: should we log here? - Ok(OpenRequestResult::NotReady) } Some(earliest_retry_at) => { @@ -763,7 +666,7 @@ impl Web3Rpcs { let mut max_count = if let Some(max_count) = max_count { max_count } else { - self.by_name.read().len() + self.by_name.load().len() }; trace!("max_count: {}", max_count); @@ -789,7 +692,7 @@ impl Web3Rpcs { // if there aren't enough synced connections, include more connections // TODO: only do this sorting if the synced_rpcs isn't enough - let mut all_rpcs: Vec<_> = self.by_name.read().values().cloned().collect(); + let mut all_rpcs: Vec<_> = self.by_name.load().values().cloned().collect(); all_rpcs.sort_by_cached_key(rpc_sync_status_sort_key); trace!("all_rpcs: {:#?}", all_rpcs); @@ -814,14 +717,14 @@ impl Web3Rpcs { if let Some(block_needed) = min_block_needed { if !rpc.has_block_data(block_needed) { - warn!("{} is missing min_block_needed. skipping", rpc); + trace!("{} is missing min_block_needed. skipping", rpc); continue; } } if let Some(block_needed) = max_block_needed { if !rpc.has_block_data(block_needed) { - warn!("{} is missing max_block_needed. skipping", rpc); + trace!("{} is missing max_block_needed. skipping", rpc); continue; } } @@ -860,7 +763,7 @@ impl Web3Rpcs { pub async fn try_send_best_consensus_head_connection( &self, authorization: &Arc, - request: JsonRpcRequest, + request: &JsonRpcRequest, request_metadata: Option<&Arc>, min_block_needed: Option<&U64>, max_block_needed: Option<&U64>, @@ -870,17 +773,17 @@ impl Web3Rpcs { let mut watch_consensus_connections = self.watch_consensus_rpcs_sender.subscribe(); - // TODO: maximum retries? right now its the total number of servers - loop { - if skip_rpcs.len() >= self.by_name.read().len() { - break; - } + let start = Instant::now(); + // TODO: get from config + let max_wait = Duration::from_secs(10); + + while start.elapsed() < max_wait { match self .best_available_rpc( authorization, request_metadata, - &skip_rpcs, + &[], min_block_needed, max_block_needed, ) @@ -890,16 +793,15 @@ impl Web3Rpcs { // save the rpc in case we get an error and want to retry on another server // TODO: look at backend_requests instead let rpc = active_request_handle.clone_connection(); - skip_rpcs.push(rpc.clone()); if let Some(request_metadata) = request_metadata { - request_metadata - .response_from_backup_rpc - .store(rpc.backup, Ordering::Release); - - request_metadata.backend_requests.lock().push(rpc); + request_metadata.backend_requests.lock().push(rpc.clone()); } + let is_backup_response = rpc.backup; + + skip_rpcs.push(rpc); + // TODO: get the log percent from the user data let response_result = active_request_handle .request( @@ -915,6 +817,13 @@ impl Web3Rpcs { request.id.clone(), ) { Ok(response) => { + // TODO: if there are multiple responses being aggregated, this will only use the last server's backup type + if let Some(request_metadata) = request_metadata { + request_metadata + .response_from_backup_rpc + .store(is_backup_response, Ordering::Release); + } + if let Some(error) = response.error.as_ref() { // trace!(?response, "rpc error"); @@ -992,11 +901,19 @@ impl Web3Rpcs { // TODO: if we get a TrySendError, reconnect. wait why do we see a trysenderror on a dual provider? shouldn't it be using reqwest - debug!( - "Backend server error on {}! Retrying on another. err={:#?}", - rpc, err + trace!( + "Backend server error on {}! Retrying {:?} on another. err={:?}", + rpc, + request, + err ); + if let Some(ref hard_limit_until) = rpc.hard_limit_until { + let retry_at = Instant::now() + Duration::from_secs(1); + + hard_limit_until.send_replace(retry_at); + } + continue; } } @@ -1033,10 +950,10 @@ impl Web3Rpcs { if watch_for_block(waiting_for, &mut watch_consensus_connections).await? { // block found! continue so we can check for another rpc - continue; } else { - // block won't be found without new servers being added - break; + // rate limits are likely keeping us from serving the head block + watch_consensus_connections.changed().await?; + watch_consensus_connections.borrow_and_update(); } } } @@ -1055,7 +972,7 @@ impl Web3Rpcs { return Ok(r); } - let num_conns = self.by_name.read().len(); + let num_conns = self.by_name.load().len(); let num_skipped = skip_rpcs.len(); let needed = min_block_needed.max(max_block_needed); @@ -1071,13 +988,15 @@ impl Web3Rpcs { "No servers synced (min {:?}, max {:?}, head {:?}) ({} known)", min_block_needed, max_block_needed, head_block_num, num_conns ); - } else if head_block_num > needed.copied() { + } else if head_block_num.as_ref() > needed { // we have synced past the needed block + // TODO: this is likely caused by rate limits. make the error message better error!( "No archive servers synced (min {:?}, max {:?}, head {:?}) ({} known)", min_block_needed, max_block_needed, head_block_num, num_conns ); } else if num_skipped == 0 { + // TODO: what should we log? } else { error!( "Requested data is not available (min {:?}, max {:?}, head {:?}) ({} skipped, {} known)", @@ -1092,7 +1011,7 @@ impl Web3Rpcs { Ok(JsonRpcForwardedResponse::from_str( "Requested data is not available", Some(-32043), - Some(request.id), + Some(request.id.clone()), )) } @@ -1111,7 +1030,12 @@ impl Web3Rpcs { ) -> Web3ProxyResult { let mut watch_consensus_rpcs = self.watch_consensus_rpcs_sender.subscribe(); - loop { + let start = Instant::now(); + + // TODO: get from config + let max_wait = Duration::from_secs(3); + + while start.elapsed() < max_wait { match self .all_connections( authorization, @@ -1123,20 +1047,16 @@ impl Web3Rpcs { .await { Ok(active_request_handles) => { - // TODO: benchmark this compared to waiting on unbounded futures - // TODO: do something with this handle? - // TODO: this is not working right. simplify - if let Some(request_metadata) = request_metadata { - let mut backup_used = false; + let mut only_backups_used = true; request_metadata.backend_requests.lock().extend( active_request_handles.iter().map(|x| { let rpc = x.clone_connection(); - if rpc.backup { + if !rpc.backup { // TODO: its possible we serve from a synced connection though. think about this more - backup_used = true; + only_backups_used = false; } x.clone_connection() @@ -1145,7 +1065,7 @@ impl Web3Rpcs { request_metadata .response_from_backup_rpc - .store(true, Ordering::Release); + .store(only_backups_used, Ordering::Release); } return self @@ -1195,12 +1115,14 @@ impl Web3Rpcs { } } } + + Err(Web3ProxyError::NoServersSynced) } pub async fn try_proxy_connection( &self, authorization: &Arc, - request: JsonRpcRequest, + request: &JsonRpcRequest, request_metadata: Option<&Arc>, min_block_needed: Option<&U64>, max_block_needed: Option<&U64>, @@ -1239,14 +1161,14 @@ impl Serialize for Web3Rpcs { let mut state = serializer.serialize_struct("Web3Rpcs", 6)?; { - let by_name = self.by_name.read(); + let by_name = self.by_name.load(); let rpcs: Vec<&Web3Rpc> = by_name.values().map(|x| x.as_ref()).collect(); // TODO: coordinate with frontend team to rename "conns" to "rpcs" state.serialize_field("conns", &rpcs)?; } { - let consensus_rpcs = self.watch_consensus_rpcs_sender.borrow(); + let consensus_rpcs = self.watch_consensus_rpcs_sender.borrow().clone(); // TODO: rename synced_connections to consensus_rpcs if let Some(consensus_rpcs) = consensus_rpcs.as_ref() { @@ -1273,25 +1195,13 @@ impl Serialize for Web3Rpcs { fn rpc_sync_status_sort_key(x: &Arc) -> (Reverse, u64, bool, OrderedFloat) { let head_block = x .head_block - .read() .as_ref() - .map(|x| *x.number()) + .and_then(|x| x.borrow().as_ref().map(|x| *x.number())) .unwrap_or_default(); let tier = x.tier; - // TODO: use request latency instead of head latency - // TODO: have the latency decay automatically - let peak_latency = x - .peak_latency - .as_ref() - .expect("peak_latency uniniialized") - .latency(); - - let active_requests = x.active_requests.load(atomic::Ordering::Relaxed) as f64; - - // TODO: i'm not sure head * active is exactly right. but we'll see - let peak_ewma = OrderedFloat(peak_latency.as_millis() as f64 * (active_requests + 1.0)); + let peak_ewma = x.peak_ewma(); let backup = x.backup; @@ -1306,7 +1216,7 @@ mod tests { use super::*; use crate::rpcs::consensus::ConsensusFinder; use crate::rpcs::{blockchain::Web3ProxyBlock, provider::Web3Provider}; - + use arc_swap::ArcSwap; use ethers::types::{Block, U256}; use latency::PeakEwmaLatency; use log::{trace, LevelFilter}; @@ -1344,46 +1254,53 @@ mod tests { .map(|x| Web3ProxyBlock::try_new(Arc::new(x)).unwrap()) .collect(); + let (tx_a, _) = watch::channel(None); + let (tx_b, _) = watch::channel(blocks.get(1).cloned()); + let (tx_c, _) = watch::channel(blocks.get(2).cloned()); + let (tx_d, _) = watch::channel(None); + let (tx_e, _) = watch::channel(blocks.get(1).cloned()); + let (tx_f, _) = watch::channel(blocks.get(2).cloned()); + let mut rpcs: Vec<_> = [ Web3Rpc { name: "a".to_string(), tier: 0, - head_block: RwLock::new(None), + head_block: Some(tx_a), peak_latency: Some(new_peak_latency()), ..Default::default() }, Web3Rpc { name: "b".to_string(), tier: 0, - head_block: RwLock::new(blocks.get(1).cloned()), + head_block: Some(tx_b), peak_latency: Some(new_peak_latency()), ..Default::default() }, Web3Rpc { name: "c".to_string(), tier: 0, - head_block: RwLock::new(blocks.get(2).cloned()), + head_block: Some(tx_c), peak_latency: Some(new_peak_latency()), ..Default::default() }, Web3Rpc { name: "d".to_string(), tier: 1, - head_block: RwLock::new(None), + head_block: Some(tx_d), peak_latency: Some(new_peak_latency()), ..Default::default() }, Web3Rpc { name: "e".to_string(), tier: 1, - head_block: RwLock::new(blocks.get(1).cloned()), + head_block: Some(tx_e), peak_latency: Some(new_peak_latency()), ..Default::default() }, Web3Rpc { name: "f".to_string(), tier: 1, - head_block: RwLock::new(blocks.get(2).cloned()), + head_block: Some(tx_f), peak_latency: Some(new_peak_latency()), ..Default::default() }, @@ -1428,11 +1345,10 @@ mod tests { let lagged_block = Arc::new(lagged_block); let head_block = Arc::new(head_block); - let mut lagged_block: Web3ProxyBlock = lagged_block.try_into().unwrap(); - let mut head_block: Web3ProxyBlock = head_block.try_into().unwrap(); - let block_data_limit = u64::MAX; + let (tx_synced, _) = watch::channel(None); + let head_rpc = Web3Rpc { name: "synced".to_string(), soft_limit: 1_000, @@ -1440,12 +1356,14 @@ mod tests { backup: false, block_data_limit: block_data_limit.into(), tier: 0, - head_block: RwLock::new(Some(head_block.clone())), + head_block: Some(tx_synced), provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), peak_latency: Some(new_peak_latency()), ..Default::default() }; + let (tx_lagged, _) = watch::channel(None); + let lagged_rpc = Web3Rpc { name: "lagged".to_string(), soft_limit: 1_000, @@ -1453,17 +1371,17 @@ mod tests { backup: false, block_data_limit: block_data_limit.into(), tier: 0, - head_block: RwLock::new(Some(lagged_block.clone())), + head_block: Some(tx_lagged), provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), peak_latency: Some(new_peak_latency()), ..Default::default() }; - assert!(head_rpc.has_block_data(lagged_block.number())); - assert!(head_rpc.has_block_data(head_block.number())); + assert!(!head_rpc.has_block_data(lagged_block.number.as_ref().unwrap())); + assert!(!head_rpc.has_block_data(head_block.number.as_ref().unwrap())); - assert!(lagged_rpc.has_block_data(lagged_block.number())); - assert!(!lagged_rpc.has_block_data(head_block.number())); + assert!(!lagged_rpc.has_block_data(lagged_block.number.as_ref().unwrap())); + assert!(!lagged_rpc.has_block_data(head_block.number.as_ref().unwrap())); let head_rpc = Arc::new(head_rpc); let lagged_rpc = Arc::new(lagged_rpc); @@ -1475,15 +1393,13 @@ mod tests { let (block_sender, _block_receiver) = flume::unbounded(); let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded(); - let (watch_consensus_rpcs_sender, _watch_consensus_rpcs_receiver) = - watch::channel(Default::default()); - let (watch_consensus_head_sender, _watch_consensus_head_receiver) = - watch::channel(Default::default()); + let (watch_consensus_rpcs_sender, _watch_consensus_rpcs_receiver) = watch::channel(None); + let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None); // TODO: make a Web3Rpcs::new let rpcs = Web3Rpcs { - block_sender, - by_name: RwLock::new(rpcs_by_name), + block_sender: block_sender.clone(), + by_name: ArcSwap::from_pointee(rpcs_by_name), http_interval_sender: None, watch_consensus_head_sender: Some(watch_consensus_head_sender), watch_consensus_rpcs_sender, @@ -1539,9 +1455,15 @@ mod tests { 2 ); - // best_synced_backend_connection requires servers to be synced with the head block + // best_synced_backend_connection which servers to be synced with the head block should not find any nodes let x = rpcs - .best_available_rpc(&authorization, None, &[], None, None) + .best_available_rpc( + &authorization, + None, + &[], + Some(head_block.number.as_ref().unwrap()), + None, + ) .await .unwrap(); @@ -1550,37 +1472,74 @@ mod tests { assert!(matches!(x, OpenRequestResult::NotReady)); // add lagged blocks to the rpcs. both servers should be allowed - lagged_block = rpcs.try_cache_block(lagged_block, true).await.unwrap(); + lagged_rpc + .send_head_block_result( + Ok(Some(lagged_block.clone())), + &block_sender, + rpcs.blocks_by_hash.clone(), + ) + .await + .unwrap(); + // TODO: this is fragile rpcs.process_block_from_rpc( &authorization, &mut consensus_finder, - Some(lagged_block.clone()), - lagged_rpc, + Some(lagged_block.clone().try_into().unwrap()), + lagged_rpc.clone(), &None, ) .await .unwrap(); + + head_rpc + .send_head_block_result( + Ok(Some(lagged_block.clone())), + &block_sender, + rpcs.blocks_by_hash.clone(), + ) + .await + .unwrap(); + + // TODO: this is fragile rpcs.process_block_from_rpc( &authorization, &mut consensus_finder, - Some(lagged_block.clone()), + Some(lagged_block.clone().try_into().unwrap()), head_rpc.clone(), &None, ) .await .unwrap(); + // TODO: how do we spawn this and wait for it to process things? subscribe and watch consensus connections? + // rpcs.process_incoming_blocks(&authorization, block_receiver, pending_tx_sender) + + assert!(head_rpc.has_block_data(lagged_block.number.as_ref().unwrap())); + assert!(!head_rpc.has_block_data(head_block.number.as_ref().unwrap())); + + assert!(lagged_rpc.has_block_data(lagged_block.number.as_ref().unwrap())); + assert!(!lagged_rpc.has_block_data(head_block.number.as_ref().unwrap())); + + // todo!("this doesn't work anymore. send_head_block_result doesn't do anything when rpcs isn't watching the block_receiver") assert_eq!(rpcs.num_synced_rpcs(), 2); // add head block to the rpcs. lagged_rpc should not be available - head_block = rpcs.try_cache_block(head_block, true).await.unwrap(); + head_rpc + .send_head_block_result( + Ok(Some(head_block.clone())), + &block_sender, + rpcs.blocks_by_hash.clone(), + ) + .await + .unwrap(); + // TODO: this is fragile rpcs.process_block_from_rpc( &authorization, &mut consensus_finder, - Some(head_block.clone()), - head_rpc, + Some(head_block.clone().try_into().unwrap()), + head_rpc.clone(), &None, ) .await @@ -1588,18 +1547,27 @@ mod tests { assert_eq!(rpcs.num_synced_rpcs(), 1); + assert!(head_rpc.has_block_data(lagged_block.number.as_ref().unwrap())); + assert!(head_rpc.has_block_data(head_block.number.as_ref().unwrap())); + + assert!(lagged_rpc.has_block_data(lagged_block.number.as_ref().unwrap())); + assert!(!lagged_rpc.has_block_data(head_block.number.as_ref().unwrap())); + + // TODO: make sure the handle is for the expected rpc assert!(matches!( rpcs.best_available_rpc(&authorization, None, &[], None, None) .await, Ok(OpenRequestResult::Handle(_)) )); + // TODO: make sure the handle is for the expected rpc assert!(matches!( rpcs.best_available_rpc(&authorization, None, &[], Some(&0.into()), None) .await, Ok(OpenRequestResult::Handle(_)) )); + // TODO: make sure the handle is for the expected rpc assert!(matches!( rpcs.best_available_rpc(&authorization, None, &[], Some(&1.into()), None) .await, @@ -1634,6 +1602,8 @@ mod tests { let head_block: Web3ProxyBlock = Arc::new(head_block).try_into().unwrap(); + let (tx_pruned, _) = watch::channel(Some(head_block.clone())); + let pruned_rpc = Web3Rpc { name: "pruned".to_string(), soft_limit: 3_000, @@ -1641,11 +1611,13 @@ mod tests { backup: false, block_data_limit: 64.into(), tier: 1, - head_block: RwLock::new(Some(head_block.clone())), + head_block: Some(tx_pruned), provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), ..Default::default() }; + let (tx_archive, _) = watch::channel(Some(head_block.clone())); + let archive_rpc = Web3Rpc { name: "archive".to_string(), soft_limit: 1_000, @@ -1653,7 +1625,7 @@ mod tests { backup: false, block_data_limit: u64::MAX.into(), tier: 2, - head_block: RwLock::new(Some(head_block.clone())), + head_block: Some(tx_archive), provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), ..Default::default() }; @@ -1673,14 +1645,13 @@ mod tests { let (block_sender, _) = flume::unbounded(); let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded(); - let (watch_consensus_rpcs_sender, _) = watch::channel(Default::default()); - let (watch_consensus_head_sender, _watch_consensus_head_receiver) = - watch::channel(Default::default()); + let (watch_consensus_rpcs_sender, _) = watch::channel(None); + let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None); // TODO: make a Web3Rpcs::new let rpcs = Web3Rpcs { block_sender, - by_name: RwLock::new(rpcs_by_name), + by_name: ArcSwap::from_pointee(rpcs_by_name), http_interval_sender: None, watch_consensus_head_sender: Some(watch_consensus_head_sender), watch_consensus_rpcs_sender, @@ -1791,6 +1762,9 @@ mod tests { let block_1: Web3ProxyBlock = Arc::new(block_1).try_into().unwrap(); let block_2: Web3ProxyBlock = Arc::new(block_2).try_into().unwrap(); + let (tx_mock_geth, _) = watch::channel(Some(block_1.clone())); + let (tx_mock_erigon_archive, _) = watch::channel(Some(block_2.clone())); + let mock_geth = Web3Rpc { name: "mock_geth".to_string(), soft_limit: 1_000, @@ -1798,7 +1772,7 @@ mod tests { backup: false, block_data_limit: 64.into(), tier: 0, - head_block: RwLock::new(Some(block_1.clone())), + head_block: Some(tx_mock_geth), provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), peak_latency: Some(new_peak_latency()), ..Default::default() @@ -1811,7 +1785,7 @@ mod tests { backup: false, block_data_limit: u64::MAX.into(), tier: 1, - head_block: RwLock::new(Some(block_2.clone())), + head_block: Some(tx_mock_erigon_archive), provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), peak_latency: Some(new_peak_latency()), ..Default::default() @@ -1835,14 +1809,13 @@ mod tests { let (block_sender, _) = flume::unbounded(); let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded(); - let (watch_consensus_rpcs_sender, _) = watch::channel(Default::default()); - let (watch_consensus_head_sender, _watch_consensus_head_receiver) = - watch::channel(Default::default()); + let (watch_consensus_rpcs_sender, _) = watch::channel(None); + let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None); // TODO: make a Web3Rpcs::new let rpcs = Web3Rpcs { block_sender, - by_name: RwLock::new(rpcs_by_name), + by_name: ArcSwap::from_pointee(rpcs_by_name), http_interval_sender: None, watch_consensus_head_sender: Some(watch_consensus_head_sender), watch_consensus_rpcs_sender, @@ -1943,6 +1916,8 @@ async fn watch_for_block( // we are past this block and no servers have this block // this happens if the block is old and all archive servers are offline // there is no chance we will get this block without adding an archive server to the config + + // TODO: i think this can also happen if we are being rate limited! return Ok(false); } } @@ -1976,3 +1951,23 @@ async fn watch_for_block( Ok(true) } + +#[cfg(test)] +mod test { + use std::cmp::Reverse; + + #[test] + fn test_block_num_sort() { + let test_vec = vec![ + Reverse(Some(3)), + Reverse(Some(2)), + Reverse(Some(1)), + Reverse(None), + ]; + + let mut sorted_vec = test_vec.clone(); + sorted_vec.sort(); + + assert_eq!(test_vec, sorted_vec); + } +} diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index 71b67e88..c541c0da 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -46,15 +46,15 @@ pub struct Web3Rpc { /// provider is in a RwLock so that we can replace it if re-connecting /// it is an async lock because we hold it open across awaits /// this provider is only used for new heads subscriptions - /// TODO: watch channel instead of a lock - /// TODO: is this only used for new heads subscriptions? if so, rename + /// TODO: benchmark ArcSwapOption and a watch::Sender pub(super) provider: AsyncRwLock>>, - /// keep track of hard limits. Optional because we skip this code for our own servers. + /// keep track of hard limits + /// this is only inside an Option so that the "Default" derive works. it will always be set. pub(super) hard_limit_until: Option>, /// rate limits are stored in a central redis so that multiple proxies can share their rate limits /// We do not use the deferred rate limiter because going over limits would cause errors pub(super) hard_limit: Option, - /// used for load balancing to the least loaded server + /// used for ensuring enough requests are available before advancing the head block pub(super) soft_limit: u32, /// use web3 queries to find the block data limit for archive/pruned nodes pub(super) automatic_block_limit: bool, @@ -65,7 +65,8 @@ pub struct Web3Rpc { /// Lower tiers are higher priority when sending requests pub(super) tier: u64, /// TODO: change this to a watch channel so that http providers can subscribe and take action on change. - pub(super) head_block: RwLock>, + /// this is only inside an Option so that the "Default" derive works. it will always be set. + pub(super) head_block: Option>>, /// Track head block latency pub(super) head_latency: RwLock, /// Track peak request latency @@ -96,8 +97,6 @@ impl Web3Rpc { // TODO: rename to http_new_head_interval_sender? http_interval_sender: Option>>, redis_pool: Option, - // TODO: think more about soft limit. watching ewma of requests is probably better. but what should the random sort be on? maybe group on tier is enough - // soft_limit: u32, block_map: BlocksByHashCache, block_sender: Option>, tx_id_sender: Option)>>, @@ -139,15 +138,9 @@ impl Web3Rpc { let automatic_block_limit = (block_data_limit.load(atomic::Ordering::Acquire) == 0) && block_sender.is_some(); - // track hard limit until on backup servers (which might surprise us with rate limit changes) + // have a sender for tracking hard limit anywhere. we use this in case we // and track on servers that have a configured hard limit - let hard_limit_until = if backup || hard_limit.is_some() { - let (sender, _) = watch::channel(Instant::now()); - - Some(sender) - } else { - None - }; + let (hard_limit_until, _) = watch::channel(Instant::now()); if config.ws_url.is_none() && config.http_url.is_none() { if let Some(url) = config.url { @@ -168,6 +161,8 @@ impl Web3Rpc { let (disconnect_sender, disconnect_receiver) = watch::channel(false); let reconnect = reconnect.into(); + let (head_block, _) = watch::channel(None); + // Spawn the task for calculting average peak latency // TODO Should these defaults be in config let peak_latency = PeakEwmaLatency::spawn( @@ -200,7 +195,7 @@ impl Web3Rpc { ws_url, http_url, hard_limit, - hard_limit_until, + hard_limit_until: Some(hard_limit_until), soft_limit: config.soft_limit, automatic_block_limit, backup, @@ -209,7 +204,7 @@ impl Web3Rpc { tier: config.tier, disconnect_watch: Some(disconnect_sender), created_at: Some(created_at), - head_block: RwLock::new(Default::default()), + head_block: Some(head_block), peak_latency: Some(peak_latency), ..Default::default() }; @@ -352,8 +347,9 @@ impl Web3Rpc { self.block_data_limit.load(atomic::Ordering::Acquire).into() } + /// TODO: get rid of this now that consensus rpcs does it pub fn has_block_data(&self, needed_block_num: &U64) -> bool { - let head_block_num = match self.head_block.read().as_ref() { + let head_block_num = match self.head_block.as_ref().unwrap().borrow().as_ref() { None => return false, Some(x) => *x.number(), }; @@ -483,8 +479,10 @@ impl Web3Rpc { } // reset sync status - let mut head_block = self.head_block.write(); - *head_block = None; + self.head_block + .as_ref() + .expect("head_block should always be set") + .send_replace(None); // disconnect the current provider // TODO: what until the block_sender's receiver finishes updating this item? @@ -587,7 +585,7 @@ impl Web3Rpc { Ok(()) } - async fn send_head_block_result( + pub(crate) async fn send_head_block_result( self: &Arc, new_head_block: Result, ProviderError>, block_sender: &flume::Sender, @@ -596,9 +594,9 @@ impl Web3Rpc { let new_head_block = match new_head_block { Ok(None) => { { - let mut head_block = self.head_block.write(); + let head_block_tx = self.head_block.as_ref().unwrap(); - if head_block.is_none() { + if head_block_tx.borrow().is_none() { // we previously sent a None. return early return Ok(()); } @@ -607,7 +605,7 @@ impl Web3Rpc { debug!("clearing head block on {} ({}ms old)!", self, age); - *head_block = None; + head_block_tx.send_replace(None); } None @@ -625,11 +623,10 @@ impl Web3Rpc { // save the block so we don't send the same one multiple times // also save so that archive checks can know how far back to query - { - let mut head_block = self.head_block.write(); - - let _ = head_block.insert(new_head_block.clone()); - } + self.head_block + .as_ref() + .unwrap() + .send_replace(Some(new_head_block.clone())); if self.block_data_limit() == U64::zero() { let authorization = Arc::new(Authorization::internal(self.db_conn.clone())?); @@ -646,11 +643,7 @@ impl Web3Rpc { Err(err) => { warn!("unable to get block from {}. err={:?}", self, err); - { - let mut head_block = self.head_block.write(); - - *head_block = None; - } + self.head_block.as_ref().unwrap().send_replace(None); None } @@ -750,7 +743,7 @@ impl Web3Rpc { if new_total_requests - old_total_requests < 10 { // TODO: if this fails too many times, reset the connection // TODO: move this into a function and the chaining should be easier - let head_block = rpc.head_block.read().clone(); + let head_block = rpc.head_block.as_ref().unwrap().borrow().clone(); if let Some((block_number, txid)) = head_block.and_then(|x| { let block = x.block; @@ -947,16 +940,25 @@ impl Web3Rpc { .await?; } Ok(Some(block)) => { - // don't send repeat blocks - let new_hash = - block.hash.expect("blocks here should always have hashes"); + if let Some(new_hash) = block.hash { + // don't send repeat blocks + if new_hash != last_hash { + // new hash! + last_hash = new_hash; - if new_hash != last_hash { - // new hash! - last_hash = new_hash; + self.send_head_block_result( + Ok(Some(block)), + &block_sender, + block_map.clone(), + ) + .await?; + } + } else { + // TODO: why is this happening? + warn!("empty head block on {}", self); self.send_head_block_result( - Ok(Some(block)), + Ok(None), &block_sender, block_map.clone(), ) @@ -1387,7 +1389,12 @@ impl Serialize for Web3Rpc { state.serialize_field("soft_limit", &self.soft_limit)?; // TODO: maybe this is too much data. serialize less? - state.serialize_field("head_block", &*self.head_block.read())?; + { + let head_block = self.head_block.as_ref().unwrap(); + let head_block = head_block.borrow(); + let head_block = head_block.as_ref(); + state.serialize_field("head_block", &head_block)?; + } state.serialize_field("head_latency", &self.head_latency.read().value())?; @@ -1445,6 +1452,8 @@ mod tests { let head_block = Web3ProxyBlock::try_new(random_block).unwrap(); let block_data_limit = u64::MAX; + let (tx, _) = watch::channel(Some(head_block.clone())); + let x = Web3Rpc { name: "name".to_string(), ws_url: Some("ws://example.com".parse::().unwrap()), @@ -1453,7 +1462,7 @@ mod tests { backup: false, block_data_limit: block_data_limit.into(), tier: 0, - head_block: RwLock::new(Some(head_block.clone())), + head_block: Some(tx), ..Default::default() }; @@ -1479,6 +1488,8 @@ mod tests { let block_data_limit = 64; + let (tx, _rx) = watch::channel(Some(head_block.clone())); + let x = Web3Rpc { name: "name".to_string(), soft_limit: 1_000, @@ -1486,7 +1497,7 @@ mod tests { backup: false, block_data_limit: block_data_limit.into(), tier: 0, - head_block: RwLock::new(Some(head_block.clone())), + head_block: Some(tx), ..Default::default() }; diff --git a/web3_proxy/src/stats/db_queries.rs b/web3_proxy/src/stats/db_queries.rs index ccc9404a..8830cbf9 100644 --- a/web3_proxy/src/stats/db_queries.rs +++ b/web3_proxy/src/stats/db_queries.rs @@ -273,6 +273,7 @@ pub async fn query_user_stats<'a>( .expect("max-age should always parse"), ); + // TODO: get this from `response` isntead of json serializing twice let cache_body = json!(response_body).to_string(); if let Err(err) = redis_conn diff --git a/web3_proxy/src/stats/influxdb_queries.rs b/web3_proxy/src/stats/influxdb_queries.rs index d946dc2d..29747c65 100644 --- a/web3_proxy/src/stats/influxdb_queries.rs +++ b/web3_proxy/src/stats/influxdb_queries.rs @@ -125,7 +125,7 @@ pub async fn query_user_stats<'a>( user_rpc_keys.append(&mut subuser_rpc_keys); - if user_rpc_keys.len() == 0 { + if user_rpc_keys.is_empty() { return Err(Web3ProxyError::BadRequest( "User has no secret RPC keys yet".to_string(), )); diff --git a/web3_proxy/src/stats/mod.rs b/web3_proxy/src/stats/mod.rs index af148e7d..930f9e04 100644 --- a/web3_proxy/src/stats/mod.rs +++ b/web3_proxy/src/stats/mod.rs @@ -2,36 +2,36 @@ //! TODO: move some of these structs/functions into their own file? pub mod db_queries; pub mod influxdb_queries; -use crate::app::AuthorizationChecks; +mod stat_buffer; + +pub use stat_buffer::{SpawnedStatBuffer, StatBuffer}; + +use crate::app::RpcSecretKeyCache; use crate::frontend::authorization::{Authorization, RequestMetadata}; +use crate::frontend::errors::Web3ProxyError; +use crate::rpcs::one::Web3Rpc; use anyhow::Context; use axum::headers::Origin; use chrono::{DateTime, Months, TimeZone, Utc}; use derive_more::From; use entities::sea_orm_active_enums::TrackingLevel; use entities::{balance, referee, referrer, rpc_accounting_v2, rpc_key, user, user_tier}; -use futures::stream; -use hashbrown::HashMap; -use influxdb2::api::write::TimestampPrecision; use influxdb2::models::DataPoint; -use log::{error, info, trace, warn}; +use log::{trace, warn}; use migration::sea_orm::prelude::Decimal; -use migration::sea_orm::ActiveModelTrait; -use migration::sea_orm::ColumnTrait; -use migration::sea_orm::IntoActiveModel; -use migration::sea_orm::{self, DatabaseConnection, EntityTrait, QueryFilter}; +use migration::sea_orm::{ + self, ActiveModelTrait, ColumnTrait, DatabaseConnection, EntityTrait, IntoActiveModel, + QueryFilter, +}; use migration::{Expr, OnConflict}; -use moka::future::Cache; use num_traits::ToPrimitive; +use parking_lot::Mutex; use std::cmp::max; use std::num::NonZeroU64; -use std::sync::atomic::Ordering; +use std::sync::atomic::{self, Ordering}; use std::sync::Arc; -use std::time::Duration; -use tokio::sync::broadcast; -use tokio::task::JoinHandle; -use tokio::time::interval; -use ulid::Ulid; + +use self::stat_buffer::BufferedRpcQueryStats; #[derive(Debug, PartialEq, Eq)] pub enum StatType { @@ -39,8 +39,9 @@ pub enum StatType { Detailed, } -// Pub is needed for migration ... I could also write a second constructor for this if needed -/// TODO: better name? +pub type BackendRequests = Mutex>>; + +/// TODO: better name? RpcQueryStatBuilder? #[derive(Clone, Debug)] pub struct RpcQueryStats { pub authorization: Arc, @@ -49,8 +50,8 @@ pub struct RpcQueryStats { pub error_response: bool, pub request_bytes: u64, /// if backend_requests is 0, there was a cache_hit - // pub frontend_request: u64, - pub backend_requests: u64, + /// no need to track frontend_request on this. a RpcQueryStats always represents one frontend request + pub backend_rpcs_used: Vec>, pub response_bytes: u64, pub response_millis: u64, pub response_timestamp: i64, @@ -58,7 +59,7 @@ pub struct RpcQueryStats { pub credits_used: Decimal, } -#[derive(Clone, From, Hash, PartialEq, Eq)] +#[derive(Clone, Debug, From, Hash, PartialEq, Eq)] pub struct RpcQueryKey { /// unix epoch time /// for the time series db, this is (close to) the time that the response was sent @@ -181,22 +182,6 @@ impl RpcQueryStats { } } -#[derive(Default)] -pub struct BufferedRpcQueryStats { - pub frontend_requests: u64, - pub backend_requests: u64, - pub backend_retries: u64, - pub no_servers: u64, - pub cache_misses: u64, - pub cache_hits: u64, - pub sum_request_bytes: u64, - pub sum_response_bytes: u64, - pub sum_response_millis: u64, - pub sum_credits_used: Decimal, - /// Balance tells us the user's balance at this point in time - pub latest_balance: Decimal, -} - /// A stat that we aggregate and then store in a database. /// For now there is just one, but I think there might be others later #[derive(Debug, From)] @@ -204,34 +189,16 @@ pub enum AppStat { RpcQuery(RpcQueryStats), } -#[derive(From)] -pub struct SpawnedStatBuffer { - pub stat_sender: flume::Sender, - /// these handles are important and must be allowed to finish - pub background_handle: JoinHandle>, -} - -pub struct StatBuffer { - chain_id: u64, - db_conn: Option, - influxdb_client: Option, - tsdb_save_interval_seconds: u32, - rpc_secret_key_cache: - Option>, - db_save_interval_seconds: u32, - billing_period_seconds: i64, - global_timeseries_buffer: HashMap, - opt_in_timeseries_buffer: HashMap, - accounting_db_buffer: HashMap, - timestamp_precision: TimestampPrecision, -} - +// TODO: move to stat_buffer.rs? impl BufferedRpcQueryStats { fn add(&mut self, stat: RpcQueryStats) { // a stat always come from just 1 frontend request self.frontend_requests += 1; - if stat.backend_requests == 0 { + // TODO: is this always okay? is it true that each backend rpc will only be queried once per request? i think so + let num_backend_rpcs_used = stat.backend_rpcs_used.len() as u64; + + if num_backend_rpcs_used == 0 { // no backend request. cache hit! self.cache_hits += 1; } else { @@ -239,7 +206,7 @@ impl BufferedRpcQueryStats { self.cache_misses += 1; // a single frontend request might have multiple backend requests - self.backend_requests += stat.backend_requests; + self.backend_requests += num_backend_rpcs_used; } self.sum_request_bytes += stat.request_bytes; @@ -261,13 +228,21 @@ impl BufferedRpcQueryStats { chain_id: u64, db_conn: &DatabaseConnection, key: RpcQueryKey, + rpc_secret_key_cache: Option<&RpcSecretKeyCache>, ) -> anyhow::Result<()> { + anyhow::ensure!( + key.response_timestamp > 0, + "no response_timestamp! This is a bug! {:?} {:?}", + key, + self + ); + let period_datetime = Utc.timestamp_opt(key.response_timestamp, 0).unwrap(); // this is a lot of variables let accounting_entry = rpc_accounting_v2::ActiveModel { id: sea_orm::NotSet, - rpc_key_id: sea_orm::Set(key.rpc_secret_key_id.map(Into::into).unwrap_or_default()), + rpc_key_id: sea_orm::Set(key.rpc_secret_key_id.map(Into::into)), chain_id: sea_orm::Set(chain_id), period_datetime: sea_orm::Set(period_datetime), archive_needed: sea_orm::Set(key.archive_needed), @@ -360,7 +335,7 @@ impl BufferedRpcQueryStats { // Technicall there should always be a user ... still let's return "Ok(())" for now let sender_user_id: u64 = match sender_rpc_key { - Some(x) => x.user_id.into(), + Some(x) => x.user_id, // Return early if the User is not found, because then it is an anonymous user // Let's also issue a warning because obviously the RPC key should correspond to a user None => { @@ -390,12 +365,14 @@ impl BufferedRpcQueryStats { // Still subtract from the user in any case, // Modify the balance of the sender completely (in mysql, next to the stats) // In any case, add this to "spent" + // TODO! we need to do the math in mysql (like with `Expr::col` above). if we do the addition here, there is a race condition active_sender_balance.used_balance = - sea_orm::Set(sender_balance.used_balance + Decimal::from(self.sum_credits_used)); + sea_orm::Set(sender_balance.used_balance + self.sum_credits_used); // Also update the available balance + // TODO! this needs to be queried from the database let new_available_balance = max( - sender_balance.available_balance - Decimal::from(self.sum_credits_used), + sender_balance.available_balance - self.sum_credits_used, Decimal::from(0), ); active_sender_balance.available_balance = sea_orm::Set(new_available_balance); @@ -424,15 +401,26 @@ impl BufferedRpcQueryStats { ))?; // Downgrade a user to premium - out of funds if there's less than 10$ in the account, and if the user was premium before + // TODO: lets let them get under $1 + // TODO: instead of checking for a specific title, downgrade if the downgrade id is set to anything if new_available_balance < Decimal::from(10u64) && downgrade_user_role.title == "Premium" { + // TODO: we could do this outside the balance low block, but I think its fine. or better, update the cache if <$10 and downgrade if <$1 + if let Some(rpc_secret_key_cache) = rpc_secret_key_cache { + todo!("expire (or probably better to update) the user cache now that the balance is low"); + // actually i think we need to have 2 caches. otherwise users with 2 keys are going to have seperate caches + // 1. rpc_secret_key_id -> AuthorizationChecks (cuz we don't want to hit the db every time) + // 2. user_id -> Balance + } + // Only downgrade the user in local process memory, not elsewhere - // app.rpc_secret_key_cache- // let mut active_downgrade_user = downgrade_user.into_active_model(); // active_downgrade_user.user_tier_id = sea_orm::Set(downgrade_user_role.id); // active_downgrade_user.save(db_conn).await?; } + // TODO: + // Get the referee, and the referrer // (2) Look up the code that this user used. This is the referee table let referee_object = match referee::Entity::find() @@ -459,6 +447,7 @@ impl BufferedRpcQueryStats { { Some(x) => x, None => { + // TODO: warn seems too verbose for this. it should be fine for a user to not have a referall code, right? warn!( "No referrer with that referral code was found {:?}", referee_object @@ -487,6 +476,7 @@ impl BufferedRpcQueryStats { } }; + // TODO: don't clone on this. use the active_model later let mut active_sender_balance = sender_balance.clone().into_active_model(); let referrer_balance = match balance::Entity::find() .filter(balance::Column::UserId.eq(user_with_that_referral_code.user_id)) @@ -513,6 +503,7 @@ impl BufferedRpcQueryStats { { // (6) If the credits have not yet been applied to the referee, apply 10M credits / $100.00 USD worth of credits. // Make it into an active model, and add credits + // TODO! race condition here! we can't set. need to let the db do the math active_sender_balance.available_balance = sea_orm::Set(sender_balance.available_balance + Decimal::from(100)); // Also mark referral as "credits_applied_for_referee" @@ -528,8 +519,7 @@ impl BufferedRpcQueryStats { let mut active_referrer_balance = referrer_balance.clone().into_active_model(); // Add 10% referral fees ... active_referrer_balance.available_balance = sea_orm::Set( - referrer_balance.available_balance - + Decimal::from(self.sum_credits_used / Decimal::from(10)), + referrer_balance.available_balance + self.sum_credits_used / Decimal::from(10), ); // Also record how much the current referrer has "provided" / "gifted" away active_referee.credits_applied_for_referrer = @@ -598,51 +588,80 @@ impl BufferedRpcQueryStats { } } -impl RpcQueryStats { - pub fn new( - method: Option, - authorization: Arc, - metadata: Arc, - response_bytes: usize, - ) -> Self { - // TODO: try_unwrap the metadata to be sure that all the stats for this request have been collected - // TODO: otherwise, i think the whole thing should be in a single lock that we can "reset" when a stat is created +impl TryFrom for RpcQueryStats { + type Error = Web3ProxyError; + + fn try_from(mut metadata: RequestMetadata) -> Result { + let mut authorization = metadata.authorization.take(); + + if authorization.is_none() { + authorization = Some(Arc::new(Authorization::internal(None)?)); + } + + let authorization = authorization.expect("Authorization will always be set"); let archive_request = metadata.archive_request.load(Ordering::Acquire); - let backend_requests = metadata.backend_requests.lock().len() as u64; - let request_bytes = metadata.request_bytes; - let error_response = metadata.error_response.load(Ordering::Acquire); - let response_millis = metadata.start_instant.elapsed().as_millis() as u64; - let response_bytes = response_bytes as u64; - // TODO: Gotta make the arithmetic here + // TODO: do this without cloning. we can take their vec + let backend_rpcs_used = metadata.backend_rpcs_used(); + + let request_bytes = metadata.request_bytes as u64; + let response_bytes = metadata.response_bytes.load(Ordering::Acquire); + + let mut error_response = metadata.error_response.load(Ordering::Acquire); + let mut response_millis = metadata.response_millis.load(atomic::Ordering::Acquire); + + let response_timestamp = match metadata.response_timestamp.load(atomic::Ordering::Acquire) { + 0 => { + // no response timestamp! + if !error_response { + // force error_response to true + // this can happen when a try operator escapes and metadata.add_response() isn't called + trace!( + "no response known, but no errors logged. investigate. {:?}", + metadata + ); + error_response = true; + } + + if response_millis == 0 { + // get something for millis even if it is a bit late + response_millis = metadata.start_instant.elapsed().as_millis() as u64 + } + + // no timestamp given. likely handling an error. set it to the current time + Utc::now().timestamp() + } + x => x, + }; + + let method = metadata.method.take(); - // TODO: Depending on the method, metadata and response bytes, pick a different number of credits used - // This can be a slightly more complex function as we ll - // TODO: Here, let's implement the formula let credits_used = Self::compute_cost( request_bytes, response_bytes, - backend_requests == 0, - &method, + backend_rpcs_used.is_empty(), + method.as_deref(), ); - let response_timestamp = Utc::now().timestamp(); - - Self { + let x = Self { authorization, archive_request, method, - backend_requests, + backend_rpcs_used, request_bytes, error_response, response_bytes, response_millis, response_timestamp, credits_used, - } - } + }; + Ok(x) + } +} + +impl RpcQueryStats { /// Compute cost per request /// All methods cost the same /// The number of bytes are based on input, and output bytes @@ -650,251 +669,39 @@ impl RpcQueryStats { request_bytes: u64, response_bytes: u64, cache_hit: bool, - _method: &Option, + method: Option<&str>, ) -> Decimal { - // TODO: Should make these lazy_static const? + // some methods should be free. there might be cases where method isn't set (though they should be uncommon) + // TODO: get this list from config (and add more to it) + if let Some(method) = method.as_ref() { + if ["eth_chainId"].contains(method) { + return 0.into(); + } + } + + // TODO: get cost_minimum, cost_free_bytes, cost_per_byte, cache_hit_divisor from config. each chain will be different // pays at least $0.000018 / credits per request let cost_minimum = Decimal::new(18, 6); + // 1kb is included on each call let cost_free_bytes = 1024; + // after that, we add cost per bytes, $0.000000006 / credits per byte + // amazon charges $.09/GB outbound + // but we also have to cover our RAM and expensive nics on the servers (haproxy/web3-proxy/blockchains) let cost_per_byte = Decimal::new(6, 9); let total_bytes = request_bytes + response_bytes; - let total_chargable_bytes = - Decimal::from(max(0, total_bytes as i64 - cost_free_bytes as i64)); - let out = cost_minimum + cost_per_byte * total_chargable_bytes; + let total_chargable_bytes = Decimal::from(total_bytes.saturating_sub(cost_free_bytes)); + + let mut cost = cost_minimum + cost_per_byte * total_chargable_bytes; + + // cache hits get a 50% discount if cache_hit { - out * Decimal::new(5, 1) - } else { - out + cost /= Decimal::from(2) } - } - /// Only used for migration from stats_v1 to stats_v2/v3 - pub fn modify_struct( - &mut self, - response_millis: u64, - response_timestamp: i64, - backend_requests: u64, - ) { - self.response_millis = response_millis; - self.response_timestamp = response_timestamp; - self.backend_requests = backend_requests; - } -} - -impl StatBuffer { - #[allow(clippy::too_many_arguments)] - pub fn try_spawn( - chain_id: u64, - bucket: String, - db_conn: Option, - influxdb_client: Option, - rpc_secret_key_cache: Option< - Cache, - >, - db_save_interval_seconds: u32, - tsdb_save_interval_seconds: u32, - billing_period_seconds: i64, - shutdown_receiver: broadcast::Receiver<()>, - ) -> anyhow::Result> { - if db_conn.is_none() && influxdb_client.is_none() { - return Ok(None); - } - - let (stat_sender, stat_receiver) = flume::unbounded(); - - let timestamp_precision = TimestampPrecision::Seconds; - let mut new = Self { - chain_id, - db_conn, - influxdb_client, - db_save_interval_seconds, - tsdb_save_interval_seconds, - rpc_secret_key_cache, - billing_period_seconds, - global_timeseries_buffer: Default::default(), - opt_in_timeseries_buffer: Default::default(), - accounting_db_buffer: Default::default(), - timestamp_precision, - }; - - // any errors inside this task will cause the application to exit - let handle = tokio::spawn(async move { - new.aggregate_and_save_loop(bucket, stat_receiver, shutdown_receiver) - .await - }); - - Ok(Some((stat_sender, handle).into())) - } - - async fn aggregate_and_save_loop( - &mut self, - bucket: String, - stat_receiver: flume::Receiver, - mut shutdown_receiver: broadcast::Receiver<()>, - ) -> anyhow::Result<()> { - let mut tsdb_save_interval = - interval(Duration::from_secs(self.tsdb_save_interval_seconds as u64)); - let mut db_save_interval = - interval(Duration::from_secs(self.db_save_interval_seconds as u64)); - - // TODO: Somewhere here we should probably be updating the balance of the user - // And also update the credits used etc. for the referred user - loop { - tokio::select! { - stat = stat_receiver.recv_async() => { - // info!("Received stat"); - // save the stat to a buffer - match stat { - Ok(AppStat::RpcQuery(stat)) => { - if self.influxdb_client.is_some() { - // TODO: round the timestamp at all? - - let global_timeseries_key = stat.global_timeseries_key(); - - self.global_timeseries_buffer.entry(global_timeseries_key).or_default().add(stat.clone()); - - if let Some(opt_in_timeseries_key) = stat.opt_in_timeseries_key() { - self.opt_in_timeseries_buffer.entry(opt_in_timeseries_key).or_default().add(stat.clone()); - } - } - - if self.db_conn.is_some() { - self.accounting_db_buffer.entry(stat.accounting_key(self.billing_period_seconds)).or_default().add(stat); - } - } - Err(err) => { - error!("error receiving stat: {:?}", err); - break; - } - } - } - _ = db_save_interval.tick() => { - // info!("DB save internal tick"); - let count = self.save_relational_stats().await; - if count > 0 { - trace!("Saved {} stats to the relational db", count); - } - } - _ = tsdb_save_interval.tick() => { - // info!("TSDB save internal tick"); - let count = self.save_tsdb_stats(&bucket).await; - if count > 0 { - trace!("Saved {} stats to the tsdb", count); - } - } - x = shutdown_receiver.recv() => { - info!("shutdown signal ---"); - match x { - Ok(_) => { - info!("stat_loop shutting down"); - }, - Err(err) => error!("stat_loop shutdown receiver err={:?}", err), - } - break; - } - } - } - - let saved_relational = self.save_relational_stats().await; - - info!("saved {} pending relational stats", saved_relational); - - let saved_tsdb = self.save_tsdb_stats(&bucket).await; - - info!("saved {} pending tsdb stats", saved_tsdb); - - info!("accounting and stat save loop complete"); - - Ok(()) - } - - async fn save_relational_stats(&mut self) -> usize { - let mut count = 0; - - if let Some(db_conn) = self.db_conn.as_ref() { - count = self.accounting_db_buffer.len(); - for (key, stat) in self.accounting_db_buffer.drain() { - // TODO: batch saves - // TODO: i don't like passing key (which came from the stat) to the function on the stat. but it works for now - if let Err(err) = stat.save_db(self.chain_id, db_conn, key).await { - error!("unable to save accounting entry! err={:?}", err); - }; - } - } - - count - } - - // TODO: bucket should be an enum so that we don't risk typos - async fn save_tsdb_stats(&mut self, bucket: &str) -> usize { - let mut count = 0; - - if let Some(influxdb_client) = self.influxdb_client.as_ref() { - // TODO: use stream::iter properly to avoid allocating this Vec - let mut points = vec![]; - - for (key, stat) in self.global_timeseries_buffer.drain() { - // TODO: i don't like passing key (which came from the stat) to the function on the stat. but it works for now - match stat - .build_timeseries_point("global_proxy", self.chain_id, key) - .await - { - Ok(point) => { - points.push(point); - } - Err(err) => { - error!("unable to build global stat! err={:?}", err); - } - }; - } - - for (key, stat) in self.opt_in_timeseries_buffer.drain() { - // TODO: i don't like passing key (which came from the stat) to the function on the stat. but it works for now - match stat - .build_timeseries_point("opt_in_proxy", self.chain_id, key) - .await - { - Ok(point) => { - points.push(point); - } - Err(err) => { - // TODO: if this errors, we throw away some of the pending stats! we should probably buffer them somewhere to be tried again - error!("unable to build opt-in stat! err={:?}", err); - } - }; - } - - count = points.len(); - - if count > 0 { - // TODO: put max_batch_size in config? - // TODO: i think the real limit is the byte size of the http request. so, a simple line count won't work very well - let max_batch_size = 100; - - let mut num_left = count; - - while num_left > 0 { - let batch_size = num_left.min(max_batch_size); - - let p = points.split_off(batch_size); - - num_left -= batch_size; - - if let Err(err) = influxdb_client - .write_with_precision(bucket, stream::iter(p), self.timestamp_precision) - .await - { - // TODO: if this errors, we throw away some of the pending stats! we should probably buffer them somewhere to be tried again - error!("unable to save {} tsdb stats! err={:?}", batch_size, err); - } - } - } - } - - count + cost } } diff --git a/web3_proxy/src/stats/stat_buffer.rs b/web3_proxy/src/stats/stat_buffer.rs new file mode 100644 index 00000000..aaca71cd --- /dev/null +++ b/web3_proxy/src/stats/stat_buffer.rs @@ -0,0 +1,269 @@ +use super::{AppStat, RpcQueryKey}; +use crate::app::RpcSecretKeyCache; +use derive_more::From; +use futures::stream; +use hashbrown::HashMap; +use influxdb2::api::write::TimestampPrecision; +use log::{error, info, trace}; +use migration::sea_orm::prelude::Decimal; +use migration::sea_orm::DatabaseConnection; +use std::time::Duration; +use tokio::sync::broadcast; +use tokio::task::JoinHandle; +use tokio::time::interval; + +#[derive(Debug, Default)] +pub struct BufferedRpcQueryStats { + pub frontend_requests: u64, + pub backend_requests: u64, + pub backend_retries: u64, + pub no_servers: u64, + pub cache_misses: u64, + pub cache_hits: u64, + pub sum_request_bytes: u64, + pub sum_response_bytes: u64, + pub sum_response_millis: u64, + pub sum_credits_used: Decimal, + /// Balance tells us the user's balance at this point in time + pub latest_balance: Decimal, +} + +#[derive(From)] +pub struct SpawnedStatBuffer { + pub stat_sender: flume::Sender, + /// these handles are important and must be allowed to finish + pub background_handle: JoinHandle>, +} +pub struct StatBuffer { + accounting_db_buffer: HashMap, + billing_period_seconds: i64, + chain_id: u64, + db_conn: Option, + db_save_interval_seconds: u32, + global_timeseries_buffer: HashMap, + influxdb_client: Option, + opt_in_timeseries_buffer: HashMap, + rpc_secret_key_cache: Option, + timestamp_precision: TimestampPrecision, + tsdb_save_interval_seconds: u32, +} + +impl StatBuffer { + #[allow(clippy::too_many_arguments)] + pub fn try_spawn( + billing_period_seconds: i64, + bucket: String, + chain_id: u64, + db_conn: Option, + db_save_interval_seconds: u32, + influxdb_client: Option, + rpc_secret_key_cache: Option, + shutdown_receiver: broadcast::Receiver<()>, + tsdb_save_interval_seconds: u32, + ) -> anyhow::Result> { + if db_conn.is_none() && influxdb_client.is_none() { + return Ok(None); + } + + let (stat_sender, stat_receiver) = flume::unbounded(); + + let timestamp_precision = TimestampPrecision::Seconds; + let mut new = Self { + accounting_db_buffer: Default::default(), + billing_period_seconds, + chain_id, + db_conn, + db_save_interval_seconds, + global_timeseries_buffer: Default::default(), + influxdb_client, + opt_in_timeseries_buffer: Default::default(), + rpc_secret_key_cache, + timestamp_precision, + tsdb_save_interval_seconds, + }; + + // any errors inside this task will cause the application to exit + let handle = tokio::spawn(async move { + new.aggregate_and_save_loop(bucket, stat_receiver, shutdown_receiver) + .await + }); + + Ok(Some((stat_sender, handle).into())) + } + + async fn aggregate_and_save_loop( + &mut self, + bucket: String, + stat_receiver: flume::Receiver, + mut shutdown_receiver: broadcast::Receiver<()>, + ) -> anyhow::Result<()> { + let mut tsdb_save_interval = + interval(Duration::from_secs(self.tsdb_save_interval_seconds as u64)); + let mut db_save_interval = + interval(Duration::from_secs(self.db_save_interval_seconds as u64)); + + // TODO: Somewhere here we should probably be updating the balance of the user + // And also update the credits used etc. for the referred user + + loop { + tokio::select! { + stat = stat_receiver.recv_async() => { + // info!("Received stat"); + // save the stat to a buffer + match stat { + Ok(AppStat::RpcQuery(stat)) => { + if self.influxdb_client.is_some() { + // TODO: round the timestamp at all? + + let global_timeseries_key = stat.global_timeseries_key(); + + self.global_timeseries_buffer.entry(global_timeseries_key).or_default().add(stat.clone()); + + if let Some(opt_in_timeseries_key) = stat.opt_in_timeseries_key() { + self.opt_in_timeseries_buffer.entry(opt_in_timeseries_key).or_default().add(stat.clone()); + } + } + + if self.db_conn.is_some() { + self.accounting_db_buffer.entry(stat.accounting_key(self.billing_period_seconds)).or_default().add(stat); + } + } + Err(err) => { + error!("error receiving stat: {:?}", err); + break; + } + } + } + _ = db_save_interval.tick() => { + // info!("DB save internal tick"); + let count = self.save_relational_stats().await; + if count > 0 { + trace!("Saved {} stats to the relational db", count); + } + } + _ = tsdb_save_interval.tick() => { + // info!("TSDB save internal tick"); + let count = self.save_tsdb_stats(&bucket).await; + if count > 0 { + trace!("Saved {} stats to the tsdb", count); + } + } + x = shutdown_receiver.recv() => { + match x { + Ok(_) => { + info!("stat_loop shutting down"); + }, + Err(err) => error!("stat_loop shutdown receiver err={:?}", err), + } + break; + } + } + } + + let saved_relational = self.save_relational_stats().await; + + info!("saved {} pending relational stat(s)", saved_relational); + + let saved_tsdb = self.save_tsdb_stats(&bucket).await; + + info!("saved {} pending tsdb stat(s)", saved_tsdb); + + info!("accounting and stat save loop complete"); + + Ok(()) + } + + async fn save_relational_stats(&mut self) -> usize { + let mut count = 0; + + if let Some(db_conn) = self.db_conn.as_ref() { + count = self.accounting_db_buffer.len(); + for (key, stat) in self.accounting_db_buffer.drain() { + // TODO: batch saves + // TODO: i don't like passing key (which came from the stat) to the function on the stat. but it works for now + if let Err(err) = stat + .save_db( + self.chain_id, + db_conn, + key, + self.rpc_secret_key_cache.as_ref(), + ) + .await + { + error!("unable to save accounting entry! err={:?}", err); + }; + } + } + + count + } + + // TODO: bucket should be an enum so that we don't risk typos + async fn save_tsdb_stats(&mut self, bucket: &str) -> usize { + let mut count = 0; + + if let Some(influxdb_client) = self.influxdb_client.as_ref() { + // TODO: use stream::iter properly to avoid allocating this Vec + let mut points = vec![]; + + for (key, stat) in self.global_timeseries_buffer.drain() { + // TODO: i don't like passing key (which came from the stat) to the function on the stat. but it works for now + match stat + .build_timeseries_point("global_proxy", self.chain_id, key) + .await + { + Ok(point) => { + points.push(point); + } + Err(err) => { + error!("unable to build global stat! err={:?}", err); + } + }; + } + + for (key, stat) in self.opt_in_timeseries_buffer.drain() { + // TODO: i don't like passing key (which came from the stat) to the function on the stat. but it works for now + match stat + .build_timeseries_point("opt_in_proxy", self.chain_id, key) + .await + { + Ok(point) => { + points.push(point); + } + Err(err) => { + // TODO: if this errors, we throw away some of the pending stats! we should probably buffer them somewhere to be tried again + error!("unable to build opt-in stat! err={:?}", err); + } + }; + } + + count = points.len(); + + if count > 0 { + // TODO: put max_batch_size in config? + // TODO: i think the real limit is the byte size of the http request. so, a simple line count won't work very well + let max_batch_size = 100; + + let mut num_left = count; + + while num_left > 0 { + let batch_size = num_left.min(max_batch_size); + + let p = points.split_off(batch_size); + + num_left -= batch_size; + + if let Err(err) = influxdb_client + .write_with_precision(bucket, stream::iter(p), self.timestamp_precision) + .await + { + // TODO: if this errors, we throw away some of the pending stats! we should probably buffer them somewhere to be tried again + error!("unable to save {} tsdb stats! err={:?}", batch_size, err); + } + } + } + } + + count + } +} From 510612d343fc51338a8a4282dcc229b50097835b Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Fri, 12 May 2023 15:47:01 -0700 Subject: [PATCH 09/66] use kanal instead of flume or tokio channels (#68) --- Cargo.lock | 129 ++++-------------- TODO.md | 2 +- latency/Cargo.toml | 1 + latency/src/peak_ewma/mod.rs | 20 +-- rate-counter/Cargo.toml | 2 +- web3_proxy/Cargo.toml | 2 +- web3_proxy/src/app/mod.rs | 2 +- web3_proxy/src/app/ws.rs | 10 +- .../src/bin/web3_proxy_cli/sentryd/mod.rs | 7 +- web3_proxy/src/config.rs | 4 +- web3_proxy/src/frontend/authorization.rs | 7 +- web3_proxy/src/frontend/errors.rs | 10 +- web3_proxy/src/frontend/rpc_proxy_ws.rs | 12 +- web3_proxy/src/rpcs/blockchain.rs | 4 +- web3_proxy/src/rpcs/many.rs | 26 ++-- web3_proxy/src/rpcs/one.rs | 24 ++-- web3_proxy/src/stats/mod.rs | 1 - web3_proxy/src/stats/stat_buffer.rs | 8 +- 18 files changed, 97 insertions(+), 174 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 5832affc..8753b543 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -792,16 +792,6 @@ dependencies = [ "cc", ] -[[package]] -name = "codespan-reporting" -version = "0.11.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3538270d33cc669650c4b093848450d380def10c331d38c768e34cac80576e6e" -dependencies = [ - "termcolor", - "unicode-width", -] - [[package]] name = "coins-bip32" version = "0.8.3" @@ -1204,50 +1194,6 @@ dependencies = [ "cipher 0.4.4", ] -[[package]] -name = "cxx" -version = "1.0.94" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f61f1b6389c3fe1c316bf8a4dccc90a38208354b330925bce1f74a6c4756eb93" -dependencies = [ - "cc", - "cxxbridge-flags", - "cxxbridge-macro", - "link-cplusplus", -] - -[[package]] -name = "cxx-build" -version = "1.0.94" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12cee708e8962df2aeb38f594aae5d827c022b6460ac71a7a3e2c3c2aae5a07b" -dependencies = [ - "cc", - "codespan-reporting", - "once_cell", - "proc-macro2", - "quote", - "scratch", - "syn 2.0.15", -] - -[[package]] -name = "cxxbridge-flags" -version = "1.0.94" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7944172ae7e4068c533afbb984114a56c46e9ccddda550499caa222902c7f7bb" - -[[package]] -name = "cxxbridge-macro" -version = "1.0.94" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2345488264226bf682893e25de0769f3360aac9957980ec49361b083ddaa5bc5" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.15", -] - [[package]] name = "dashmap" version = "4.0.2" @@ -2052,19 +1998,6 @@ dependencies = [ "miniz_oxide 0.7.1", ] -[[package]] -name = "flume" -version = "0.10.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1657b4441c3403d9f7b3409e47575237dac27b1b5726df654a6ecbf92f0f7577" -dependencies = [ - "futures-core", - "futures-sink", - "nanorand", - "pin-project", - "spin 0.9.8", -] - [[package]] name = "fnv" version = "1.0.7" @@ -2376,9 +2309,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.3.18" +version = "0.3.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17f8a914c2987b688368b5138aa05321db91f4090cf26118185672ad588bce21" +checksum = "d357c7ae988e7d2182f7d7871d0b963962420b0678b0997ce7de72001aeab782" dependencies = [ "bytes", "fnv", @@ -2712,12 +2645,11 @@ dependencies = [ [[package]] name = "iana-time-zone-haiku" -version = "0.1.1" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0703ae284fc167426161c2e3f1da3ea71d94b21bedbcc9494e92b28e334e3dca" +checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" dependencies = [ - "cxx", - "cxx-build", + "cc", ] [[package]] @@ -2976,6 +2908,16 @@ dependencies = [ "signature 2.1.0", ] +[[package]] +name = "kanal" +version = "0.1.0-pre8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b05d55519627edaf7fd0f29981f6dc03fb52df3f5b257130eb8d0bf2801ea1d7" +dependencies = [ + "futures-core", + "lock_api", +] + [[package]] name = "keccak" version = "0.1.4" @@ -3021,6 +2963,7 @@ name = "latency" version = "0.1.0" dependencies = [ "ewma", + "kanal", "log", "serde", "tokio", @@ -3069,15 +3012,6 @@ dependencies = [ "vcpkg", ] -[[package]] -name = "link-cplusplus" -version = "1.0.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ecd207c9c713c34f95a097a5b029ac2ce6010530c7b49d7fea24d977dede04f5" -dependencies = [ - "cc", -] - [[package]] name = "linux-raw-sys" version = "0.3.7" @@ -3274,15 +3208,6 @@ dependencies = [ "uuid 1.3.2", ] -[[package]] -name = "nanorand" -version = "0.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a51313c5820b0b02bd422f4b44776fbf47961755c74ce64afc73bfad10226c3" -dependencies = [ - "getrandom", -] - [[package]] name = "native-tls" version = "0.2.11" @@ -4277,7 +4202,7 @@ dependencies = [ name = "rate-counter" version = "0.1.0" dependencies = [ - "flume", + "kanal", "tokio", ] @@ -4323,9 +4248,9 @@ dependencies = [ [[package]] name = "rdkafka-sys" -version = "4.3.0+1.9.2" +version = "4.4.0+1.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d222a401698c7f2010e3967353eae566d9934dcda49c29910da922414ab4e3f4" +checksum = "87ac9d87c3aba1748e3112318459f2ac8bff80bfff7359e338e0463549590249" dependencies = [ "cmake", "libc", @@ -4777,12 +4702,6 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" -[[package]] -name = "scratch" -version = "1.0.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1792db035ce95be60c3f8853017b3999209281c24e2ba5bc8e59bf97a0c590c1" - [[package]] name = "scrypt" version = "0.10.0" @@ -5004,9 +4923,9 @@ dependencies = [ [[package]] name = "security-framework" -version = "2.8.2" +version = "2.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a332be01508d814fed64bf28f798a146d73792121129962fdf335bb3c49a4254" +checksum = "ca2855b3715770894e67cbfa3df957790aa0c9edc3bf06efa1a84d77fa0839d1" dependencies = [ "bitflags", "core-foundation", @@ -5017,9 +4936,9 @@ dependencies = [ [[package]] name = "security-framework-sys" -version = "2.8.0" +version = "2.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31c9bb296072e961fcbd8853511dd39c2d8be2deb1e17c6860b1d30732b323b4" +checksum = "f51d0c0d83bec45f16480d0ce0058397a69e48fcdc52d1dc8855fb68acbd31a7" dependencies = [ "core-foundation-sys", "libc", @@ -6658,7 +6577,6 @@ dependencies = [ "ethers", "ewma", "fdlimit", - "flume", "fstrings", "futures", "gethostname", @@ -6673,6 +6591,7 @@ dependencies = [ "influxdb2-structmap", "ipnet", "itertools", + "kanal", "latency", "listenfd", "log", diff --git a/TODO.md b/TODO.md index b405b251..690007b4 100644 --- a/TODO.md +++ b/TODO.md @@ -189,7 +189,7 @@ These are roughly in order of completition - [x] graceful shutdown. stop taking new requests and don't stop until all outstanding queries are handled - https://github.com/tokio-rs/mini-redis/blob/master/src/shutdown.rs - we need this because we need to be sure all the queries are saved in the db. maybe put stuff in Drop - - need an flume::watch on unflushed stats that we can subscribe to. wait for it to flip to true + - need an kanal::watch on unflushed stats that we can subscribe to. wait for it to flip to true - [x] don't use unix timestamps for response_millis since leap seconds will confuse it - [x] config to allow origins even on the anonymous endpoints - [x] send logs to sentry diff --git a/latency/Cargo.toml b/latency/Cargo.toml index eb51eba9..8583e9ab 100644 --- a/latency/Cargo.toml +++ b/latency/Cargo.toml @@ -7,6 +7,7 @@ edition = "2021" [dependencies] ewma = "0.1.1" +kanal = "0.1.0-pre8" log = "0.4.17" serde = { version = "1.0.163", features = [] } tokio = { version = "1.28.1", features = ["full"] } diff --git a/latency/src/peak_ewma/mod.rs b/latency/src/peak_ewma/mod.rs index 9b6f2f8b..533ff3af 100644 --- a/latency/src/peak_ewma/mod.rs +++ b/latency/src/peak_ewma/mod.rs @@ -2,9 +2,8 @@ mod rtt_estimate; use std::sync::Arc; +use kanal::SendError; use log::error; -use tokio::sync::mpsc; -use tokio::sync::mpsc::error::TrySendError; use tokio::task::JoinHandle; use tokio::time::{Duration, Instant}; @@ -20,7 +19,7 @@ pub struct PeakEwmaLatency { /// Join handle for the latency calculation task pub join_handle: JoinHandle<()>, /// Send to update with each request duration - request_tx: mpsc::Sender, + request_tx: kanal::AsyncSender, /// Latency average and last update time rtt_estimate: Arc, /// Decay time @@ -34,7 +33,7 @@ impl PeakEwmaLatency { /// average latency. pub fn spawn(decay_ns: f64, buf_size: usize, start_latency: Duration) -> Self { debug_assert!(decay_ns > 0.0, "decay_ns must be positive"); - let (request_tx, request_rx) = mpsc::channel(buf_size); + let (request_tx, request_rx) = kanal::bounded_async(buf_size); let rtt_estimate = Arc::new(AtomicRttEstimate::new(start_latency)); let task = PeakEwmaLatencyTask { request_rx, @@ -71,16 +70,19 @@ impl PeakEwmaLatency { /// Should only be called from the Web3Rpc that owns it. pub fn report(&self, duration: Duration) { match self.request_tx.try_send(duration) { - Ok(()) => {} - Err(TrySendError::Full(_)) => { + Ok(true) => {} + Ok(false) => { // We don't want to block if the channel is full, just // report the error error!("Latency report channel full"); // TODO: could we spawn a new tokio task to report tthis later? } - Err(TrySendError::Closed(_)) => { + Err(SendError::Closed) => { unreachable!("Owner should keep channel open"); } + Err(SendError::ReceiveClosed) => { + unreachable!("Receiver should keep channel open"); + } }; //.expect("Owner should keep channel open"); } @@ -90,7 +92,7 @@ impl PeakEwmaLatency { #[derive(Debug)] struct PeakEwmaLatencyTask { /// Receive new request timings for update - request_rx: mpsc::Receiver, + request_rx: kanal::AsyncReceiver, /// Current estimate and update time rtt_estimate: Arc, /// Last update time, used for decay calculation @@ -102,7 +104,7 @@ struct PeakEwmaLatencyTask { impl PeakEwmaLatencyTask { /// Run the loop for updating latency async fn run(mut self) { - while let Some(rtt) = self.request_rx.recv().await { + while let Ok(rtt) = self.request_rx.recv().await { self.update(rtt); } } diff --git a/rate-counter/Cargo.toml b/rate-counter/Cargo.toml index 7bf027e4..6e25777a 100644 --- a/rate-counter/Cargo.toml +++ b/rate-counter/Cargo.toml @@ -5,5 +5,5 @@ authors = ["Bryan Stitt "] edition = "2021" [dependencies] -flume = "0.10.14" +kanal = "0.1.0-pre8" tokio = { version = "1.28.1", features = ["time"] } diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index 4883f9a2..4d4a926c 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -43,7 +43,6 @@ env_logger = "0.10.0" ethers = { version = "2.0.4", default-features = false, features = ["rustls", "ws"] } ewma = "0.1.1" fdlimit = "0.2.1" -flume = "0.10.14" fstrings = "0.2" futures = { version = "0.3.28", features = ["thread-pool"] } gethostname = "0.4.2" @@ -58,6 +57,7 @@ influxdb2 = { git = "https://github.com/llamanodes/influxdb2", features = ["rust influxdb2-structmap = { git = "https://github.com/llamanodes/influxdb2/"} ipnet = "2.7.2" itertools = "0.10.5" +kanal = "0.1.0-pre8" listenfd = "1.0.1" log = "0.4.17" mimalloc = { version = "0.1.37", optional = true} diff --git a/web3_proxy/src/app/mod.rs b/web3_proxy/src/app/mod.rs index 63c3f9f5..09dfb2ed 100644 --- a/web3_proxy/src/app/mod.rs +++ b/web3_proxy/src/app/mod.rs @@ -265,7 +265,7 @@ pub struct Web3ProxyApp { Cache, hashbrown::hash_map::DefaultHashBuilder>, pub kafka_producer: Option, /// channel for sending stats in a background task - pub stat_sender: Option>, + pub stat_sender: Option>, } /// flatten a JoinError into an anyhow error diff --git a/web3_proxy/src/app/ws.rs b/web3_proxy/src/app/ws.rs index db348577..6e8e61f1 100644 --- a/web3_proxy/src/app/ws.rs +++ b/web3_proxy/src/app/ws.rs @@ -24,7 +24,7 @@ impl Web3ProxyApp { jsonrpc_request: JsonRpcRequest, subscription_count: &'a AtomicUsize, // TODO: taking a sender for Message instead of the exact json we are planning to send feels wrong, but its easier for now - response_sender: flume::Sender, + response_sender: kanal::AsyncSender, ) -> Web3ProxyResult<(AbortHandle, JsonRpcForwardedResponse)> { let request_metadata = RequestMetadata::new( self, @@ -94,7 +94,7 @@ impl Web3ProxyApp { // TODO: can we check a content type header? let response_msg = Message::Text(response_str); - if response_sender.send_async(response_msg).await.is_err() { + if response_sender.send(response_msg).await.is_err() { // TODO: increment error_response? i don't think so. i think this will happen once every time a client disconnects. // TODO: cancel this subscription earlier? select on head_block_receiver.next() and an abort handle? break; @@ -158,7 +158,7 @@ impl Web3ProxyApp { // TODO: do clients support binary messages? let response_msg = Message::Text(response_str); - if response_sender.send_async(response_msg).await.is_err() { + if response_sender.send(response_msg).await.is_err() { // TODO: cancel this subscription earlier? select on head_block_receiver.next() and an abort handle? break; }; @@ -221,7 +221,7 @@ impl Web3ProxyApp { // TODO: do clients support binary messages? let response_msg = Message::Text(response_str); - if response_sender.send_async(response_msg).await.is_err() { + if response_sender.send(response_msg).await.is_err() { // TODO: cancel this subscription earlier? select on head_block_receiver.next() and an abort handle? break; }; @@ -285,7 +285,7 @@ impl Web3ProxyApp { // TODO: do clients support binary messages? let response_msg = Message::Text(response_str); - if response_sender.send_async(response_msg).await.is_err() { + if response_sender.send(response_msg).await.is_err() { // TODO: cancel this subscription earlier? select on head_block_receiver.next() and an abort handle? break; }; diff --git a/web3_proxy/src/bin/web3_proxy_cli/sentryd/mod.rs b/web3_proxy/src/bin/web3_proxy_cli/sentryd/mod.rs index 5e0af642..a1d9f50d 100644 --- a/web3_proxy/src/bin/web3_proxy_cli/sentryd/mod.rs +++ b/web3_proxy/src/bin/web3_proxy_cli/sentryd/mod.rs @@ -11,7 +11,6 @@ use log::{error, info}; use pagerduty_rs::{eventsv2async::EventsV2 as PagerdutyAsyncEventsV2, types::Event}; use serde_json::json; use std::time::Duration; -use tokio::sync::mpsc; use tokio::time::{interval, MissedTickBehavior}; use web3_proxy::{config::TopConfig, pagerduty::pagerduty_alert}; @@ -116,7 +115,7 @@ impl SentrydSubCommand { let mut handles = FuturesUnordered::new(); // channels and a task for sending errors to logs/pagerduty - let (error_sender, mut error_receiver) = mpsc::channel::(10); + let (error_sender, error_receiver) = kanal::bounded_async::(10); { let error_handler_f = async move { @@ -124,7 +123,7 @@ impl SentrydSubCommand { info!("set PAGERDUTY_INTEGRATION_KEY to send create alerts for errors"); } - while let Some(err) = error_receiver.recv().await { + while let Ok(err) = error_receiver.recv().await { log::log!(err.level, "check failed: {:#?}", err); if matches!(err.level, log::Level::Error) { @@ -258,7 +257,7 @@ async fn a_loop( class: &str, seconds: u64, error_level: log::Level, - error_sender: mpsc::Sender, + error_sender: kanal::AsyncSender, f: impl Fn(SentrydErrorBuilder) -> T, ) -> anyhow::Result<()> where diff --git a/web3_proxy/src/config.rs b/web3_proxy/src/config.rs index 05a947d5..51fd4099 100644 --- a/web3_proxy/src/config.rs +++ b/web3_proxy/src/config.rs @@ -285,8 +285,8 @@ impl Web3RpcConfig { http_client: Option, http_interval_sender: Option>>, blocks_by_hash_cache: BlocksByHashCache, - block_sender: Option>, - tx_id_sender: Option>, + block_sender: Option>, + tx_id_sender: Option>, reconnect: bool, ) -> anyhow::Result<(Arc, AnyhowJoinHandle<()>)> { if !self.extra.is_empty() { diff --git a/web3_proxy/src/frontend/authorization.rs b/web3_proxy/src/frontend/authorization.rs index 5d951956..bb1d6ecb 100644 --- a/web3_proxy/src/frontend/authorization.rs +++ b/web3_proxy/src/frontend/authorization.rs @@ -267,7 +267,7 @@ pub struct RequestMetadata { pub kafka_debug_logger: Option>, /// Channel to send stats to - pub stat_sender: Option>, + pub stat_sender: Option>, } impl Default for RequestMetadata { @@ -457,8 +457,11 @@ impl RequestMetadata { let stat: AppStat = stat.into(); + // can't use async because a Drop can call this + let stat_sender = stat_sender.to_sync(); + if let Err(err) = stat_sender.send(stat) { - error!("failed sending stats for {:?}: {:?}", err.0, err); + error!("failed sending stat: {:?}", err); // TODO: return it? that seems like it might cause an infinite loop }; diff --git a/web3_proxy/src/frontend/errors.rs b/web3_proxy/src/frontend/errors.rs index b7d1a669..1ae29e23 100644 --- a/web3_proxy/src/frontend/errors.rs +++ b/web3_proxy/src/frontend/errors.rs @@ -51,7 +51,6 @@ pub enum Web3ProxyError { EthersHttpClientError(ethers::prelude::HttpClientError), EthersProviderError(ethers::prelude::ProviderError), EthersWsClientError(ethers::prelude::WsClientError), - FlumeRecvError(flume::RecvError), GasEstimateNotU256, Headers(headers::Error), HeaderToString(ToStrError), @@ -78,6 +77,8 @@ pub enum Web3ProxyError { #[display(fmt = "{:?}", _0)] #[error(ignore)] JsonRpcForwardedError(JsonRpcForwardedResponse), + KanalReceiveError(kanal::ReceiveError), + KanalSendError(kanal::SendError), #[display(fmt = "{:?}", _0)] #[error(ignore)] MsgPackEncode(rmp_serde::encode::Error), @@ -112,7 +113,6 @@ pub enum Web3ProxyError { #[from(ignore)] RefererNotAllowed(headers::Referer), SemaphoreAcquireError(AcquireError), - SendAppStatError(flume::SendError), SerdeJson(serde_json::Error), /// simple way to return an error message to the user and an anyhow to our logs #[display(fmt = "{}, {}, {:?}", _0, _1, _2)] @@ -261,8 +261,8 @@ impl Web3ProxyError { ), ) } - Self::FlumeRecvError(err) => { - warn!("FlumeRecvError err={:#?}", err); + Self::KanalReceiveError(err) => { + warn!("KanalRecvError err={:#?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, JsonRpcForwardedResponse::from_str( @@ -701,7 +701,7 @@ impl Web3ProxyError { ), ) } - Self::SendAppStatError(err) => { + Self::KanalSendError(err) => { error!("SendAppStatError err={:?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, diff --git a/web3_proxy/src/frontend/rpc_proxy_ws.rs b/web3_proxy/src/frontend/rpc_proxy_ws.rs index e0522b00..e562f297 100644 --- a/web3_proxy/src/frontend/rpc_proxy_ws.rs +++ b/web3_proxy/src/frontend/rpc_proxy_ws.rs @@ -311,7 +311,7 @@ async fn proxy_web3_socket( let (ws_tx, ws_rx) = socket.split(); // create a channel for our reader and writer can communicate. todo: benchmark different channels - let (response_sender, response_receiver) = flume::unbounded::(); + let (response_sender, response_receiver) = kanal::unbounded_async::(); tokio::spawn(write_web3_socket(response_receiver, ws_tx)); tokio::spawn(read_web3_socket(app, authorization, ws_rx, response_sender)); @@ -323,7 +323,7 @@ async fn handle_socket_payload( app: Arc, authorization: &Arc, payload: &str, - response_sender: &flume::Sender, + response_sender: &kanal::AsyncSender, subscription_count: &AtomicUsize, subscriptions: Arc>>, ) -> Web3ProxyResult<(Message, Option)> { @@ -452,7 +452,7 @@ async fn read_web3_socket( app: Arc, authorization: Arc, mut ws_rx: SplitStream, - response_sender: flume::Sender, + response_sender: kanal::AsyncSender, ) { // RwLock should be fine here. a user isn't going to be opening tons of subscriptions let subscriptions = Arc::new(RwLock::new(HashMap::new())); @@ -528,7 +528,7 @@ async fn read_web3_socket( } }; - if response_sender.send_async(response_msg).await.is_err() { + if response_sender.send(response_msg).await.is_err() { let _ = close_sender.send(true); return; }; @@ -549,13 +549,13 @@ async fn read_web3_socket( } async fn write_web3_socket( - response_rx: flume::Receiver, + response_rx: kanal::AsyncReceiver, mut ws_tx: SplitSink, ) { // TODO: increment counter for open websockets // TODO: is there any way to make this stream receive. - while let Ok(msg) = response_rx.recv_async().await { + while let Ok(msg) = response_rx.recv().await { // a response is ready // TODO: poke rate limits for this user? diff --git a/web3_proxy/src/rpcs/blockchain.rs b/web3_proxy/src/rpcs/blockchain.rs index 3d6ac3dd..dc1ef8c2 100644 --- a/web3_proxy/src/rpcs/blockchain.rs +++ b/web3_proxy/src/rpcs/blockchain.rs @@ -365,7 +365,7 @@ impl Web3Rpcs { pub(super) async fn process_incoming_blocks( &self, authorization: &Arc, - block_receiver: flume::Receiver, + block_receiver: kanal::AsyncReceiver, // TODO: document that this is a watch sender and not a broadcast! if things get busy, blocks might get missed // Geth's subscriptions have the same potential for skipping blocks. pending_tx_sender: Option>, @@ -373,7 +373,7 @@ impl Web3Rpcs { let mut connection_heads = ConsensusFinder::new(self.max_block_age, self.max_block_lag); loop { - match block_receiver.recv_async().await { + match block_receiver.recv().await { Ok((new_block, rpc)) => { let rpc_name = rpc.name.clone(); diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index 9d27ad43..6addeb6f 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -43,7 +43,7 @@ use tokio::time::{interval, sleep, sleep_until, Duration, Instant, MissedTickBeh #[derive(From)] pub struct Web3Rpcs { /// if watch_consensus_head_sender is some, Web3Rpc inside self will send blocks here when they get them - pub(crate) block_sender: flume::Sender<(Option, Arc)>, + pub(crate) block_sender: kanal::AsyncSender<(Option, Arc)>, /// any requests will be forwarded to one (or more) of these connections pub(crate) by_name: ArcSwap>>, /// notify all http providers to check their blocks at the same time @@ -57,8 +57,8 @@ pub struct Web3Rpcs { pub(super) watch_consensus_head_sender: Option>>, pub(super) pending_transaction_cache: Cache, - pub(super) pending_tx_id_receiver: flume::Receiver, - pub(super) pending_tx_id_sender: flume::Sender, + pub(super) pending_tx_id_receiver: kanal::AsyncReceiver, + pub(super) pending_tx_id_sender: kanal::AsyncSender, /// TODO: this map is going to grow forever unless we do some sort of pruning. maybe store pruned in redis? /// all blocks, including orphans pub(super) blocks_by_hash: BlocksByHashCache, @@ -94,8 +94,8 @@ impl Web3Rpcs { watch::Receiver>>, // watch::Receiver>, )> { - let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded(); - let (block_sender, block_receiver) = flume::unbounded::(); + let (pending_tx_id_sender, pending_tx_id_receiver) = kanal::unbounded_async(); + let (block_sender, block_receiver) = kanal::unbounded_async::(); // TODO: query the rpc to get the actual expected block time, or get from config? maybe have this be part of a health check? let expected_block_time_ms = match chain_id { @@ -347,7 +347,7 @@ impl Web3Rpcs { async fn subscribe( self: Arc, authorization: Arc, - block_receiver: flume::Receiver, + block_receiver: kanal::AsyncReceiver, pending_tx_sender: Option>, ) -> anyhow::Result<()> { let mut futures = vec![]; @@ -362,7 +362,7 @@ impl Web3Rpcs { let pending_tx_id_receiver = self.pending_tx_id_receiver.clone(); let handle = tokio::task::spawn(async move { // TODO: set up this future the same as the block funnel - while let Ok((pending_tx_id, rpc)) = pending_tx_id_receiver.recv_async().await { + while let Ok((pending_tx_id, rpc)) = pending_tx_id_receiver.recv().await { let f = clone.clone().process_incoming_tx_id( authorization.clone(), rpc, @@ -1391,8 +1391,8 @@ mod tests { (lagged_rpc.name.clone(), lagged_rpc.clone()), ]); - let (block_sender, _block_receiver) = flume::unbounded(); - let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded(); + let (block_sender, _block_receiver) = kanal::unbounded_async(); + let (pending_tx_id_sender, pending_tx_id_receiver) = kanal::unbounded_async(); let (watch_consensus_rpcs_sender, _watch_consensus_rpcs_receiver) = watch::channel(None); let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None); @@ -1643,8 +1643,8 @@ mod tests { (archive_rpc.name.clone(), archive_rpc.clone()), ]); - let (block_sender, _) = flume::unbounded(); - let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded(); + let (block_sender, _) = kanal::unbounded_async(); + let (pending_tx_id_sender, pending_tx_id_receiver) = kanal::unbounded_async(); let (watch_consensus_rpcs_sender, _) = watch::channel(None); let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None); @@ -1807,8 +1807,8 @@ mod tests { ), ]); - let (block_sender, _) = flume::unbounded(); - let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded(); + let (block_sender, _) = kanal::unbounded_async(); + let (pending_tx_id_sender, pending_tx_id_receiver) = kanal::unbounded_async(); let (watch_consensus_rpcs_sender, _) = watch::channel(None); let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None); diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index c541c0da..01575384 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -98,8 +98,8 @@ impl Web3Rpc { http_interval_sender: Option>>, redis_pool: Option, block_map: BlocksByHashCache, - block_sender: Option>, - tx_id_sender: Option)>>, + block_sender: Option>, + tx_id_sender: Option)>>, reconnect: bool, ) -> anyhow::Result<(Arc, AnyhowJoinHandle<()>)> { let created_at = Instant::now(); @@ -388,7 +388,7 @@ impl Web3Rpc { /// TODO: maybe it would be better to use "Full Jitter". The "Full Jitter" approach uses less work, but slightly more time. pub async fn retrying_connect( self: &Arc, - block_sender: Option<&flume::Sender>, + block_sender: Option<&kanal::AsyncSender>, chain_id: u64, db_conn: Option<&DatabaseConnection>, delay_start: bool, @@ -451,7 +451,7 @@ impl Web3Rpc { /// connect to the web3 provider async fn connect( self: &Arc, - block_sender: Option<&flume::Sender>, + block_sender: Option<&kanal::AsyncSender>, chain_id: u64, db_conn: Option<&DatabaseConnection>, ) -> anyhow::Result<()> { @@ -473,7 +473,7 @@ impl Web3Rpc { // tell the block subscriber that this rpc doesn't have any blocks if let Some(block_sender) = block_sender { block_sender - .send_async((None, self.clone())) + .send((None, self.clone())) .await .context("block_sender during connect")?; } @@ -588,7 +588,7 @@ impl Web3Rpc { pub(crate) async fn send_head_block_result( self: &Arc, new_head_block: Result, ProviderError>, - block_sender: &flume::Sender, + block_sender: &kanal::AsyncSender, block_map: BlocksByHashCache, ) -> anyhow::Result<()> { let new_head_block = match new_head_block { @@ -651,7 +651,7 @@ impl Web3Rpc { // send an empty block to take this server out of rotation block_sender - .send_async((new_head_block, self.clone())) + .send((new_head_block, self.clone())) .await .context("block_sender")?; @@ -670,11 +670,11 @@ impl Web3Rpc { self: Arc, authorization: &Arc, block_map: BlocksByHashCache, - block_sender: Option>, + block_sender: Option>, chain_id: u64, disconnect_receiver: watch::Receiver, http_interval_sender: Option>>, - tx_id_sender: Option)>>, + tx_id_sender: Option)>>, ) -> anyhow::Result<()> { let error_handler = if self.backup { RequestErrorHandler::DebugLevel @@ -895,7 +895,7 @@ impl Web3Rpc { self: Arc, authorization: Arc, http_interval_receiver: Option>, - block_sender: flume::Sender, + block_sender: kanal::AsyncSender, block_map: BlocksByHashCache, ) -> anyhow::Result<()> { trace!("watching new heads on {}", self); @@ -1090,7 +1090,7 @@ impl Web3Rpc { async fn subscribe_pending_transactions( self: Arc, authorization: Arc, - tx_id_sender: flume::Sender<(TxHash, Arc)>, + tx_id_sender: kanal::AsyncSender<(TxHash, Arc)>, ) -> anyhow::Result<()> { // TODO: give this a separate client. don't use new_head_client for everything. especially a firehose this big // TODO: timeout @@ -1115,7 +1115,7 @@ impl Web3Rpc { while let Some(pending_tx_id) = stream.next().await { tx_id_sender - .send_async((pending_tx_id, self.clone())) + .send((pending_tx_id, self.clone())) .await .context("tx_id_sender")?; diff --git a/web3_proxy/src/stats/mod.rs b/web3_proxy/src/stats/mod.rs index 930f9e04..14759611 100644 --- a/web3_proxy/src/stats/mod.rs +++ b/web3_proxy/src/stats/mod.rs @@ -672,7 +672,6 @@ impl RpcQueryStats { method: Option<&str>, ) -> Decimal { // some methods should be free. there might be cases where method isn't set (though they should be uncommon) - // TODO: get this list from config (and add more to it) if let Some(method) = method.as_ref() { if ["eth_chainId"].contains(method) { return 0.into(); diff --git a/web3_proxy/src/stats/stat_buffer.rs b/web3_proxy/src/stats/stat_buffer.rs index aaca71cd..e71f90af 100644 --- a/web3_proxy/src/stats/stat_buffer.rs +++ b/web3_proxy/src/stats/stat_buffer.rs @@ -30,7 +30,7 @@ pub struct BufferedRpcQueryStats { #[derive(From)] pub struct SpawnedStatBuffer { - pub stat_sender: flume::Sender, + pub stat_sender: kanal::AsyncSender, /// these handles are important and must be allowed to finish pub background_handle: JoinHandle>, } @@ -65,7 +65,7 @@ impl StatBuffer { return Ok(None); } - let (stat_sender, stat_receiver) = flume::unbounded(); + let (stat_sender, stat_receiver) = kanal::unbounded_async(); let timestamp_precision = TimestampPrecision::Seconds; let mut new = Self { @@ -94,7 +94,7 @@ impl StatBuffer { async fn aggregate_and_save_loop( &mut self, bucket: String, - stat_receiver: flume::Receiver, + stat_receiver: kanal::AsyncReceiver, mut shutdown_receiver: broadcast::Receiver<()>, ) -> anyhow::Result<()> { let mut tsdb_save_interval = @@ -107,7 +107,7 @@ impl StatBuffer { loop { tokio::select! { - stat = stat_receiver.recv_async() => { + stat = stat_receiver.recv() => { // info!("Received stat"); // save the stat to a buffer match stat { From 0c990b07553b16aa776670bc0c55d1e3cf4e5ae7 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Fri, 12 May 2023 16:02:43 -0700 Subject: [PATCH 10/66] use a cancel-safe channel for stats --- web3_proxy/src/app/mod.rs | 4 ++-- web3_proxy/src/frontend/authorization.rs | 12 +++++------- web3_proxy/src/stats/stat_buffer.rs | 14 +++++++------- 3 files changed, 14 insertions(+), 16 deletions(-) diff --git a/web3_proxy/src/app/mod.rs b/web3_proxy/src/app/mod.rs index 09dfb2ed..a7cfc851 100644 --- a/web3_proxy/src/app/mod.rs +++ b/web3_proxy/src/app/mod.rs @@ -54,7 +54,7 @@ use std::num::NonZeroU64; use std::str::FromStr; use std::sync::{atomic, Arc}; use std::time::Duration; -use tokio::sync::{broadcast, watch, Semaphore}; +use tokio::sync::{broadcast, mpsc, watch, Semaphore}; use tokio::task::JoinHandle; use tokio::time::{sleep, timeout}; use ulid::Ulid; @@ -265,7 +265,7 @@ pub struct Web3ProxyApp { Cache, hashbrown::hash_map::DefaultHashBuilder>, pub kafka_producer: Option, /// channel for sending stats in a background task - pub stat_sender: Option>, + pub stat_sender: Option>, } /// flatten a JoinError into an anyhow error diff --git a/web3_proxy/src/frontend/authorization.rs b/web3_proxy/src/frontend/authorization.rs index bb1d6ecb..15982954 100644 --- a/web3_proxy/src/frontend/authorization.rs +++ b/web3_proxy/src/frontend/authorization.rs @@ -33,7 +33,7 @@ use std::mem; use std::sync::atomic::{self, AtomicBool, AtomicI64, AtomicU64, AtomicUsize}; use std::time::Duration; use std::{net::IpAddr, str::FromStr, sync::Arc}; -use tokio::sync::{OwnedSemaphorePermit, Semaphore}; +use tokio::sync::{mpsc, OwnedSemaphorePermit, Semaphore}; use tokio::task::JoinHandle; use tokio::time::Instant; use ulid::Ulid; @@ -266,8 +266,8 @@ pub struct RequestMetadata { /// TODO: maybe this shouldn't be determined by ProxyMode. A request param should probably enable this pub kafka_debug_logger: Option>, - /// Channel to send stats to - pub stat_sender: Option>, + /// Cancel-safe channel to send stats to + pub stat_sender: Option>, } impl Default for RequestMetadata { @@ -457,12 +457,10 @@ impl RequestMetadata { let stat: AppStat = stat.into(); - // can't use async because a Drop can call this - let stat_sender = stat_sender.to_sync(); - if let Err(err) = stat_sender.send(stat) { - error!("failed sending stat: {:?}", err); + error!("failed sending stat {:?}: {:?}", err.0, err); // TODO: return it? that seems like it might cause an infinite loop + // TODO: but dropping stats is bad... hmm... i guess better to undercharge customers than overcharge }; Ok(None) diff --git a/web3_proxy/src/stats/stat_buffer.rs b/web3_proxy/src/stats/stat_buffer.rs index e71f90af..a9d14329 100644 --- a/web3_proxy/src/stats/stat_buffer.rs +++ b/web3_proxy/src/stats/stat_buffer.rs @@ -8,7 +8,7 @@ use log::{error, info, trace}; use migration::sea_orm::prelude::Decimal; use migration::sea_orm::DatabaseConnection; use std::time::Duration; -use tokio::sync::broadcast; +use tokio::sync::{broadcast, mpsc}; use tokio::task::JoinHandle; use tokio::time::interval; @@ -30,7 +30,7 @@ pub struct BufferedRpcQueryStats { #[derive(From)] pub struct SpawnedStatBuffer { - pub stat_sender: kanal::AsyncSender, + pub stat_sender: mpsc::UnboundedSender, /// these handles are important and must be allowed to finish pub background_handle: JoinHandle>, } @@ -65,7 +65,7 @@ impl StatBuffer { return Ok(None); } - let (stat_sender, stat_receiver) = kanal::unbounded_async(); + let (stat_sender, stat_receiver) = mpsc::unbounded_channel(); let timestamp_precision = TimestampPrecision::Seconds; let mut new = Self { @@ -94,7 +94,7 @@ impl StatBuffer { async fn aggregate_and_save_loop( &mut self, bucket: String, - stat_receiver: kanal::AsyncReceiver, + mut stat_receiver: mpsc::UnboundedReceiver, mut shutdown_receiver: broadcast::Receiver<()>, ) -> anyhow::Result<()> { let mut tsdb_save_interval = @@ -111,7 +111,7 @@ impl StatBuffer { // info!("Received stat"); // save the stat to a buffer match stat { - Ok(AppStat::RpcQuery(stat)) => { + Some(AppStat::RpcQuery(stat)) => { if self.influxdb_client.is_some() { // TODO: round the timestamp at all? @@ -128,8 +128,8 @@ impl StatBuffer { self.accounting_db_buffer.entry(stat.accounting_key(self.billing_period_seconds)).or_default().add(stat); } } - Err(err) => { - error!("error receiving stat: {:?}", err); + None => { + info!("done receiving stats"); break; } } From 3cea4c3123b3ef3fdd8ea796fffc1fd0d8814473 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Fri, 12 May 2023 22:18:19 -0700 Subject: [PATCH 11/66] update peak_ewma to use request latency and fix off by one --- web3_proxy/src/rpcs/one.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index 01575384..597263cd 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -237,14 +237,14 @@ impl Web3Rpc { pub fn peak_ewma(&self) -> OrderedFloat { // TODO: use request instead of head latency? that was killing perf though - let head_ewma = self.head_latency.read().value(); + let peak_latency = self.peak_latency.as_ref().unwrap().latency().as_secs_f64(); // TODO: what ordering? - let active_requests = self.active_requests.load(atomic::Ordering::Relaxed) as f64; + let active_requests = self.active_requests.load(atomic::Ordering::Relaxed) as f64 + 1.0; // TODO: i'm not sure head * active is exactly right. but we'll see // TODO: i don't think this actually counts as peak. investigate with atomics.rs and peak_ewma.rs - OrderedFloat(head_ewma * active_requests) + OrderedFloat(peak_latency * active_requests) } // TODO: would be great if rpcs exposed this. see https://github.com/ledgerwatch/erigon/issues/6391 From 914c3e03a8275028e66666c51fa617cf0a064691 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Fri, 12 May 2023 22:22:21 -0700 Subject: [PATCH 12/66] remove stale comments --- web3_proxy/src/rpcs/one.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index 597263cd..416055a8 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -236,14 +236,11 @@ impl Web3Rpc { } pub fn peak_ewma(&self) -> OrderedFloat { - // TODO: use request instead of head latency? that was killing perf though let peak_latency = self.peak_latency.as_ref().unwrap().latency().as_secs_f64(); // TODO: what ordering? let active_requests = self.active_requests.load(atomic::Ordering::Relaxed) as f64 + 1.0; - // TODO: i'm not sure head * active is exactly right. but we'll see - // TODO: i don't think this actually counts as peak. investigate with atomics.rs and peak_ewma.rs OrderedFloat(peak_latency * active_requests) } From c54970da0ad77201fdf1503ab6ab465a79db5dc2 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Fri, 12 May 2023 23:00:03 -0700 Subject: [PATCH 13/66] change ordering and move fetch_add and fetch_sub --- README.md | 2 +- deferred-rate-limiter/src/lib.rs | 2 +- latency/src/peak_ewma/rtt_estimate.rs | 4 +- latency/src/util/atomic_f32_pair.rs | 6 +-- web3_proxy/src/app/mod.rs | 4 +- web3_proxy/src/frontend/authorization.rs | 4 +- web3_proxy/src/rpcs/many.rs | 10 ++--- web3_proxy/src/rpcs/one.rs | 24 ++++++++--- web3_proxy/src/rpcs/request.rs | 54 ++++++++++++++---------- 9 files changed, 65 insertions(+), 45 deletions(-) diff --git a/README.md b/README.md index af5dabf1..4461169e 100644 --- a/README.md +++ b/README.md @@ -6,7 +6,7 @@ Web3_proxy is a fast caching and load balancing proxy for web3 (Ethereum or simi Signed transactions (eth_sendRawTransaction) are sent in parallel to the configured private RPCs (eden, ethermine, flashbots, etc.). -All other requests are sent to an RPC server on the latest block (alchemy, moralis, rivet, your own node, or one of many other providers). If multiple servers are in sync, they are prioritized by `active_requests/soft_limit`. Note that this means that the fastest server is most likely to serve requests and slow servers are unlikely to ever get any requests. +All other requests are sent to an RPC server on the latest block (llamanodes, alchemy, moralis, rivet, your own node, or one of many other providers). If multiple servers are in sync, they are prioritized by `active_requests` and request latency. Note that this means that the fastest server is most likely to serve requests and slow servers are unlikely to ever get any requests. Each server has different limits to configure. The `soft_limit` is the number of parallel active requests where a server starts to slow down. The `hard_limit` is where a server starts giving rate limits or other errors. diff --git a/deferred-rate-limiter/src/lib.rs b/deferred-rate-limiter/src/lib.rs index 8f055ce3..f02c69f8 100644 --- a/deferred-rate-limiter/src/lib.rs +++ b/deferred-rate-limiter/src/lib.rs @@ -139,7 +139,7 @@ where Ok(deferred_rate_limit_result) } else { // we have a cached amount here - let cached_key_count = local_key_count.fetch_add(count, Ordering::Acquire); + let cached_key_count = local_key_count.fetch_add(count, Ordering::AcqRel); // assuming no other parallel futures incremented this key, this is the count that redis has let expected_key_count = cached_key_count + count; diff --git a/latency/src/peak_ewma/rtt_estimate.rs b/latency/src/peak_ewma/rtt_estimate.rs index be56fe9c..c169ea4a 100644 --- a/latency/src/peak_ewma/rtt_estimate.rs +++ b/latency/src/peak_ewma/rtt_estimate.rs @@ -97,7 +97,7 @@ impl AtomicRttEstimate { /// This method omits the ordering argument since loads may use /// slightly stale data to avoid adding additional latency. pub fn load(&self) -> RttEstimate { - RttEstimate::from_pair(self.pair.load(Ordering::Relaxed), self.start_time) + RttEstimate::from_pair(self.pair.load(Ordering::Acquire), self.start_time) } /// Fetches the value, and applies a function to it that returns an @@ -114,7 +114,7 @@ impl AtomicRttEstimate { let mut update_at = Instant::now(); let mut rtt = Duration::ZERO; self.pair - .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |pair| { + .fetch_update(Ordering::Release, Ordering::Acquire, |pair| { rtt = f(RttEstimate::from_pair(pair, self.start_time)); // Save the new update_at inside the function in case it // is run multiple times diff --git a/latency/src/util/atomic_f32_pair.rs b/latency/src/util/atomic_f32_pair.rs index fa74fa0b..18b356c3 100644 --- a/latency/src/util/atomic_f32_pair.rs +++ b/latency/src/util/atomic_f32_pair.rs @@ -69,7 +69,7 @@ mod tests { fn test_atomic_f32_pair_load() { let pair = [f32::consts::PI, f32::consts::E]; let atomic = AtomicF32Pair::new(pair); - assert_eq!(pair, atomic.load(Ordering::Relaxed)); + assert_eq!(pair, atomic.load(Ordering::Acquire)); } #[test] @@ -77,13 +77,13 @@ mod tests { let pair = [f32::consts::PI, f32::consts::E]; let atomic = AtomicF32Pair::new(pair); atomic - .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |[f1, f2]| { + .fetch_update(Ordering::Release, Ordering::Acquire, |[f1, f2]| { Some([f1 + 1.0, f2 + 1.0]) }) .unwrap(); assert_eq!( [pair[0] + 1.0, pair[1] + 1.0], - atomic.load(Ordering::Relaxed) + atomic.load(Ordering::Acquire) ); } } diff --git a/web3_proxy/src/app/mod.rs b/web3_proxy/src/app/mod.rs index a7cfc851..ce1ac9ba 100644 --- a/web3_proxy/src/app/mod.rs +++ b/web3_proxy/src/app/mod.rs @@ -1780,7 +1780,7 @@ impl Web3ProxyApp { if block_depth < self.config.archive_depth { request_metadata .archive_request - .store(true, atomic::Ordering::Relaxed); + .store(true, atomic::Ordering::Release); } let request_block = self @@ -1810,7 +1810,7 @@ impl Web3ProxyApp { if block_depth < self.config.archive_depth { request_metadata .archive_request - .store(true, atomic::Ordering::Relaxed); + .store(true, atomic::Ordering::Release); } let from_block = self diff --git a/web3_proxy/src/frontend/authorization.rs b/web3_proxy/src/frontend/authorization.rs index 15982954..6ba772f0 100644 --- a/web3_proxy/src/frontend/authorization.rs +++ b/web3_proxy/src/frontend/authorization.rs @@ -205,7 +205,7 @@ impl KafkaDebugLogger { let payload = rmp_serde::to_vec(&request).expect("requests should always serialize with rmp"); - self.num_requests.fetch_add(1, atomic::Ordering::SeqCst); + self.num_requests.fetch_add(1, atomic::Ordering::AcqRel); self.background_log(payload) } @@ -217,7 +217,7 @@ impl KafkaDebugLogger { let payload = rmp_serde::to_vec(&response).expect("requests should always serialize with rmp"); - self.num_responses.fetch_add(1, atomic::Ordering::SeqCst); + self.num_responses.fetch_add(1, atomic::Ordering::AcqRel); self.background_log(payload) } diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index 6addeb6f..82924eb0 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -620,7 +620,7 @@ impl Web3Rpcs { } if let Some(request_metadata) = request_metadata { - request_metadata.no_servers.fetch_add(1, Ordering::Release); + request_metadata.no_servers.fetch_add(1, Ordering::AcqRel); } match earliest_retry_at { @@ -929,7 +929,7 @@ impl Web3Rpcs { // TODO: have a separate column for rate limited? if let Some(request_metadata) = request_metadata { - request_metadata.no_servers.fetch_add(1, Ordering::Release); + request_metadata.no_servers.fetch_add(1, Ordering::AcqRel); } tokio::select! { @@ -943,7 +943,7 @@ impl Web3Rpcs { } OpenRequestResult::NotReady => { if let Some(request_metadata) = request_metadata { - request_metadata.no_servers.fetch_add(1, Ordering::Release); + request_metadata.no_servers.fetch_add(1, Ordering::AcqRel); } let waiting_for = min_block_needed.max(max_block_needed); @@ -1086,7 +1086,7 @@ impl Web3Rpcs { if let Some(request_metadata) = &request_metadata { // TODO: if this times out, i think we drop this - request_metadata.no_servers.fetch_add(1, Ordering::Release); + request_metadata.no_servers.fetch_add(1, Ordering::AcqRel); } watch_consensus_rpcs.changed().await?; @@ -1101,7 +1101,7 @@ impl Web3Rpcs { warn!("All rate limits exceeded. Sleeping"); if let Some(request_metadata) = &request_metadata { - request_metadata.no_servers.fetch_add(1, Ordering::Release); + request_metadata.no_servers.fetch_add(1, Ordering::AcqRel); } tokio::select! { diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index 416055a8..ee792049 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -239,7 +239,7 @@ impl Web3Rpc { let peak_latency = self.peak_latency.as_ref().unwrap().latency().as_secs_f64(); // TODO: what ordering? - let active_requests = self.active_requests.load(atomic::Ordering::Relaxed) as f64 + 1.0; + let active_requests = self.active_requests.load(atomic::Ordering::Acquire) as f64 + 1.0; OrderedFloat(peak_latency * active_requests) } @@ -734,7 +734,7 @@ impl Web3Rpc { // health check as a way of keeping this rpc's request_ewma accurate // TODO: do something different if this is a backup server? - new_total_requests = rpc.total_requests.load(atomic::Ordering::Relaxed); + new_total_requests = rpc.total_requests.load(atomic::Ordering::Acquire); // TODO: how many requests should we require in order to skip a health check? if new_total_requests - old_total_requests < 10 { @@ -1363,7 +1363,7 @@ impl Serialize for Web3Rpc { S: Serializer, { // 3 is the number of fields in the struct. - let mut state = serializer.serialize_struct("Web3Rpc", 9)?; + let mut state = serializer.serialize_struct("Web3Rpc", 12)?; // the url is excluded because it likely includes private information. just show the name that we use in keys state.serialize_field("name", &self.name)?; @@ -1372,7 +1372,7 @@ impl Serialize for Web3Rpc { state.serialize_field("backup", &self.backup)?; - match self.block_data_limit.load(atomic::Ordering::Relaxed) { + match self.block_data_limit.load(atomic::Ordering::Acquire) { u64::MAX => { state.serialize_field("block_data_limit", &None::<()>)?; } @@ -1395,9 +1395,21 @@ impl Serialize for Web3Rpc { state.serialize_field("head_latency", &self.head_latency.read().value())?; + state.serialize_field( + "peak_latency", + &self.peak_latency.as_ref().unwrap().latency(), + )?; + + state.serialize_field("peak_ewma", self.peak_ewma().as_ref())?; + + state.serialize_field( + "active_requests", + &self.active_requests.load(atomic::Ordering::Acquire), + )?; + state.serialize_field( "total_requests", - &self.total_requests.load(atomic::Ordering::Relaxed), + &self.total_requests.load(atomic::Ordering::Acquire), )?; state.end() @@ -1410,7 +1422,7 @@ impl fmt::Debug for Web3Rpc { f.field("name", &self.name); - let block_data_limit = self.block_data_limit.load(atomic::Ordering::Relaxed); + let block_data_limit = self.block_data_limit.load(atomic::Ordering::Acquire); if block_data_limit == u64::MAX { f.field("blocks", &"all"); } else { diff --git a/web3_proxy/src/rpcs/request.rs b/web3_proxy/src/rpcs/request.rs index ed0474cc..445f5d7a 100644 --- a/web3_proxy/src/rpcs/request.rs +++ b/web3_proxy/src/rpcs/request.rs @@ -7,7 +7,7 @@ use entities::revert_log; use entities::sea_orm_active_enums::Method; use ethers::providers::ProviderError; use ethers::types::{Address, Bytes}; -use log::{debug, error, trace, warn, Level}; +use log::{debug, error, info, trace, warn, Level}; use migration::sea_orm::{self, ActiveEnum, ActiveModelTrait}; use serde_json::json; use std::fmt; @@ -121,14 +121,22 @@ impl Authorization { } } +impl Drop for OpenRequestHandle { + fn drop(&mut self) { + let x = self + .rpc + .active_requests + .fetch_sub(1, atomic::Ordering::AcqRel); + } +} + impl OpenRequestHandle { pub async fn new(authorization: Arc, rpc: Arc) -> Self { // TODO: take request_id as an argument? // TODO: attach a unique id to this? customer requests have one, but not internal queries // TODO: what ordering?! - // TODO: should we be using metered, or not? i think not because we want stats for each handle - // TODO: these should maybe be sent to an influxdb instance? - rpc.active_requests.fetch_add(1, atomic::Ordering::Relaxed); + rpc.active_requests + .fetch_add(1, std::sync::atomic::Ordering::AcqRel); Self { authorization, rpc } } @@ -188,11 +196,9 @@ impl OpenRequestHandle { self.rpc .total_requests - .fetch_add(1, std::sync::atomic::Ordering::Relaxed); + .fetch_add(1, std::sync::atomic::Ordering::AcqRel); - self.rpc - .active_requests - .fetch_add(1, std::sync::atomic::Ordering::Relaxed); + // we used to fetch_add the active_request count here, but sometimes a request is made without going through this function (like with subscriptions) let start = Instant::now(); @@ -212,16 +218,16 @@ impl OpenRequestHandle { }; // note. we intentionally do not record this latency now. we do NOT want to measure errors - // let latency = latency.elapsed(); + let latency = start.elapsed(); - self.rpc - .active_requests - .fetch_sub(1, std::sync::atomic::Ordering::Relaxed); + // we used to fetch_sub the active_request count here, but sometimes the handle is dropped without request being called! - // TODO: i think ethers already has trace logging (and does it much more fancy) trace!( "response from {} for {} {:?}: {:?}", - self.rpc, method, params, response, + self.rpc, + method, + params, + response, ); if let Err(err) = &response { @@ -352,19 +358,21 @@ impl OpenRequestHandle { // TODO: do not unwrap! (doesn't matter much since we check method as a string above) let method: Method = Method::try_from_value(&method.to_string()).unwrap(); - // TODO: DO NOT UNWRAP! But also figure out the best way to keep returning ProviderErrors here - let params: EthCallParams = serde_json::from_value(json!(params)) - .context("parsing params to EthCallParams") - .unwrap(); + match serde_json::from_value::(json!(params)) { + Ok(params) => { + // spawn saving to the database so we don't slow down the request + let f = self.authorization.clone().save_revert(method, params.0 .0); - // spawn saving to the database so we don't slow down the request - let f = self.authorization.clone().save_revert(method, params.0 .0); - - tokio::spawn(f); + tokio::spawn(f); + } + Err(err) => { + warn!("failed parsing eth_call params. unable to save revert"); + } + } } } } else if let Some(peak_latency) = &self.rpc.peak_latency { - peak_latency.report(start.elapsed()); + peak_latency.report(latency); } else { unreachable!("peak_latency not initialized"); } From a3b97fda9bb20c301af1268667e903026819f8ad Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Fri, 12 May 2023 23:04:56 -0700 Subject: [PATCH 14/66] lint --- web3_proxy/src/rpcs/request.rs | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/web3_proxy/src/rpcs/request.rs b/web3_proxy/src/rpcs/request.rs index 445f5d7a..8103b5e6 100644 --- a/web3_proxy/src/rpcs/request.rs +++ b/web3_proxy/src/rpcs/request.rs @@ -7,7 +7,7 @@ use entities::revert_log; use entities::sea_orm_active_enums::Method; use ethers::providers::ProviderError; use ethers::types::{Address, Bytes}; -use log::{debug, error, info, trace, warn, Level}; +use log::{debug, error, trace, warn, Level}; use migration::sea_orm::{self, ActiveEnum, ActiveModelTrait}; use serde_json::json; use std::fmt; @@ -123,8 +123,7 @@ impl Authorization { impl Drop for OpenRequestHandle { fn drop(&mut self) { - let x = self - .rpc + self.rpc .active_requests .fetch_sub(1, atomic::Ordering::AcqRel); } @@ -366,7 +365,10 @@ impl OpenRequestHandle { tokio::spawn(f); } Err(err) => { - warn!("failed parsing eth_call params. unable to save revert"); + warn!( + "failed parsing eth_call params. unable to save revert. {}", + err + ); } } } From 74a602bc9c4c4aa67193260e375157c9b066faec Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Fri, 12 May 2023 23:20:59 -0700 Subject: [PATCH 15/66] wip --- latency/src/peak_ewma/mod.rs | 8 ++++++-- web3_proxy/src/rpcs/consensus.rs | 6 +++++- web3_proxy/src/rpcs/request.rs | 3 ++- 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/latency/src/peak_ewma/mod.rs b/latency/src/peak_ewma/mod.rs index 533ff3af..a6c248d4 100644 --- a/latency/src/peak_ewma/mod.rs +++ b/latency/src/peak_ewma/mod.rs @@ -3,7 +3,7 @@ mod rtt_estimate; use std::sync::Arc; use kanal::SendError; -use log::error; +use log::{error, info, trace}; use tokio::task::JoinHandle; use tokio::time::{Duration, Instant}; @@ -70,7 +70,9 @@ impl PeakEwmaLatency { /// Should only be called from the Web3Rpc that owns it. pub fn report(&self, duration: Duration) { match self.request_tx.try_send(duration) { - Ok(true) => {} + Ok(true) => { + trace!("success"); + } Ok(false) => { // We don't want to block if the channel is full, just // report the error @@ -113,6 +115,8 @@ impl PeakEwmaLatencyTask { fn update(&mut self, rtt: Duration) { let rtt = nanos(rtt); + info!("updating rtt: {}ns", rtt); + let now = Instant::now(); debug_assert!( self.update_at <= now, diff --git a/web3_proxy/src/rpcs/consensus.rs b/web3_proxy/src/rpcs/consensus.rs index e6261412..3b0d5060 100644 --- a/web3_proxy/src/rpcs/consensus.rs +++ b/web3_proxy/src/rpcs/consensus.rs @@ -407,7 +407,11 @@ impl ConsensusFinder { { Ok(parent_block) => block_to_check = parent_block, Err(err) => { - warn!("Problem fetching parent block of {:#?} during consensus finding: {:#?}", block_to_check, err); + warn!( + "Problem fetching parent block of {:?} during consensus finding: {:#?}", + block_to_check.hash(), + err + ); break; } } diff --git a/web3_proxy/src/rpcs/request.rs b/web3_proxy/src/rpcs/request.rs index 8103b5e6..18e4ca2e 100644 --- a/web3_proxy/src/rpcs/request.rs +++ b/web3_proxy/src/rpcs/request.rs @@ -7,7 +7,7 @@ use entities::revert_log; use entities::sea_orm_active_enums::Method; use ethers::providers::ProviderError; use ethers::types::{Address, Bytes}; -use log::{debug, error, trace, warn, Level}; +use log::{debug, error, info, trace, warn, Level}; use migration::sea_orm::{self, ActiveEnum, ActiveModelTrait}; use serde_json::json; use std::fmt; @@ -374,6 +374,7 @@ impl OpenRequestHandle { } } } else if let Some(peak_latency) = &self.rpc.peak_latency { + trace!("updating peak_latency: {}", latency.as_secs_f64()); peak_latency.report(latency); } else { unreachable!("peak_latency not initialized"); From 4b7f88367da13aad921df4fa1b6af8dc01ac7e74 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Fri, 12 May 2023 23:22:20 -0700 Subject: [PATCH 16/66] fix serializing /status --- web3_proxy/src/rpcs/consensus.rs | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/web3_proxy/src/rpcs/consensus.rs b/web3_proxy/src/rpcs/consensus.rs index 3b0d5060..8807895d 100644 --- a/web3_proxy/src/rpcs/consensus.rs +++ b/web3_proxy/src/rpcs/consensus.rs @@ -92,10 +92,20 @@ pub type RankedRpcMap = BTreeMap>>; pub struct ConsensusWeb3Rpcs { pub(crate) tier: u64, pub(crate) backups_needed: bool, + + // TODO: this is already inside best_rpcs. give that a shorter serialize here and then include this again + #[serde(skip_serializing)] pub(crate) head_block: Web3ProxyBlock, + + // TODO: smaller serialize pub(crate) best_rpcs: Vec>, + + // TODO: make this work. the key needs to be a string + #[serde(skip_serializing)] pub(crate) other_rpcs: RankedRpcMap, + // TODO: make this work. the key needs to be a string + #[serde(skip_serializing)] rpc_data: HashMap, RpcData>, } From 1639405f6d337cf55787da0e8149bfccc7971621 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Fri, 12 May 2023 23:27:19 -0700 Subject: [PATCH 17/66] remove loud log --- latency/src/peak_ewma/mod.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/latency/src/peak_ewma/mod.rs b/latency/src/peak_ewma/mod.rs index a6c248d4..94dca1ab 100644 --- a/latency/src/peak_ewma/mod.rs +++ b/latency/src/peak_ewma/mod.rs @@ -115,8 +115,6 @@ impl PeakEwmaLatencyTask { fn update(&mut self, rtt: Duration) { let rtt = nanos(rtt); - info!("updating rtt: {}ns", rtt); - let now = Instant::now(); debug_assert!( self.update_at <= now, From c3cdf7ef4360985448006f717508ab3b0401954f Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Fri, 12 May 2023 23:40:15 -0700 Subject: [PATCH 18/66] fix consensus finding bug --- latency/src/peak_ewma/mod.rs | 5 ++++- web3_proxy/src/rpcs/blockchain.rs | 2 +- web3_proxy/src/rpcs/consensus.rs | 4 +++- web3_proxy/src/rpcs/one.rs | 6 +++++- 4 files changed, 13 insertions(+), 4 deletions(-) diff --git a/latency/src/peak_ewma/mod.rs b/latency/src/peak_ewma/mod.rs index 94dca1ab..640318c9 100644 --- a/latency/src/peak_ewma/mod.rs +++ b/latency/src/peak_ewma/mod.rs @@ -122,8 +122,11 @@ impl PeakEwmaLatencyTask { self.update_at, ); - self.rtt_estimate + let x = self + .rtt_estimate .fetch_update(|mut rtt_estimate| rtt_estimate.update(rtt, self.decay_ns, now)); + + info!("x: {:?}", x); } } diff --git a/web3_proxy/src/rpcs/blockchain.rs b/web3_proxy/src/rpcs/blockchain.rs index dc1ef8c2..f0c1bdbd 100644 --- a/web3_proxy/src/rpcs/blockchain.rs +++ b/web3_proxy/src/rpcs/blockchain.rs @@ -435,7 +435,7 @@ impl Web3Rpcs { Ok(Some(x)) => x, }; - trace!("new_synced_connections: {:?}", new_synced_connections); + trace!("new_synced_connections: {:#?}", new_synced_connections); let watch_consensus_head_sender = self.watch_consensus_head_sender.as_ref().unwrap(); let consensus_tier = new_synced_connections.tier; diff --git a/web3_proxy/src/rpcs/consensus.rs b/web3_proxy/src/rpcs/consensus.rs index 8807895d..a5658522 100644 --- a/web3_proxy/src/rpcs/consensus.rs +++ b/web3_proxy/src/rpcs/consensus.rs @@ -375,9 +375,11 @@ impl ConsensusFinder { trace!("first_tier: {}", current_tier); + trace!("rpc_heads_by_tier: {:#?}", rpc_heads_by_tier); + // loop over all the rpc heads (grouped by tier) and their parents to find consensus // TODO: i'm sure theres a lot of shortcuts that could be taken, but this is simplest to implement - for (rpc, rpc_head) in self.rpc_heads.iter() { + for (rpc, rpc_head) in rpc_heads_by_tier.into_iter() { if current_tier != rpc.tier { // we finished processing a tier. check for primary results if let Some(consensus) = self.count_votes(&primary_votes, web3_rpcs) { diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index ee792049..be0487e0 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -236,7 +236,11 @@ impl Web3Rpc { } pub fn peak_ewma(&self) -> OrderedFloat { - let peak_latency = self.peak_latency.as_ref().unwrap().latency().as_secs_f64(); + let peak_latency = if let Some(peak_latency) = self.peak_latency.as_ref() { + peak_latency.latency().as_secs_f64() + } else { + 0.0 + }; // TODO: what ordering? let active_requests = self.active_requests.load(atomic::Ordering::Acquire) as f64 + 1.0; From b493f02c3d173d7d2373da0cbb3e00ea1f55c78e Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Sat, 13 May 2023 09:56:31 -0700 Subject: [PATCH 19/66] put relaxed back --- latency/src/peak_ewma/rtt_estimate.rs | 4 ++-- latency/src/util/atomic_f32_pair.rs | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/latency/src/peak_ewma/rtt_estimate.rs b/latency/src/peak_ewma/rtt_estimate.rs index c169ea4a..be56fe9c 100644 --- a/latency/src/peak_ewma/rtt_estimate.rs +++ b/latency/src/peak_ewma/rtt_estimate.rs @@ -97,7 +97,7 @@ impl AtomicRttEstimate { /// This method omits the ordering argument since loads may use /// slightly stale data to avoid adding additional latency. pub fn load(&self) -> RttEstimate { - RttEstimate::from_pair(self.pair.load(Ordering::Acquire), self.start_time) + RttEstimate::from_pair(self.pair.load(Ordering::Relaxed), self.start_time) } /// Fetches the value, and applies a function to it that returns an @@ -114,7 +114,7 @@ impl AtomicRttEstimate { let mut update_at = Instant::now(); let mut rtt = Duration::ZERO; self.pair - .fetch_update(Ordering::Release, Ordering::Acquire, |pair| { + .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |pair| { rtt = f(RttEstimate::from_pair(pair, self.start_time)); // Save the new update_at inside the function in case it // is run multiple times diff --git a/latency/src/util/atomic_f32_pair.rs b/latency/src/util/atomic_f32_pair.rs index 18b356c3..fa74fa0b 100644 --- a/latency/src/util/atomic_f32_pair.rs +++ b/latency/src/util/atomic_f32_pair.rs @@ -69,7 +69,7 @@ mod tests { fn test_atomic_f32_pair_load() { let pair = [f32::consts::PI, f32::consts::E]; let atomic = AtomicF32Pair::new(pair); - assert_eq!(pair, atomic.load(Ordering::Acquire)); + assert_eq!(pair, atomic.load(Ordering::Relaxed)); } #[test] @@ -77,13 +77,13 @@ mod tests { let pair = [f32::consts::PI, f32::consts::E]; let atomic = AtomicF32Pair::new(pair); atomic - .fetch_update(Ordering::Release, Ordering::Acquire, |[f1, f2]| { + .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |[f1, f2]| { Some([f1 + 1.0, f2 + 1.0]) }) .unwrap(); assert_eq!( [pair[0] + 1.0, pair[1] + 1.0], - atomic.load(Ordering::Acquire) + atomic.load(Ordering::Relaxed) ); } } From 20807398654412b0a0fbcae9cc3ee77390eaafbe Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Sat, 13 May 2023 11:13:02 -0700 Subject: [PATCH 20/66] Quick cache (#70) * quick cache and allocate less * improve /status cache * prepare to cache raw transaction hashes so we dont dos our backends * simple benchmark for /health and /status * mut not needed with atomics * DRY all the status pages * use u64 instead of bytes for subscriptions * fix setting earliest_retry_at and improve logs * Revert "use kanal instead of flume or tokio channels (#68)" This reverts commit 510612d343fc51338a8a4282dcc229b50097835b. * fix automatic retries * put relaxed back * convert error message time to seconds * assert instead of debug_assert while we debug * ns instead of seconds * disable peak_latency for now * null is the default * cargo fmt * comments * remove request caching for now * log on exit * unit weigher for now * make cache smaller. we need a weigher for prod. just debugging * oops. we need async * add todo * no need for to_string on a RawValue --- Cargo.lock | 143 +++- README.md | 10 +- TODO.md | 2 +- latency/Cargo.toml | 1 - latency/src/peak_ewma/mod.rs | 47 +- migration/src/lib.rs | 4 +- rate-counter/Cargo.toml | 2 +- web3_proxy/Cargo.toml | 4 +- web3_proxy/src/app/mod.rs | 470 +++++------- web3_proxy/src/app/ws.rs | 20 +- .../src/bin/web3_proxy_cli/sentryd/mod.rs | 7 +- web3_proxy/src/config.rs | 4 +- web3_proxy/src/frontend/authorization.rs | 10 +- web3_proxy/src/frontend/errors.rs | 706 +++++++++--------- web3_proxy/src/frontend/mod.rs | 29 +- web3_proxy/src/frontend/rpc_proxy_http.rs | 2 + web3_proxy/src/frontend/rpc_proxy_ws.rs | 44 +- web3_proxy/src/frontend/status.rs | 139 +++- .../src/frontend/users/authentication.rs | 2 +- web3_proxy/src/jsonrpc.rs | 77 +- web3_proxy/src/lib.rs | 1 + web3_proxy/src/response_cache.rs | 137 ++++ web3_proxy/src/rpcs/blockchain.rs | 33 +- web3_proxy/src/rpcs/consensus.rs | 4 +- web3_proxy/src/rpcs/many.rs | 297 ++++---- web3_proxy/src/rpcs/one.rs | 44 +- web3_proxy/src/rpcs/request.rs | 25 +- web3_proxy/src/stats/mod.rs | 1 + web3_proxy/src/stats/stat_buffer.rs | 18 +- 29 files changed, 1282 insertions(+), 1001 deletions(-) create mode 100644 web3_proxy/src/response_cache.rs diff --git a/Cargo.lock b/Cargo.lock index 8753b543..9b4c1ba9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -68,6 +68,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2c99f64d1e06488f620f932677e24bc6e2897582980441ae90a671415bd7ec2f" dependencies = [ "cfg-if", + "getrandom", "once_cell", "version_check", ] @@ -792,6 +793,16 @@ dependencies = [ "cc", ] +[[package]] +name = "codespan-reporting" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3538270d33cc669650c4b093848450d380def10c331d38c768e34cac80576e6e" +dependencies = [ + "termcolor", + "unicode-width", +] + [[package]] name = "coins-bip32" version = "0.8.3" @@ -1194,6 +1205,50 @@ dependencies = [ "cipher 0.4.4", ] +[[package]] +name = "cxx" +version = "1.0.94" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f61f1b6389c3fe1c316bf8a4dccc90a38208354b330925bce1f74a6c4756eb93" +dependencies = [ + "cc", + "cxxbridge-flags", + "cxxbridge-macro", + "link-cplusplus", +] + +[[package]] +name = "cxx-build" +version = "1.0.94" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "12cee708e8962df2aeb38f594aae5d827c022b6460ac71a7a3e2c3c2aae5a07b" +dependencies = [ + "cc", + "codespan-reporting", + "once_cell", + "proc-macro2", + "quote", + "scratch", + "syn 2.0.15", +] + +[[package]] +name = "cxxbridge-flags" +version = "1.0.94" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7944172ae7e4068c533afbb984114a56c46e9ccddda550499caa222902c7f7bb" + +[[package]] +name = "cxxbridge-macro" +version = "1.0.94" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2345488264226bf682893e25de0769f3360aac9957980ec49361b083ddaa5bc5" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.15", +] + [[package]] name = "dashmap" version = "4.0.2" @@ -1998,6 +2053,19 @@ dependencies = [ "miniz_oxide 0.7.1", ] +[[package]] +name = "flume" +version = "0.10.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1657b4441c3403d9f7b3409e47575237dac27b1b5726df654a6ecbf92f0f7577" +dependencies = [ + "futures-core", + "futures-sink", + "nanorand", + "pin-project", + "spin 0.9.8", +] + [[package]] name = "fnv" version = "1.0.7" @@ -2309,9 +2377,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.3.19" +version = "0.3.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d357c7ae988e7d2182f7d7871d0b963962420b0678b0997ce7de72001aeab782" +checksum = "17f8a914c2987b688368b5138aa05321db91f4090cf26118185672ad588bce21" dependencies = [ "bytes", "fnv", @@ -2645,11 +2713,12 @@ dependencies = [ [[package]] name = "iana-time-zone-haiku" -version = "0.1.2" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" +checksum = "0703ae284fc167426161c2e3f1da3ea71d94b21bedbcc9494e92b28e334e3dca" dependencies = [ - "cc", + "cxx", + "cxx-build", ] [[package]] @@ -2908,16 +2977,6 @@ dependencies = [ "signature 2.1.0", ] -[[package]] -name = "kanal" -version = "0.1.0-pre8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b05d55519627edaf7fd0f29981f6dc03fb52df3f5b257130eb8d0bf2801ea1d7" -dependencies = [ - "futures-core", - "lock_api", -] - [[package]] name = "keccak" version = "0.1.4" @@ -2963,7 +3022,6 @@ name = "latency" version = "0.1.0" dependencies = [ "ewma", - "kanal", "log", "serde", "tokio", @@ -3012,6 +3070,15 @@ dependencies = [ "vcpkg", ] +[[package]] +name = "link-cplusplus" +version = "1.0.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ecd207c9c713c34f95a097a5b029ac2ce6010530c7b49d7fea24d977dede04f5" +dependencies = [ + "cc", +] + [[package]] name = "linux-raw-sys" version = "0.3.7" @@ -3208,6 +3275,15 @@ dependencies = [ "uuid 1.3.2", ] +[[package]] +name = "nanorand" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6a51313c5820b0b02bd422f4b44776fbf47961755c74ce64afc73bfad10226c3" +dependencies = [ + "getrandom", +] + [[package]] name = "native-tls" version = "0.2.11" @@ -4138,6 +4214,17 @@ dependencies = [ "unicase", ] +[[package]] +name = "quick_cache" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5253a3a0d56548d5b0be25414171dc780cc6870727746d05bd2bde352eee96c5" +dependencies = [ + "ahash 0.8.3", + "hashbrown 0.13.2", + "parking_lot 0.12.1", +] + [[package]] name = "quote" version = "1.0.27" @@ -4202,7 +4289,7 @@ dependencies = [ name = "rate-counter" version = "0.1.0" dependencies = [ - "kanal", + "flume", "tokio", ] @@ -4248,9 +4335,9 @@ dependencies = [ [[package]] name = "rdkafka-sys" -version = "4.4.0+1.9.2" +version = "4.3.0+1.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87ac9d87c3aba1748e3112318459f2ac8bff80bfff7359e338e0463549590249" +checksum = "d222a401698c7f2010e3967353eae566d9934dcda49c29910da922414ab4e3f4" dependencies = [ "cmake", "libc", @@ -4702,6 +4789,12 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" +[[package]] +name = "scratch" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1792db035ce95be60c3f8853017b3999209281c24e2ba5bc8e59bf97a0c590c1" + [[package]] name = "scrypt" version = "0.10.0" @@ -4923,9 +5016,9 @@ dependencies = [ [[package]] name = "security-framework" -version = "2.9.0" +version = "2.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca2855b3715770894e67cbfa3df957790aa0c9edc3bf06efa1a84d77fa0839d1" +checksum = "a332be01508d814fed64bf28f798a146d73792121129962fdf335bb3c49a4254" dependencies = [ "bitflags", "core-foundation", @@ -4936,9 +5029,9 @@ dependencies = [ [[package]] name = "security-framework-sys" -version = "2.9.0" +version = "2.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f51d0c0d83bec45f16480d0ce0058397a69e48fcdc52d1dc8855fb68acbd31a7" +checksum = "31c9bb296072e961fcbd8853511dd39c2d8be2deb1e17c6860b1d30732b323b4" dependencies = [ "core-foundation-sys", "libc", @@ -6577,6 +6670,7 @@ dependencies = [ "ethers", "ewma", "fdlimit", + "flume", "fstrings", "futures", "gethostname", @@ -6591,7 +6685,6 @@ dependencies = [ "influxdb2-structmap", "ipnet", "itertools", - "kanal", "latency", "listenfd", "log", @@ -6606,6 +6699,7 @@ dependencies = [ "parking_lot 0.12.1", "prettytable", "proctitle", + "quick_cache", "rdkafka", "redis-rate-limiter", "regex", @@ -6616,6 +6710,7 @@ dependencies = [ "serde_json", "serde_prometheus", "siwe", + "strum", "thread-fast-rng", "time 0.3.21", "tokio", diff --git a/README.md b/README.md index 4461169e..5c78789f 100644 --- a/README.md +++ b/README.md @@ -60,13 +60,13 @@ Check that the websocket is working: ``` $ websocat ws://127.0.0.1:8544 -{"id": 1, "method": "eth_subscribe", "params": ["newHeads"]} +{"jsonrpc": "2.0", "id": 1, "method": "eth_subscribe", "params": ["newHeads"]} -{"id": 2, "method": "eth_subscribe", "params": ["newPendingTransactions"]} +{"jsonrpc": "2.0", "id": 2, "method": "eth_subscribe", "params": ["newPendingTransactions"]} -{"id": 3, "method": "eth_subscribe", "params": ["newPendingFullTransactions"]} +{"jsonrpc": "2.0", "id": 3, "method": "eth_subscribe", "params": ["newPendingFullTransactions"]} -{"id": 4, "method": "eth_subscribe", "params": ["newPendingRawTransactions"]} +{"jsonrpc": "2.0", "id": 4, "method": "eth_subscribe", "params": ["newPendingRawTransactions"]} ``` You can copy `config/example.toml` to `config/production-$CHAINNAME.toml` and then run `docker-compose up --build -d` start proxies for many chains. @@ -149,6 +149,8 @@ TODO: also enable debug symbols in the release build by modifying the root Cargo Test the proxy: + wrk -t12 -c400 -d30s --latency http://127.0.0.1:8544/health + wrk -t12 -c400 -d30s --latency http://127.0.0.1:8544/status wrk -s ./wrk/getBlockNumber.lua -t12 -c400 -d30s --latency http://127.0.0.1:8544/u/$API_KEY wrk -s ./wrk/getLatestBlockByNumber.lua -t12 -c400 -d30s --latency http://127.0.0.1:8544/u/$API_KEY diff --git a/TODO.md b/TODO.md index 690007b4..b405b251 100644 --- a/TODO.md +++ b/TODO.md @@ -189,7 +189,7 @@ These are roughly in order of completition - [x] graceful shutdown. stop taking new requests and don't stop until all outstanding queries are handled - https://github.com/tokio-rs/mini-redis/blob/master/src/shutdown.rs - we need this because we need to be sure all the queries are saved in the db. maybe put stuff in Drop - - need an kanal::watch on unflushed stats that we can subscribe to. wait for it to flip to true + - need an flume::watch on unflushed stats that we can subscribe to. wait for it to flip to true - [x] don't use unix timestamps for response_millis since leap seconds will confuse it - [x] config to allow origins even on the anonymous endpoints - [x] send logs to sentry diff --git a/latency/Cargo.toml b/latency/Cargo.toml index 8583e9ab..eb51eba9 100644 --- a/latency/Cargo.toml +++ b/latency/Cargo.toml @@ -7,7 +7,6 @@ edition = "2021" [dependencies] ewma = "0.1.1" -kanal = "0.1.0-pre8" log = "0.4.17" serde = { version = "1.0.163", features = [] } tokio = { version = "1.28.1", features = ["full"] } diff --git a/latency/src/peak_ewma/mod.rs b/latency/src/peak_ewma/mod.rs index 640318c9..e3b27cdb 100644 --- a/latency/src/peak_ewma/mod.rs +++ b/latency/src/peak_ewma/mod.rs @@ -2,8 +2,9 @@ mod rtt_estimate; use std::sync::Arc; -use kanal::SendError; -use log::{error, info, trace}; +use log::{error, info}; +use tokio::sync::mpsc; +use tokio::sync::mpsc::error::TrySendError; use tokio::task::JoinHandle; use tokio::time::{Duration, Instant}; @@ -19,7 +20,7 @@ pub struct PeakEwmaLatency { /// Join handle for the latency calculation task pub join_handle: JoinHandle<()>, /// Send to update with each request duration - request_tx: kanal::AsyncSender, + request_tx: mpsc::Sender, /// Latency average and last update time rtt_estimate: Arc, /// Decay time @@ -33,7 +34,7 @@ impl PeakEwmaLatency { /// average latency. pub fn spawn(decay_ns: f64, buf_size: usize, start_latency: Duration) -> Self { debug_assert!(decay_ns > 0.0, "decay_ns must be positive"); - let (request_tx, request_rx) = kanal::bounded_async(buf_size); + let (request_tx, request_rx) = mpsc::channel(buf_size); let rtt_estimate = Arc::new(AtomicRttEstimate::new(start_latency)); let task = PeakEwmaLatencyTask { request_rx, @@ -55,10 +56,10 @@ impl PeakEwmaLatency { let mut estimate = self.rtt_estimate.load(); let now = Instant::now(); - debug_assert!( + assert!( estimate.update_at <= now, - "update_at={:?} in the future", - estimate.update_at, + "update_at is {}ns in the future", + estimate.update_at.duration_since(now).as_nanos(), ); // Update the RTT estimate to account for decay since the last update. @@ -67,26 +68,20 @@ impl PeakEwmaLatency { /// Report latency from a single request /// - /// Should only be called from the Web3Rpc that owns it. + /// Should only be called with a duration from the Web3Rpc that owns it. pub fn report(&self, duration: Duration) { match self.request_tx.try_send(duration) { - Ok(true) => { - trace!("success"); - } - Ok(false) => { + Ok(()) => {} + Err(TrySendError::Full(_)) => { // We don't want to block if the channel is full, just // report the error error!("Latency report channel full"); // TODO: could we spawn a new tokio task to report tthis later? } - Err(SendError::Closed) => { + Err(TrySendError::Closed(_)) => { unreachable!("Owner should keep channel open"); } - Err(SendError::ReceiveClosed) => { - unreachable!("Receiver should keep channel open"); - } }; - //.expect("Owner should keep channel open"); } } @@ -94,7 +89,7 @@ impl PeakEwmaLatency { #[derive(Debug)] struct PeakEwmaLatencyTask { /// Receive new request timings for update - request_rx: kanal::AsyncReceiver, + request_rx: mpsc::Receiver, /// Current estimate and update time rtt_estimate: Arc, /// Last update time, used for decay calculation @@ -106,27 +101,25 @@ struct PeakEwmaLatencyTask { impl PeakEwmaLatencyTask { /// Run the loop for updating latency async fn run(mut self) { - while let Ok(rtt) = self.request_rx.recv().await { + while let Some(rtt) = self.request_rx.recv().await { self.update(rtt); } + info!("latency loop exited"); } /// Update the estimate object atomically. - fn update(&mut self, rtt: Duration) { + fn update(&self, rtt: Duration) { let rtt = nanos(rtt); let now = Instant::now(); - debug_assert!( + assert!( self.update_at <= now, - "update_at={:?} in the future", - self.update_at, + "update_at is {}ns in the future", + self.update_at.duration_since(now).as_nanos(), ); - let x = self - .rtt_estimate + self.rtt_estimate .fetch_update(|mut rtt_estimate| rtt_estimate.update(rtt, self.decay_ns, now)); - - info!("x: {:?}", x); } } diff --git a/migration/src/lib.rs b/migration/src/lib.rs index 182f5a1e..ae9adaf7 100644 --- a/migration/src/lib.rs +++ b/migration/src/lib.rs @@ -17,12 +17,12 @@ mod m20230119_204135_better_free_tier; mod m20230125_204810_stats_v2; mod m20230130_124740_read_only_login_logic; mod m20230130_165144_prepare_admin_imitation_pre_login; -mod m20230215_152254_admin_trail; -mod m20230307_002623_migrate_rpc_accounting_to_rpc_accounting_v2; mod m20230205_130035_create_balance; mod m20230205_133755_create_referrals; mod m20230214_134254_increase_balance_transactions; +mod m20230215_152254_admin_trail; mod m20230221_230953_track_spend; +mod m20230307_002623_migrate_rpc_accounting_to_rpc_accounting_v2; mod m20230412_171916_modify_secondary_user_add_primary_user; mod m20230422_172555_premium_downgrade_logic; mod m20230511_161214_remove_columns_statsv2_origin_and_method; diff --git a/rate-counter/Cargo.toml b/rate-counter/Cargo.toml index 6e25777a..7bf027e4 100644 --- a/rate-counter/Cargo.toml +++ b/rate-counter/Cargo.toml @@ -5,5 +5,5 @@ authors = ["Bryan Stitt "] edition = "2021" [dependencies] -kanal = "0.1.0-pre8" +flume = "0.10.14" tokio = { version = "1.28.1", features = ["time"] } diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index 4d4a926c..65a1cb87 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -43,6 +43,7 @@ env_logger = "0.10.0" ethers = { version = "2.0.4", default-features = false, features = ["rustls", "ws"] } ewma = "0.1.1" fdlimit = "0.2.1" +flume = "0.10.14" fstrings = "0.2" futures = { version = "0.3.28", features = ["thread-pool"] } gethostname = "0.4.2" @@ -57,7 +58,6 @@ influxdb2 = { git = "https://github.com/llamanodes/influxdb2", features = ["rust influxdb2-structmap = { git = "https://github.com/llamanodes/influxdb2/"} ipnet = "2.7.2" itertools = "0.10.5" -kanal = "0.1.0-pre8" listenfd = "1.0.1" log = "0.4.17" mimalloc = { version = "0.1.37", optional = true} @@ -70,6 +70,7 @@ pagerduty-rs = { version = "0.1.6", default-features = false, features = ["async parking_lot = { version = "0.12.1", features = ["arc_lock"] } prettytable = "*" proctitle = "0.1.1" +quick_cache = "0.3.0" rdkafka = { version = "0.29.0" } regex = "1.8.1" reqwest = { version = "0.11.17", default-features = false, features = ["json", "tokio-rustls"] } @@ -79,6 +80,7 @@ serde = { version = "1.0.163", features = [] } serde_json = { version = "1.0.96", default-features = false, features = ["alloc", "raw_value"] } serde_prometheus = "0.2.2" siwe = "0.5.0" +strum = { version = "0.24.1", features = ["derive"] } time = "0.3.21" tokio = { version = "1.28.1", features = ["full"] } tokio-console = { version = "*", optional = true } diff --git a/web3_proxy/src/app/mod.rs b/web3_proxy/src/app/mod.rs index ce1ac9ba..a8e8ad01 100644 --- a/web3_proxy/src/app/mod.rs +++ b/web3_proxy/src/app/mod.rs @@ -4,12 +4,16 @@ mod ws; use crate::block_number::{block_needed, BlockNeeded}; use crate::config::{AppConfig, TopConfig}; use crate::frontend::authorization::{ - Authorization, RequestMetadata, RequestOrMethod, RpcSecretKey, + Authorization, RequestMetadata, RequestOrMethod, ResponseOrBytes, RpcSecretKey, }; use crate::frontend::errors::{Web3ProxyError, Web3ProxyErrorContext, Web3ProxyResult}; use crate::frontend::rpc_proxy_ws::ProxyMode; use crate::jsonrpc::{ - JsonRpcForwardedResponse, JsonRpcForwardedResponseEnum, JsonRpcRequest, JsonRpcRequestEnum, + JsonRpcErrorData, JsonRpcForwardedResponse, JsonRpcForwardedResponseEnum, JsonRpcRequest, + JsonRpcRequestEnum, +}; +use crate::response_cache::{ + JsonRpcQueryCache, JsonRpcQueryCacheKey, JsonRpcQueryWeigher, JsonRpcResponseData, }; use crate::rpcs::blockchain::Web3ProxyBlock; use crate::rpcs::consensus::ConsensusWeb3Rpcs; @@ -46,15 +50,14 @@ use redis_rate_limiter::redis::AsyncCommands; use redis_rate_limiter::{redis, DeadpoolRuntime, RedisConfig, RedisPool, RedisRateLimiter}; use serde::Serialize; use serde_json::json; -use serde_json::value::to_raw_value; +use std::borrow::Cow; use std::fmt; -use std::hash::{Hash, Hasher}; use std::net::IpAddr; use std::num::NonZeroU64; use std::str::FromStr; use std::sync::{atomic, Arc}; use std::time::Duration; -use tokio::sync::{broadcast, mpsc, watch, Semaphore}; +use tokio::sync::{broadcast, watch, Semaphore}; use tokio::task::JoinHandle; use tokio::time::{sleep, timeout}; use ulid::Ulid; @@ -71,88 +74,6 @@ pub static APP_USER_AGENT: &str = concat!( // aggregate across 1 week pub const BILLING_PERIOD_SECONDS: i64 = 60 * 60 * 24 * 7; -#[derive(Debug, From)] -struct ResponseCacheKey { - // if none, this is cached until evicted - from_block: Option, - // to_block is only set when ranges of blocks are requested (like with eth_getLogs) - to_block: Option, - method: String, - params: Option, - cache_errors: bool, -} - -impl ResponseCacheKey { - fn weight(&self) -> usize { - let mut w = self.method.len(); - - if let Some(ref p) = self.params { - w += p.to_string().len(); - } - - w - } -} - -impl PartialEq for ResponseCacheKey { - fn eq(&self, other: &Self) -> bool { - if self.cache_errors != other.cache_errors { - return false; - } - - match (self.from_block.as_ref(), other.from_block.as_ref()) { - (None, None) => {} - (None, Some(_)) => { - return false; - } - (Some(_), None) => { - return false; - } - (Some(s), Some(o)) => { - if s != o { - return false; - } - } - } - - match (self.to_block.as_ref(), other.to_block.as_ref()) { - (None, None) => {} - (None, Some(_)) => { - return false; - } - (Some(_), None) => { - return false; - } - (Some(s), Some(o)) => { - if s != o { - return false; - } - } - } - - if self.method != other.method { - return false; - } - - self.params == other.params - } -} - -impl Eq for ResponseCacheKey {} - -impl Hash for ResponseCacheKey { - fn hash(&self, state: &mut H) { - self.from_block.as_ref().map(|x| x.hash()).hash(state); - self.to_block.as_ref().map(|x| x.hash()).hash(state); - self.method.hash(state); - self.params.as_ref().map(|x| x.to_string()).hash(state); - self.cache_errors.hash(state) - } -} - -type ResponseCache = - Cache; - pub type AnyhowJoinHandle = JoinHandle>; /// TODO: move this @@ -224,7 +145,7 @@ pub struct Web3ProxyApp { /// TODO: include another type so that we can use private miner relays that do not use JSONRPC requests pub private_rpcs: Option>, /// track JSONRPC responses - response_cache: ResponseCache, + pub jsonrpc_query_cache: JsonRpcQueryCache, /// rpc clients that subscribe to newHeads use this channel /// don't drop this or the sender will stop working /// TODO: broadcast channel instead? @@ -265,7 +186,7 @@ pub struct Web3ProxyApp { Cache, hashbrown::hash_map::DefaultHashBuilder>, pub kafka_producer: Option, /// channel for sending stats in a background task - pub stat_sender: Option>, + pub stat_sender: Option>, } /// flatten a JoinError into an anyhow error @@ -695,23 +616,12 @@ impl Web3ProxyApp { // responses can be very different in sizes, so this is a cache with a max capacity and a weigher // TODO: don't allow any response to be bigger than X% of the cache - let response_cache = Cache::builder() - .max_capacity(top_config.app.response_cache_max_bytes) - .weigher(|k: &ResponseCacheKey, v| { - // TODO: is this good enough? - if let Ok(v) = serde_json::to_string(v) { - let weight = k.weight() + v.len(); - - // the or in unwrap_or is probably never called - weight.try_into().unwrap_or(u32::MAX) - } else { - // this seems impossible - u32::MAX - } - }) - // TODO: what should we set? 10 minutes is arbitrary. the nodes themselves hold onto transactions for much longer - .time_to_live(Duration::from_secs(600)) - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); + // TODO: we should emit stats to calculate a more accurate expected cache size + let response_cache = JsonRpcQueryCache::with_weighter( + (top_config.app.response_cache_max_bytes / 2048) as usize, + top_config.app.response_cache_max_bytes, + JsonRpcQueryWeigher, + ); // create semaphores for concurrent connection limits // TODO: what should tti be for semaphores? @@ -816,7 +726,7 @@ impl Web3ProxyApp { http_client, kafka_producer, private_rpcs, - response_cache, + jsonrpc_query_cache: response_cache, watch_consensus_head_receiver, pending_tx_sender, pending_transactions, @@ -1229,8 +1139,7 @@ impl Web3ProxyApp { authorization: &Arc, request: &JsonRpcRequest, request_metadata: Arc, - num_public_rpcs: Option, - ) -> Web3ProxyResult { + ) -> Web3ProxyResult { if let Some(protected_rpcs) = self.private_rpcs.as_ref() { if !protected_rpcs.is_empty() { let protected_response = protected_rpcs @@ -1250,6 +1159,17 @@ impl Web3ProxyApp { } } + let num_public_rpcs = match authorization.checks.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 @@ -1277,6 +1197,11 @@ impl Web3ProxyApp { // TODO: move this code to another module so that its easy to turn this trace logging on in dev trace!("Received request: {:?}", request); + // save the id so we can attach it to the response + // TODO: we don't always need to clone this. if we come from the cache, we can just take from the request + // TODO: store on the request_metadata? + let response_id = request.id.clone(); + let request_metadata = RequestMetadata::new( self, authorization.clone(), @@ -1285,7 +1210,7 @@ impl Web3ProxyApp { ) .await; - let (status_code, response) = match self + let (status_code, response_data): (_, JsonRpcResponseData) = match self ._proxy_cached_request(authorization, request, head_block_num, &request_metadata) .await { @@ -1293,7 +1218,10 @@ impl Web3ProxyApp { Err(err) => err.into_response_parts(), }; - request_metadata.add_response(&response); + let response = JsonRpcForwardedResponse::from_response_data(response_data, response_id); + + // TODO: this serializes twice :/ + request_metadata.add_response(ResponseOrBytes::Response(&response)); // TODO: with parallel request sending, I think there could be a race on this let rpcs = request_metadata.backend_rpcs_used(); @@ -1308,14 +1236,12 @@ impl Web3ProxyApp { request: &mut JsonRpcRequest, head_block_num: Option, request_metadata: &Arc, - ) -> Web3ProxyResult { - // save the id so we can attach it to the response - let request_id = request.id.clone(); - // TODO: don't clone + ) -> Web3ProxyResult { + // TODO: don't clone? let request_method = request.method.clone(); // TODO: serve net_version without querying the backend - let response: JsonRpcForwardedResponse = match request_method.as_ref() { + let response_data: JsonRpcResponseData = match request_method.as_ref() { // lots of commands are blocked method @ ("db_getHex" | "db_getString" @@ -1324,6 +1250,9 @@ impl Web3ProxyApp { | "debug_accountRange" | "debug_backtraceAt" | "debug_blockProfile" + | "debug_bundler_clearState" + | "debug_bundler_dumpMempool" + | "debug_bundler_sendBundleNow" | "debug_chaindbCompact" | "debug_chaindbProperty" | "debug_cpuProfile" @@ -1337,8 +1266,8 @@ impl Web3ProxyApp { | "debug_setGCPercent" | "debug_setHead" | "debug_setMutexProfileFraction" - | "debug_standardTraceBlockToFile" | "debug_standardTraceBadBlockToFile" + | "debug_standardTraceBlockToFile" | "debug_startCPUProfile" | "debug_startGoTrace" | "debug_stopCPUProfile" @@ -1346,6 +1275,7 @@ impl Web3ProxyApp { | "debug_writeBlockProfile" | "debug_writeMemProfile" | "debug_writeMutexProfile" + | "erigon_cacheCheck" | "eth_compileLLL" | "eth_compileSerpent" | "eth_compileSolidity" @@ -1355,24 +1285,23 @@ impl Web3ProxyApp { | "eth_signTransaction" | "eth_submitHashrate" | "eth_submitWork" - | "erigon_cacheCheck" | "les_addBalance" | "les_setClientParams" | "les_setDefaultParams" + | "miner_setEtherbase" | "miner_setExtra" + | "miner_setGasLimit" | "miner_setGasPrice" | "miner_start" | "miner_stop" - | "miner_setEtherbase" - | "miner_setGasLimit" + | "personal_ecRecover" | "personal_importRawKey" | "personal_listAccounts" | "personal_lockAccount" | "personal_newAccount" - | "personal_unlockAccount" | "personal_sendTransaction" | "personal_sign" - | "personal_ecRecover" + | "personal_unlockAccount" | "shh_addToGroup" | "shh_getFilterChanges" | "shh_getMessages" @@ -1383,13 +1312,12 @@ impl Web3ProxyApp { | "shh_post" | "shh_uninstallFilter" | "shh_version") => { - // i don't think we will ever support these methods + // i don't think we will ever support these methods. maybe do Forbidden? // TODO: what error code? - JsonRpcForwardedResponse::from_string( - format!("method unsupported: {}", method), - None, - Some(request_id), - ) + JsonRpcErrorData::from(format!( + "the method {} does not exist/is not available", + method + )).into() } // TODO: implement these commands method @ ("eth_getFilterChanges" @@ -1401,21 +1329,11 @@ impl Web3ProxyApp { | "eth_uninstallFilter") => { // TODO: unsupported command stat. use the count to prioritize new features // TODO: what error code? - JsonRpcForwardedResponse::from_string( - format!("not yet implemented: {}", method), - None, - Some(request_id), - ) - } - method @ ("debug_bundler_sendBundleNow" - | "debug_bundler_clearState" - | "debug_bundler_dumpMempool") => { - JsonRpcForwardedResponse::from_string( - // TODO: we should probably have some escaping on this. but maybe serde will protect us enough - format!("method unsupported: {}", method), - None, - Some(request_id), - ) + JsonRpcErrorData::from(format!( + "the method {} is not yet implemented. contact us if you need this", + method + )) + .into() } _method @ ("eth_sendUserOperation" | "eth_estimateUserOperationGas" @@ -1435,18 +1353,14 @@ impl Web3ProxyApp { } None => { // TODO: stats even when we error! - // TODO: use Web3ProxyError? dedicated error for no 4337 bundlers - return Err(anyhow::anyhow!("no bundler_4337_rpcs available").into()); + // TODO: dedicated error for no 4337 bundlers + return Err(Web3ProxyError::NoServersSynced); } }, - "eth_accounts" => { - JsonRpcForwardedResponse::from_value(serde_json::Value::Array(vec![]), request_id) - } + "eth_accounts" => JsonRpcResponseData::from(serde_json::Value::Array(vec![])), "eth_blockNumber" => { match head_block_num.or(self.balanced_rpcs.head_block_num()) { - Some(head_block_num) => { - JsonRpcForwardedResponse::from_value(json!(head_block_num), request_id) - } + Some(head_block_num) => JsonRpcResponseData::from(json!(head_block_num)), None => { // TODO: what does geth do if this happens? // TODO: standard not synced error @@ -1454,19 +1368,16 @@ impl Web3ProxyApp { } } } - "eth_chainId" => JsonRpcForwardedResponse::from_value( - json!(U64::from(self.config.chain_id)), - request_id, - ), + "eth_chainId" => JsonRpcResponseData::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" => { // no need for serving coinbase - JsonRpcForwardedResponse::from_value(json!(Address::zero()), request_id) + JsonRpcResponseData::from(json!(Address::zero())) } "eth_estimateGas" => { - let mut response = self + let response_data = self .balanced_rpcs .try_proxy_connection( authorization, @@ -1477,8 +1388,8 @@ impl Web3ProxyApp { ) .await?; - if let Some(gas_estimate) = response.result.take() { - let mut gas_estimate: U256 = serde_json::from_str(gas_estimate.get()) + if let JsonRpcResponseData::Result { value, .. } = response_data { + let mut gas_estimate: U256 = serde_json::from_str(value.get()) .or(Err(Web3ProxyError::GasEstimateNotU256))?; let gas_increase = if let Some(gas_increase_percent) = @@ -1495,14 +1406,14 @@ impl Web3ProxyApp { gas_estimate += gas_increase; - JsonRpcForwardedResponse::from_value(json!(gas_estimate), request_id) + JsonRpcResponseData::from(json!(gas_estimate)) } else { - response + response_data } } "eth_getTransactionReceipt" | "eth_getTransactionByHash" => { // try to get the transaction without specifying a min_block_height - let mut response = self + let mut response_data = self .balanced_rpcs .try_proxy_connection( authorization, @@ -1514,13 +1425,13 @@ impl Web3ProxyApp { .await?; // if we got "null", it is probably because the tx is old. retry on nodes with old block data - if let Some(ref result) = response.result { - if result.get() == "null" { + if let JsonRpcResponseData::Result { value, .. } = &response_data { + if value.get() == "null" { request_metadata .archive_request .store(true, atomic::Ordering::Release); - response = self + response_data = self .balanced_rpcs .try_proxy_connection( authorization, @@ -1533,44 +1444,39 @@ impl Web3ProxyApp { } } - response + response_data } // TODO: eth_gasPrice that does awesome magic to predict the future - "eth_hashrate" => JsonRpcForwardedResponse::from_value(json!(U64::zero()), request_id), - "eth_mining" => { - JsonRpcForwardedResponse::from_value(serde_json::Value::Bool(false), request_id) - } + "eth_hashrate" => JsonRpcResponseData::from(json!(U64::zero())), + "eth_mining" => JsonRpcResponseData::from(serde_json::Value::Bool(false)), // TODO: eth_sendBundle (flashbots/eden command) // broadcast transactions to all private rpcs at once "eth_sendRawTransaction" => { - let num_public_rpcs = match authorization.checks.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, - }; + // TODO: decode the transaction - let mut response = self + // TODO: error if the chain_id is incorrect + + // TODO: check the cache to see if we have sent this transaction recently + // TODO: if so, use a cached response. + // TODO: if not, + // TODO: - cache successes for up to 1 minute + // TODO: - cache failures for 1 block (we see transactions skipped because out of funds. but that can change block to block) + + let mut response_data = self .try_send_protected( authorization, request, request_metadata.clone(), - num_public_rpcs, ) .await?; // 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 Some(ref response_error) = response.error { - if response_error.code == -32000 - && (response_error.message == "ALREADY_EXISTS: already known" - || response_error.message - == "INTERNAL_ERROR: existing tx with same hash") + if let JsonRpcResponseData::Error { value, .. } = &response_data { + if value.code == -32000 + && (value.message == "ALREADY_EXISTS: already known" + || value.message == "INTERNAL_ERROR: existing tx with same hash") { let params = request .params @@ -1608,21 +1514,19 @@ impl Web3ProxyApp { trace!("tx_hash: {:#?}", tx_hash); - let tx_hash = to_raw_value(&tx_hash).unwrap(); - - response.error = None; - response.result = Some(tx_hash); + response_data = JsonRpcResponseData::from(tx_hash); } } } // emit transaction count stats + // TODO: use this cache to avoid sending duplicate transactions? if let Some(ref salt) = self.config.public_recent_ips_salt { - if let Some(ref tx_hash) = response.result { + if let JsonRpcResponseData::Result { value, .. } = &response_data { let now = Utc::now().timestamp(); let app = self.clone(); - let salted_tx_hash = format!("{}:{}", salt, tx_hash); + let salted_tx_hash = format!("{}:{}", salt, value.get()); let f = async move { match app.redis_conn().await { @@ -1653,35 +1557,38 @@ impl Web3ProxyApp { } } - response + response_data } "eth_syncing" => { // no stats on this. its cheap // TODO: return a real response if all backends are syncing or if no servers in sync - JsonRpcForwardedResponse::from_value(serde_json::Value::Bool(false), request_id) + // TODO: const + JsonRpcResponseData::from(serde_json::Value::Bool(false)) } - "eth_subscribe" => JsonRpcForwardedResponse::from_str( - "notifications not supported. eth_subscribe is only available over a websocket", - Some(-32601), - Some(request_id), - ), - "eth_unsubscribe" => JsonRpcForwardedResponse::from_str( - "notifications not supported. eth_unsubscribe is only available over a websocket", - Some(-32601), - Some(request_id), - ), + "eth_subscribe" => JsonRpcErrorData { + message: Cow::Borrowed( + "notifications not supported. eth_subscribe is only available over a websocket", + ), + code: -32601, + data: None, + } + .into(), + "eth_unsubscribe" => JsonRpcErrorData { + message: Cow::Borrowed("notifications not supported. eth_unsubscribe is only available over a websocket"), + code: -32601, + data: None, + }.into(), "net_listening" => { // TODO: only true if there are some backends on balanced_rpcs? - JsonRpcForwardedResponse::from_value(serde_json::Value::Bool(true), request_id) + // TODO: const + JsonRpcResponseData::from(serde_json::Value::Bool(true)) } - "net_peerCount" => JsonRpcForwardedResponse::from_value( - json!(U64::from(self.balanced_rpcs.num_synced_rpcs())), - request_id, - ), - "web3_clientVersion" => JsonRpcForwardedResponse::from_value( - serde_json::Value::String(APP_USER_AGENT.to_string()), - request_id, - ), + "net_peerCount" => + JsonRpcResponseData::from(json!(U64::from(self.balanced_rpcs.num_synced_rpcs()))) + , + "web3_clientVersion" => + JsonRpcResponseData::from(serde_json::Value::String(APP_USER_AGENT.to_string())) + , "web3_sha3" => { // returns Keccak-256 (not the standardized SHA3-256) of the given data. match &request.params { @@ -1692,11 +1599,11 @@ impl Web3ProxyApp { { // TODO: what error code? // TODO: use Web3ProxyError::BadRequest - JsonRpcForwardedResponse::from_str( - "Invalid request", - Some(-32600), - Some(request_id), - ) + JsonRpcErrorData { + message: Cow::Borrowed("Invalid request"), + code: -32600, + data: None + }.into() } else { // TODO: BadRequest instead of web3_context let param = Bytes::from_str( @@ -1714,25 +1621,25 @@ impl Web3ProxyApp { let hash = H256::from(keccak256(param)); - JsonRpcForwardedResponse::from_value(json!(hash), request_id) + JsonRpcResponseData::from(json!(hash)) } } _ => { // TODO: this needs the correct error code in the response // TODO: Web3ProxyError::BadRequest instead? - JsonRpcForwardedResponse::from_str( - "invalid request", - Some(StatusCode::BAD_REQUEST.as_u16().into()), - Some(request_id), - ) + JsonRpcErrorData { + message: Cow::Borrowed("invalid request"), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }.into() } } } - "test" => JsonRpcForwardedResponse::from_str( - "The method test does not exist/is not available.", - Some(-32601), - Some(request_id), - ), + "test" => JsonRpcErrorData { + message: Cow::Borrowed("The method test does not exist/is not available."), + code: -32601, + data: None, + }.into(), // anything else gets sent to backend rpcs and cached method => { if method.starts_with("admin_") { @@ -1746,12 +1653,12 @@ impl Web3ProxyApp { .ok_or(Web3ProxyError::NoServersSynced)?; // TODO: don't clone. this happens way too much. maybe &mut? - let mut request = request.clone(); + // let mut request = request.clone(); // we do this check before checking caches because it might modify the request params // TODO: add a stat for archive vs full since they should probably cost different // TODO: this cache key can be rather large. is that okay? - let cache_key: Option = match block_needed( + let cache_key: Option = match block_needed( authorization, method, request.params.as_mut(), @@ -1760,7 +1667,7 @@ impl Web3ProxyApp { ) .await? { - BlockNeeded::CacheSuccessForever => Some(ResponseCacheKey { + BlockNeeded::CacheSuccessForever => Some(JsonRpcQueryCacheKey { from_block: None, to_block: None, method: method.to_string(), @@ -1786,9 +1693,10 @@ impl Web3ProxyApp { let request_block = self .balanced_rpcs .block(authorization, &request_block_hash, None) - .await?; + .await? + .block; - Some(ResponseCacheKey { + Some(JsonRpcQueryCacheKey { from_block: Some(request_block), to_block: None, method: method.to_string(), @@ -1816,7 +1724,8 @@ impl Web3ProxyApp { let from_block = self .balanced_rpcs .block(authorization, &from_block_hash, None) - .await?; + .await? + .block; let (to_block_hash, _) = self .balanced_rpcs @@ -1826,75 +1735,62 @@ impl Web3ProxyApp { let to_block = self .balanced_rpcs .block(authorization, &to_block_hash, None) - .await?; + .await? + .block; - Some(ResponseCacheKey { + Some(JsonRpcQueryCacheKey { from_block: Some(from_block), to_block: Some(to_block), method: method.to_string(), - // TODO: hash here? params: request.params.clone(), cache_errors, }) } }; - trace!("cache_key: {:#?}", cache_key); - let mut response = { - let request_metadata = request_metadata.clone(); + let authorization = authorization.clone(); - let authorization = authorization.clone(); + if let Some(cache_key) = cache_key { + let from_block_num = cache_key.from_block.as_ref().map(|x| x.number.unwrap()); + let to_block_num = cache_key.to_block.as_ref().map(|x| x.number.unwrap()); - if let Some(cache_key) = cache_key { - let from_block_num = cache_key.from_block.as_ref().map(|x| *x.number()); - let to_block_num = cache_key.to_block.as_ref().map(|x| *x.number()); + match self + .jsonrpc_query_cache + .get_value_or_guard_async(&cache_key).await + { + Ok(x) => x, + Err(x) => { + let response_data = self.balanced_rpcs + .try_proxy_connection( + &authorization, + request, + Some(request_metadata), + from_block_num.as_ref(), + to_block_num.as_ref(), + ) + .await?; - self.response_cache - .try_get_with(cache_key, async move { - // TODO: put the hash here instead of the block number? its in the request already. - let mut response = self - .balanced_rpcs - .try_proxy_connection( - &authorization, - &request, - Some(&request_metadata), - from_block_num.as_ref(), - to_block_num.as_ref(), - ) - .await?; + // TODO: convert the Box to an Arc + x.insert(response_data.clone()); - // discard their id by replacing it with an empty - response.id = Default::default(); - - // TODO: only cache the inner response - // TODO: how are we going to stream this? - // TODO: check response size. if its very large, return it in a custom Error type that bypasses caching? or will moka do that for us? - Ok::<_, Web3ProxyError>(response) - }) - // TODO: add context (error while caching and forwarding response {}) - .await? - } else { - self.balanced_rpcs - .try_proxy_connection( - &authorization, - &request, - Some(&request_metadata), - None, - None, - ) - .await? + response_data + } } - }; - - // since this data likely came out of a cache, the response.id is not going to match the request.id - // replace the id with our request's id. - response.id = request_id; - - response + } else { + self.balanced_rpcs + .try_proxy_connection( + &authorization, + request, + Some(request_metadata), + None, + None, + ) + .await? + } } }; - Ok(response) + Ok(response_data) } } diff --git a/web3_proxy/src/app/ws.rs b/web3_proxy/src/app/ws.rs index 6e8e61f1..07ba9b59 100644 --- a/web3_proxy/src/app/ws.rs +++ b/web3_proxy/src/app/ws.rs @@ -5,9 +5,10 @@ use crate::frontend::authorization::{Authorization, RequestMetadata, RequestOrMe use crate::frontend::errors::{Web3ProxyError, Web3ProxyResult}; use crate::jsonrpc::JsonRpcForwardedResponse; use crate::jsonrpc::JsonRpcRequest; +use crate::response_cache::JsonRpcResponseData; use crate::rpcs::transactions::TxStatus; use axum::extract::ws::Message; -use ethers::prelude::U64; +use ethers::types::U64; use futures::future::AbortHandle; use futures::future::Abortable; use futures::stream::StreamExt; @@ -24,7 +25,7 @@ impl Web3ProxyApp { jsonrpc_request: JsonRpcRequest, subscription_count: &'a AtomicUsize, // TODO: taking a sender for Message instead of the exact json we are planning to send feels wrong, but its easier for now - response_sender: kanal::AsyncSender, + response_sender: flume::Sender, ) -> Web3ProxyResult<(AbortHandle, JsonRpcForwardedResponse)> { let request_metadata = RequestMetadata::new( self, @@ -39,7 +40,7 @@ impl Web3ProxyApp { // TODO: this only needs to be unique per connection. we don't need it globably unique // TODO: have a max number of subscriptions per key/ip. have a global max number of subscriptions? how should this be calculated? let subscription_id = subscription_count.fetch_add(1, atomic::Ordering::SeqCst); - let subscription_id = U64::from(subscription_id); + let subscription_id = U64::from(subscription_id as u64); // save the id so we can use it in the response let id = jsonrpc_request.id.clone(); @@ -94,7 +95,7 @@ impl Web3ProxyApp { // TODO: can we check a content type header? let response_msg = Message::Text(response_str); - if response_sender.send(response_msg).await.is_err() { + if response_sender.send_async(response_msg).await.is_err() { // TODO: increment error_response? i don't think so. i think this will happen once every time a client disconnects. // TODO: cancel this subscription earlier? select on head_block_receiver.next() and an abort handle? break; @@ -158,7 +159,7 @@ impl Web3ProxyApp { // TODO: do clients support binary messages? let response_msg = Message::Text(response_str); - if response_sender.send(response_msg).await.is_err() { + if response_sender.send_async(response_msg).await.is_err() { // TODO: cancel this subscription earlier? select on head_block_receiver.next() and an abort handle? break; }; @@ -221,7 +222,7 @@ impl Web3ProxyApp { // TODO: do clients support binary messages? let response_msg = Message::Text(response_str); - if response_sender.send(response_msg).await.is_err() { + if response_sender.send_async(response_msg).await.is_err() { // TODO: cancel this subscription earlier? select on head_block_receiver.next() and an abort handle? break; }; @@ -285,7 +286,7 @@ impl Web3ProxyApp { // TODO: do clients support binary messages? let response_msg = Message::Text(response_str); - if response_sender.send(response_msg).await.is_err() { + if response_sender.send_async(response_msg).await.is_err() { // TODO: cancel this subscription earlier? select on head_block_receiver.next() and an abort handle? break; }; @@ -304,8 +305,11 @@ impl Web3ProxyApp { // TODO: do something with subscription_join_handle? - let response = JsonRpcForwardedResponse::from_value(json!(subscription_id), id); + let response_data = JsonRpcResponseData::from(json!(subscription_id)); + let response = JsonRpcForwardedResponse::from_response_data(response_data, id); + + // TODO: this serializes twice request_metadata.add_response(&response); // TODO: make a `SubscriptonHandle(AbortHandle, JoinHandle)` struct? diff --git a/web3_proxy/src/bin/web3_proxy_cli/sentryd/mod.rs b/web3_proxy/src/bin/web3_proxy_cli/sentryd/mod.rs index a1d9f50d..5e0af642 100644 --- a/web3_proxy/src/bin/web3_proxy_cli/sentryd/mod.rs +++ b/web3_proxy/src/bin/web3_proxy_cli/sentryd/mod.rs @@ -11,6 +11,7 @@ use log::{error, info}; use pagerduty_rs::{eventsv2async::EventsV2 as PagerdutyAsyncEventsV2, types::Event}; use serde_json::json; use std::time::Duration; +use tokio::sync::mpsc; use tokio::time::{interval, MissedTickBehavior}; use web3_proxy::{config::TopConfig, pagerduty::pagerduty_alert}; @@ -115,7 +116,7 @@ impl SentrydSubCommand { let mut handles = FuturesUnordered::new(); // channels and a task for sending errors to logs/pagerduty - let (error_sender, error_receiver) = kanal::bounded_async::(10); + let (error_sender, mut error_receiver) = mpsc::channel::(10); { let error_handler_f = async move { @@ -123,7 +124,7 @@ impl SentrydSubCommand { info!("set PAGERDUTY_INTEGRATION_KEY to send create alerts for errors"); } - while let Ok(err) = error_receiver.recv().await { + while let Some(err) = error_receiver.recv().await { log::log!(err.level, "check failed: {:#?}", err); if matches!(err.level, log::Level::Error) { @@ -257,7 +258,7 @@ async fn a_loop( class: &str, seconds: u64, error_level: log::Level, - error_sender: kanal::AsyncSender, + error_sender: mpsc::Sender, f: impl Fn(SentrydErrorBuilder) -> T, ) -> anyhow::Result<()> where diff --git a/web3_proxy/src/config.rs b/web3_proxy/src/config.rs index 51fd4099..05a947d5 100644 --- a/web3_proxy/src/config.rs +++ b/web3_proxy/src/config.rs @@ -285,8 +285,8 @@ impl Web3RpcConfig { http_client: Option, http_interval_sender: Option>>, blocks_by_hash_cache: BlocksByHashCache, - block_sender: Option>, - tx_id_sender: Option>, + block_sender: Option>, + tx_id_sender: Option>, reconnect: bool, ) -> anyhow::Result<(Arc, AnyhowJoinHandle<()>)> { if !self.extra.is_empty() { diff --git a/web3_proxy/src/frontend/authorization.rs b/web3_proxy/src/frontend/authorization.rs index 6ba772f0..34a1b398 100644 --- a/web3_proxy/src/frontend/authorization.rs +++ b/web3_proxy/src/frontend/authorization.rs @@ -33,7 +33,7 @@ use std::mem; use std::sync::atomic::{self, AtomicBool, AtomicI64, AtomicU64, AtomicUsize}; use std::time::Duration; use std::{net::IpAddr, str::FromStr, sync::Arc}; -use tokio::sync::{mpsc, OwnedSemaphorePermit, Semaphore}; +use tokio::sync::{OwnedSemaphorePermit, Semaphore}; use tokio::task::JoinHandle; use tokio::time::Instant; use ulid::Ulid; @@ -266,8 +266,8 @@ pub struct RequestMetadata { /// TODO: maybe this shouldn't be determined by ProxyMode. A request param should probably enable this pub kafka_debug_logger: Option>, - /// Cancel-safe channel to send stats to - pub stat_sender: Option>, + /// Cancel-safe channel for sending stats to the buffer + pub stat_sender: Option>, } impl Default for RequestMetadata { @@ -355,7 +355,9 @@ impl ResponseOrBytes<'_> { Self::Json(x) => serde_json::to_string(x) .expect("this should always serialize") .len(), - Self::Response(x) => x.num_bytes(), + Self::Response(x) => serde_json::to_string(x) + .expect("this should always serialize") + .len(), Self::Bytes(num_bytes) => *num_bytes, } } diff --git a/web3_proxy/src/frontend/errors.rs b/web3_proxy/src/frontend/errors.rs index 1ae29e23..e16b674e 100644 --- a/web3_proxy/src/frontend/errors.rs +++ b/web3_proxy/src/frontend/errors.rs @@ -1,10 +1,12 @@ //! Utlities for logging errors for admins and displaying errors to users. use super::authorization::Authorization; -use crate::jsonrpc::JsonRpcForwardedResponse; +use crate::jsonrpc::{JsonRpcErrorData, JsonRpcForwardedResponse}; +use crate::response_cache::JsonRpcResponseData; -use std::net::IpAddr; +use std::error::Error; use std::sync::Arc; +use std::{borrow::Cow, net::IpAddr}; use axum::{ headers, @@ -51,6 +53,7 @@ pub enum Web3ProxyError { EthersHttpClientError(ethers::prelude::HttpClientError), EthersProviderError(ethers::prelude::ProviderError), EthersWsClientError(ethers::prelude::WsClientError), + FlumeRecvError(flume::RecvError), GasEstimateNotU256, Headers(headers::Error), HeaderToString(ToStrError), @@ -64,7 +67,7 @@ pub enum Web3ProxyError { InvalidHeaderValue(InvalidHeaderValue), InvalidEip, InvalidInviteCode, - InvalidReferralCode, + UnknownReferralCode, InvalidReferer, InvalidSignatureLength, InvalidUserAgent, @@ -76,11 +79,6 @@ pub enum Web3ProxyError { JoinError(JoinError), #[display(fmt = "{:?}", _0)] #[error(ignore)] - JsonRpcForwardedError(JsonRpcForwardedResponse), - KanalReceiveError(kanal::ReceiveError), - KanalSendError(kanal::SendError), - #[display(fmt = "{:?}", _0)] - #[error(ignore)] MsgPackEncode(rmp_serde::encode::Error), NoBlockNumberOrHash, NoBlocksKnown, @@ -113,6 +111,7 @@ pub enum Web3ProxyError { #[from(ignore)] RefererNotAllowed(headers::Referer), SemaphoreAcquireError(AcquireError), + SendAppStatError(flume::SendError), SerdeJson(serde_json::Error), /// simple way to return an error message to the user and an anyhow to our logs #[display(fmt = "{}, {}, {:?}", _0, _1, _2)] @@ -139,42 +138,41 @@ pub enum Web3ProxyError { } impl Web3ProxyError { - pub fn into_response_parts(self) -> (StatusCode, JsonRpcForwardedResponse) { - match self { + pub fn into_response_parts(self) -> (StatusCode, JsonRpcResponseData) { + let (code, err): (StatusCode, JsonRpcErrorData) = match self { Self::AccessDenied => { // TODO: attach something to this trace. probably don't include much in the message though. don't want to leak creds by accident trace!("access denied"); ( StatusCode::FORBIDDEN, - JsonRpcForwardedResponse::from_string( - // TODO: is it safe to expose all of our anyhow strings? - "FORBIDDEN".to_string(), - Some(StatusCode::FORBIDDEN.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("FORBIDDEN"), + code: StatusCode::FORBIDDEN.as_u16().into(), + data: None, + }, ) } Self::Anyhow(err) => { warn!("anyhow. err={:?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_string( + JsonRpcErrorData { // TODO: is it safe to expose all of our anyhow strings? - err.to_string(), - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + message: Cow::Owned(err.to_string()), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } Self::BadRequest(err) => { debug!("BAD_REQUEST: {}", err); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - &format!("bad request: {}", err), - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(format!("bad request: {}", err)), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::BadResponse(err) => { @@ -182,33 +180,33 @@ impl Web3ProxyError { debug!("BAD_RESPONSE: {}", err); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_str( - &format!("bad response: {}", err), - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(format!("bad response: {}", err)), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } Self::BadRouting => { error!("BadRouting"); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_str( - "bad routing", - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("bad routing"), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } Self::Database(err) => { error!("database err={:?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_str( - "database error!", - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("database error!"), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } Self::EipVerificationFailed(err_1, err_191) => { @@ -218,81 +216,81 @@ impl Web3ProxyError { ); ( StatusCode::UNAUTHORIZED, - JsonRpcForwardedResponse::from_string( - format!( + JsonRpcErrorData { + message: Cow::Owned(format!( "both the primary and eip191 verification failed: {:#?}; {:#?}", err_1, err_191 - ), - Some(StatusCode::UNAUTHORIZED.as_u16().into()), - None, - ), + )), + code: StatusCode::UNAUTHORIZED.as_u16().into(), + data: None, + }, ) } Self::EthersHttpClientError(err) => { warn!("EthersHttpClientError err={:?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_str( - "ether http client error", - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("ether http client error"), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } Self::EthersProviderError(err) => { warn!("EthersProviderError err={:?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_str( - "ether provider error", - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("ether provider error"), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } Self::EthersWsClientError(err) => { warn!("EthersWsClientError err={:?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_str( - "ether ws client error", - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("ether ws client error"), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } - Self::KanalReceiveError(err) => { - warn!("KanalRecvError err={:#?}", err); + Self::FlumeRecvError(err) => { + warn!("FlumeRecvError err={:#?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_str( - "flume recv error!", - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("flume recv error!"), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } - Self::JsonRpcForwardedError(x) => (StatusCode::OK, x), + // Self::JsonRpcForwardedError(x) => (StatusCode::OK, x), Self::GasEstimateNotU256 => { warn!("GasEstimateNotU256"); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_str( - "gas estimate result is not an U256", - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("gas estimate result is not an U256"), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } Self::Headers(err) => { warn!("HeadersError {:?}", err); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - &format!("{}", err), - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(format!("{}", err)), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::InfluxDb2RequestError(err) => { @@ -300,135 +298,135 @@ impl Web3ProxyError { error!("influxdb2 err={:?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_str( - "influxdb2 error!", - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("influxdb2 error!"), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } Self::InvalidBlockBounds { min, max } => { debug!("InvalidBlockBounds min={} max={}", min, max); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_string( - format!( + JsonRpcErrorData { + message: Cow::Owned(format!( "Invalid blocks bounds requested. min ({}) > max ({})", min, max - ), - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + )), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::IpAddrParse(err) => { - warn!("IpAddrParse err={:?}", err); + debug!("IpAddrParse err={:?}", err); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - &format!("{}", err), - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(err.to_string()), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::IpNotAllowed(ip) => { - warn!("IpNotAllowed ip={})", ip); + debug!("IpNotAllowed ip={})", ip); ( StatusCode::FORBIDDEN, - JsonRpcForwardedResponse::from_string( - format!("IP ({}) is not allowed!", ip), - Some(StatusCode::FORBIDDEN.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(format!("IP ({}) is not allowed!", ip)), + code: StatusCode::FORBIDDEN.as_u16().into(), + data: None, + }, ) } Self::InvalidHeaderValue(err) => { - warn!("InvalidHeaderValue err={:?}", err); + debug!("InvalidHeaderValue err={:?}", err); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - &format!("{}", err), - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(format!("{}", err)), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::InvalidEip => { - warn!("InvalidEip"); + debug!("InvalidEip"); ( - StatusCode::UNAUTHORIZED, - JsonRpcForwardedResponse::from_str( - "invalid message eip given", - Some(StatusCode::UNAUTHORIZED.as_u16().into()), - None, - ), + StatusCode::BAD_REQUEST, + JsonRpcErrorData { + message: Cow::Borrowed("invalid message eip given"), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::InvalidInviteCode => { - warn!("InvalidInviteCode"); + debug!("InvalidInviteCode"); ( StatusCode::UNAUTHORIZED, - JsonRpcForwardedResponse::from_str( - "invalid invite code", - Some(StatusCode::UNAUTHORIZED.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("invalid invite code"), + code: StatusCode::UNAUTHORIZED.as_u16().into(), + data: None, + }, ) } - Self::InvalidReferralCode => { - warn!("InvalidReferralCode"); + Self::UnknownReferralCode => { + debug!("UnknownReferralCode"); ( StatusCode::UNAUTHORIZED, - JsonRpcForwardedResponse::from_str( - "invalid referral code", - Some(StatusCode::UNAUTHORIZED.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("invalid referral code"), + code: StatusCode::UNAUTHORIZED.as_u16().into(), + data: None, + }, ) } Self::InvalidReferer => { - warn!("InvalidReferer"); + debug!("InvalidReferer"); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - "invalid referer!", - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("invalid referer!"), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::InvalidSignatureLength => { - warn!("InvalidSignatureLength"); + debug!("InvalidSignatureLength"); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - "invalid signature length", - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("invalid signature length"), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::InvalidUserAgent => { - warn!("InvalidUserAgent"); + debug!("InvalidUserAgent"); ( - StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - "invalid user agent!", - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + StatusCode::FORBIDDEN, + JsonRpcErrorData { + message: Cow::Borrowed("invalid user agent!"), + code: StatusCode::FORBIDDEN.as_u16().into(), + data: None, + }, ) } Self::InvalidUserKey => { warn!("InvalidUserKey"); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - "UserKey was not a ULID or UUID", - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("UserKey was not a ULID or UUID"), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::JoinError(err) => { @@ -442,78 +440,78 @@ impl Web3ProxyError { ( code, - JsonRpcForwardedResponse::from_str( - // TODO: different messages, too? - "Unable to complete request", - Some(code.as_u16().into()), - None, - ), + JsonRpcErrorData { + // TODO: different messages of cancelled or not? + message: Cow::Borrowed("Unable to complete request"), + code: code.as_u16().into(), + data: None, + }, ) } Self::MsgPackEncode(err) => { warn!("MsgPackEncode Error: {}", err); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_str( - &format!("msgpack encode error: {}", err), - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(format!("msgpack encode error: {}", err)), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } Self::NoBlockNumberOrHash => { warn!("NoBlockNumberOrHash"); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_str( - "Blocks here must have a number or hash", - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("Blocks here must have a number or hash"), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } Self::NoBlocksKnown => { error!("NoBlocksKnown"); ( StatusCode::BAD_GATEWAY, - JsonRpcForwardedResponse::from_str( - "no blocks known", - Some(StatusCode::BAD_GATEWAY.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("no blocks known"), + code: StatusCode::BAD_GATEWAY.as_u16().into(), + data: None, + }, ) } Self::NoConsensusHeadBlock => { error!("NoConsensusHeadBlock"); ( StatusCode::BAD_GATEWAY, - JsonRpcForwardedResponse::from_str( - "no consensus head block", - Some(StatusCode::BAD_GATEWAY.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("no consensus head block"), + code: StatusCode::BAD_GATEWAY.as_u16().into(), + data: None, + }, ) } Self::NoHandleReady => { error!("NoHandleReady"); ( StatusCode::BAD_GATEWAY, - JsonRpcForwardedResponse::from_str( - "unable to retry for request handle", - Some(StatusCode::BAD_GATEWAY.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("unable to retry for request handle"), + code: StatusCode::BAD_GATEWAY.as_u16().into(), + data: None, + }, ) } Self::NoServersSynced => { warn!("NoServersSynced"); ( StatusCode::BAD_GATEWAY, - JsonRpcForwardedResponse::from_str( - "no servers synced", - Some(StatusCode::BAD_GATEWAY.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("no servers synced"), + code: StatusCode::BAD_GATEWAY.as_u16().into(), + data: None, + }, ) } Self::NotEnoughRpcs { @@ -523,11 +521,14 @@ impl Web3ProxyError { error!("NotEnoughRpcs {}/{}", num_known, min_head_rpcs); ( StatusCode::BAD_GATEWAY, - JsonRpcForwardedResponse::from_string( - format!("not enough rpcs connected {}/{}", num_known, min_head_rpcs), - Some(StatusCode::BAD_GATEWAY.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(format!( + "not enough rpcs connected {}/{}", + num_known, min_head_rpcs + )), + code: StatusCode::BAD_GATEWAY.as_u16().into(), + data: None, + }, ) } Self::NotFound => { @@ -535,88 +536,88 @@ impl Web3ProxyError { // TODO: instead of an error, show a normal html page for 404? ( StatusCode::NOT_FOUND, - JsonRpcForwardedResponse::from_str( - "not found!", - Some(StatusCode::NOT_FOUND.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("not found!"), + code: StatusCode::NOT_FOUND.as_u16().into(), + data: None, + }, ) } Self::NotImplemented => { trace!("NotImplemented"); ( StatusCode::NOT_IMPLEMENTED, - JsonRpcForwardedResponse::from_str( - "work in progress", - Some(StatusCode::NOT_IMPLEMENTED.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("work in progress"), + code: StatusCode::NOT_IMPLEMENTED.as_u16().into(), + data: None, + }, ) } Self::OriginRequired => { trace!("OriginRequired"); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - "Origin required", - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("Origin required"), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::OriginNotAllowed(origin) => { trace!("OriginNotAllowed origin={}", origin); ( StatusCode::FORBIDDEN, - JsonRpcForwardedResponse::from_string( - format!("Origin ({}) is not allowed!", origin), - Some(StatusCode::FORBIDDEN.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(format!("Origin ({}) is not allowed!", origin)), + code: StatusCode::FORBIDDEN.as_u16().into(), + data: None, + }, ) } Self::ParseBytesError(err) => { trace!("ParseBytesError err={:?}", err); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - "parse bytes error!", - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("parse bytes error!"), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::ParseMsgError(err) => { trace!("ParseMsgError err={:?}", err); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - "parse message error!", - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("parse message error!"), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::ParseAddressError => { trace!("ParseAddressError"); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - "unable to parse address", - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("unable to parse address"), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::PaymentRequired => { trace!("PaymentRequiredError"); ( StatusCode::PAYMENT_REQUIRED, - JsonRpcForwardedResponse::from_str( - "Payment is required and user is not premium.", - Some(StatusCode::PAYMENT_REQUIRED.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("Payment is required and user is not premium"), + code: StatusCode::PAYMENT_REQUIRED.as_u16().into(), + data: None, + }, ) } // TODO: this should actually by the id of the key. multiple users might control one key @@ -638,89 +639,91 @@ impl Web3ProxyError { format!( "too many requests from rpc key #{}.{}", authorization.checks.rpc_secret_key_id.unwrap(), - retry_msg + retry_msg, ) }; ( StatusCode::TOO_MANY_REQUESTS, - JsonRpcForwardedResponse::from_string( - msg, - Some(StatusCode::TOO_MANY_REQUESTS.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(msg), + code: StatusCode::TOO_MANY_REQUESTS.as_u16().into(), + data: None, + }, ) } Self::Redis(err) => { warn!("redis err={:?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_str( - "redis error!", - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("redis error!"), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } Self::RefererRequired => { warn!("referer required"); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - "Referer required", - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("Referer required"), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::RefererNotAllowed(referer) => { warn!("referer not allowed referer={:?}", referer); ( StatusCode::FORBIDDEN, - JsonRpcForwardedResponse::from_string( - format!("Referer ({:?}) is not allowed", referer), - Some(StatusCode::FORBIDDEN.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(format!("Referer ({:?}) is not allowed", referer)), + code: StatusCode::FORBIDDEN.as_u16().into(), + data: None, + }, ) } - Self::Arc(err) => match Arc::try_unwrap(err) { - Ok(err) => err, - Err(err) => Self::Anyhow(anyhow::anyhow!("{}", err)), + Self::Arc(err) => { + return match Arc::try_unwrap(err) { + Ok(err) => err, + Err(err) => Self::Anyhow(anyhow::anyhow!("{}", err)), + } + .into_response_parts(); } - .into_response_parts(), Self::SemaphoreAcquireError(err) => { warn!("semaphore acquire err={:?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_string( + JsonRpcErrorData { // TODO: is it safe to expose all of our anyhow strings? - "semaphore acquire error".to_string(), - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + message: Cow::Borrowed("semaphore acquire error"), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } - Self::KanalSendError(err) => { + Self::SendAppStatError(err) => { error!("SendAppStatError err={:?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_str( - "error stat_sender sending response_stat", - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("error stat_sender sending response_stat"), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } Self::SerdeJson(err) => { - warn!("serde json err={:?}", err); + warn!("serde json err={:?} source={:?}", err, err.source()); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - "de/serialization error!", - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(format!("de/serialization error! {}", err)), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::StatusCode(status_code, err_msg, err) => { @@ -734,80 +737,84 @@ impl Web3ProxyError { ( status_code, - JsonRpcForwardedResponse::from_str(&err_msg, Some(code.into()), None), + JsonRpcErrorData { + message: Cow::Owned(err_msg), + code: code.into(), + data: None, + }, ) } Self::Timeout(x) => ( StatusCode::REQUEST_TIMEOUT, - JsonRpcForwardedResponse::from_str( - &format!("request timed out: {:?}", x), - Some(StatusCode::REQUEST_TIMEOUT.as_u16().into()), + JsonRpcErrorData { + message: Cow::Owned(format!("request timed out: {:?}", x)), + code: StatusCode::REQUEST_TIMEOUT.as_u16().into(), // TODO: include the actual id! - None, - ), + data: None, + }, ), Self::HeaderToString(err) => { // trace!(?err, "HeaderToString"); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - &format!("{}", err), - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(err.to_string()), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::UlidDecode(err) => { // trace!(?err, "UlidDecodeError"); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - &format!("{}", err), - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(format!("{}", err)), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::UnknownBlockNumber => { error!("UnknownBlockNumber"); ( StatusCode::BAD_GATEWAY, - JsonRpcForwardedResponse::from_str( - "no servers synced. unknown eth_blockNumber", - Some(StatusCode::BAD_GATEWAY.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("no servers synced. unknown eth_blockNumber"), + code: StatusCode::BAD_GATEWAY.as_u16().into(), + data: None, + }, ) } // TODO: stat? Self::UnknownKey => ( StatusCode::UNAUTHORIZED, - JsonRpcForwardedResponse::from_str( - "unknown api key!", - Some(StatusCode::UNAUTHORIZED.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("unknown api key!"), + code: StatusCode::UNAUTHORIZED.as_u16().into(), + data: None, + }, ), Self::UserAgentRequired => { warn!("UserAgentRequired"); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - "User agent required", - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("User agent required"), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::UserAgentNotAllowed(ua) => { warn!("UserAgentNotAllowed ua={}", ua); ( StatusCode::FORBIDDEN, - JsonRpcForwardedResponse::from_string( - format!("User agent ({}) is not allowed!", ua), - Some(StatusCode::FORBIDDEN.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(format!("User agent ({}) is not allowed!", ua)), + code: StatusCode::FORBIDDEN.as_u16().into(), + data: None, + }, ) } Self::UserIdZero => { @@ -815,75 +822,79 @@ impl Web3ProxyError { // TODO: this might actually be an application error and not a BAD_REQUEST ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - "user ids should always be non-zero", - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("user ids should always be non-zero"), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::VerificationError(err) => { trace!("VerificationError err={:?}", err); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - "verification error!", - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("verification error!"), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::WatchRecvError(err) => { error!("WatchRecvError err={:?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_str( - "watch recv error!", - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("watch recv error!"), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } Self::WatchSendError => { error!("WatchSendError"); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_str( - "watch send error!", - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed("watch send error!"), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } Self::WebsocketOnly => { trace!("WebsocketOnly"); ( StatusCode::BAD_REQUEST, - JsonRpcForwardedResponse::from_str( - "redirect_public_url not set. only websockets work here", - Some(StatusCode::BAD_REQUEST.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Borrowed( + "redirect_public_url not set. only websockets work here", + ), + code: StatusCode::BAD_REQUEST.as_u16().into(), + data: None, + }, ) } Self::WithContext(err, msg) => match err { Some(err) => { warn!("{:#?} w/ context {}", err, msg); - err.into_response_parts() + return err.into_response_parts(); } None => { warn!("error w/ context {}", msg); ( StatusCode::INTERNAL_SERVER_ERROR, - JsonRpcForwardedResponse::from_string( - msg, - Some(StatusCode::INTERNAL_SERVER_ERROR.as_u16().into()), - None, - ), + JsonRpcErrorData { + message: Cow::Owned(msg), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, ) } }, - } + }; + + (code, JsonRpcResponseData::from(err)) } } @@ -903,7 +914,12 @@ impl IntoResponse for Web3ProxyError { fn into_response(self) -> Response { // TODO: include the request id in these so that users can give us something that will point to logs // TODO: status code is in the jsonrpc response and is also the first item in the tuple. DRY - let (status_code, response) = self.into_response_parts(); + let (status_code, response_data) = self.into_response_parts(); + + // this will be missing the jsonrpc id! + // its better to get request id and call from_response_data with it then to use this IntoResponse helper. + let response = + JsonRpcForwardedResponse::from_response_data(response_data, Default::default()); (status_code, Json(response)).into_response() } diff --git a/web3_proxy/src/frontend/mod.rs b/web3_proxy/src/frontend/mod.rs index 728978da..0aca1b02 100644 --- a/web3_proxy/src/frontend/mod.rs +++ b/web3_proxy/src/frontend/mod.rs @@ -16,26 +16,27 @@ use axum::{ routing::{get, post, put}, Extension, Router, }; -use http::header::AUTHORIZATION; +use http::{header::AUTHORIZATION, StatusCode}; use listenfd::ListenFd; use log::info; -use moka::future::Cache; +use std::iter::once; use std::net::SocketAddr; use std::sync::Arc; -use std::{iter::once, time::Duration}; -use tokio::sync::broadcast; +use strum::{EnumCount, EnumIter}; +use tokio::{sync::broadcast, time::Instant}; use tower_http::cors::CorsLayer; use tower_http::sensitive_headers::SetSensitiveRequestHeadersLayer; /// simple keys for caching responses -#[derive(Clone, Hash, PartialEq, Eq)] -pub enum FrontendResponseCaches { +#[derive(Copy, Clone, Hash, PartialEq, Eq, EnumCount, EnumIter)] +pub enum FrontendResponseCacheKey { + BackupsNeeded, + Health, Status, } pub type FrontendJsonResponseCache = - Cache, hashbrown::hash_map::DefaultHashBuilder>; -pub type FrontendHealthCache = Cache<(), bool, hashbrown::hash_map::DefaultHashBuilder>; + quick_cache::sync::Cache; /// Start the frontend server. pub async fn serve( @@ -47,14 +48,9 @@ pub async fn serve( // setup caches for whatever the frontend needs // no need for max items since it is limited by the enum key // TODO: latest moka allows for different ttls for different - let json_response_cache: FrontendJsonResponseCache = Cache::builder() - .time_to_live(Duration::from_secs(2)) - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); + let response_cache_size = FrontendResponseCacheKey::COUNT; - // /health gets a cache with a shorter lifetime - let health_cache: FrontendHealthCache = Cache::builder() - .time_to_live(Duration::from_millis(100)) - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); + let json_response_cache = FrontendJsonResponseCache::new(response_cache_size); // TODO: read config for if fastest/versus should be available publicly. default off @@ -220,8 +216,7 @@ pub async fn serve( // application state .layer(Extension(proxy_app)) // frontend caches - .layer(Extension(json_response_cache)) - .layer(Extension(health_cache)) + .layer(Extension(Arc::new(json_response_cache))) // 404 for any unknown routes .fallback(errors::handler_404); diff --git a/web3_proxy/src/frontend/rpc_proxy_http.rs b/web3_proxy/src/frontend/rpc_proxy_http.rs index 06b55603..1b39678c 100644 --- a/web3_proxy/src/frontend/rpc_proxy_http.rs +++ b/web3_proxy/src/frontend/rpc_proxy_http.rs @@ -63,6 +63,8 @@ async fn _proxy_web3_rpc( let authorization = Arc::new(authorization); + // TODO: calculate payload bytes here (before turning into serde_json::Value). that will save serializing later + let (status_code, response, rpcs, _semaphore) = app .proxy_web3_rpc(authorization, payload) .await diff --git a/web3_proxy/src/frontend/rpc_proxy_ws.rs b/web3_proxy/src/frontend/rpc_proxy_ws.rs index e562f297..03aa9843 100644 --- a/web3_proxy/src/frontend/rpc_proxy_ws.rs +++ b/web3_proxy/src/frontend/rpc_proxy_ws.rs @@ -20,7 +20,7 @@ use axum::{ }; use axum_client_ip::InsecureClientIp; use axum_macros::debug_handler; -use ethers::types::Bytes; +use ethers::types::U64; use fstrings::{f, format_args_f}; use futures::SinkExt; use futures::{ @@ -311,7 +311,7 @@ async fn proxy_web3_socket( let (ws_tx, ws_rx) = socket.split(); // create a channel for our reader and writer can communicate. todo: benchmark different channels - let (response_sender, response_receiver) = kanal::unbounded_async::(); + let (response_sender, response_receiver) = flume::unbounded::(); tokio::spawn(write_web3_socket(response_receiver, ws_tx)); tokio::spawn(read_web3_socket(app, authorization, ws_rx, response_sender)); @@ -323,25 +323,28 @@ async fn handle_socket_payload( app: Arc, authorization: &Arc, payload: &str, - response_sender: &kanal::AsyncSender, + response_sender: &flume::Sender, subscription_count: &AtomicUsize, - subscriptions: Arc>>, + subscriptions: Arc>>, ) -> Web3ProxyResult<(Message, Option)> { let (authorization, semaphore) = match authorization.check_again(&app).await { Ok((a, s)) => (a, s), Err(err) => { let (_, err) = err.into_response_parts(); - let err = serde_json::to_string(&err).expect("to_string should always work here"); + let err = JsonRpcForwardedResponse::from_response_data(err, Default::default()); + + let err = serde_json::to_string(&err)?; return Ok((Message::Text(err), None)); } }; // TODO: do any clients send batches over websockets? - let (id, response) = match serde_json::from_str::(payload) { + // TODO: change response into response_data + let (response_id, response) = match serde_json::from_str::(payload) { Ok(json_request) => { - let id = json_request.id.clone(); + let response_id = json_request.id.clone(); // TODO: move this to a seperate function so we can use the try operator let response: Web3ProxyResult = @@ -366,9 +369,9 @@ async fn handle_socket_payload( .as_ref() .context("there should be a result here")?; - // TODO: there must be a better way to do this - let k: Bytes = serde_json::from_str(result.get()) - .context("subscription ids must be bytes")?; + // TODO: there must be a better way to turn a RawValue + let k: U64 = serde_json::from_str(result.get()) + .context("subscription ids must be U64s")?; x.insert(k, handle); }; @@ -384,7 +387,7 @@ async fn handle_socket_payload( .await; #[derive(serde::Deserialize)] - struct EthUnsubscribeParams([Bytes; 1]); + struct EthUnsubscribeParams([U64; 1]); if let Some(params) = json_request.params { match serde_json::from_value(params) { @@ -403,9 +406,10 @@ async fn handle_socket_payload( } }; + // TODO: don't create the response here. use a JsonRpcResponseData instead let response = JsonRpcForwardedResponse::from_value( json!(partial_response), - id.clone(), + response_id.clone(), ); request_metadata.add_response(&response); @@ -428,7 +432,7 @@ async fn handle_socket_payload( .map(|(status_code, response, _)| response), }; - (id, response) + (response_id, response) } Err(err) => { let id = JsonRpcId::None.to_raw_value(); @@ -439,8 +443,10 @@ async fn handle_socket_payload( let response_str = match response { Ok(x) => serde_json::to_string(&x).expect("to_string should always work here"), Err(err) => { - let (_, mut response) = err.into_response_parts(); - response.id = id; + let (_, response_data) = err.into_response_parts(); + + let response = JsonRpcForwardedResponse::from_response_data(response_data, response_id); + serde_json::to_string(&response).expect("to_string should always work here") } }; @@ -452,7 +458,7 @@ async fn read_web3_socket( app: Arc, authorization: Arc, mut ws_rx: SplitStream, - response_sender: kanal::AsyncSender, + response_sender: flume::Sender, ) { // RwLock should be fine here. a user isn't going to be opening tons of subscriptions let subscriptions = Arc::new(RwLock::new(HashMap::new())); @@ -528,7 +534,7 @@ async fn read_web3_socket( } }; - if response_sender.send(response_msg).await.is_err() { + if response_sender.send_async(response_msg).await.is_err() { let _ = close_sender.send(true); return; }; @@ -549,13 +555,13 @@ async fn read_web3_socket( } async fn write_web3_socket( - response_rx: kanal::AsyncReceiver, + response_rx: flume::Receiver, mut ws_tx: SplitSink, ) { // TODO: increment counter for open websockets // TODO: is there any way to make this stream receive. - while let Ok(msg) = response_rx.recv().await { + while let Ok(msg) = response_rx.recv_async().await { // a response is ready // TODO: poke rate limits for this user? diff --git a/web3_proxy/src/frontend/status.rs b/web3_proxy/src/frontend/status.rs index 0e46c21a..4bc30aa3 100644 --- a/web3_proxy/src/frontend/status.rs +++ b/web3_proxy/src/frontend/status.rs @@ -3,32 +3,97 @@ //! For ease of development, users can currently access these endponts. //! They will eventually move to another port. -use super::{FrontendHealthCache, FrontendJsonResponseCache, FrontendResponseCaches}; -use crate::app::{Web3ProxyApp, APP_USER_AGENT}; -use axum::{http::StatusCode, response::IntoResponse, Extension, Json}; +use super::{FrontendJsonResponseCache, FrontendResponseCacheKey}; +use crate::{ + app::{Web3ProxyApp, APP_USER_AGENT}, + frontend::errors::Web3ProxyError, +}; +use axum::{body::Bytes, http::StatusCode, response::IntoResponse, Extension}; use axum_macros::debug_handler; +use futures::Future; +use once_cell::sync::Lazy; use serde_json::json; -use std::sync::Arc; +use std::{sync::Arc, time::Duration}; +use tokio::time::Instant; + +static HEALTH_OK: Lazy = Lazy::new(|| Bytes::from("OK\n")); +static HEALTH_NOT_OK: Lazy = Lazy::new(|| Bytes::from(":(\n")); + +static BACKUPS_NEEDED_TRUE: Lazy = Lazy::new(|| Bytes::from("true\n")); +static BACKUPS_NEEDED_FALSE: Lazy = Lazy::new(|| Bytes::from("false\n")); + +/// simple ttl for +// TODO: make this generic for any cache/key +async fn _quick_cache_ttl( + app: Arc, + cache: Arc, + key: FrontendResponseCacheKey, + f: impl Fn(Arc) -> Fut, +) -> (StatusCode, Bytes) +where + Fut: Future, +{ + let mut response; + let expire_at; + + (response, expire_at) = cache + .get_or_insert_async::(&key, async { + let expire_at = Instant::now() + Duration::from_millis(1000); + + let response = f(app.clone()).await; + + Ok((response, expire_at)) + }) + .await + .unwrap(); + + if Instant::now() >= expire_at { + // TODO: this expiration isn't perfect + // parallel requests could overwrite eachother + // its good enough for now + let expire_at = Instant::now() + Duration::from_millis(1000); + + response = f(app).await; + + cache.insert(key, (response.clone(), expire_at)); + } + + response +} /// Health check page for load balancers to use. #[debug_handler] pub async fn health( Extension(app): Extension>, - Extension(health_cache): Extension, + Extension(cache): Extension>, ) -> impl IntoResponse { - let synced = health_cache - .get_with((), async { app.balanced_rpcs.synced() }) - .await; + _quick_cache_ttl(app, cache, FrontendResponseCacheKey::Health, _health).await +} - if synced { - (StatusCode::OK, "OK") +async fn _health(app: Arc) -> (StatusCode, Bytes) { + if app.balanced_rpcs.synced() { + (StatusCode::OK, HEALTH_OK.clone()) } else { - (StatusCode::SERVICE_UNAVAILABLE, ":(") + (StatusCode::SERVICE_UNAVAILABLE, HEALTH_NOT_OK.clone()) } } /// Easy alerting if backup servers are in use. -pub async fn backups_needed(Extension(app): Extension>) -> impl IntoResponse { +#[debug_handler] +pub async fn backups_needed( + Extension(app): Extension>, + Extension(cache): Extension>, +) -> impl IntoResponse { + _quick_cache_ttl( + app, + cache, + FrontendResponseCacheKey::BackupsNeeded, + _backups_needed, + ) + .await +} + +async fn _backups_needed(app: Arc) -> (StatusCode, Bytes) { let code = { let consensus_rpcs = app .balanced_rpcs @@ -49,9 +114,9 @@ pub async fn backups_needed(Extension(app): Extension>) -> imp }; if matches!(code, StatusCode::OK) { - (code, "no backups needed. :)") + (code, BACKUPS_NEEDED_FALSE.clone()) } else { - (code, "backups needed! :(") + (code, BACKUPS_NEEDED_TRUE.clone()) } } @@ -61,23 +126,33 @@ pub async fn backups_needed(Extension(app): Extension>) -> imp #[debug_handler] pub async fn status( Extension(app): Extension>, - Extension(response_cache): Extension, + Extension(cache): Extension>, ) -> impl IntoResponse { - let body = response_cache - .get_with(FrontendResponseCaches::Status, async { - // TODO: what else should we include? uptime, cache hit rates, cpu load, memory used - // TODO: the hostname is probably not going to change. only get once at the start? - let body = json!({ - "version": APP_USER_AGENT, - "chain_id": app.config.chain_id, - "balanced_rpcs": app.balanced_rpcs, - "private_rpcs": app.private_rpcs, - "hostname": app.hostname, - }); - - Arc::new(body) - }) - .await; - - Json(body) + _quick_cache_ttl(app, cache, FrontendResponseCacheKey::Status, _status).await +} + +// TODO: this doesn't need to be async, but _quick_cache_ttl needs an async function +async fn _status(app: Arc) -> (StatusCode, Bytes) { + // TODO: what else should we include? uptime, cache hit rates, cpu load, memory used + // TODO: the hostname is probably not going to change. only get once at the start? + let body = json!({ + "version": APP_USER_AGENT, + "chain_id": app.config.chain_id, + "balanced_rpcs": app.balanced_rpcs, + "private_rpcs": app.private_rpcs, + "bundler_4337_rpcs": app.bundler_4337_rpcs, + "hostname": app.hostname, + }); + + let body = body.to_string().into_bytes(); + + let body = Bytes::from(body); + + let code = if app.balanced_rpcs.synced() { + StatusCode::OK + } else { + StatusCode::INTERNAL_SERVER_ERROR + }; + + (code, body) } diff --git a/web3_proxy/src/frontend/users/authentication.rs b/web3_proxy/src/frontend/users/authentication.rs index e681ea41..e4a70ca3 100644 --- a/web3_proxy/src/frontend/users/authentication.rs +++ b/web3_proxy/src/frontend/users/authentication.rs @@ -321,7 +321,7 @@ pub async fn user_login_post( .filter(referrer::Column::ReferralCode.eq(referral_code)) .one(db_replica.conn()) .await? - .ok_or(Web3ProxyError::InvalidReferralCode)?; + .ok_or(Web3ProxyError::UnknownReferralCode)?; // Create a new item in the database, // marking this guy as the referrer (and ignoring a duplicate insert, if there is any...) diff --git a/web3_proxy/src/jsonrpc.rs b/web3_proxy/src/jsonrpc.rs index eabd3d0b..677cc069 100644 --- a/web3_proxy/src/jsonrpc.rs +++ b/web3_proxy/src/jsonrpc.rs @@ -1,20 +1,17 @@ use crate::frontend::errors::{Web3ProxyError, Web3ProxyResult}; +use crate::response_cache::JsonRpcResponseData; use derive_more::From; use ethers::prelude::ProviderError; use serde::de::{self, Deserializer, MapAccess, SeqAccess, Visitor}; use serde::{Deserialize, Serialize}; use serde_json::json; use serde_json::value::{to_raw_value, RawValue}; +use std::borrow::Cow; use std::fmt; -fn default_jsonrpc() -> String { - "2.0".to_string() -} - +// TODO: &str here instead of String should save a lot of allocations #[derive(Clone, Deserialize, Serialize)] pub struct JsonRpcRequest { - // TODO: skip jsonrpc entirely? its against spec to drop it, but some servers bad - #[serde(default = "default_jsonrpc")] pub jsonrpc: String, /// id could be a stricter type, but many rpcs do things against the spec pub id: Box, @@ -51,7 +48,7 @@ impl JsonRpcRequest { params: Option, ) -> anyhow::Result { let x = Self { - jsonrpc: default_jsonrpc(), + jsonrpc: "2.0".to_string(), id: id.to_raw_value(), method, params, @@ -194,19 +191,38 @@ pub struct JsonRpcErrorData { /// The error code pub code: i64, /// The error message - pub message: String, + pub message: Cow<'static, str>, /// Additional data #[serde(skip_serializing_if = "Option::is_none")] pub data: Option, } +impl From<&'static str> for JsonRpcErrorData { + fn from(value: &'static str) -> Self { + Self { + code: -32000, + message: Cow::Borrowed(value), + data: None, + } + } +} + +impl From for JsonRpcErrorData { + fn from(value: String) -> Self { + Self { + code: -32000, + message: Cow::Owned(value), + data: None, + } + } +} + /// A complete response /// TODO: better Debug response #[derive(Clone, Debug, Deserialize, Serialize)] pub struct JsonRpcForwardedResponse { // TODO: jsonrpc a &str? - #[serde(default = "default_jsonrpc")] - pub jsonrpc: String, + pub jsonrpc: &'static str, pub id: Box, #[serde(skip_serializing_if = "Option::is_none")] pub result: Option>, @@ -242,40 +258,40 @@ impl JsonRpcForwardedResponse { // TODO: this is too verbose. plenty of errors are valid, like users giving an invalid address. no need to log that // TODO: can we somehow get the initial request here? if we put that into a tracing span, will things slow down a ton? JsonRpcForwardedResponse { - jsonrpc: "2.0".to_string(), - id: id.unwrap_or_else(|| JsonRpcId::None.to_raw_value()), + jsonrpc: "2.0", + id: id.unwrap_or_default(), result: None, error: Some(JsonRpcErrorData { code: code.unwrap_or(-32099), - message, + message: Cow::Owned(message), // TODO: accept data as an argument data: None, }), } } - pub fn from_response(partial_response: Box, id: Box) -> Self { + pub fn from_raw_response(result: Box, id: Box) -> Self { JsonRpcForwardedResponse { - jsonrpc: "2.0".to_string(), + jsonrpc: "2.0", id, // TODO: since we only use the result here, should that be all we return from try_send_request? - result: Some(partial_response), + result: Some(result), error: None, } } - pub fn from_value(partial_response: serde_json::Value, id: Box) -> Self { - let partial_response = - to_raw_value(&partial_response).expect("Value to RawValue should always work"); + pub fn from_value(result: serde_json::Value, id: Box) -> Self { + let partial_response = to_raw_value(&result).expect("Value to RawValue should always work"); JsonRpcForwardedResponse { - jsonrpc: "2.0".to_string(), + jsonrpc: "2.0", id, result: Some(partial_response), error: None, } } + // TODO: delete this. its on JsonRpcErrorData pub fn from_ethers_error(e: ProviderError, id: Box) -> Web3ProxyResult { // TODO: move turning ClientError into json to a helper function? let code; @@ -302,12 +318,12 @@ impl JsonRpcForwardedResponse { } Ok(Self { - jsonrpc: "2.0".to_string(), + jsonrpc: "2.0", id, result: None, error: Some(JsonRpcErrorData { code, - message, + message: Cow::Owned(message), data, }), }) @@ -318,16 +334,21 @@ impl JsonRpcForwardedResponse { id: Box, ) -> Web3ProxyResult { match result { - Ok(response) => Ok(Self::from_response(response, id)), + Ok(response) => Ok(Self::from_raw_response(response, id)), Err(e) => Self::from_ethers_error(e, id), } } - pub fn num_bytes(&self) -> usize { - // TODO: not sure how to do this without wasting a ton of allocations - serde_json::to_string(self) - .expect("this should always be valid json") - .len() + pub fn from_response_data(data: JsonRpcResponseData, id: Box) -> Self { + match data { + JsonRpcResponseData::Result { value, .. } => Self::from_raw_response(value, id), + JsonRpcResponseData::Error { value, .. } => JsonRpcForwardedResponse { + jsonrpc: "2.0", + id, + result: None, + error: Some(value), + }, + } } } diff --git a/web3_proxy/src/lib.rs b/web3_proxy/src/lib.rs index 1f29beb3..ce4ae400 100644 --- a/web3_proxy/src/lib.rs +++ b/web3_proxy/src/lib.rs @@ -8,6 +8,7 @@ pub mod jsonrpc; pub mod pagerduty; pub mod prometheus; pub mod referral_code; +pub mod response_cache; pub mod rpcs; pub mod stats; pub mod user_token; diff --git a/web3_proxy/src/response_cache.rs b/web3_proxy/src/response_cache.rs new file mode 100644 index 00000000..c7e710bd --- /dev/null +++ b/web3_proxy/src/response_cache.rs @@ -0,0 +1,137 @@ +use crate::{ + frontend::errors::Web3ProxyError, jsonrpc::JsonRpcErrorData, rpcs::blockchain::ArcBlock, +}; +use derive_more::From; +use ethers::providers::ProviderError; +use quick_cache::{sync::Cache as QuickCache, Weighter}; +use serde_json::value::RawValue; +use std::{ + borrow::Cow, + hash::{Hash, Hasher}, + num::NonZeroU32, +}; + +#[derive(Clone, Debug, From, PartialEq, Eq)] +pub struct JsonRpcQueryCacheKey { + pub from_block: Option, + pub to_block: Option, + pub method: String, + pub params: Option, + pub cache_errors: bool, +} + +impl Hash for JsonRpcQueryCacheKey { + fn hash(&self, state: &mut H) { + self.from_block.as_ref().map(|x| x.hash).hash(state); + self.to_block.as_ref().map(|x| x.hash).hash(state); + self.method.hash(state); + + // make sure preserve_order feature is OFF + self.params.as_ref().map(|x| x.to_string()).hash(state); + + self.cache_errors.hash(state) + } +} + +pub type JsonRpcQueryCache = + QuickCache; + +#[derive(Clone)] +pub struct JsonRpcQueryWeigher; + +#[derive(Clone)] +pub enum JsonRpcResponseData { + Result { + value: Box, + size: Option, + }, + Error { + value: JsonRpcErrorData, + size: Option, + }, +} + +impl JsonRpcResponseData { + pub fn num_bytes(&self) -> NonZeroU32 { + // TODO: dry this somehow + match self { + JsonRpcResponseData::Result { value, size } => size.unwrap_or_else(|| { + let size = value.get().len(); + + NonZeroU32::new(size.clamp(1, u32::MAX as usize) as u32).unwrap() + }), + JsonRpcResponseData::Error { value, size } => size.unwrap_or_else(|| { + let size = serde_json::to_string(value).unwrap().len(); + + NonZeroU32::new(size.clamp(1, u32::MAX as usize) as u32).unwrap() + }), + } + } +} + +impl From for JsonRpcResponseData { + fn from(value: serde_json::Value) -> Self { + let value = RawValue::from_string(value.to_string()).unwrap(); + + Self::Result { value, size: None } + } +} + +impl From> for JsonRpcResponseData { + fn from(value: Box) -> Self { + Self::Result { value, size: None } + } +} + +impl From for JsonRpcResponseData { + fn from(value: JsonRpcErrorData) -> Self { + Self::Error { value, size: None } + } +} + +impl TryFrom for JsonRpcErrorData { + type Error = Web3ProxyError; + + fn try_from(e: ProviderError) -> Result { + // TODO: move turning ClientError into json to a helper function? + let code; + let message: String; + let data; + + match e { + ProviderError::JsonRpcClientError(err) => { + if let Some(err) = err.as_error_response() { + code = err.code; + message = err.message.clone(); + data = err.data.clone(); + } else if let Some(err) = err.as_serde_error() { + // this is not an rpc error. keep it as an error + return Err(Web3ProxyError::BadResponse(format!( + "bad response: {}", + err + ))); + } else { + return Err(anyhow::anyhow!("unexpected ethers error! {:?}", err).into()); + } + } + e => return Err(e.into()), + } + + Ok(JsonRpcErrorData { + code, + message: Cow::Owned(message), + data, + }) + } +} + +impl Weighter for JsonRpcQueryWeigher { + fn weight( + &self, + _key: &JsonRpcQueryCacheKey, + _qey: &(), + value: &JsonRpcResponseData, + ) -> NonZeroU32 { + value.num_bytes() + } +} diff --git a/web3_proxy/src/rpcs/blockchain.rs b/web3_proxy/src/rpcs/blockchain.rs index f0c1bdbd..1ec3bc2c 100644 --- a/web3_proxy/src/rpcs/blockchain.rs +++ b/web3_proxy/src/rpcs/blockchain.rs @@ -5,6 +5,7 @@ use super::one::Web3Rpc; use super::transactions::TxStatus; use crate::frontend::authorization::Authorization; use crate::frontend::errors::{Web3ProxyError, Web3ProxyErrorContext, Web3ProxyResult}; +use crate::response_cache::JsonRpcResponseData; use crate::{config::BlockAndRpc, jsonrpc::JsonRpcRequest}; use derive_more::From; use ethers::prelude::{Block, TxHash, H256, U64}; @@ -250,15 +251,14 @@ impl Web3Rpcs { ) .await?; - if response.error.is_some() { - return Err(response.into()); - } + let value = match response { + JsonRpcResponseData::Error { .. } => { + return Err(anyhow::anyhow!("failed fetching block").into()); + } + JsonRpcResponseData::Result { value, .. } => value, + }; - let block = response - .result - .web3_context("no error, but also no block")?; - - let block: Option = serde_json::from_str(block.get())?; + let block: Option = serde_json::from_str(value.get())?; let block: ArcBlock = block.web3_context("no block in the response")?; @@ -346,13 +346,14 @@ impl Web3Rpcs { .try_send_best_consensus_head_connection(authorization, &request, None, Some(num), None) .await?; - if response.error.is_some() { - return Err(response.into()); - } + let value = match response { + JsonRpcResponseData::Error { .. } => { + return Err(anyhow::anyhow!("failed fetching block").into()); + } + JsonRpcResponseData::Result { value, .. } => value, + }; - let raw_block = response.result.web3_context("no cannonical block result")?; - - let block: ArcBlock = serde_json::from_str(raw_block.get())?; + let block: ArcBlock = serde_json::from_str(value.get())?; let block = Web3ProxyBlock::try_from(block)?; @@ -365,7 +366,7 @@ impl Web3Rpcs { pub(super) async fn process_incoming_blocks( &self, authorization: &Arc, - block_receiver: kanal::AsyncReceiver, + block_receiver: flume::Receiver, // TODO: document that this is a watch sender and not a broadcast! if things get busy, blocks might get missed // Geth's subscriptions have the same potential for skipping blocks. pending_tx_sender: Option>, @@ -373,7 +374,7 @@ impl Web3Rpcs { let mut connection_heads = ConsensusFinder::new(self.max_block_age, self.max_block_lag); loop { - match block_receiver.recv().await { + match block_receiver.recv_async().await { Ok((new_block, rpc)) => { let rpc_name = rpc.name.clone(); diff --git a/web3_proxy/src/rpcs/consensus.rs b/web3_proxy/src/rpcs/consensus.rs index a5658522..fbe10fa6 100644 --- a/web3_proxy/src/rpcs/consensus.rs +++ b/web3_proxy/src/rpcs/consensus.rs @@ -373,9 +373,9 @@ impl ConsensusFinder { .0 .tier; - trace!("first_tier: {}", current_tier); + // trace!("first_tier: {}", current_tier); - trace!("rpc_heads_by_tier: {:#?}", rpc_heads_by_tier); + // trace!("rpc_heads_by_tier: {:#?}", rpc_heads_by_tier); // loop over all the rpc heads (grouped by tier) and their parents to find consensus // TODO: i'm sure theres a lot of shortcuts that could be taken, but this is simplest to implement diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index 82924eb0..e9183d86 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -8,7 +8,8 @@ use crate::config::{BlockAndRpc, TxHashAndRpc, Web3RpcConfig}; use crate::frontend::authorization::{Authorization, RequestMetadata}; use crate::frontend::errors::{Web3ProxyError, Web3ProxyResult}; use crate::frontend::rpc_proxy_ws::ProxyMode; -use crate::jsonrpc::{JsonRpcForwardedResponse, JsonRpcRequest}; +use crate::jsonrpc::{JsonRpcErrorData, JsonRpcRequest}; +use crate::response_cache::JsonRpcResponseData; use crate::rpcs::consensus::{RankedRpcMap, RpcRanking}; use crate::rpcs::transactions::TxStatus; use anyhow::Context; @@ -23,12 +24,13 @@ use hashbrown::{HashMap, HashSet}; use itertools::Itertools; use log::{debug, error, info, trace, warn, Level}; use migration::sea_orm::DatabaseConnection; -use moka::future::{Cache, ConcurrentCacheExt}; +use moka::future::Cache; use ordered_float::OrderedFloat; use serde::ser::{SerializeStruct, Serializer}; use serde::Serialize; use serde_json::json; use serde_json::value::RawValue; +use std::borrow::Cow; use std::cmp::{min_by_key, Reverse}; use std::collections::BTreeMap; use std::fmt; @@ -43,7 +45,7 @@ use tokio::time::{interval, sleep, sleep_until, Duration, Instant, MissedTickBeh #[derive(From)] pub struct Web3Rpcs { /// if watch_consensus_head_sender is some, Web3Rpc inside self will send blocks here when they get them - pub(crate) block_sender: kanal::AsyncSender<(Option, Arc)>, + pub(crate) block_sender: flume::Sender<(Option, Arc)>, /// any requests will be forwarded to one (or more) of these connections pub(crate) by_name: ArcSwap>>, /// notify all http providers to check their blocks at the same time @@ -55,10 +57,11 @@ pub struct Web3Rpcs { pub(crate) watch_consensus_rpcs_sender: watch::Sender>>, /// this head receiver makes it easy to wait until there is a new block pub(super) watch_consensus_head_sender: Option>>, + /// keep track of transactions that we have sent through subscriptions pub(super) pending_transaction_cache: Cache, - pub(super) pending_tx_id_receiver: kanal::AsyncReceiver, - pub(super) pending_tx_id_sender: kanal::AsyncSender, + pub(super) pending_tx_id_receiver: flume::Receiver, + pub(super) pending_tx_id_sender: flume::Sender, /// TODO: this map is going to grow forever unless we do some sort of pruning. maybe store pruned in redis? /// all blocks, including orphans pub(super) blocks_by_hash: BlocksByHashCache, @@ -94,8 +97,8 @@ impl Web3Rpcs { watch::Receiver>>, // watch::Receiver>, )> { - let (pending_tx_id_sender, pending_tx_id_receiver) = kanal::unbounded_async(); - let (block_sender, block_receiver) = kanal::unbounded_async::(); + let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded(); + let (block_sender, block_receiver) = flume::unbounded::(); // TODO: query the rpc to get the actual expected block time, or get from config? maybe have this be part of a health check? let expected_block_time_ms = match chain_id { @@ -347,7 +350,7 @@ impl Web3Rpcs { async fn subscribe( self: Arc, authorization: Arc, - block_receiver: kanal::AsyncReceiver, + block_receiver: flume::Receiver, pending_tx_sender: Option>, ) -> anyhow::Result<()> { let mut futures = vec![]; @@ -362,7 +365,7 @@ impl Web3Rpcs { let pending_tx_id_receiver = self.pending_tx_id_receiver.clone(); let handle = tokio::task::spawn(async move { // TODO: set up this future the same as the block funnel - while let Ok((pending_tx_id, rpc)) = pending_tx_id_receiver.recv().await { + while let Ok((pending_tx_id, rpc)) = pending_tx_id_receiver.recv_async().await { let f = clone.clone().process_incoming_tx_id( authorization.clone(), rpc, @@ -423,12 +426,11 @@ impl Web3Rpcs { pub async fn try_send_parallel_requests( &self, active_request_handles: Vec, - id: Box, method: &str, params: Option<&serde_json::Value>, error_level: Level, // TODO: remove this box once i figure out how to do the options - ) -> Web3ProxyResult { + ) -> Web3ProxyResult { // TODO: if only 1 active_request_handles, do self.try_send_request? let responses = active_request_handles @@ -447,24 +449,16 @@ impl Web3Rpcs { let mut count_map: HashMap = HashMap::new(); let mut counts: Counter = Counter::new(); let mut any_ok_with_json_result = false; - let mut any_ok_but_maybe_json_error = false; for partial_response in responses { if partial_response.is_ok() { any_ok_with_json_result = true; } - let response = - JsonRpcForwardedResponse::try_from_response_result(partial_response, id.clone()); - - // TODO: better key? - let s = format!("{:?}", response); + // TODO: better key! + let s = format!("{:?}", partial_response); if count_map.get(&s).is_none() { - if response.is_ok() { - any_ok_but_maybe_json_error = true; - } - - count_map.insert(s.clone(), response); + count_map.insert(s.clone(), partial_response); } counts.update([s].into_iter()); @@ -477,19 +471,18 @@ impl Web3Rpcs { match most_common { Ok(x) => { - if any_ok_with_json_result && x.error.is_some() { - // this one may be an "Ok", but the json has an error inside it - continue; - } // return the most common success - return Ok(x); + return Ok(x.into()); } Err(err) => { - if any_ok_but_maybe_json_error { + if any_ok_with_json_result { // the most common is an error, but there is an Ok in here somewhere. loop to find it continue; } - return Err(err); + + let err: JsonRpcErrorData = err.try_into()?; + + return Ok(err.into()); } } } @@ -601,7 +594,18 @@ impl Web3Rpcs { return Ok(OpenRequestResult::Handle(handle)); } Ok(OpenRequestResult::RetryAt(retry_at)) => { - earliest_retry_at = earliest_retry_at.min(Some(retry_at)); + trace!( + "{:?} - retry on {} @ {}", + request_ulid, + best_rpc, + retry_at.duration_since(Instant::now()).as_secs_f32() + ); + + if earliest_retry_at.is_none() { + earliest_retry_at = Some(retry_at); + } else { + earliest_retry_at = earliest_retry_at.min(Some(retry_at)); + } } Ok(OpenRequestResult::NotReady) => { // TODO: log a warning? emit a stat? @@ -641,7 +645,16 @@ impl Web3Rpcs { Ok(OpenRequestResult::NotReady) } Some(earliest_retry_at) => { - warn!("no servers on {:?}! {:?}", self, earliest_retry_at); + // TODO: log the server that retry_at came from + // TODO: `self` doesn't log well. get a pretty name for this group of servers + warn!( + "{:?} - no servers on {:?}! retry in {:?}s", + request_ulid, + self, + earliest_retry_at + .duration_since(Instant::now()) + .as_secs_f32() + ); Ok(OpenRequestResult::RetryAt(earliest_retry_at)) } @@ -733,7 +746,7 @@ impl Web3Rpcs { match rpc.try_request_handle(authorization, None).await { Ok(OpenRequestResult::RetryAt(retry_at)) => { // this rpc is not available. skip it - warn!("{} is rate limited. skipping", rpc); + trace!("{} is rate limited. skipping", rpc); earliest_retry_at = earliest_retry_at.min(Some(retry_at)); } Ok(OpenRequestResult::Handle(handle)) => { @@ -767,7 +780,7 @@ impl Web3Rpcs { request_metadata: Option<&Arc>, min_block_needed: Option<&U64>, max_block_needed: Option<&U64>, - ) -> Web3ProxyResult { + ) -> Web3ProxyResult { let mut skip_rpcs = vec![]; let mut method_not_available_response = None; @@ -783,7 +796,7 @@ impl Web3Rpcs { .best_available_rpc( authorization, request_metadata, - &[], + &skip_rpcs, min_block_needed, max_block_needed, ) @@ -800,10 +813,11 @@ impl Web3Rpcs { let is_backup_response = rpc.backup; + // TODO: instead of entirely skipping, maybe demote a tier? skip_rpcs.push(rpc); // TODO: get the log percent from the user data - let response_result = active_request_handle + let response_result: Result, _> = active_request_handle .request( &request.method, &json!(request.params), @@ -812,10 +826,7 @@ impl Web3Rpcs { ) .await; - match JsonRpcForwardedResponse::try_from_response_result( - response_result, - request.id.clone(), - ) { + match response_result { Ok(response) => { // TODO: if there are multiple responses being aggregated, this will only use the last server's backup type if let Some(request_metadata) = request_metadata { @@ -824,97 +835,98 @@ impl Web3Rpcs { .store(is_backup_response, Ordering::Release); } - if let Some(error) = response.error.as_ref() { - // trace!(?response, "rpc error"); + return Ok(response.into()); + } + Err(error) => { + // trace!(?response, "rpc error"); - if let Some(request_metadata) = request_metadata { - request_metadata - .error_response - .store(true, Ordering::Release); + // TODO: separate jsonrpc error and web3 proxy error! + if let Some(request_metadata) = request_metadata { + request_metadata + .error_response + .store(true, Ordering::Release); + } + + let error: JsonRpcErrorData = error.try_into()?; + + // some errors should be retried on other nodes + let error_msg = error.message.as_ref(); + + // different providers do different codes. check all of them + // TODO: there's probably more strings to add here + let rate_limit_substrings = ["limit", "exceeded", "quota usage"]; + for rate_limit_substr in rate_limit_substrings { + if error_msg.contains(rate_limit_substr) { + warn!("rate limited by {}", skip_rpcs.last().unwrap()); + continue; } + } - // some errors should be retried on other nodes - let error_msg = error.message.as_str(); + match error.code { + -32000 => { + // TODO: regex? + let retry_prefixes = [ + "header not found", + "header for hash not found", + "missing trie node", + "node not started", + "RPC timeout", + ]; + for retry_prefix in retry_prefixes { + if error_msg.starts_with(retry_prefix) { + // TODO: too verbose + debug!("retrying on another server"); + continue; + } + } + } + -32601 => { + let error_msg = error.message.as_ref(); - // different providers do different codes. check all of them - // TODO: there's probably more strings to add here - let rate_limit_substrings = ["limit", "exceeded", "quota usage"]; - for rate_limit_substr in rate_limit_substrings { - if error_msg.contains(rate_limit_substr) { - warn!("rate limited by {}", skip_rpcs.last().unwrap()); + // sometimes a provider does not support all rpc methods + // we check other connections rather than returning the error + // but sometimes the method is something that is actually unsupported, + // so we save the response here to return it later + + // some providers look like this + if error_msg.starts_with("the method") + && error_msg.ends_with("is not available") + { + method_not_available_response = Some(error); + continue; + } + + // others look like this (this is the example in the official spec) + if error_msg == "Method not found" { + method_not_available_response = Some(error); continue; } } - - match error.code { - -32000 => { - // TODO: regex? - let retry_prefixes = [ - "header not found", - "header for hash not found", - "missing trie node", - "node not started", - "RPC timeout", - ]; - for retry_prefix in retry_prefixes { - if error_msg.starts_with(retry_prefix) { - continue; - } - } - } - -32601 => { - let error_msg = error.message.as_str(); - - // sometimes a provider does not support all rpc methods - // we check other connections rather than returning the error - // but sometimes the method is something that is actually unsupported, - // so we save the response here to return it later - - // some providers look like this - if error_msg.starts_with("the method") - && error_msg.ends_with("is not available") - { - method_not_available_response = Some(response); - continue; - } - - // others look like this (this is the example in the official spec) - if error_msg == "Method not found" { - method_not_available_response = Some(response); - continue; - } - } - _ => {} - } - } else { - // trace!(?response, "rpc success"); + _ => {} } - return Ok(response); - } - Err(err) => { - let rpc = skip_rpcs - .last() - .expect("there must have been a provider if we got an error"); + // let rpc = skip_rpcs + // .last() + // .expect("there must have been a provider if we got an error"); // TODO: emit a stat. if a server is getting skipped a lot, something is not right // TODO: if we get a TrySendError, reconnect. wait why do we see a trysenderror on a dual provider? shouldn't it be using reqwest - trace!( - "Backend server error on {}! Retrying {:?} on another. err={:?}", - rpc, - request, - err - ); + // TODO! WRONG! ONLY SET RETRY_AT IF THIS IS A SERVER/CONNECTION ERROR. JSONRPC "error" is FINE + // trace!( + // "Backend server error on {}! Retrying {:?} on another. err={:?}", + // rpc, + // request, + // error, + // ); + // if let Some(ref hard_limit_until) = rpc.hard_limit_until { + // let retry_at = Instant::now() + Duration::from_secs(1); - if let Some(ref hard_limit_until) = rpc.hard_limit_until { - let retry_at = Instant::now() + Duration::from_secs(1); + // hard_limit_until.send_replace(retry_at); + // } - hard_limit_until.send_replace(retry_at); - } - - continue; + return Ok(error.into()); } } } @@ -923,8 +935,8 @@ impl Web3Rpcs { // sleep (TODO: with a lock?) until our rate limits should be available // TODO: if a server catches up sync while we are waiting, we could stop waiting warn!( - "All rate limits exceeded. waiting for change in synced servers or {:?}", - retry_at + "All rate limits exceeded. waiting for change in synced servers or {:?}s", + retry_at.duration_since(Instant::now()).as_secs_f32() ); // TODO: have a separate column for rate limited? @@ -934,6 +946,7 @@ impl Web3Rpcs { tokio::select! { _ = sleep_until(retry_at) => { + trace!("slept!"); skip_rpcs.pop(); } _ = watch_consensus_connections.changed() => { @@ -948,6 +961,8 @@ impl Web3Rpcs { let waiting_for = min_block_needed.max(max_block_needed); + info!("waiting for {:?}", waiting_for); + if watch_for_block(waiting_for, &mut watch_consensus_connections).await? { // block found! continue so we can check for another rpc } else { @@ -966,10 +981,12 @@ impl Web3Rpcs { .store(true, Ordering::Release); } - if let Some(r) = method_not_available_response { + if let Some(err) = method_not_available_response { // TODO: this error response is likely the user's fault. do we actually want it marked as an error? maybe keep user and server error bools? // TODO: emit a stat for unsupported methods? it would be best to block them at the proxy instead of at the backend - return Ok(r); + // TODO: this is too verbose! + debug!("{}", serde_json::to_string(&err)?); + return Ok(err.into()); } let num_conns = self.by_name.load().len(); @@ -995,8 +1012,6 @@ impl Web3Rpcs { "No archive servers synced (min {:?}, max {:?}, head {:?}) ({} known)", min_block_needed, max_block_needed, head_block_num, num_conns ); - } else if num_skipped == 0 { - // TODO: what should we log? } else { error!( "Requested data is not available (min {:?}, max {:?}, head {:?}) ({} skipped, {} known)", @@ -1008,11 +1023,12 @@ impl Web3Rpcs { // TODO: what error code? // cloudflare gives {"jsonrpc":"2.0","error":{"code":-32043,"message":"Requested data cannot be older than 128 blocks."},"id":1} - Ok(JsonRpcForwardedResponse::from_str( - "Requested data is not available", - Some(-32043), - Some(request.id.clone()), - )) + Ok(JsonRpcErrorData { + message: Cow::Borrowed("Requested data is not available"), + code: -32043, + data: None, + } + .into()) } /// be sure there is a timeout on this or it might loop forever @@ -1027,7 +1043,7 @@ impl Web3Rpcs { error_level: Level, max_count: Option, always_include_backups: bool, - ) -> Web3ProxyResult { + ) -> Web3ProxyResult { let mut watch_consensus_rpcs = self.watch_consensus_rpcs_sender.subscribe(); let start = Instant::now(); @@ -1071,7 +1087,6 @@ impl Web3Rpcs { return self .try_send_parallel_requests( active_request_handles, - request.id.clone(), request.method.as_ref(), request.params.as_ref(), error_level, @@ -1126,7 +1141,7 @@ impl Web3Rpcs { request_metadata: Option<&Arc>, min_block_needed: Option<&U64>, max_block_needed: Option<&U64>, - ) -> Web3ProxyResult { + ) -> Web3ProxyResult { match authorization.checks.proxy_mode { ProxyMode::Debug | ProxyMode::Best => { self.try_send_best_consensus_head_connection( @@ -1158,7 +1173,7 @@ impl Serialize for Web3Rpcs { where S: Serializer, { - let mut state = serializer.serialize_struct("Web3Rpcs", 6)?; + let mut state = serializer.serialize_struct("Web3Rpcs", 1)?; { let by_name = self.by_name.load(); @@ -1178,12 +1193,12 @@ impl Serialize for Web3Rpcs { } } - self.blocks_by_hash.sync(); - self.blocks_by_number.sync(); - state.serialize_field("block_hashes_count", &self.blocks_by_hash.entry_count())?; - state.serialize_field("block_hashes_size", &self.blocks_by_hash.weighted_size())?; - state.serialize_field("block_numbers_count", &self.blocks_by_number.entry_count())?; - state.serialize_field("block_numbers_size", &self.blocks_by_number.weighted_size())?; + // self.blocks_by_hash.sync(); + // self.blocks_by_number.sync(); + // state.serialize_field("block_hashes_count", &self.blocks_by_hash.entry_count())?; + // state.serialize_field("block_hashes_size", &self.blocks_by_hash.weighted_size())?; + // state.serialize_field("block_numbers_count", &self.blocks_by_number.entry_count())?; + // state.serialize_field("block_numbers_size", &self.blocks_by_number.weighted_size())?; state.end() } } @@ -1391,8 +1406,8 @@ mod tests { (lagged_rpc.name.clone(), lagged_rpc.clone()), ]); - let (block_sender, _block_receiver) = kanal::unbounded_async(); - let (pending_tx_id_sender, pending_tx_id_receiver) = kanal::unbounded_async(); + let (block_sender, _block_receiver) = flume::unbounded(); + let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded(); let (watch_consensus_rpcs_sender, _watch_consensus_rpcs_receiver) = watch::channel(None); let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None); @@ -1643,8 +1658,8 @@ mod tests { (archive_rpc.name.clone(), archive_rpc.clone()), ]); - let (block_sender, _) = kanal::unbounded_async(); - let (pending_tx_id_sender, pending_tx_id_receiver) = kanal::unbounded_async(); + let (block_sender, _) = flume::unbounded(); + let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded(); let (watch_consensus_rpcs_sender, _) = watch::channel(None); let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None); @@ -1807,8 +1822,8 @@ mod tests { ), ]); - let (block_sender, _) = kanal::unbounded_async(); - let (pending_tx_id_sender, pending_tx_id_receiver) = kanal::unbounded_async(); + let (block_sender, _) = flume::unbounded(); + let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded(); let (watch_consensus_rpcs_sender, _) = watch::channel(None); let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None); diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index be0487e0..b9faea75 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -98,8 +98,8 @@ impl Web3Rpc { http_interval_sender: Option>>, redis_pool: Option, block_map: BlocksByHashCache, - block_sender: Option>, - tx_id_sender: Option)>>, + block_sender: Option>, + tx_id_sender: Option)>>, reconnect: bool, ) -> anyhow::Result<(Arc, AnyhowJoinHandle<()>)> { let created_at = Instant::now(); @@ -389,7 +389,7 @@ impl Web3Rpc { /// TODO: maybe it would be better to use "Full Jitter". The "Full Jitter" approach uses less work, but slightly more time. pub async fn retrying_connect( self: &Arc, - block_sender: Option<&kanal::AsyncSender>, + block_sender: Option<&flume::Sender>, chain_id: u64, db_conn: Option<&DatabaseConnection>, delay_start: bool, @@ -452,7 +452,7 @@ impl Web3Rpc { /// connect to the web3 provider async fn connect( self: &Arc, - block_sender: Option<&kanal::AsyncSender>, + block_sender: Option<&flume::Sender>, chain_id: u64, db_conn: Option<&DatabaseConnection>, ) -> anyhow::Result<()> { @@ -474,7 +474,7 @@ impl Web3Rpc { // tell the block subscriber that this rpc doesn't have any blocks if let Some(block_sender) = block_sender { block_sender - .send((None, self.clone())) + .send_async((None, self.clone())) .await .context("block_sender during connect")?; } @@ -589,7 +589,7 @@ impl Web3Rpc { pub(crate) async fn send_head_block_result( self: &Arc, new_head_block: Result, ProviderError>, - block_sender: &kanal::AsyncSender, + block_sender: &flume::Sender, block_map: BlocksByHashCache, ) -> anyhow::Result<()> { let new_head_block = match new_head_block { @@ -652,7 +652,7 @@ impl Web3Rpc { // send an empty block to take this server out of rotation block_sender - .send((new_head_block, self.clone())) + .send_async((new_head_block, self.clone())) .await .context("block_sender")?; @@ -671,11 +671,11 @@ impl Web3Rpc { self: Arc, authorization: &Arc, block_map: BlocksByHashCache, - block_sender: Option>, + block_sender: Option>, chain_id: u64, disconnect_receiver: watch::Receiver, http_interval_sender: Option>>, - tx_id_sender: Option)>>, + tx_id_sender: Option)>>, ) -> anyhow::Result<()> { let error_handler = if self.backup { RequestErrorHandler::DebugLevel @@ -896,7 +896,7 @@ impl Web3Rpc { self: Arc, authorization: Arc, http_interval_receiver: Option>, - block_sender: kanal::AsyncSender, + block_sender: flume::Sender, block_map: BlocksByHashCache, ) -> anyhow::Result<()> { trace!("watching new heads on {}", self); @@ -1091,7 +1091,7 @@ impl Web3Rpc { async fn subscribe_pending_transactions( self: Arc, authorization: Arc, - tx_id_sender: kanal::AsyncSender<(TxHash, Arc)>, + tx_id_sender: flume::Sender<(TxHash, Arc)>, ) -> anyhow::Result<()> { // TODO: give this a separate client. don't use new_head_client for everything. especially a firehose this big // TODO: timeout @@ -1116,7 +1116,7 @@ impl Web3Rpc { while let Some(pending_tx_id) = stream.next().await { tx_id_sender - .send((pending_tx_id, self.clone())) + .send_async((pending_tx_id, self.clone())) .await .context("tx_id_sender")?; @@ -1397,6 +1397,16 @@ impl Serialize for Web3Rpc { state.serialize_field("head_block", &head_block)?; } + state.serialize_field( + "total_requests", + &self.total_requests.load(atomic::Ordering::Acquire), + )?; + + state.serialize_field( + "active_requests", + &self.active_requests.load(atomic::Ordering::Relaxed), + )?; + state.serialize_field("head_latency", &self.head_latency.read().value())?; state.serialize_field( @@ -1406,16 +1416,6 @@ impl Serialize for Web3Rpc { state.serialize_field("peak_ewma", self.peak_ewma().as_ref())?; - state.serialize_field( - "active_requests", - &self.active_requests.load(atomic::Ordering::Acquire), - )?; - - state.serialize_field( - "total_requests", - &self.total_requests.load(atomic::Ordering::Acquire), - )?; - state.end() } } diff --git a/web3_proxy/src/rpcs/request.rs b/web3_proxy/src/rpcs/request.rs index 18e4ca2e..c50dd202 100644 --- a/web3_proxy/src/rpcs/request.rs +++ b/web3_proxy/src/rpcs/request.rs @@ -7,7 +7,7 @@ use entities::revert_log; use entities::sea_orm_active_enums::Method; use ethers::providers::ProviderError; use ethers::types::{Address, Bytes}; -use log::{debug, error, info, trace, warn, Level}; +use log::{debug, error, trace, warn, Level}; use migration::sea_orm::{self, ActiveEnum, ActiveModelTrait}; use serde_json::json; use std::fmt; @@ -283,12 +283,19 @@ impl OpenRequestHandle { _ => err.as_error_response().map(|x| x.message.clone()), }; + trace!("error message: {:?}", msg); + if let Some(msg) = msg { if msg.starts_with("execution reverted") { trace!("revert from {}", self.rpc); ResponseTypes::Revert } else if msg.contains("limit") || msg.contains("request") { - trace!("rate limit from {}", self.rpc); + // TODO: too verbose + if self.rpc.backup { + trace!("rate limit from {}", self.rpc); + } else { + warn!("rate limit from {}", self.rpc); + } ResponseTypes::RateLimit } else { ResponseTypes::Error @@ -303,6 +310,15 @@ impl OpenRequestHandle { if matches!(response_type, ResponseTypes::RateLimit) { if let Some(hard_limit_until) = self.rpc.hard_limit_until.as_ref() { // TODO: how long should we actually wait? different providers have different times + // TODO: if rate_limit_period_seconds is set, use that + // TODO: check response headers for rate limits too + // TODO: warn if production, debug if backup + if self.rpc.backup { + debug!("unexpected rate limit on {}!", self.rpc); + } else { + warn!("unexpected rate limit on {}!", self.rpc); + } + let retry_at = Instant::now() + Duration::from_secs(1); trace!("retry {} at: {:?}", self.rpc, retry_at); @@ -374,8 +390,9 @@ impl OpenRequestHandle { } } } else if let Some(peak_latency) = &self.rpc.peak_latency { - trace!("updating peak_latency: {}", latency.as_secs_f64()); - peak_latency.report(latency); + // trace!("updating peak_latency: {}", latency.as_secs_f64()); + // peak_latency.report(latency); + trace!("peak latency disabled for now"); } else { unreachable!("peak_latency not initialized"); } diff --git a/web3_proxy/src/stats/mod.rs b/web3_proxy/src/stats/mod.rs index 14759611..930f9e04 100644 --- a/web3_proxy/src/stats/mod.rs +++ b/web3_proxy/src/stats/mod.rs @@ -672,6 +672,7 @@ impl RpcQueryStats { method: Option<&str>, ) -> Decimal { // some methods should be free. there might be cases where method isn't set (though they should be uncommon) + // TODO: get this list from config (and add more to it) if let Some(method) = method.as_ref() { if ["eth_chainId"].contains(method) { return 0.into(); diff --git a/web3_proxy/src/stats/stat_buffer.rs b/web3_proxy/src/stats/stat_buffer.rs index a9d14329..5de91e8b 100644 --- a/web3_proxy/src/stats/stat_buffer.rs +++ b/web3_proxy/src/stats/stat_buffer.rs @@ -8,7 +8,7 @@ use log::{error, info, trace}; use migration::sea_orm::prelude::Decimal; use migration::sea_orm::DatabaseConnection; use std::time::Duration; -use tokio::sync::{broadcast, mpsc}; +use tokio::sync::broadcast; use tokio::task::JoinHandle; use tokio::time::interval; @@ -30,7 +30,7 @@ pub struct BufferedRpcQueryStats { #[derive(From)] pub struct SpawnedStatBuffer { - pub stat_sender: mpsc::UnboundedSender, + pub stat_sender: flume::Sender, /// these handles are important and must be allowed to finish pub background_handle: JoinHandle>, } @@ -65,7 +65,7 @@ impl StatBuffer { return Ok(None); } - let (stat_sender, stat_receiver) = mpsc::unbounded_channel(); + let (stat_sender, stat_receiver) = flume::unbounded(); let timestamp_precision = TimestampPrecision::Seconds; let mut new = Self { @@ -94,7 +94,7 @@ impl StatBuffer { async fn aggregate_and_save_loop( &mut self, bucket: String, - mut stat_receiver: mpsc::UnboundedReceiver, + stat_receiver: flume::Receiver, mut shutdown_receiver: broadcast::Receiver<()>, ) -> anyhow::Result<()> { let mut tsdb_save_interval = @@ -107,11 +107,11 @@ impl StatBuffer { loop { tokio::select! { - stat = stat_receiver.recv() => { - // info!("Received stat"); + stat = stat_receiver.recv_async() => { + // trace!("Received stat"); // save the stat to a buffer match stat { - Some(AppStat::RpcQuery(stat)) => { + Ok(AppStat::RpcQuery(stat)) => { if self.influxdb_client.is_some() { // TODO: round the timestamp at all? @@ -128,8 +128,8 @@ impl StatBuffer { self.accounting_db_buffer.entry(stat.accounting_key(self.billing_period_seconds)).or_default().add(stat); } } - None => { - info!("done receiving stats"); + Err(err) => { + info!("error receiving stat: {}", err); break; } } From 685c1d039a904f9d61212c9cea7037c84e75ac67 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Sat, 13 May 2023 15:00:24 -0700 Subject: [PATCH 21/66] cargo upgrade --- Cargo.lock | 14 +++++++------- web3_proxy/Cargo.toml | 4 ++-- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 9b4c1ba9..15b0d7ab 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2302,12 +2302,12 @@ dependencies = [ [[package]] name = "gethostname" -version = "0.4.2" +version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "199523ba70af2b447640715e8c4bd2b5360313a71d2d69361ae4dd1dc31487dd" +checksum = "0176e0459c2e4a1fe232f984bca6890e681076abb9934f6cea7c326f3fc47818" dependencies = [ "libc", - "windows", + "windows-targets 0.48.0", ] [[package]] @@ -4317,9 +4317,9 @@ dependencies = [ [[package]] name = "rdkafka" -version = "0.29.0" +version = "0.30.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd7c5d6d17442bcb9f943aae96d67d98c6d36af60442dd5da62aaa7fcbb25c48" +checksum = "97640b53443880ca65df40e9373a8193f9ad58b3f7419bc7206067f4a952500d" dependencies = [ "futures-channel", "futures-util", @@ -4335,9 +4335,9 @@ dependencies = [ [[package]] name = "rdkafka-sys" -version = "4.3.0+1.9.2" +version = "4.4.0+1.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d222a401698c7f2010e3967353eae566d9934dcda49c29910da922414ab4e3f4" +checksum = "87ac9d87c3aba1748e3112318459f2ac8bff80bfff7359e338e0463549590249" dependencies = [ "cmake", "libc", diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index 65a1cb87..91c2b26a 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -46,7 +46,7 @@ fdlimit = "0.2.1" flume = "0.10.14" fstrings = "0.2" futures = { version = "0.3.28", features = ["thread-pool"] } -gethostname = "0.4.2" +gethostname = "0.4.3" glob = "0.3.1" handlebars = "4.3.7" hashbrown = { version = "0.13.2", features = ["serde"] } @@ -71,7 +71,7 @@ parking_lot = { version = "0.12.1", features = ["arc_lock"] } prettytable = "*" proctitle = "0.1.1" quick_cache = "0.3.0" -rdkafka = { version = "0.29.0" } +rdkafka = { version = "0.30.0" } regex = "1.8.1" reqwest = { version = "0.11.17", default-features = false, features = ["json", "tokio-rustls"] } rmp-serde = "1.1.1" From 9621cfdccd5ffc0f4be5fdd658f49e30608cf1fc Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Sat, 13 May 2023 16:28:27 -0700 Subject: [PATCH 22/66] add quick cache wrapper for ttl expiration --- Cargo.lock | 9 +++ Cargo.toml | 1 + quick_cache_ttl/Cargo.toml | 14 +++++ quick_cache_ttl/src/lib.rs | 125 +++++++++++++++++++++++++++++++++++++ 4 files changed, 149 insertions(+) create mode 100644 quick_cache_ttl/Cargo.toml create mode 100644 quick_cache_ttl/src/lib.rs diff --git a/Cargo.lock b/Cargo.lock index 15b0d7ab..0cdee203 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4225,6 +4225,15 @@ dependencies = [ "parking_lot 0.12.1", ] +[[package]] +name = "quick_cache_ttl" +version = "0.1.0" +dependencies = [ + "flume", + "quick_cache", + "tokio", +] + [[package]] name = "quote" version = "1.0.27" diff --git a/Cargo.toml b/Cargo.toml index 3c0a9c22..12175b98 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -4,6 +4,7 @@ members = [ "entities", "latency", "migration", + "quick_cache_ttl", "rate-counter", "redis-rate-limiter", "thread-fast-rng", diff --git a/quick_cache_ttl/Cargo.toml b/quick_cache_ttl/Cargo.toml new file mode 100644 index 00000000..d0102b72 --- /dev/null +++ b/quick_cache_ttl/Cargo.toml @@ -0,0 +1,14 @@ +[package] +name = "quick_cache_ttl" +version = "0.1.0" +edition = "2021" + +# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html + +[dependencies] +flume = "0.10.14" +quick_cache = "0.3.0" +tokio = { version = "1.28.1", features = ["full"] } + +[dev-dependencies] +tokio = { version = "1.28.1", features = ["full", "test-util"] } diff --git a/quick_cache_ttl/src/lib.rs b/quick_cache_ttl/src/lib.rs new file mode 100644 index 00000000..11184357 --- /dev/null +++ b/quick_cache_ttl/src/lib.rs @@ -0,0 +1,125 @@ +use quick_cache::sync::Cache; +use quick_cache::{PlaceholderGuard, Weighter}; +use std::hash::{BuildHasher, Hash}; +use std::sync::Arc; +use std::time::Duration; +use tokio::task::JoinHandle; +use tokio::time::{sleep_until, Instant}; + +pub struct QuickCache { + cache: Arc>, + pub task_handle: JoinHandle<()>, + ttl: Duration, + tx: flume::Sender<(Instant, Key)>, +} + +// TODO: join handle that +struct QuickCacheTask { + cache: Arc>, + rx: flume::Receiver<(Instant, Key)>, +} + +pub struct Guard<'a, Key, Qey, Val, We, B> { + inner: PlaceholderGuard<'a, Key, Qey, Val, We, B>, + key: Key, + ttl: Duration, + tx: &'a flume::Sender<(Instant, Key)>, +} + +impl< + 'a, + Key: Clone + Eq + Hash + PartialEq, + Qey: Eq + Hash, + Val: Clone, + We: Weighter, + B: BuildHasher, + > Guard<'a, Key, Qey, Val, We, B> +{ + pub fn insert(self, val: Val) { + let expire_at = Instant::now() + self.ttl; + + self.inner.insert(val); + + self.tx.send((expire_at, self.key)).unwrap(); + } +} + +impl< + Key: Clone + Eq + Hash + Send + Sync + 'static, + Val: Clone + Send + Sync + 'static, + We: Weighter + Clone + Send + Sync + 'static, + B: BuildHasher + Clone + Send + Sync + 'static, + > QuickCache +{ + pub async fn spawn( + estimated_items_capacity: usize, + weight_capacity: u64, + weighter: We, + hash_builder: B, + ttl: Duration, + ) -> Self { + let (tx, rx) = flume::unbounded(); + + let cache = Cache::with( + estimated_items_capacity, + weight_capacity, + weighter, + hash_builder, + ); + + let cache = Arc::new(cache); + + let task = QuickCacheTask { + cache: cache.clone(), + rx, + }; + + let task_handle = tokio::spawn(task.run()); + + Self { + cache, + task_handle, + ttl, + tx, + } + } + + pub fn insert(&self, key: Key, val: Val) { + let expire_at = Instant::now() + self.ttl; + + self.cache.insert(key.clone(), val); + + self.tx.send((expire_at, key)).unwrap(); + } + + pub async fn get_value_or_guard_async( + &self, + key: Key, + ) -> Result> { + match self.cache.get_value_or_guard_async(&key).await { + Ok(x) => Ok(x), + Err(inner) => Err(Guard { + inner, + key, + ttl: self.ttl, + tx: &self.tx, + }), + } + } + + pub fn remove(&self, key: &Key) -> bool { + self.cache.remove(key) + } +} + +impl + Clone, B: BuildHasher + Clone> + QuickCacheTask +{ + async fn run(self) { + while let Ok((expire_at, key)) = self.rx.recv_async().await { + sleep_until(expire_at).await; + + self.cache.remove(&key); + } + } +} From b204a1f8ca0c1e666c895f04bada956241a20d9e Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Sat, 13 May 2023 21:03:45 -0700 Subject: [PATCH 23/66] kqcache too --- latency/src/peak_ewma/mod.rs | 1 + quick_cache_ttl/src/lib.rs | 128 ++--------------------------------- 2 files changed, 5 insertions(+), 124 deletions(-) diff --git a/latency/src/peak_ewma/mod.rs b/latency/src/peak_ewma/mod.rs index e3b27cdb..f4031545 100644 --- a/latency/src/peak_ewma/mod.rs +++ b/latency/src/peak_ewma/mod.rs @@ -63,6 +63,7 @@ impl PeakEwmaLatency { ); // Update the RTT estimate to account for decay since the last update. + // TODO: having an update here means we don't actually write from just one thread!! Thats how we get partially written stuff i think estimate.update(0.0, self.decay_ns, now) } diff --git a/quick_cache_ttl/src/lib.rs b/quick_cache_ttl/src/lib.rs index 11184357..a538704e 100644 --- a/quick_cache_ttl/src/lib.rs +++ b/quick_cache_ttl/src/lib.rs @@ -1,125 +1,5 @@ -use quick_cache::sync::Cache; -use quick_cache::{PlaceholderGuard, Weighter}; -use std::hash::{BuildHasher, Hash}; -use std::sync::Arc; -use std::time::Duration; -use tokio::task::JoinHandle; -use tokio::time::{sleep_until, Instant}; +mod cache; +mod kq_cache; -pub struct QuickCache { - cache: Arc>, - pub task_handle: JoinHandle<()>, - ttl: Duration, - tx: flume::Sender<(Instant, Key)>, -} - -// TODO: join handle that -struct QuickCacheTask { - cache: Arc>, - rx: flume::Receiver<(Instant, Key)>, -} - -pub struct Guard<'a, Key, Qey, Val, We, B> { - inner: PlaceholderGuard<'a, Key, Qey, Val, We, B>, - key: Key, - ttl: Duration, - tx: &'a flume::Sender<(Instant, Key)>, -} - -impl< - 'a, - Key: Clone + Eq + Hash + PartialEq, - Qey: Eq + Hash, - Val: Clone, - We: Weighter, - B: BuildHasher, - > Guard<'a, Key, Qey, Val, We, B> -{ - pub fn insert(self, val: Val) { - let expire_at = Instant::now() + self.ttl; - - self.inner.insert(val); - - self.tx.send((expire_at, self.key)).unwrap(); - } -} - -impl< - Key: Clone + Eq + Hash + Send + Sync + 'static, - Val: Clone + Send + Sync + 'static, - We: Weighter + Clone + Send + Sync + 'static, - B: BuildHasher + Clone + Send + Sync + 'static, - > QuickCache -{ - pub async fn spawn( - estimated_items_capacity: usize, - weight_capacity: u64, - weighter: We, - hash_builder: B, - ttl: Duration, - ) -> Self { - let (tx, rx) = flume::unbounded(); - - let cache = Cache::with( - estimated_items_capacity, - weight_capacity, - weighter, - hash_builder, - ); - - let cache = Arc::new(cache); - - let task = QuickCacheTask { - cache: cache.clone(), - rx, - }; - - let task_handle = tokio::spawn(task.run()); - - Self { - cache, - task_handle, - ttl, - tx, - } - } - - pub fn insert(&self, key: Key, val: Val) { - let expire_at = Instant::now() + self.ttl; - - self.cache.insert(key.clone(), val); - - self.tx.send((expire_at, key)).unwrap(); - } - - pub async fn get_value_or_guard_async( - &self, - key: Key, - ) -> Result> { - match self.cache.get_value_or_guard_async(&key).await { - Ok(x) => Ok(x), - Err(inner) => Err(Guard { - inner, - key, - ttl: self.ttl, - tx: &self.tx, - }), - } - } - - pub fn remove(&self, key: &Key) -> bool { - self.cache.remove(key) - } -} - -impl + Clone, B: BuildHasher + Clone> - QuickCacheTask -{ - async fn run(self) { - while let Ok((expire_at, key)) = self.rx.recv_async().await { - sleep_until(expire_at).await; - - self.cache.remove(&key); - } - } -} +pub use cache::CacheWithTTL; +pub use kq_cache::{KQCacheWithTTL, PlaceholderGuardWithTTL}; From c66eb6d864cd87f1b1f18d04abf354ff5fe483a8 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 16 May 2023 12:18:59 -0700 Subject: [PATCH 24/66] head latency instead of peak latency for now --- TODO.md | 2 +- latency/src/ewma.rs | 16 +++++++++------- web3_proxy/src/rpcs/consensus.rs | 9 +++++---- web3_proxy/src/rpcs/one.rs | 15 ++++++++------- 4 files changed, 23 insertions(+), 19 deletions(-) diff --git a/TODO.md b/TODO.md index b405b251..195c293d 100644 --- a/TODO.md +++ b/TODO.md @@ -745,4 +745,4 @@ in another repo: event subscriber - [ ] tests for config reloading - [ ] use pin instead of arc for a bunch of things? - https://fasterthanli.me/articles/pin-and-suffering -- [ ] calculate archive depth automatically based on block_data_limits +- [ ] calculate archive depth automatically based on block_data_limits \ No newline at end of file diff --git a/latency/src/ewma.rs b/latency/src/ewma.rs index 073dad54..fe5b51f9 100644 --- a/latency/src/ewma.rs +++ b/latency/src/ewma.rs @@ -17,18 +17,19 @@ impl Serialize for EwmaLatency { } impl EwmaLatency { - #[inline(always)] + #[inline] pub fn record(&mut self, duration: Duration) { self.record_ms(duration.as_secs_f64() * 1000.0); } - #[inline(always)] + #[inline] pub fn record_ms(&mut self, milliseconds: f64) { - self.ewma.add(milliseconds); + // don't let it go under 0.1ms + self.ewma.add(milliseconds.max(0.1)); } /// Current EWMA value in milliseconds - #[inline(always)] + #[inline] pub fn value(&self) -> f64 { self.ewma.value() } @@ -36,10 +37,11 @@ impl EwmaLatency { impl Default for EwmaLatency { fn default() -> Self { - // TODO: what should the default span be? 25 requests? - let span = 25.0; + // TODO: what should the default span be? 10 requests? + let span = 10.0; - let start = 1000.0; + // TODO: what should the defautt start be? + let start = 1.0; Self::new(span, start) } diff --git a/web3_proxy/src/rpcs/consensus.rs b/web3_proxy/src/rpcs/consensus.rs index fbe10fa6..f2a4b576 100644 --- a/web3_proxy/src/rpcs/consensus.rs +++ b/web3_proxy/src/rpcs/consensus.rs @@ -7,7 +7,7 @@ use derive_more::Constructor; use ethers::prelude::{H256, U64}; use hashbrown::{HashMap, HashSet}; use itertools::{Itertools, MinMaxResult}; -use log::{trace, warn}; +use log::{debug, info, trace, warn}; use moka::future::Cache; use serde::Serialize; use std::cmp::Reverse; @@ -266,15 +266,16 @@ impl ConsensusFinder { async fn insert(&mut self, rpc: Arc, block: Web3ProxyBlock) -> Option { let first_seen = self .first_seen - .get_with_by_ref(block.hash(), async move { Instant::now() }) + .get_with_by_ref(block.hash(), async { Instant::now() }) .await; - // TODO: this should be 0 if we are first seen, but i think it will be slightly non-zero. - // calculate elapsed time before trying to lock. + // calculate elapsed time before trying to lock let latency = first_seen.elapsed(); + // record the time behind the fastest node rpc.head_latency.write().record(latency); + // update the local mapping of rpc -> block self.rpc_heads.insert(rpc, block) } diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index b9faea75..686e20d2 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -70,7 +70,6 @@ pub struct Web3Rpc { /// Track head block latency pub(super) head_latency: RwLock, /// Track peak request latency - /// /// This is only inside an Option so that the "Default" derive works. it will always be set. pub(super) peak_latency: Option, /// Track total requests served @@ -236,16 +235,18 @@ impl Web3Rpc { } pub fn peak_ewma(&self) -> OrderedFloat { - let peak_latency = if let Some(peak_latency) = self.peak_latency.as_ref() { - peak_latency.latency().as_secs_f64() - } else { - 0.0 - }; + // TODO: bug inside peak ewma somewhere. possible with atomics being relaxed or the conversion to pair and back + // let peak_latency = if let Some(peak_latency) = self.peak_latency.as_ref() { + // peak_latency.latency().as_secs_f64() + // } else { + // 0.0 + // }; + let head_latency = self.head_latency.read().value(); // TODO: what ordering? let active_requests = self.active_requests.load(atomic::Ordering::Acquire) as f64 + 1.0; - OrderedFloat(peak_latency * active_requests) + OrderedFloat(head_latency * active_requests) } // TODO: would be great if rpcs exposed this. see https://github.com/ledgerwatch/erigon/issues/6391 From 978c385b3c277852d9c81ff0e8e0b0ea36fce12b Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 16 May 2023 13:26:39 -0700 Subject: [PATCH 25/66] improve wait for block --- web3_proxy/src/rpcs/blockchain.rs | 2 +- web3_proxy/src/rpcs/consensus.rs | 25 ++++++++++++-- web3_proxy/src/rpcs/many.rs | 54 +++++++++++++++---------------- web3_proxy/src/stats/mod.rs | 6 ++-- 4 files changed, 53 insertions(+), 34 deletions(-) diff --git a/web3_proxy/src/rpcs/blockchain.rs b/web3_proxy/src/rpcs/blockchain.rs index 1ec3bc2c..828eacb3 100644 --- a/web3_proxy/src/rpcs/blockchain.rs +++ b/web3_proxy/src/rpcs/blockchain.rs @@ -444,7 +444,7 @@ impl Web3Rpcs { let total_tiers = consensus_finder.worst_tier().unwrap_or(10); let backups_needed = new_synced_connections.backups_needed; let consensus_head_block = new_synced_connections.head_block.clone(); - let num_consensus_rpcs = new_synced_connections.num_conns(); + let num_consensus_rpcs = new_synced_connections.num_consensus_rpcs(); let num_active_rpcs = consensus_finder.len(); let total_rpcs = self.by_name.load().len(); diff --git a/web3_proxy/src/rpcs/consensus.rs b/web3_proxy/src/rpcs/consensus.rs index f2a4b576..31ffe385 100644 --- a/web3_proxy/src/rpcs/consensus.rs +++ b/web3_proxy/src/rpcs/consensus.rs @@ -7,7 +7,7 @@ use derive_more::Constructor; use ethers::prelude::{H256, U64}; use hashbrown::{HashMap, HashSet}; use itertools::{Itertools, MinMaxResult}; -use log::{debug, info, trace, warn}; +use log::{trace, warn}; use moka::future::Cache; use serde::Serialize; use std::cmp::Reverse; @@ -111,10 +111,31 @@ pub struct ConsensusWeb3Rpcs { impl ConsensusWeb3Rpcs { #[inline] - pub fn num_conns(&self) -> usize { + pub fn num_consensus_rpcs(&self) -> usize { self.best_rpcs.len() } + pub fn best_block_num(&self, skip_rpcs: &[Arc]) -> Option<&U64> { + if self.best_rpcs.iter().all(|rpc| skip_rpcs.contains(rpc)) { + // all of the consensus rpcs are skipped + // iterate the other rpc tiers to find the next best block + let mut best_block = None; + for (next_ranking, next_rpcs) in self.other_rpcs.iter() { + if next_rpcs.iter().all(|rpc| skip_rpcs.contains(rpc)) { + // everything in this ranking is skipped + continue; + } + + best_block = best_block.max(next_ranking.head_num.as_ref()); + } + + best_block + } else { + // not all the best synced rpcs are skipped yet. use the best head block + Some(self.head_block.number()) + } + } + pub fn has_block_data(&self, rpc: &Web3Rpc, block_num: &U64) -> bool { self.rpc_data .get(rpc) diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index e9183d86..35dd5d2e 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -37,7 +37,6 @@ use std::fmt; use std::sync::atomic::Ordering; use std::sync::Arc; use thread_fast_rng::rand::seq::SliceRandom; -use tokio; use tokio::sync::{broadcast, watch}; use tokio::time::{interval, sleep, sleep_until, Duration, Instant, MissedTickBehavior}; @@ -630,7 +629,7 @@ impl Web3Rpcs { match earliest_retry_at { None => { // none of the servers gave us a time to retry at - debug!("no servers on {:?} gave a retry time", self); + debug!("no servers on {:?} gave a retry time. {:?}", self, skip); // TODO: bring this back? need to think about how to do this with `allow_backups` // we could return an error here, but maybe waiting a second will fix the problem @@ -784,7 +783,7 @@ impl Web3Rpcs { let mut skip_rpcs = vec![]; let mut method_not_available_response = None; - let mut watch_consensus_connections = self.watch_consensus_rpcs_sender.subscribe(); + let mut watch_consensus_rpcs = self.watch_consensus_rpcs_sender.subscribe(); let start = Instant::now(); @@ -949,8 +948,8 @@ impl Web3Rpcs { trace!("slept!"); skip_rpcs.pop(); } - _ = watch_consensus_connections.changed() => { - watch_consensus_connections.borrow_and_update(); + _ = watch_consensus_rpcs.changed() => { + watch_consensus_rpcs.borrow_and_update(); } } } @@ -961,14 +960,12 @@ impl Web3Rpcs { let waiting_for = min_block_needed.max(max_block_needed); - info!("waiting for {:?}", waiting_for); - - if watch_for_block(waiting_for, &mut watch_consensus_connections).await? { + if watch_for_block(waiting_for, &skip_rpcs, &mut watch_consensus_rpcs).await? { // block found! continue so we can check for another rpc } else { // rate limits are likely keeping us from serving the head block - watch_consensus_connections.changed().await?; - watch_consensus_connections.borrow_and_update(); + watch_consensus_rpcs.changed().await?; + watch_consensus_rpcs.borrow_and_update(); } } } @@ -994,7 +991,7 @@ impl Web3Rpcs { let needed = min_block_needed.max(max_block_needed); - let head_block_num = watch_consensus_connections + let head_block_num = watch_consensus_rpcs .borrow() .as_ref() .map(|x| *x.head_block.number()); @@ -1536,7 +1533,6 @@ mod tests { assert!(lagged_rpc.has_block_data(lagged_block.number.as_ref().unwrap())); assert!(!lagged_rpc.has_block_data(head_block.number.as_ref().unwrap())); - // todo!("this doesn't work anymore. send_head_block_result doesn't do anything when rpcs isn't watching the block_receiver") assert_eq!(rpcs.num_synced_rpcs(), 2); // add head block to the rpcs. lagged_rpc should not be available @@ -1917,22 +1913,24 @@ mod tests { /// returns `true` when the desired block number is available /// TODO: max wait time? max number of blocks to wait for? time is probably best async fn watch_for_block( - block_num: Option<&U64>, - watch_consensus_connections: &mut watch::Receiver>>, + needed_block_num: Option<&U64>, + skip_rpcs: &[Arc], + watch_consensus_rpcs: &mut watch::Receiver>>, ) -> Web3ProxyResult { - let mut head_block_num = watch_consensus_connections + info!("waiting for {:?}", needed_block_num); + + let mut best_block_num: Option = watch_consensus_rpcs .borrow_and_update() .as_ref() - .map(|x| *x.head_block.number()); + .and_then(|x| x.best_block_num(skip_rpcs).copied()); - match (block_num, head_block_num) { - (Some(x), Some(ref head)) => { - if x <= head { - // we are past this block and no servers have this block + match (needed_block_num, best_block_num.as_ref()) { + (Some(x), Some(best)) => { + if x <= best { + // the best block is past the needed block and no servers have the needed data // this happens if the block is old and all archive servers are offline // there is no chance we will get this block without adding an archive server to the config - - // TODO: i think this can also happen if we are being rate limited! + // TODO: i think this can also happen if we are being rate limited! but then waiting might work. need skip_rpcs to be smarter return Ok(false); } } @@ -1944,6 +1942,7 @@ async fn watch_for_block( } (Some(_), None) => { // block requested but no servers synced. we will wait + // TODO: if the web3rpcs connected to this consensus isn't watching head blocks, exit with an erorr (waiting for blocks won't ever work) } (None, Some(head)) => { // i don't think this is possible @@ -1955,13 +1954,14 @@ async fn watch_for_block( // future block is requested // wait for the block to arrive - while head_block_num < block_num.copied() { - watch_consensus_connections.changed().await?; + while best_block_num.as_ref() < needed_block_num { + watch_consensus_rpcs.changed().await?; - head_block_num = watch_consensus_connections - .borrow_and_update() + let consensus_rpcs = watch_consensus_rpcs.borrow_and_update(); + + best_block_num = consensus_rpcs .as_ref() - .map(|x| *x.head_block.number()); + .and_then(|x| x.best_block_num(skip_rpcs).copied()); } Ok(true) diff --git a/web3_proxy/src/stats/mod.rs b/web3_proxy/src/stats/mod.rs index 930f9e04..352adb58 100644 --- a/web3_proxy/src/stats/mod.rs +++ b/web3_proxy/src/stats/mod.rs @@ -17,7 +17,7 @@ use derive_more::From; use entities::sea_orm_active_enums::TrackingLevel; use entities::{balance, referee, referrer, rpc_accounting_v2, rpc_key, user, user_tier}; use influxdb2::models::DataPoint; -use log::{trace, warn}; +use log::{error, trace, warn}; use migration::sea_orm::prelude::Decimal; use migration::sea_orm::{ self, ActiveModelTrait, ColumnTrait, DatabaseConnection, EntityTrait, IntoActiveModel, @@ -406,7 +406,7 @@ impl BufferedRpcQueryStats { if new_available_balance < Decimal::from(10u64) && downgrade_user_role.title == "Premium" { // TODO: we could do this outside the balance low block, but I think its fine. or better, update the cache if <$10 and downgrade if <$1 if let Some(rpc_secret_key_cache) = rpc_secret_key_cache { - todo!("expire (or probably better to update) the user cache now that the balance is low"); + error!("expire (or probably better to update) the user cache now that the balance is low"); // actually i think we need to have 2 caches. otherwise users with 2 keys are going to have seperate caches // 1. rpc_secret_key_id -> AuthorizationChecks (cuz we don't want to hit the db every time) // 2. user_id -> Balance @@ -419,8 +419,6 @@ impl BufferedRpcQueryStats { // active_downgrade_user.save(db_conn).await?; } - // TODO: - // Get the referee, and the referrer // (2) Look up the code that this user used. This is the referee table let referee_object = match referee::Entity::find() From 4d2b405b4efb384a374282b6b0ee89415a2a1ffc Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 16 May 2023 14:09:10 -0700 Subject: [PATCH 26/66] lower log level --- latency/src/peak_ewma/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/latency/src/peak_ewma/mod.rs b/latency/src/peak_ewma/mod.rs index e3b27cdb..24712229 100644 --- a/latency/src/peak_ewma/mod.rs +++ b/latency/src/peak_ewma/mod.rs @@ -104,7 +104,7 @@ impl PeakEwmaLatencyTask { while let Some(rtt) = self.request_rx.recv().await { self.update(rtt); } - info!("latency loop exited"); + trace!("latency loop exited"); } /// Update the estimate object atomically. From f4cebde53f31c0b1ae128a263b8c2b1a71d1f51a Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Mon, 15 May 2023 10:48:59 -0700 Subject: [PATCH 27/66] use ttl quick cache --- Cargo.lock | 2 +- quick_cache_ttl/src/cache.rs | 78 ++++++++++++++++ quick_cache_ttl/src/kq_cache.rs | 143 ++++++++++++++++++++++++++++++ quick_cache_ttl/src/lib.rs | 1 + web3_proxy/Cargo.toml | 2 +- web3_proxy/src/app/mod.rs | 11 ++- web3_proxy/src/frontend/mod.rs | 19 ++-- web3_proxy/src/frontend/status.rs | 81 +++++------------ web3_proxy/src/response_cache.rs | 11 ++- 9 files changed, 278 insertions(+), 70 deletions(-) create mode 100644 quick_cache_ttl/src/cache.rs create mode 100644 quick_cache_ttl/src/kq_cache.rs diff --git a/Cargo.lock b/Cargo.lock index 0cdee203..4292411b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6708,7 +6708,7 @@ dependencies = [ "parking_lot 0.12.1", "prettytable", "proctitle", - "quick_cache", + "quick_cache_ttl", "rdkafka", "redis-rate-limiter", "regex", diff --git a/quick_cache_ttl/src/cache.rs b/quick_cache_ttl/src/cache.rs new file mode 100644 index 00000000..6afceb23 --- /dev/null +++ b/quick_cache_ttl/src/cache.rs @@ -0,0 +1,78 @@ +use quick_cache::{DefaultHashBuilder, UnitWeighter, Weighter}; +use std::{ + future::Future, + hash::{BuildHasher, Hash}, + time::Duration, +}; + +use crate::{KQCacheWithTTL, PlaceholderGuardWithTTL}; + +pub struct CacheWithTTL(KQCacheWithTTL); + +impl + CacheWithTTL +{ + pub async fn new_with_unit_weights(estimated_items_capacity: usize, ttl: Duration) -> Self { + Self::new( + estimated_items_capacity, + estimated_items_capacity as u64, + UnitWeighter, + DefaultHashBuilder::default(), + ttl, + ) + .await + } +} + +impl< + Key: Eq + Hash + Clone + Send + Sync + 'static, + Val: Clone + Send + Sync + 'static, + We: Weighter + Clone + Send + Sync + 'static, + B: BuildHasher + Clone + Send + Sync + 'static, + > CacheWithTTL +{ + pub async fn new( + estimated_items_capacity: usize, + weight_capacity: u64, + weighter: We, + hash_builder: B, + ttl: Duration, + ) -> Self { + let inner = KQCacheWithTTL::new( + estimated_items_capacity, + weight_capacity, + weighter, + hash_builder, + ttl, + ) + .await; + + Self(inner) + } + + #[inline] + pub async fn get_or_insert_async(&self, key: &Key, f: Fut) -> Result + where + Fut: Future>, + { + self.0.get_or_insert_async(key, &(), f).await + } + + #[inline] + pub async fn get_value_or_guard_async( + &self, + key: Key, + ) -> Result> { + self.0.get_value_or_guard_async(key, ()).await + } + + #[inline] + pub fn insert(&self, key: Key, val: Val) { + self.0.insert(key, (), val) + } + + #[inline] + pub fn remove(&self, key: &Key) -> bool { + self.0.remove(key, &()) + } +} diff --git a/quick_cache_ttl/src/kq_cache.rs b/quick_cache_ttl/src/kq_cache.rs new file mode 100644 index 00000000..5859cb8d --- /dev/null +++ b/quick_cache_ttl/src/kq_cache.rs @@ -0,0 +1,143 @@ +use quick_cache::sync::KQCache; +use quick_cache::{PlaceholderGuard, Weighter}; +use std::future::Future; +use std::hash::{BuildHasher, Hash}; +use std::sync::Arc; +use std::time::Duration; +use tokio::task::JoinHandle; +use tokio::time::{sleep_until, Instant}; + +pub struct KQCacheWithTTL { + pub(crate) cache: Arc>, + pub task_handle: JoinHandle<()>, + ttl: Duration, + pub(crate) tx: flume::Sender<(Instant, Key, Qey)>, +} + +struct KQCacheWithTTLTask { + cache: Arc>, + rx: flume::Receiver<(Instant, Key, Qey)>, +} + +pub struct PlaceholderGuardWithTTL<'a, Key, Qey, Val, We, B> { + inner: PlaceholderGuard<'a, Key, Qey, Val, We, B>, + key: Key, + qey: Qey, + ttl: Duration, + tx: &'a flume::Sender<(Instant, Key, Qey)>, +} + +impl< + Key: Eq + Hash + Clone + Send + Sync + 'static, + Qey: Eq + Hash + Clone + Send + Sync + 'static, + Val: Clone + Send + Sync + 'static, + We: Weighter + Clone + Send + Sync + 'static, + B: BuildHasher + Clone + Send + Sync + 'static, + > KQCacheWithTTL +{ + pub async fn new( + estimated_items_capacity: usize, + weight_capacity: u64, + weighter: We, + hash_builder: B, + ttl: Duration, + ) -> Self { + let (tx, rx) = flume::unbounded(); + + let cache = KQCache::with( + estimated_items_capacity, + weight_capacity, + weighter, + hash_builder, + ); + + let cache = Arc::new(cache); + + let task = KQCacheWithTTLTask { + cache: cache.clone(), + rx, + }; + + let task_handle = tokio::spawn(task.run()); + + Self { + cache, + task_handle, + ttl, + tx, + } + } + + #[inline] + pub async fn get_or_insert_async(&self, key: &Key, qey: &Qey, f: Fut) -> Result + where + Fut: Future>, + { + self.cache.get_or_insert_async(key, qey, f).await + } + + #[inline] + pub async fn get_value_or_guard_async( + &self, + key: Key, + qey: Qey, + ) -> Result> { + match self.cache.get_value_or_guard_async(&key, &qey).await { + Ok(x) => Ok(x), + Err(inner) => Err(PlaceholderGuardWithTTL { + inner, + key, + qey, + ttl: self.ttl, + tx: &self.tx, + }), + } + } + + pub fn insert(&self, key: Key, qey: Qey, val: Val) { + let expire_at = Instant::now() + self.ttl; + + self.cache.insert(key.clone(), qey.clone(), val); + + self.tx.send((expire_at, key, qey)).unwrap(); + } + + pub fn remove(&self, key: &Key, qey: &Qey) -> bool { + self.cache.remove(key, qey) + } +} + +impl< + Key: Eq + Hash, + Qey: Eq + Hash, + Val: Clone, + We: Weighter + Clone, + B: BuildHasher + Clone, + > KQCacheWithTTLTask +{ + async fn run(self) { + while let Ok((expire_at, key, qey)) = self.rx.recv_async().await { + sleep_until(expire_at).await; + + self.cache.remove(&key, &qey); + } + } +} + +impl< + 'a, + Key: Clone + Hash + Eq, + Qey: Clone + Hash + Eq, + Val: Clone, + We: Weighter, + B: BuildHasher, + > PlaceholderGuardWithTTL<'a, Key, Qey, Val, We, B> +{ + pub fn insert(self, val: Val) { + let expire_at = Instant::now() + self.ttl; + + self.inner.insert(val); + + self.tx.send((expire_at, self.key, self.qey)).unwrap(); + } +} diff --git a/quick_cache_ttl/src/lib.rs b/quick_cache_ttl/src/lib.rs index a538704e..b2a05332 100644 --- a/quick_cache_ttl/src/lib.rs +++ b/quick_cache_ttl/src/lib.rs @@ -3,3 +3,4 @@ mod kq_cache; pub use cache::CacheWithTTL; pub use kq_cache::{KQCacheWithTTL, PlaceholderGuardWithTTL}; +pub use quick_cache::{DefaultHashBuilder, UnitWeighter, Weighter}; diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index 91c2b26a..bca468c6 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -19,6 +19,7 @@ deferred-rate-limiter = { path = "../deferred-rate-limiter" } entities = { path = "../entities" } latency = { path = "../latency" } migration = { path = "../migration" } +quick_cache_ttl = { path = "../quick_cache_ttl" } redis-rate-limiter = { path = "../redis-rate-limiter" } thread-fast-rng = { path = "../thread-fast-rng" } @@ -70,7 +71,6 @@ pagerduty-rs = { version = "0.1.6", default-features = false, features = ["async parking_lot = { version = "0.12.1", features = ["arc_lock"] } prettytable = "*" proctitle = "0.1.1" -quick_cache = "0.3.0" rdkafka = { version = "0.30.0" } regex = "1.8.1" reqwest = { version = "0.11.17", default-features = false, features = ["json", "tokio-rustls"] } diff --git a/web3_proxy/src/app/mod.rs b/web3_proxy/src/app/mod.rs index a8e8ad01..0a1c188a 100644 --- a/web3_proxy/src/app/mod.rs +++ b/web3_proxy/src/app/mod.rs @@ -36,6 +36,7 @@ use ethers::types::U256; use ethers::utils::rlp::{Decodable, Rlp}; use futures::future::join_all; use futures::stream::{FuturesUnordered, StreamExt}; +use hashbrown::hash_map::DefaultHashBuilder; use hashbrown::{HashMap, HashSet}; use ipnet::IpNet; use log::{debug, error, info, trace, warn, Level}; @@ -617,11 +618,15 @@ impl Web3ProxyApp { // responses can be very different in sizes, so this is a cache with a max capacity and a weigher // TODO: don't allow any response to be bigger than X% of the cache // TODO: we should emit stats to calculate a more accurate expected cache size - let response_cache = JsonRpcQueryCache::with_weighter( + // TODO: do we actually want a TTL on this? + let response_cache = JsonRpcQueryCache::new( (top_config.app.response_cache_max_bytes / 2048) as usize, top_config.app.response_cache_max_bytes, JsonRpcQueryWeigher, - ); + DefaultHashBuilder::default(), + Duration::from_secs(3600), + ) + .await; // create semaphores for concurrent connection limits // TODO: what should tti be for semaphores? @@ -1756,7 +1761,7 @@ impl Web3ProxyApp { match self .jsonrpc_query_cache - .get_value_or_guard_async(&cache_key).await + .get_value_or_guard_async(cache_key).await { Ok(x) => x, Err(x) => { diff --git a/web3_proxy/src/frontend/mod.rs b/web3_proxy/src/frontend/mod.rs index 0aca1b02..b5ebbbe5 100644 --- a/web3_proxy/src/frontend/mod.rs +++ b/web3_proxy/src/frontend/mod.rs @@ -19,11 +19,12 @@ use axum::{ use http::{header::AUTHORIZATION, StatusCode}; use listenfd::ListenFd; use log::info; -use std::iter::once; +use quick_cache_ttl::UnitWeighter; use std::net::SocketAddr; use std::sync::Arc; +use std::{iter::once, time::Duration}; use strum::{EnumCount, EnumIter}; -use tokio::{sync::broadcast, time::Instant}; +use tokio::sync::broadcast; use tower_http::cors::CorsLayer; use tower_http::sensitive_headers::SetSensitiveRequestHeadersLayer; @@ -35,8 +36,12 @@ pub enum FrontendResponseCacheKey { Status, } -pub type FrontendJsonResponseCache = - quick_cache::sync::Cache; +pub type FrontendJsonResponseCache = quick_cache_ttl::CacheWithTTL< + FrontendResponseCacheKey, + (StatusCode, axum::body::Bytes), + UnitWeighter, + quick_cache_ttl::DefaultHashBuilder, +>; /// Start the frontend server. pub async fn serve( @@ -50,7 +55,11 @@ pub async fn serve( // TODO: latest moka allows for different ttls for different let response_cache_size = FrontendResponseCacheKey::COUNT; - let json_response_cache = FrontendJsonResponseCache::new(response_cache_size); + let json_response_cache = FrontendJsonResponseCache::new_with_unit_weights( + response_cache_size, + Duration::from_secs(1), + ) + .await; // TODO: read config for if fastest/versus should be available publicly. default off diff --git a/web3_proxy/src/frontend/status.rs b/web3_proxy/src/frontend/status.rs index 4bc30aa3..a23b0a70 100644 --- a/web3_proxy/src/frontend/status.rs +++ b/web3_proxy/src/frontend/status.rs @@ -4,17 +4,12 @@ //! They will eventually move to another port. use super::{FrontendJsonResponseCache, FrontendResponseCacheKey}; -use crate::{ - app::{Web3ProxyApp, APP_USER_AGENT}, - frontend::errors::Web3ProxyError, -}; +use crate::app::{Web3ProxyApp, APP_USER_AGENT}; use axum::{body::Bytes, http::StatusCode, response::IntoResponse, Extension}; use axum_macros::debug_handler; -use futures::Future; use once_cell::sync::Lazy; use serde_json::json; -use std::{sync::Arc, time::Duration}; -use tokio::time::Instant; +use std::{convert::Infallible, sync::Arc}; static HEALTH_OK: Lazy = Lazy::new(|| Bytes::from("OK\n")); static HEALTH_NOT_OK: Lazy = Lazy::new(|| Bytes::from(":(\n")); @@ -22,54 +17,21 @@ static HEALTH_NOT_OK: Lazy = Lazy::new(|| Bytes::from(":(\n")); static BACKUPS_NEEDED_TRUE: Lazy = Lazy::new(|| Bytes::from("true\n")); static BACKUPS_NEEDED_FALSE: Lazy = Lazy::new(|| Bytes::from("false\n")); -/// simple ttl for -// TODO: make this generic for any cache/key -async fn _quick_cache_ttl( - app: Arc, - cache: Arc, - key: FrontendResponseCacheKey, - f: impl Fn(Arc) -> Fut, -) -> (StatusCode, Bytes) -where - Fut: Future, -{ - let mut response; - let expire_at; - - (response, expire_at) = cache - .get_or_insert_async::(&key, async { - let expire_at = Instant::now() + Duration::from_millis(1000); - - let response = f(app.clone()).await; - - Ok((response, expire_at)) - }) - .await - .unwrap(); - - if Instant::now() >= expire_at { - // TODO: this expiration isn't perfect - // parallel requests could overwrite eachother - // its good enough for now - let expire_at = Instant::now() + Duration::from_millis(1000); - - response = f(app).await; - - cache.insert(key, (response.clone(), expire_at)); - } - - response -} - /// Health check page for load balancers to use. #[debug_handler] pub async fn health( Extension(app): Extension>, Extension(cache): Extension>, ) -> impl IntoResponse { - _quick_cache_ttl(app, cache, FrontendResponseCacheKey::Health, _health).await + cache + .get_or_insert_async::(&FrontendResponseCacheKey::Health, async move { + Ok(_health(app).await) + }) + .await } +// TODO: _health doesn't need to be async, but _quick_cache_ttl needs an async function +#[inline] async fn _health(app: Arc) -> (StatusCode, Bytes) { if app.balanced_rpcs.synced() { (StatusCode::OK, HEALTH_OK.clone()) @@ -84,15 +46,15 @@ pub async fn backups_needed( Extension(app): Extension>, Extension(cache): Extension>, ) -> impl IntoResponse { - _quick_cache_ttl( - app, - cache, - FrontendResponseCacheKey::BackupsNeeded, - _backups_needed, - ) - .await + cache + .get_or_insert_async::( + &FrontendResponseCacheKey::BackupsNeeded, + async move { Ok(_backups_needed(app).await) }, + ) + .await } +#[inline] async fn _backups_needed(app: Arc) -> (StatusCode, Bytes) { let code = { let consensus_rpcs = app @@ -122,16 +84,21 @@ async fn _backups_needed(app: Arc) -> (StatusCode, Bytes) { /// Very basic status page. /// -/// TODO: replace this with proper stats and monitoring +/// TODO: replace this with proper stats and monitoring. frontend uses it for their public dashboards though #[debug_handler] pub async fn status( Extension(app): Extension>, Extension(cache): Extension>, ) -> impl IntoResponse { - _quick_cache_ttl(app, cache, FrontendResponseCacheKey::Status, _status).await + cache + .get_or_insert_async::(&FrontendResponseCacheKey::Status, async move { + Ok(_status(app).await) + }) + .await } -// TODO: this doesn't need to be async, but _quick_cache_ttl needs an async function +// TODO: _status doesn't need to be async, but _quick_cache_ttl needs an async function +#[inline] async fn _status(app: Arc) -> (StatusCode, Bytes) { // TODO: what else should we include? uptime, cache hit rates, cpu load, memory used // TODO: the hostname is probably not going to change. only get once at the start? diff --git a/web3_proxy/src/response_cache.rs b/web3_proxy/src/response_cache.rs index c7e710bd..7b269847 100644 --- a/web3_proxy/src/response_cache.rs +++ b/web3_proxy/src/response_cache.rs @@ -3,7 +3,8 @@ use crate::{ }; use derive_more::From; use ethers::providers::ProviderError; -use quick_cache::{sync::Cache as QuickCache, Weighter}; +use hashbrown::hash_map::DefaultHashBuilder; +use quick_cache_ttl::{CacheWithTTL, Weighter}; use serde_json::value::RawValue; use std::{ borrow::Cow, @@ -33,8 +34,12 @@ impl Hash for JsonRpcQueryCacheKey { } } -pub type JsonRpcQueryCache = - QuickCache; +pub type JsonRpcQueryCache = CacheWithTTL< + JsonRpcQueryCacheKey, + JsonRpcResponseData, + JsonRpcQueryWeigher, + DefaultHashBuilder, +>; #[derive(Clone)] pub struct JsonRpcQueryWeigher; From 525f6d4a34fc7c25d4e14133bcd3baa4453cc223 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 16 May 2023 14:58:00 -0700 Subject: [PATCH 28/66] better variable names --- web3_proxy/src/frontend/mod.rs | 17 +++++++---------- web3_proxy/src/frontend/status.rs | 19 +++++++++---------- 2 files changed, 16 insertions(+), 20 deletions(-) diff --git a/web3_proxy/src/frontend/mod.rs b/web3_proxy/src/frontend/mod.rs index b5ebbbe5..aa827909 100644 --- a/web3_proxy/src/frontend/mod.rs +++ b/web3_proxy/src/frontend/mod.rs @@ -30,14 +30,14 @@ use tower_http::sensitive_headers::SetSensitiveRequestHeadersLayer; /// simple keys for caching responses #[derive(Copy, Clone, Hash, PartialEq, Eq, EnumCount, EnumIter)] -pub enum FrontendResponseCacheKey { +pub enum ResponseCacheKey { BackupsNeeded, Health, Status, } -pub type FrontendJsonResponseCache = quick_cache_ttl::CacheWithTTL< - FrontendResponseCacheKey, +pub type ResponseCache = quick_cache_ttl::CacheWithTTL< + ResponseCacheKey, (StatusCode, axum::body::Bytes), UnitWeighter, quick_cache_ttl::DefaultHashBuilder, @@ -53,13 +53,10 @@ pub async fn serve( // setup caches for whatever the frontend needs // no need for max items since it is limited by the enum key // TODO: latest moka allows for different ttls for different - let response_cache_size = FrontendResponseCacheKey::COUNT; + let response_cache_size = ResponseCacheKey::COUNT; - let json_response_cache = FrontendJsonResponseCache::new_with_unit_weights( - response_cache_size, - Duration::from_secs(1), - ) - .await; + let response_cache = + ResponseCache::new_with_unit_weights(response_cache_size, Duration::from_secs(1)).await; // TODO: read config for if fastest/versus should be available publicly. default off @@ -225,7 +222,7 @@ pub async fn serve( // application state .layer(Extension(proxy_app)) // frontend caches - .layer(Extension(Arc::new(json_response_cache))) + .layer(Extension(Arc::new(response_cache))) // 404 for any unknown routes .fallback(errors::handler_404); diff --git a/web3_proxy/src/frontend/status.rs b/web3_proxy/src/frontend/status.rs index a23b0a70..afa9ebb0 100644 --- a/web3_proxy/src/frontend/status.rs +++ b/web3_proxy/src/frontend/status.rs @@ -3,7 +3,7 @@ //! For ease of development, users can currently access these endponts. //! They will eventually move to another port. -use super::{FrontendJsonResponseCache, FrontendResponseCacheKey}; +use super::{ResponseCache, ResponseCacheKey}; use crate::app::{Web3ProxyApp, APP_USER_AGENT}; use axum::{body::Bytes, http::StatusCode, response::IntoResponse, Extension}; use axum_macros::debug_handler; @@ -21,10 +21,10 @@ static BACKUPS_NEEDED_FALSE: Lazy = Lazy::new(|| Bytes::from("false\n")); #[debug_handler] pub async fn health( Extension(app): Extension>, - Extension(cache): Extension>, + Extension(cache): Extension>, ) -> impl IntoResponse { cache - .get_or_insert_async::(&FrontendResponseCacheKey::Health, async move { + .get_or_insert_async::(&ResponseCacheKey::Health, async move { Ok(_health(app).await) }) .await @@ -44,13 +44,12 @@ async fn _health(app: Arc) -> (StatusCode, Bytes) { #[debug_handler] pub async fn backups_needed( Extension(app): Extension>, - Extension(cache): Extension>, + Extension(cache): Extension>, ) -> impl IntoResponse { cache - .get_or_insert_async::( - &FrontendResponseCacheKey::BackupsNeeded, - async move { Ok(_backups_needed(app).await) }, - ) + .get_or_insert_async::(&ResponseCacheKey::BackupsNeeded, async move { + Ok(_backups_needed(app).await) + }) .await } @@ -88,10 +87,10 @@ async fn _backups_needed(app: Arc) -> (StatusCode, Bytes) { #[debug_handler] pub async fn status( Extension(app): Extension>, - Extension(cache): Extension>, + Extension(cache): Extension>, ) -> impl IntoResponse { cache - .get_or_insert_async::(&FrontendResponseCacheKey::Status, async move { + .get_or_insert_async::(&ResponseCacheKey::Status, async move { Ok(_status(app).await) }) .await From efaaf1bb5f7f108e9bd29d1f19584968a4e56341 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 16 May 2023 15:27:18 -0700 Subject: [PATCH 29/66] send the correct content type --- web3_proxy/src/frontend/mod.rs | 2 +- web3_proxy/src/frontend/status.rs | 57 ++++++++++++++++++++++++------- 2 files changed, 46 insertions(+), 13 deletions(-) diff --git a/web3_proxy/src/frontend/mod.rs b/web3_proxy/src/frontend/mod.rs index aa827909..9eb2f6d9 100644 --- a/web3_proxy/src/frontend/mod.rs +++ b/web3_proxy/src/frontend/mod.rs @@ -38,7 +38,7 @@ pub enum ResponseCacheKey { pub type ResponseCache = quick_cache_ttl::CacheWithTTL< ResponseCacheKey, - (StatusCode, axum::body::Bytes), + (StatusCode, &'static str, axum::body::Bytes), UnitWeighter, quick_cache_ttl::DefaultHashBuilder, >; diff --git a/web3_proxy/src/frontend/status.rs b/web3_proxy/src/frontend/status.rs index afa9ebb0..1f1cb94b 100644 --- a/web3_proxy/src/frontend/status.rs +++ b/web3_proxy/src/frontend/status.rs @@ -5,7 +5,12 @@ use super::{ResponseCache, ResponseCacheKey}; use crate::app::{Web3ProxyApp, APP_USER_AGENT}; -use axum::{body::Bytes, http::StatusCode, response::IntoResponse, Extension}; +use axum::{ + body::{Bytes, Full}, + http::StatusCode, + response::{IntoResponse, Response}, + Extension, +}; use axum_macros::debug_handler; use once_cell::sync::Lazy; use serde_json::json; @@ -17,26 +22,40 @@ static HEALTH_NOT_OK: Lazy = Lazy::new(|| Bytes::from(":(\n")); static BACKUPS_NEEDED_TRUE: Lazy = Lazy::new(|| Bytes::from("true\n")); static BACKUPS_NEEDED_FALSE: Lazy = Lazy::new(|| Bytes::from("false\n")); +static CONTENT_TYPE_JSON: &str = "application/json"; +static CONTENT_TYPE_PLAIN: &str = "text/plain"; + /// Health check page for load balancers to use. #[debug_handler] pub async fn health( Extension(app): Extension>, Extension(cache): Extension>, ) -> impl IntoResponse { - cache + let (code, content_type, body) = cache .get_or_insert_async::(&ResponseCacheKey::Health, async move { Ok(_health(app).await) }) .await + .unwrap(); + + Response::builder() + .status(code) + .header("content-type", content_type) + .body(Full::from(body)) + .unwrap() } // TODO: _health doesn't need to be async, but _quick_cache_ttl needs an async function #[inline] -async fn _health(app: Arc) -> (StatusCode, Bytes) { +async fn _health(app: Arc) -> (StatusCode, &'static str, Bytes) { if app.balanced_rpcs.synced() { - (StatusCode::OK, HEALTH_OK.clone()) + (StatusCode::OK, CONTENT_TYPE_PLAIN, HEALTH_OK.clone()) } else { - (StatusCode::SERVICE_UNAVAILABLE, HEALTH_NOT_OK.clone()) + ( + StatusCode::SERVICE_UNAVAILABLE, + CONTENT_TYPE_PLAIN, + HEALTH_NOT_OK.clone(), + ) } } @@ -46,15 +65,22 @@ pub async fn backups_needed( Extension(app): Extension>, Extension(cache): Extension>, ) -> impl IntoResponse { - cache + let (code, content_type, body) = cache .get_or_insert_async::(&ResponseCacheKey::BackupsNeeded, async move { Ok(_backups_needed(app).await) }) .await + .unwrap(); + + Response::builder() + .status(code) + .header("content-type", content_type) + .body(Full::from(body)) + .unwrap() } #[inline] -async fn _backups_needed(app: Arc) -> (StatusCode, Bytes) { +async fn _backups_needed(app: Arc) -> (StatusCode, &'static str, Bytes) { let code = { let consensus_rpcs = app .balanced_rpcs @@ -75,9 +101,9 @@ async fn _backups_needed(app: Arc) -> (StatusCode, Bytes) { }; if matches!(code, StatusCode::OK) { - (code, BACKUPS_NEEDED_FALSE.clone()) + (code, CONTENT_TYPE_PLAIN, BACKUPS_NEEDED_FALSE.clone()) } else { - (code, BACKUPS_NEEDED_TRUE.clone()) + (code, CONTENT_TYPE_PLAIN, BACKUPS_NEEDED_TRUE.clone()) } } @@ -89,16 +115,23 @@ pub async fn status( Extension(app): Extension>, Extension(cache): Extension>, ) -> impl IntoResponse { - cache + let (code, content_type, body) = cache .get_or_insert_async::(&ResponseCacheKey::Status, async move { Ok(_status(app).await) }) .await + .unwrap(); + + Response::builder() + .status(code) + .header("content-type", content_type) + .body(Full::from(body)) + .unwrap() } // TODO: _status doesn't need to be async, but _quick_cache_ttl needs an async function #[inline] -async fn _status(app: Arc) -> (StatusCode, Bytes) { +async fn _status(app: Arc) -> (StatusCode, &'static str, Bytes) { // TODO: what else should we include? uptime, cache hit rates, cpu load, memory used // TODO: the hostname is probably not going to change. only get once at the start? let body = json!({ @@ -120,5 +153,5 @@ async fn _status(app: Arc) -> (StatusCode, Bytes) { StatusCode::INTERNAL_SERVER_ERROR }; - (code, body) + (code, CONTENT_TYPE_JSON, body) } From 5d64524aa61ee85b584b1baa0911d7446196c1f7 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 16 May 2023 16:04:17 -0700 Subject: [PATCH 30/66] add name to web3rpcs --- README.md | 3 +++ latency/src/peak_ewma/mod.rs | 2 +- web3_proxy/src/app/mod.rs | 3 +++ web3_proxy/src/rpcs/many.rs | 40 +++++++++++++++++++++++++----------- 4 files changed, 35 insertions(+), 13 deletions(-) diff --git a/README.md b/README.md index 5c78789f..8b55df42 100644 --- a/README.md +++ b/README.md @@ -54,6 +54,9 @@ Check that the proxy is working: ``` curl -X POST -H "Content-Type: application/json" --data '{"jsonrpc":"2.0","method":"web3_clientVersion","id":1}' 127.0.0.1:8544 ``` +``` +curl -X POST -H "Content-Type: application/json" --data '{"jsonrpc":"2.0","method":"eth_getBalance", "params": ["0x0000000000000000000000000000000000000000", "latest"],"id":1}' 127.0.0.1:8544 +``` Check that the websocket is working: diff --git a/latency/src/peak_ewma/mod.rs b/latency/src/peak_ewma/mod.rs index 059ba06e..a795c383 100644 --- a/latency/src/peak_ewma/mod.rs +++ b/latency/src/peak_ewma/mod.rs @@ -2,7 +2,7 @@ mod rtt_estimate; use std::sync::Arc; -use log::{error, info}; +use log::{error, trace}; use tokio::sync::mpsc; use tokio::sync::mpsc::error::TrySendError; use tokio::task::JoinHandle; diff --git a/web3_proxy/src/app/mod.rs b/web3_proxy/src/app/mod.rs index 0a1c188a..7876f2ed 100644 --- a/web3_proxy/src/app/mod.rs +++ b/web3_proxy/src/app/mod.rs @@ -648,6 +648,7 @@ impl Web3ProxyApp { top_config.app.max_block_lag, top_config.app.min_synced_rpcs, top_config.app.min_sum_soft_limit, + "balanced".to_string(), pending_transactions.clone(), Some(pending_tx_sender.clone()), Some(watch_consensus_head_sender), @@ -675,6 +676,7 @@ impl Web3ProxyApp { None, 0, 0, + "protected".to_string(), pending_transactions.clone(), // TODO: subscribe to pending transactions on the private rpcs? they seem to have low rate limits, but they should have None, @@ -708,6 +710,7 @@ impl Web3ProxyApp { None, 0, 0, + "eip4337".to_string(), pending_transactions.clone(), None, None, diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index 35dd5d2e..e48d485b 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -33,7 +33,7 @@ use serde_json::value::RawValue; use std::borrow::Cow; use std::cmp::{min_by_key, Reverse}; use std::collections::BTreeMap; -use std::fmt; +use std::fmt::{self, Display}; use std::sync::atomic::Ordering; use std::sync::Arc; use thread_fast_rng::rand::seq::SliceRandom; @@ -43,6 +43,7 @@ use tokio::time::{interval, sleep, sleep_until, Duration, Instant, MissedTickBeh /// A collection of web3 connections. Sends requests either the current best server or all servers. #[derive(From)] pub struct Web3Rpcs { + pub(crate) name: String, /// if watch_consensus_head_sender is some, Web3Rpc inside self will send blocks here when they get them pub(crate) block_sender: flume::Sender<(Option, Arc)>, /// any requests will be forwarded to one (or more) of these connections @@ -87,6 +88,7 @@ impl Web3Rpcs { max_block_lag: Option, min_head_rpcs: usize, min_sum_soft_limit: u32, + name: String, pending_transaction_cache: Cache, pending_tx_sender: Option>, watch_consensus_head_sender: Option>>, @@ -184,19 +186,20 @@ impl Web3Rpcs { let connections = Arc::new(Self { block_sender, - by_name, - http_interval_sender, - watch_consensus_rpcs_sender, - watch_consensus_head_sender, - pending_transaction_cache, - pending_tx_id_sender, - pending_tx_id_receiver, blocks_by_hash, blocks_by_number, - min_sum_soft_limit, - min_head_rpcs, + by_name, + http_interval_sender, max_block_age, max_block_lag, + min_head_rpcs, + min_sum_soft_limit, + name, + pending_transaction_cache, + pending_tx_id_receiver, + pending_tx_id_sender, + watch_consensus_head_sender, + watch_consensus_rpcs_sender, }); let authorization = Arc::new(Authorization::internal(db_conn)?); @@ -629,7 +632,10 @@ impl Web3Rpcs { match earliest_retry_at { None => { // none of the servers gave us a time to retry at - debug!("no servers on {:?} gave a retry time. {:?}", self, skip); + debug!( + "{:?} - no servers on {:?} gave a retry time! Skipped {:?}", + request_ulid, self, skip + ); // TODO: bring this back? need to think about how to do this with `allow_backups` // we could return an error here, but maybe waiting a second will fix the problem @@ -647,9 +653,10 @@ impl Web3Rpcs { // TODO: log the server that retry_at came from // TODO: `self` doesn't log well. get a pretty name for this group of servers warn!( - "{:?} - no servers on {:?}! retry in {:?}s", + "{:?} - no servers in {} ready! Skipped {:?}. Retry in {:?}s", request_ulid, self, + skip, earliest_retry_at .duration_since(Instant::now()) .as_secs_f32() @@ -1156,6 +1163,12 @@ impl Web3Rpcs { } } +impl Display for Web3Rpcs { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.write_str(&self.name) + } +} + impl fmt::Debug for Web3Rpcs { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { // TODO: the default formatter takes forever to write. this is too quiet though @@ -1413,6 +1426,7 @@ mod tests { block_sender: block_sender.clone(), by_name: ArcSwap::from_pointee(rpcs_by_name), http_interval_sender: None, + name: "test".to_string(), watch_consensus_head_sender: Some(watch_consensus_head_sender), watch_consensus_rpcs_sender, pending_transaction_cache: Cache::builder() @@ -1664,6 +1678,7 @@ mod tests { block_sender, by_name: ArcSwap::from_pointee(rpcs_by_name), http_interval_sender: None, + name: "test".to_string(), watch_consensus_head_sender: Some(watch_consensus_head_sender), watch_consensus_rpcs_sender, pending_transaction_cache: Cache::builder() @@ -1828,6 +1843,7 @@ mod tests { block_sender, by_name: ArcSwap::from_pointee(rpcs_by_name), http_interval_sender: None, + name: "test".to_string(), watch_consensus_head_sender: Some(watch_consensus_head_sender), watch_consensus_rpcs_sender, pending_transaction_cache: Cache::builder() From 30ea532c701978c18423ff042300383b1cc95ed4 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 16 May 2023 16:13:56 -0700 Subject: [PATCH 31/66] display, not debug --- web3_proxy/src/app/mod.rs | 6 +++--- web3_proxy/src/rpcs/many.rs | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/web3_proxy/src/app/mod.rs b/web3_proxy/src/app/mod.rs index 7876f2ed..70b0b6e4 100644 --- a/web3_proxy/src/app/mod.rs +++ b/web3_proxy/src/app/mod.rs @@ -648,7 +648,7 @@ impl Web3ProxyApp { top_config.app.max_block_lag, top_config.app.min_synced_rpcs, top_config.app.min_sum_soft_limit, - "balanced".to_string(), + "balanced rpcs".to_string(), pending_transactions.clone(), Some(pending_tx_sender.clone()), Some(watch_consensus_head_sender), @@ -676,7 +676,7 @@ impl Web3ProxyApp { None, 0, 0, - "protected".to_string(), + "protected rpcs".to_string(), pending_transactions.clone(), // TODO: subscribe to pending transactions on the private rpcs? they seem to have low rate limits, but they should have None, @@ -710,7 +710,7 @@ impl Web3ProxyApp { None, 0, 0, - "eip4337".to_string(), + "eip4337 rpcs".to_string(), pending_transactions.clone(), None, None, diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index e48d485b..e761f691 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -633,7 +633,7 @@ impl Web3Rpcs { None => { // none of the servers gave us a time to retry at debug!( - "{:?} - no servers on {:?} gave a retry time! Skipped {:?}", + "{:?} - no servers in {} gave a retry time! Skipped {:?}", request_ulid, self, skip ); From a0d8218eb1e8997ad2886b56bf3771e9bb18475a Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 16 May 2023 18:18:56 -0700 Subject: [PATCH 32/66] best block num has to check has_block_data --- web3_proxy/src/rpcs/consensus.rs | 73 +++++++++++++++++++++++++++----- web3_proxy/src/rpcs/many.rs | 6 +-- 2 files changed, 66 insertions(+), 13 deletions(-) diff --git a/web3_proxy/src/rpcs/consensus.rs b/web3_proxy/src/rpcs/consensus.rs index 31ffe385..5a943f67 100644 --- a/web3_proxy/src/rpcs/consensus.rs +++ b/web3_proxy/src/rpcs/consensus.rs @@ -7,10 +7,10 @@ use derive_more::Constructor; use ethers::prelude::{H256, U64}; use hashbrown::{HashMap, HashSet}; use itertools::{Itertools, MinMaxResult}; -use log::{trace, warn}; +use log::{debug, trace, warn}; use moka::future::Cache; use serde::Serialize; -use std::cmp::Reverse; +use std::cmp::{Ordering, Reverse}; use std::collections::BTreeMap; use std::fmt; use std::sync::Arc; @@ -115,24 +115,76 @@ impl ConsensusWeb3Rpcs { self.best_rpcs.len() } - pub fn best_block_num(&self, skip_rpcs: &[Arc]) -> Option<&U64> { - if self.best_rpcs.iter().all(|rpc| skip_rpcs.contains(rpc)) { + pub fn best_block_num( + &self, + needed_block_num: Option<&U64>, + skip_rpcs: &[Arc], + ) -> Option<&U64> { + // TODO: dry this up with `filter`? + fn _best_block_num_filter( + x: &ConsensusWeb3Rpcs, + rpc: &Arc, + needed_block_num: Option<&U64>, + skip_rpcs: &[Arc], + ) -> bool { + // return true if this rpc will never work for us. "false" is good + if skip_rpcs.contains(rpc) { + // if rpc is skipped, it must have already been determined it is unable to serve the request + true + } else if let Some(needed_block_num) = needed_block_num { + if let Some(rpc_data) = x.rpc_data.get(rpc).as_ref() { + match rpc_data.head_block_num.cmp(needed_block_num) { + Ordering::Less => { + // rpc is not synced. let it catch up + false + } + Ordering::Greater | Ordering::Equal => { + // rpc is synced past the needed block. make sure the block isn't too old for it + !x.has_block_data(rpc, needed_block_num) + } + } + } else { + // no rpc data for this rpc. thats not promising + true + } + } else { + false + } + } + + if self + .best_rpcs + .iter() + .all(|rpc| _best_block_num_filter(self, rpc, needed_block_num, skip_rpcs)) + { // all of the consensus rpcs are skipped // iterate the other rpc tiers to find the next best block - let mut best_block = None; + let mut best_num = None; for (next_ranking, next_rpcs) in self.other_rpcs.iter() { - if next_rpcs.iter().all(|rpc| skip_rpcs.contains(rpc)) { - // everything in this ranking is skipped + if next_rpcs + .iter() + .all(|rpc| _best_block_num_filter(self, rpc, needed_block_num, skip_rpcs)) + { + // TODO: too verbose + debug!("everything in this ranking ({:?}) is skipped", next_ranking); continue; } - best_block = best_block.max(next_ranking.head_num.as_ref()); + best_num = best_num.max(next_ranking.head_num.as_ref()); } - best_block + // TODO: too verbose + debug!("best (old) block: {:?}", best_num); + + best_num } else { // not all the best synced rpcs are skipped yet. use the best head block - Some(self.head_block.number()) + let best_num = self.head_block.number(); + + // TODO: too verbose + debug!("best (head) block: {}", best_num); + + Some(best_num) } } @@ -143,6 +195,7 @@ impl ConsensusWeb3Rpcs { .unwrap_or(false) } + // TODO: better name for this pub fn filter( &self, skip: &[Arc], diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index e761f691..cf8e7f1c 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -1933,12 +1933,12 @@ async fn watch_for_block( skip_rpcs: &[Arc], watch_consensus_rpcs: &mut watch::Receiver>>, ) -> Web3ProxyResult { - info!("waiting for {:?}", needed_block_num); + debug!("waiting for {:?}", needed_block_num); let mut best_block_num: Option = watch_consensus_rpcs .borrow_and_update() .as_ref() - .and_then(|x| x.best_block_num(skip_rpcs).copied()); + .and_then(|x| x.best_block_num(needed_block_num, skip_rpcs).copied()); match (needed_block_num, best_block_num.as_ref()) { (Some(x), Some(best)) => { @@ -1977,7 +1977,7 @@ async fn watch_for_block( best_block_num = consensus_rpcs .as_ref() - .and_then(|x| x.best_block_num(skip_rpcs).copied()); + .and_then(|x| x.best_block_num(needed_block_num, skip_rpcs).copied()); } Ok(true) From 58b7799cc5f9d8392af5224e304b3b3bc6ba42a4 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 16 May 2023 18:25:56 -0700 Subject: [PATCH 33/66] for now, always return 0 for cost --- web3_proxy/src/stats/mod.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/web3_proxy/src/stats/mod.rs b/web3_proxy/src/stats/mod.rs index 352adb58..deb8563d 100644 --- a/web3_proxy/src/stats/mod.rs +++ b/web3_proxy/src/stats/mod.rs @@ -669,6 +669,9 @@ impl RpcQueryStats { cache_hit: bool, method: Option<&str>, ) -> Decimal { + // for now, always return 0 for cost + return 0.into(); + // some methods should be free. there might be cases where method isn't set (though they should be uncommon) // TODO: get this list from config (and add more to it) if let Some(method) = method.as_ref() { From 0f79d2e60d5b0279b19910dab78fe93503b7d1fe Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 16 May 2023 18:29:56 -0700 Subject: [PATCH 34/66] more loud logs --- web3_proxy/src/rpcs/many.rs | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index cf8e7f1c..3bfe94ad 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -633,8 +633,8 @@ impl Web3Rpcs { None => { // none of the servers gave us a time to retry at debug!( - "{:?} - no servers in {} gave a retry time! Skipped {:?}", - request_ulid, self, skip + "{:?} - no servers in {} gave a retry time! Skipped {:?}. {:#?}", + request_ulid, self, skip, usable_rpcs_by_tier_and_head_number ); // TODO: bring this back? need to think about how to do this with `allow_backups` @@ -1933,13 +1933,16 @@ async fn watch_for_block( skip_rpcs: &[Arc], watch_consensus_rpcs: &mut watch::Receiver>>, ) -> Web3ProxyResult { - debug!("waiting for {:?}", needed_block_num); - let mut best_block_num: Option = watch_consensus_rpcs .borrow_and_update() .as_ref() .and_then(|x| x.best_block_num(needed_block_num, skip_rpcs).copied()); + debug!( + "waiting for {:?}. best {:?}", + needed_block_num, best_block_num + ); + match (needed_block_num, best_block_num.as_ref()) { (Some(x), Some(best)) => { if x <= best { @@ -1947,6 +1950,7 @@ async fn watch_for_block( // this happens if the block is old and all archive servers are offline // there is no chance we will get this block without adding an archive server to the config // TODO: i think this can also happen if we are being rate limited! but then waiting might work. need skip_rpcs to be smarter + warn!("watching for block {} will never succeed. best {}", x, best); return Ok(false); } } From a07da300420fb3e35fdafef0ba30783b61998a8f Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 16 May 2023 18:32:33 -0700 Subject: [PATCH 35/66] keep usable_rpcs_by_tier_and_head_number around longer --- web3_proxy/src/rpcs/many.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index 3bfe94ad..a1b67f7d 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -504,7 +504,7 @@ impl Web3Rpcs { // TODO: use tracing and add this so logs are easy let request_ulid = request_metadata.map(|x| &x.request_ulid); - let usable_rpcs_by_tier_and_head_number = { + let mut usable_rpcs_by_tier_and_head_number = { let mut m: RankedRpcMap = BTreeMap::new(); if let Some(consensus_rpcs) = self.watch_consensus_rpcs_sender.borrow().as_ref() { @@ -568,7 +568,7 @@ impl Web3Rpcs { let mut earliest_retry_at = None; - for mut usable_rpcs in usable_rpcs_by_tier_and_head_number.into_values() { + for usable_rpcs in usable_rpcs_by_tier_and_head_number.values_mut() { // sort the tier randomly if usable_rpcs.len() == 1 { // TODO: include an rpc from the next tier? @@ -582,7 +582,7 @@ impl Web3Rpcs { // now that the rpcs are shuffled, try to get an active request handle for one of them // pick the first two and try the one with the lower rpc.latency.ewma // TODO: chunks or tuple windows? - for (rpc_a, rpc_b) in usable_rpcs.into_iter().circular_tuple_windows() { + for (rpc_a, rpc_b) in usable_rpcs.iter().circular_tuple_windows() { trace!("{:?} - {} vs {}", request_ulid, rpc_a, rpc_b); // TODO: cached key to save a read lock // TODO: ties to the server with the smallest block_data_limit From 89961331af3a4ca3141ec9f9d4d975fc6f5d543d Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 16 May 2023 22:29:36 -0700 Subject: [PATCH 36/66] refactor wait_for_block into should_wait_for_block --- web3_proxy/src/rpcs/consensus.rs | 168 ++++++++++++++++++------------- web3_proxy/src/rpcs/many.rs | 90 ++++------------- web3_proxy/src/rpcs/one.rs | 3 + 3 files changed, 121 insertions(+), 140 deletions(-) diff --git a/web3_proxy/src/rpcs/consensus.rs b/web3_proxy/src/rpcs/consensus.rs index 5a943f67..90385901 100644 --- a/web3_proxy/src/rpcs/consensus.rs +++ b/web3_proxy/src/rpcs/consensus.rs @@ -86,25 +86,31 @@ impl PartialOrd for RpcRanking { pub type RankedRpcMap = BTreeMap>>; +pub enum ShouldWaitForBlock { + Ready, + Wait { current: Option }, + NeverReady, +} + /// A collection of Web3Rpcs that are on the same block. -/// Serialize is so we can print it on our debug endpoint +/// Serialize is so we can print it on our /status endpoint #[derive(Clone, Serialize)] pub struct ConsensusWeb3Rpcs { pub(crate) tier: u64, pub(crate) backups_needed: bool, - // TODO: this is already inside best_rpcs. give that a shorter serialize here and then include this again + // TODO: this is already inside best_rpcs. Don't skip, instead make a shorter serialize #[serde(skip_serializing)] pub(crate) head_block: Web3ProxyBlock, // TODO: smaller serialize - pub(crate) best_rpcs: Vec>, + pub(crate) head_rpcs: Vec>, - // TODO: make this work. the key needs to be a string + // TODO: make this work. the key needs to be a string. I think we need `serialize_with` #[serde(skip_serializing)] pub(crate) other_rpcs: RankedRpcMap, - // TODO: make this work. the key needs to be a string + // TODO: make this work. the key needs to be a string. I think we need `serialize_with` #[serde(skip_serializing)] rpc_data: HashMap, RpcData>, } @@ -112,79 +118,65 @@ pub struct ConsensusWeb3Rpcs { impl ConsensusWeb3Rpcs { #[inline] pub fn num_consensus_rpcs(&self) -> usize { - self.best_rpcs.len() + self.head_rpcs.len() } - pub fn best_block_num( + /// will tell you if you should wait for a block + /// TODO: also include method (or maybe an enum representing the different prune types) + pub fn should_wait_for_block( &self, needed_block_num: Option<&U64>, skip_rpcs: &[Arc], - ) -> Option<&U64> { - // TODO: dry this up with `filter`? - fn _best_block_num_filter( - x: &ConsensusWeb3Rpcs, - rpc: &Arc, - needed_block_num: Option<&U64>, - skip_rpcs: &[Arc], - ) -> bool { - // return true if this rpc will never work for us. "false" is good - if skip_rpcs.contains(rpc) { - // if rpc is skipped, it must have already been determined it is unable to serve the request - true - } else if let Some(needed_block_num) = needed_block_num { - if let Some(rpc_data) = x.rpc_data.get(rpc).as_ref() { - match rpc_data.head_block_num.cmp(needed_block_num) { - Ordering::Less => { - // rpc is not synced. let it catch up - false - } - Ordering::Greater | Ordering::Equal => { - // rpc is synced past the needed block. make sure the block isn't too old for it - !x.has_block_data(rpc, needed_block_num) - } - } - } else { - // no rpc data for this rpc. thats not promising - true - } - } else { - false + ) -> ShouldWaitForBlock { + if self + .head_rpcs + .iter() + .any(|rpc| self.rpc_will_work_eventually(rpc, needed_block_num, skip_rpcs)) + { + let head_num = self.head_block.number(); + + if Some(head_num) >= needed_block_num { + debug!("best (head) block: {}", head_num); + return ShouldWaitForBlock::Ready; } } - if self - .best_rpcs - .iter() - .all(|rpc| _best_block_num_filter(self, rpc, needed_block_num, skip_rpcs)) - { - // all of the consensus rpcs are skipped - // iterate the other rpc tiers to find the next best block - let mut best_num = None; - for (next_ranking, next_rpcs) in self.other_rpcs.iter() { - if next_rpcs - .iter() - .all(|rpc| _best_block_num_filter(self, rpc, needed_block_num, skip_rpcs)) - { - // TODO: too verbose - debug!("everything in this ranking ({:?}) is skipped", next_ranking); - continue; - } + // all of the head rpcs are skipped - best_num = best_num.max(next_ranking.head_num.as_ref()); + let mut best_num = None; + + // iterate the other rpc tiers to find the next best block + for (next_ranking, next_rpcs) in self.other_rpcs.iter() { + if !next_rpcs + .iter() + .any(|rpc| self.rpc_will_work_eventually(rpc, needed_block_num, skip_rpcs)) + { + // TODO: too verbose + debug!("everything in this ranking ({:?}) is skipped", next_ranking); + continue; } + let next_head_num = next_ranking.head_num.as_ref(); + + if next_head_num >= needed_block_num { + debug!("best (head) block: {:?}", next_head_num); + return ShouldWaitForBlock::Ready; + } + + best_num = next_head_num; + } + + // TODO: this seems wrong + if best_num.is_some() { // TODO: too verbose debug!("best (old) block: {:?}", best_num); - - best_num + ShouldWaitForBlock::Wait { + current: best_num.copied(), + } } else { - // not all the best synced rpcs are skipped yet. use the best head block - let best_num = self.head_block.number(); - // TODO: too verbose - debug!("best (head) block: {}", best_num); - - Some(best_num) + debug!("never ready"); + ShouldWaitForBlock::NeverReady } } @@ -195,8 +187,48 @@ impl ConsensusWeb3Rpcs { .unwrap_or(false) } + // TODO: take method as a param, too. mark nodes with supported methods (maybe do it optimistically? on) + fn rpc_will_work_eventually( + &self, + rpc: &Arc, + needed_block_num: Option<&U64>, + skip_rpcs: &[Arc], + ) -> bool { + // return true if this rpc will never work for us. "false" is good + if skip_rpcs.contains(rpc) { + // if rpc is skipped, it must have already been determined it is unable to serve the request + return false; + } + + if let Some(needed_block_num) = needed_block_num { + if let Some(rpc_data) = self.rpc_data.get(rpc) { + match rpc_data.head_block_num.cmp(needed_block_num) { + Ordering::Less => { + debug!("{} is behind. let it catch up", rpc); + return true; + } + Ordering::Greater | Ordering::Equal => { + // rpc is synced past the needed block. make sure the block isn't too old for it + if self.has_block_data(rpc, needed_block_num) { + debug!("{} has {}", rpc, needed_block_num); + return true; + } else { + debug!("{} does not have {}", rpc, needed_block_num); + return false; + } + } + } + } + + // no rpc data for this rpc. thats not promising + return true; + } + + false + } + // TODO: better name for this - pub fn filter( + pub fn rpc_will_work_now( &self, skip: &[Arc], min_block_needed: Option<&U64>, @@ -244,7 +276,7 @@ impl fmt::Debug for ConsensusWeb3Rpcs { // TODO: print the actual conns? f.debug_struct("ConsensusWeb3Rpcs") .field("head_block", &self.head_block) - .field("num_conns", &self.best_rpcs.len()) + .field("num_conns", &self.head_rpcs.len()) .finish_non_exhaustive() } } @@ -272,7 +304,7 @@ impl Web3Rpcs { let consensus = self.watch_consensus_rpcs_sender.borrow(); if let Some(consensus) = consensus.as_ref() { - !consensus.best_rpcs.is_empty() + !consensus.head_rpcs.is_empty() } else { false } @@ -282,7 +314,7 @@ impl Web3Rpcs { let consensus = self.watch_consensus_rpcs_sender.borrow(); if let Some(consensus) = consensus.as_ref() { - consensus.best_rpcs.len() + consensus.head_rpcs.len() } else { 0 } @@ -598,7 +630,7 @@ impl ConsensusFinder { let consensus = ConsensusWeb3Rpcs { tier, head_block: maybe_head_block.clone(), - best_rpcs: consensus_rpcs, + head_rpcs: consensus_rpcs, other_rpcs, backups_needed, rpc_data, diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index a1b67f7d..d5440115 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -1,6 +1,6 @@ ///! Load balanced communication with a group of web3 rpc providers use super::blockchain::{BlocksByHashCache, Web3ProxyBlock}; -use super::consensus::ConsensusWeb3Rpcs; +use super::consensus::{ConsensusWeb3Rpcs, ShouldWaitForBlock}; use super::one::Web3Rpc; use super::request::{OpenRequestHandle, OpenRequestResult, RequestErrorHandler}; use crate::app::{flatten_handle, AnyhowJoinHandle, Web3ProxyApp}; @@ -521,8 +521,8 @@ impl Web3Rpcs { ); // todo: for now, build the map m here. once that works, do as much of it as possible while building ConsensusWeb3Rpcs - for x in consensus_rpcs.best_rpcs.iter().filter(|rpc| { - consensus_rpcs.filter(skip, min_block_needed, max_block_needed, rpc) + for x in consensus_rpcs.head_rpcs.iter().filter(|rpc| { + consensus_rpcs.rpc_will_work_now(skip, min_block_needed, max_block_needed, rpc) }) { m.entry(best_key).or_insert_with(Vec::new).push(x.clone()); } @@ -533,7 +533,12 @@ impl Web3Rpcs { let v: Vec<_> = v .iter() .filter(|rpc| { - consensus_rpcs.filter(skip, min_block_needed, max_block_needed, rpc) + consensus_rpcs.rpc_will_work_now( + skip, + min_block_needed, + max_block_needed, + rpc, + ) }) .cloned() .collect(); @@ -698,7 +703,7 @@ impl Web3Rpcs { let synced_rpcs = self.watch_consensus_rpcs_sender.borrow(); if let Some(synced_rpcs) = synced_rpcs.as_ref() { - synced_rpcs.best_rpcs.clone() + synced_rpcs.head_rpcs.clone() } else { vec![] } @@ -967,13 +972,15 @@ impl Web3Rpcs { let waiting_for = min_block_needed.max(max_block_needed); - if watch_for_block(waiting_for, &skip_rpcs, &mut watch_consensus_rpcs).await? { - // block found! continue so we can check for another rpc - } else { - // rate limits are likely keeping us from serving the head block - watch_consensus_rpcs.changed().await?; - watch_consensus_rpcs.borrow_and_update(); + if let Some(consensus_rpcs) = watch_consensus_rpcs.borrow_and_update().as_ref() + { + match consensus_rpcs.should_wait_for_block(waiting_for, &skip_rpcs) { + ShouldWaitForBlock::NeverReady => break, + ShouldWaitForBlock::Ready => continue, + ShouldWaitForBlock::Wait { .. } => {} + } } + watch_consensus_rpcs.changed().await; } } } @@ -1926,67 +1933,6 @@ mod tests { } } -/// returns `true` when the desired block number is available -/// TODO: max wait time? max number of blocks to wait for? time is probably best -async fn watch_for_block( - needed_block_num: Option<&U64>, - skip_rpcs: &[Arc], - watch_consensus_rpcs: &mut watch::Receiver>>, -) -> Web3ProxyResult { - let mut best_block_num: Option = watch_consensus_rpcs - .borrow_and_update() - .as_ref() - .and_then(|x| x.best_block_num(needed_block_num, skip_rpcs).copied()); - - debug!( - "waiting for {:?}. best {:?}", - needed_block_num, best_block_num - ); - - match (needed_block_num, best_block_num.as_ref()) { - (Some(x), Some(best)) => { - if x <= best { - // the best block is past the needed block and no servers have the needed data - // this happens if the block is old and all archive servers are offline - // there is no chance we will get this block without adding an archive server to the config - // TODO: i think this can also happen if we are being rate limited! but then waiting might work. need skip_rpcs to be smarter - warn!("watching for block {} will never succeed. best {}", x, best); - return Ok(false); - } - } - (None, None) => { - // i don't think this is possible - // maybe if we internally make a request for the latest block and all our servers are disconnected? - warn!("how'd this None/None happen?"); - return Ok(false); - } - (Some(_), None) => { - // block requested but no servers synced. we will wait - // TODO: if the web3rpcs connected to this consensus isn't watching head blocks, exit with an erorr (waiting for blocks won't ever work) - } - (None, Some(head)) => { - // i don't think this is possible - // maybe if we internally make a request for the latest block and all our servers are disconnected? - warn!("how'd this None/Some({}) happen?", head); - return Ok(false); - } - }; - - // future block is requested - // wait for the block to arrive - while best_block_num.as_ref() < needed_block_num { - watch_consensus_rpcs.changed().await?; - - let consensus_rpcs = watch_consensus_rpcs.borrow_and_update(); - - best_block_num = consensus_rpcs - .as_ref() - .and_then(|x| x.best_block_num(needed_block_num, skip_rpcs).copied()); - } - - Ok(true) -} - #[cfg(test)] mod test { use std::cmp::Reverse; diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index 686e20d2..b37a4425 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -47,6 +47,8 @@ pub struct Web3Rpc { /// it is an async lock because we hold it open across awaits /// this provider is only used for new heads subscriptions /// TODO: benchmark ArcSwapOption and a watch::Sender + /// TODO: only the websocket provider needs to be behind an asyncrwlock! + /// TODO: the http provider is just an http_client pub(super) provider: AsyncRwLock>>, /// keep track of hard limits /// this is only inside an Option so that the "Default" derive works. it will always be set. @@ -1216,6 +1218,7 @@ impl Web3Rpc { if unlocked_provider.is_some() || self.provider.read().await.is_some() { // we already have an unlocked provider. no need to lock } else { + warn!("no provider on {}", self); return Ok(OpenRequestResult::NotReady); } From c818a8fcd70f9a95a8c82aa1d0d134af52cf3769 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 16 May 2023 23:40:32 -0700 Subject: [PATCH 37/66] default balance to zero instead of panicing --- web3_proxy/src/frontend/authorization.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/web3_proxy/src/frontend/authorization.rs b/web3_proxy/src/frontend/authorization.rs index 34a1b398..13403ac7 100644 --- a/web3_proxy/src/frontend/authorization.rs +++ b/web3_proxy/src/frontend/authorization.rs @@ -1116,8 +1116,8 @@ impl Web3ProxyApp { .filter(balance::Column::UserId.eq(user_model.id)) .one(db_replica.conn()) .await? - .expect("related balance") - .available_balance; + .map(|x| x.available_balance) + .unwrap_or_default(); let user_tier_model = user_tier::Entity::find_by_id(user_model.user_tier_id) From 1c61390ad1554a928019d2d654d5c405b587cfda Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 16 May 2023 23:42:42 -0700 Subject: [PATCH 38/66] check if watch_consensus_rpcs.changed has failed i dont think it ever will, but better safe than sorry --- web3_proxy/src/rpcs/many.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index d5440115..6030a7d4 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -980,7 +980,7 @@ impl Web3Rpcs { ShouldWaitForBlock::Wait { .. } => {} } } - watch_consensus_rpcs.changed().await; + watch_consensus_rpcs.changed().await?; } } } From 0f367d903515016fe8c1017a0fde20a55a3b22b3 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Thu, 18 May 2023 00:04:30 -0700 Subject: [PATCH 39/66] more quick cache ttl --- Cargo.lock | 2 +- deferred-rate-limiter/Cargo.toml | 2 +- deferred-rate-limiter/src/lib.rs | 34 ++++++++++++++++++++------------ web3_proxy/src/app/mod.rs | 14 +++++-------- 4 files changed, 28 insertions(+), 24 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 4292411b..43ccb657 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1310,7 +1310,7 @@ dependencies = [ "anyhow", "hashbrown 0.13.2", "log", - "moka", + "quick_cache_ttl", "redis-rate-limiter", "tokio", ] diff --git a/deferred-rate-limiter/Cargo.toml b/deferred-rate-limiter/Cargo.toml index 699c9927..fad3722f 100644 --- a/deferred-rate-limiter/Cargo.toml +++ b/deferred-rate-limiter/Cargo.toml @@ -5,10 +5,10 @@ authors = ["Bryan Stitt "] edition = "2021" [dependencies] +quick_cache_ttl = { path = "../quick_cache_ttl" } redis-rate-limiter = { path = "../redis-rate-limiter" } anyhow = "1.0.71" hashbrown = "0.13.2" log = "0.4.17" -moka = { version = "0.11.0", default-features = false, features = ["future"] } tokio = "1.28.1" diff --git a/deferred-rate-limiter/src/lib.rs b/deferred-rate-limiter/src/lib.rs index f02c69f8..65f320a3 100644 --- a/deferred-rate-limiter/src/lib.rs +++ b/deferred-rate-limiter/src/lib.rs @@ -1,6 +1,6 @@ //#![warn(missing_docs)] use log::error; -use moka::future::Cache; +use quick_cache_ttl::{CacheWithTTL, UnitWeighter}; use redis_rate_limiter::{RedisRateLimitResult, RedisRateLimiter}; use std::cmp::Eq; use std::fmt::{Debug, Display}; @@ -16,7 +16,8 @@ pub struct DeferredRateLimiter where K: Send + Sync, { - local_cache: Cache, hashbrown::hash_map::DefaultHashBuilder>, + local_cache: + CacheWithTTL, UnitWeighter, hashbrown::hash_map::DefaultHashBuilder>, prefix: String, rrl: RedisRateLimiter, /// if None, defers to the max on rrl @@ -33,9 +34,9 @@ impl DeferredRateLimiter where K: Copy + Debug + Display + Hash + Eq + Send + Sync + 'static, { - pub fn new( + pub async fn new( // TODO: change this to cache_size in bytes - cache_size: u64, + cache_size: usize, prefix: &str, rrl: RedisRateLimiter, default_max_requests_per_second: Option, @@ -45,11 +46,18 @@ where // TODO: time to live is not exactly right. we want this ttl counter to start only after redis is down. this works for now // TODO: what do these weigh? // TODO: allow skipping max_capacity - let local_cache = Cache::builder() - .time_to_live(Duration::from_secs(ttl)) - .max_capacity(cache_size) - .name(prefix) - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); + let local_cache = CacheWithTTL::new( + cache_size, + cache_size as u64, + UnitWeighter, + hashbrown::hash_map::DefaultHashBuilder::default(), + Duration::from_secs(ttl), + ) + .await; + // .time_to_live(Duration::from_secs(ttl)) + // .max_capacity(cache_size) + // .name(prefix) + // .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); Self { local_cache, @@ -87,9 +95,9 @@ where let redis_key = redis_key.clone(); let rrl = Arc::new(self.rrl.clone()); - // set arc_deferred_rate_limit_result and return the coun + // set arc_deferred_rate_limit_result and return the count self.local_cache - .get_with_by_ref(&key, async move { + .get_or_insert_async::(&key, async move { // we do not use the try operator here because we want to be okay with redis errors let redis_count = match rrl .throttle_label(&redis_key, Some(max_requests_per_period), count) @@ -126,9 +134,9 @@ where } }; - Arc::new(AtomicU64::new(redis_count)) + Ok(Arc::new(AtomicU64::new(redis_count))) }) - .await + .await? }; let mut locked = deferred_rate_limit_result.lock().await; diff --git a/web3_proxy/src/app/mod.rs b/web3_proxy/src/app/mod.rs index 70b0b6e4..ed77cfe8 100644 --- a/web3_proxy/src/app/mod.rs +++ b/web3_proxy/src/app/mod.rs @@ -573,15 +573,11 @@ impl Web3ProxyApp { // 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 - frontend_ip_rate_limiter = Some(DeferredRateLimiter::::new( - 10_000, - "ip", - rpc_rrl.clone(), - None, - )); - frontend_registered_user_rate_limiter = Some(DeferredRateLimiter::::new( - 10_000, "key", rpc_rrl, None, - )); + frontend_ip_rate_limiter = Some( + DeferredRateLimiter::::new(20_000, "ip", rpc_rrl.clone(), None).await, + ); + frontend_registered_user_rate_limiter = + Some(DeferredRateLimiter::::new(10_000, "key", rpc_rrl, None).await); } // login rate limiter From 24439c51431775a63e875219672c5256c034297a Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Thu, 18 May 2023 00:43:33 -0700 Subject: [PATCH 40/66] disable balance join for now --- web3_proxy/src/stats/influxdb_queries.rs | 33 +++++++++++++----------- web3_proxy/src/stats/stat_buffer.rs | 11 ++++++-- 2 files changed, 27 insertions(+), 17 deletions(-) diff --git a/web3_proxy/src/stats/influxdb_queries.rs b/web3_proxy/src/stats/influxdb_queries.rs index 29747c65..368df1f1 100644 --- a/web3_proxy/src/stats/influxdb_queries.rs +++ b/web3_proxy/src/stats/influxdb_queries.rs @@ -184,7 +184,8 @@ pub async fn query_user_stats<'a>( {filter_chain_id} {drop_method} - cumsum = base + // cumsum = base + base |> aggregateWindow(every: {query_window_seconds}s, fn: sum, createEmpty: false) |> pivot(rowKey: ["_time"], columnKey: ["_field"], valueColumn: "_value") |> drop(columns: ["balance"]) @@ -199,19 +200,19 @@ pub async fn query_user_stats<'a>( |> group() |> sort(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"], desc: true) - balance = base - |> toFloat() - |> aggregateWindow(every: {query_window_seconds}s, fn: mean, createEmpty: false) - |> pivot(rowKey: ["_time"], columnKey: ["_field"], valueColumn: "_value") - |> group(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"]) - |> mean(column: "balance") - |> group() - |> sort(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"], desc: true) + // balance = base + // |> toFloat() + // |> aggregateWindow(every: {query_window_seconds}s, fn: mean, createEmpty: false) + // |> pivot(rowKey: ["_time"], columnKey: ["_field"], valueColumn: "_value") + // |> group(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"]) + // |> mean(column: "balance") + // |> group() + // |> sort(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"], desc: true) - join( - tables: {{cumsum, balance}}, - on: {join_candidates} - ) + // join( + // tables: {{cumsum, balance}}, + // on: {join_candidates} + // ) "#); info!("Raw query to db is: {:?}", query); @@ -219,8 +220,10 @@ pub async fn query_user_stats<'a>( info!("Query to db is: {:?}", query); // Make the query and collect all data - let raw_influx_responses: Vec = - influxdb_client.query_raw(Some(query.clone())).await?; + let raw_influx_responses: Vec = influxdb_client + .query_raw(Some(query.clone())) + .await + .context("failed parsing query result into a FluxRecord")?; // Basically rename all items to be "total", // calculate number of "archive_needed" and "error_responses" through their boolean representations ... diff --git a/web3_proxy/src/stats/stat_buffer.rs b/web3_proxy/src/stats/stat_buffer.rs index 5de91e8b..d7a5dea1 100644 --- a/web3_proxy/src/stats/stat_buffer.rs +++ b/web3_proxy/src/stats/stat_buffer.rs @@ -4,7 +4,7 @@ use derive_more::From; use futures::stream; use hashbrown::HashMap; use influxdb2::api::write::TimestampPrecision; -use log::{error, info, trace}; +use log::{debug, error, info, trace}; use migration::sea_orm::prelude::Decimal; use migration::sea_orm::DatabaseConnection; use std::time::Duration; @@ -249,17 +249,24 @@ impl StatBuffer { while num_left > 0 { let batch_size = num_left.min(max_batch_size); + // TODO: there has to be a better way to chunk this up. chunk on the stream with the stream being an iter? let p = points.split_off(batch_size); num_left -= batch_size; if let Err(err) = influxdb_client - .write_with_precision(bucket, stream::iter(p), self.timestamp_precision) + .write_with_precision( + bucket, + stream::iter(points), + self.timestamp_precision, + ) .await { // TODO: if this errors, we throw away some of the pending stats! we should probably buffer them somewhere to be tried again error!("unable to save {} tsdb stats! err={:?}", batch_size, err); } + + points = p; } } } From b61675e9282c0790a499d2ace40355c80d0d0dc8 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Thu, 18 May 2023 13:34:22 -0700 Subject: [PATCH 41/66] replace all moka with quick_cache_ttl --- Cargo.lock | 193 +---------------------- TODO.md | 2 +- deferred-rate-limiter/src/lib.rs | 19 +-- quick_cache_ttl/src/cache.rs | 48 +++++- quick_cache_ttl/src/kq_cache.rs | 9 +- quick_cache_ttl/src/lib.rs | 1 + web3_proxy/Cargo.toml | 1 - web3_proxy/src/app/mod.rs | 54 +++---- web3_proxy/src/frontend/authorization.rs | 43 +++-- web3_proxy/src/frontend/errors.rs | 10 -- web3_proxy/src/frontend/mod.rs | 2 +- web3_proxy/src/rpcs/blockchain.rs | 33 ++-- web3_proxy/src/rpcs/consensus.rs | 14 +- web3_proxy/src/rpcs/many.rs | 81 +++++----- web3_proxy/src/rpcs/one.rs | 8 +- web3_proxy/src/rpcs/transactions.rs | 2 +- web3_proxy/src/stats/stat_buffer.rs | 2 +- 17 files changed, 184 insertions(+), 338 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 43ccb657..410724bc 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -164,35 +164,6 @@ dependencies = [ "term", ] -[[package]] -name = "async-io" -version = "1.13.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fc5b45d93ef0529756f812ca52e44c221b35341892d3dcc34132ac02f3dd2af" -dependencies = [ - "async-lock", - "autocfg", - "cfg-if", - "concurrent-queue", - "futures-lite", - "log", - "parking", - "polling", - "rustix", - "slab", - "socket2", - "waker-fn", -] - -[[package]] -name = "async-lock" -version = "2.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa24f727524730b077666307f2734b4a1a1c57acb79193127dcc8914d5242dd7" -dependencies = [ - "event-listener", -] - [[package]] name = "async-stream" version = "0.3.5" @@ -593,12 +564,6 @@ dependencies = [ "syn 1.0.109", ] -[[package]] -name = "bytecount" -version = "0.6.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c676a478f63e9fa2dd5368a42f28bba0d6c560b775f38583c8bbaa7fcd67c9c" - [[package]] name = "byteorder" version = "1.4.3" @@ -653,19 +618,6 @@ dependencies = [ "serde", ] -[[package]] -name = "cargo_metadata" -version = "0.14.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4acbb09d9ee8e23699b9634375c72795d095bf268439da88562cf9b501f181fa" -dependencies = [ - "camino", - "cargo-platform", - "semver", - "serde", - "serde_json", -] - [[package]] name = "cargo_metadata" version = "0.15.4" @@ -901,15 +853,6 @@ dependencies = [ "tokio-util", ] -[[package]] -name = "concurrent-queue" -version = "2.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62ec6771ecfa0762d24683ee5a32ad78487a3d3afdc0fb8cae19d2c5deb50b7c" -dependencies = [ - "crossbeam-utils", -] - [[package]] name = "console" version = "0.14.1" @@ -1626,15 +1569,6 @@ dependencies = [ "libc", ] -[[package]] -name = "error-chain" -version = "0.12.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d2f06b9cac1506ece98fe3231e3cc9c4410ec3d5b1f24ae1c8946f0742cdefc" -dependencies = [ - "version_check", -] - [[package]] name = "eth-keystore" version = "0.5.0" @@ -1803,7 +1737,7 @@ checksum = "198ea9efa8480fa69f73d31d41b1601dace13d053c6fe4be6f5878d9dfcf0108" dependencies = [ "arrayvec", "bytes", - "cargo_metadata 0.15.4", + "cargo_metadata", "chrono", "elliptic-curve 0.13.4", "ethabi", @@ -2204,21 +2138,6 @@ version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4fff74096e71ed47f8e023204cfd0aa1289cd54ae5430a9523be060cdb849964" -[[package]] -name = "futures-lite" -version = "1.13.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49a9d51ce47660b1e808d3c990b4709f2f415d928835a17dfd16991515c46bce" -dependencies = [ - "fastrand", - "futures-core", - "futures-io", - "memchr", - "parking", - "pin-project-lite", - "waker-fn", -] - [[package]] name = "futures-locks" version = "0.7.1" @@ -3250,31 +3169,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "moka" -version = "0.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "934030d03f6191edbb4ba16835ccdb80d560788ac686570a8e2986a0fb59ded8" -dependencies = [ - "async-io", - "async-lock", - "crossbeam-channel", - "crossbeam-epoch", - "crossbeam-utils", - "futures-util", - "num_cpus", - "once_cell", - "parking_lot 0.12.1", - "rustc_version", - "scheduled-thread-pool", - "skeptic", - "smallvec", - "tagptr", - "thiserror", - "triomphe", - "uuid 1.3.2", -] - [[package]] name = "nanorand" version = "0.7.0" @@ -3704,12 +3598,6 @@ dependencies = [ "syn 1.0.109", ] -[[package]] -name = "parking" -version = "2.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14f2252c834a40ed9bb5422029649578e63aa341ac401f74e719dd1afda8394e" - [[package]] name = "parking_lot" version = "0.11.2" @@ -4016,22 +3904,6 @@ version = "0.3.27" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "26072860ba924cbfa98ea39c8c19b4dd6a4a25423dbdf219c1eca91aa0cf6964" -[[package]] -name = "polling" -version = "2.8.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b2d323e8ca7996b3e23126511a523f7e62924d93ecd5ae73b333815b0eb3dce" -dependencies = [ - "autocfg", - "bitflags", - "cfg-if", - "concurrent-queue", - "libc", - "log", - "pin-project-lite", - "windows-sys 0.48.0", -] - [[package]] name = "ppv-lite86" version = "0.2.17" @@ -4203,17 +4075,6 @@ dependencies = [ "syn 1.0.109", ] -[[package]] -name = "pulldown-cmark" -version = "0.9.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d9cc634bc78768157b5cbfe988ffcd1dcba95cd2b2f03a88316c08c6d00ed63" -dependencies = [ - "bitflags", - "memchr", - "unicase", -] - [[package]] name = "quick_cache" version = "0.3.0" @@ -4777,15 +4638,6 @@ dependencies = [ "windows-sys 0.42.0", ] -[[package]] -name = "scheduled-thread-pool" -version = "0.2.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3cbc66816425a074528352f5789333ecff06ca41b36b0b0efdfbb29edc391a19" -dependencies = [ - "parking_lot 0.12.1", -] - [[package]] name = "scoped-tls" version = "1.0.1" @@ -5400,21 +5252,6 @@ dependencies = [ "time 0.3.21", ] -[[package]] -name = "skeptic" -version = "0.13.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16d23b015676c90a0f01c197bfdc786c20342c73a0afdda9025adb0bc42940a8" -dependencies = [ - "bytecount", - "cargo_metadata 0.14.2", - "error-chain", - "glob", - "pulldown-cmark", - "tempfile", - "walkdir", -] - [[package]] name = "slab" version = "0.4.8" @@ -5754,12 +5591,6 @@ version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" -[[package]] -name = "tagptr" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b2093cf4c8eb1e67749a6762251bc9cd836b6fc171623bd0a9d324d37af2417" - [[package]] name = "tap" version = "1.0.1" @@ -6310,12 +6141,6 @@ dependencies = [ "tracing-log", ] -[[package]] -name = "triomphe" -version = "0.1.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1ee9bd9239c339d714d657fac840c6d2a4f9c45f4f9ec7b0975113458be78db" - [[package]] name = "try-lock" version = "0.2.4" @@ -6400,15 +6225,6 @@ dependencies = [ "libc", ] -[[package]] -name = "unicase" -version = "2.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "50f37be617794602aabbeee0be4f259dc1778fabe05e2d67ee8f79326d5cb4f6" -dependencies = [ - "version_check", -] - [[package]] name = "unicode-bidi" version = "0.3.13" @@ -6531,12 +6347,6 @@ version = "0.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" -[[package]] -name = "waker-fn" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d5b2c62b4012a3e1eca5a7e077d13b3bf498c4073e33ccd58626607748ceeca" - [[package]] name = "walkdir" version = "2.3.3" @@ -6699,7 +6509,6 @@ dependencies = [ "log", "migration", "mimalloc", - "moka", "num", "num-traits", "once_cell", diff --git a/TODO.md b/TODO.md index 195c293d..5bab04ae 100644 --- a/TODO.md +++ b/TODO.md @@ -128,7 +128,7 @@ These are roughly in order of completition - this was intentional so that recently confirmed transactions go to a server that is more likely to have the tx. - but under heavy load, we hit their rate limits. need a "retry_until_success" function that goes to balanced_rpcs. or maybe store in redis the txids that we broadcast privately and use that to route. - [x] some of the DashMaps grow unbounded! Make/find a "SizedDashMap" that cleans up old rows with some garbage collection task - - moka is exactly what we need + - moka has all the features that we need and more - [x] if block data limit is 0, say Unknown in Debug output - [x] basic request method stats (using the user_id and other fields that are in the tracing frame) - [x] refactor from_anyhow_error to have consistent error codes and http codes. maybe implement the Error trait diff --git a/deferred-rate-limiter/src/lib.rs b/deferred-rate-limiter/src/lib.rs index 65f320a3..99cf5a2e 100644 --- a/deferred-rate-limiter/src/lib.rs +++ b/deferred-rate-limiter/src/lib.rs @@ -1,6 +1,6 @@ //#![warn(missing_docs)] use log::error; -use quick_cache_ttl::{CacheWithTTL, UnitWeighter}; +use quick_cache_ttl::CacheWithTTL; use redis_rate_limiter::{RedisRateLimitResult, RedisRateLimiter}; use std::cmp::Eq; use std::fmt::{Debug, Display}; @@ -16,8 +16,7 @@ pub struct DeferredRateLimiter where K: Send + Sync, { - local_cache: - CacheWithTTL, UnitWeighter, hashbrown::hash_map::DefaultHashBuilder>, + local_cache: CacheWithTTL>, prefix: String, rrl: RedisRateLimiter, /// if None, defers to the max on rrl @@ -46,18 +45,8 @@ where // TODO: time to live is not exactly right. we want this ttl counter to start only after redis is down. this works for now // TODO: what do these weigh? // TODO: allow skipping max_capacity - let local_cache = CacheWithTTL::new( - cache_size, - cache_size as u64, - UnitWeighter, - hashbrown::hash_map::DefaultHashBuilder::default(), - Duration::from_secs(ttl), - ) - .await; - // .time_to_live(Duration::from_secs(ttl)) - // .max_capacity(cache_size) - // .name(prefix) - // .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); + let local_cache = + CacheWithTTL::new_with_capacity(cache_size, Duration::from_secs(ttl)).await; Self { local_cache, diff --git a/quick_cache_ttl/src/cache.rs b/quick_cache_ttl/src/cache.rs index 6afceb23..6f2ac650 100644 --- a/quick_cache_ttl/src/cache.rs +++ b/quick_cache_ttl/src/cache.rs @@ -2,26 +2,61 @@ use quick_cache::{DefaultHashBuilder, UnitWeighter, Weighter}; use std::{ future::Future, hash::{BuildHasher, Hash}, + sync::Arc, time::Duration, }; use crate::{KQCacheWithTTL, PlaceholderGuardWithTTL}; -pub struct CacheWithTTL(KQCacheWithTTL); +pub struct CacheWithTTL( + KQCacheWithTTL, +); impl CacheWithTTL { - pub async fn new_with_unit_weights(estimated_items_capacity: usize, ttl: Duration) -> Self { + pub async fn new_with_capacity(capacity: usize, ttl: Duration) -> Self { Self::new( - estimated_items_capacity, - estimated_items_capacity as u64, + capacity, + capacity as u64, UnitWeighter, DefaultHashBuilder::default(), ttl, ) .await } + + pub async fn arc_with_capacity(capacity: usize, ttl: Duration) -> Arc { + let x = Self::new_with_capacity(capacity, ttl).await; + + Arc::new(x) + } +} + +impl< + Key: Eq + Hash + Clone + Send + Sync + 'static, + Val: Clone + Send + Sync + 'static, + We: Weighter + Clone + Send + Sync + 'static, + B: BuildHasher + Clone + Default + Send + Sync + 'static, + > CacheWithTTL +{ + pub async fn new_with_weights( + estimated_items_capacity: usize, + weight_capacity: u64, + weighter: We, + ttl: Duration, + ) -> Self { + let inner = KQCacheWithTTL::new( + estimated_items_capacity, + weight_capacity, + weighter, + B::default(), + ttl, + ) + .await; + + Self(inner) + } } impl< @@ -50,6 +85,11 @@ impl< Self(inner) } + #[inline] + pub fn get(&self, key: &Key) -> Option { + self.0.get(key, &()) + } + #[inline] pub async fn get_or_insert_async(&self, key: &Key, f: Fut) -> Result where diff --git a/quick_cache_ttl/src/kq_cache.rs b/quick_cache_ttl/src/kq_cache.rs index 5859cb8d..e566c127 100644 --- a/quick_cache_ttl/src/kq_cache.rs +++ b/quick_cache_ttl/src/kq_cache.rs @@ -8,10 +8,10 @@ use tokio::task::JoinHandle; use tokio::time::{sleep_until, Instant}; pub struct KQCacheWithTTL { - pub(crate) cache: Arc>, + cache: Arc>, pub task_handle: JoinHandle<()>, ttl: Duration, - pub(crate) tx: flume::Sender<(Instant, Key, Qey)>, + tx: flume::Sender<(Instant, Key, Qey)>, } struct KQCacheWithTTLTask { @@ -68,6 +68,11 @@ impl< } } + #[inline] + pub fn get(&self, key: &Key, qey: &Qey) -> Option { + self.cache.get(key, qey) + } + #[inline] pub async fn get_or_insert_async(&self, key: &Key, qey: &Qey, f: Fut) -> Result where diff --git a/quick_cache_ttl/src/lib.rs b/quick_cache_ttl/src/lib.rs index b2a05332..ada0c5b0 100644 --- a/quick_cache_ttl/src/lib.rs +++ b/quick_cache_ttl/src/lib.rs @@ -3,4 +3,5 @@ mod kq_cache; pub use cache::CacheWithTTL; pub use kq_cache::{KQCacheWithTTL, PlaceholderGuardWithTTL}; +pub use quick_cache::sync::{Cache, KQCache}; pub use quick_cache::{DefaultHashBuilder, UnitWeighter, Weighter}; diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index bca468c6..1c6b6003 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -62,7 +62,6 @@ itertools = "0.10.5" listenfd = "1.0.1" log = "0.4.17" mimalloc = { version = "0.1.37", optional = true} -moka = { version = "0.11.0", default-features = false, features = ["future"] } num = "0.4.0" num-traits = "0.2.15" once_cell = { version = "1.17.1" } diff --git a/web3_proxy/src/app/mod.rs b/web3_proxy/src/app/mod.rs index ed77cfe8..bdc70c90 100644 --- a/web3_proxy/src/app/mod.rs +++ b/web3_proxy/src/app/mod.rs @@ -46,7 +46,7 @@ use migration::sea_orm::{ }; use migration::sea_query::table::ColumnDef; use migration::{Alias, DbErr, Migrator, MigratorTrait, Table}; -use moka::future::Cache; +use quick_cache_ttl::{Cache, CacheWithTTL}; use redis_rate_limiter::redis::AsyncCommands; use redis_rate_limiter::{redis, DeadpoolRuntime, RedisConfig, RedisPool, RedisRateLimiter}; use serde::Serialize; @@ -61,7 +61,6 @@ use std::time::Duration; use tokio::sync::{broadcast, watch, Semaphore}; use tokio::task::JoinHandle; use tokio::time::{sleep, timeout}; -use ulid::Ulid; // TODO: make this customizable? // 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 @@ -127,9 +126,8 @@ impl DatabaseReplica { } } -// TODO: this should be a the secret key id, not the key itself! -pub type RpcSecretKeyCache = - Cache; +/// Cache data from the database about rpc keys +pub type RpcSecretKeyCache = Arc>; /// The application // TODO: i'm sure this is more arcs than necessary, but spawning futures makes references hard @@ -161,7 +159,7 @@ pub struct Web3ProxyApp { pub hostname: Option, /// store pending transactions that we've seen so that we don't send duplicates to subscribers /// TODO: think about this more. might be worth storing if we sent the transaction or not and using this for automatic retries - pub pending_transactions: Cache, + pub pending_transactions: Arc>, /// rate limit anonymous users pub frontend_ip_rate_limiter: Option>, /// rate limit authenticated users @@ -178,13 +176,11 @@ pub struct Web3ProxyApp { // TODO: should the key be our RpcSecretKey class instead of Ulid? pub rpc_secret_key_cache: RpcSecretKeyCache, /// concurrent/parallel RPC request limits for authenticated users - pub registered_user_semaphores: - Cache, hashbrown::hash_map::DefaultHashBuilder>, + pub rpc_key_semaphores: Cache>, /// concurrent/parallel request limits for anonymous users - pub ip_semaphores: Cache, hashbrown::hash_map::DefaultHashBuilder>, + pub ip_semaphores: Cache>, /// concurrent/parallel application request limits for authenticated users - pub bearer_token_semaphores: - Cache, hashbrown::hash_map::DefaultHashBuilder>, + pub bearer_token_semaphores: Cache>, pub kafka_producer: Option, /// channel for sending stats in a background task pub stat_sender: Option>, @@ -510,10 +506,8 @@ impl Web3ProxyApp { // 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 - let rpc_secret_key_cache = Cache::builder() - .max_capacity(10_000) - .time_to_live(Duration::from_secs(600)) - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); + let rpc_secret_key_cache = + CacheWithTTL::arc_with_capacity(10_000, Duration::from_secs(600)).await; // create a channel for receiving stats // we do this in a channel so we don't slow down our response to the users @@ -603,13 +597,11 @@ impl Web3ProxyApp { // TODO: capacity from configs // all these are the same size, so no need for a weigher // TODO: this used to have a time_to_idle - let pending_transactions = Cache::builder() - .max_capacity(10_000) - // TODO: different chains might handle this differently - // TODO: what should we set? 5 minutes is arbitrary. the nodes themselves hold onto transactions for much longer - // TODO: this used to be time_to_update, but - .time_to_live(Duration::from_secs(300)) - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); + // TODO: different chains might handle this differently + // TODO: what should we set? 5 minutes is arbitrary. the nodes themselves hold onto transactions for much longer + // TODO: this used to be time_to_update, but + let pending_transactions = + CacheWithTTL::arc_with_capacity(10_000, Duration::from_secs(300)).await; // responses can be very different in sizes, so this is a cache with a max capacity and a weigher // TODO: don't allow any response to be bigger than X% of the cache @@ -624,17 +616,15 @@ impl Web3ProxyApp { ) .await; + // TODO: how should we handle hitting this max? + let max_users = 20_000; + // create semaphores for concurrent connection limits + // TODO: how can we implement time til idle? // TODO: what should tti be for semaphores? - let bearer_token_semaphores = Cache::builder() - .time_to_idle(Duration::from_secs(120)) - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); - let ip_semaphores = Cache::builder() - .time_to_idle(Duration::from_secs(120)) - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); - let registered_user_semaphores = Cache::builder() - .time_to_idle(Duration::from_secs(120)) - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); + let bearer_token_semaphores = Cache::new(max_users); + let ip_semaphores = Cache::new(max_users); + let registered_user_semaphores = Cache::new(max_users); let (balanced_rpcs, balanced_handle, consensus_connections_watcher) = Web3Rpcs::spawn( top_config.app.chain_id, @@ -745,7 +735,7 @@ impl Web3ProxyApp { rpc_secret_key_cache, bearer_token_semaphores, ip_semaphores, - registered_user_semaphores, + rpc_key_semaphores: registered_user_semaphores, stat_sender, }; diff --git a/web3_proxy/src/frontend/authorization.rs b/web3_proxy/src/frontend/authorization.rs index 13403ac7..51ad4834 100644 --- a/web3_proxy/src/frontend/authorization.rs +++ b/web3_proxy/src/frontend/authorization.rs @@ -28,7 +28,9 @@ use rdkafka::producer::{FutureProducer, FutureRecord}; use rdkafka::util::Timeout as KafkaTimeout; use redis_rate_limiter::redis::AsyncCommands; use redis_rate_limiter::RedisRateLimitResult; +use std::convert::Infallible; use std::fmt::Display; +use std::hash::Hash; use std::mem; use std::sync::atomic::{self, AtomicBool, AtomicI64, AtomicU64, AtomicUsize}; use std::time::Duration; @@ -47,6 +49,15 @@ pub enum RpcSecretKey { Uuid(Uuid), } +impl Hash for RpcSecretKey { + fn hash(&self, state: &mut H) { + match self { + Self::Ulid(x) => state.write_u128(x.0), + Self::Uuid(x) => state.write_u128(x.as_u128()), + } + } +} + /// TODO: should this have IpAddr and Origin or AuthorizationChecks? #[derive(Debug)] pub enum RateLimitResult { @@ -872,12 +883,12 @@ impl Web3ProxyApp { if let Some(max_concurrent_requests) = self.config.public_max_concurrent_requests { let semaphore = self .ip_semaphores - .get_with_by_ref(ip, async move { + .get_or_insert_async::(ip, async move { // TODO: set max_concurrent_requests dynamically based on load? let s = Semaphore::new(max_concurrent_requests); - Arc::new(s) + Ok(Arc::new(s)) }) - .await; + .await?; // if semaphore.available_permits() == 0 { // // TODO: concurrent limit hit! emit a stat? less important for anon users @@ -901,17 +912,17 @@ impl Web3ProxyApp { let user_id = authorization_checks .user_id .try_into() - .or(Err(Web3ProxyError::UserIdZero)) - .web3_context("user ids should always be non-zero")?; + .or(Err(Web3ProxyError::UserIdZero))?; let semaphore = self - .registered_user_semaphores - .get_with(user_id, async move { + .rpc_key_semaphores + .get_or_insert_async(&user_id, async move { let s = Semaphore::new(max_concurrent_requests as usize); // trace!("new semaphore for user_id {}", user_id); - Arc::new(s) + Ok::<_, Infallible>(Arc::new(s)) }) - .await; + .await + .unwrap(); // if semaphore.available_permits() == 0 { // // TODO: concurrent limit hit! emit a stat? this has a race condition though. @@ -939,11 +950,12 @@ impl Web3ProxyApp { // limit concurrent requests let semaphore = self .bearer_token_semaphores - .get_with_by_ref(&user_bearer_token, async move { + .get_or_insert_async::(&user_bearer_token, async move { let s = Semaphore::new(self.config.bearer_token_max_concurrent_requests as usize); - Arc::new(s) + Ok(Arc::new(s)) }) - .await; + .await + .unwrap(); let semaphore_permit = semaphore.acquire_owned().await?; @@ -1086,9 +1098,9 @@ impl Web3ProxyApp { proxy_mode: ProxyMode, rpc_secret_key: RpcSecretKey, ) -> Web3ProxyResult { - let authorization_checks: Result<_, Arc> = self + let authorization_checks: Result<_, Web3ProxyError> = self .rpc_secret_key_cache - .try_get_with(rpc_secret_key.into(), async move { + .get_or_insert_async(&rpc_secret_key, async move { // trace!(?rpc_secret_key, "user cache miss"); let db_replica = self @@ -1107,6 +1119,7 @@ impl Web3ProxyApp { Some(rpc_key_model) => { // TODO: move these splits into helper functions // TODO: can we have sea orm handle this for us? + // TODO: don't expect. return an application error let user_model = user::Entity::find_by_id(rpc_key_model.user_id) .one(db_replica.conn()) .await? @@ -1209,7 +1222,7 @@ impl Web3ProxyApp { }) .await; - authorization_checks.map_err(Web3ProxyError::Arc) + authorization_checks } /// Authorized the ip/origin/referer/useragent and rate limit and concurrency diff --git a/web3_proxy/src/frontend/errors.rs b/web3_proxy/src/frontend/errors.rs index e16b674e..9126526e 100644 --- a/web3_proxy/src/frontend/errors.rs +++ b/web3_proxy/src/frontend/errors.rs @@ -5,7 +5,6 @@ use crate::jsonrpc::{JsonRpcErrorData, JsonRpcForwardedResponse}; use crate::response_cache::JsonRpcResponseData; use std::error::Error; -use std::sync::Arc; use std::{borrow::Cow, net::IpAddr}; use axum::{ @@ -33,13 +32,11 @@ impl From for Web3ProxyResult<()> { } } -// TODO: #[derive(Debug, Display, Error, From)] pub enum Web3ProxyError { AccessDenied, #[error(ignore)] Anyhow(anyhow::Error), - Arc(Arc), #[error(ignore)] #[from(ignore)] BadRequest(String), @@ -685,13 +682,6 @@ impl Web3ProxyError { }, ) } - Self::Arc(err) => { - return match Arc::try_unwrap(err) { - Ok(err) => err, - Err(err) => Self::Anyhow(anyhow::anyhow!("{}", err)), - } - .into_response_parts(); - } Self::SemaphoreAcquireError(err) => { warn!("semaphore acquire err={:?}", err); ( diff --git a/web3_proxy/src/frontend/mod.rs b/web3_proxy/src/frontend/mod.rs index 9eb2f6d9..e1496960 100644 --- a/web3_proxy/src/frontend/mod.rs +++ b/web3_proxy/src/frontend/mod.rs @@ -56,7 +56,7 @@ pub async fn serve( let response_cache_size = ResponseCacheKey::COUNT; let response_cache = - ResponseCache::new_with_unit_weights(response_cache_size, Duration::from_secs(1)).await; + ResponseCache::new_with_capacity(response_cache_size, Duration::from_secs(1)).await; // TODO: read config for if fastest/versus should be available publicly. default off diff --git a/web3_proxy/src/rpcs/blockchain.rs b/web3_proxy/src/rpcs/blockchain.rs index 828eacb3..ff2c08c4 100644 --- a/web3_proxy/src/rpcs/blockchain.rs +++ b/web3_proxy/src/rpcs/blockchain.rs @@ -10,10 +10,11 @@ use crate::{config::BlockAndRpc, jsonrpc::JsonRpcRequest}; use derive_more::From; use ethers::prelude::{Block, TxHash, H256, U64}; use log::{debug, trace, warn, Level}; -use moka::future::Cache; +use quick_cache_ttl::CacheWithTTL; use serde::ser::SerializeStruct; use serde::Serialize; use serde_json::json; +use std::convert::Infallible; use std::hash::Hash; use std::{cmp::Ordering, fmt::Display, sync::Arc}; use tokio::sync::broadcast; @@ -22,7 +23,8 @@ use tokio::time::Duration; // TODO: type for Hydrated Blocks with their full transactions? pub type ArcBlock = Arc>; -pub type BlocksByHashCache = Cache; +pub type BlocksByHashCache = Arc>; +pub type BlocksByNumberCache = Arc>; /// A block and its age. #[derive(Clone, Debug, Default, From)] @@ -168,9 +170,7 @@ impl Web3Rpcs { heaviest_chain: bool, ) -> Web3ProxyResult { // TODO: i think we can rearrange this function to make it faster on the hot path - let block_hash = block.hash(); - - if block_hash.is_zero() { + if block.hash().is_zero() { debug!("Skipping block without hash!"); return Ok(block); } @@ -182,15 +182,18 @@ impl Web3Rpcs { // this is the only place that writes to block_numbers // multiple inserts should be okay though // TODO: info that there was a fork? - self.blocks_by_number.insert(*block_num, *block_hash).await; + self.blocks_by_number.insert(*block_num, *block.hash()); } // this block is very likely already in block_hashes // TODO: use their get_with + let block_hash = *block.hash(); + let block = self .blocks_by_hash - .get_with(*block_hash, async move { block }) - .await; + .get_or_insert_async::(&block_hash, async move { Ok(block) }) + .await + .unwrap(); Ok(block) } @@ -423,7 +426,7 @@ impl Web3Rpcs { return Ok(()); } - let new_synced_connections = match consensus_finder + let new_consensus_rpcs = match consensus_finder .find_consensus_connections(authorization, self) .await { @@ -436,21 +439,21 @@ impl Web3Rpcs { Ok(Some(x)) => x, }; - trace!("new_synced_connections: {:#?}", new_synced_connections); + trace!("new_synced_connections: {:#?}", new_consensus_rpcs); let watch_consensus_head_sender = self.watch_consensus_head_sender.as_ref().unwrap(); - let consensus_tier = new_synced_connections.tier; + let consensus_tier = new_consensus_rpcs.tier; // TODO: think more about this unwrap let total_tiers = consensus_finder.worst_tier().unwrap_or(10); - let backups_needed = new_synced_connections.backups_needed; - let consensus_head_block = new_synced_connections.head_block.clone(); - let num_consensus_rpcs = new_synced_connections.num_consensus_rpcs(); + let backups_needed = new_consensus_rpcs.backups_needed; + let consensus_head_block = new_consensus_rpcs.head_block.clone(); + let num_consensus_rpcs = new_consensus_rpcs.num_consensus_rpcs(); let num_active_rpcs = consensus_finder.len(); let total_rpcs = self.by_name.load().len(); let old_consensus_head_connections = self .watch_consensus_rpcs_sender - .send_replace(Some(Arc::new(new_synced_connections))); + .send_replace(Some(Arc::new(new_consensus_rpcs))); let backups_voted_str = if backups_needed { "B " } else { "" }; diff --git a/web3_proxy/src/rpcs/consensus.rs b/web3_proxy/src/rpcs/consensus.rs index 90385901..806dc448 100644 --- a/web3_proxy/src/rpcs/consensus.rs +++ b/web3_proxy/src/rpcs/consensus.rs @@ -8,10 +8,11 @@ use ethers::prelude::{H256, U64}; use hashbrown::{HashMap, HashSet}; use itertools::{Itertools, MinMaxResult}; use log::{debug, trace, warn}; -use moka::future::Cache; +use quick_cache_ttl::Cache; use serde::Serialize; use std::cmp::{Ordering, Reverse}; use std::collections::BTreeMap; +use std::convert::Infallible; use std::fmt; use std::sync::Arc; use tokio::time::Instant; @@ -321,7 +322,7 @@ impl Web3Rpcs { } } -type FirstSeenCache = Cache; +type FirstSeenCache = Cache; /// A ConsensusConnections builder that tracks all connection heads across multiple groups of servers pub struct ConsensusFinder { @@ -342,9 +343,7 @@ impl ConsensusFinder { pub fn new(max_block_age: Option, max_block_lag: Option) -> Self { // TODO: what's a good capacity for this? it shouldn't need to be very large // TODO: if we change Web3ProxyBlock to store the instance, i think we could use the block_by_hash cache - let first_seen = Cache::builder() - .max_capacity(16) - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); + let first_seen = Cache::new(16); // TODO: hard coding 0-9 isn't great, but its easier than refactoring this to be smart about config reloading let rpc_heads = HashMap::new(); @@ -372,8 +371,9 @@ impl ConsensusFinder { async fn insert(&mut self, rpc: Arc, block: Web3ProxyBlock) -> Option { let first_seen = self .first_seen - .get_with_by_ref(block.hash(), async { Instant::now() }) - .await; + .get_or_insert_async::(block.hash(), async { Ok(Instant::now()) }) + .await + .unwrap(); // calculate elapsed time before trying to lock let latency = first_seen.elapsed(); diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index 6030a7d4..6311b5b7 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -1,5 +1,5 @@ ///! Load balanced communication with a group of web3 rpc providers -use super::blockchain::{BlocksByHashCache, Web3ProxyBlock}; +use super::blockchain::{BlocksByHashCache, BlocksByNumberCache, Web3ProxyBlock}; use super::consensus::{ConsensusWeb3Rpcs, ShouldWaitForBlock}; use super::one::Web3Rpc; use super::request::{OpenRequestHandle, OpenRequestResult, RequestErrorHandler}; @@ -16,7 +16,7 @@ use anyhow::Context; use arc_swap::ArcSwap; use counter::Counter; use derive_more::From; -use ethers::prelude::{ProviderError, TxHash, H256, U64}; +use ethers::prelude::{ProviderError, TxHash, U64}; use futures::future::try_join_all; use futures::stream::FuturesUnordered; use futures::StreamExt; @@ -24,8 +24,8 @@ use hashbrown::{HashMap, HashSet}; use itertools::Itertools; use log::{debug, error, info, trace, warn, Level}; use migration::sea_orm::DatabaseConnection; -use moka::future::Cache; use ordered_float::OrderedFloat; +use quick_cache_ttl::CacheWithTTL; use serde::ser::{SerializeStruct, Serializer}; use serde::Serialize; use serde_json::json; @@ -58,15 +58,14 @@ pub struct Web3Rpcs { /// this head receiver makes it easy to wait until there is a new block pub(super) watch_consensus_head_sender: Option>>, /// keep track of transactions that we have sent through subscriptions - pub(super) pending_transaction_cache: - Cache, + pub(super) pending_transaction_cache: Arc>, pub(super) pending_tx_id_receiver: flume::Receiver, pub(super) pending_tx_id_sender: flume::Sender, /// TODO: this map is going to grow forever unless we do some sort of pruning. maybe store pruned in redis? /// all blocks, including orphans pub(super) blocks_by_hash: BlocksByHashCache, /// blocks on the heaviest chain - pub(super) blocks_by_number: Cache, + pub(super) blocks_by_number: BlocksByNumberCache, /// the number of rpcs required to agree on consensus for the head block (thundering herd protection) pub(super) min_head_rpcs: usize, /// the soft limit required to agree on consensus for the head block. (thundering herd protection) @@ -89,7 +88,7 @@ impl Web3Rpcs { min_head_rpcs: usize, min_sum_soft_limit: u32, name: String, - pending_transaction_cache: Cache, + pending_transaction_cache: Arc>, pending_tx_sender: Option>, watch_consensus_head_sender: Option>>, ) -> anyhow::Result<( @@ -159,24 +158,23 @@ impl Web3Rpcs { }; // these blocks don't have full transactions, but they do have rather variable amounts of transaction hashes - // TODO: how can we do the weigher better? need to know actual allocated size + // TODO: how can we do the weigher this? need to know actual allocated size // TODO: time_to_idle instead? // TODO: limits from config - let blocks_by_hash: BlocksByHashCache = Cache::builder() - .max_capacity(1024 * 1024 * 1024) - .weigher(|_k, v: &Web3ProxyBlock| { - 1 + v.block.transactions.len().try_into().unwrap_or(u32::MAX) - }) - .time_to_live(Duration::from_secs(30 * 60)) - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); + let blocks_by_hash: BlocksByHashCache = + Arc::new(CacheWithTTL::new_with_capacity(10_000, Duration::from_secs(30 * 60)).await); + // .max_capacity(1024 * 1024 * 1024) + // .weigher(|_k, v: &Web3ProxyBlock| { + // 1 + v.block.transactions.len().try_into().unwrap_or(u32::MAX) + // }) + // .time_to_live(Duration::from_secs(30 * 60)) + // .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); // all block numbers are the same size, so no need for weigher // TODO: limits from config // TODO: time_to_idle instead? - let blocks_by_number = Cache::builder() - .time_to_live(Duration::from_secs(30 * 60)) - .max_capacity(10_000) - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); + let blocks_by_number = + Arc::new(CacheWithTTL::new_with_capacity(10_000, Duration::from_secs(30 * 60)).await); let (watch_consensus_rpcs_sender, consensus_connections_watcher) = watch::channel(Default::default()); @@ -264,7 +262,7 @@ impl Web3Rpcs { }; let pending_tx_id_sender = Some(self.pending_tx_id_sender.clone()); - let blocks_by_hash = self.blocks_by_hash.clone(); + let blocks_by_hash_cache = self.blocks_by_hash.clone(); let http_interval_sender = self.http_interval_sender.clone(); let chain_id = app.config.chain_id; @@ -277,7 +275,7 @@ impl Web3Rpcs { chain_id, http_client, http_interval_sender, - blocks_by_hash, + blocks_by_hash_cache, block_sender, pending_tx_id_sender, true, @@ -1249,6 +1247,7 @@ mod tests { use crate::rpcs::consensus::ConsensusFinder; use crate::rpcs::{blockchain::Web3ProxyBlock, provider::Web3Provider}; use arc_swap::ArcSwap; + use ethers::types::H256; use ethers::types::{Block, U256}; use latency::PeakEwmaLatency; use log::{trace, LevelFilter}; @@ -1436,14 +1435,15 @@ mod tests { name: "test".to_string(), watch_consensus_head_sender: Some(watch_consensus_head_sender), watch_consensus_rpcs_sender, - pending_transaction_cache: Cache::builder() - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()), + pending_transaction_cache: CacheWithTTL::arc_with_capacity( + 100, + Duration::from_secs(60), + ) + .await, pending_tx_id_receiver, pending_tx_id_sender, - blocks_by_hash: Cache::builder() - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()), - blocks_by_number: Cache::builder() - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()), + blocks_by_hash: CacheWithTTL::arc_with_capacity(100, Duration::from_secs(60)).await, + blocks_by_number: CacheWithTTL::arc_with_capacity(100, Duration::from_secs(60)).await, // TODO: test max_block_age? max_block_age: None, // TODO: test max_block_lag? @@ -1688,14 +1688,15 @@ mod tests { name: "test".to_string(), watch_consensus_head_sender: Some(watch_consensus_head_sender), watch_consensus_rpcs_sender, - pending_transaction_cache: Cache::builder() - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()), + pending_transaction_cache: CacheWithTTL::arc_with_capacity( + 100, + Duration::from_secs(120), + ) + .await, pending_tx_id_receiver, pending_tx_id_sender, - blocks_by_hash: Cache::builder() - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()), - blocks_by_number: Cache::builder() - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()), + blocks_by_hash: CacheWithTTL::arc_with_capacity(100, Duration::from_secs(120)).await, + blocks_by_number: CacheWithTTL::arc_with_capacity(100, Duration::from_secs(120)).await, min_head_rpcs: 1, min_sum_soft_limit: 4_000, max_block_age: None, @@ -1853,14 +1854,16 @@ mod tests { name: "test".to_string(), watch_consensus_head_sender: Some(watch_consensus_head_sender), watch_consensus_rpcs_sender, - pending_transaction_cache: Cache::builder() - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()), + pending_transaction_cache: CacheWithTTL::arc_with_capacity( + 10_000, + Duration::from_secs(120), + ) + .await, pending_tx_id_receiver, pending_tx_id_sender, - blocks_by_hash: Cache::builder() - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()), - blocks_by_number: Cache::builder() - .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()), + blocks_by_hash: CacheWithTTL::arc_with_capacity(10_000, Duration::from_secs(120)).await, + blocks_by_number: CacheWithTTL::arc_with_capacity(10_000, Duration::from_secs(120)) + .await, min_head_rpcs: 1, min_sum_soft_limit: 1_000, max_block_age: None, diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index b37a4425..6095c872 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -23,6 +23,7 @@ use serde::Serialize; use serde_json::json; use std::borrow::Cow; use std::cmp::min; +use std::convert::Infallible; use std::fmt; use std::hash::{Hash, Hasher}; use std::sync::atomic::{self, AtomicBool, AtomicU64, AtomicUsize}; @@ -622,8 +623,11 @@ impl Web3Rpc { // if we already have this block saved, set new_head_block to that arc. otherwise store this copy let new_head_block = block_map - .get_with(new_hash, async move { new_head_block }) - .await; + .get_or_insert_async::( + &new_hash, + async move { Ok(new_head_block) }, + ) + .await?; // save the block so we don't send the same one multiple times // also save so that archive checks can know how far back to query diff --git a/web3_proxy/src/rpcs/transactions.rs b/web3_proxy/src/rpcs/transactions.rs index 687c5224..d8c007ee 100644 --- a/web3_proxy/src/rpcs/transactions.rs +++ b/web3_proxy/src/rpcs/transactions.rs @@ -82,7 +82,7 @@ impl Web3Rpcs { } // trace!(?pending_tx_id, "checking pending_transactions on {}", rpc); - if self.pending_transaction_cache.contains_key(&pending_tx_id) { + if self.pending_transaction_cache.get(&pending_tx_id).is_some() { // this transaction has already been processed return Ok(()); } diff --git a/web3_proxy/src/stats/stat_buffer.rs b/web3_proxy/src/stats/stat_buffer.rs index d7a5dea1..f8fd2db8 100644 --- a/web3_proxy/src/stats/stat_buffer.rs +++ b/web3_proxy/src/stats/stat_buffer.rs @@ -4,7 +4,7 @@ use derive_more::From; use futures::stream; use hashbrown::HashMap; use influxdb2::api::write::TimestampPrecision; -use log::{debug, error, info, trace}; +use log::{error, info, trace}; use migration::sea_orm::prelude::Decimal; use migration::sea_orm::DatabaseConnection; use std::time::Duration; From 8898ffe57884d3c188be97fb664d6d308bb5dc3a Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Thu, 18 May 2023 13:36:35 -0700 Subject: [PATCH 42/66] cargo upgrade --- Cargo.lock | 112 +++++++++++++++++++++++++++--------------- entities/Cargo.toml | 2 +- web3_proxy/Cargo.toml | 8 +-- 3 files changed, 77 insertions(+), 45 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 410724bc..7d73a5b1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1243,7 +1243,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bef552e6f588e446098f6ba40d89ac146c8c7b64aade83c051ee00bb5d2bc18d" dependencies = [ "serde", - "uuid 1.3.2", + "uuid 1.3.3", ] [[package]] @@ -1532,7 +1532,7 @@ dependencies = [ "sea-orm", "serde", "ulid", - "uuid 1.3.2", + "uuid 1.3.3", ] [[package]] @@ -1708,7 +1708,7 @@ dependencies = [ "serde_json", "syn 2.0.15", "tokio", - "toml 0.7.3", + "toml 0.7.4", "url", "walkdir", ] @@ -2580,15 +2580,15 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.23.2" +version = "0.24.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1788965e61b367cd03a62950836d5cd41560c3577d90e40e0819373194d1661c" +checksum = "0646026eb1b3eea4cd9ba47912ea5ce9cc07713d105b1a14698f4e6433d348b7" dependencies = [ "http", "hyper", - "rustls", + "rustls 0.21.1", "tokio", - "tokio-rustls", + "tokio-rustls 0.24.0", ] [[package]] @@ -3011,7 +3011,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e0500463acd96259d219abb05dc57e5a076ef04b2db9a2112846929b5f174c96" dependencies = [ "libc", - "uuid 1.3.2", + "uuid 1.3.3", "winapi", ] @@ -4187,9 +4187,9 @@ dependencies = [ [[package]] name = "rdkafka" -version = "0.30.0" +version = "0.31.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97640b53443880ca65df40e9373a8193f9ad58b3f7419bc7206067f4a952500d" +checksum = "88383df3a85a38adfa2aa447d3ab6eb9cedcb49613adcf18e7e7ebb3b62e9b03" dependencies = [ "futures-channel", "futures-util", @@ -4319,9 +4319,9 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.11.17" +version = "0.11.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13293b639a097af28fc8a90f22add145a9c954e49d77da06263d58cf44d5fb91" +checksum = "cde824a14b7c14f85caff81225f411faacc04a2013f41670f41443742b1c1c55" dependencies = [ "base64 0.21.0", "bytes", @@ -4342,14 +4342,14 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls", + "rustls 0.21.1", "rustls-pemfile", "serde", "serde_json", "serde_urlencoded", "tokio", "tokio-native-tls", - "tokio-rustls", + "tokio-rustls 0.24.0", "tokio-util", "tower-service", "url", @@ -4566,6 +4566,18 @@ dependencies = [ "webpki", ] +[[package]] +name = "rustls" +version = "0.21.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c911ba11bc8433e811ce56fde130ccf32f5127cab0e0194e9c68c5a5b671791e" +dependencies = [ + "log", + "ring", + "rustls-webpki", + "sct", +] + [[package]] name = "rustls-pemfile" version = "1.0.2" @@ -4575,6 +4587,16 @@ dependencies = [ "base64 0.21.0", ] +[[package]] +name = "rustls-webpki" +version = "0.100.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6207cd5ed3d8dca7816f8f3725513a34609c0c765bf652b8c3cb4cfd87db46b" +dependencies = [ + "ring", + "untrusted", +] + [[package]] name = "rustversion" version = "1.0.12" @@ -4703,7 +4725,7 @@ dependencies = [ "time 0.3.21", "tracing", "url", - "uuid 1.3.2", + "uuid 1.3.3", ] [[package]] @@ -4764,7 +4786,7 @@ dependencies = [ "sea-query-derive", "serde_json", "time 0.3.21", - "uuid 1.3.2", + "uuid 1.3.3", ] [[package]] @@ -4780,7 +4802,7 @@ dependencies = [ "serde_json", "sqlx", "time 0.3.21", - "uuid 1.3.2", + "uuid 1.3.3", ] [[package]] @@ -4927,7 +4949,7 @@ checksum = "6c3d7f8bf7373e75222452fcdd9347d857452a92d0eec738f941bc4656c5b5df" dependencies = [ "httpdate", "reqwest", - "rustls", + "rustls 0.20.8", "sentry-anyhow", "sentry-backtrace", "sentry-contexts", @@ -5023,7 +5045,7 @@ dependencies = [ "thiserror", "time 0.3.21", "url", - "uuid 1.3.2", + "uuid 1.3.3", ] [[package]] @@ -5101,9 +5123,9 @@ dependencies = [ [[package]] name = "serde_spanned" -version = "0.6.1" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0efd8caf556a6cebd3b285caf480045fcc1ac04f6bd786b09a6f11af30c4fcf4" +checksum = "93107647184f6027e3b7dcb2e11034cf95ffa1e3a682c67951963ac69c1c007d" dependencies = [ "serde", ] @@ -5415,7 +5437,7 @@ dependencies = [ "rand", "rsa", "rust_decimal", - "rustls", + "rustls 0.20.8", "rustls-pemfile", "serde", "serde_json", @@ -5429,7 +5451,7 @@ dependencies = [ "time 0.3.21", "tokio-stream", "url", - "uuid 1.3.2", + "uuid 1.3.3", "webpki-roots", ] @@ -5460,7 +5482,7 @@ checksum = "804d3f245f894e61b1e6263c84b23ca675d96753b5abfd5cc8597d86806e8024" dependencies = [ "once_cell", "tokio", - "tokio-rustls", + "tokio-rustls 0.23.4", ] [[package]] @@ -5844,11 +5866,21 @@ version = "0.23.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c43ee83903113e03984cb9e5cebe6c04a5116269e900e3ddba8f068a62adda59" dependencies = [ - "rustls", + "rustls 0.20.8", "tokio", "webpki", ] +[[package]] +name = "tokio-rustls" +version = "0.24.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e0d409377ff5b1e3ca6437aa86c1eb7d40c134bfec254e44c830defa92669db5" +dependencies = [ + "rustls 0.21.1", + "tokio", +] + [[package]] name = "tokio-stream" version = "0.1.14" @@ -5869,9 +5901,9 @@ checksum = "54319c93411147bced34cb5609a80e0a8e44c5999c93903a81cd866630ec0bfd" dependencies = [ "futures-util", "log", - "rustls", + "rustls 0.20.8", "tokio", - "tokio-rustls", + "tokio-rustls 0.23.4", "tungstenite", "webpki", "webpki-roots", @@ -5916,9 +5948,9 @@ dependencies = [ [[package]] name = "toml" -version = "0.7.3" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b403acf6f2bb0859c93c7f0d967cb4a75a7ac552100f9322faf64dc047669b21" +checksum = "d6135d499e69981f9ff0ef2167955a5333c35e36f6937d382974566b3d5b94ec" dependencies = [ "serde", "serde_spanned", @@ -5928,18 +5960,18 @@ dependencies = [ [[package]] name = "toml_datetime" -version = "0.6.1" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ab8ed2edee10b50132aed5f331333428b011c99402b5a534154ed15746f9622" +checksum = "5a76a9312f5ba4c2dec6b9161fdf25d87ad8a09256ccea5a556fef03c706a10f" dependencies = [ "serde", ] [[package]] name = "toml_edit" -version = "0.19.8" +version = "0.19.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "239410c8609e8125456927e6707163a3b1fdb40561e4b803bc041f466ccfdc13" +checksum = "92d964908cec0d030b812013af25a0e57fddfadb1e066ecc6681d86253129d4f" dependencies = [ "indexmap", "serde", @@ -6173,7 +6205,7 @@ dependencies = [ "httparse", "log", "rand", - "rustls", + "rustls 0.20.8", "sha1", "thiserror", "url", @@ -6213,7 +6245,7 @@ checksum = "13a3aaa69b04e5b66cc27309710a569ea23593612387d67daaf102e73aa974fd" dependencies = [ "rand", "serde", - "uuid 1.3.2", + "uuid 1.3.3", ] [[package]] @@ -6285,7 +6317,7 @@ dependencies = [ "base64 0.13.1", "log", "once_cell", - "rustls", + "rustls 0.20.8", "url", "webpki", "webpki-roots", @@ -6321,9 +6353,9 @@ dependencies = [ [[package]] name = "uuid" -version = "1.3.2" +version = "1.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dad5567ad0cf5b760e5665964bec1b47dfd077ba8a2544b513f3556d3d239a2" +checksum = "345444e32442451b267fc254ae85a209c64be56d2890e601a0c37ff0c3c5ecd2" dependencies = [ "getrandom", "serde", @@ -6535,12 +6567,12 @@ dependencies = [ "tokio-console", "tokio-stream", "tokio-uring", - "toml 0.7.3", + "toml 0.7.4", "tower", "tower-http", "ulid", "url", - "uuid 1.3.2", + "uuid 1.3.3", ] [[package]] diff --git a/entities/Cargo.toml b/entities/Cargo.toml index c88d7668..0c6b4df8 100644 --- a/entities/Cargo.toml +++ b/entities/Cargo.toml @@ -12,6 +12,6 @@ path = "src/mod.rs" [dependencies] sea-orm = "0.11.3" serde = "1.0.163" -uuid = "1.3.2" +uuid = "1.3.3" ethers = "2.0.4" ulid = "1.0.0" diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index 1c6b6003..a91eddb6 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -70,9 +70,9 @@ pagerduty-rs = { version = "0.1.6", default-features = false, features = ["async parking_lot = { version = "0.12.1", features = ["arc_lock"] } prettytable = "*" proctitle = "0.1.1" -rdkafka = { version = "0.30.0" } +rdkafka = { version = "0.31.0" } regex = "1.8.1" -reqwest = { version = "0.11.17", default-features = false, features = ["json", "tokio-rustls"] } +reqwest = { version = "0.11.18", default-features = false, features = ["json", "tokio-rustls"] } rmp-serde = "1.1.1" sentry = { version = "0.31.0", default-features = false, features = ["backtrace", "contexts", "panic", "anyhow", "reqwest", "rustls", "log", "sentry-log"] } serde = { version = "1.0.163", features = [] } @@ -85,9 +85,9 @@ tokio = { version = "1.28.1", features = ["full"] } tokio-console = { version = "*", optional = true } tokio-stream = { version = "0.1.14", features = ["sync"] } tokio-uring = { version = "0.4.0", optional = true } -toml = "0.7.3" +toml = "0.7.4" tower = "0.4.13" tower-http = { version = "0.4.0", features = ["cors", "sensitive-headers"] } ulid = { version = "1.0.0", features = ["uuid", "serde"] } url = "2.3.1" -uuid = "1.3.2" +uuid = "1.3.3" From a92c93706bafad9bf4f656afbbda92113adda015 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Thu, 18 May 2023 13:47:00 -0700 Subject: [PATCH 43/66] dont check heads while waiting this isn't great. but should work better than what we have --- web3_proxy/src/rpcs/consensus.rs | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/web3_proxy/src/rpcs/consensus.rs b/web3_proxy/src/rpcs/consensus.rs index 806dc448..1502966c 100644 --- a/web3_proxy/src/rpcs/consensus.rs +++ b/web3_proxy/src/rpcs/consensus.rs @@ -129,18 +129,19 @@ impl ConsensusWeb3Rpcs { needed_block_num: Option<&U64>, skip_rpcs: &[Arc], ) -> ShouldWaitForBlock { - if self - .head_rpcs - .iter() - .any(|rpc| self.rpc_will_work_eventually(rpc, needed_block_num, skip_rpcs)) - { - let head_num = self.head_block.number(); + // TODO: i think checking synced is always a waste of time. though i guess there could be a race + // if self + // .head_rpcs + // .iter() + // .any(|rpc| self.rpc_will_work_eventually(rpc, needed_block_num, skip_rpcs)) + // { + // let head_num = self.head_block.number(); - if Some(head_num) >= needed_block_num { - debug!("best (head) block: {}", head_num); - return ShouldWaitForBlock::Ready; - } - } + // if Some(head_num) >= needed_block_num { + // debug!("best (head) block: {}", head_num); + // return ShouldWaitForBlock::Ready; + // } + // } // all of the head rpcs are skipped From 9c584354d9ab8c3f1d623fb00094f9d6191e6d16 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Thu, 18 May 2023 13:51:28 -0700 Subject: [PATCH 44/66] add to skip list earlier --- web3_proxy/src/frontend/users/payment.rs | 6 ++--- web3_proxy/src/rpcs/consensus.rs | 22 ++++++++--------- web3_proxy/src/rpcs/many.rs | 31 ++++++++++++++---------- 3 files changed, 32 insertions(+), 27 deletions(-) diff --git a/web3_proxy/src/frontend/users/payment.rs b/web3_proxy/src/frontend/users/payment.rs index b49c3705..fb2f0d91 100644 --- a/web3_proxy/src/frontend/users/payment.rs +++ b/web3_proxy/src/frontend/users/payment.rs @@ -153,7 +153,7 @@ pub async fn user_balance_post( // Just make an rpc request, idk if i need to call this super extensive code let transaction_receipt: TransactionReceipt = match app .balanced_rpcs - .best_available_rpc(&authorization, None, &[], None, None) + .best_available_rpc(&authorization, None, &mut vec![], None, None) .await { Ok(OpenRequestResult::Handle(handle)) => { @@ -188,7 +188,7 @@ pub async fn user_balance_post( debug!("Transaction receipt is: {:?}", transaction_receipt); let accepted_token: Address = match app .balanced_rpcs - .best_available_rpc(&authorization, None, &[], None, None) + .best_available_rpc(&authorization, None, &mut vec![], None, None) .await { Ok(OpenRequestResult::Handle(handle)) => { @@ -243,7 +243,7 @@ pub async fn user_balance_post( debug!("Accepted token is: {:?}", accepted_token); let decimals: u32 = match app .balanced_rpcs - .best_available_rpc(&authorization, None, &[], None, None) + .best_available_rpc(&authorization, None, &mut vec![], None, None) .await { Ok(OpenRequestResult::Handle(handle)) => { diff --git a/web3_proxy/src/rpcs/consensus.rs b/web3_proxy/src/rpcs/consensus.rs index 1502966c..2ca28519 100644 --- a/web3_proxy/src/rpcs/consensus.rs +++ b/web3_proxy/src/rpcs/consensus.rs @@ -130,18 +130,18 @@ impl ConsensusWeb3Rpcs { skip_rpcs: &[Arc], ) -> ShouldWaitForBlock { // TODO: i think checking synced is always a waste of time. though i guess there could be a race - // if self - // .head_rpcs - // .iter() - // .any(|rpc| self.rpc_will_work_eventually(rpc, needed_block_num, skip_rpcs)) - // { - // let head_num = self.head_block.number(); + if self + .head_rpcs + .iter() + .any(|rpc| self.rpc_will_work_eventually(rpc, needed_block_num, skip_rpcs)) + { + let head_num = self.head_block.number(); - // if Some(head_num) >= needed_block_num { - // debug!("best (head) block: {}", head_num); - // return ShouldWaitForBlock::Ready; - // } - // } + if Some(head_num) >= needed_block_num { + debug!("best (head) block: {}", head_num); + return ShouldWaitForBlock::Ready; + } + } // all of the head rpcs are skipped diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index 6311b5b7..5921f895 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -495,7 +495,7 @@ impl Web3Rpcs { &self, authorization: &Arc, request_metadata: Option<&Arc>, - skip: &[Arc], + skip: &mut Vec>, min_block_needed: Option<&U64>, max_block_needed: Option<&U64>, ) -> Web3ProxyResult { @@ -592,6 +592,8 @@ impl Web3Rpcs { let best_rpc = min_by_key(rpc_a, rpc_b, |x| x.peak_ewma()); trace!("{:?} - winner: {}", request_ulid, best_rpc); + skip.push(best_rpc.clone()); + // just because it has lower latency doesn't mean we are sure to get a connection match best_rpc.try_request_handle(authorization, None).await { Ok(OpenRequestResult::Handle(handle)) => { @@ -805,7 +807,7 @@ impl Web3Rpcs { .best_available_rpc( authorization, request_metadata, - &skip_rpcs, + &mut skip_rpcs, min_block_needed, max_block_needed, ) @@ -822,9 +824,6 @@ impl Web3Rpcs { let is_backup_response = rpc.backup; - // TODO: instead of entirely skipping, maybe demote a tier? - skip_rpcs.push(rpc); - // TODO: get the log percent from the user data let response_result: Result, _> = active_request_handle .request( @@ -1493,7 +1492,7 @@ mod tests { .best_available_rpc( &authorization, None, - &[], + &mut vec![], Some(head_block.number.as_ref().unwrap()), None, ) @@ -1587,28 +1586,28 @@ mod tests { // TODO: make sure the handle is for the expected rpc assert!(matches!( - rpcs.best_available_rpc(&authorization, None, &[], None, None) + rpcs.best_available_rpc(&authorization, None, &mut vec![], None, None) .await, Ok(OpenRequestResult::Handle(_)) )); // TODO: make sure the handle is for the expected rpc assert!(matches!( - rpcs.best_available_rpc(&authorization, None, &[], Some(&0.into()), None) + rpcs.best_available_rpc(&authorization, None, &mut vec![], Some(&0.into()), None) .await, Ok(OpenRequestResult::Handle(_)) )); // TODO: make sure the handle is for the expected rpc assert!(matches!( - rpcs.best_available_rpc(&authorization, None, &[], Some(&1.into()), None) + rpcs.best_available_rpc(&authorization, None, &mut vec![], Some(&1.into()), None) .await, Ok(OpenRequestResult::Handle(_)) )); // future block should not get a handle let future_rpc = rpcs - .best_available_rpc(&authorization, None, &[], Some(&2.into()), None) + .best_available_rpc(&authorization, None, &mut vec![], Some(&2.into()), None) .await; assert!(matches!(future_rpc, Ok(OpenRequestResult::NotReady))); } @@ -1733,7 +1732,13 @@ mod tests { // best_synced_backend_connection requires servers to be synced with the head block // TODO: test with and without passing the head_block.number? let best_available_server = rpcs - .best_available_rpc(&authorization, None, &[], Some(head_block.number()), None) + .best_available_rpc( + &authorization, + None, + &mut vec![], + Some(head_block.number()), + None, + ) .await; debug!("best_available_server: {:#?}", best_available_server); @@ -1744,13 +1749,13 @@ mod tests { )); let _best_available_server_from_none = rpcs - .best_available_rpc(&authorization, None, &[], None, None) + .best_available_rpc(&authorization, None, &mut vec![], None, None) .await; // assert_eq!(best_available_server, best_available_server_from_none); let best_archive_server = rpcs - .best_available_rpc(&authorization, None, &[], Some(&1.into()), None) + .best_available_rpc(&authorization, None, &mut vec![], Some(&1.into()), None) .await; match best_archive_server { From 5aae318620963bbd9eb9ff5f346c2b7d13a86e9c Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Thu, 18 May 2023 15:11:29 -0700 Subject: [PATCH 45/66] logs and comments --- web3_proxy/src/rpcs/one.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index 6095c872..c3d8b34f 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -1223,9 +1223,11 @@ impl Web3Rpc { // we already have an unlocked provider. no need to lock } else { warn!("no provider on {}", self); + // TODO: wait for provider? that will probably slow us down more than we want return Ok(OpenRequestResult::NotReady); } + // check cached rate limits if let Some(hard_limit_until) = self.hard_limit_until.as_ref() { let hard_limit_ready = *hard_limit_until.borrow(); let now = Instant::now(); @@ -1234,7 +1236,7 @@ impl Web3Rpc { } } - // check rate limits + // check shared rate limits if let Some(ratelimiter) = self.hard_limit.as_ref() { // TODO: how should we know if we should set expire or not? match ratelimiter @@ -1247,6 +1249,7 @@ impl Web3Rpc { } RedisRateLimitResult::RetryAt(retry_at, _) => { // rate limit gave us a wait time + // if not a backup server, warn. backups hit rate limits often if !self.backup { let when = retry_at.duration_since(Instant::now()); warn!( @@ -1263,6 +1266,7 @@ impl Web3Rpc { return Ok(OpenRequestResult::RetryAt(retry_at)); } RedisRateLimitResult::RetryNever => { + warn!("how did retry never on {} happen?", self); return Ok(OpenRequestResult::NotReady); } } @@ -1342,7 +1346,7 @@ impl Hash for Web3Rpc { self.ws_url.hash(state); self.automatic_block_limit.hash(state); self.backup.hash(state); - // TODO: including soft_limit might need to change if we change them to be dynamic + // TODO: don't include soft_limit if we change them to be dynamic self.soft_limit.hash(state); self.tier.hash(state); self.created_at.hash(state); From c443f766235eb1b86e169a0990f204ab95b9c557 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Thu, 18 May 2023 22:43:07 -0700 Subject: [PATCH 46/66] refactor tiers again --- Cargo.lock | 122 +++---- web3_proxy/Cargo.toml | 2 +- web3_proxy/src/frontend/users/payment.rs | 6 +- web3_proxy/src/frontend/users/referral.rs | 28 +- web3_proxy/src/referral_code.rs | 10 +- web3_proxy/src/rpcs/blockchain.rs | 10 +- web3_proxy/src/rpcs/consensus.rs | 2 +- web3_proxy/src/rpcs/many.rs | 402 +++++++++++++--------- 8 files changed, 317 insertions(+), 265 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7d73a5b1..95aec00b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -183,7 +183,7 @@ checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -194,7 +194,7 @@ checksum = "b9ccdd8f2a161be9bd5c023df56f1b2a0bd1d83872ae53b71a84a12c9bf6e842" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -328,7 +328,7 @@ dependencies = [ "heck 0.4.1", "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -1172,7 +1172,7 @@ dependencies = [ "proc-macro2", "quote", "scratch", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -1189,7 +1189,7 @@ checksum = "2345488264226bf682893e25de0769f3360aac9957980ec49361b083ddaa5bc5" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -1706,7 +1706,7 @@ dependencies = [ "reqwest", "serde", "serde_json", - "syn 2.0.15", + "syn 2.0.16", "tokio", "toml 0.7.4", "url", @@ -1726,7 +1726,7 @@ dependencies = [ "proc-macro2", "quote", "serde_json", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -1753,7 +1753,7 @@ dependencies = [ "serde", "serde_json", "strum", - "syn 2.0.15", + "syn 2.0.16", "tempfile", "thiserror", "tiny-keccak", @@ -2156,7 +2156,7 @@ checksum = "89ca545a94061b6365f2c7355b4b32bd20df3ff95f02da9329b34ccc3bd6ee72" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -2296,9 +2296,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.3.18" +version = "0.3.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17f8a914c2987b688368b5138aa05321db91f4090cf26118185672ad588bce21" +checksum = "d357c7ae988e7d2182f7d7871d0b963962420b0678b0997ce7de72001aeab782" dependencies = [ "bytes", "fnv", @@ -2862,9 +2862,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.62" +version = "0.3.63" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68c16e1bfd491478ab155fd8b4896b86f9ede344949b641e61501e07c2b8b4d5" +checksum = "2f37a4a5928311ac501dee68b3c7613a1037d0edb30c8e5427bd832d55d1b790" dependencies = [ "wasm-bindgen", ] @@ -3380,7 +3380,7 @@ dependencies = [ "proc-macro-crate 1.3.1", "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -3458,7 +3458,7 @@ checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -3739,7 +3739,7 @@ dependencies = [ "pest_meta", "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -3923,7 +3923,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1ceca8aaf45b5c46ec7ed39fff75f57290368c1846d33d24a122ca81416ab058" dependencies = [ "proc-macro2", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -4005,9 +4005,9 @@ checksum = "dc375e1527247fe1a97d8b7156678dfe7c1af2fc075c9a4db3690ecd2a148068" [[package]] name = "proc-macro2" -version = "1.0.56" +version = "1.0.58" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b63bdb0cd06f1f4dedf69b254734f9b45af66e4a031e42a7480257d9898b435" +checksum = "fa1fb82fc0c281dd9671101b66b771ebbe1eaf967b96ac8740dcba4b70005ca8" dependencies = [ "unicode-ident", ] @@ -4899,9 +4899,9 @@ dependencies = [ [[package]] name = "security-framework" -version = "2.8.2" +version = "2.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a332be01508d814fed64bf28f798a146d73792121129962fdf335bb3c49a4254" +checksum = "ca2855b3715770894e67cbfa3df957790aa0c9edc3bf06efa1a84d77fa0839d1" dependencies = [ "bitflags", "core-foundation", @@ -4912,9 +4912,9 @@ dependencies = [ [[package]] name = "security-framework-sys" -version = "2.8.0" +version = "2.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31c9bb296072e961fcbd8853511dd39c2d8be2deb1e17c6860b1d30732b323b4" +checksum = "f51d0c0d83bec45f16480d0ce0058397a69e48fcdc52d1dc8855fb68acbd31a7" dependencies = [ "core-foundation-sys", "libc", @@ -4943,9 +4943,9 @@ checksum = "cd0b0ec5f1c1ca621c432a25813d8d60c88abe6d3e08a3eb9cf37d97a0fe3d73" [[package]] name = "sentry" -version = "0.31.0" +version = "0.31.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c3d7f8bf7373e75222452fcdd9347d857452a92d0eec738f941bc4656c5b5df" +checksum = "37dd6c0cdca6b1d1ca44cde7fff289f2592a97965afec870faa7b81b9fc87745" dependencies = [ "httpdate", "reqwest", @@ -4963,9 +4963,9 @@ dependencies = [ [[package]] name = "sentry-anyhow" -version = "0.31.0" +version = "0.31.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ef7f47c57a1146d553b4976f20e8bba370195a88858bdf6945a63c529549236" +checksum = "d9c3d7032fff178c77c107c32c6d3337b12847adf67165ccc876c898e7154b00" dependencies = [ "anyhow", "sentry-backtrace", @@ -4974,9 +4974,9 @@ dependencies = [ [[package]] name = "sentry-backtrace" -version = "0.31.0" +version = "0.31.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03b7cdefbdca51f1146f0f24a3cb4ecb6428951f030ff5c720cfb5c60bd174c0" +checksum = "c029fe8317cdd75cb2b52c600bab4e2ef64c552198e669ba874340447f330962" dependencies = [ "backtrace", "once_cell", @@ -4986,9 +4986,9 @@ dependencies = [ [[package]] name = "sentry-contexts" -version = "0.31.0" +version = "0.31.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6af4cb29066e0e8df0cc3111211eb93543ccb09e1ccbe71de6d88b4bb459a2b1" +checksum = "bc575098d73c8b942b589ab453b06e4c43527556dd8f95532220d1b54d7c6b4b" dependencies = [ "hostname", "libc", @@ -5000,9 +5000,9 @@ dependencies = [ [[package]] name = "sentry-core" -version = "0.31.0" +version = "0.31.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e781b55761e47a60d1ff326ae8059de22b0e6b0cee68eab1c5912e4fb199a76" +checksum = "20216140001bbf05895f013abd0dae4df58faee24e016d54cbf107f070bac56b" dependencies = [ "once_cell", "rand", @@ -5013,9 +5013,9 @@ dependencies = [ [[package]] name = "sentry-log" -version = "0.31.0" +version = "0.31.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ee978a28d4cb15c89618dc035ab56ea18fe82d5a248ddcd771d153876ccbdf56" +checksum = "a43934e48e9c8e2c7d0dcb9c6cbcfcbe3ee109a14fc0c821e8944acd4faa2c25" dependencies = [ "log", "sentry-core", @@ -5023,9 +5023,9 @@ dependencies = [ [[package]] name = "sentry-panic" -version = "0.31.0" +version = "0.31.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e0b877981990d9e84ae6916df61993d188fdf76afb59521f0aeaf9b8e6d26d0" +checksum = "4e45cd0a113fc06d6edba01732010518816cdc8ce3bccc70f5e41570046bf046" dependencies = [ "sentry-backtrace", "sentry-core", @@ -5033,9 +5033,9 @@ dependencies = [ [[package]] name = "sentry-types" -version = "0.31.0" +version = "0.31.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d642a04657cc77d8de52ae7c6d93a15cb02284eb219344a89c1e2b26bbaf578c" +checksum = "d7f6959d8cb3a77be27e588eef6ce9a2a469651a556d9de662e4d07e5ace4232" dependencies = [ "debugid", "getrandom", @@ -5065,7 +5065,7 @@ checksum = "8c805777e3930c8883389c602315a24224bcc738b63905ef87cd1420353ea93e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -5598,9 +5598,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.15" +version = "2.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a34fcf3e8b60f57e6a14301a2e916d323af98b0ea63c599441eec8558660c822" +checksum = "a6f671d4b5ffdb8eadec19c0ae67fe2639df8684bd7bc4b83d986b8db549cf01" dependencies = [ "proc-macro2", "quote", @@ -5685,7 +5685,7 @@ checksum = "f9456a42c5b0d803c8cd86e73dd7cc9edd429499f37a3550d286d5e86720569f" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -5847,7 +5847,7 @@ checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -6079,7 +6079,7 @@ checksum = "0f57e3ca2a01450b1a921183a9c9cbfda207fd822cef4ccb00a65402cbba7a74" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -6413,9 +6413,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.85" +version = "0.2.86" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b6cb788c4e39112fbe1822277ef6fb3c55cd86b95cb3d3c4c1c9597e4ac74b4" +checksum = "5bba0e8cb82ba49ff4e229459ff22a191bbe9a1cb3a341610c9c33efc27ddf73" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -6423,24 +6423,24 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.85" +version = "0.2.86" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35e522ed4105a9d626d885b35d62501b30d9666283a5c8be12c14a8bdafe7822" +checksum = "19b04bc93f9d6bdee709f6bd2118f57dd6679cf1176a1af464fca3ab0d66d8fb" dependencies = [ "bumpalo", "log", "once_cell", "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-futures" -version = "0.4.35" +version = "0.4.36" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "083abe15c5d88556b77bdf7aef403625be9e327ad37c62c4e4129af740168163" +checksum = "2d1985d03709c53167ce907ff394f5316aa22cb4e12761295c5dc57dacb6297e" dependencies = [ "cfg-if", "js-sys", @@ -6450,9 +6450,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.85" +version = "0.2.86" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "358a79a0cb89d21db8120cbfb91392335913e4890665b1a7981d9e956903b434" +checksum = "14d6b024f1a526bb0234f52840389927257beb670610081360e5a03c5df9c258" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -6460,22 +6460,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.85" +version = "0.2.86" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4783ce29f09b9d93134d41297aded3a712b7b979e9c6f28c32cb88c973a94869" +checksum = "e128beba882dd1eb6200e1dc92ae6c5dbaa4311aa7bb211ca035779e5efc39f8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.85" +version = "0.2.86" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a901d592cafaa4d711bc324edfaff879ac700b19c3dfd60058d2b445be2691eb" +checksum = "ed9d5b4305409d1fc9482fee2d7f9bcbf24b3972bf59817ef757e23982242a93" [[package]] name = "wasm-streams" @@ -6492,9 +6492,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.62" +version = "0.3.63" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16b5f940c7edfdc6d12126d98c9ef4d1b3d470011c47c76a6581df47ad9ba721" +checksum = "3bdd9ef4e984da1187bf8110c5cf5b845fbc87a23602cdf912386a76fcd3a7c2" dependencies = [ "js-sys", "wasm-bindgen", diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index a91eddb6..c51e9c8e 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -74,7 +74,7 @@ rdkafka = { version = "0.31.0" } regex = "1.8.1" reqwest = { version = "0.11.18", default-features = false, features = ["json", "tokio-rustls"] } rmp-serde = "1.1.1" -sentry = { version = "0.31.0", default-features = false, features = ["backtrace", "contexts", "panic", "anyhow", "reqwest", "rustls", "log", "sentry-log"] } +sentry = { version = "0.31.1", default-features = false, features = ["backtrace", "contexts", "panic", "anyhow", "reqwest", "rustls", "log", "sentry-log"] } serde = { version = "1.0.163", features = [] } serde_json = { version = "1.0.96", default-features = false, features = ["alloc", "raw_value"] } serde_prometheus = "0.2.2" diff --git a/web3_proxy/src/frontend/users/payment.rs b/web3_proxy/src/frontend/users/payment.rs index fb2f0d91..975fb3da 100644 --- a/web3_proxy/src/frontend/users/payment.rs +++ b/web3_proxy/src/frontend/users/payment.rs @@ -153,7 +153,7 @@ pub async fn user_balance_post( // Just make an rpc request, idk if i need to call this super extensive code let transaction_receipt: TransactionReceipt = match app .balanced_rpcs - .best_available_rpc(&authorization, None, &mut vec![], None, None) + .wait_for_best_rpc(&authorization, None, &mut vec![], None, None) .await { Ok(OpenRequestResult::Handle(handle)) => { @@ -188,7 +188,7 @@ pub async fn user_balance_post( debug!("Transaction receipt is: {:?}", transaction_receipt); let accepted_token: Address = match app .balanced_rpcs - .best_available_rpc(&authorization, None, &mut vec![], None, None) + .wait_for_best_rpc(&authorization, None, &mut vec![], None, None) .await { Ok(OpenRequestResult::Handle(handle)) => { @@ -243,7 +243,7 @@ pub async fn user_balance_post( debug!("Accepted token is: {:?}", accepted_token); let decimals: u32 = match app .balanced_rpcs - .best_available_rpc(&authorization, None, &mut vec![], None, None) + .wait_for_best_rpc(&authorization, None, &mut vec![], None, None) .await { Ok(OpenRequestResult::Handle(handle)) => { diff --git a/web3_proxy/src/frontend/users/referral.rs b/web3_proxy/src/frontend/users/referral.rs index c4f613ac..391fccc5 100644 --- a/web3_proxy/src/frontend/users/referral.rs +++ b/web3_proxy/src/frontend/users/referral.rs @@ -1,6 +1,6 @@ //! Handle registration, logins, and managing account data. use crate::app::Web3ProxyApp; -use crate::frontend::errors::{Web3ProxyError, Web3ProxyResponse}; +use crate::frontend::errors::Web3ProxyResponse; use crate::referral_code::ReferralCode; use anyhow::Context; use axum::{ @@ -10,10 +10,9 @@ use axum::{ Extension, Json, TypedHeader, }; use axum_macros::debug_handler; -use entities::{referrer, user_tier}; +use entities::referrer; use hashbrown::HashMap; use http::StatusCode; -use log::warn; use migration::sea_orm; use migration::sea_orm::ActiveModelTrait; use migration::sea_orm::ColumnTrait; @@ -38,20 +37,7 @@ pub async fn user_referral_link_get( .db_replica() .context("getting replica db for user's revert logs")?; - // Second, check if the user is a premium user - let user_tier = user_tier::Entity::find() - .filter(user_tier::Column::Id.eq(user.user_tier_id)) - .one(db_replica.conn()) - .await? - .ok_or(Web3ProxyError::UnknownKey)?; - - warn!("User tier is: {:?}", user_tier); - // TODO: This shouldn't be hardcoded. Also, it should be an enum, not sth like this ... - if user_tier.id != 6 { - return Err(Web3ProxyError::PaymentRequired); - } - - // Then get the referral token + // Then get the referral token. If one doesn't exist, create one let user_referrer = referrer::Entity::find() .filter(referrer::Column::UserId.eq(user.id)) .one(db_replica.conn()) @@ -60,18 +46,18 @@ pub async fn user_referral_link_get( let (referral_code, status_code) = match user_referrer { Some(x) => (x.referral_code, StatusCode::OK), None => { - // Connect to the database for mutable write + // Connect to the database for writes let db_conn = app.db_conn().context("getting db_conn")?; - let referral_code = ReferralCode::default().0; - // Log that this guy was referred by another guy - // Do not automatically create a new + let referral_code = ReferralCode::default().to_string(); + let referrer_entry = referrer::ActiveModel { user_id: sea_orm::ActiveValue::Set(user.id), referral_code: sea_orm::ActiveValue::Set(referral_code.clone()), ..Default::default() }; referrer_entry.save(&db_conn).await?; + (referral_code, StatusCode::CREATED) } }; diff --git a/web3_proxy/src/referral_code.rs b/web3_proxy/src/referral_code.rs index d5343e84..49b8b08c 100644 --- a/web3_proxy/src/referral_code.rs +++ b/web3_proxy/src/referral_code.rs @@ -1,7 +1,9 @@ +use std::fmt::Display; + use anyhow::{self, Result}; use ulid::Ulid; -pub struct ReferralCode(pub String); +pub struct ReferralCode(String); impl Default for ReferralCode { fn default() -> Self { @@ -10,6 +12,12 @@ impl Default for ReferralCode { } } +impl Display for ReferralCode { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.write_str(&self.0) + } +} + impl TryFrom for ReferralCode { type Error = anyhow::Error; diff --git a/web3_proxy/src/rpcs/blockchain.rs b/web3_proxy/src/rpcs/blockchain.rs index ff2c08c4..9c181e0d 100644 --- a/web3_proxy/src/rpcs/blockchain.rs +++ b/web3_proxy/src/rpcs/blockchain.rs @@ -245,13 +245,7 @@ impl Web3Rpcs { // TODO: request_metadata? maybe we should put it in the authorization? // TODO: think more about this wait_for_sync let response = self - .try_send_best_consensus_head_connection( - authorization, - &request, - None, - None, - None, - ) + .try_send_best_connection(authorization, &request, None, None, None) .await?; let value = match response { @@ -346,7 +340,7 @@ impl Web3Rpcs { let request: JsonRpcRequest = serde_json::from_value(request)?; let response = self - .try_send_best_consensus_head_connection(authorization, &request, None, Some(num), None) + .try_send_best_connection(authorization, &request, None, Some(num), None) .await?; let value = match response { diff --git a/web3_proxy/src/rpcs/consensus.rs b/web3_proxy/src/rpcs/consensus.rs index 2ca28519..b11fb794 100644 --- a/web3_proxy/src/rpcs/consensus.rs +++ b/web3_proxy/src/rpcs/consensus.rs @@ -190,7 +190,7 @@ impl ConsensusWeb3Rpcs { } // TODO: take method as a param, too. mark nodes with supported methods (maybe do it optimistically? on) - fn rpc_will_work_eventually( + pub fn rpc_will_work_eventually( &self, rpc: &Arc, needed_block_num: Option<&U64>, diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index 5921f895..5f281bee 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -10,7 +10,6 @@ use crate::frontend::errors::{Web3ProxyError, Web3ProxyResult}; use crate::frontend::rpc_proxy_ws::ProxyMode; use crate::jsonrpc::{JsonRpcErrorData, JsonRpcRequest}; use crate::response_cache::JsonRpcResponseData; -use crate::rpcs::consensus::{RankedRpcMap, RpcRanking}; use crate::rpcs::transactions::TxStatus; use anyhow::Context; use arc_swap::ArcSwap; @@ -32,7 +31,6 @@ use serde_json::json; use serde_json::value::RawValue; use std::borrow::Cow; use std::cmp::{min_by_key, Reverse}; -use std::collections::BTreeMap; use std::fmt::{self, Display}; use std::sync::atomic::Ordering; use std::sync::Arc; @@ -491,141 +489,241 @@ impl Web3Rpcs { unimplemented!("this shouldn't be possible") } - pub async fn best_available_rpc( + async fn _best_available_rpc( + &self, + authorization: &Arc, + potential_rpcs: &[Arc], + skip: &mut Vec>, + ) -> OpenRequestResult { + let mut earliest_retry_at = None; + + for (rpc_a, rpc_b) in potential_rpcs.iter().circular_tuple_windows() { + trace!("{} vs {}", rpc_a, rpc_b); + // TODO: cached key to save a read lock + // TODO: ties to the server with the smallest block_data_limit + let faster_rpc = min_by_key(rpc_a, rpc_b, |x| x.peak_ewma()); + trace!("winner: {}", faster_rpc); + + // add to the skip list in case this one fails + skip.push(Arc::clone(faster_rpc)); + + // just because it has lower latency doesn't mean we are sure to get a connection. there might be rate limits + match faster_rpc.try_request_handle(authorization, None).await { + Ok(OpenRequestResult::Handle(handle)) => { + trace!("opened handle: {}", faster_rpc); + return OpenRequestResult::Handle(handle); + } + Ok(OpenRequestResult::RetryAt(retry_at)) => { + trace!( + "retry on {} @ {}", + faster_rpc, + retry_at.duration_since(Instant::now()).as_secs_f32() + ); + + if earliest_retry_at.is_none() { + earliest_retry_at = Some(retry_at); + } else { + earliest_retry_at = earliest_retry_at.min(Some(retry_at)); + } + } + Ok(OpenRequestResult::NotReady) => { + // TODO: log a warning? emit a stat? + trace!("best_rpc not ready: {}", faster_rpc); + } + Err(err) => { + trace!("No request handle for {}. err={:?}", faster_rpc, err) + } + } + } + + if let Some(retry_at) = earliest_retry_at { + OpenRequestResult::RetryAt(retry_at) + } else { + OpenRequestResult::NotReady + } + } + + pub async fn wait_for_best_rpc( &self, authorization: &Arc, request_metadata: Option<&Arc>, - skip: &mut Vec>, + skip_rpcs: &mut Vec>, min_block_needed: Option<&U64>, max_block_needed: Option<&U64>, ) -> Web3ProxyResult { - // TODO: use tracing and add this so logs are easy - let request_ulid = request_metadata.map(|x| &x.request_ulid); + let mut earliest_retry_at: Option = None; - let mut usable_rpcs_by_tier_and_head_number = { - let mut m: RankedRpcMap = BTreeMap::new(); + if self.watch_consensus_head_sender.is_none() { + trace!("this Web3Rpcs is not tracking head blocks. pick any server"); - if let Some(consensus_rpcs) = self.watch_consensus_rpcs_sender.borrow().as_ref() { - // first place is the blocks that are synced close to head. if those don't work. try all the rpcs. if those don't work, keep trying for a few seconds + let by_name = self.by_name.load(); - let head_block = &consensus_rpcs.head_block; + let mut potential_rpcs: Vec<_> = by_name + .values() + .filter(|rpc| !skip_rpcs.contains(rpc)) + .filter(|rpc| { + min_block_needed + .map(|x| rpc.has_block_data(x)) + .unwrap_or(true) + }) + .filter(|rpc| { + max_block_needed + .map(|x| rpc.has_block_data(x)) + .unwrap_or(true) + }) + .cloned() + .collect(); - let head_block_num = *head_block.number(); + potential_rpcs.shuffle(&mut thread_fast_rng::thread_fast_rng()); - let best_key = RpcRanking::new( - consensus_rpcs.tier, - consensus_rpcs.backups_needed, - Some(head_block_num), - ); - - // todo: for now, build the map m here. once that works, do as much of it as possible while building ConsensusWeb3Rpcs - for x in consensus_rpcs.head_rpcs.iter().filter(|rpc| { - consensus_rpcs.rpc_will_work_now(skip, min_block_needed, max_block_needed, rpc) - }) { - m.entry(best_key).or_insert_with(Vec::new).push(x.clone()); - } - - let tier_offset = consensus_rpcs.tier + 1; - - for (k, v) in consensus_rpcs.other_rpcs.iter() { - let v: Vec<_> = v - .iter() - .filter(|rpc| { - consensus_rpcs.rpc_will_work_now( - skip, - min_block_needed, - max_block_needed, - rpc, - ) - }) - .cloned() - .collect(); - - let offset_ranking = k.add_offset(tier_offset); - - m.entry(offset_ranking).or_insert_with(Vec::new).extend(v); - } - } else if self.watch_consensus_head_sender.is_none() { - trace!("this Web3Rpcs is not tracking head blocks. pick any server"); - - for x in self.by_name.load().values() { - if skip.contains(x) { - trace!("{:?} - already skipped. {}", request_ulid, x); - continue; + match self + ._best_available_rpc(authorization, &potential_rpcs, skip_rpcs) + .await + { + OpenRequestResult::Handle(x) => return Ok(OpenRequestResult::Handle(x)), + OpenRequestResult::NotReady => {} + OpenRequestResult::RetryAt(retry_at) => { + if earliest_retry_at.is_none() { + earliest_retry_at = Some(retry_at); + } else { + earliest_retry_at = earliest_retry_at.min(Some(retry_at)); } - - let key = RpcRanking::default_with_backup(x.backup); - - m.entry(key).or_insert_with(Vec::new).push(x.clone()); } } + } else { + let start = Instant::now(); - m - }; + // TODO: get from config or argument + let max_wait = Duration::from_secs(10); - trace!( - "{:?} - usable_rpcs_by_tier_and_head_number: {:#?}", - request_ulid, - usable_rpcs_by_tier_and_head_number - ); + let mut watch_consensus_rpcs = self.watch_consensus_rpcs_sender.subscribe(); - let mut earliest_retry_at = None; + let mut potential_rpcs = Vec::with_capacity(self.by_name.load().len()); - for usable_rpcs in usable_rpcs_by_tier_and_head_number.values_mut() { - // sort the tier randomly - if usable_rpcs.len() == 1 { - // TODO: include an rpc from the next tier? - } else { - // we can't get the rng outside of this loop because it is not Send - // this function should be pretty fast anyway, so it shouldn't matter too much - let mut rng = thread_fast_rng::thread_fast_rng(); - usable_rpcs.shuffle(&mut rng); - }; + while start.elapsed() < max_wait { + let consensus_rpcs = watch_consensus_rpcs.borrow_and_update().clone(); - // now that the rpcs are shuffled, try to get an active request handle for one of them - // pick the first two and try the one with the lower rpc.latency.ewma - // TODO: chunks or tuple windows? - for (rpc_a, rpc_b) in usable_rpcs.iter().circular_tuple_windows() { - trace!("{:?} - {} vs {}", request_ulid, rpc_a, rpc_b); - // TODO: cached key to save a read lock - // TODO: ties to the server with the smallest block_data_limit - let best_rpc = min_by_key(rpc_a, rpc_b, |x| x.peak_ewma()); - trace!("{:?} - winner: {}", request_ulid, best_rpc); + potential_rpcs.clear(); - skip.push(best_rpc.clone()); + // first check everything that is synced + // even though we might be querying an old block that an unsynced server can handle, + // it is best to not send queries to a syncing server. that slows down sync and can bloat erigon's disk usage. + if let Some(consensus_rpcs) = consensus_rpcs { + potential_rpcs.extend( + consensus_rpcs + .head_rpcs + .iter() + .filter(|rpc| { + consensus_rpcs.rpc_will_work_now( + skip_rpcs, + min_block_needed, + max_block_needed, + rpc, + ) + }) + .cloned(), + ); - // just because it has lower latency doesn't mean we are sure to get a connection - match best_rpc.try_request_handle(authorization, None).await { - Ok(OpenRequestResult::Handle(handle)) => { - trace!("{:?} - opened handle: {}", request_ulid, best_rpc); - return Ok(OpenRequestResult::Handle(handle)); + potential_rpcs.shuffle(&mut thread_fast_rng::thread_fast_rng()); + + if potential_rpcs.len() >= self.min_head_rpcs { + // we have enough potential rpcs. try to load balance + + match self + ._best_available_rpc(authorization, &potential_rpcs, skip_rpcs) + .await + { + OpenRequestResult::Handle(x) => { + return Ok(OpenRequestResult::Handle(x)) + } + OpenRequestResult::NotReady => {} + OpenRequestResult::RetryAt(retry_at) => { + if earliest_retry_at.is_none() { + earliest_retry_at = Some(retry_at); + } else { + earliest_retry_at = earliest_retry_at.min(Some(retry_at)); + } + } + } + + // these rpcs were tried. don't try them again + potential_rpcs.clear(); } - Ok(OpenRequestResult::RetryAt(retry_at)) => { - trace!( - "{:?} - retry on {} @ {}", - request_ulid, - best_rpc, - retry_at.duration_since(Instant::now()).as_secs_f32() - ); - if earliest_retry_at.is_none() { - earliest_retry_at = Some(retry_at); - } else { - earliest_retry_at = earliest_retry_at.min(Some(retry_at)); + for next_rpcs in consensus_rpcs.other_rpcs.values() { + // we have to collect in order to shuffle + let mut more_rpcs: Vec<_> = next_rpcs + .iter() + .filter(|rpc| { + consensus_rpcs.rpc_will_work_now( + skip_rpcs, + min_block_needed, + max_block_needed, + rpc, + ) + }) + .cloned() + .collect(); + + // shuffle only the new entries. that way the highest tier still gets preference + more_rpcs.shuffle(&mut thread_fast_rng::thread_fast_rng()); + + potential_rpcs.extend(more_rpcs.into_iter()); + + if potential_rpcs.len() >= self.min_head_rpcs { + // we have enough potential rpcs. try to load balance + match self + ._best_available_rpc(authorization, &potential_rpcs, skip_rpcs) + .await + { + OpenRequestResult::Handle(x) => { + return Ok(OpenRequestResult::Handle(x)) + } + OpenRequestResult::NotReady => {} + OpenRequestResult::RetryAt(retry_at) => { + if earliest_retry_at.is_none() { + earliest_retry_at = Some(retry_at); + } else { + earliest_retry_at = earliest_retry_at.min(Some(retry_at)); + } + } + } + + // these rpcs were tried. don't try them again + potential_rpcs.clear(); } } - Ok(OpenRequestResult::NotReady) => { - // TODO: log a warning? emit a stat? - trace!("{:?} - best_rpc not ready: {}", request_ulid, best_rpc); + + if !potential_rpcs.is_empty() { + // even after scanning all the tiers, there are not enough rpcs that can serve this request. try anyways + match self + ._best_available_rpc(authorization, &potential_rpcs, skip_rpcs) + .await + { + OpenRequestResult::Handle(x) => { + return Ok(OpenRequestResult::Handle(x)) + } + OpenRequestResult::NotReady => {} + OpenRequestResult::RetryAt(retry_at) => { + if earliest_retry_at.is_none() { + earliest_retry_at = Some(retry_at); + } else { + earliest_retry_at = earliest_retry_at.min(Some(retry_at)); + } + } + } } - Err(err) => { - trace!( - "{:?} - No request handle for {}. err={:?}", - request_ulid, - best_rpc, - err - ) + + let waiting_for = min_block_needed.max(max_block_needed); + + match consensus_rpcs.should_wait_for_block(waiting_for, skip_rpcs) { + ShouldWaitForBlock::NeverReady => break, + ShouldWaitForBlock::Ready => continue, + ShouldWaitForBlock::Wait { .. } => {} } + + // TODO: select on consensus_rpcs changing and on earliest_retry_at + watch_consensus_rpcs.changed().await?; } } } @@ -634,41 +732,20 @@ impl Web3Rpcs { request_metadata.no_servers.fetch_add(1, Ordering::AcqRel); } - match earliest_retry_at { - None => { - // none of the servers gave us a time to retry at - debug!( - "{:?} - no servers in {} gave a retry time! Skipped {:?}. {:#?}", - request_ulid, self, skip, usable_rpcs_by_tier_and_head_number - ); + if let Some(retry_at) = earliest_retry_at { + // TODO: log the server that retry_at came from + warn!( + "no servers in {} ready! Skipped {:?}. Retry in {:?}s", + self, + skip_rpcs, + retry_at.duration_since(Instant::now()).as_secs_f32() + ); - // TODO: bring this back? need to think about how to do this with `allow_backups` - // we could return an error here, but maybe waiting a second will fix the problem - // TODO: configurable max wait? the whole max request time, or just some portion? - // let handle = sorted_rpcs - // .get(0) - // .expect("at least 1 is available") - // .wait_for_request_handle(authorization, Duration::from_secs(3), false) - // .await?; - // Ok(OpenRequestResult::Handle(handle)) + Ok(OpenRequestResult::RetryAt(retry_at)) + } else { + warn!("no servers in {} ready! Skipped {:?}", self, skip_rpcs); - Ok(OpenRequestResult::NotReady) - } - Some(earliest_retry_at) => { - // TODO: log the server that retry_at came from - // TODO: `self` doesn't log well. get a pretty name for this group of servers - warn!( - "{:?} - no servers in {} ready! Skipped {:?}. Retry in {:?}s", - request_ulid, - self, - skip, - earliest_retry_at - .duration_since(Instant::now()) - .as_secs_f32() - ); - - Ok(OpenRequestResult::RetryAt(earliest_retry_at)) - } + Ok(OpenRequestResult::NotReady) } } @@ -784,7 +861,7 @@ impl Web3Rpcs { /// be sure there is a timeout on this or it might loop forever /// TODO: think more about wait_for_sync - pub async fn try_send_best_consensus_head_connection( + pub async fn try_send_best_connection( &self, authorization: &Arc, request: &JsonRpcRequest, @@ -802,9 +879,10 @@ impl Web3Rpcs { // TODO: get from config let max_wait = Duration::from_secs(10); + // TODO: the loop here feels somewhat redundant with the loop in best_available_rpc while start.elapsed() < max_wait { match self - .best_available_rpc( + .wait_for_best_rpc( authorization, request_metadata, &mut skip_rpcs, @@ -963,21 +1041,7 @@ impl Web3Rpcs { } } OpenRequestResult::NotReady => { - if let Some(request_metadata) = request_metadata { - request_metadata.no_servers.fetch_add(1, Ordering::AcqRel); - } - - let waiting_for = min_block_needed.max(max_block_needed); - - if let Some(consensus_rpcs) = watch_consensus_rpcs.borrow_and_update().as_ref() - { - match consensus_rpcs.should_wait_for_block(waiting_for, &skip_rpcs) { - ShouldWaitForBlock::NeverReady => break, - ShouldWaitForBlock::Ready => continue, - ShouldWaitForBlock::Wait { .. } => {} - } - } - watch_consensus_rpcs.changed().await?; + break; } } } @@ -1152,7 +1216,7 @@ impl Web3Rpcs { ) -> Web3ProxyResult { match authorization.checks.proxy_mode { ProxyMode::Debug | ProxyMode::Best => { - self.try_send_best_consensus_head_connection( + self.try_send_best_connection( authorization, request, request_metadata, @@ -1489,7 +1553,7 @@ mod tests { // best_synced_backend_connection which servers to be synced with the head block should not find any nodes let x = rpcs - .best_available_rpc( + .wait_for_best_rpc( &authorization, None, &mut vec![], @@ -1586,28 +1650,28 @@ mod tests { // TODO: make sure the handle is for the expected rpc assert!(matches!( - rpcs.best_available_rpc(&authorization, None, &mut vec![], None, None) + rpcs.wait_for_best_rpc(&authorization, None, &mut vec![], None, None) .await, Ok(OpenRequestResult::Handle(_)) )); // TODO: make sure the handle is for the expected rpc assert!(matches!( - rpcs.best_available_rpc(&authorization, None, &mut vec![], Some(&0.into()), None) + rpcs.wait_for_best_rpc(&authorization, None, &mut vec![], Some(&0.into()), None) .await, Ok(OpenRequestResult::Handle(_)) )); // TODO: make sure the handle is for the expected rpc assert!(matches!( - rpcs.best_available_rpc(&authorization, None, &mut vec![], Some(&1.into()), None) + rpcs.wait_for_best_rpc(&authorization, None, &mut vec![], Some(&1.into()), None) .await, Ok(OpenRequestResult::Handle(_)) )); // future block should not get a handle let future_rpc = rpcs - .best_available_rpc(&authorization, None, &mut vec![], Some(&2.into()), None) + .wait_for_best_rpc(&authorization, None, &mut vec![], Some(&2.into()), None) .await; assert!(matches!(future_rpc, Ok(OpenRequestResult::NotReady))); } @@ -1732,7 +1796,7 @@ mod tests { // best_synced_backend_connection requires servers to be synced with the head block // TODO: test with and without passing the head_block.number? let best_available_server = rpcs - .best_available_rpc( + .wait_for_best_rpc( &authorization, None, &mut vec![], @@ -1749,13 +1813,13 @@ mod tests { )); let _best_available_server_from_none = rpcs - .best_available_rpc(&authorization, None, &mut vec![], None, None) + .wait_for_best_rpc(&authorization, None, &mut vec![], None, None) .await; // assert_eq!(best_available_server, best_available_server_from_none); let best_archive_server = rpcs - .best_available_rpc(&authorization, None, &mut vec![], Some(&1.into()), None) + .wait_for_best_rpc(&authorization, None, &mut vec![], Some(&1.into()), None) .await; match best_archive_server { From 795548580a94b9c96a6b1ea5493ee87d56070c13 Mon Sep 17 00:00:00 2001 From: yenicelik Date: Sun, 21 May 2023 16:40:42 -0400 Subject: [PATCH 47/66] now should also group by archive_needed (turns out this was wanted) --- scripts/generate-requests-and-stats.sh | 2 +- scripts/manual-tests/42-simple-balance.sh | 14 +++++++------- web3_proxy/src/stats/influxdb_queries.rs | 23 ++++------------------- 3 files changed, 12 insertions(+), 27 deletions(-) diff --git a/scripts/generate-requests-and-stats.sh b/scripts/generate-requests-and-stats.sh index ecae4466..a05e55e7 100644 --- a/scripts/generate-requests-and-stats.sh +++ b/scripts/generate-requests-and-stats.sh @@ -5,4 +5,4 @@ # https://github.com/INFURA/versus # ./ethspam | ./versus --stop-after 100 "http://localhost:8544/" # Pipe into the endpoint ..., add a bearer token and all that -./ethspam http://127.0.0.1:8544 | ./versus --stop-after 100 http://localhost:8544 +./ethspam http://127.0.0.1:8544/rpc/01H0ZZJDNNEW49FRFS4D9SPR8B | ./versus --concurrency=4 --stop-after 100 http://localhost:8544/rpc/01H0ZZJDNNEW49FRFS4D9SPR8B diff --git a/scripts/manual-tests/42-simple-balance.sh b/scripts/manual-tests/42-simple-balance.sh index ce6e32da..724d4809 100644 --- a/scripts/manual-tests/42-simple-balance.sh +++ b/scripts/manual-tests/42-simple-balance.sh @@ -24,14 +24,14 @@ curl -X POST http://127.0.0.1:8544/user/login \ -H 'Content-Type: application/json' \ -d '{ "address": "0xeb3e928a2e54be013ef8241d4c9eaf4dfae94d5a", - "msg": "0x6c6c616d616e6f6465732e636f6d2077616e747320796f7520746f207369676e20696e207769746820796f757220457468657265756d206163636f756e743a0a3078654233453932384132453534424530313345463832343164344339456146344466414539344435610a0af09fa699f09fa699f09fa699f09fa699f09fa6990a0a5552493a2068747470733a2f2f6c6c616d616e6f6465732e636f6d2f0a56657273696f6e3a20310a436861696e2049443a20310a4e6f6e63653a203031475a4b384b4847305259474737514e5132475037464444470a4973737565642041743a20323032332d30352d30345431313a33333a32312e3533363734355a0a45787069726174696f6e2054696d653a20323032332d30352d30345431313a35333a32312e3533363734355a", - "sig": "cebd9effff15f4517e53522dbe91798d59dc0df0299faaec25d3f6443fa121f847e4311d5ca7386e75b87d6d45df92b8ced58c822117519c666ab1a6b2fc7bd21b", + "msg": "0x6c6c616d616e6f6465732e636f6d2077616e747320796f7520746f207369676e20696e207769746820796f757220457468657265756d206163636f756e743a0a3078654233453932384132453534424530313345463832343164344339456146344466414539344435610a0af09fa699f09fa699f09fa699f09fa699f09fa6990a0a5552493a2068747470733a2f2f6c6c616d616e6f6465732e636f6d2f0a56657273696f6e3a20310a436861696e2049443a20310a4e6f6e63653a20303148305a5a48434356324b32324738544850535758485131480a4973737565642041743a20323032332d30352d32315432303a32303a34332e3033353539315a0a45787069726174696f6e2054696d653a20323032332d30352d32315432303a34303a34332e3033353539315a", + "sig": "7591251840bf75d2ab7c895bc566a49d2f4c3ad6bb14d7256258a59e52055fc94c11f8f3836f5311b52fc18aca40867cd85802636645e1d757494800631cad381c", "version": "3", "signer": "MEW" }' -# bearer token is: 01GZK8MHHGQWK4VPGF97HS91MB -# scret key is: 01GZK65YNV0P0WN2SCXYTW3R9S +# bearer token is: 01H0ZZJDQ2F02MAXZR5K1X5NCP +# scret key is: 01H0ZZJDNNEW49FRFS4D9SPR8B # 01GZH2PS89EJJY6V8JFCVTQ4BX # 01GZH2PS7CTHA3TAZ4HXCTX6KQ @@ -42,7 +42,7 @@ curl -X POST http://127.0.0.1:8544/user/login \ # Check the balance of the user # Balance seems to be returning properly (0, in this test case) curl \ --H "Authorization: Bearer 01GZK8MHHGQWK4VPGF97HS91MB" \ +-H "Authorization: Bearer 01H0ZZJDQ2F02MAXZR5K1X5NCP" \ -X GET "127.0.0.1:8544/user/balance" @@ -73,10 +73,10 @@ curl \ ## Check if calling an RPC endpoint logs the stats ## This one does already even it seems -for i in {1..100} +for i in {1..300} do curl \ - -X POST "127.0.0.1:8544/rpc/01GZK65YNV0P0WN2SCXYTW3R9S" \ + -X POST "127.0.0.1:8544/rpc/01H0ZZJDNNEW49FRFS4D9SPR8B" \ -H "Content-Type: application/json" \ --data '{"method":"eth_blockNumber","params":[],"id":1,"jsonrpc":"2.0"}' done diff --git a/web3_proxy/src/stats/influxdb_queries.rs b/web3_proxy/src/stats/influxdb_queries.rs index 368df1f1..0d75881c 100644 --- a/web3_proxy/src/stats/influxdb_queries.rs +++ b/web3_proxy/src/stats/influxdb_queries.rs @@ -73,6 +73,7 @@ pub async fn query_user_stats<'a>( let mut join_candidates: Vec = vec![ "_time".to_string(), "_measurement".to_string(), + "archive_needed".to_string(), "chain_id".to_string(), ]; @@ -184,35 +185,19 @@ pub async fn query_user_stats<'a>( {filter_chain_id} {drop_method} - // cumsum = base base |> aggregateWindow(every: {query_window_seconds}s, fn: sum, createEmpty: false) |> pivot(rowKey: ["_time"], columnKey: ["_field"], valueColumn: "_value") |> drop(columns: ["balance"]) - |> map(fn: (r) => ({{ r with "archive_needed": if r.archive_needed == "true" then r.frontend_requests else 0}})) |> map(fn: (r) => ({{ r with "error_response": if r.error_response == "true" then r.frontend_requests else 0}})) - |> group(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"]) + |> group(columns: ["_time", "_measurement", "archive_needed", "chain_id", "method", "rpc_secret_key_id"]) |> sort(columns: ["frontend_requests"]) |> map(fn:(r) => ({{ r with "sum_credits_used": float(v: r["sum_credits_used"]) }})) - |> cumulativeSum(columns: ["archive_needed", "error_response", "backend_requests", "cache_hits", "cache_misses", "frontend_requests", "sum_credits_used", "sum_request_bytes", "sum_response_bytes", "sum_response_millis"]) + |> cumulativeSum(columns: ["error_response", "backend_requests", "cache_hits", "cache_misses", "frontend_requests", "sum_credits_used", "sum_request_bytes", "sum_response_bytes", "sum_response_millis"]) |> sort(columns: ["frontend_requests"], desc: true) |> limit(n: 1) |> group() - |> sort(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"], desc: true) - - // balance = base - // |> toFloat() - // |> aggregateWindow(every: {query_window_seconds}s, fn: mean, createEmpty: false) - // |> pivot(rowKey: ["_time"], columnKey: ["_field"], valueColumn: "_value") - // |> group(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"]) - // |> mean(column: "balance") - // |> group() - // |> sort(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"], desc: true) - - // join( - // tables: {{cumsum, balance}}, - // on: {join_candidates} - // ) + |> sort(columns: ["_time", "_measurement", "archive_needed", "chain_id", "method", "rpc_secret_key_id"], desc: true) "#); info!("Raw query to db is: {:?}", query); From 46afe8b1e4f61292cbee48136c91c9a903e3ee46 Mon Sep 17 00:00:00 2001 From: yenicelik Date: Sun, 21 May 2023 17:13:15 -0400 Subject: [PATCH 48/66] influxdb query also groups by archive_needed now --- web3_proxy/src/stats/influxdb_queries.rs | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/web3_proxy/src/stats/influxdb_queries.rs b/web3_proxy/src/stats/influxdb_queries.rs index 0d75881c..eb4a56b8 100644 --- a/web3_proxy/src/stats/influxdb_queries.rs +++ b/web3_proxy/src/stats/influxdb_queries.rs @@ -423,14 +423,20 @@ pub async fn query_user_stats<'a>( } } else if key == "archive_needed" { match value { - influxdb2_structmap::value::Value::Long(inner) => { + influxdb2_structmap::value::Value::String(inner) => { out.insert( "archive_needed".to_owned(), - serde_json::Value::Number(inner.into()), + if inner == "true" { + serde_json::Value::Bool(true) + } else if inner == "false" { + serde_json::Value::Bool(false) + } else { + serde_json::Value::String("error".to_owned()) + }, ); } _ => { - error!("archive_needed should always be a Long!"); + error!("archive_needed should always be a String!"); } } } else if key == "error_response" { From 29678b025ebab5bdaa51374a6b087df99a254774 Mon Sep 17 00:00:00 2001 From: yenicelik Date: Sun, 21 May 2023 20:27:46 -0400 Subject: [PATCH 49/66] updated error_response --- web3_proxy/src/stats/influxdb_queries.rs | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/web3_proxy/src/stats/influxdb_queries.rs b/web3_proxy/src/stats/influxdb_queries.rs index eb4a56b8..f31bf2e9 100644 --- a/web3_proxy/src/stats/influxdb_queries.rs +++ b/web3_proxy/src/stats/influxdb_queries.rs @@ -75,6 +75,7 @@ pub async fn query_user_stats<'a>( "_measurement".to_string(), "archive_needed".to_string(), "chain_id".to_string(), + "error_response".to_string(), ]; // Include a hashmap to go from rpc_secret_key_id to the rpc_secret_key @@ -190,14 +191,14 @@ pub async fn query_user_stats<'a>( |> pivot(rowKey: ["_time"], columnKey: ["_field"], valueColumn: "_value") |> drop(columns: ["balance"]) |> map(fn: (r) => ({{ r with "error_response": if r.error_response == "true" then r.frontend_requests else 0}})) - |> group(columns: ["_time", "_measurement", "archive_needed", "chain_id", "method", "rpc_secret_key_id"]) + |> group(columns: ["_time", "_measurement", "archive_needed", "chain_id", "error_response", "method", "rpc_secret_key_id"]) |> sort(columns: ["frontend_requests"]) |> map(fn:(r) => ({{ r with "sum_credits_used": float(v: r["sum_credits_used"]) }})) - |> cumulativeSum(columns: ["error_response", "backend_requests", "cache_hits", "cache_misses", "frontend_requests", "sum_credits_used", "sum_request_bytes", "sum_response_bytes", "sum_response_millis"]) + |> cumulativeSum(columns: ["backend_requests", "cache_hits", "cache_misses", "frontend_requests", "sum_credits_used", "sum_request_bytes", "sum_response_bytes", "sum_response_millis"]) |> sort(columns: ["frontend_requests"], desc: true) |> limit(n: 1) |> group() - |> sort(columns: ["_time", "_measurement", "archive_needed", "chain_id", "method", "rpc_secret_key_id"], desc: true) + |> sort(columns: ["_time", "_measurement", "archive_needed", "chain_id", "error_response", "method", "rpc_secret_key_id"], desc: true) "#); info!("Raw query to db is: {:?}", query); @@ -441,10 +442,16 @@ pub async fn query_user_stats<'a>( } } else if key == "error_response" { match value { - influxdb2_structmap::value::Value::Long(inner) => { + influxdb2_structmap::value::Value::String(inner) => { out.insert( "error_response".to_owned(), - serde_json::Value::Number(inner.into()), + if inner == "true" { + serde_json::Value::Bool(true) + } else if inner == "false" { + serde_json::Value::Bool(false) + } else { + serde_json::Value::String("error".to_owned()) + }, ); } _ => { From 59634139c902bde1c8ba6541be14ee73ab28b420 Mon Sep 17 00:00:00 2001 From: yenicelik Date: Sun, 21 May 2023 20:27:52 -0400 Subject: [PATCH 50/66] updated error_response --- web3_proxy/src/stats/influxdb_queries.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/web3_proxy/src/stats/influxdb_queries.rs b/web3_proxy/src/stats/influxdb_queries.rs index f31bf2e9..69d1110c 100644 --- a/web3_proxy/src/stats/influxdb_queries.rs +++ b/web3_proxy/src/stats/influxdb_queries.rs @@ -190,7 +190,6 @@ pub async fn query_user_stats<'a>( |> aggregateWindow(every: {query_window_seconds}s, fn: sum, createEmpty: false) |> pivot(rowKey: ["_time"], columnKey: ["_field"], valueColumn: "_value") |> drop(columns: ["balance"]) - |> map(fn: (r) => ({{ r with "error_response": if r.error_response == "true" then r.frontend_requests else 0}})) |> group(columns: ["_time", "_measurement", "archive_needed", "chain_id", "error_response", "method", "rpc_secret_key_id"]) |> sort(columns: ["frontend_requests"]) |> map(fn:(r) => ({{ r with "sum_credits_used": float(v: r["sum_credits_used"]) }})) From 91eeee23e2f2eee2158551da2a649033baa4a3b7 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Mon, 22 May 2023 15:32:15 -0700 Subject: [PATCH 51/66] use ether's reconnects instead of our own which need a lock (wip) --- web3_proxy/src/config.rs | 2 - web3_proxy/src/frontend/users/payment.rs | 5 +- web3_proxy/src/rpcs/blockchain.rs | 3 +- web3_proxy/src/rpcs/many.rs | 20 +- web3_proxy/src/rpcs/one.rs | 404 ++++++----------------- web3_proxy/src/rpcs/provider.rs | 155 ++++----- web3_proxy/src/rpcs/request.rs | 52 +-- web3_proxy/src/rpcs/transactions.rs | 3 +- 8 files changed, 191 insertions(+), 453 deletions(-) diff --git a/web3_proxy/src/config.rs b/web3_proxy/src/config.rs index 05a947d5..fc156631 100644 --- a/web3_proxy/src/config.rs +++ b/web3_proxy/src/config.rs @@ -287,7 +287,6 @@ impl Web3RpcConfig { blocks_by_hash_cache: BlocksByHashCache, block_sender: Option>, tx_id_sender: Option>, - reconnect: bool, ) -> anyhow::Result<(Arc, AnyhowJoinHandle<()>)> { if !self.extra.is_empty() { warn!("unknown Web3RpcConfig fields!: {:?}", self.extra.keys()); @@ -304,7 +303,6 @@ impl Web3RpcConfig { blocks_by_hash_cache, block_sender, tx_id_sender, - reconnect, ) .await } diff --git a/web3_proxy/src/frontend/users/payment.rs b/web3_proxy/src/frontend/users/payment.rs index 975fb3da..728728d0 100644 --- a/web3_proxy/src/frontend/users/payment.rs +++ b/web3_proxy/src/frontend/users/payment.rs @@ -166,7 +166,6 @@ pub async fn user_balance_post( "eth_getTransactionReceipt", &vec![format!("0x{}", hex::encode(tx_hash))], Level::Trace.into(), - None, ) .await // TODO: What kind of error would be here @@ -217,7 +216,7 @@ pub async fn user_balance_post( ]); debug!("Params are: {:?}", ¶ms); let accepted_token: String = handle - .request("eth_call", ¶ms, Level::Trace.into(), None) + .request("eth_call", ¶ms, Level::Trace.into()) .await // TODO: What kind of error would be here .map_err(|err| Web3ProxyError::Anyhow(err.into()))?; @@ -267,7 +266,7 @@ pub async fn user_balance_post( ]); debug!("ERC20 Decimal request params are: {:?}", ¶ms); let decimals: String = handle - .request("eth_call", ¶ms, Level::Trace.into(), None) + .request("eth_call", ¶ms, Level::Trace.into()) .await .map_err(|err| Web3ProxyError::Anyhow(err.into()))?; debug!("Decimals response is: {:?}", decimals); diff --git a/web3_proxy/src/rpcs/blockchain.rs b/web3_proxy/src/rpcs/blockchain.rs index 9c181e0d..2b71b1fe 100644 --- a/web3_proxy/src/rpcs/blockchain.rs +++ b/web3_proxy/src/rpcs/blockchain.rs @@ -219,13 +219,12 @@ impl Web3Rpcs { // TODO: if error, retry? let block: Web3ProxyBlock = match rpc { Some(rpc) => rpc - .wait_for_request_handle(authorization, Some(Duration::from_secs(30)), None) + .wait_for_request_handle(authorization, Some(Duration::from_secs(30))) .await? .request::<_, Option>( "eth_getBlockByHash", &json!(get_block_params), Level::Error.into(), - None, ) .await? .and_then(|x| { diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index 5f281bee..74ef6776 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -276,7 +276,6 @@ impl Web3Rpcs { blocks_by_hash_cache, block_sender, pending_tx_id_sender, - true, )); Some(handle) @@ -303,9 +302,9 @@ impl Web3Rpcs { while new_head_receiver.borrow_and_update().is_none() { new_head_receiver.changed().await?; } - } - old_rpc.disconnect().await.context("disconnect old rpc")?; + // TODO: tell ethers to disconnect? is there a function for that? + } } // TODO: what should we do with the new handle? make sure error logs aren't dropped @@ -435,7 +434,7 @@ impl Web3Rpcs { .into_iter() .map(|active_request_handle| async move { let result: Result, _> = active_request_handle - .request(method, &json!(¶ms), error_level.into(), None) + .request(method, &json!(¶ms), error_level.into()) .await; result }) @@ -508,7 +507,7 @@ impl Web3Rpcs { skip.push(Arc::clone(faster_rpc)); // just because it has lower latency doesn't mean we are sure to get a connection. there might be rate limits - match faster_rpc.try_request_handle(authorization, None).await { + match faster_rpc.try_request_handle(authorization).await { Ok(OpenRequestResult::Handle(handle)) => { trace!("opened handle: {}", faster_rpc); return OpenRequestResult::Handle(handle); @@ -831,7 +830,7 @@ impl Web3Rpcs { } // check rate limits and increment our connection counter - match rpc.try_request_handle(authorization, None).await { + match rpc.try_request_handle(authorization).await { Ok(OpenRequestResult::RetryAt(retry_at)) => { // this rpc is not available. skip it trace!("{} is rate limited. skipping", rpc); @@ -908,7 +907,6 @@ impl Web3Rpcs { &request.method, &json!(request.params), RequestErrorHandler::Save, - None, ) .await; @@ -1307,8 +1305,8 @@ mod tests { use std::time::{SystemTime, UNIX_EPOCH}; use super::*; + use crate::rpcs::blockchain::Web3ProxyBlock; use crate::rpcs::consensus::ConsensusFinder; - use crate::rpcs::{blockchain::Web3ProxyBlock, provider::Web3Provider}; use arc_swap::ArcSwap; use ethers::types::H256; use ethers::types::{Block, U256}; @@ -1451,7 +1449,6 @@ mod tests { block_data_limit: block_data_limit.into(), tier: 0, head_block: Some(tx_synced), - provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), peak_latency: Some(new_peak_latency()), ..Default::default() }; @@ -1466,7 +1463,6 @@ mod tests { block_data_limit: block_data_limit.into(), tier: 0, head_block: Some(tx_lagged), - provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), peak_latency: Some(new_peak_latency()), ..Default::default() }; @@ -1707,7 +1703,6 @@ mod tests { block_data_limit: 64.into(), tier: 1, head_block: Some(tx_pruned), - provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), ..Default::default() }; @@ -1721,7 +1716,6 @@ mod tests { block_data_limit: u64::MAX.into(), tier: 2, head_block: Some(tx_archive), - provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), ..Default::default() }; @@ -1876,7 +1870,6 @@ mod tests { block_data_limit: 64.into(), tier: 0, head_block: Some(tx_mock_geth), - provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), peak_latency: Some(new_peak_latency()), ..Default::default() }; @@ -1889,7 +1882,6 @@ mod tests { block_data_limit: u64::MAX.into(), tier: 1, head_block: Some(tx_mock_erigon_archive), - provider: AsyncRwLock::new(Some(Arc::new(Web3Provider::Mock))), peak_latency: Some(new_peak_latency()), ..Default::default() }; diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index c3d8b34f..dd84fa4b 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -1,6 +1,6 @@ ///! Rate-limited communication with a web3 provider. use super::blockchain::{ArcBlock, BlocksByHashCache, Web3ProxyBlock}; -use super::provider::Web3Provider; +use super::provider::{connect_http, connect_ws, EthersHttpProvider, EthersWsProvider}; use super::request::{OpenRequestHandle, OpenRequestResult}; use crate::app::{flatten_handle, AnyhowJoinHandle}; use crate::config::{BlockAndRpc, Web3RpcConfig}; @@ -40,17 +40,10 @@ pub struct Web3Rpc { pub name: String, pub display_name: Option, pub db_conn: Option, - pub(super) ws_url: Option, - pub(super) http_url: Option, - /// Some connections use an http_client. we keep a clone for reconnecting - pub(super) http_client: Option, - /// provider is in a RwLock so that we can replace it if re-connecting - /// it is an async lock because we hold it open across awaits - /// this provider is only used for new heads subscriptions - /// TODO: benchmark ArcSwapOption and a watch::Sender - /// TODO: only the websocket provider needs to be behind an asyncrwlock! - /// TODO: the http provider is just an http_client - pub(super) provider: AsyncRwLock>>, + /// most all requests prefer use the http_provider + pub(super) http_provider: Option, + /// the websocket provider is only used for subscriptions + pub(super) ws_provider: Option, /// keep track of hard limits /// this is only inside an Option so that the "Default" derive works. it will always be set. pub(super) hard_limit_until: Option>, @@ -79,7 +72,6 @@ pub struct Web3Rpc { /// TODO: maybe move this to graphana pub(super) total_requests: AtomicUsize, pub(super) active_requests: AtomicUsize, - pub(super) reconnect: AtomicBool, /// this is only inside an Option so that the "Default" derive works. it will always be set. pub(super) disconnect_watch: Option>, pub(super) created_at: Option, @@ -102,7 +94,6 @@ impl Web3Rpc { block_map: BlocksByHashCache, block_sender: Option>, tx_id_sender: Option)>>, - reconnect: bool, ) -> anyhow::Result<(Arc, AnyhowJoinHandle<()>)> { let created_at = Instant::now(); @@ -161,7 +152,6 @@ impl Web3Rpc { } let (disconnect_sender, disconnect_receiver) = watch::channel(false); - let reconnect = reconnect.into(); let (head_block, _) = watch::channel(None); @@ -169,6 +159,7 @@ impl Web3Rpc { // TODO Should these defaults be in config let peak_latency = PeakEwmaLatency::spawn( // Decay over 15s + // TODO! This is wrong! needs to be as_nanos! Duration::from_secs(15).as_millis() as f64, // Peak requests so far around 5k, we will use an order of magnitude // more to be safe. Should only use about 50mb RAM @@ -177,41 +168,46 @@ impl Web3Rpc { Duration::from_secs(1), ); - let http_url = if let Some(http_url) = config.http_url { - Some(http_url.parse()?) + let http_provider = if let Some(http_url) = config.http_url { + let http_url = http_url.parse::()?; + + Some(connect_http(Cow::Owned(http_url), http_client)?) + + // TODO: check the provider is on the right chain } else { None }; - let ws_url = if let Some(ws_url) = config.ws_url { - Some(ws_url.parse()?) + let ws_provider = if let Some(ws_url) = config.ws_url { + let ws_url = ws_url.parse::()?; + + Some(connect_ws(Cow::Owned(ws_url), usize::MAX).await?) + + // TODO: check the provider is on the right chain } else { None }; - let new_connection = Self { - name, - db_conn: db_conn.clone(), - display_name: config.display_name, - http_client, - ws_url, - http_url, - hard_limit, - hard_limit_until: Some(hard_limit_until), - soft_limit: config.soft_limit, + let new_rpc = Self { automatic_block_limit, backup, block_data_limit, - reconnect, - tier: config.tier, - disconnect_watch: Some(disconnect_sender), created_at: Some(created_at), + db_conn: db_conn.clone(), + disconnect_watch: Some(disconnect_sender), + display_name: config.display_name, + hard_limit, + hard_limit_until: Some(hard_limit_until), head_block: Some(head_block), + http_provider, + name, peak_latency: Some(peak_latency), + soft_limit: config.soft_limit, + tier: config.tier, ..Default::default() }; - let new_connection = Arc::new(new_connection); + let new_connection = Arc::new(new_rpc); // subscribe to new blocks and new transactions // subscribing starts the connection (with retries) @@ -256,7 +252,6 @@ impl Web3Rpc { async fn check_block_data_limit( self: &Arc, authorization: &Arc, - unlocked_provider: Option>, ) -> anyhow::Result> { if !self.automatic_block_limit { // TODO: is this a good thing to return? @@ -270,16 +265,13 @@ impl Web3Rpc { // TODO: binary search between 90k and max? // TODO: start at 0 or 1? for block_data_limit in [0, 32, 64, 128, 256, 512, 1024, 90_000, u64::MAX] { - let handle = self - .wait_for_request_handle(authorization, None, unlocked_provider.clone()) - .await?; + let handle = self.wait_for_request_handle(authorization, None).await?; let head_block_num_future = handle.request::, U256>( "eth_blockNumber", &None, // error here are expected, so keep the level low Level::Debug.into(), - unlocked_provider.clone(), ); let head_block_num = timeout(Duration::from_secs(5), head_block_num_future) @@ -297,9 +289,7 @@ impl Web3Rpc { // TODO: wait for the handle BEFORE we check the current block number. it might be delayed too! // TODO: what should the request be? - let handle = self - .wait_for_request_handle(authorization, None, unlocked_provider.clone()) - .await?; + let handle = self.wait_for_request_handle(authorization, None).await?; let archive_result: Result = handle .request( @@ -310,7 +300,6 @@ impl Web3Rpc { )), // error here are expected, so keep the level low Level::Trace.into(), - unlocked_provider.clone(), ) .await; @@ -388,204 +377,50 @@ impl Web3Rpc { true } - /// reconnect to the provider. errors are retried forever with exponential backoff with jitter. - /// We use the "Decorrelated" jitter from - /// TODO: maybe it would be better to use "Full Jitter". The "Full Jitter" approach uses less work, but slightly more time. - pub async fn retrying_connect( + /// query the web3 provider to confirm it is on the expected chain with the expected data available + async fn check_provider( self: &Arc, block_sender: Option<&flume::Sender>, chain_id: u64, db_conn: Option<&DatabaseConnection>, - delay_start: bool, ) -> anyhow::Result<()> { - // there are several crates that have retry helpers, but they all seem more complex than necessary - // TODO: move this backoff logic into a helper function so we can use it when doing database locking - let base_ms = 500; - let cap_ms = 30_000; - let range_multiplier = 3; + let authorization = Arc::new(Authorization::internal(db_conn.cloned())?); - // sleep once before the initial retry attempt - // TODO: now that we use this method for our initial connection, do we still want this sleep? - let mut sleep_ms = if delay_start { - let first_sleep_ms = min( - cap_ms, - thread_fast_rng().gen_range(base_ms..(base_ms * range_multiplier)), - ); - let reconnect_in = Duration::from_millis(first_sleep_ms); + // check the server's chain_id here + // TODO: some public rpcs (on bsc and fantom) do not return an id and so this ends up being an error + // TODO: what should the timeout be? should there be a request timeout? + // trace!("waiting on chain id for {}", self); + let found_chain_id: Result = self + .wait_for_request_handle(&authorization, None) + .await + .context(format!("waiting for request handle on {}", self))? + .request("eth_chainId", &json!(Vec::<()>::new()), Level::Trace.into()) + .await; + trace!("found_chain_id: {:#?}", found_chain_id); - info!("Reconnect to {} in {}ms", self, reconnect_in.as_millis()); - - sleep(reconnect_in).await; - - first_sleep_ms - } else { - base_ms - }; - - // retry until we succeed - while let Err(err) = self.connect(block_sender, chain_id, db_conn).await { - // thread_rng is crytographically secure. we don't need that here. use thread_fast_rng instead - // TODO: min of 1 second? sleep longer if rate limited? - sleep_ms = min( - cap_ms, - thread_fast_rng().gen_range(base_ms..(sleep_ms * range_multiplier)), - ); - - let retry_in = Duration::from_millis(sleep_ms); - - let error_level = if self.backup { - log::Level::Debug - } else { - log::Level::Info - }; - - log::log!( - error_level, - "Failed (re)connect to {}! Retry in {}ms. err={:?}", - self, - retry_in.as_millis(), - err, - ); - - sleep(retry_in).await; + match found_chain_id { + Ok(found_chain_id) => { + // TODO: there has to be a cleaner way to do this + if chain_id != found_chain_id.as_u64() { + return Err(anyhow::anyhow!( + "incorrect chain id! Config has {}, but RPC has {}", + chain_id, + found_chain_id + ) + .context(format!("failed @ {}", self))); + } + } + Err(e) => { + return Err(anyhow::Error::from(e) + .context(format!("unable to parse eth_chainId from {}", self))); + } } - Ok(()) - } + self.check_block_data_limit(&authorization) + .await + .context(format!("unable to check_block_data_limit of {}", self))?; - /// connect to the web3 provider - async fn connect( - self: &Arc, - block_sender: Option<&flume::Sender>, - chain_id: u64, - db_conn: Option<&DatabaseConnection>, - ) -> anyhow::Result<()> { - if let Ok(mut unlocked_provider) = self.provider.try_write() { - #[cfg(test)] - if let Some(Web3Provider::Mock) = unlocked_provider.as_deref() { - return Ok(()); - } - - *unlocked_provider = if let Some(ws_url) = self.ws_url.as_ref() { - // set up ws client - match &*unlocked_provider { - None => { - info!("connecting to {}", self); - } - Some(_) => { - debug!("reconnecting to {}", self); - - // tell the block subscriber that this rpc doesn't have any blocks - if let Some(block_sender) = block_sender { - block_sender - .send_async((None, self.clone())) - .await - .context("block_sender during connect")?; - } - - // reset sync status - self.head_block - .as_ref() - .expect("head_block should always be set") - .send_replace(None); - - // disconnect the current provider - // TODO: what until the block_sender's receiver finishes updating this item? - *unlocked_provider = None; - } - } - - let p = Web3Provider::new(Cow::Borrowed(ws_url), None) - .await - .context(format!("failed connecting to {}", ws_url))?; - - assert!(p.ws().is_some()); - - Some(Arc::new(p)) - } else { - // http client - if let Some(url) = &self.http_url { - let p = Web3Provider::new(Cow::Borrowed(url), self.http_client.clone()) - .await - .context(format!("failed connecting to {}", url))?; - - assert!(p.http().is_some()); - - Some(Arc::new(p)) - } else { - None - } - }; - - let authorization = Arc::new(Authorization::internal(db_conn.cloned())?); - - // check the server's chain_id here - // TODO: some public rpcs (on bsc and fantom) do not return an id and so this ends up being an error - // TODO: what should the timeout be? should there be a request timeout? - // trace!("waiting on chain id for {}", self); - let found_chain_id: Result = self - .wait_for_request_handle(&authorization, None, unlocked_provider.clone()) - .await - .context(format!("waiting for request handle on {}", self))? - .request( - "eth_chainId", - &json!(Vec::<()>::new()), - Level::Trace.into(), - unlocked_provider.clone(), - ) - .await; - trace!("found_chain_id: {:#?}", found_chain_id); - - match found_chain_id { - Ok(found_chain_id) => { - // TODO: there has to be a cleaner way to do this - if chain_id != found_chain_id.as_u64() { - return Err(anyhow::anyhow!( - "incorrect chain id! Config has {}, but RPC has {}", - chain_id, - found_chain_id - ) - .context(format!("failed @ {}", self))); - } - } - Err(e) => { - return Err(anyhow::Error::from(e) - .context(format!("unable to parse eth_chainId from {}", self))); - } - } - - self.check_block_data_limit(&authorization, unlocked_provider.clone()) - .await - .context(format!("unable to check_block_data_limit of {}", self))?; - - drop(unlocked_provider); - - info!("successfully connected to {}", self); - } else if self.provider.read().await.is_none() { - return Err(anyhow!("failed waiting for client {}", self)); - }; - - Ok(()) - } - - pub async fn disconnect(&self) -> anyhow::Result<()> { - let age = self.created_at.unwrap().elapsed().as_secs(); - - info!("disconnecting {} ({}s old)", self, age); - - self.reconnect.store(false, atomic::Ordering::Release); - - if let Err(err) = self.disconnect_watch.as_ref().unwrap().send(true) { - warn!("failed sending disconnect watch: {:?}", err); - }; - - trace!("disconnecting (locking) {} ({}s old)", self, age); - - let mut provider = self.provider.write().await; - - trace!("disconnecting (clearing provider) {} ({}s old)", self, age); - - *provider = None; + info!("successfully connected to {}", self); Ok(()) } @@ -638,7 +473,7 @@ impl Web3Rpc { if self.block_data_limit() == U64::zero() { let authorization = Arc::new(Authorization::internal(self.db_conn.clone())?); - if let Err(err) = self.check_block_data_limit(&authorization, None).await { + if let Err(err) = self.check_block_data_limit(&authorization).await { warn!( "failed checking block limit after {} finished syncing. {:?}", self, err @@ -670,7 +505,7 @@ impl Web3Rpc { *self.disconnect_watch.as_ref().unwrap().borrow() } - /// subscribe to blocks and transactions with automatic reconnects + /// subscribe to blocks and transactions /// This should only exit when the program is exiting. /// TODO: should more of these args be on self? #[allow(clippy::too_many_arguments)] @@ -690,15 +525,12 @@ impl Web3Rpc { RequestErrorHandler::ErrorLevel }; - let mut delay_start = false; + todo!(); - // this does loop. just only when reconnect is enabled - #[allow(clippy::never_loop)] - loop { - trace!("subscription loop started on {}", self); - - let mut futures = vec![]; + /* + let mut futures = vec![]; + while false { let http_interval_receiver = http_interval_sender.as_ref().map(|x| x.subscribe()); { @@ -741,7 +573,7 @@ impl Web3Rpc { // TODO: what if we just happened to have this check line up with another restart? // TODO: think more about this - if let Some(client) = rpc.provider.read().await.clone() { + if let Some(client) = rpc.ws_provider.read().await.clone() { // health check as a way of keeping this rpc's request_ewma accurate // TODO: do something different if this is a backup server? @@ -785,6 +617,7 @@ impl Web3Rpc { let code = match to { Err(err) => { + // TODO: an "error" here just means that the hash wasn't available. i dont think its truly an "error" if rpc.backup { debug!( "{} failed health check query! {:#?}", @@ -893,6 +726,8 @@ impl Web3Rpc { } } + + */ info!("all subscriptions on {} completed", self); Ok(()) @@ -908,8 +743,15 @@ impl Web3Rpc { ) -> anyhow::Result<()> { trace!("watching new heads on {}", self); - let provider = self.wait_for_provider().await; + if let Some(ws_provider) = self.ws_provider.as_ref() { + todo!("subscribe") + } else if let Some(http_provider) = self.http_provider.as_ref() { + todo!("poll") + } else { + unimplemented!("no ws or http provider!") + } + /* match provider.as_ref() { Web3Provider::Http(_client) => { // there is a "watch_blocks" function, but a lot of public nodes do not support the necessary rpc endpoints @@ -922,17 +764,13 @@ impl Web3Rpc { while !self.should_disconnect() { // TODO: what should the max_wait be? // we do not pass unlocked_provider because we want to get a new one each call. otherwise we might re-use an old one - match self - .wait_for_request_handle(&authorization, None, None) - .await - { + match self.wait_for_request_handle(&authorization, None).await { Ok(active_request_handle) => { let block: Result, _> = active_request_handle .request( "eth_getBlockByNumber", &json!(("latest", false)), Level::Warn.into(), - None, ) .await; @@ -1017,9 +855,8 @@ impl Web3Rpc { } Web3Provider::Both(_, client) | Web3Provider::Ws(client) => { // todo: move subscribe_blocks onto the request handle? - let active_request_handle = self - .wait_for_request_handle(&authorization, None, Some(provider.clone())) - .await; + let active_request_handle = + self.wait_for_request_handle(&authorization, None).await; let mut stream = client.subscribe_blocks().await?; drop(active_request_handle); @@ -1029,13 +866,12 @@ impl Web3Rpc { // TODO: how does this get wrapped in an arc? does ethers handle that? // TODO: do this part over http? let block: Result, _> = self - .wait_for_request_handle(&authorization, None, Some(provider.clone())) + .wait_for_request_handle(&authorization, None) .await? .request( "eth_getBlockByNumber", &json!(("latest", false)), Level::Warn.into(), - Some(provider.clone()), ) .await; @@ -1082,6 +918,7 @@ impl Web3Rpc { #[cfg(test)] Web3Provider::Mock => unimplemented!(), } + */ // clear the head block. this might not be needed, but it won't hurt self.send_head_block_result(Ok(None), &block_sender, block_map) @@ -1100,10 +937,10 @@ impl Web3Rpc { authorization: Arc, tx_id_sender: flume::Sender<(TxHash, Arc)>, ) -> anyhow::Result<()> { - // TODO: give this a separate client. don't use new_head_client for everything. especially a firehose this big - // TODO: timeout - let provider = self.wait_for_provider().await; + // TODO: make this subscription optional + self.wait_for_disconnect().await?; + /* trace!("watching pending transactions on {}", self); // TODO: does this keep the lock open for too long? match provider.as_ref() { @@ -1144,6 +981,7 @@ impl Web3Rpc { self.wait_for_disconnect().await?; } } + */ if self.should_disconnect() { Ok(()) @@ -1155,21 +993,15 @@ impl Web3Rpc { } /// be careful with this; it might wait forever! - /// `allow_not_ready` is only for use by health checks while starting the provider - /// TODO: don't use anyhow. use specific error type pub async fn wait_for_request_handle<'a>( self: &'a Arc, authorization: &'a Arc, max_wait: Option, - unlocked_provider: Option>, ) -> Web3ProxyResult { let max_wait = max_wait.map(|x| Instant::now() + x); loop { - match self - .try_request_handle(authorization, unlocked_provider.clone()) - .await - { + match self.try_request_handle(authorization).await { Ok(OpenRequestResult::Handle(handle)) => return Ok(handle), Ok(OpenRequestResult::RetryAt(retry_at)) => { // TODO: emit a stat? @@ -1214,18 +1046,8 @@ impl Web3Rpc { pub async fn try_request_handle( self: &Arc, authorization: &Arc, - // TODO: borrow on this instead of needing to clone the Arc? - unlocked_provider: Option>, ) -> Web3ProxyResult { - // TODO: think more about this read block - // TODO: this should *not* be new_head_client. this should be a separate object - if unlocked_provider.is_some() || self.provider.read().await.is_some() { - // we already have an unlocked provider. no need to lock - } else { - warn!("no provider on {}", self); - // TODO: wait for provider? that will probably slow us down more than we want - return Ok(OpenRequestResult::NotReady); - } + // TODO: if websocket is reconnecting, return an error? // check cached rate limits if let Some(hard_limit_until) = self.hard_limit_until.as_ref() { @@ -1291,59 +1113,34 @@ impl Web3Rpc { } } - async fn wait_for_provider(&self) -> Arc { - let mut provider = self.provider.read().await.clone(); - - let mut logged = false; - while provider.is_none() { - // trace!("waiting on unlocked_provider: locking..."); - sleep(Duration::from_millis(100)).await; - - if !logged { - debug!("waiting for provider on {}", self); - logged = true; - } - - provider = self.provider.read().await.clone(); - } - - provider.unwrap() - } - pub async fn wait_for_query( self: &Arc, method: &str, params: &P, revert_handler: RequestErrorHandler, authorization: Arc, - unlocked_provider: Option>, ) -> anyhow::Result where // TODO: not sure about this type. would be better to not need clones, but measure and spawns combine to need it P: Clone + fmt::Debug + serde::Serialize + Send + Sync + 'static, R: serde::Serialize + serde::de::DeserializeOwned + fmt::Debug + Send, { - self.wait_for_request_handle(&authorization, None, None) + self.wait_for_request_handle(&authorization, None) .await? - .request::(method, params, revert_handler, unlocked_provider) + .request::(method, params, revert_handler) .await .context("ProviderError from the backend") } } -impl fmt::Debug for Web3Provider { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - // TODO: the default Debug takes forever to write. this is too quiet though. we at least need the url - f.debug_struct("Web3Provider").finish_non_exhaustive() - } -} - impl Hash for Web3Rpc { fn hash(&self, state: &mut H) { self.name.hash(state); self.display_name.hash(state); - self.http_url.hash(state); - self.ws_url.hash(state); + self.http_provider.as_ref().map(|x| x.url()).hash(state); + // TODO: figure out how to get the url for the provider + // TODO: url does NOT include the authorization data. i think created_at should protect us if auth changes without anything else + // self.ws_provider.map(|x| x.url()).hash(state); self.automatic_block_limit.hash(state); self.backup.hash(state); // TODO: don't include soft_limit if we change them to be dynamic @@ -1481,7 +1278,6 @@ mod tests { let x = Web3Rpc { name: "name".to_string(), - ws_url: Some("ws://example.com".parse::().unwrap()), soft_limit: 1_000, automatic_block_limit: false, backup: false, diff --git a/web3_proxy/src/rpcs/provider.rs b/web3_proxy/src/rpcs/provider.rs index a70bc88f..1829b0a5 100644 --- a/web3_proxy/src/rpcs/provider.rs +++ b/web3_proxy/src/rpcs/provider.rs @@ -5,92 +5,81 @@ use std::{borrow::Cow, time::Duration}; use url::Url; // TODO: our own structs for these that handle streaming large responses -type EthersHttpProvider = ethers::providers::Provider; -type EthersWsProvider = ethers::providers::Provider; +pub type EthersHttpProvider = ethers::providers::Provider; +pub type EthersWsProvider = ethers::providers::Provider; -/// Use HTTP and WS providers. -// TODO: instead of an enum, I tried to use Box, but hit -// TODO: custom types that let us stream JSON responses -#[derive(From)] -pub enum Web3Provider { - Both(EthersHttpProvider, EthersWsProvider), - Http(EthersHttpProvider), - // TODO: deadpool? custom tokio-tungstenite - Ws(EthersWsProvider), - #[cfg(test)] - Mock, -} +pub fn extract_auth(url: &mut Cow<'_, Url>) -> Option { + if let Some(pass) = url.password().map(|x| x.to_string()) { + // to_string is needed because we are going to remove these items from the url + let user = url.username().to_string(); -impl Web3Provider { - pub fn http(&self) -> Option<&EthersHttpProvider> { - match self { - Self::Http(x) => Some(x), - _ => None, - } - } + // clear username and password from the url + let mut_url = url.to_mut(); - pub fn ws(&self) -> Option<&EthersWsProvider> { - match self { - Self::Both(_, x) | Self::Ws(x) => Some(x), - _ => None, - } - } + mut_url + .set_username("") + .expect("unable to clear username on websocket"); + mut_url + .set_password(None) + .expect("unable to clear password on websocket"); - /// Note, if the http url has an authority the http_client param is ignored and a dedicated http_client will be used - /// TODO: take a reqwest::Client or a reqwest::ClientBuilder. that way we can do things like set compression even when auth is set - pub async fn new( - mut url: Cow<'_, Url>, - http_client: Option, - ) -> anyhow::Result { - let auth = if let Some(pass) = url.password().map(|x| x.to_string()) { - // to_string is needed because we are going to remove these items from the url - let user = url.username().to_string(); - - // clear username and password from the url - let mut_url = url.to_mut(); - - mut_url - .set_username("") - .map_err(|_| anyhow!("unable to clear username on websocket"))?; - mut_url - .set_password(None) - .map_err(|_| anyhow!("unable to clear password on websocket"))?; - - // keep them - Some(Authorization::basic(user, pass)) - } else { - None - }; - - let provider = if url.scheme().starts_with("http") { - let provider = if let Some(auth) = auth { - ethers::providers::Http::new_with_auth(url.into_owned(), auth)? - } else if let Some(http_client) = http_client { - ethers::providers::Http::new_with_client(url.into_owned(), http_client) - } else { - ethers::providers::Http::new(url.into_owned()) - }; - - // TODO: i don't think this interval matters for our uses, but we should probably set it to like `block time / 2` - ethers::providers::Provider::new(provider) - .interval(Duration::from_secs(12)) - .into() - } else if url.scheme().starts_with("ws") { - let provider = if auth.is_some() { - let connection_details = ConnectionDetails::new(url.as_str(), auth); - - ethers::providers::Ws::connect(connection_details).await? - } else { - ethers::providers::Ws::connect(url.as_str()).await? - }; - - // TODO: dry this up (needs https://github.com/gakonst/ethers-rs/issues/592) - // TODO: i don't think this interval matters - ethers::providers::Provider::new(provider).into() - } else { - return Err(anyhow::anyhow!("only http and ws servers are supported")); - }; - - Ok(provider) + // keep them + Some(Authorization::basic(user, pass)) + } else { + None } } + +/// Note, if the http url has an authority the http_client param is ignored and a dedicated http_client will be used +/// TODO: take a reqwest::Client or a reqwest::ClientBuilder. that way we can do things like set compression even when auth is set +pub fn connect_http( + mut url: Cow<'_, Url>, + http_client: Option, +) -> anyhow::Result { + let auth = extract_auth(&mut url); + + let provider = if url.scheme().starts_with("http") { + let provider = if let Some(auth) = auth { + ethers::providers::Http::new_with_auth(url.into_owned(), auth)? + } else if let Some(http_client) = http_client { + ethers::providers::Http::new_with_client(url.into_owned(), http_client) + } else { + ethers::providers::Http::new(url.into_owned()) + }; + + // TODO: i don't think this interval matters for our uses, but we should probably set it to like `block time / 2` + ethers::providers::Provider::new(provider) + .interval(Duration::from_secs(12)) + .into() + } else { + return Err(anyhow::anyhow!("only http servers are supported")); + }; + + Ok(provider) +} + +pub async fn connect_ws( + mut url: Cow<'_, Url>, + reconnects: usize, +) -> anyhow::Result { + let auth = extract_auth(&mut url); + + let provider = if url.scheme().starts_with("ws") { + let provider = if auth.is_some() { + let connection_details = ConnectionDetails::new(url.as_str(), auth); + + // if they error, we do our own reconnection with backoff + ethers::providers::Ws::connect_with_reconnects(connection_details, reconnects).await? + } else { + ethers::providers::Ws::connect_with_reconnects(url.as_str(), reconnects).await? + }; + + // TODO: dry this up (needs https://github.com/gakonst/ethers-rs/issues/592) + // TODO: i don't think this interval matters + ethers::providers::Provider::new(provider).into() + } else { + return Err(anyhow::anyhow!("ws servers are supported")); + }; + + Ok(provider) +} diff --git a/web3_proxy/src/rpcs/request.rs b/web3_proxy/src/rpcs/request.rs index c50dd202..7a13335f 100644 --- a/web3_proxy/src/rpcs/request.rs +++ b/web3_proxy/src/rpcs/request.rs @@ -1,5 +1,4 @@ use super::one::Web3Rpc; -use super::provider::Web3Provider; use crate::frontend::authorization::Authorization; use anyhow::Context; use chrono::Utc; @@ -158,7 +157,6 @@ impl OpenRequestHandle { method: &str, params: &P, mut error_handler: RequestErrorHandler, - unlocked_provider: Option>, ) -> Result where // TODO: not sure about this type. would be better to not need clones, but measure and spawns combine to need it @@ -170,29 +168,6 @@ impl OpenRequestHandle { // trace!(rpc=%self.rpc, %method, "request"); trace!("requesting from {}", self.rpc); - let mut provider = if unlocked_provider.is_some() { - unlocked_provider - } else { - self.rpc.provider.read().await.clone() - }; - - let mut logged = false; - // TODO: instead of a lock, i guess it should be a watch? - while provider.is_none() { - // trace!("waiting on provider: locking..."); - // TODO: i dont like this. subscribing to a channel could be better - sleep(Duration::from_millis(100)).await; - - if !logged { - debug!("no provider for open handle on {}", self.rpc); - logged = true; - } - - provider = self.rpc.provider.read().await.clone(); - } - - let provider = provider.expect("provider was checked already"); - self.rpc .total_requests .fetch_add(1, std::sync::atomic::Ordering::AcqRel); @@ -202,21 +177,16 @@ impl OpenRequestHandle { let start = Instant::now(); // TODO: replace ethers-rs providers with our own that supports streaming the responses - let response = match provider.as_ref() { - #[cfg(test)] - Web3Provider::Mock => { - return Err(ProviderError::CustomError( - "mock provider can't respond".to_string(), - )) - } - Web3Provider::Ws(p) => p.request(method, params).await, - Web3Provider::Http(p) | Web3Provider::Both(p, _) => { - // TODO: i keep hearing that http is faster. but ws has always been better for me. investigate more with actual benchmarks - p.request(method, params).await - } + // TODO: replace ethers-rs providers with our own that handles "id" being null + let response: Result = if let Some(ref p) = self.rpc.http_provider { + p.request(method, params).await + } else if let Some(ref p) = self.rpc.ws_provider { + p.request(method, params).await + } else { + unimplemented!("no provider. cannot send request") }; - // note. we intentionally do not record this latency now. we do NOT want to measure errors + // we do NOT want to measure errors, so we intentionally do not record this latency now. let latency = start.elapsed(); // we used to fetch_sub the active_request count here, but sometimes the handle is dropped without request being called! @@ -277,11 +247,7 @@ impl OpenRequestHandle { // TODO: move this info a function on ResponseErrorType let response_type = if let ProviderError::JsonRpcClientError(err) = err { // Http and Ws errors are very similar, but different types - let msg = match &*provider { - #[cfg(test)] - Web3Provider::Mock => unimplemented!(), - _ => err.as_error_response().map(|x| x.message.clone()), - }; + let msg = err.as_error_response().map(|x| x.message.clone()); trace!("error message: {:?}", msg); diff --git a/web3_proxy/src/rpcs/transactions.rs b/web3_proxy/src/rpcs/transactions.rs index d8c007ee..c46da986 100644 --- a/web3_proxy/src/rpcs/transactions.rs +++ b/web3_proxy/src/rpcs/transactions.rs @@ -29,14 +29,13 @@ impl Web3Rpcs { // TODO: yearn devs have had better luck with batching these, but i think that's likely just adding a delay itself // TODO: if one rpc fails, try another? // TODO: try_request_handle, or wait_for_request_handle? I think we want wait here - let tx: Transaction = match rpc.try_request_handle(authorization, None).await { + let tx: Transaction = match rpc.try_request_handle(authorization).await { Ok(OpenRequestResult::Handle(handle)) => { handle .request( "eth_getTransactionByHash", &(pending_tx_id,), Level::Error.into(), - None, ) .await? } From 3ac15558e38f7fd65df69b460999d0af2ac2be38 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Mon, 22 May 2023 15:43:39 -0700 Subject: [PATCH 52/66] fix decay time --- Cargo.lock | 1 + latency/Cargo.toml | 1 + latency/src/peak_ewma/mod.rs | 55 ++++++++++++++------------- latency/src/peak_ewma/rtt_estimate.rs | 3 +- web3_proxy/Cargo.toml | 3 ++ web3_proxy/src/rpcs/many.rs | 2 +- web3_proxy/src/rpcs/one.rs | 5 +-- web3_proxy/src/rpcs/request.rs | 4 +- 8 files changed, 40 insertions(+), 34 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 95aec00b..922bdae7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2941,6 +2941,7 @@ name = "latency" version = "0.1.0" dependencies = [ "ewma", + "flume", "log", "serde", "tokio", diff --git a/latency/Cargo.toml b/latency/Cargo.toml index eb51eba9..7cea2c16 100644 --- a/latency/Cargo.toml +++ b/latency/Cargo.toml @@ -7,6 +7,7 @@ edition = "2021" [dependencies] ewma = "0.1.1" +flume = "0.10.14" log = "0.4.17" serde = { version = "1.0.163", features = [] } tokio = { version = "1.28.1", features = ["full"] } diff --git a/latency/src/peak_ewma/mod.rs b/latency/src/peak_ewma/mod.rs index a795c383..0d4dd25b 100644 --- a/latency/src/peak_ewma/mod.rs +++ b/latency/src/peak_ewma/mod.rs @@ -2,9 +2,7 @@ mod rtt_estimate; use std::sync::Arc; -use log::{error, trace}; -use tokio::sync::mpsc; -use tokio::sync::mpsc::error::TrySendError; +use log::{error, log_enabled, trace}; use tokio::task::JoinHandle; use tokio::time::{Duration, Instant}; @@ -20,7 +18,7 @@ pub struct PeakEwmaLatency { /// Join handle for the latency calculation task pub join_handle: JoinHandle<()>, /// Send to update with each request duration - request_tx: mpsc::Sender, + request_tx: flume::Sender, /// Latency average and last update time rtt_estimate: Arc, /// Decay time @@ -32,9 +30,12 @@ impl PeakEwmaLatency { /// /// Returns a handle that can also be used to read the current /// average latency. - pub fn spawn(decay_ns: f64, buf_size: usize, start_latency: Duration) -> Self { + pub fn spawn(decay: Duration, buf_size: usize, start_latency: Duration) -> Self { + let decay_ns = decay.as_nanos() as f64; + debug_assert!(decay_ns > 0.0, "decay_ns must be positive"); - let (request_tx, request_rx) = mpsc::channel(buf_size); + + let (request_tx, request_rx) = flume::bounded(buf_size); let rtt_estimate = Arc::new(AtomicRttEstimate::new(start_latency)); let task = PeakEwmaLatencyTask { request_rx, @@ -56,15 +57,19 @@ impl PeakEwmaLatency { let mut estimate = self.rtt_estimate.load(); let now = Instant::now(); - assert!( - estimate.update_at <= now, - "update_at is {}ns in the future", - estimate.update_at.duration_since(now).as_nanos(), - ); - // Update the RTT estimate to account for decay since the last update. - // TODO: having an update here means we don't actually write from just one thread!! Thats how we get partially written stuff i think - estimate.update(0.0, self.decay_ns, now) + if estimate.update_at > now { + if log_enabled!(log::Level::Trace) { + trace!( + "update_at is {}ns in the future", + estimate.update_at.duration_since(now).as_nanos() + ); + } + estimate.rtt + } else { + // Update the RTT estimate to account for decay since the last update. + estimate.update(0.0, self.decay_ns, now) + } } /// Report latency from a single request @@ -73,15 +78,12 @@ impl PeakEwmaLatency { pub fn report(&self, duration: Duration) { match self.request_tx.try_send(duration) { Ok(()) => {} - Err(TrySendError::Full(_)) => { + Err(err) => { // We don't want to block if the channel is full, just // report the error - error!("Latency report channel full"); + error!("Latency report channel full. {}", err); // TODO: could we spawn a new tokio task to report tthis later? } - Err(TrySendError::Closed(_)) => { - unreachable!("Owner should keep channel open"); - } }; } } @@ -90,7 +92,7 @@ impl PeakEwmaLatency { #[derive(Debug)] struct PeakEwmaLatencyTask { /// Receive new request timings for update - request_rx: mpsc::Receiver, + request_rx: flume::Receiver, /// Current estimate and update time rtt_estimate: Arc, /// Last update time, used for decay calculation @@ -101,8 +103,8 @@ struct PeakEwmaLatencyTask { impl PeakEwmaLatencyTask { /// Run the loop for updating latency - async fn run(mut self) { - while let Some(rtt) = self.request_rx.recv().await { + async fn run(self) { + while let Ok(rtt) = self.request_rx.recv_async().await { self.update(rtt); } trace!("latency loop exited"); @@ -128,14 +130,15 @@ impl PeakEwmaLatencyTask { mod tests { use tokio::time::{self, Duration}; - use crate::util::nanos::NANOS_PER_MILLI; - /// The default RTT estimate decays, so that new nodes are considered if the /// default RTT is too high. #[tokio::test(start_paused = true)] async fn default_decay() { - let estimate = - super::PeakEwmaLatency::spawn(NANOS_PER_MILLI * 1_000.0, 8, Duration::from_millis(10)); + let estimate = super::PeakEwmaLatency::spawn( + Duration::from_millis(1_000), + 8, + Duration::from_millis(10), + ); let load = estimate.latency(); assert_eq!(load, Duration::from_millis(10)); diff --git a/latency/src/peak_ewma/rtt_estimate.rs b/latency/src/peak_ewma/rtt_estimate.rs index be56fe9c..1850d9e4 100644 --- a/latency/src/peak_ewma/rtt_estimate.rs +++ b/latency/src/peak_ewma/rtt_estimate.rs @@ -44,6 +44,7 @@ impl RttEstimate { ); Duration::from_nanos(next_estimate as u64) }; + self.rtt } @@ -101,7 +102,7 @@ impl AtomicRttEstimate { } /// Fetches the value, and applies a function to it that returns an - /// new rtt. Retrns the new RttEstimate with new update_at. + /// new rtt. Returns the new RttEstimate with new update_at. /// /// Automatically updates the update_at with Instant::now(). This /// method omits ordering arguments, defaulting to Relaxed since diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index c51e9c8e..90764578 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -91,3 +91,6 @@ tower-http = { version = "0.4.0", features = ["cors", "sensitive-headers"] } ulid = { version = "1.0.0", features = ["uuid", "serde"] } url = "2.3.1" uuid = "1.3.3" + +[dev-dependencies] +tokio = { version = "1.28.1", features = ["full", "test-util"] } diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index 74ef6776..b6f924de 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -1318,7 +1318,7 @@ mod tests { #[cfg(test)] fn new_peak_latency() -> PeakEwmaLatency { const NANOS_PER_MILLI: f64 = 1_000_000.0; - PeakEwmaLatency::spawn(1_000.0 * NANOS_PER_MILLI, 4, Duration::from_secs(1)) + PeakEwmaLatency::spawn(Duration::from_secs(1), 4, Duration::from_secs(1)) } #[tokio::test] diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index dd84fa4b..9015dd18 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -159,8 +159,7 @@ impl Web3Rpc { // TODO Should these defaults be in config let peak_latency = PeakEwmaLatency::spawn( // Decay over 15s - // TODO! This is wrong! needs to be as_nanos! - Duration::from_secs(15).as_millis() as f64, + Duration::from_secs(15), // Peak requests so far around 5k, we will use an order of magnitude // more to be safe. Should only use about 50mb RAM 50_000, @@ -1220,7 +1219,7 @@ impl Serialize for Web3Rpc { state.serialize_field( "peak_latency", - &self.peak_latency.as_ref().unwrap().latency(), + &self.peak_latency.as_ref().unwrap().latency().as_millis(), )?; state.serialize_field("peak_ewma", self.peak_ewma().as_ref())?; diff --git a/web3_proxy/src/rpcs/request.rs b/web3_proxy/src/rpcs/request.rs index 7a13335f..34110bf7 100644 --- a/web3_proxy/src/rpcs/request.rs +++ b/web3_proxy/src/rpcs/request.rs @@ -356,9 +356,7 @@ impl OpenRequestHandle { } } } else if let Some(peak_latency) = &self.rpc.peak_latency { - // trace!("updating peak_latency: {}", latency.as_secs_f64()); - // peak_latency.report(latency); - trace!("peak latency disabled for now"); + peak_latency.report(latency); } else { unreachable!("peak_latency not initialized"); } From 05f862f3e43cf89917e635e165f10c218e4f7f39 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Mon, 22 May 2023 15:50:00 -0700 Subject: [PATCH 53/66] cargo upgrade --- Cargo.lock | 10 +++++----- web3_proxy/Cargo.toml | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 922bdae7..51b231ce 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4279,13 +4279,13 @@ dependencies = [ [[package]] name = "regex" -version = "1.8.1" +version = "1.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af83e617f331cc6ae2da5443c602dfa5af81e517212d9d611a5b3ba1777b5370" +checksum = "d1a59b5d8e97dee33696bf13c5ba8ab85341c002922fba050069326b9c498974" dependencies = [ "aho-corasick", "memchr", - "regex-syntax 0.7.1", + "regex-syntax 0.7.2", ] [[package]] @@ -4305,9 +4305,9 @@ checksum = "f162c6dd7b008981e4d40210aca20b4bd0f9b60ca9271061b07f78537722f2e1" [[package]] name = "regex-syntax" -version = "0.7.1" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5996294f19bd3aae0453a862ad728f60e6600695733dd5df01da90c54363a3c" +checksum = "436b050e76ed2903236f032a59761c1eb99e1b0aead2c257922771dab1fc8c78" [[package]] name = "rend" diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index 90764578..52e517c7 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -71,7 +71,7 @@ parking_lot = { version = "0.12.1", features = ["arc_lock"] } prettytable = "*" proctitle = "0.1.1" rdkafka = { version = "0.31.0" } -regex = "1.8.1" +regex = "1.8.2" reqwest = { version = "0.11.18", default-features = false, features = ["json", "tokio-rustls"] } rmp-serde = "1.1.1" sentry = { version = "0.31.1", default-features = false, features = ["backtrace", "contexts", "panic", "anyhow", "reqwest", "rustls", "log", "sentry-log"] } From ddfd1dbff10004d665c3054f1530b0deb28a86b9 Mon Sep 17 00:00:00 2001 From: David Date: Mon, 22 May 2023 20:07:17 -0400 Subject: [PATCH 54/66] Added admin balance increase, and also test (#72) * added admin balance increase, and also test * slightly hacky, but does do an update expression * added note column to entities * removed redundant printlns * flattened items * removed user tier logic from admin increase balance completely. admin will have to manually increase the user tier --- .../src/admin_increase_balance_receipt.rs | 49 ++++++ entities/src/mod.rs | 1 + entities/src/prelude.rs | 1 + migration/src/lib.rs | 6 +- .../src/m20230514_114803_admin_add_credits.rs | 97 ++++++++++++ scripts/manual-tests/45-admin-add-balance.sh | 44 ++++++ web3_proxy/src/frontend/admin.rs | 146 +++++++++++++++++- web3_proxy/src/frontend/mod.rs | 4 + 8 files changed, 343 insertions(+), 5 deletions(-) create mode 100644 entities/src/admin_increase_balance_receipt.rs create mode 100644 migration/src/m20230514_114803_admin_add_credits.rs create mode 100644 scripts/manual-tests/45-admin-add-balance.sh diff --git a/entities/src/admin_increase_balance_receipt.rs b/entities/src/admin_increase_balance_receipt.rs new file mode 100644 index 00000000..d3d3cf27 --- /dev/null +++ b/entities/src/admin_increase_balance_receipt.rs @@ -0,0 +1,49 @@ +//! `SeaORM` Entity. Generated by sea-orm-codegen 0.10.6 + +use sea_orm::entity::prelude::*; + +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] +#[sea_orm(table_name = "admin_increase_balance_receipt")] +pub struct Model { + #[sea_orm(primary_key)] + pub id: i32, + #[sea_orm(column_type = "Decimal(Some((20, 10)))")] + pub amount: Decimal, + pub admin_id: u64, + pub deposit_to_user_id: u64, + pub note: String, +} + +#[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] +pub enum Relation { + #[sea_orm( + belongs_to = "super::admin::Entity", + from = "Column::AdminId", + to = "super::admin::Column::Id", + on_update = "NoAction", + on_delete = "NoAction" + )] + Admin, + #[sea_orm( + belongs_to = "super::user::Entity", + from = "Column::DepositToUserId", + to = "super::user::Column::Id", + on_update = "NoAction", + on_delete = "NoAction" + )] + User, +} + +impl Related for Entity { + fn to() -> RelationDef { + Relation::Admin.def() + } +} + +impl Related for Entity { + fn to() -> RelationDef { + Relation::User.def() + } +} + +impl ActiveModelBehavior for ActiveModel {} diff --git a/entities/src/mod.rs b/entities/src/mod.rs index 91a8a460..00510fae 100644 --- a/entities/src/mod.rs +++ b/entities/src/mod.rs @@ -3,6 +3,7 @@ pub mod prelude; pub mod admin; +pub mod admin_increase_balance_receipt; pub mod admin_trail; pub mod balance; pub mod increase_on_chain_balance_receipt; diff --git a/entities/src/prelude.rs b/entities/src/prelude.rs index 9d5f4cc0..20b764f3 100644 --- a/entities/src/prelude.rs +++ b/entities/src/prelude.rs @@ -1,6 +1,7 @@ //! `SeaORM` Entity. Generated by sea-orm-codegen 0.10.7 pub use super::admin::Entity as Admin; +pub use super::admin_increase_balance_receipt::Entity as AdminIncreaseBalanceReceipt; pub use super::admin_trail::Entity as AdminTrail; pub use super::balance::Entity as Balance; pub use super::increase_on_chain_balance_receipt::Entity as IncreaseOnChainBalanceReceipt; diff --git a/migration/src/lib.rs b/migration/src/lib.rs index ae9adaf7..91c45391 100644 --- a/migration/src/lib.rs +++ b/migration/src/lib.rs @@ -27,6 +27,7 @@ mod m20230412_171916_modify_secondary_user_add_primary_user; mod m20230422_172555_premium_downgrade_logic; mod m20230511_161214_remove_columns_statsv2_origin_and_method; mod m20230512_220213_allow_null_rpc_key_id_in_stats_v2; +mod m20230514_114803_admin_add_credits; pub struct Migrator; @@ -51,16 +52,17 @@ impl MigratorTrait for Migrator { Box::new(m20230125_204810_stats_v2::Migration), Box::new(m20230130_124740_read_only_login_logic::Migration), Box::new(m20230130_165144_prepare_admin_imitation_pre_login::Migration), - Box::new(m20230215_152254_admin_trail::Migration), - Box::new(m20230307_002623_migrate_rpc_accounting_to_rpc_accounting_v2::Migration), Box::new(m20230205_130035_create_balance::Migration), Box::new(m20230205_133755_create_referrals::Migration), Box::new(m20230214_134254_increase_balance_transactions::Migration), + Box::new(m20230215_152254_admin_trail::Migration), Box::new(m20230221_230953_track_spend::Migration), + Box::new(m20230307_002623_migrate_rpc_accounting_to_rpc_accounting_v2::Migration), Box::new(m20230412_171916_modify_secondary_user_add_primary_user::Migration), Box::new(m20230422_172555_premium_downgrade_logic::Migration), Box::new(m20230511_161214_remove_columns_statsv2_origin_and_method::Migration), Box::new(m20230512_220213_allow_null_rpc_key_id_in_stats_v2::Migration), + Box::new(m20230514_114803_admin_add_credits::Migration), ] } } diff --git a/migration/src/m20230514_114803_admin_add_credits.rs b/migration/src/m20230514_114803_admin_add_credits.rs new file mode 100644 index 00000000..0344d5d6 --- /dev/null +++ b/migration/src/m20230514_114803_admin_add_credits.rs @@ -0,0 +1,97 @@ +use sea_orm_migration::prelude::*; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .create_table( + Table::create() + .table(AdminIncreaseBalanceReceipt::Table) + .if_not_exists() + .col( + ColumnDef::new(AdminIncreaseBalanceReceipt::Id) + .integer() + .not_null() + .auto_increment() + .primary_key(), + ) + .col( + ColumnDef::new(AdminIncreaseBalanceReceipt::Amount) + .decimal_len(20, 10) + .not_null(), + ) + .col( + ColumnDef::new(AdminIncreaseBalanceReceipt::AdminId) + .big_unsigned() + .not_null(), + ) + .foreign_key( + ForeignKey::create() + .name("fk-admin_id") + .from( + AdminIncreaseBalanceReceipt::Table, + AdminIncreaseBalanceReceipt::AdminId, + ) + .to(Admin::Table, Admin::Id), + ) + .col( + ColumnDef::new(AdminIncreaseBalanceReceipt::DepositToUserId) + .big_unsigned() + .not_null(), + ) + .foreign_key( + ForeignKey::create() + .name("fk-admin_deposits_to_user_id") + .from( + AdminIncreaseBalanceReceipt::Table, + AdminIncreaseBalanceReceipt::DepositToUserId, + ) + .to(User::Table, User::Id), + ) + .col( + ColumnDef::new(AdminIncreaseBalanceReceipt::Note) + .string() + .not_null(), + ) + .to_owned(), + ) + .await + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .drop_table( + Table::drop() + .table(AdminIncreaseBalanceReceipt::Table) + .to_owned(), + ) + .await + } +} + +#[derive(Iden)] +enum Admin { + Table, + Id, +} + +/// Learn more at https://docs.rs/sea-query#iden +#[derive(Iden)] +enum User { + Table, + Id, +} + +/// Learn more at https://docs.rs/sea-query#iden +#[derive(Iden)] +enum AdminIncreaseBalanceReceipt { + Table, + Id, + Amount, + AdminId, + DepositToUserId, + Note, +} diff --git a/scripts/manual-tests/45-admin-add-balance.sh b/scripts/manual-tests/45-admin-add-balance.sh new file mode 100644 index 00000000..61b194f8 --- /dev/null +++ b/scripts/manual-tests/45-admin-add-balance.sh @@ -0,0 +1,44 @@ + +# Create / Login user1 +curl -X GET "http://127.0.0.1:8544/user/login/0xeb3e928a2e54be013ef8241d4c9eaf4dfae94d5a" +curl -X POST http://127.0.0.1:8544/user/login \ + -H 'Content-Type: application/json' \ + -d '{ + "address": "0xeb3e928a2e54be013ef8241d4c9eaf4dfae94d5a", + "msg": "0x6c6c616d616e6f6465732e636f6d2077616e747320796f7520746f207369676e20696e207769746820796f757220457468657265756d206163636f756e743a0a3078654233453932384132453534424530313345463832343164344339456146344466414539344435610a0af09fa699f09fa699f09fa699f09fa699f09fa6990a0a5552493a2068747470733a2f2f6c6c616d616e6f6465732e636f6d2f0a56657273696f6e3a20310a436861696e2049443a20310a4e6f6e63653a203031483044573642334a48355a4b384a4e3947504d594e4d4b370a4973737565642041743a20323032332d30352d31345431393a33353a35352e3736323632395a0a45787069726174696f6e2054696d653a20323032332d30352d31345431393a35353a35352e3736323632395a", + "sig": "f88b42d638246f8e51637c753052cab3a13b2a138faf3107c921ce2f0027d6506b9adcd3a7b72af830cdf50d20e6e9cb3f9f456dd1be47f6543990ea050909791c", + "version": "3", + "signer": "MEW" + }' + +# 01H0DW6VFCP365B9TXVQVVMHHY +# 01H0DVZNDJWQ7YG8RBHXQHJ301 + +# Make user1 an admin +cargo run change_admin_status 0xeB3E928A2E54BE013EF8241d4C9EaF4DfAE94D5a true + +# Create/Login user2 +curl -X GET "http://127.0.0.1:8544/user/login/0x762390ae7a3c4D987062a398C1eA8767029AB08E" + +curl -X POST http://127.0.0.1:8544/user/login \ + -H 'Content-Type: application/json' \ + -d '{ + "address": "0x762390ae7a3c4d987062a398c1ea8767029ab08e", + "msg": "0x6c6c616d616e6f6465732e636f6d2077616e747320796f7520746f207369676e20696e207769746820796f757220457468657265756d206163636f756e743a0a3078373632333930616537613363344439383730363261333938433165413837363730323941423038450a0af09fa699f09fa699f09fa699f09fa699f09fa6990a0a5552493a2068747470733a2f2f6c6c616d616e6f6465732e636f6d2f0a56657273696f6e3a20310a436861696e2049443a20310a4e6f6e63653a20303148304457384233304e534447594e484d33514d4a31434e530a4973737565642041743a20323032332d30352d31345431393a33373a30312e3238303338355a0a45787069726174696f6e2054696d653a20323032332d30352d31345431393a35373a30312e3238303338355a", + "sig": "c545235557b7952a789dffa2af153af5cf663dcc05449bcc4b651b04cda57de05bcef55c0f5cbf6aa2432369582eb6a40927d14ad0a2d15f48fa45f32fbf273f1c", + "version": "3", + "signer": "MEW" + }' + +# 01H0DWPXRQA7XX2VFSNR02CG1N +# 01H0DWPXQQ951Y3R90QMF6MYGE + +curl \ +-H "Authorization: Bearer 01H0DWPXRQA7XX2VFSNR02CG1N" \ +-X GET "127.0.0.1:8544/user/balance" + + +# Admin add balance +curl \ +-H "Authorization: Bearer 01H0DW6VFCP365B9TXVQVVMHHY" \ +-X GET "127.0.0.1:8544/admin/increase_balance?user_address=0x762390ae7a3c4D987062a398C1eA8767029AB08E&amount=100.0" diff --git a/web3_proxy/src/frontend/admin.rs b/web3_proxy/src/frontend/admin.rs index c3ddf453..e2527b93 100644 --- a/web3_proxy/src/frontend/admin.rs +++ b/web3_proxy/src/frontend/admin.rs @@ -5,8 +5,11 @@ use super::errors::Web3ProxyResponse; use crate::admin_queries::query_admin_modify_usertier; use crate::app::Web3ProxyApp; use crate::frontend::errors::{Web3ProxyError, Web3ProxyErrorContext}; +use crate::http_params::get_user_id_from_params; use crate::user_token::UserBearerToken; use crate::PostLogin; +use anyhow::Context; +use axum::body::HttpBody; use axum::{ extract::{Path, Query}, headers::{authorization::Bearer, Authorization}, @@ -16,15 +19,20 @@ use axum::{ use axum_client_ip::InsecureClientIp; use axum_macros::debug_handler; use chrono::{TimeZone, Utc}; -use entities::{admin_trail, login, pending_login, rpc_key, user}; +use entities::{ + admin, admin_increase_balance_receipt, admin_trail, balance, login, pending_login, rpc_key, + user, user_tier, +}; use ethers::{prelude::Address, types::Bytes}; use hashbrown::HashMap; use http::StatusCode; use log::{debug, info, warn}; -use migration::sea_orm::prelude::Uuid; +use migration::sea_orm::prelude::{Decimal, Uuid}; use migration::sea_orm::{ - self, ActiveModelTrait, ColumnTrait, EntityTrait, IntoActiveModel, QueryFilter, + self, ActiveModelTrait, ActiveValue, ColumnTrait, EntityTrait, IntoActiveModel, QueryFilter, + TransactionTrait, Update, }; +use migration::{ConnectionTrait, Expr, OnConflict}; use serde_json::json; use siwe::{Message, VerificationOpts}; use std::ops::Add; @@ -33,6 +41,138 @@ use std::sync::Arc; use time::{Duration, OffsetDateTime}; use ulid::Ulid; +/// `GET /admin/increase_balance` -- As an admin, modify a user's user-tier +/// +/// - user_address that is to credited balance +/// - user_role_tier that is supposed to be adapted +#[debug_handler] +pub async fn admin_increase_balance( + Extension(app): Extension>, + TypedHeader(Authorization(bearer)): TypedHeader>, + Query(params): Query>, +) -> Web3ProxyResponse { + let (caller, _) = app.bearer_is_authorized(bearer).await?; + let caller_id = caller.id; + + // Establish connections + let db_conn = app + .db_conn() + .context("query_admin_modify_user needs a db")?; + + // Check if the caller is an admin (if not, return early) + let admin_entry: admin::Model = admin::Entity::find() + .filter(admin::Column::UserId.eq(caller_id)) + .one(&db_conn) + .await? + .ok_or(Web3ProxyError::AccessDenied)?; + + // Get the user from params + let user_address: Address = params + .get("user_address") + .ok_or_else(|| { + Web3ProxyError::BadRequest("Unable to find user_address key in request".to_string()) + })? + .parse::
() + .map_err(|_| { + Web3ProxyError::BadRequest("Unable to parse user_address as an Address".to_string()) + })?; + let user_address_bytes: Vec = user_address.clone().to_fixed_bytes().into(); + let note: String = params + .get("note") + .ok_or_else(|| { + Web3ProxyError::BadRequest("Unable to find 'note' key in request".to_string()) + })? + .parse::() + .map_err(|_| { + Web3ProxyError::BadRequest("Unable to parse 'note' as a String".to_string()) + })?; + // Get the amount from params + // Decimal::from_str + let amount: Decimal = params + .get("amount") + .ok_or_else(|| { + Web3ProxyError::BadRequest("Unable to get the amount key from the request".to_string()) + }) + .map(|x| Decimal::from_str(x))? + .or_else(|err| { + Err(Web3ProxyError::BadRequest(format!( + "Unable to parse amount from the request {:?}", + err + ))) + })?; + + let user_entry: user::Model = user::Entity::find() + .filter(user::Column::Address.eq(user_address_bytes.clone())) + .one(&db_conn) + .await? + .ok_or(Web3ProxyError::BadRequest( + "No user with this id found".to_string(), + ))?; + + let increase_balance_receipt = admin_increase_balance_receipt::ActiveModel { + amount: sea_orm::Set(amount), + admin_id: sea_orm::Set(admin_entry.id), + deposit_to_user_id: sea_orm::Set(user_entry.id), + note: sea_orm::Set(note), + ..Default::default() + }; + increase_balance_receipt.save(&db_conn).await?; + + let mut out = HashMap::new(); + out.insert( + "user", + serde_json::Value::String(format!("{:?}", user_address)), + ); + out.insert("amount", serde_json::Value::String(amount.to_string())); + + // Get the balance row + let balance_entry: balance::Model = balance::Entity::find() + .filter(balance::Column::UserId.eq(user_entry.id)) + .one(&db_conn) + .await? + .context("User does not have a balance row")?; + + // Finally make the user premium if balance is above 10$ + let premium_user_tier = user_tier::Entity::find() + .filter(user_tier::Column::Title.eq("Premium")) + .one(&db_conn) + .await? + .context("Premium tier was not found!")?; + + let balance_entry = balance_entry.into_active_model(); + balance::Entity::insert(balance_entry) + .on_conflict( + OnConflict::new() + .values([ + // ( + // balance::Column::Id, + // Expr::col(balance::Column::Id).add(self.frontend_requests), + // ), + ( + balance::Column::AvailableBalance, + Expr::col(balance::Column::AvailableBalance).add(amount), + ), + // ( + // balance::Column::Used, + // Expr::col(balance::Column::UsedBalance).add(self.backend_retries), + // ), + // ( + // balance::Column::UserId, + // Expr::col(balance::Column::UserId).add(self.no_servers), + // ), + ]) + .to_owned(), + ) + .exec(&db_conn) + .await?; + // TODO: Downgrade otherwise, right now not functioning properly + + // Then read and save in one transaction + let response = (StatusCode::OK, Json(out)).into_response(); + + Ok(response) +} + /// `GET /admin/modify_role` -- As an admin, modify a user's user-tier /// /// - user_address that is to be modified diff --git a/web3_proxy/src/frontend/mod.rs b/web3_proxy/src/frontend/mod.rs index e1496960..107360c4 100644 --- a/web3_proxy/src/frontend/mod.rs +++ b/web3_proxy/src/frontend/mod.rs @@ -199,6 +199,10 @@ pub async fn serve( "/user/logout", post(users::authentication::user_logout_post), ) + .route( + "/admin/increase_balance", + get(admin::admin_increase_balance), + ) .route("/admin/modify_role", get(admin::admin_change_user_roles)) .route( "/admin/imitate-login/:admin_address/:user_address", From 27333a0028f04df6743fa5f12ea16a34d980b233 Mon Sep 17 00:00:00 2001 From: David Date: Mon, 22 May 2023 20:08:35 -0400 Subject: [PATCH 55/66] David/devel/debug graph (#82) * now should also group by archive_needed (turns out this was wanted) * influxdb query also groups by archive_needed now * updated error_response * updated error_response --- scripts/generate-requests-and-stats.sh | 2 +- scripts/manual-tests/42-simple-balance.sh | 14 +++---- web3_proxy/src/stats/influxdb_queries.rs | 47 +++++++++++------------ 3 files changed, 30 insertions(+), 33 deletions(-) diff --git a/scripts/generate-requests-and-stats.sh b/scripts/generate-requests-and-stats.sh index ecae4466..a05e55e7 100644 --- a/scripts/generate-requests-and-stats.sh +++ b/scripts/generate-requests-and-stats.sh @@ -5,4 +5,4 @@ # https://github.com/INFURA/versus # ./ethspam | ./versus --stop-after 100 "http://localhost:8544/" # Pipe into the endpoint ..., add a bearer token and all that -./ethspam http://127.0.0.1:8544 | ./versus --stop-after 100 http://localhost:8544 +./ethspam http://127.0.0.1:8544/rpc/01H0ZZJDNNEW49FRFS4D9SPR8B | ./versus --concurrency=4 --stop-after 100 http://localhost:8544/rpc/01H0ZZJDNNEW49FRFS4D9SPR8B diff --git a/scripts/manual-tests/42-simple-balance.sh b/scripts/manual-tests/42-simple-balance.sh index ce6e32da..724d4809 100644 --- a/scripts/manual-tests/42-simple-balance.sh +++ b/scripts/manual-tests/42-simple-balance.sh @@ -24,14 +24,14 @@ curl -X POST http://127.0.0.1:8544/user/login \ -H 'Content-Type: application/json' \ -d '{ "address": "0xeb3e928a2e54be013ef8241d4c9eaf4dfae94d5a", - "msg": "0x6c6c616d616e6f6465732e636f6d2077616e747320796f7520746f207369676e20696e207769746820796f757220457468657265756d206163636f756e743a0a3078654233453932384132453534424530313345463832343164344339456146344466414539344435610a0af09fa699f09fa699f09fa699f09fa699f09fa6990a0a5552493a2068747470733a2f2f6c6c616d616e6f6465732e636f6d2f0a56657273696f6e3a20310a436861696e2049443a20310a4e6f6e63653a203031475a4b384b4847305259474737514e5132475037464444470a4973737565642041743a20323032332d30352d30345431313a33333a32312e3533363734355a0a45787069726174696f6e2054696d653a20323032332d30352d30345431313a35333a32312e3533363734355a", - "sig": "cebd9effff15f4517e53522dbe91798d59dc0df0299faaec25d3f6443fa121f847e4311d5ca7386e75b87d6d45df92b8ced58c822117519c666ab1a6b2fc7bd21b", + "msg": "0x6c6c616d616e6f6465732e636f6d2077616e747320796f7520746f207369676e20696e207769746820796f757220457468657265756d206163636f756e743a0a3078654233453932384132453534424530313345463832343164344339456146344466414539344435610a0af09fa699f09fa699f09fa699f09fa699f09fa6990a0a5552493a2068747470733a2f2f6c6c616d616e6f6465732e636f6d2f0a56657273696f6e3a20310a436861696e2049443a20310a4e6f6e63653a20303148305a5a48434356324b32324738544850535758485131480a4973737565642041743a20323032332d30352d32315432303a32303a34332e3033353539315a0a45787069726174696f6e2054696d653a20323032332d30352d32315432303a34303a34332e3033353539315a", + "sig": "7591251840bf75d2ab7c895bc566a49d2f4c3ad6bb14d7256258a59e52055fc94c11f8f3836f5311b52fc18aca40867cd85802636645e1d757494800631cad381c", "version": "3", "signer": "MEW" }' -# bearer token is: 01GZK8MHHGQWK4VPGF97HS91MB -# scret key is: 01GZK65YNV0P0WN2SCXYTW3R9S +# bearer token is: 01H0ZZJDQ2F02MAXZR5K1X5NCP +# scret key is: 01H0ZZJDNNEW49FRFS4D9SPR8B # 01GZH2PS89EJJY6V8JFCVTQ4BX # 01GZH2PS7CTHA3TAZ4HXCTX6KQ @@ -42,7 +42,7 @@ curl -X POST http://127.0.0.1:8544/user/login \ # Check the balance of the user # Balance seems to be returning properly (0, in this test case) curl \ --H "Authorization: Bearer 01GZK8MHHGQWK4VPGF97HS91MB" \ +-H "Authorization: Bearer 01H0ZZJDQ2F02MAXZR5K1X5NCP" \ -X GET "127.0.0.1:8544/user/balance" @@ -73,10 +73,10 @@ curl \ ## Check if calling an RPC endpoint logs the stats ## This one does already even it seems -for i in {1..100} +for i in {1..300} do curl \ - -X POST "127.0.0.1:8544/rpc/01GZK65YNV0P0WN2SCXYTW3R9S" \ + -X POST "127.0.0.1:8544/rpc/01H0ZZJDNNEW49FRFS4D9SPR8B" \ -H "Content-Type: application/json" \ --data '{"method":"eth_blockNumber","params":[],"id":1,"jsonrpc":"2.0"}' done diff --git a/web3_proxy/src/stats/influxdb_queries.rs b/web3_proxy/src/stats/influxdb_queries.rs index 368df1f1..69d1110c 100644 --- a/web3_proxy/src/stats/influxdb_queries.rs +++ b/web3_proxy/src/stats/influxdb_queries.rs @@ -73,7 +73,9 @@ pub async fn query_user_stats<'a>( let mut join_candidates: Vec = vec![ "_time".to_string(), "_measurement".to_string(), + "archive_needed".to_string(), "chain_id".to_string(), + "error_response".to_string(), ]; // Include a hashmap to go from rpc_secret_key_id to the rpc_secret_key @@ -184,35 +186,18 @@ pub async fn query_user_stats<'a>( {filter_chain_id} {drop_method} - // cumsum = base base |> aggregateWindow(every: {query_window_seconds}s, fn: sum, createEmpty: false) |> pivot(rowKey: ["_time"], columnKey: ["_field"], valueColumn: "_value") |> drop(columns: ["balance"]) - |> map(fn: (r) => ({{ r with "archive_needed": if r.archive_needed == "true" then r.frontend_requests else 0}})) - |> map(fn: (r) => ({{ r with "error_response": if r.error_response == "true" then r.frontend_requests else 0}})) - |> group(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"]) + |> group(columns: ["_time", "_measurement", "archive_needed", "chain_id", "error_response", "method", "rpc_secret_key_id"]) |> sort(columns: ["frontend_requests"]) |> map(fn:(r) => ({{ r with "sum_credits_used": float(v: r["sum_credits_used"]) }})) - |> cumulativeSum(columns: ["archive_needed", "error_response", "backend_requests", "cache_hits", "cache_misses", "frontend_requests", "sum_credits_used", "sum_request_bytes", "sum_response_bytes", "sum_response_millis"]) + |> cumulativeSum(columns: ["backend_requests", "cache_hits", "cache_misses", "frontend_requests", "sum_credits_used", "sum_request_bytes", "sum_response_bytes", "sum_response_millis"]) |> sort(columns: ["frontend_requests"], desc: true) |> limit(n: 1) |> group() - |> sort(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"], desc: true) - - // balance = base - // |> toFloat() - // |> aggregateWindow(every: {query_window_seconds}s, fn: mean, createEmpty: false) - // |> pivot(rowKey: ["_time"], columnKey: ["_field"], valueColumn: "_value") - // |> group(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"]) - // |> mean(column: "balance") - // |> group() - // |> sort(columns: ["_time", "_measurement", "chain_id", "method", "rpc_secret_key_id"], desc: true) - - // join( - // tables: {{cumsum, balance}}, - // on: {join_candidates} - // ) + |> sort(columns: ["_time", "_measurement", "archive_needed", "chain_id", "error_response", "method", "rpc_secret_key_id"], desc: true) "#); info!("Raw query to db is: {:?}", query); @@ -438,22 +423,34 @@ pub async fn query_user_stats<'a>( } } else if key == "archive_needed" { match value { - influxdb2_structmap::value::Value::Long(inner) => { + influxdb2_structmap::value::Value::String(inner) => { out.insert( "archive_needed".to_owned(), - serde_json::Value::Number(inner.into()), + if inner == "true" { + serde_json::Value::Bool(true) + } else if inner == "false" { + serde_json::Value::Bool(false) + } else { + serde_json::Value::String("error".to_owned()) + }, ); } _ => { - error!("archive_needed should always be a Long!"); + error!("archive_needed should always be a String!"); } } } else if key == "error_response" { match value { - influxdb2_structmap::value::Value::Long(inner) => { + influxdb2_structmap::value::Value::String(inner) => { out.insert( "error_response".to_owned(), - serde_json::Value::Number(inner.into()), + if inner == "true" { + serde_json::Value::Bool(true) + } else if inner == "false" { + serde_json::Value::Bool(false) + } else { + serde_json::Value::String("error".to_owned()) + }, ); } _ => { From 78f247fc6cad8008694465f0ebc96fadb0e9f050 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 23 May 2023 14:40:34 -0700 Subject: [PATCH 56/66] split http and ws provider --- Cargo.lock | 1 + web3_proxy/Cargo.toml | 1 + web3_proxy/src/app/mod.rs | 38 +- web3_proxy/src/config.rs | 32 +- web3_proxy/src/frontend/authorization.rs | 73 +-- web3_proxy/src/frontend/errors.rs | 67 ++- web3_proxy/src/frontend/mod.rs | 4 +- web3_proxy/src/frontend/status.rs | 6 +- web3_proxy/src/prometheus.rs | 3 +- web3_proxy/src/rpcs/blockchain.rs | 8 +- web3_proxy/src/rpcs/consensus.rs | 2 +- web3_proxy/src/rpcs/many.rs | 118 +--- web3_proxy/src/rpcs/one.rs | 669 ++++++++--------------- web3_proxy/src/rpcs/provider.rs | 43 +- web3_proxy/src/rpcs/request.rs | 5 +- web3_proxy/src/rpcs/transactions.rs | 4 +- web3_proxy/src/stats/mod.rs | 23 +- web3_proxy/src/stats/stat_buffer.rs | 8 +- 18 files changed, 444 insertions(+), 661 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 51b231ce..7a79b0c2 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6533,6 +6533,7 @@ dependencies = [ "hex_fmt", "hostname", "http", + "hyper", "influxdb2", "influxdb2-structmap", "ipnet", diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index 52e517c7..17072f1c 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -55,6 +55,7 @@ hdrhistogram = "7.5.2" hex_fmt = "0.3.0" hostname = "0.3.1" http = "0.2.9" +hyper = { version = "0.14.26", features = ["full"] } influxdb2 = { git = "https://github.com/llamanodes/influxdb2", features = ["rustls"] } influxdb2-structmap = { git = "https://github.com/llamanodes/influxdb2/"} ipnet = "2.7.2" diff --git a/web3_proxy/src/app/mod.rs b/web3_proxy/src/app/mod.rs index bdc70c90..42003674 100644 --- a/web3_proxy/src/app/mod.rs +++ b/web3_proxy/src/app/mod.rs @@ -74,7 +74,7 @@ pub static APP_USER_AGENT: &str = concat!( // aggregate across 1 week pub const BILLING_PERIOD_SECONDS: i64 = 60 * 60 * 24 * 7; -pub type AnyhowJoinHandle = JoinHandle>; +pub type Web3ProxyJoinHandle = JoinHandle>; /// TODO: move this #[derive(Clone, Debug, Default, From)] @@ -176,7 +176,7 @@ pub struct Web3ProxyApp { // TODO: should the key be our RpcSecretKey class instead of Ulid? pub rpc_secret_key_cache: RpcSecretKeyCache, /// concurrent/parallel RPC request limits for authenticated users - pub rpc_key_semaphores: Cache>, + pub user_semaphores: Cache>, /// concurrent/parallel request limits for anonymous users pub ip_semaphores: Cache>, /// concurrent/parallel application request limits for authenticated users @@ -188,7 +188,7 @@ pub struct Web3ProxyApp { /// flatten a JoinError into an anyhow error /// Useful when joining multiple futures. -pub async fn flatten_handle(handle: AnyhowJoinHandle) -> anyhow::Result { +pub async fn flatten_handle(handle: Web3ProxyJoinHandle) -> Web3ProxyResult { match handle.await { Ok(Ok(result)) => Ok(result), Ok(Err(err)) => Err(err), @@ -198,8 +198,8 @@ pub async fn flatten_handle(handle: AnyhowJoinHandle) -> anyhow::Result /// return the first error, or Ok if everything worked pub async fn flatten_handles( - mut handles: FuturesUnordered>, -) -> anyhow::Result<()> { + mut handles: FuturesUnordered>, +) -> Web3ProxyResult<()> { while let Some(x) = handles.next().await { match x { Err(e) => return Err(e.into()), @@ -315,9 +315,9 @@ pub struct Web3ProxyAppSpawn { /// the app. probably clone this to use in other groups of handles pub app: Arc, /// handles for the balanced and private rpcs - pub app_handles: FuturesUnordered>, + pub app_handles: FuturesUnordered>, /// these are important and must be allowed to finish - pub background_handles: FuturesUnordered>, + pub background_handles: FuturesUnordered>, /// config changes are sent here pub new_top_config_sender: watch::Sender, /// watch this to know when to start the app @@ -359,10 +359,12 @@ impl Web3ProxyApp { } // these futures are key parts of the app. if they stop running, the app has encountered an irrecoverable error - let app_handles = FuturesUnordered::new(); + // TODO: this is a small enough group, that a vec with try_join_all is probably fine + let app_handles: FuturesUnordered> = FuturesUnordered::new(); // we must wait for these to end on their own (and they need to subscribe to shutdown_sender) - let important_background_handles = FuturesUnordered::new(); + let important_background_handles: FuturesUnordered> = + FuturesUnordered::new(); // connect to the database and make sure the latest migrations have run let mut db_conn = None::; @@ -624,12 +626,10 @@ impl Web3ProxyApp { // TODO: what should tti be for semaphores? let bearer_token_semaphores = Cache::new(max_users); let ip_semaphores = Cache::new(max_users); - let registered_user_semaphores = Cache::new(max_users); + let user_semaphores = Cache::new(max_users); let (balanced_rpcs, balanced_handle, consensus_connections_watcher) = Web3Rpcs::spawn( - top_config.app.chain_id, db_conn.clone(), - http_client.clone(), top_config.app.max_block_age, top_config.app.max_block_lag, top_config.app.min_synced_rpcs, @@ -654,9 +654,7 @@ impl Web3ProxyApp { // TODO: Merge // let (private_rpcs, private_rpcs_handle) = Web3Rpcs::spawn( let (private_rpcs, private_handle, _) = Web3Rpcs::spawn( - top_config.app.chain_id, db_conn.clone(), - http_client.clone(), // private rpcs don't get subscriptions, so no need for max_block_age or max_block_lag None, None, @@ -688,9 +686,7 @@ impl Web3ProxyApp { } else { // TODO: do something with the spawn handle let (bundler_4337_rpcs, bundler_4337_rpcs_handle, _) = Web3Rpcs::spawn( - top_config.app.chain_id, db_conn.clone(), - http_client.clone(), // bundler_4337_rpcs don't get subscriptions, so no need for max_block_age or max_block_lag None, None, @@ -735,7 +731,7 @@ impl Web3ProxyApp { rpc_secret_key_cache, bearer_token_semaphores, ip_semaphores, - rpc_key_semaphores: registered_user_semaphores, + user_semaphores, stat_sender, }; @@ -752,9 +748,9 @@ impl Web3ProxyApp { loop { let new_top_config = new_top_config_receiver.borrow_and_update().to_owned(); - app.apply_top_config(new_top_config) - .await - .context("failed applying new top_config")?; + if let Err(err) = app.apply_top_config(new_top_config).await { + error!("unable to apply config! {:?}", err); + }; new_top_config_receiver .changed() @@ -790,7 +786,7 @@ impl Web3ProxyApp { .into()) } - pub async fn apply_top_config(&self, new_top_config: TopConfig) -> anyhow::Result<()> { + pub async fn apply_top_config(&self, new_top_config: TopConfig) -> Web3ProxyResult<()> { // TODO: also update self.config from new_top_config.app // connect to the backends diff --git a/web3_proxy/src/config.rs b/web3_proxy/src/config.rs index fc156631..df3ad199 100644 --- a/web3_proxy/src/config.rs +++ b/web3_proxy/src/config.rs @@ -1,4 +1,4 @@ -use crate::app::AnyhowJoinHandle; +use crate::app::Web3ProxyJoinHandle; use crate::rpcs::blockchain::{BlocksByHashCache, Web3ProxyBlock}; use crate::rpcs::one::Web3Rpc; use argh::FromArgs; @@ -9,7 +9,7 @@ use log::warn; use migration::sea_orm::DatabaseConnection; use serde::Deserialize; use std::sync::Arc; -use tokio::sync::broadcast; +use std::time::Duration; pub type BlockAndRpc = (Option, Arc); pub type TxHashAndRpc = (TxHash, Arc); @@ -283,23 +283,45 @@ impl Web3RpcConfig { redis_pool: Option, chain_id: u64, http_client: Option, - http_interval_sender: Option>>, blocks_by_hash_cache: BlocksByHashCache, block_sender: Option>, tx_id_sender: Option>, - ) -> anyhow::Result<(Arc, AnyhowJoinHandle<()>)> { + ) -> anyhow::Result<(Arc, Web3ProxyJoinHandle<()>)> { if !self.extra.is_empty() { warn!("unknown Web3RpcConfig fields!: {:?}", self.extra.keys()); } + // TODO: get this from config? a helper function? where does this belong? + let block_interval = match chain_id { + // ethereum + 1 => Duration::from_secs(12), + // ethereum-goerli + 5 => Duration::from_secs(12), + // polygon + 137 => Duration::from_secs(2), + // fantom + 250 => Duration::from_secs(1), + // arbitrum + 42161 => Duration::from_millis(500), + // anything else + _ => { + let default = 10; + warn!( + "unexpected chain_id ({}). polling every {} seconds", + chain_id, default + ); + Duration::from_secs(default) + } + }; + Web3Rpc::spawn( self, name, chain_id, db_conn, http_client, - http_interval_sender, redis_pool, + block_interval, blocks_by_hash_cache, block_sender, tx_id_sender, diff --git a/web3_proxy/src/frontend/authorization.rs b/web3_proxy/src/frontend/authorization.rs index 51ad4834..dcc3b614 100644 --- a/web3_proxy/src/frontend/authorization.rs +++ b/web3_proxy/src/frontend/authorization.rs @@ -30,7 +30,7 @@ use redis_rate_limiter::redis::AsyncCommands; use redis_rate_limiter::RedisRateLimitResult; use std::convert::Infallible; use std::fmt::Display; -use std::hash::Hash; +use std::hash::{Hash, Hasher}; use std::mem; use std::sync::atomic::{self, AtomicBool, AtomicI64, AtomicU64, AtomicUsize}; use std::time::Duration; @@ -42,22 +42,12 @@ use ulid::Ulid; use uuid::Uuid; /// This lets us use UUID and ULID while we transition to only ULIDs -/// TODO: include the key's description. #[derive(Copy, Clone, Debug, Eq, PartialEq, serde::Deserialize, serde::Serialize)] pub enum RpcSecretKey { Ulid(Ulid), Uuid(Uuid), } -impl Hash for RpcSecretKey { - fn hash(&self, state: &mut H) { - match self { - Self::Ulid(x) => state.write_u128(x.0), - Self::Uuid(x) => state.write_u128(x.as_u128()), - } - } -} - /// TODO: should this have IpAddr and Origin or AuthorizationChecks? #[derive(Debug)] pub enum RateLimitResult { @@ -99,6 +89,17 @@ pub struct KafkaDebugLogger { num_responses: AtomicUsize, } +impl Hash for RpcSecretKey { + fn hash(&self, state: &mut H) { + let x = match self { + Self::Ulid(x) => x.0, + Self::Uuid(x) => x.as_u128(), + }; + + x.hash(state); + } +} + impl fmt::Debug for KafkaDebugLogger { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("KafkaDebugLogger") @@ -883,17 +884,13 @@ impl Web3ProxyApp { if let Some(max_concurrent_requests) = self.config.public_max_concurrent_requests { let semaphore = self .ip_semaphores - .get_or_insert_async::(ip, async move { + .get_or_insert_async::(ip, async move { // TODO: set max_concurrent_requests dynamically based on load? let s = Semaphore::new(max_concurrent_requests); Ok(Arc::new(s)) }) - .await?; - - // if semaphore.available_permits() == 0 { - // // TODO: concurrent limit hit! emit a stat? less important for anon users - // // TODO: there is probably a race here - // } + .await + .expect("infallible"); let semaphore_permit = semaphore.acquire_owned().await?; @@ -903,8 +900,8 @@ impl Web3ProxyApp { } } - /// Limit the number of concurrent requests from the given rpc key. - pub async fn registered_user_semaphore( + /// Limit the number of concurrent requests for a given user across all of their keys + pub async fn user_semaphore( &self, authorization_checks: &AuthorizationChecks, ) -> Web3ProxyResult> { @@ -915,25 +912,19 @@ impl Web3ProxyApp { .or(Err(Web3ProxyError::UserIdZero))?; let semaphore = self - .rpc_key_semaphores - .get_or_insert_async(&user_id, async move { + .user_semaphores + .get_or_insert_async::(&user_id, async move { let s = Semaphore::new(max_concurrent_requests as usize); - // trace!("new semaphore for user_id {}", user_id); - Ok::<_, Infallible>(Arc::new(s)) + Ok(Arc::new(s)) }) .await - .unwrap(); - - // if semaphore.available_permits() == 0 { - // // TODO: concurrent limit hit! emit a stat? this has a race condition though. - // // TODO: maybe have a stat on how long we wait to acquire the semaphore instead? - // } + .expect("infallible"); let semaphore_permit = semaphore.acquire_owned().await?; Ok(Some(semaphore_permit)) } else { - // unlimited requests allowed + // unlimited concurrency Ok(None) } } @@ -955,7 +946,7 @@ impl Web3ProxyApp { Ok(Arc::new(s)) }) .await - .unwrap(); + .expect("infallible"); let semaphore_permit = semaphore.acquire_owned().await?; @@ -1043,7 +1034,7 @@ impl Web3ProxyApp { // they do check origin because we can override rate limits for some origins let authorization = Authorization::external( allowed_origin_requests_per_period, - self.db_conn.clone(), + self.db_conn(), ip, origin, proxy_mode, @@ -1098,8 +1089,7 @@ impl Web3ProxyApp { proxy_mode: ProxyMode, rpc_secret_key: RpcSecretKey, ) -> Web3ProxyResult { - let authorization_checks: Result<_, Web3ProxyError> = self - .rpc_secret_key_cache + self.rpc_secret_key_cache .get_or_insert_async(&rpc_secret_key, async move { // trace!(?rpc_secret_key, "user cache miss"); @@ -1119,7 +1109,6 @@ impl Web3ProxyApp { Some(rpc_key_model) => { // TODO: move these splits into helper functions // TODO: can we have sea orm handle this for us? - // TODO: don't expect. return an application error let user_model = user::Entity::find_by_id(rpc_key_model.user_id) .one(db_replica.conn()) .await? @@ -1129,8 +1118,8 @@ impl Web3ProxyApp { .filter(balance::Column::UserId.eq(user_model.id)) .one(db_replica.conn()) .await? - .map(|x| x.available_balance) - .unwrap_or_default(); + .expect("related balance") + .available_balance; let user_tier_model = user_tier::Entity::find_by_id(user_model.user_tier_id) @@ -1220,9 +1209,7 @@ impl Web3ProxyApp { None => Ok(AuthorizationChecks::default()), } }) - .await; - - authorization_checks + .await } /// Authorized the ip/origin/referer/useragent and rate limit and concurrency @@ -1246,9 +1233,7 @@ impl Web3ProxyApp { // only allow this rpc_key to run a limited amount of concurrent requests // TODO: rate limit should be BEFORE the semaphore! - let semaphore = self - .registered_user_semaphore(&authorization_checks) - .await?; + let semaphore = self.user_semaphore(&authorization_checks).await?; let authorization = Authorization::try_new( authorization_checks, diff --git a/web3_proxy/src/frontend/errors.rs b/web3_proxy/src/frontend/errors.rs index 9126526e..c970c849 100644 --- a/web3_proxy/src/frontend/errors.rs +++ b/web3_proxy/src/frontend/errors.rs @@ -47,14 +47,15 @@ pub enum Web3ProxyError { Database(DbErr), #[display(fmt = "{:#?}, {:#?}", _0, _1)] EipVerificationFailed(Box, Box), - EthersHttpClientError(ethers::prelude::HttpClientError), - EthersProviderError(ethers::prelude::ProviderError), - EthersWsClientError(ethers::prelude::WsClientError), - FlumeRecvError(flume::RecvError), + EthersHttpClient(ethers::prelude::HttpClientError), + EthersProvider(ethers::prelude::ProviderError), + EthersWsClient(ethers::prelude::WsClientError), + FlumeRecv(flume::RecvError), GasEstimateNotU256, Headers(headers::Error), HeaderToString(ToStrError), - InfluxDb2RequestError(influxdb2::RequestError), + Hyper(hyper::Error), + InfluxDb2Request(influxdb2::RequestError), #[display(fmt = "{} > {}", min, max)] #[from(ignore)] InvalidBlockBounds { @@ -64,6 +65,7 @@ pub enum Web3ProxyError { InvalidHeaderValue(InvalidHeaderValue), InvalidEip, InvalidInviteCode, + Io(std::io::Error), UnknownReferralCode, InvalidReferer, InvalidSignatureLength, @@ -88,6 +90,12 @@ pub enum Web3ProxyError { num_known: usize, min_head_rpcs: usize, }, + #[display(fmt = "{}/{}", available, needed)] + #[from(ignore)] + NotEnoughSoftLimit { + available: u32, + needed: u32, + }, NotFound, NotImplemented, OriginRequired, @@ -136,6 +144,7 @@ pub enum Web3ProxyError { impl Web3ProxyError { pub fn into_response_parts(self) -> (StatusCode, JsonRpcResponseData) { + // TODO: include a unique request id in the data let (code, err): (StatusCode, JsonRpcErrorData) = match self { Self::AccessDenied => { // TODO: attach something to this trace. probably don't include much in the message though. don't want to leak creds by accident @@ -223,7 +232,7 @@ impl Web3ProxyError { }, ) } - Self::EthersHttpClientError(err) => { + Self::EthersHttpClient(err) => { warn!("EthersHttpClientError err={:?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, @@ -234,7 +243,7 @@ impl Web3ProxyError { }, ) } - Self::EthersProviderError(err) => { + Self::EthersProvider(err) => { warn!("EthersProviderError err={:?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, @@ -245,7 +254,7 @@ impl Web3ProxyError { }, ) } - Self::EthersWsClientError(err) => { + Self::EthersWsClient(err) => { warn!("EthersWsClientError err={:?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, @@ -256,7 +265,7 @@ impl Web3ProxyError { }, ) } - Self::FlumeRecvError(err) => { + Self::FlumeRecv(err) => { warn!("FlumeRecvError err={:#?}", err); ( StatusCode::INTERNAL_SERVER_ERROR, @@ -290,7 +299,19 @@ impl Web3ProxyError { }, ) } - Self::InfluxDb2RequestError(err) => { + Self::Hyper(err) => { + warn!("hyper err={:?}", err); + ( + StatusCode::INTERNAL_SERVER_ERROR, + JsonRpcErrorData { + // TODO: is it safe to expose these error strings? + message: Cow::Owned(err.to_string()), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, + ) + } + Self::InfluxDb2Request(err) => { // TODO: attach a request id to the message and to this error so that if people report problems, we can dig in sentry to find out more error!("influxdb2 err={:?}", err); ( @@ -371,6 +392,18 @@ impl Web3ProxyError { }, ) } + Self::Io(err) => { + warn!("std io err={:?}", err); + ( + StatusCode::INTERNAL_SERVER_ERROR, + JsonRpcErrorData { + // TODO: is it safe to expose our io error strings? + message: Cow::Owned(err.to_string()), + code: StatusCode::INTERNAL_SERVER_ERROR.as_u16().into(), + data: None, + }, + ) + } Self::UnknownReferralCode => { debug!("UnknownReferralCode"); ( @@ -528,6 +561,20 @@ impl Web3ProxyError { }, ) } + Self::NotEnoughSoftLimit { available, needed } => { + error!("NotEnoughSoftLimit {}/{}", available, needed); + ( + StatusCode::BAD_GATEWAY, + JsonRpcErrorData { + message: Cow::Owned(format!( + "not enough soft limit available {}/{}", + available, needed + )), + code: StatusCode::BAD_GATEWAY.as_u16().into(), + data: None, + }, + ) + } Self::NotFound => { // TODO: emit a stat? // TODO: instead of an error, show a normal html page for 404? diff --git a/web3_proxy/src/frontend/mod.rs b/web3_proxy/src/frontend/mod.rs index e1496960..28de5880 100644 --- a/web3_proxy/src/frontend/mod.rs +++ b/web3_proxy/src/frontend/mod.rs @@ -28,6 +28,8 @@ use tokio::sync::broadcast; use tower_http::cors::CorsLayer; use tower_http::sensitive_headers::SetSensitiveRequestHeadersLayer; +use self::errors::Web3ProxyResult; + /// simple keys for caching responses #[derive(Copy, Clone, Hash, PartialEq, Eq, EnumCount, EnumIter)] pub enum ResponseCacheKey { @@ -49,7 +51,7 @@ pub async fn serve( proxy_app: Arc, mut shutdown_receiver: broadcast::Receiver<()>, shutdown_complete_sender: broadcast::Sender<()>, -) -> anyhow::Result<()> { +) -> Web3ProxyResult<()> { // setup caches for whatever the frontend needs // no need for max items since it is limited by the enum key // TODO: latest moka allows for different ttls for different diff --git a/web3_proxy/src/frontend/status.rs b/web3_proxy/src/frontend/status.rs index 1f1cb94b..632f2839 100644 --- a/web3_proxy/src/frontend/status.rs +++ b/web3_proxy/src/frontend/status.rs @@ -36,7 +36,7 @@ pub async fn health( Ok(_health(app).await) }) .await - .unwrap(); + .expect("this cache get is infallible"); Response::builder() .status(code) @@ -70,7 +70,7 @@ pub async fn backups_needed( Ok(_backups_needed(app).await) }) .await - .unwrap(); + .expect("this cache get is infallible"); Response::builder() .status(code) @@ -120,7 +120,7 @@ pub async fn status( Ok(_status(app).await) }) .await - .unwrap(); + .expect("this cache get is infallible"); Response::builder() .status(code) diff --git a/web3_proxy/src/prometheus.rs b/web3_proxy/src/prometheus.rs index 2c582c24..c17b8ecd 100644 --- a/web3_proxy/src/prometheus.rs +++ b/web3_proxy/src/prometheus.rs @@ -8,13 +8,14 @@ use std::sync::Arc; use tokio::sync::broadcast; use crate::app::Web3ProxyApp; +use crate::frontend::errors::Web3ProxyResult; /// Run a prometheus metrics server on the given port. pub async fn serve( app: Arc, port: u16, mut shutdown_receiver: broadcast::Receiver<()>, -) -> anyhow::Result<()> { +) -> Web3ProxyResult<()> { // routes should be ordered most to least common let app = Router::new().route("/", get(root)).layer(Extension(app)); diff --git a/web3_proxy/src/rpcs/blockchain.rs b/web3_proxy/src/rpcs/blockchain.rs index 2b71b1fe..df79ac68 100644 --- a/web3_proxy/src/rpcs/blockchain.rs +++ b/web3_proxy/src/rpcs/blockchain.rs @@ -18,7 +18,6 @@ use std::convert::Infallible; use std::hash::Hash; use std::{cmp::Ordering, fmt::Display, sync::Arc}; use tokio::sync::broadcast; -use tokio::time::Duration; // TODO: type for Hydrated Blocks with their full transactions? pub type ArcBlock = Arc>; @@ -193,7 +192,7 @@ impl Web3Rpcs { .blocks_by_hash .get_or_insert_async::(&block_hash, async move { Ok(block) }) .await - .unwrap(); + .expect("this cache get is infallible"); Ok(block) } @@ -219,12 +218,11 @@ impl Web3Rpcs { // TODO: if error, retry? let block: Web3ProxyBlock = match rpc { Some(rpc) => rpc - .wait_for_request_handle(authorization, Some(Duration::from_secs(30))) - .await? .request::<_, Option>( "eth_getBlockByHash", &json!(get_block_params), Level::Error.into(), + authorization.clone(), ) .await? .and_then(|x| { @@ -366,7 +364,7 @@ impl Web3Rpcs { // TODO: document that this is a watch sender and not a broadcast! if things get busy, blocks might get missed // Geth's subscriptions have the same potential for skipping blocks. pending_tx_sender: Option>, - ) -> anyhow::Result<()> { + ) -> Web3ProxyResult<()> { let mut connection_heads = ConsensusFinder::new(self.max_block_age, self.max_block_lag); loop { diff --git a/web3_proxy/src/rpcs/consensus.rs b/web3_proxy/src/rpcs/consensus.rs index b11fb794..b7237ccb 100644 --- a/web3_proxy/src/rpcs/consensus.rs +++ b/web3_proxy/src/rpcs/consensus.rs @@ -374,7 +374,7 @@ impl ConsensusFinder { .first_seen .get_or_insert_async::(block.hash(), async { Ok(Instant::now()) }) .await - .unwrap(); + .expect("this cache get is infallible"); // calculate elapsed time before trying to lock let latency = first_seen.elapsed(); diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index b6f924de..310a5686 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -3,7 +3,7 @@ use super::blockchain::{BlocksByHashCache, BlocksByNumberCache, Web3ProxyBlock}; use super::consensus::{ConsensusWeb3Rpcs, ShouldWaitForBlock}; use super::one::Web3Rpc; use super::request::{OpenRequestHandle, OpenRequestResult, RequestErrorHandler}; -use crate::app::{flatten_handle, AnyhowJoinHandle, Web3ProxyApp}; +use crate::app::{flatten_handle, Web3ProxyApp, Web3ProxyJoinHandle}; use crate::config::{BlockAndRpc, TxHashAndRpc, Web3RpcConfig}; use crate::frontend::authorization::{Authorization, RequestMetadata}; use crate::frontend::errors::{Web3ProxyError, Web3ProxyResult}; @@ -11,7 +11,6 @@ use crate::frontend::rpc_proxy_ws::ProxyMode; use crate::jsonrpc::{JsonRpcErrorData, JsonRpcRequest}; use crate::response_cache::JsonRpcResponseData; use crate::rpcs::transactions::TxStatus; -use anyhow::Context; use arc_swap::ArcSwap; use counter::Counter; use derive_more::From; @@ -36,7 +35,7 @@ use std::sync::atomic::Ordering; use std::sync::Arc; use thread_fast_rng::rand::seq::SliceRandom; use tokio::sync::{broadcast, watch}; -use tokio::time::{interval, sleep, sleep_until, Duration, Instant, MissedTickBehavior}; +use tokio::time::{sleep, sleep_until, Duration, Instant}; /// A collection of web3 connections. Sends requests either the current best server or all servers. #[derive(From)] @@ -46,8 +45,6 @@ pub struct Web3Rpcs { pub(crate) block_sender: flume::Sender<(Option, Arc)>, /// any requests will be forwarded to one (or more) of these connections pub(crate) by_name: ArcSwap>>, - /// notify all http providers to check their blocks at the same time - pub(crate) http_interval_sender: Option>>, /// all providers with the same consensus head block. won't update if there is no `self.watch_consensus_head_sender` /// TODO: document that this is a watch sender and not a broadcast! if things get busy, blocks might get missed /// TODO: why is watch_consensus_head_sender in an Option, but this one isn't? @@ -78,9 +75,7 @@ impl Web3Rpcs { /// Spawn durable connections to multiple Web3 providers. #[allow(clippy::too_many_arguments)] pub async fn spawn( - chain_id: u64, db_conn: Option, - http_client: Option, max_block_age: Option, max_block_lag: Option, min_head_rpcs: usize, @@ -91,82 +86,18 @@ impl Web3Rpcs { watch_consensus_head_sender: Option>>, ) -> anyhow::Result<( Arc, - AnyhowJoinHandle<()>, + Web3ProxyJoinHandle<()>, watch::Receiver>>, // watch::Receiver>, )> { let (pending_tx_id_sender, pending_tx_id_receiver) = flume::unbounded(); let (block_sender, block_receiver) = flume::unbounded::(); - // TODO: query the rpc to get the actual expected block time, or get from config? maybe have this be part of a health check? - let expected_block_time_ms = match chain_id { - // ethereum - 1 => 12_000, - // ethereum-goerli - 5 => 12_000, - // polygon - 137 => 2_000, - // fantom - 250 => 1_000, - // arbitrum - 42161 => 500, - // anything else - _ => { - warn!( - "unexpected chain_id ({}). polling every {} seconds", - chain_id, 10 - ); - 10_000 - } - }; - - let http_interval_sender = if http_client.is_some() { - let (sender, _) = broadcast::channel(1); - - // TODO: what interval? follow a websocket also? maybe by watching synced connections with a timeout. will need debounce - let mut interval = interval(Duration::from_millis(expected_block_time_ms / 2)); - interval.set_missed_tick_behavior(MissedTickBehavior::Delay); - - let sender = Arc::new(sender); - - let f = { - let sender = sender.clone(); - - async move { - loop { - interval.tick().await; - - // trace!("http interval ready"); - - if sender.send(()).is_err() { - // errors are okay. they mean that all receivers have been dropped, or the rpcs just haven't started yet - // TODO: i'm seeing this error a lot more than expected - trace!("no http receivers"); - }; - } - } - }; - - // TODO: do something with this handle? - tokio::spawn(f); - - Some(sender) - } else { - None - }; - // these blocks don't have full transactions, but they do have rather variable amounts of transaction hashes - // TODO: how can we do the weigher this? need to know actual allocated size + // TODO: actual weighter on this // TODO: time_to_idle instead? - // TODO: limits from config let blocks_by_hash: BlocksByHashCache = Arc::new(CacheWithTTL::new_with_capacity(10_000, Duration::from_secs(30 * 60)).await); - // .max_capacity(1024 * 1024 * 1024) - // .weigher(|_k, v: &Web3ProxyBlock| { - // 1 + v.block.transactions.len().try_into().unwrap_or(u32::MAX) - // }) - // .time_to_live(Duration::from_secs(30 * 60)) - // .build_with_hasher(hashbrown::hash_map::DefaultHashBuilder::default()); // all block numbers are the same size, so no need for weigher // TODO: limits from config @@ -185,7 +116,6 @@ impl Web3Rpcs { blocks_by_hash, blocks_by_number, by_name, - http_interval_sender, max_block_age, max_block_lag, min_head_rpcs, @@ -214,7 +144,7 @@ impl Web3Rpcs { &self, app: &Web3ProxyApp, rpc_configs: HashMap, - ) -> anyhow::Result<()> { + ) -> Web3ProxyResult<()> { // safety checks if rpc_configs.len() < app.config.min_synced_rpcs { // TODO: don't count disabled servers! @@ -232,15 +162,14 @@ impl Web3Rpcs { let sum_soft_limit = rpc_configs.values().fold(0, |acc, x| acc + x.soft_limit); // TODO: require a buffer? - anyhow::ensure!( - sum_soft_limit >= self.min_sum_soft_limit, - "Only {}/{} soft limit! Add more rpcs, increase soft limits, or reduce min_sum_soft_limit.", - sum_soft_limit, - self.min_sum_soft_limit, - ); + if sum_soft_limit < self.min_sum_soft_limit { + return Err(Web3ProxyError::NotEnoughSoftLimit { + available: sum_soft_limit, + needed: self.min_sum_soft_limit, + }); + } // turn configs into connections (in parallel) - // TODO: move this into a helper function. then we can use it when configs change (will need a remove function too) let mut spawn_handles: FuturesUnordered<_> = rpc_configs .into_iter() .filter_map(|(server_name, server_config)| { @@ -261,7 +190,6 @@ impl Web3Rpcs { let pending_tx_id_sender = Some(self.pending_tx_id_sender.clone()); let blocks_by_hash_cache = self.blocks_by_hash.clone(); - let http_interval_sender = self.http_interval_sender.clone(); let chain_id = app.config.chain_id; debug!("spawning {}", server_name); @@ -272,7 +200,6 @@ impl Web3Rpcs { vredis_pool, chain_id, http_client, - http_interval_sender, blocks_by_hash_cache, block_sender, pending_tx_id_sender, @@ -312,7 +239,7 @@ impl Web3Rpcs { Ok(Err(err)) => { // if we got an error here, the app can continue on // TODO: include context about which connection failed - // TODO: will this retry automatically? i don't think so + // TODO: retry automatically error!("Unable to create connection. err={:?}", err); } Err(err) => { @@ -322,6 +249,15 @@ impl Web3Rpcs { } } + let num_rpcs = self.by_name.load().len(); + + if num_rpcs < self.min_head_rpcs { + return Err(Web3ProxyError::NotEnoughRpcs { + num_known: num_rpcs, + min_head_rpcs: self.min_head_rpcs, + }); + } + Ok(()) } @@ -349,7 +285,7 @@ impl Web3Rpcs { authorization: Arc, block_receiver: flume::Receiver, pending_tx_sender: Option>, - ) -> anyhow::Result<()> { + ) -> Web3ProxyResult<()> { let mut futures = vec![]; // setup the transaction funnel @@ -1317,7 +1253,6 @@ mod tests { #[cfg(test)] fn new_peak_latency() -> PeakEwmaLatency { - const NANOS_PER_MILLI: f64 = 1_000_000.0; PeakEwmaLatency::spawn(Duration::from_secs(1), 4, Duration::from_secs(1)) } @@ -1490,7 +1425,6 @@ mod tests { let rpcs = Web3Rpcs { block_sender: block_sender.clone(), by_name: ArcSwap::from_pointee(rpcs_by_name), - http_interval_sender: None, name: "test".to_string(), watch_consensus_head_sender: Some(watch_consensus_head_sender), watch_consensus_rpcs_sender, @@ -1568,7 +1502,7 @@ mod tests { .send_head_block_result( Ok(Some(lagged_block.clone())), &block_sender, - rpcs.blocks_by_hash.clone(), + &rpcs.blocks_by_hash, ) .await .unwrap(); @@ -1588,7 +1522,7 @@ mod tests { .send_head_block_result( Ok(Some(lagged_block.clone())), &block_sender, - rpcs.blocks_by_hash.clone(), + &rpcs.blocks_by_hash, ) .await .unwrap(); @@ -1620,7 +1554,7 @@ mod tests { .send_head_block_result( Ok(Some(head_block.clone())), &block_sender, - rpcs.blocks_by_hash.clone(), + &rpcs.blocks_by_hash, ) .await .unwrap(); @@ -1741,7 +1675,6 @@ mod tests { let rpcs = Web3Rpcs { block_sender, by_name: ArcSwap::from_pointee(rpcs_by_name), - http_interval_sender: None, name: "test".to_string(), watch_consensus_head_sender: Some(watch_consensus_head_sender), watch_consensus_rpcs_sender, @@ -1911,7 +1844,6 @@ mod tests { let rpcs = Web3Rpcs { block_sender, by_name: ArcSwap::from_pointee(rpcs_by_name), - http_interval_sender: None, name: "test".to_string(), watch_consensus_head_sender: Some(watch_consensus_head_sender), watch_consensus_rpcs_sender, diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index 9015dd18..e3e3b0ea 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -2,18 +2,18 @@ use super::blockchain::{ArcBlock, BlocksByHashCache, Web3ProxyBlock}; use super::provider::{connect_http, connect_ws, EthersHttpProvider, EthersWsProvider}; use super::request::{OpenRequestHandle, OpenRequestResult}; -use crate::app::{flatten_handle, AnyhowJoinHandle}; +use crate::app::{flatten_handle, Web3ProxyJoinHandle}; use crate::config::{BlockAndRpc, Web3RpcConfig}; use crate::frontend::authorization::Authorization; use crate::frontend::errors::{Web3ProxyError, Web3ProxyResult}; use crate::rpcs::request::RequestErrorHandler; use anyhow::{anyhow, Context}; -use ethers::prelude::{Bytes, Middleware, ProviderError, TxHash, H256, U64}; +use ethers::prelude::{Bytes, Middleware, TxHash, U64}; use ethers::types::{Address, Transaction, U256}; use futures::future::try_join_all; use futures::StreamExt; use latency::{EwmaLatency, PeakEwmaLatency}; -use log::{debug, error, info, trace, warn, Level}; +use log::{debug, info, trace, warn, Level}; use migration::sea_orm::DatabaseConnection; use ordered_float::OrderedFloat; use parking_lot::RwLock; @@ -21,16 +21,12 @@ use redis_rate_limiter::{RedisPool, RedisRateLimitResult, RedisRateLimiter}; use serde::ser::{SerializeStruct, Serializer}; use serde::Serialize; use serde_json::json; -use std::borrow::Cow; -use std::cmp::min; use std::convert::Infallible; use std::fmt; use std::hash::{Hash, Hasher}; -use std::sync::atomic::{self, AtomicBool, AtomicU64, AtomicUsize}; +use std::sync::atomic::{self, AtomicU64, AtomicUsize}; use std::{cmp::Ordering, sync::Arc}; -use thread_fast_rng::rand::Rng; -use thread_fast_rng::thread_fast_rng; -use tokio::sync::{broadcast, oneshot, watch, RwLock as AsyncRwLock}; +use tokio::sync::watch; use tokio::time::{sleep, sleep_until, timeout, Duration, Instant}; use url::Url; @@ -88,13 +84,12 @@ impl Web3Rpc { db_conn: Option, // optional because this is only used for http providers. websocket providers don't use it http_client: Option, - // TODO: rename to http_new_head_interval_sender? - http_interval_sender: Option>>, redis_pool: Option, + block_interval: Duration, block_map: BlocksByHashCache, block_sender: Option>, tx_id_sender: Option)>>, - ) -> anyhow::Result<(Arc, AnyhowJoinHandle<()>)> { + ) -> anyhow::Result<(Arc, Web3ProxyJoinHandle<()>)> { let created_at = Instant::now(); let hard_limit = match (config.hard_limit, redis_pool) { @@ -151,8 +146,6 @@ impl Web3Rpc { } } - let (disconnect_sender, disconnect_receiver) = watch::channel(false); - let (head_block, _) = watch::channel(None); // Spawn the task for calculting average peak latency @@ -170,7 +163,7 @@ impl Web3Rpc { let http_provider = if let Some(http_url) = config.http_url { let http_url = http_url.parse::()?; - Some(connect_http(Cow::Owned(http_url), http_client)?) + Some(connect_http(http_url, http_client, block_interval)?) // TODO: check the provider is on the right chain } else { @@ -180,20 +173,21 @@ impl Web3Rpc { let ws_provider = if let Some(ws_url) = config.ws_url { let ws_url = ws_url.parse::()?; - Some(connect_ws(Cow::Owned(ws_url), usize::MAX).await?) + Some(connect_ws(ws_url, usize::MAX).await?) // TODO: check the provider is on the right chain } else { None }; + let (disconnect_watch, _) = watch::channel(false); + let new_rpc = Self { automatic_block_limit, backup, block_data_limit, created_at: Some(created_at), db_conn: db_conn.clone(), - disconnect_watch: Some(disconnect_sender), display_name: config.display_name, hard_limit, hard_limit_until: Some(hard_limit_until), @@ -203,6 +197,8 @@ impl Web3Rpc { peak_latency: Some(peak_latency), soft_limit: config.soft_limit, tier: config.tier, + ws_provider, + disconnect_watch: Some(disconnect_watch), ..Default::default() }; @@ -221,8 +217,6 @@ impl Web3Rpc { block_map, block_sender, chain_id, - disconnect_receiver, - http_interval_sender, tx_id_sender, ) .await @@ -264,13 +258,12 @@ impl Web3Rpc { // TODO: binary search between 90k and max? // TODO: start at 0 or 1? for block_data_limit in [0, 32, 64, 128, 256, 512, 1024, 90_000, u64::MAX] { - let handle = self.wait_for_request_handle(authorization, None).await?; - - let head_block_num_future = handle.request::, U256>( + let head_block_num_future = self.request::, U256>( "eth_blockNumber", &None, // error here are expected, so keep the level low Level::Debug.into(), + authorization.clone(), ); let head_block_num = timeout(Duration::from_secs(5), head_block_num_future) @@ -288,9 +281,7 @@ impl Web3Rpc { // TODO: wait for the handle BEFORE we check the current block number. it might be delayed too! // TODO: what should the request be? - let handle = self.wait_for_request_handle(authorization, None).await?; - - let archive_result: Result = handle + let archive_result: Result = self .request( "eth_getCode", &json!(( @@ -299,6 +290,7 @@ impl Web3Rpc { )), // error here are expected, so keep the level low Level::Trace.into(), + authorization.clone(), ) .await; @@ -377,23 +369,20 @@ impl Web3Rpc { } /// query the web3 provider to confirm it is on the expected chain with the expected data available - async fn check_provider( - self: &Arc, - block_sender: Option<&flume::Sender>, - chain_id: u64, - db_conn: Option<&DatabaseConnection>, - ) -> anyhow::Result<()> { - let authorization = Arc::new(Authorization::internal(db_conn.cloned())?); + async fn check_provider(self: &Arc, chain_id: u64) -> Web3ProxyResult<()> { + let authorization = Arc::new(Authorization::internal(self.db_conn.clone())?); // check the server's chain_id here // TODO: some public rpcs (on bsc and fantom) do not return an id and so this ends up being an error // TODO: what should the timeout be? should there be a request timeout? // trace!("waiting on chain id for {}", self); let found_chain_id: Result = self - .wait_for_request_handle(&authorization, None) - .await - .context(format!("waiting for request handle on {}", self))? - .request("eth_chainId", &json!(Vec::<()>::new()), Level::Trace.into()) + .request( + "eth_chainId", + &json!(Vec::<()>::new()), + Level::Trace.into(), + authorization.clone(), + ) .await; trace!("found_chain_id: {:#?}", found_chain_id); @@ -406,12 +395,14 @@ impl Web3Rpc { chain_id, found_chain_id ) - .context(format!("failed @ {}", self))); + .context(format!("failed @ {}", self)) + .into()); } } Err(e) => { return Err(anyhow::Error::from(e) - .context(format!("unable to parse eth_chainId from {}", self))); + .context(format!("unable to parse eth_chainId from {}", self)) + .into()); } } @@ -426,27 +417,25 @@ impl Web3Rpc { pub(crate) async fn send_head_block_result( self: &Arc, - new_head_block: Result, ProviderError>, + new_head_block: Web3ProxyResult>, block_sender: &flume::Sender, - block_map: BlocksByHashCache, - ) -> anyhow::Result<()> { + block_map: &BlocksByHashCache, + ) -> Web3ProxyResult<()> { let new_head_block = match new_head_block { Ok(None) => { - { - let head_block_tx = self.head_block.as_ref().unwrap(); + let head_block_tx = self.head_block.as_ref().unwrap(); - if head_block_tx.borrow().is_none() { - // we previously sent a None. return early - return Ok(()); - } - - let age = self.created_at.unwrap().elapsed().as_millis(); - - debug!("clearing head block on {} ({}ms old)!", self, age); - - head_block_tx.send_replace(None); + if head_block_tx.borrow().is_none() { + // we previously sent a None. return early + return Ok(()); } + let age = self.created_at.unwrap().elapsed().as_millis(); + + debug!("clearing head block on {} ({}ms old)!", self, age); + + head_block_tx.send_replace(None); + None } Ok(Some(new_head_block)) => { @@ -461,7 +450,8 @@ impl Web3Rpc { &new_hash, async move { Ok(new_head_block) }, ) - .await?; + .await + .expect("this cache get is infallible"); // save the block so we don't send the same one multiple times // also save so that archive checks can know how far back to query @@ -504,9 +494,61 @@ impl Web3Rpc { *self.disconnect_watch.as_ref().unwrap().borrow() } + async fn healthcheck( + self: &Arc, + authorization: &Arc, + error_handler: RequestErrorHandler, + ) -> Web3ProxyResult<()> { + let head_block = self.head_block.as_ref().unwrap().borrow().clone(); + + if let Some(head_block) = head_block { + let head_block = head_block.block; + + // TODO: if head block is very old and not expected to be syncing, emit warning + + let block_number = head_block.number.context("no block number")?; + + let to = if let Some(txid) = head_block.transactions.last().cloned() { + let tx = self + .request::<_, Option>( + "eth_getTransactionByHash", + &(txid,), + error_handler, + authorization.clone(), + ) + .await? + .context("no transaction")?; + + // TODO: what default? something real? + tx.to.unwrap_or_else(|| { + "0xdead00000000000000000000000000000000beef" + .parse::
() + .expect("deafbeef") + }) + } else { + "0xdead00000000000000000000000000000000beef" + .parse::
() + .expect("deafbeef") + }; + + let _code = self + .request::<_, Option>( + "eth_getCode", + &(to, block_number), + error_handler, + authorization.clone(), + ) + .await?; + } else { + // TODO: if head block is none for too long, give an error + } + + Ok(()) + } + /// subscribe to blocks and transactions /// This should only exit when the program is exiting. - /// TODO: should more of these args be on self? + /// TODO: should more of these args be on self? chain_id for sure #[allow(clippy::too_many_arguments)] async fn subscribe( self: Arc, @@ -514,220 +556,97 @@ impl Web3Rpc { block_map: BlocksByHashCache, block_sender: Option>, chain_id: u64, - disconnect_receiver: watch::Receiver, - http_interval_sender: Option>>, tx_id_sender: Option)>>, - ) -> anyhow::Result<()> { + ) -> Web3ProxyResult<()> { let error_handler = if self.backup { RequestErrorHandler::DebugLevel } else { RequestErrorHandler::ErrorLevel }; - todo!(); + debug!("starting subscriptions on {}", self); + + self.check_provider(chain_id).await?; - /* let mut futures = vec![]; - while false { - let http_interval_receiver = http_interval_sender.as_ref().map(|x| x.subscribe()); + // health check that runs if there haven't been any recent requests + { + // TODO: move this into a proper function + let authorization = authorization.clone(); + let rpc = self.clone(); - { - // TODO: move this into a proper function - let authorization = authorization.clone(); - let block_sender = block_sender.clone(); - let rpc = self.clone(); - let (ready_tx, ready_rx) = oneshot::channel(); - let f = async move { - // initial sleep to allow for the initial connection - rpc.retrying_connect( - block_sender.as_ref(), - chain_id, - authorization.db_conn.as_ref(), - delay_start, - ) - .await?; + // TODO: how often? different depending on the chain? + // TODO: reset this timeout when a new block is seen? we need to keep request_latency updated though + let health_sleep_seconds = 10; - // provider is ready - ready_tx.send(()).unwrap(); + // health check loop + let f = async move { + // TODO: benchmark this and lock contention + let mut old_total_requests = 0; + let mut new_total_requests; - // TODO: how often? different depending on the chain? - // TODO: reset this timeout when a new block is seen? we need to keep request_latency updated though - let health_sleep_seconds = 10; + // TODO: errors here should not cause the loop to exit! + while !rpc.should_disconnect() { + new_total_requests = rpc.total_requests.load(atomic::Ordering::Relaxed); - // TODO: benchmark this and lock contention - let mut old_total_requests = 0; - let mut new_total_requests; - - // health check loop - loop { - // TODO: do we need this to be abortable? - if rpc.should_disconnect() { - break; - } - - sleep(Duration::from_secs(health_sleep_seconds)).await; - - trace!("health check on {}", rpc); - - // TODO: what if we just happened to have this check line up with another restart? - // TODO: think more about this - if let Some(client) = rpc.ws_provider.read().await.clone() { - // health check as a way of keeping this rpc's request_ewma accurate - // TODO: do something different if this is a backup server? - - new_total_requests = rpc.total_requests.load(atomic::Ordering::Acquire); - - // TODO: how many requests should we require in order to skip a health check? - if new_total_requests - old_total_requests < 10 { - // TODO: if this fails too many times, reset the connection - // TODO: move this into a function and the chaining should be easier - let head_block = rpc.head_block.as_ref().unwrap().borrow().clone(); - - if let Some((block_number, txid)) = head_block.and_then(|x| { - let block = x.block; - - let block_number = block.number?; - let txid = block.transactions.last().cloned()?; - - Some((block_number, txid)) - }) { - let to = rpc - .wait_for_query::<_, Option>( - "eth_getTransactionByHash", - &(txid,), - error_handler, - authorization.clone(), - Some(client.clone()), - ) - .await - .and_then(|tx| { - let tx = tx.context("no transaction found")?; - - // TODO: what default? something real? - let to = tx.to.unwrap_or_else(|| { - "0xdead00000000000000000000000000000000beef" - .parse::
() - .expect("deafbeef") - }); - - Ok(to) - }); - - let code = match to { - Err(err) => { - // TODO: an "error" here just means that the hash wasn't available. i dont think its truly an "error" - if rpc.backup { - debug!( - "{} failed health check query! {:#?}", - rpc, err - ); - } else { - warn!( - "{} failed health check query! {:#?}", - rpc, err - ); - } - continue; - } - Ok(to) => { - rpc.wait_for_query::<_, Option>( - "eth_getCode", - &(to, block_number), - error_handler, - authorization.clone(), - Some(client), - ) - .await - } - }; - - if let Err(err) = code { - if rpc.backup { - debug!("{} failed health check query! {:#?}", rpc, err); - } else { - warn!("{} failed health check query! {:#?}", rpc, err); - } - continue; - } - } - } - - old_total_requests = new_total_requests; + if new_total_requests - old_total_requests < 10 { + // TODO: if this fails too many times, reset the connection + // TODO: move this into a function and the chaining should be easier + if let Err(err) = rpc.healthcheck(&authorization, error_handler).await { + // TODO: different level depending on the error handler + warn!("health checking {} failed: {:?}", rpc, err); } } - debug!("health checks for {} exited", rpc); - Ok(()) - }; + // TODO: should we count the requests done inside this health check + old_total_requests = new_total_requests; - futures.push(flatten_handle(tokio::spawn(f))); - - // wait on the initial connection - ready_rx.await?; - } - - if let Some(block_sender) = &block_sender { - // TODO: do we need this to be abortable? - let f = self.clone().subscribe_new_heads( - authorization.clone(), - http_interval_receiver, - block_sender.clone(), - block_map.clone(), - ); - - futures.push(flatten_handle(tokio::spawn(f))); - } - - if let Some(tx_id_sender) = &tx_id_sender { - // TODO: do we need this to be abortable? - let f = self - .clone() - .subscribe_pending_transactions(authorization.clone(), tx_id_sender.clone()); - - futures.push(flatten_handle(tokio::spawn(f))); - } - - match try_join_all(futures).await { - Ok(_) => { - // future exited without error - // TODO: think about this more. we never set it to false. this can't be right - break; + sleep(Duration::from_secs(health_sleep_seconds)).await; } - Err(err) => { - let disconnect_sender = self.disconnect_watch.as_ref().unwrap(); - if self.reconnect.load(atomic::Ordering::Acquire) { - warn!("{} connection ended. reconnecting. err={:?}", self, err); + debug!("healthcheck loop on {} exited", rpc); - // TODO: i'm not sure if this is necessary, but telling everything to disconnect seems like a better idea than relying on timeouts and dropped futures. - disconnect_sender.send_replace(true); - disconnect_sender.send_replace(false); + Ok(()) + }; - // we call retrying_connect here with initial_delay=true. above, initial_delay=false - delay_start = true; - - continue; - } - - // reconnect is not enabled. - if *disconnect_receiver.borrow() { - info!("{} is disconnecting", self); - break; - } else { - error!("{} subscription exited. err={:?}", self, err); - - disconnect_sender.send_replace(true); - - break; - } - } - } + futures.push(flatten_handle(tokio::spawn(f))); } + // subscribe to new heads + if let Some(block_sender) = &block_sender { + // TODO: do we need this to be abortable? + let f = self.clone().subscribe_new_heads( + authorization.clone(), + block_sender.clone(), + block_map.clone(), + ); - */ - info!("all subscriptions on {} completed", self); + futures.push(flatten_handle(tokio::spawn(f))); + } + + // subscribe pending transactions + // TODO: make this opt-in. its a lot of bandwidth + if let Some(tx_id_sender) = tx_id_sender { + // TODO: do we need this to be abortable? + let f = self + .clone() + .subscribe_pending_transactions(authorization.clone(), tx_id_sender); + + futures.push(flatten_handle(tokio::spawn(f))); + } + + // try_join on the futures + if let Err(err) = try_join_all(futures).await { + warn!("subscription erred: {:?}", err); + } + + debug!("subscriptions on {} exited", self); + + self.disconnect_watch + .as_ref() + .expect("disconnect_watch should always be set") + .send_replace(true); Ok(()) } @@ -736,197 +655,76 @@ impl Web3Rpc { async fn subscribe_new_heads( self: Arc, authorization: Arc, - http_interval_receiver: Option>, block_sender: flume::Sender, block_map: BlocksByHashCache, - ) -> anyhow::Result<()> { - trace!("watching new heads on {}", self); + ) -> Web3ProxyResult<()> { + debug!("subscribing to new heads on {}", self); if let Some(ws_provider) = self.ws_provider.as_ref() { - todo!("subscribe") + // todo: move subscribe_blocks onto the request handle + let active_request_handle = self.wait_for_request_handle(&authorization, None).await; + let mut blocks = ws_provider.subscribe_blocks().await?; + drop(active_request_handle); + + // query the block once since the subscription doesn't send the current block + // there is a very small race condition here where the stream could send us a new block right now + // but all seeing the same block twice won't break anything + // TODO: how does this get wrapped in an arc? does ethers handle that? + // TODO: can we force this to use the websocket? + let latest_block: Result, _> = self + .request( + "eth_getBlockByNumber", + &json!(("latest", false)), + Level::Warn.into(), + authorization, + ) + .await; + + self.send_head_block_result(latest_block, &block_sender, &block_map) + .await?; + + while let Some(block) = blocks.next().await { + if self.should_disconnect() { + break; + } + + let block = Arc::new(block); + + self.send_head_block_result(Ok(Some(block)), &block_sender, &block_map) + .await?; + } } else if let Some(http_provider) = self.http_provider.as_ref() { - todo!("poll") + // there is a "watch_blocks" function, but a lot of public nodes do not support the necessary rpc endpoints + let mut blocks = http_provider.watch_blocks().await?; + + while let Some(block_hash) = blocks.next().await { + if self.should_disconnect() { + break; + } + + let block = if let Some(block) = block_map.get(&block_hash) { + block.block + } else if let Some(block) = http_provider.get_block(block_hash).await? { + Arc::new(block) + } else { + continue; + }; + + self.send_head_block_result(Ok(Some(block)), &block_sender, &block_map) + .await?; + } } else { unimplemented!("no ws or http provider!") } - /* - match provider.as_ref() { - Web3Provider::Http(_client) => { - // there is a "watch_blocks" function, but a lot of public nodes do not support the necessary rpc endpoints - // TODO: try watch_blocks and fall back to this? - - let mut http_interval_receiver = http_interval_receiver.unwrap(); - - let mut last_hash = H256::zero(); - - while !self.should_disconnect() { - // TODO: what should the max_wait be? - // we do not pass unlocked_provider because we want to get a new one each call. otherwise we might re-use an old one - match self.wait_for_request_handle(&authorization, None).await { - Ok(active_request_handle) => { - let block: Result, _> = active_request_handle - .request( - "eth_getBlockByNumber", - &json!(("latest", false)), - Level::Warn.into(), - ) - .await; - - match block { - Ok(None) => { - warn!("no head block on {}", self); - - self.send_head_block_result( - Ok(None), - &block_sender, - block_map.clone(), - ) - .await?; - } - Ok(Some(block)) => { - if let Some(new_hash) = block.hash { - // don't send repeat blocks - if new_hash != last_hash { - // new hash! - last_hash = new_hash; - - self.send_head_block_result( - Ok(Some(block)), - &block_sender, - block_map.clone(), - ) - .await?; - } - } else { - // TODO: why is this happening? - warn!("empty head block on {}", self); - - self.send_head_block_result( - Ok(None), - &block_sender, - block_map.clone(), - ) - .await?; - } - } - Err(err) => { - // we did not get a block back. something is up with the server. take it out of rotation - self.send_head_block_result( - Err(err), - &block_sender, - block_map.clone(), - ) - .await?; - } - } - } - Err(err) => { - warn!("Internal error on latest block from {}. {:?}", self, err); - - self.send_head_block_result(Ok(None), &block_sender, block_map.clone()) - .await?; - - // TODO: what should we do? sleep? extra time? - } - } - - // wait for the next interval - // TODO: if error or rate limit, increase interval? - while let Err(err) = http_interval_receiver.recv().await { - match err { - broadcast::error::RecvError::Closed => { - // channel is closed! that's not good. bubble the error up - return Err(err.into()); - } - broadcast::error::RecvError::Lagged(lagged) => { - // querying the block was delayed - // this can happen if tokio is very busy or waiting for requests limits took too long - if self.backup { - debug!("http interval on {} lagging by {}!", self, lagged); - } else { - warn!("http interval on {} lagging by {}!", self, lagged); - } - } - } - } - } - } - Web3Provider::Both(_, client) | Web3Provider::Ws(client) => { - // todo: move subscribe_blocks onto the request handle? - let active_request_handle = - self.wait_for_request_handle(&authorization, None).await; - let mut stream = client.subscribe_blocks().await?; - drop(active_request_handle); - - // query the block once since the subscription doesn't send the current block - // there is a very small race condition here where the stream could send us a new block right now - // but all that does is print "new block" for the same block as current block - // TODO: how does this get wrapped in an arc? does ethers handle that? - // TODO: do this part over http? - let block: Result, _> = self - .wait_for_request_handle(&authorization, None) - .await? - .request( - "eth_getBlockByNumber", - &json!(("latest", false)), - Level::Warn.into(), - ) - .await; - - let mut last_hash = match &block { - Ok(Some(new_block)) => new_block - .hash - .expect("blocks should always have a hash here"), - _ => H256::zero(), - }; - - self.send_head_block_result(block, &block_sender, block_map.clone()) - .await?; - - while let Some(new_block) = stream.next().await { - // TODO: select on disconnect_watch instead of waiting for a block to arrive - if self.should_disconnect() { - break; - } - - // TODO: check the new block's hash to be sure we don't send dupes - let new_hash = new_block - .hash - .expect("blocks should always have a hash here"); - - if new_hash == last_hash { - // some rpcs like to give us duplicates. don't waste our time on them - continue; - } else { - last_hash = new_hash; - } - - self.send_head_block_result( - Ok(Some(Arc::new(new_block))), - &block_sender, - block_map.clone(), - ) - .await?; - } - - // TODO: is this always an error? - // TODO: we probably don't want a warn and to return error - debug!("new_heads subscription to {} ended", self); - } - #[cfg(test)] - Web3Provider::Mock => unimplemented!(), - } - */ - // clear the head block. this might not be needed, but it won't hurt - self.send_head_block_result(Ok(None), &block_sender, block_map) + self.send_head_block_result(Ok(None), &block_sender, &block_map) .await?; if self.should_disconnect() { Ok(()) } else { - Err(anyhow!("new_heads subscription exited. reconnect needed")) + Err(anyhow!("new_heads subscription exited. reconnect needed").into()) } } @@ -935,7 +733,7 @@ impl Web3Rpc { self: Arc, authorization: Arc, tx_id_sender: flume::Sender<(TxHash, Arc)>, - ) -> anyhow::Result<()> { + ) -> Web3ProxyResult<()> { // TODO: make this subscription optional self.wait_for_disconnect().await?; @@ -985,9 +783,7 @@ impl Web3Rpc { if self.should_disconnect() { Ok(()) } else { - Err(anyhow!( - "pending_transactions subscription exited. reconnect needed" - )) + Err(anyhow!("pending_transactions subscription exited. reconnect needed").into()) } } @@ -1034,7 +830,7 @@ impl Web3Rpc { } // TODO: sleep how long? maybe just error? - // TODO: instead of an arbitrary sleep, subscribe to the head block on this + // TODO: instead of an arbitrary sleep, subscribe to the head block on this? sleep(Duration::from_millis(10)).await; } Err(err) => return Err(err), @@ -1099,36 +895,39 @@ impl Web3Rpc { } async fn wait_for_disconnect(&self) -> Result<(), tokio::sync::watch::error::RecvError> { - let mut disconnect_watch = self.disconnect_watch.as_ref().unwrap().subscribe(); + let mut disconnect_subscription = self.disconnect_watch.as_ref().unwrap().subscribe(); loop { - if *disconnect_watch.borrow_and_update() { + if *disconnect_subscription.borrow_and_update() { // disconnect watch is set to "true" return Ok(()); } - // wait for disconnect_watch to change - disconnect_watch.changed().await?; + // wait for disconnect_subscription to change + disconnect_subscription.changed().await?; } } - pub async fn wait_for_query( + pub async fn request( self: &Arc, method: &str, params: &P, revert_handler: RequestErrorHandler, authorization: Arc, - ) -> anyhow::Result + ) -> Web3ProxyResult where // TODO: not sure about this type. would be better to not need clones, but measure and spawns combine to need it P: Clone + fmt::Debug + serde::Serialize + Send + Sync + 'static, R: serde::Serialize + serde::de::DeserializeOwned + fmt::Debug + Send, { - self.wait_for_request_handle(&authorization, None) + // TODO: take max_wait as a function argument? + let x = self + .wait_for_request_handle(&authorization, None) .await? .request::(method, params, revert_handler) - .await - .context("ProviderError from the backend") + .await?; + + Ok(x) } } @@ -1255,7 +1054,7 @@ impl fmt::Display for Web3Rpc { mod tests { #![allow(unused_imports)] use super::*; - use ethers::types::{Block, U256}; + use ethers::types::{Block, H256, U256}; #[test] fn test_archive_node_has_block_data() { diff --git a/web3_proxy/src/rpcs/provider.rs b/web3_proxy/src/rpcs/provider.rs index 1829b0a5..45c82147 100644 --- a/web3_proxy/src/rpcs/provider.rs +++ b/web3_proxy/src/rpcs/provider.rs @@ -1,26 +1,20 @@ -use anyhow::anyhow; -use derive_more::From; use ethers::providers::{Authorization, ConnectionDetails}; -use std::{borrow::Cow, time::Duration}; +use std::time::Duration; use url::Url; // TODO: our own structs for these that handle streaming large responses pub type EthersHttpProvider = ethers::providers::Provider; pub type EthersWsProvider = ethers::providers::Provider; -pub fn extract_auth(url: &mut Cow<'_, Url>) -> Option { +pub fn extract_auth(url: &mut Url) -> Option { if let Some(pass) = url.password().map(|x| x.to_string()) { // to_string is needed because we are going to remove these items from the url let user = url.username().to_string(); // clear username and password from the url - let mut_url = url.to_mut(); - - mut_url - .set_username("") + url.set_username("") .expect("unable to clear username on websocket"); - mut_url - .set_password(None) + url.set_password(None) .expect("unable to clear password on websocket"); // keep them @@ -33,35 +27,36 @@ pub fn extract_auth(url: &mut Cow<'_, Url>) -> Option { /// Note, if the http url has an authority the http_client param is ignored and a dedicated http_client will be used /// TODO: take a reqwest::Client or a reqwest::ClientBuilder. that way we can do things like set compression even when auth is set pub fn connect_http( - mut url: Cow<'_, Url>, + mut url: Url, http_client: Option, + interval: Duration, ) -> anyhow::Result { let auth = extract_auth(&mut url); - let provider = if url.scheme().starts_with("http") { + let mut provider = if url.scheme().starts_with("http") { let provider = if let Some(auth) = auth { - ethers::providers::Http::new_with_auth(url.into_owned(), auth)? + ethers::providers::Http::new_with_auth(url, auth)? } else if let Some(http_client) = http_client { - ethers::providers::Http::new_with_client(url.into_owned(), http_client) + ethers::providers::Http::new_with_client(url, http_client) } else { - ethers::providers::Http::new(url.into_owned()) + ethers::providers::Http::new(url) }; // TODO: i don't think this interval matters for our uses, but we should probably set it to like `block time / 2` - ethers::providers::Provider::new(provider) - .interval(Duration::from_secs(12)) - .into() + ethers::providers::Provider::new(provider).interval(Duration::from_secs(2)) } else { - return Err(anyhow::anyhow!("only http servers are supported")); + return Err(anyhow::anyhow!( + "only http servers are supported. cannot use {}", + url + )); }; + provider.set_interval(interval); + Ok(provider) } -pub async fn connect_ws( - mut url: Cow<'_, Url>, - reconnects: usize, -) -> anyhow::Result { +pub async fn connect_ws(mut url: Url, reconnects: usize) -> anyhow::Result { let auth = extract_auth(&mut url); let provider = if url.scheme().starts_with("ws") { @@ -76,7 +71,7 @@ pub async fn connect_ws( // TODO: dry this up (needs https://github.com/gakonst/ethers-rs/issues/592) // TODO: i don't think this interval matters - ethers::providers::Provider::new(provider).into() + ethers::providers::Provider::new(provider) } else { return Err(anyhow::anyhow!("ws servers are supported")); }; diff --git a/web3_proxy/src/rpcs/request.rs b/web3_proxy/src/rpcs/request.rs index 34110bf7..9e91f75c 100644 --- a/web3_proxy/src/rpcs/request.rs +++ b/web3_proxy/src/rpcs/request.rs @@ -1,5 +1,6 @@ use super::one::Web3Rpc; use crate::frontend::authorization::Authorization; +use crate::frontend::errors::Web3ProxyResult; use anyhow::Context; use chrono::Utc; use entities::revert_log; @@ -13,7 +14,7 @@ use std::fmt; use std::sync::atomic; use std::sync::Arc; use thread_fast_rng::rand::Rng; -use tokio::time::{sleep, Duration, Instant}; +use tokio::time::{Duration, Instant}; #[derive(Debug)] pub enum OpenRequestResult { @@ -75,7 +76,7 @@ impl Authorization { self: Arc, method: Method, params: EthCallFirstParams, - ) -> anyhow::Result<()> { + ) -> Web3ProxyResult<()> { let rpc_key_id = match self.checks.rpc_secret_key_id { Some(rpc_key_id) => rpc_key_id.into(), None => { diff --git a/web3_proxy/src/rpcs/transactions.rs b/web3_proxy/src/rpcs/transactions.rs index c46da986..c9602e6c 100644 --- a/web3_proxy/src/rpcs/transactions.rs +++ b/web3_proxy/src/rpcs/transactions.rs @@ -1,4 +1,4 @@ -use crate::frontend::authorization::Authorization; +use crate::frontend::{authorization::Authorization, errors::Web3ProxyResult}; use super::many::Web3Rpcs; ///! Load balanced communication with a group of web3 providers @@ -70,7 +70,7 @@ impl Web3Rpcs { rpc: Arc, pending_tx_id: TxHash, pending_tx_sender: broadcast::Sender, - ) -> anyhow::Result<()> { + ) -> Web3ProxyResult<()> { // TODO: how many retries? until some timestamp is hit is probably better. maybe just loop and call this with a timeout // TODO: after more investigation, i don't think retries will help. i think this is because chains of transactions get dropped from memory // TODO: also check the "confirmed transactions" mapping? maybe one shared mapping with TxState in it? diff --git a/web3_proxy/src/stats/mod.rs b/web3_proxy/src/stats/mod.rs index deb8563d..0aedb115 100644 --- a/web3_proxy/src/stats/mod.rs +++ b/web3_proxy/src/stats/mod.rs @@ -8,9 +8,9 @@ pub use stat_buffer::{SpawnedStatBuffer, StatBuffer}; use crate::app::RpcSecretKeyCache; use crate::frontend::authorization::{Authorization, RequestMetadata}; -use crate::frontend::errors::Web3ProxyError; +use crate::frontend::errors::{Web3ProxyError, Web3ProxyResult}; use crate::rpcs::one::Web3Rpc; -use anyhow::Context; +use anyhow::{anyhow, Context}; use axum::headers::Origin; use chrono::{DateTime, Months, TimeZone, Utc}; use derive_more::From; @@ -229,13 +229,14 @@ impl BufferedRpcQueryStats { db_conn: &DatabaseConnection, key: RpcQueryKey, rpc_secret_key_cache: Option<&RpcSecretKeyCache>, - ) -> anyhow::Result<()> { - anyhow::ensure!( - key.response_timestamp > 0, - "no response_timestamp! This is a bug! {:?} {:?}", - key, - self - ); + ) -> Web3ProxyResult<()> { + if key.response_timestamp == 0 { + return Err(Web3ProxyError::Anyhow(anyhow!( + "no response_timestamp! This is a bug! {:?} {:?}", + key, + self + ))); + } let period_datetime = Utc.timestamp_opt(key.response_timestamp, 0).unwrap(); @@ -670,8 +671,9 @@ impl RpcQueryStats { method: Option<&str>, ) -> Decimal { // for now, always return 0 for cost - return 0.into(); + 0.into() + /* // some methods should be free. there might be cases where method isn't set (though they should be uncommon) // TODO: get this list from config (and add more to it) if let Some(method) = method.as_ref() { @@ -704,5 +706,6 @@ impl RpcQueryStats { } cost + */ } } diff --git a/web3_proxy/src/stats/stat_buffer.rs b/web3_proxy/src/stats/stat_buffer.rs index f8fd2db8..c7028204 100644 --- a/web3_proxy/src/stats/stat_buffer.rs +++ b/web3_proxy/src/stats/stat_buffer.rs @@ -1,5 +1,6 @@ use super::{AppStat, RpcQueryKey}; -use crate::app::RpcSecretKeyCache; +use crate::app::{RpcSecretKeyCache, Web3ProxyJoinHandle}; +use crate::frontend::errors::Web3ProxyResult; use derive_more::From; use futures::stream; use hashbrown::HashMap; @@ -9,7 +10,6 @@ use migration::sea_orm::prelude::Decimal; use migration::sea_orm::DatabaseConnection; use std::time::Duration; use tokio::sync::broadcast; -use tokio::task::JoinHandle; use tokio::time::interval; #[derive(Debug, Default)] @@ -32,7 +32,7 @@ pub struct BufferedRpcQueryStats { pub struct SpawnedStatBuffer { pub stat_sender: flume::Sender, /// these handles are important and must be allowed to finish - pub background_handle: JoinHandle>, + pub background_handle: Web3ProxyJoinHandle<()>, } pub struct StatBuffer { accounting_db_buffer: HashMap, @@ -96,7 +96,7 @@ impl StatBuffer { bucket: String, stat_receiver: flume::Receiver, mut shutdown_receiver: broadcast::Receiver<()>, - ) -> anyhow::Result<()> { + ) -> Web3ProxyResult<()> { let mut tsdb_save_interval = interval(Duration::from_secs(self.tsdb_save_interval_seconds as u64)); let mut db_save_interval = From 651f0fcad3a4ecd801351f816258eeddf555d0a0 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 23 May 2023 14:51:34 -0700 Subject: [PATCH 57/66] lint --- web3_proxy/src/frontend/admin.rs | 16 +++++----------- web3_proxy/src/rpcs/request.rs | 4 +++- 2 files changed, 8 insertions(+), 12 deletions(-) diff --git a/web3_proxy/src/frontend/admin.rs b/web3_proxy/src/frontend/admin.rs index e2527b93..ba4d4eaa 100644 --- a/web3_proxy/src/frontend/admin.rs +++ b/web3_proxy/src/frontend/admin.rs @@ -5,11 +5,9 @@ use super::errors::Web3ProxyResponse; use crate::admin_queries::query_admin_modify_usertier; use crate::app::Web3ProxyApp; use crate::frontend::errors::{Web3ProxyError, Web3ProxyErrorContext}; -use crate::http_params::get_user_id_from_params; use crate::user_token::UserBearerToken; use crate::PostLogin; use anyhow::Context; -use axum::body::HttpBody; use axum::{ extract::{Path, Query}, headers::{authorization::Bearer, Authorization}, @@ -29,10 +27,9 @@ use http::StatusCode; use log::{debug, info, warn}; use migration::sea_orm::prelude::{Decimal, Uuid}; use migration::sea_orm::{ - self, ActiveModelTrait, ActiveValue, ColumnTrait, EntityTrait, IntoActiveModel, QueryFilter, - TransactionTrait, Update, + self, ActiveModelTrait, ColumnTrait, EntityTrait, IntoActiveModel, QueryFilter, }; -use migration::{ConnectionTrait, Expr, OnConflict}; +use migration::{Expr, OnConflict}; use serde_json::json; use siwe::{Message, VerificationOpts}; use std::ops::Add; @@ -76,7 +73,7 @@ pub async fn admin_increase_balance( .map_err(|_| { Web3ProxyError::BadRequest("Unable to parse user_address as an Address".to_string()) })?; - let user_address_bytes: Vec = user_address.clone().to_fixed_bytes().into(); + let user_address_bytes: Vec = user_address.to_fixed_bytes().into(); let note: String = params .get("note") .ok_or_else(|| { @@ -94,11 +91,8 @@ pub async fn admin_increase_balance( Web3ProxyError::BadRequest("Unable to get the amount key from the request".to_string()) }) .map(|x| Decimal::from_str(x))? - .or_else(|err| { - Err(Web3ProxyError::BadRequest(format!( - "Unable to parse amount from the request {:?}", - err - ))) + .map_err(|err| { + Web3ProxyError::BadRequest(format!("Unable to parse amount from the request {:?}", err)) })?; let user_entry: user::Model = user::Entity::find() diff --git a/web3_proxy/src/rpcs/request.rs b/web3_proxy/src/rpcs/request.rs index 9e91f75c..7a8a0003 100644 --- a/web3_proxy/src/rpcs/request.rs +++ b/web3_proxy/src/rpcs/request.rs @@ -184,7 +184,9 @@ impl OpenRequestHandle { } else if let Some(ref p) = self.rpc.ws_provider { p.request(method, params).await } else { - unimplemented!("no provider. cannot send request") + return Err(ProviderError::CustomError( + "no provider configured!".to_string(), + )); }; // we do NOT want to measure errors, so we intentionally do not record this latency now. From 77c9b0ab126eed710eeb9abe6f535e7f9e68c213 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 23 May 2023 14:52:45 -0700 Subject: [PATCH 58/66] peak latency instead of head latency --- web3_proxy/src/rpcs/one.rs | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index e3e3b0ea..89825e29 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -227,18 +227,16 @@ impl Web3Rpc { } pub fn peak_ewma(&self) -> OrderedFloat { - // TODO: bug inside peak ewma somewhere. possible with atomics being relaxed or the conversion to pair and back - // let peak_latency = if let Some(peak_latency) = self.peak_latency.as_ref() { - // peak_latency.latency().as_secs_f64() - // } else { - // 0.0 - // }; - let head_latency = self.head_latency.read().value(); + let peak_latency = if let Some(peak_latency) = self.peak_latency.as_ref() { + peak_latency.latency().as_secs_f64() + } else { + 1.0 + }; // TODO: what ordering? let active_requests = self.active_requests.load(atomic::Ordering::Acquire) as f64 + 1.0; - OrderedFloat(head_latency * active_requests) + OrderedFloat(peak_latency * active_requests) } // TODO: would be great if rpcs exposed this. see https://github.com/ledgerwatch/erigon/issues/6391 From dd347fd91663a7d06210df7929954654bbf42c15 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 23 May 2023 14:57:24 -0700 Subject: [PATCH 59/66] add units to serialized variables --- web3_proxy/src/rpcs/one.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/web3_proxy/src/rpcs/one.rs b/web3_proxy/src/rpcs/one.rs index 89825e29..c74a156e 100644 --- a/web3_proxy/src/rpcs/one.rs +++ b/web3_proxy/src/rpcs/one.rs @@ -1012,14 +1012,14 @@ impl Serialize for Web3Rpc { &self.active_requests.load(atomic::Ordering::Relaxed), )?; - state.serialize_field("head_latency", &self.head_latency.read().value())?; + state.serialize_field("head_latency_ms", &self.head_latency.read().value())?; state.serialize_field( - "peak_latency", + "peak_latency_ms", &self.peak_latency.as_ref().unwrap().latency().as_millis(), )?; - state.serialize_field("peak_ewma", self.peak_ewma().as_ref())?; + state.serialize_field("peak_ewma_s", self.peak_ewma().as_ref())?; state.end() } From 4c7a7b87cb2ff5dd571532e1a4676d8e0f930c5b Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 23 May 2023 14:58:13 -0700 Subject: [PATCH 60/66] cargo upgrade --- Cargo.lock | 60 +++++++++++++++++++++---------------------- web3_proxy/Cargo.toml | 2 +- 2 files changed, 31 insertions(+), 31 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7a79b0c2..12339225 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -263,7 +263,7 @@ checksum = "f8175979259124331c1d7bf6586ee7e0da434155e4b2d48ec2c8386281d8df39" dependencies = [ "async-trait", "axum-core", - "base64 0.21.0", + "base64 0.21.1", "bitflags", "bytes", "futures-util", @@ -379,9 +379,9 @@ checksum = "9e1b586273c5702936fe7b7d6896644d8be71e6314cfe09d3167c95f712589e8" [[package]] name = "base64" -version = "0.21.0" +version = "0.21.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4a4ddaa51a5bc52a6948f74c06d20aaaddb71924eab79b8c97a8c556e942d6a" +checksum = "3f1e31e207a6b8fb791a38ea3105e6cb541f55e4d029902d3039a4ad07cc4105" [[package]] name = "base64ct" @@ -532,9 +532,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.12.2" +version = "3.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c6ed94e98ecff0c12dd1b04c15ec0d7d9458ca8fe806cea6f12954efe74c63b" +checksum = "a3e2c3daef883ecc1b5d58c15adae93470a91d425f3532ba1695849656af3fc1" [[package]] name = "byte-slice-cast" @@ -797,7 +797,7 @@ version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9b949a1c63fb7eb591eb7ba438746326aedf0ae843e51ec92ba6bec5bb382c4f" dependencies = [ - "base64 0.21.0", + "base64 0.21.1", "bech32", "bs58", "digest 0.10.6", @@ -1812,7 +1812,7 @@ checksum = "1009041f40476b972b5d79346cc512e97c662b1a0a2f78285eabe9a122909783" dependencies = [ "async-trait", "auto_impl", - "base64 0.21.0", + "base64 0.21.1", "bytes", "enr", "ethers-core", @@ -3001,9 +3001,9 @@ dependencies = [ [[package]] name = "linux-raw-sys" -version = "0.3.7" +version = "0.3.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ece97ea872ece730aed82664c424eb4c8291e1ff2480247ccf7409044bc6479f" +checksum = "ef53942eb7bf7ff43a617b3e2c1c4a5ecf5944a7c1bc12d7ee39bbb15e5c1519" [[package]] name = "listenfd" @@ -4324,7 +4324,7 @@ version = "0.11.18" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cde824a14b7c14f85caff81225f411faacc04a2013f41670f41443742b1c1c55" dependencies = [ - "base64 0.21.0", + "base64 0.21.1", "bytes", "encoding_rs", "futures-core", @@ -4585,7 +4585,7 @@ version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d194b56d58803a43635bdc398cd17e383d6f71f9182b9a192c127ca42494a59b" dependencies = [ - "base64 0.21.0", + "base64 0.21.1", ] [[package]] @@ -4900,9 +4900,9 @@ dependencies = [ [[package]] name = "security-framework" -version = "2.9.0" +version = "2.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca2855b3715770894e67cbfa3df957790aa0c9edc3bf06efa1a84d77fa0839d1" +checksum = "1fc758eb7bffce5b308734e9b0c1468893cae9ff70ebf13e7090be8dcbcc83a8" dependencies = [ "bitflags", "core-foundation", @@ -4944,9 +4944,9 @@ checksum = "cd0b0ec5f1c1ca621c432a25813d8d60c88abe6d3e08a3eb9cf37d97a0fe3d73" [[package]] name = "sentry" -version = "0.31.1" +version = "0.31.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37dd6c0cdca6b1d1ca44cde7fff289f2592a97965afec870faa7b81b9fc87745" +checksum = "234f6e133d27140ad5ea3b369a7665f7fbc060fe246f81d8168665b38c08b600" dependencies = [ "httpdate", "reqwest", @@ -4964,9 +4964,9 @@ dependencies = [ [[package]] name = "sentry-anyhow" -version = "0.31.1" +version = "0.31.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9c3d7032fff178c77c107c32c6d3337b12847adf67165ccc876c898e7154b00" +checksum = "47e940be4c63b29006b4ac422cacea932c2cb5f8c209647ee86446ed27595a42" dependencies = [ "anyhow", "sentry-backtrace", @@ -4975,9 +4975,9 @@ dependencies = [ [[package]] name = "sentry-backtrace" -version = "0.31.1" +version = "0.31.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c029fe8317cdd75cb2b52c600bab4e2ef64c552198e669ba874340447f330962" +checksum = "d89b6b53de06308dd5ac08934b597bcd72a9aae0c20bc3ab06da69cb34d468e3" dependencies = [ "backtrace", "once_cell", @@ -4987,9 +4987,9 @@ dependencies = [ [[package]] name = "sentry-contexts" -version = "0.31.1" +version = "0.31.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc575098d73c8b942b589ab453b06e4c43527556dd8f95532220d1b54d7c6b4b" +checksum = "0769b66763e59976cd5c0fd817dcd51ccce404de8bebac0cd0e886c55b0fffa8" dependencies = [ "hostname", "libc", @@ -5001,9 +5001,9 @@ dependencies = [ [[package]] name = "sentry-core" -version = "0.31.1" +version = "0.31.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "20216140001bbf05895f013abd0dae4df58faee24e016d54cbf107f070bac56b" +checksum = "a1f954f1b89e8cd82576dc49bfab80304c9a6201343b4fe5c68c819f7a9bbed2" dependencies = [ "once_cell", "rand", @@ -5014,9 +5014,9 @@ dependencies = [ [[package]] name = "sentry-log" -version = "0.31.1" +version = "0.31.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a43934e48e9c8e2c7d0dcb9c6cbcfcbe3ee109a14fc0c821e8944acd4faa2c25" +checksum = "958f539c85854acf7fa0fa46f07077be9050c7b28602ddfb5a651e9d11b27740" dependencies = [ "log", "sentry-core", @@ -5024,9 +5024,9 @@ dependencies = [ [[package]] name = "sentry-panic" -version = "0.31.1" +version = "0.31.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e45cd0a113fc06d6edba01732010518816cdc8ce3bccc70f5e41570046bf046" +checksum = "94dc2ab494362ad51308c7c19f44e9ab70e426a931621e4a05f378a1e74558c2" dependencies = [ "sentry-backtrace", "sentry-core", @@ -5034,9 +5034,9 @@ dependencies = [ [[package]] name = "sentry-types" -version = "0.31.1" +version = "0.31.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7f6959d8cb3a77be27e588eef6ce9a2a469651a556d9de662e4d07e5ace4232" +checksum = "85c53caf80cb1c6fcdf4d82b7bfff8477f50841e4caad7bf8e5e57a152b564cb" dependencies = [ "debugid", "getrandom", @@ -5989,7 +5989,7 @@ checksum = "3082666a3a6433f7f511c7192923fa1fe07c69332d3c6a2e6bb040b569199d5a" dependencies = [ "async-trait", "axum", - "base64 0.21.0", + "base64 0.21.1", "bytes", "futures-core", "futures-util", diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index 17072f1c..61334872 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -75,7 +75,7 @@ rdkafka = { version = "0.31.0" } regex = "1.8.2" reqwest = { version = "0.11.18", default-features = false, features = ["json", "tokio-rustls"] } rmp-serde = "1.1.1" -sentry = { version = "0.31.1", default-features = false, features = ["backtrace", "contexts", "panic", "anyhow", "reqwest", "rustls", "log", "sentry-log"] } +sentry = { version = "0.31.2", default-features = false, features = ["backtrace", "contexts", "panic", "anyhow", "reqwest", "rustls", "log", "sentry-log"] } serde = { version = "1.0.163", features = [] } serde_json = { version = "1.0.96", default-features = false, features = ["alloc", "raw_value"] } serde_prometheus = "0.2.2" From 55f9c5782707bc02fb63f3017688324fb433747b Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 23 May 2023 15:03:16 -0700 Subject: [PATCH 61/66] include error in warning --- web3_proxy/src/rpcs/many.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index 310a5686..5bad1334 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -877,7 +877,11 @@ impl Web3Rpcs { let rate_limit_substrings = ["limit", "exceeded", "quota usage"]; for rate_limit_substr in rate_limit_substrings { if error_msg.contains(rate_limit_substr) { - warn!("rate limited by {}", skip_rpcs.last().unwrap()); + warn!( + "rate limited ({}) by {}", + error_msg, + skip_rpcs.last().unwrap() + ); continue; } } From dafb69fae1268d9ac8d73f7d1fb088053f286a6f Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 23 May 2023 15:50:29 -0700 Subject: [PATCH 62/66] move timeout deeper --- web3_proxy/src/app/mod.rs | 76 +++++++++++++++++++-------------------- 1 file changed, 37 insertions(+), 39 deletions(-) diff --git a/web3_proxy/src/app/mod.rs b/web3_proxy/src/app/mod.rs index 42003674..a6ade181 100644 --- a/web3_proxy/src/app/mod.rs +++ b/web3_proxy/src/app/mod.rs @@ -1011,20 +1011,12 @@ impl Web3ProxyApp { ) -> Web3ProxyResult<(StatusCode, JsonRpcForwardedResponseEnum, Vec>)> { // trace!(?request, "proxy_web3_rpc"); - // even though we have timeouts on the requests to our backend providers, - // we need a timeout for the incoming request so that retries don't run forever - // TODO: take this as an optional argument. check for a different max from the user_tier? - // TODO: how much time was spent on this request alredy? - let max_time = Duration::from_secs(240); - // TODO: use streams and buffers so we don't overwhelm our server let response = match request { JsonRpcRequestEnum::Single(mut request) => { - let (status_code, response, rpcs) = timeout( - max_time, - self.proxy_cached_request(&authorization, &mut request, None), - ) - .await?; + let (status_code, response, rpcs) = self + .proxy_cached_request(&authorization, &mut request, None) + .await; ( status_code, @@ -1033,11 +1025,9 @@ impl Web3ProxyApp { ) } JsonRpcRequestEnum::Batch(requests) => { - let (responses, rpcs) = timeout( - max_time, - self.proxy_web3_rpc_requests(&authorization, requests), - ) - .await??; + let (responses, rpcs) = self + .proxy_web3_rpc_requests(&authorization, requests) + .await?; // TODO: real status code ( @@ -1331,7 +1321,8 @@ impl Web3ProxyApp { | "eth_getUserOperationReceipt" | "eth_supportedEntryPoints") => match self.bundler_4337_rpcs.as_ref() { Some(bundler_4337_rpcs) => { - bundler_4337_rpcs + // TODO: timeout + bundler_4337_rpcs .try_proxy_connection( authorization, request, @@ -1367,6 +1358,7 @@ impl Web3ProxyApp { JsonRpcResponseData::from(json!(Address::zero())) } "eth_estimateGas" => { + // TODO: timeout let response_data = self .balanced_rpcs .try_proxy_connection( @@ -1403,6 +1395,7 @@ impl Web3ProxyApp { } "eth_getTransactionReceipt" | "eth_getTransactionByHash" => { // try to get the transaction without specifying a min_block_height + // TODO: timeout let mut response_data = self .balanced_rpcs .try_proxy_connection( @@ -1446,12 +1439,7 @@ impl Web3ProxyApp { // TODO: error if the chain_id is incorrect - // TODO: check the cache to see if we have sent this transaction recently - // TODO: if so, use a cached response. - // TODO: if not, - // TODO: - cache successes for up to 1 minute - // TODO: - cache failures for 1 block (we see transactions skipped because out of funds. but that can change block to block) - + // TODO: timeout let mut response_data = self .try_send_protected( authorization, @@ -1581,6 +1569,7 @@ impl Web3ProxyApp { , "web3_sha3" => { // returns Keccak-256 (not the standardized SHA3-256) of the given data. + // TODO: timeout match &request.params { Some(serde_json::Value::Array(params)) => { // TODO: make a struct and use serde conversion to clean this up @@ -1740,6 +1729,9 @@ impl Web3ProxyApp { let authorization = authorization.clone(); + // TODO: different timeouts for different user tiers + let duration = Duration::from_secs(240); + if let Some(cache_key) = cache_key { let from_block_num = cache_key.from_block.as_ref().map(|x| x.number.unwrap()); let to_block_num = cache_key.to_block.as_ref().map(|x| x.number.unwrap()); @@ -1750,15 +1742,18 @@ impl Web3ProxyApp { { Ok(x) => x, Err(x) => { - let response_data = self.balanced_rpcs - .try_proxy_connection( - &authorization, - request, - Some(request_metadata), - from_block_num.as_ref(), - to_block_num.as_ref(), + let response_data = timeout( + duration, + self.balanced_rpcs + .try_proxy_connection( + &authorization, + request, + Some(request_metadata), + from_block_num.as_ref(), + to_block_num.as_ref(), + ) ) - .await?; + .await??; // TODO: convert the Box to an Arc x.insert(response_data.clone()); @@ -1767,15 +1762,18 @@ impl Web3ProxyApp { } } } else { - self.balanced_rpcs - .try_proxy_connection( - &authorization, - request, - Some(request_metadata), - None, - None, + timeout( + duration, + self.balanced_rpcs + .try_proxy_connection( + &authorization, + request, + Some(request_metadata), + None, + None, + ) ) - .await? + .await?? } } }; From b1a0bcac575542aea3c11f413ba88dd0d877e46e Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 23 May 2023 20:46:27 -0700 Subject: [PATCH 63/66] add max wait to fix tests --- web3_proxy/src/frontend/users/payment.rs | 6 +- web3_proxy/src/rpcs/many.rs | 86 ++++++++++++++++++------ 2 files changed, 70 insertions(+), 22 deletions(-) diff --git a/web3_proxy/src/frontend/users/payment.rs b/web3_proxy/src/frontend/users/payment.rs index 728728d0..1ceb8532 100644 --- a/web3_proxy/src/frontend/users/payment.rs +++ b/web3_proxy/src/frontend/users/payment.rs @@ -153,7 +153,7 @@ pub async fn user_balance_post( // Just make an rpc request, idk if i need to call this super extensive code let transaction_receipt: TransactionReceipt = match app .balanced_rpcs - .wait_for_best_rpc(&authorization, None, &mut vec![], None, None) + .wait_for_best_rpc(&authorization, None, &mut vec![], None, None, None) .await { Ok(OpenRequestResult::Handle(handle)) => { @@ -187,7 +187,7 @@ pub async fn user_balance_post( debug!("Transaction receipt is: {:?}", transaction_receipt); let accepted_token: Address = match app .balanced_rpcs - .wait_for_best_rpc(&authorization, None, &mut vec![], None, None) + .wait_for_best_rpc(&authorization, None, &mut vec![], None, None, None) .await { Ok(OpenRequestResult::Handle(handle)) => { @@ -242,7 +242,7 @@ pub async fn user_balance_post( debug!("Accepted token is: {:?}", accepted_token); let decimals: u32 = match app .balanced_rpcs - .wait_for_best_rpc(&authorization, None, &mut vec![], None, None) + .wait_for_best_rpc(&authorization, None, &mut vec![], None, None, None) .await { Ok(OpenRequestResult::Handle(handle)) => { diff --git a/web3_proxy/src/rpcs/many.rs b/web3_proxy/src/rpcs/many.rs index 5bad1334..4bf536cf 100644 --- a/web3_proxy/src/rpcs/many.rs +++ b/web3_proxy/src/rpcs/many.rs @@ -34,6 +34,7 @@ use std::fmt::{self, Display}; use std::sync::atomic::Ordering; use std::sync::Arc; use thread_fast_rng::rand::seq::SliceRandom; +use tokio::select; use tokio::sync::{broadcast, watch}; use tokio::time::{sleep, sleep_until, Duration, Instant}; @@ -485,6 +486,7 @@ impl Web3Rpcs { skip_rpcs: &mut Vec>, min_block_needed: Option<&U64>, max_block_needed: Option<&U64>, + max_wait: Option, ) -> Web3ProxyResult { let mut earliest_retry_at: Option = None; @@ -526,16 +528,14 @@ impl Web3Rpcs { } } } else { - let start = Instant::now(); - - // TODO: get from config or argument - let max_wait = Duration::from_secs(10); + let stop_trying_at = + Instant::now() + max_wait.unwrap_or_else(|| Duration::from_secs(10)); let mut watch_consensus_rpcs = self.watch_consensus_rpcs_sender.subscribe(); let mut potential_rpcs = Vec::with_capacity(self.by_name.load().len()); - while start.elapsed() < max_wait { + loop { let consensus_rpcs = watch_consensus_rpcs.borrow_and_update().clone(); potential_rpcs.clear(); @@ -653,12 +653,16 @@ impl Web3Rpcs { match consensus_rpcs.should_wait_for_block(waiting_for, skip_rpcs) { ShouldWaitForBlock::NeverReady => break, - ShouldWaitForBlock::Ready => continue, - ShouldWaitForBlock::Wait { .. } => {} + ShouldWaitForBlock::Ready => {} + ShouldWaitForBlock::Wait { .. } => select! { + _ = watch_consensus_rpcs.changed() => {}, + _ = sleep_until(stop_trying_at) => {}, + }, } + } - // TODO: select on consensus_rpcs changing and on earliest_retry_at - watch_consensus_rpcs.changed().await?; + if Instant::now() > stop_trying_at { + break; } } } @@ -823,6 +827,7 @@ impl Web3Rpcs { &mut skip_rpcs, min_block_needed, max_block_needed, + None, ) .await? { @@ -1493,6 +1498,7 @@ mod tests { &mut vec![], Some(head_block.number.as_ref().unwrap()), None, + Some(Duration::from_secs(0)), ) .await .unwrap(); @@ -1584,28 +1590,56 @@ mod tests { // TODO: make sure the handle is for the expected rpc assert!(matches!( - rpcs.wait_for_best_rpc(&authorization, None, &mut vec![], None, None) - .await, + rpcs.wait_for_best_rpc( + &authorization, + None, + &mut vec![], + None, + None, + Some(Duration::from_secs(0)) + ) + .await, Ok(OpenRequestResult::Handle(_)) )); // TODO: make sure the handle is for the expected rpc assert!(matches!( - rpcs.wait_for_best_rpc(&authorization, None, &mut vec![], Some(&0.into()), None) - .await, + rpcs.wait_for_best_rpc( + &authorization, + None, + &mut vec![], + Some(&0.into()), + None, + Some(Duration::from_secs(0)), + ) + .await, Ok(OpenRequestResult::Handle(_)) )); // TODO: make sure the handle is for the expected rpc assert!(matches!( - rpcs.wait_for_best_rpc(&authorization, None, &mut vec![], Some(&1.into()), None) - .await, + rpcs.wait_for_best_rpc( + &authorization, + None, + &mut vec![], + Some(&1.into()), + None, + Some(Duration::from_secs(0)), + ) + .await, Ok(OpenRequestResult::Handle(_)) )); // future block should not get a handle let future_rpc = rpcs - .wait_for_best_rpc(&authorization, None, &mut vec![], Some(&2.into()), None) + .wait_for_best_rpc( + &authorization, + None, + &mut vec![], + Some(&2.into()), + None, + Some(Duration::from_secs(0)), + ) .await; assert!(matches!(future_rpc, Ok(OpenRequestResult::NotReady))); } @@ -1675,7 +1709,6 @@ mod tests { let (watch_consensus_rpcs_sender, _) = watch::channel(None); let (watch_consensus_head_sender, _watch_consensus_head_receiver) = watch::channel(None); - // TODO: make a Web3Rpcs::new let rpcs = Web3Rpcs { block_sender, by_name: ArcSwap::from_pointee(rpcs_by_name), @@ -1733,6 +1766,7 @@ mod tests { &mut vec![], Some(head_block.number()), None, + Some(Duration::from_secs(0)), ) .await; @@ -1744,13 +1778,27 @@ mod tests { )); let _best_available_server_from_none = rpcs - .wait_for_best_rpc(&authorization, None, &mut vec![], None, None) + .wait_for_best_rpc( + &authorization, + None, + &mut vec![], + None, + None, + Some(Duration::from_secs(0)), + ) .await; // assert_eq!(best_available_server, best_available_server_from_none); let best_archive_server = rpcs - .wait_for_best_rpc(&authorization, None, &mut vec![], Some(&1.into()), None) + .wait_for_best_rpc( + &authorization, + None, + &mut vec![], + Some(&1.into()), + None, + Some(Duration::from_secs(0)), + ) .await; match best_archive_server { From 0d15d5baf7fbcf75a6172c1c8b2b6f04daaad722 Mon Sep 17 00:00:00 2001 From: Bryan Stitt Date: Tue, 23 May 2023 20:58:29 -0700 Subject: [PATCH 64/66] bump versions to match number of migrations --- Cargo.lock | 6 +++--- entities/Cargo.toml | 2 +- migration/Cargo.toml | 2 +- web3_proxy/Cargo.toml | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 12339225..0c211658 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1526,7 +1526,7 @@ dependencies = [ [[package]] name = "entities" -version = "0.27.0" +version = "0.28.0" dependencies = [ "ethers", "sea-orm", @@ -3091,7 +3091,7 @@ dependencies = [ [[package]] name = "migration" -version = "0.27.0" +version = "0.28.0" dependencies = [ "sea-orm-migration", "tokio", @@ -6503,7 +6503,7 @@ dependencies = [ [[package]] name = "web3_proxy" -version = "0.27.0" +version = "0.28.0" dependencies = [ "anyhow", "arc-swap", diff --git a/entities/Cargo.toml b/entities/Cargo.toml index 0c6b4df8..b0cf89e7 100644 --- a/entities/Cargo.toml +++ b/entities/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "entities" -version = "0.27.0" +version = "0.28.0" edition = "2021" [lib] diff --git a/migration/Cargo.toml b/migration/Cargo.toml index 97f07b5e..bd2e3b11 100644 --- a/migration/Cargo.toml +++ b/migration/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "migration" -version = "0.27.0" +version = "0.28.0" edition = "2021" publish = false diff --git a/web3_proxy/Cargo.toml b/web3_proxy/Cargo.toml index 61334872..0f4c2af2 100644 --- a/web3_proxy/Cargo.toml +++ b/web3_proxy/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "web3_proxy" -version = "0.27.0" +version = "0.28.0" edition = "2021" default-run = "web3_proxy_cli" From 47f07261edec7f911c4a0e2885d141ccfb908ded Mon Sep 17 00:00:00 2001 From: yenicelik Date: Wed, 24 May 2023 13:34:50 +0200 Subject: [PATCH 65/66] detailed request must come with a bearer token --- web3_proxy/src/stats/influxdb_queries.rs | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/web3_proxy/src/stats/influxdb_queries.rs b/web3_proxy/src/stats/influxdb_queries.rs index 69d1110c..6246ac54 100644 --- a/web3_proxy/src/stats/influxdb_queries.rs +++ b/web3_proxy/src/stats/influxdb_queries.rs @@ -41,6 +41,13 @@ pub async fn query_user_stats<'a>( None => 0, }; + // Return an error if the bearer is set, but the StatType is Detailed + if stat_response_type == StatType::Detailed && user_id == 0 { + return Err(Web3ProxyError::BadRequest( + "Detailed Stats Response requires you to authorize with a bearer token".to_owned(), + )); + } + let db_replica = app .db_replica() .context("query_user_stats needs a db replica")?; From b6cab88c4eb5678f08af3422085273e769e93b3f Mon Sep 17 00:00:00 2001 From: yenicelik Date: Wed, 24 May 2023 13:35:10 +0200 Subject: [PATCH 66/66] removed joined_on variable that was not used --- web3_proxy/src/stats/influxdb_queries.rs | 18 +----------------- 1 file changed, 1 insertion(+), 17 deletions(-) diff --git a/web3_proxy/src/stats/influxdb_queries.rs b/web3_proxy/src/stats/influxdb_queries.rs index 6246ac54..4b597118 100644 --- a/web3_proxy/src/stats/influxdb_queries.rs +++ b/web3_proxy/src/stats/influxdb_queries.rs @@ -77,14 +77,6 @@ pub async fn query_user_stats<'a>( "opt_in_proxy" }; - let mut join_candidates: Vec = vec![ - "_time".to_string(), - "_measurement".to_string(), - "archive_needed".to_string(), - "chain_id".to_string(), - "error_response".to_string(), - ]; - // Include a hashmap to go from rpc_secret_key_id to the rpc_secret_key let mut rpc_key_id_to_key = HashMap::new(); @@ -140,9 +132,6 @@ pub async fn query_user_stats<'a>( )); } - // Make the tables join on the rpc_key_id as well: - join_candidates.push("rpc_secret_key_id".to_string()); - // Iterate, pop and add to string f!( r#"|> filter(fn: (r) => contains(value: r["rpc_secret_key_id"], set: {:?}))"#, @@ -177,13 +166,8 @@ pub async fn query_user_stats<'a>( let drop_method = match stat_response_type { StatType::Aggregated => f!(r#"|> drop(columns: ["method"])"#), - StatType::Detailed => { - // Make the tables join on the method column as well - join_candidates.push("method".to_string()); - "".to_string() - } + StatType::Detailed => "".to_string(), }; - let join_candidates = f!(r#"{:?}"#, join_candidates); let query = f!(r#" base = from(bucket: "{bucket}")