From c7ec54ea69f529a4077d48609cab9faf82db7848 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Tue, 9 Apr 2024 19:00:22 -0400 Subject: [PATCH 01/46] Attempt at smarter HBONE pooling between ztunnels Signed-off-by: Benjamin Leggett --- Cargo.lock | 49 +++ Cargo.toml | 2 + src/config.rs | 18 +- src/identity/manager.rs | 2 + src/proxy.rs | 27 +- src/proxy/inbound.rs | 1 + src/proxy/outbound.rs | 97 +--- src/proxy/pool.rs | 951 ++++++++++++++++++++++++++++++++-------- src/proxyfactory.rs | 6 + 9 files changed, 892 insertions(+), 261 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 91fb7fd58..8a72c3da4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -573,6 +573,15 @@ dependencies = [ "crossbeam-utils", ] +[[package]] +name = "crossbeam-queue" +version = "0.3.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "df0346b5d5e76ac2fe4e327c5fd1118d6be7c51dfb18f9b7922923f287471e35" +dependencies = [ + "crossbeam-utils", +] + [[package]] name = "crossbeam-utils" version = "0.8.19" @@ -1550,6 +1559,15 @@ version = "0.4.21" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c" +[[package]] +name = "lru" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d3262e75e648fce39813cb56ac41f3c3e3f65217ebf3844d818d1f9398cfb0dc" +dependencies = [ + "hashbrown 0.14.3", +] + [[package]] name = "lru-cache" version = "0.1.2" @@ -1978,6 +1996,35 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" +[[package]] +name = "pingora-pool" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b4569e3bef52b0abab239a5cf3287c71307615ca61be7fc7799d71fdaab33d81" +dependencies = [ + "crossbeam-queue", + "log", + "lru", + "parking_lot", + "pingora-timeout", + "thread_local", + "tokio", +] + +[[package]] +name = "pingora-timeout" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0be182194d34e1b28608eaa49ee0fb86e5b7ab1d21a1d7a2b4d402446fda47e1" +dependencies = [ + "futures", + "once_cell", + "parking_lot", + "pin-project-lite", + "thread_local", + "tokio", +] + [[package]] name = "plotters" version = "0.3.5" @@ -3658,6 +3705,8 @@ dependencies = [ "nix 0.28.0", "oid-registry", "once_cell", + "pin-project-lite", + "pingora-pool", "ppp", "pprof", "prometheus-client", diff --git a/Cargo.toml b/Cargo.toml index e46d367b2..e2028a336 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -97,6 +97,8 @@ url = "2.2" x509-parser = { version = "0.16", default-features = false } tracing-log = "0.2" backoff = "0.4.0" +pin-project-lite = "0.2" +pingora-pool = "0.1.0" [target.'cfg(target_os = "linux")'.dependencies] netns-rs = "0.1" diff --git a/src/config.rs b/src/config.rs index 0af5ca85a..3cacb53f2 100644 --- a/src/config.rs +++ b/src/config.rs @@ -49,6 +49,7 @@ const CA_ADDRESS: &str = "CA_ADDRESS"; const SECRET_TTL: &str = "SECRET_TTL"; const FAKE_CA: &str = "FAKE_CA"; const ZTUNNEL_WORKER_THREADS: &str = "ZTUNNEL_WORKER_THREADS"; +const POOL_MAX_STREAMS_PER_CONNECTION: &str = "POOL_MAX_STREAMS_PER_CONNECTION"; const ENABLE_ORIG_SRC: &str = "ENABLE_ORIG_SRC"; const PROXY_CONFIG: &str = "PROXY_CONFIG"; @@ -125,6 +126,19 @@ pub struct Config { pub connection_window_size: u32, pub frame_size: u32, + // The limit of how many streams a single HBONE pool connection will be limited to, before + // spawning a new conn rather than reusing an existing one, even to a dest that already has an open connection. + // + // This can be used to effect flow control for "connection storms" when workload clients + // (such as loadgen clients) open many connections all at once. + // + // Note that this will only be checked and inner conns rebalanced accordingly when a new connection + // is requested from the pool, and not on every stream queue on that connection. + // So if you request a single connection from a pool configured wiht a max streamcount of 200, + // and queue 500 streams on it, you will still exceed this limit and are at the mercy of hyper's + // default stream queuing. + pub pool_max_streams_per_conn: u16, + pub socks5_addr: Option, pub admin_addr: SocketAddr, pub stats_addr: SocketAddr, @@ -321,6 +335,8 @@ pub fn construct_config(pc: ProxyConfig) -> Result { .get(DNS_CAPTURE_METADATA) .map_or(false, |value| value.to_lowercase() == "true"), + pool_max_streams_per_conn: parse_default(POOL_MAX_STREAMS_PER_CONNECTION, 250)?, + window_size: 4 * 1024 * 1024, connection_window_size: 4 * 1024 * 1024, frame_size: 1024 * 1024, @@ -342,7 +358,7 @@ pub fn construct_config(pc: ProxyConfig) -> Result { DEFAULT_READINESS_PORT, // There is no config for this in ProxyConfig currently ), - socks5_addr, + socks5_addr: SocketAddr::new(IpAddr::V4(Ipv4Addr::LOCALHOST), 15080), inbound_addr: SocketAddr::new(IpAddr::V6(Ipv6Addr::UNSPECIFIED), 15008), inbound_plaintext_addr: SocketAddr::new(IpAddr::V6(Ipv6Addr::UNSPECIFIED), 15006), outbound_addr: SocketAddr::new(IpAddr::V6(Ipv6Addr::UNSPECIFIED), 15001), diff --git a/src/identity/manager.rs b/src/identity/manager.rs index f054813e1..4b80ce826 100644 --- a/src/identity/manager.rs +++ b/src/identity/manager.rs @@ -117,6 +117,8 @@ impl fmt::Display for Identity { } } +// TODO we shouldn't have a "default identity" outside of tests +// #[cfg(test)] impl Default for Identity { fn default() -> Self { const TRUST_DOMAIN: &str = "cluster.local"; diff --git a/src/proxy.rs b/src/proxy.rs index 610a3b98e..10e3191a4 100644 --- a/src/proxy.rs +++ b/src/proxy.rs @@ -49,7 +49,7 @@ mod inbound_passthrough; #[allow(non_camel_case_types)] pub mod metrics; mod outbound; -mod pool; +pub mod pool; mod socks5; mod util; @@ -105,7 +105,7 @@ pub(super) struct ProxyInputs { hbone_port: u16, pub state: DemandProxyState, metrics: Arc, - pool: pool::Pool, + pool: pool::WorkloadHBONEPool, socket_factory: Arc, proxy_workload_info: Option>, } @@ -119,6 +119,7 @@ impl ProxyInputs { metrics: Arc, socket_factory: Arc, proxy_workload_info: Option, + pool: pool::WorkloadHBONEPool, ) -> Self { Self { cfg, @@ -126,7 +127,7 @@ impl ProxyInputs { cert_manager, metrics, connection_manager, - pool: pool::Pool::new(), + pool, hbone_port: 0, socket_factory, proxy_workload_info: proxy_workload_info.map(Arc::new), @@ -143,15 +144,23 @@ impl Proxy { drain: Watch, ) -> Result { let metrics = Arc::new(metrics); + let socket_factory = Arc::new(DefaultSocketFactory); + + let pool = pool::WorkloadHBONEPool::new( + cfg.clone(), + socket_factory.clone(), + cert_manager.clone(), + drain.clone(), + ); let pi = ProxyInputs { cfg, state, cert_manager, connection_manager: ConnectionManager::default(), metrics, - pool: pool::Pool::new(), + pool, hbone_port: 0, - socket_factory: Arc::new(DefaultSocketFactory), + socket_factory, proxy_workload_info: None, }; Self::from_inputs(pi, drain).await @@ -245,11 +254,13 @@ pub enum Error { AuthorizationPolicyRejection, #[error("pool is already connecting")] - PoolAlreadyConnecting, + WorkloadHBONEPoolAlreadyConnecting, - #[error("pool: {0}")] - Pool(#[from] hyper_util::client::legacy::pool::Error), + #[error("connection streams maxed out")] + WorkloadHBONEPoolConnStreamsMaxed, + // #[error("pool: {0}")] + // WorkloadHBONEPool(#[from] custom_http2_pool::Error), #[error("{0}")] Generic(Box), diff --git a/src/proxy/inbound.rs b/src/proxy/inbound.rs index 379a72326..78352d2db 100644 --- a/src/proxy/inbound.rs +++ b/src/proxy/inbound.rs @@ -117,6 +117,7 @@ impl Inbound { debug!(%conn, "accepted connection"); let enable_original_source = self.pi.cfg.enable_original_source; let serve = crate::hyper_util::http2_server() + // .max_concurrent_streams(100) // TODO BML test .initial_stream_window_size(self.pi.cfg.window_size) .initial_connection_window_size(self.pi.cfg.connection_window_size) .max_frame_size(self.pi.cfg.frame_size) diff --git a/src/proxy/outbound.rs b/src/proxy/outbound.rs index 096fc5b12..8d7baec9f 100644 --- a/src/proxy/outbound.rs +++ b/src/proxy/outbound.rs @@ -20,7 +20,6 @@ use std::time::Instant; use bytes::Bytes; use drain::Watch; use http_body_util::Empty; -use hyper::client::conn::http2; use hyper::header::FORWARDED; use tokio::net::{TcpListener, TcpStream}; @@ -37,7 +36,7 @@ use crate::proxy::{util, Error, ProxyInputs, TraceParent, BAGGAGE_HEADER, TRACEP use crate::state::service::ServiceDescription; use crate::state::workload::gatewayaddress::Destination; use crate::state::workload::{address::Address, NetworkAddress, Protocol, Workload}; -use crate::{hyper_util, proxy, socket}; +use crate::{proxy, socket}; pub struct Outbound { pi: ProxyInputs, @@ -85,7 +84,6 @@ impl Outbound { let socket = self.listener.accept().await; let start_outbound_instant = Instant::now(); let outbound_drain = sub_drain.clone(); - let outer_conn_drain = sub_drain.clone(); match socket { Ok((stream, _remote)) => { let mut oc = OutboundConnection { @@ -101,7 +99,7 @@ impl Outbound { _ = outbound_drain.signaled() => { debug!("outbound drain signaled"); } - _ = oc.proxy(stream, outer_conn_drain.clone()) => {} + _ = oc.proxy(stream) => {} } debug!(dur=?start_outbound_instant.elapsed(), id=%oc.id, "outbound spawn DONE"); }) @@ -138,7 +136,7 @@ pub(super) struct OutboundConnection { } impl OutboundConnection { - async fn proxy(&mut self, source_stream: TcpStream, outer_conn_drain: Watch) { + async fn proxy(&mut self, source_stream: TcpStream) { let source_addr = socket::to_canonical(source_stream.peer_addr().expect("must receive peer addr")); let dst_addr = socket::orig_dst_addr_or_default(&source_stream); @@ -147,7 +145,6 @@ impl OutboundConnection { source_addr, dst_addr, false, - Some(outer_conn_drain), ) .await; } @@ -169,16 +166,15 @@ impl OutboundConnection { ) { match out_drain { Some(drain) => { - let outer_conn_drain = drain.clone(); tokio::select! { _ = drain.signaled() => { info!("socks drain signaled"); } - res = self.proxy_to(stream, remote_addr, orig_dst_addr, block_passthrough, Some(outer_conn_drain)) => res + res = self.proxy_to(stream, remote_addr, orig_dst_addr, block_passthrough) => res } } None => { - self.proxy_to(stream, remote_addr, orig_dst_addr, block_passthrough, None) + self.proxy_to(stream, remote_addr, orig_dst_addr, block_passthrough) .await; } } @@ -190,7 +186,6 @@ impl OutboundConnection { source_addr: SocketAddr, dest_addr: SocketAddr, block_passthrough: bool, - outer_conn_drain: Option, ) { let start = Instant::now(); @@ -252,7 +247,6 @@ impl OutboundConnection { self.proxy_to_hbone( &mut source_stream, source_addr, - outer_conn_drain, &req, &result_tracker, ) @@ -270,7 +264,6 @@ impl OutboundConnection { &mut self, stream: &mut TcpStream, remote_addr: SocketAddr, - outer_conn_drain: Option, req: &Request, connection_stats: &ConnectionResult, ) -> Result<(), Error> { @@ -296,75 +289,14 @@ impl OutboundConnection { ); let dst_identity = allowed_sans; - let pool_key = pool::Key { + let pool_key = pool::WorkloadKey { src_id: req.source.identity(), dst_id: dst_identity.clone(), src: remote_addr.ip(), dst: req.gateway, }; - // Setup our connection future. This won't always run if we have an existing connection - // in the pool. - let connect = async { - let mut builder = http2::Builder::new(hyper_util::TokioExecutor); - let builder = builder - .initial_stream_window_size(self.pi.cfg.window_size) - .max_frame_size(self.pi.cfg.frame_size) - .initial_connection_window_size(self.pi.cfg.connection_window_size); - - let local = self - .pi - .cfg - .enable_original_source - .unwrap_or_default() - .then_some(remote_addr.ip()); - let id = &req.source.identity(); - let cert = self.pi.cert_manager.fetch_certificate(id).await?; - let connector = cert.outbound_connector(dst_identity)?; - let tcp_stream = - super::freebind_connect(local, req.gateway, self.pi.socket_factory.as_ref()) - .await?; - tcp_stream.set_nodelay(true)?; // TODO: this is backwards of expectations - let tls_stream = connector.connect(tcp_stream).await?; - let (request_sender, connection) = builder - .handshake(::hyper_util::rt::TokioIo::new(tls_stream)) - .await - .map_err(Error::HttpHandshake)?; - - // spawn a task to poll the connection and drive the HTTP state - // if we got a drain for that connection, respect it in a race - match outer_conn_drain { - Some(conn_drain) => { - tokio::spawn(async move { - tokio::select! { - _ = conn_drain.signaled() => { - debug!("draining outer HBONE connection"); - } - res = connection=> { - match res { - Err(e) => { - error!("Error in HBONE connection handshake: {:?}", e); - } - Ok(_) => { - debug!("done with HBONE connection handshake: {:?}", res); - } - } - } - } - }); - } - None => { - tokio::spawn(async move { - if let Err(e) = connection.await { - error!("Error in HBONE connection handshake: {:?}", e); - } - }); - } - } - - Ok(request_sender) - }; - let mut connection = self.pi.pool.connect(pool_key.clone(), connect).await?; + let mut connection = self.pi.pool.connect(pool_key.clone()).await?; let mut f = http_types::proxies::Forwarded::new(); f.add_for(remote_addr.to_string()); @@ -708,15 +640,24 @@ mod tests { XdsAddressType::Workload(wl) => new_proxy_state(&[source, waypoint, wl], &[], &[]), XdsAddressType::Service(svc) => new_proxy_state(&[source, waypoint], &[svc], &[]), }; + + let sock_fact = std::sync::Arc::new(crate::proxy::DefaultSocketFactory); + let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); + let (_, sub_drain) = drain::channel(); let outbound = OutboundConnection { pi: ProxyInputs { cert_manager: identity::mock::new_secret_manager(Duration::from_secs(10)), state, hbone_port: 15008, - cfg, + cfg: cfg.clone(), metrics: test_proxy_metrics(), - pool: pool::Pool::new(), - socket_factory: std::sync::Arc::new(crate::proxy::DefaultSocketFactory), + pool: pool::WorkloadHBONEPool::new( + cfg, + sock_fact.clone(), + cert_mgr.clone(), + sub_drain, + ), + socket_factory: sock_fact, proxy_workload_info: None, connection_manager: ConnectionManager::default(), }, diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index f274617d2..f36e7d770 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -12,80 +12,334 @@ // See the License for the specific language governing permissions and // limitations under the License. +use super::{Error, SocketFactory}; use bytes::Bytes; -use futures::pin_mut; -use futures_util::future; -use futures_util::future::Either; +use drain::Watch; use http_body_util::Empty; use hyper::body::Incoming; use hyper::client::conn::http2; use hyper::http::{Request, Response}; -use hyper_util::client::legacy::pool; -use hyper_util::client::legacy::pool::{Pool as HyperPool, Poolable, Pooled, Reservation}; -use hyper_util::rt::TokioTimer; + +use std::collections::hash_map::DefaultHasher; use std::future::Future; +use std::hash::{Hash, Hasher}; use std::net::IpAddr; use std::net::SocketAddr; -use std::time::Duration; -use tracing::debug; +use std::sync::atomic::{AtomicI32, AtomicU16, Ordering}; +use std::sync::Arc; + +use tokio::sync::watch; +use tokio::sync::{Mutex, RwLock}; +use tracing::{debug, error}; + +use crate::config; +use crate::identity::{Identity, SecretManager}; + +use std::collections::HashMap; -use crate::identity::Identity; -use crate::proxy::Error; +use pingora_pool; +static GLOBAL_CONN_COUNT: AtomicI32 = AtomicI32::new(0); + +// A relatively nonstandard HTTP/2 connection pool designed to allow multiplexing proxied workload connections +// over a (smaller) number of HTTP/2 mTLS tunnels. +// +// The following invariants apply to this pool: +// - Every workload (inpod mode) gets its own connpool. +// - Every unique src/dest key gets their own dedicated connections inside the pool. +// - Every unique src/dest key gets 1-n dedicated connections, where N is (currently) unbounded but practically limited +// by flow control throttling. #[derive(Clone)] -pub struct Pool { - pool: HyperPool, +pub struct WorkloadHBONEPool { + pool_notifier: watch::Sender, + pool_watcher: watch::Receiver, + max_streamcount: u16, + // this is effectively just a convenience data type - a rwlocked hashmap with keying and LRU drops + // and has no actual hyper/http/connection logic. + connected_pool: Arc>, + cfg: config::Config, + socket_factory: Arc, + cert_manager: Arc, + drainer: Watch, + // this must be a readlockable list-of-locks, so we can lock per-key, not globally, and avoid holding up all conn attempts + established_conn_writelock: Arc>>>>, } -impl Pool { - pub fn new() -> Pool { +impl WorkloadHBONEPool { + pub fn new( + cfg: crate::config::Config, + socket_factory: Arc, + cert_manager: Arc, + drainer: Watch, //when signaled, will stop driving all conns in the pool, effectively draining the pool. + ) -> WorkloadHBONEPool { + let (tx, rx) = watch::channel(false); + debug!( + "constructing pool with {:#?} streams per conn", + cfg.pool_max_streams_per_conn + ); Self { - pool: HyperPool::new( - hyper_util::client::legacy::pool::Config { - idle_timeout: Some(Duration::from_secs(90)), - max_idle_per_host: std::usize::MAX, - }, - TokioExec, - Some(TokioTimer::new()), - ), + pool_notifier: tx, + pool_watcher: rx, + max_streamcount: cfg.pool_max_streams_per_conn, + // the number here is simply the number of unique src/dest keys + // the pool is expected to track before the inner hashmap resizes. + connected_pool: Arc::new(pingora_pool::ConnectionPool::new(50000)), + cfg, + socket_factory, + cert_manager, + drainer, + established_conn_writelock: Arc::new(RwLock::new(HashMap::new())), } } -} -#[derive(Clone)] -pub struct TokioExec; + pub async fn connect(&mut self, key: WorkloadKey) -> Result { + // TODO BML this may not be collision resistant + // it may also be slow as shit + let mut s = DefaultHasher::new(); + key.hash(&mut s); + let hash_key = s.finish(); + let pool_key = pingora_pool::ConnectionMeta::new( + hash_key, + GLOBAL_CONN_COUNT.fetch_add(1, Ordering::Relaxed), + ); + + let existing_conn = self + .first_checkout_conn_from_pool(&key, hash_key, &pool_key) + .await; + + if existing_conn.is_some() { + debug!("using existing conn, connect future will be dropped on the floor"); + return Ok(existing_conn.unwrap()); + } else { + // critical block - this writelocks the entire pool for all tasks/threads + // as we check to see if anyone has inserted a sharded mutex for this key. + // So we want to hold this for as short as possible a time, and drop it + // before we hold it over an await. + // + // this is is the only block where we should hold a writelock on the whole mutex map + { + let mut map_write_lock = self.established_conn_writelock.write().await; + match map_write_lock.get(&hash_key) { + Some(_) => { + debug!("already have conn for key {:#?}", hash_key); + } + None => { + debug!("inserting conn mutex for key {:#?}", hash_key); + map_write_lock.insert(hash_key, Some(Mutex::new(()))); + } + }; + drop(map_write_lock); + } + + // Now we know _someone_ won the race to insert a conn mutex, we don't need a writelock + // on the outer map anymore - so we can just readlock the outer map, + // and get the inner mutex for this connkey that we care about. + // + // This unblocks other tasks spawning connections against other keys, but blocks other + // tasks spawning connections against THIS key - which is what we want. + + // NOTE: This inner, key-specific mutex is a tokio::async::Mutex, and not a stdlib sync mutex. + // these differ from the stdlib sync mutex in that they are (slightly) slower + // (they effectively sleep the current task) and they can be held over an await. + // The tokio docs (rightly) advise you to not use these, + // because holding a lock over an await is a great way to create deadlocks if the await you + // hold it over does not resolve. + // + // HOWEVER. Here we know this connection will either establish or timeout + // and we WANT other tasks to go back to sleep if there is an outstanding lock. + // So the downsides are actually useful (we WANT task contention - + // to block other parallel tasks from trying to spawn a connection if we are already doing so) + let map_read_lock = self.established_conn_writelock.read().await; + let exist_conn_lock = map_read_lock.get(&hash_key).unwrap(); + let found_conn = match exist_conn_lock.as_ref().unwrap().try_lock() { + Ok(_guard) => { + // if we get here, either we won the connlock race and can create one, + // or someone else won, but the streamcount for the one they added is already hit, + // so we should start and insert another. + debug!("appears we need a new conn, retaining connlock"); + debug!("nothing else is creating a conn, make one"); + let pool_conn = self.spawn_new_pool_conn(key.clone()).await; + let client = Client( + pool_conn?, + Arc::new(AtomicU16::new(0)), + self.max_streamcount, + ); + + debug!( + "starting new conn for key {:#?} with pk {:#?}", + key, pool_key + ); + debug!("dropping lock"); + Some(client) + } + Err(_) => { + // The sharded mutex for this connkey is already locked - someone else must be making a conn + // if they are, try to wait for it, but bail if we find one and it's got a maxed streamcount. + debug!("something else is creating a conn, wait for it"); + let waiter = self.pool_watcher.changed(); + tokio::pin!(waiter); + + loop { + tokio::select! { + _ = &mut waiter => { + debug!("notified a new conn was enpooled, checking for hash {:#?}", hash_key); + + let existing_conn = self.connected_pool.get(&hash_key);// .and_then(|e_conn| { + + // debug!("while waiting for new conn, got existing conn for key {:#?}", key); + // }); + match existing_conn { + None => { + debug!("got nothin"); + continue; + } + Some(e_conn) => { + debug!("found existing conn after waiting"); + if e_conn.at_max_streamcount() { + debug!("found existing conn for key {:#?}, but streamcount is maxed", key); + break None; + } + break Some(e_conn); + } + } + } + + } + } + } + }; + + match found_conn { + Some(f_conn) => { + self.connected_pool.put(&pool_key, f_conn.clone()); + let _ = self.pool_notifier.send(true); + return Ok(f_conn); + } + + None => { + debug!("spawning new conn for key {:#?} to replace", key); + let pool_conn = self.spawn_new_pool_conn(key.clone()).await; + let r_conn = Client( + pool_conn?, + Arc::new(AtomicU16::new(0)), + self.max_streamcount, + ); + self.connected_pool.put(&pool_key, r_conn.clone()); + let _ = self.pool_notifier.send(true); + return Ok(r_conn); + } + } + } + } -impl hyper::rt::Executor for TokioExec -where - F: std::future::Future + Send + 'static, - F::Output: Send + 'static, -{ - fn execute(&self, fut: F) { - tokio::spawn(fut); + async fn first_checkout_conn_from_pool( + &self, + key: &WorkloadKey, + hash_key: u64, + pool_key: &pingora_pool::ConnectionMeta, + ) -> Option { + let map_read_lock = self.established_conn_writelock.read().await; + match map_read_lock.get(&hash_key) { + Some(exist_conn_lock) => { + let _conn_lock = exist_conn_lock.as_ref().unwrap().lock().await; + + debug!("getting conn for key {:#?} and hash {:#?}", key, hash_key); + self.connected_pool.get(&hash_key).and_then(|e_conn| { + debug!("got existing conn for key {:#?}", key); + if e_conn.at_max_streamcount() { + debug!("got conn for key {:#?}, but streamcount is maxed", key); + None + // Some(Err(Error::WorkloadHBONEPoolConnStreamsMaxed)) + } else { + self.connected_pool.put(&pool_key, e_conn.clone()); + let _ = self.pool_notifier.send(true); + Some(e_conn) + } + }) + } + None => None, + } } -} -#[derive(Debug, Clone)] -struct Client(http2::SendRequest>); + async fn spawn_new_pool_conn( + &self, + key: WorkloadKey, + ) -> Result>, Error> { + let clone_key = key.clone(); + let mut c_builder = http2::Builder::new(crate::hyper_util::TokioExecutor); + let builder = c_builder + .initial_stream_window_size(self.cfg.window_size) + .max_frame_size(self.cfg.frame_size) + .initial_connection_window_size(self.cfg.connection_window_size); + + let local = self + .cfg + .enable_original_source + .unwrap_or_default() + .then_some(key.src); + let cert = self.cert_manager.fetch_certificate(&key.src_id).await?; + let connector = cert.outbound_connector(key.dst_id)?; + let tcp_stream = + super::freebind_connect(local, key.dst, self.socket_factory.as_ref()).await?; + tcp_stream.set_nodelay(true)?; // TODO: this is backwards of expectations + let tls_stream = connector.connect(tcp_stream).await?; + let (request_sender, connection) = builder + .handshake(::hyper_util::rt::TokioIo::new(tls_stream)) + .await + .map_err(Error::HttpHandshake)?; -impl Poolable for Client { - fn is_open(&self) -> bool { - self.0.is_ready() + // spawn a task to poll the connection and drive the HTTP state + // if we got a drain for that connection, respect it in a race + // it is important to have a drain here, or this connection will never terminate + let driver_drain = self.drainer.clone(); + tokio::spawn(async move { + debug!("starting a connection driver for {:?}", clone_key); + tokio::select! { + _ = driver_drain.signaled() => { + debug!("draining outer HBONE connection"); + } + res = connection=> { + match res { + Err(e) => { + error!("Error in HBONE connection handshake: {:?}", e); + } + Ok(_) => { + debug!("done with HBONE connection handshake: {:?}", res); + } + } + } + } + }); + + Ok(request_sender) } +} + +#[derive(Debug, Clone)] +pub struct Client(http2::SendRequest>, Arc, u16); - fn reserve(self) -> Reservation { - let b = self.clone(); - let a = self; - Reservation::Shared(a, b) +impl Client { + pub fn at_max_streamcount(&self) -> bool { + let curr_count = self.1.load(Ordering::Relaxed); + debug!("checking streamcount: {curr_count}"); + if curr_count >= self.2 { + return true; + } + false } - fn can_share(&self) -> bool { - true // http2 always shares + pub fn send_request( + &mut self, + req: Request>, + ) -> impl Future>> { + // TODO should we enforce streamcount per-sent-request? This would be slow. + self.1.fetch_add(1, Ordering::Relaxed); + self.0.send_request(req) } } #[derive(PartialEq, Eq, Hash, Clone, Debug)] -pub struct Key { +pub struct WorkloadKey { pub src_id: Identity, pub dst_id: Vec, // In theory we can just use src,dst,node. However, the dst has a check that @@ -96,157 +350,506 @@ pub struct Key { pub src: IpAddr, } -#[derive(Debug)] -pub struct Connection(Pooled); - -impl Connection { - pub fn send_request( - &mut self, - req: Request>, - ) -> impl Future>> { - self.0 .0.send_request(req) - } -} - -impl Pool { - pub async fn connect(&self, key: Key, connect: F) -> Result - where - F: Future>, Error>>, - { - let reuse_connection = self.pool.checkout(key.clone()); - - let connect_pool = async { - let ver = pool::Ver::Http2; - let Some(connecting) = self.pool.connecting(&key, ver) else { - // There is already an existing connection establishment in flight. - // Return an error so - return Err(Error::PoolAlreadyConnecting); - }; - let pc = Client(connect.await?); - let pooled = self.pool.pooled(connecting, pc); - Ok::<_, Error>(pooled) - }; - pin_mut!(connect_pool); - let request_sender: Pooled = - match future::select(reuse_connection, connect_pool).await { - // Checkout won. - Either::Left((Ok(conn), _)) => { - debug!(?key, "fetched existing connection"); - conn - } - // Checkout won, but had an error. - Either::Left((Err(err), connecting)) => match err { - // Checked out a closed connection. Just keep connecting then - pool::Error::CheckedOutClosedValue => connecting.await?, - // Some other error, bubble it up - _ => return Err(Error::Pool(err)), - }, - // Connect won, checkout can just be dropped. - Either::Right((Ok(request_sender), _checkout)) => { - debug!(?key, "established new connection"); - request_sender - } - // Connect won, checkout can just be dropped. - Either::Right((Err(err), checkout)) => { - debug!( - ?key, - "connect won, but wait for existing pooled connection to establish" - ); - match err { - // Connect won but we already had an in-flight connection, so use that. - Error::PoolAlreadyConnecting => checkout.await?, - // Some other connection error - err => return Err(err), - } - } - }; - - Ok(Connection(request_sender)) - } -} #[cfg(test)] mod test { use std::convert::Infallible; use std::net::SocketAddr; + use std::time::Instant; + use crate::identity; + + use drain::Watch; + use futures_util::StreamExt; use hyper::body::Incoming; + use hyper::service::service_fn; use hyper::{Request, Response}; - use tokio::net::{TcpListener, TcpStream}; + use std::sync::atomic::AtomicU32; + use std::time::Duration; + use tokio::io::AsyncWriteExt; + use tokio::net::TcpListener; + use tokio::task::{self}; use tracing::{error, info}; + use ztunnel::test_helpers::*; + use super::*; #[tokio::test] - async fn test_pool() { + async fn test_pool_reuses_conn_for_same_key() { + // crate::telemetry::setup_logging(); + + let (server_drain_signal, server_drain) = drain::channel(); + + let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + + let cfg = crate::config::Config { + local_node: Some("local-node".to_string()), + pool_max_streams_per_conn: 6, + ..crate::config::parse_config().unwrap() + }; + let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); + let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); + + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + + let key1 = WorkloadKey { + src_id: Identity::default(), + dst_id: vec![Identity::default()], + src: IpAddr::from([127, 0, 0, 2]), + dst: server_addr, + }; + let client1 = spawn_client(pool.clone(), key1.clone(), server_addr, 2).await; + let client2 = spawn_client(pool.clone(), key1.clone(), server_addr, 2).await; + let client3 = spawn_client(pool.clone(), key1, server_addr, 2).await; + + drop(pool); + server_drain_signal.drain().await; + let real_conncount = server_handle.await.unwrap(); + assert!(real_conncount == 1, "actual conncount was {real_conncount}"); + + assert!(!client1.is_err()); + assert!(!client2.is_err()); + assert!(!client3.is_err()); + } + + #[tokio::test] + async fn test_pool_does_not_reuse_conn_for_diff_key() { + let (server_drain_signal, server_drain) = drain::channel(); + let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + + // crate::telemetry::setup_logging(); + + let cfg = crate::config::Config { + local_node: Some("local-node".to_string()), + pool_max_streams_per_conn: 10, + ..crate::config::parse_config().unwrap() + }; + let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); + let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + + let key1 = WorkloadKey { + src_id: Identity::default(), + dst_id: vec![Identity::default()], + src: IpAddr::from([127, 0, 0, 2]), + dst: server_addr, + }; + let key2 = WorkloadKey { + src_id: Identity::default(), + dst_id: vec![Identity::default()], + src: IpAddr::from([127, 0, 0, 3]), + dst: server_addr, + }; + + let client1 = spawn_client(pool.clone(), key1, server_addr, 2).await; + let client2 = spawn_client(pool.clone(), key2, server_addr, 2).await; + + drop(pool); + server_drain_signal.drain().await; + let real_conncount = server_handle.await.unwrap(); + assert!(real_conncount == 2, "actual conncount was {real_conncount}"); + + assert!(!client1.is_err()); + assert!(!client2.is_err()); // expect this to panic - we used a new key + } + + #[tokio::test] + async fn test_pool_respects_per_conn_stream_limit() { + let (server_drain_signal, server_drain) = drain::channel(); + let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + + let cfg = crate::config::Config { + local_node: Some("local-node".to_string()), + pool_max_streams_per_conn: 3, + ..crate::config::parse_config().unwrap() + }; + let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); + let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + + let key1 = WorkloadKey { + src_id: Identity::default(), + dst_id: vec![Identity::default()], + src: IpAddr::from([127, 0, 0, 2]), + dst: server_addr, + }; + let client1 = spawn_client(pool.clone(), key1.clone(), server_addr, 4).await; + let client2 = spawn_client(pool.clone(), key1, server_addr, 2).await; + + drop(pool); + server_drain_signal.drain().await; + + let real_conncount = server_handle.await.unwrap(); + assert!(real_conncount == 2, "actual conncount was {real_conncount}"); + + assert!(!client1.is_err()); + assert!(!client2.is_err()); // expect this to panic - same key, but stream limit of 3 + } + + #[tokio::test] + async fn test_pool_handles_many_conns_per_key() { + let (server_drain_signal, server_drain) = drain::channel(); + let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + + let cfg = crate::config::Config { + local_node: Some("local-node".to_string()), + pool_max_streams_per_conn: 2, + ..crate::config::parse_config().unwrap() + }; + let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); + let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); + + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + + let key1 = WorkloadKey { + src_id: Identity::default(), + dst_id: vec![Identity::default()], + src: IpAddr::from([127, 0, 0, 2]), + dst: server_addr, + }; + let client1 = spawn_client(pool.clone(), key1.clone(), server_addr, 4).await; + let client2 = spawn_client(pool.clone(), key1.clone(), server_addr, 4).await; + + drop(pool); + server_drain_signal.drain().await; + + let real_conncount = server_handle.await.unwrap(); + assert!(real_conncount == 2, "actual conncount was {real_conncount}"); + + assert!(!client1.is_err()); + assert!(!client2.is_err()); + } + + #[tokio::test] + async fn test_pool_100_clients_streamexhaust() { + // crate::telemetry::setup_logging(); + + let (server_drain_signal, server_drain) = drain::channel(); + let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + + let cfg = crate::config::Config { + local_node: Some("local-node".to_string()), + pool_max_streams_per_conn: 50, + ..crate::config::parse_config().unwrap() + }; + let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); + let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + + let key1 = WorkloadKey { + src_id: Identity::default(), + dst_id: vec![Identity::default()], + src: IpAddr::from([127, 0, 0, 2]), + dst: server_addr, + }; + let client_count = 100; + let mut count = 0u32; + let mut tasks = futures::stream::FuturesUnordered::new(); + loop { + count += 1; + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 100)); + + if count == client_count { + break; + } + } + while let Some(_) = tasks.next().await { + continue; + } + + drop(pool); + + server_drain_signal.drain().await; + let real_conncount = server_handle.await.unwrap(); + assert!(real_conncount == 2, "actual conncount was {real_conncount}"); + } + + #[tokio::test] + async fn test_pool_100_clients_singleconn() { + let _cfg = crate::config::Config { + local_node: Some("local-node".to_string()), + ..crate::config::parse_config().unwrap() + }; + + // crate::telemetry::setup_logging(); + + let (server_drain_signal, server_drain) = drain::channel(); + let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + + let cfg = crate::config::Config { + local_node: Some("local-node".to_string()), + pool_max_streams_per_conn: 1000, + ..crate::config::parse_config().unwrap() + }; + let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); + let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + + let key1 = WorkloadKey { + src_id: Identity::default(), + dst_id: vec![Identity::default()], + src: IpAddr::from([127, 0, 0, 2]), + dst: server_addr, + }; + let client_count = 100; + let mut count = 0u32; + let mut tasks = futures::stream::FuturesUnordered::new(); + loop { + count += 1; + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 100)); + + if count == client_count { + break; + } + } + while let Some(_) = tasks.next().await { + continue; + } + + drop(pool); + + server_drain_signal.drain().await; + let real_conncount = server_handle.await.unwrap(); + assert!(real_conncount == 1, "actual conncount was {real_conncount}"); + } + + #[tokio::test] + async fn test_pool_100_clients_100_srcs() { + let _cfg = crate::config::Config { + local_node: Some("local-node".to_string()), + ..crate::config::parse_config().unwrap() + }; + + // crate::telemetry::setup_logging(); + + let (server_drain_signal, server_drain) = drain::channel(); + let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + + let cfg = crate::config::Config { + local_node: Some("local-node".to_string()), + pool_max_streams_per_conn: 100, + ..crate::config::parse_config().unwrap() + }; + let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); + let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + + let client_count = 100; + let mut count = 0u8; + let mut tasks = futures::stream::FuturesUnordered::new(); + loop { + count += 1; + + let key1 = WorkloadKey { + src_id: Identity::default(), + dst_id: vec![Identity::default()], + src: IpAddr::from([127, 0, 0, count.into()]), + dst: server_addr, + }; + // key1.src = IpAddr::from([127, 0, 0, count]); + + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 100)); + + if count == client_count { + break; + } + } + while let Some(_) = tasks.next().await { + continue; + } + + drop(pool); + + server_drain_signal.drain().await; + let real_conncount = server_handle.await.unwrap(); + assert!( + real_conncount == 100, + "actual conncount was {real_conncount}" + ); + } + + #[tokio::test] + async fn test_pool_1000_clients_3_srcs() { + let _cfg = crate::config::Config { + local_node: Some("local-node".to_string()), + ..crate::config::parse_config().unwrap() + }; + + // crate::telemetry::setup_logging(); + + let (server_drain_signal, server_drain) = drain::channel(); + let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + + let cfg = crate::config::Config { + local_node: Some("local-node".to_string()), + pool_max_streams_per_conn: 1000, + ..crate::config::parse_config().unwrap() + }; + let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); + let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + + let mut key1 = WorkloadKey { + src_id: Identity::default(), + dst_id: vec![Identity::default()], + src: IpAddr::from([127, 0, 0, 1]), + dst: server_addr, + }; + + let client_count = 1000; + let mut count = 0u32; + let mut tasks = futures::stream::FuturesUnordered::new(); + loop { + count += 1; + if count % 2 == 0 { + debug!("using key 2"); + key1.src = IpAddr::from([127, 0, 0, 4]); + } else if count % 3 == 0 { + debug!("using key 3"); + key1.src = IpAddr::from([127, 0, 0, 6]); + } else { + debug!("using key 1"); + key1.src = IpAddr::from([127, 0, 0, 2]); + } + + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 1)); + + if count == client_count { + break; + } + } + while let Some(_) = tasks.next().await { + continue; + } + + drop(pool); + + server_drain_signal.drain().await; + let real_conncount = server_handle.await.unwrap(); + assert!(real_conncount == 3, "actual conncount was {real_conncount}"); + } + + fn spawn_client( + mut pool: WorkloadHBONEPool, + key: WorkloadKey, + remote_addr: SocketAddr, + req_count: u32, + ) -> task::JoinHandle<()> { + tokio::spawn(async move { + let req = || { + hyper::Request::builder() + .uri(format!("{remote_addr}")) + .method(hyper::Method::CONNECT) + .version(hyper::Version::HTTP_2) + .body(Empty::::new()) + .unwrap() + }; + + let start = Instant::now(); + + let mut c1 = pool + .connect(key.clone()) + // needs tokio_unstable, but useful + // .instrument(tracing::debug_span!("client_tid", tid=%tokio::task::id())) + .await + .unwrap(); + debug!( + "client spent {}ms waiting for conn", + start.elapsed().as_millis() + ); + + let mut count = 0u32; + loop { + count += 1; + if c1.send_request(req()).await.unwrap().status() != 200 { + panic!("nope") + } + + if count >= req_count { + break; + } + } + }) + } + + async fn spawn_server(stop: Watch) -> (SocketAddr, task::JoinHandle) { // We'll bind to 127.0.0.1:3000 let addr = SocketAddr::from(([127, 0, 0, 1], 0)); + let test_cfg = test_config(); async fn hello_world(req: Request) -> Result>, Infallible> { - info!("got req {req:?}"); - Ok(Response::builder().status(200).body(Empty::new()).unwrap()) + info!("hello world: received request"); + tokio::task::spawn(async move { + match hyper::upgrade::on(req).await { + Ok(upgraded) => { + let (mut ri, mut wi) = + tokio::io::split(hyper_util::rt::TokioIo::new(upgraded)); + // Signal we are the waypoint so tests can validate this + wi.write_all(b"waypoint\n").await.unwrap(); + tcp::handle_stream(tcp::Mode::ReadWrite, &mut ri, &mut wi).await; + } + Err(e) => error!("No upgrade {e}"), + } + }); + Ok::<_, Infallible>(Response::new(http_body_util::Empty::::new())) } + let conn_count: Arc = Arc::new(AtomicU32::new(0)); + let _drop_conn_count: Arc = Arc::new(AtomicU32::new(0)); + // We create a TcpListener and bind it to 127.0.0.1:3000 let listener = TcpListener::bind(addr).await.unwrap(); + let bound_addr = listener.local_addr().unwrap(); - let addr = listener.local_addr().unwrap(); - tokio::spawn(async move { + let certs = crate::tls::mock::generate_test_certs( + &Identity::default().into(), + Duration::from_secs(0), + Duration::from_secs(100), + ); + let acceptor = crate::tls::mock::MockServerCertProvider::new(certs); + let mut tls_stream = crate::hyper_util::tls_server(acceptor, listener); + + let srv_handle = tokio::spawn(async move { // We start a loop to continuously accept incoming connections - loop { - let (stream, _) = listener.accept().await.unwrap(); - - // Spawn a tokio task to serve multiple connections concurrently - tokio::task::spawn(async move { - // Finally, we bind the incoming connection to our `hello` service - if let Err(err) = crate::hyper_util::http2_server() - .serve_connection( - hyper_util::rt::TokioIo::new(stream), - service_fn(hello_world), - ) - .await - { - println!("Error serving connection: {:?}", err); - } - }); - } - }); - let pool = Pool::new(); - let key = Key { - src_id: Identity::default(), - dst_id: vec![Identity::default()], - src: IpAddr::from([127, 0, 0, 2]), - dst: addr, - }; - let connect = || async { - let builder = http2::Builder::new(TokioExec); - - let tcp_stream = TcpStream::connect(addr).await?; - let (request_sender, connection) = builder - .handshake(hyper_util::rt::TokioIo::new(tcp_stream)) - .await?; - // spawn a task to poll the connection and drive the HTTP state - tokio::spawn(async move { - if let Err(e) = connection.await { - error!("Error in connection handshake: {:?}", e); + // and also count them + let movable_count = conn_count.clone(); + let accept = async move { + loop { + let stream = tls_stream.next().await.unwrap(); + movable_count.fetch_add(1, Ordering::Relaxed); + debug!("bump serverconn"); + + // Spawn a tokio task to serve multiple connections concurrently + tokio::task::spawn(async move { + // Finally, we bind the incoming connection to our `hello` service + if let Err(err) = crate::hyper_util::http2_server() + .initial_stream_window_size(test_cfg.window_size) + .initial_connection_window_size(test_cfg.connection_window_size) + .max_frame_size(test_cfg.frame_size) + // 64KB max; default is 16MB driven from Golang's defaults + // Since we know we are going to recieve a bounded set of headers, more is overkill. + .max_header_list_size(65536) + .serve_connection( + hyper_util::rt::TokioIo::new(stream), + service_fn(hello_world), + ) + .await + { + println!("Error serving connection: {:?}", err); + } + }); } - }); - Ok(request_sender) - }; - let req = || { - hyper::Request::builder() - .uri(format!("http://{addr}")) - .method(hyper::Method::GET) - .version(hyper::Version::HTTP_2) - .body(Empty::::new()) - .unwrap() - }; - let mut c1 = pool.connect(key.clone(), connect()).await.unwrap(); - let mut c2 = pool - .connect(key, async { unreachable!("should use pooled connection") }) - .await - .unwrap(); - assert_eq!(c1.send_request(req()).await.unwrap().status(), 200); - assert_eq!(c1.send_request(req()).await.unwrap().status(), 200); - assert_eq!(c2.send_request(req()).await.unwrap().status(), 200); + }; + tokio::select! { + _ = accept => {} + _ = stop.signaled() => { + debug!("GOT STOP SERVER"); + } + }; + + return conn_count.load(Ordering::Relaxed); + }); + + (bound_addr, srv_handle) } } diff --git a/src/proxyfactory.rs b/src/proxyfactory.rs index 99295068f..9a93a5f3d 100644 --- a/src/proxyfactory.rs +++ b/src/proxyfactory.rs @@ -100,6 +100,12 @@ impl ProxyFactory { self.proxy_metrics.clone().unwrap(), socket_factory.clone(), proxy_workload_info, + crate::proxy::pool::WorkloadHBONEPool::new( + self.config.clone(), + socket_factory.clone(), + self.cert_manager.clone(), + drain.clone(), + ), ); result.connection_manager = Some(cm); result.proxy = Some(Proxy::from_inputs(pi, drain.clone()).await?); From af0d232cb148e444f87b543ee976f2874b920cf4 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Tue, 16 Apr 2024 00:34:43 -0400 Subject: [PATCH 02/46] Lints Signed-off-by: Benjamin Leggett --- fuzz/Cargo.lock | 49 +++++++++++++++++++++++++++++++++++++++++++++++ src/proxy/pool.rs | 4 ++-- 2 files changed, 51 insertions(+), 2 deletions(-) diff --git a/fuzz/Cargo.lock b/fuzz/Cargo.lock index 7c1963d2a..c63703336 100644 --- a/fuzz/Cargo.lock +++ b/fuzz/Cargo.lock @@ -444,6 +444,15 @@ dependencies = [ "crossbeam-utils", ] +[[package]] +name = "crossbeam-queue" +version = "0.3.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "df0346b5d5e76ac2fe4e327c5fd1118d6be7c51dfb18f9b7922923f287471e35" +dependencies = [ + "crossbeam-utils", +] + [[package]] name = "crossbeam-utils" version = "0.8.19" @@ -1302,6 +1311,15 @@ version = "0.4.20" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b5e6163cb8c49088c2c36f57875e58ccd8c87c7427f7fbd50ea6710b2f3f2e8f" +[[package]] +name = "lru" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d3262e75e648fce39813cb56ac41f3c3e3f65217ebf3844d818d1f9398cfb0dc" +dependencies = [ + "hashbrown 0.14.3", +] + [[package]] name = "lru-cache" version = "0.1.2" @@ -1641,6 +1659,35 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" +[[package]] +name = "pingora-pool" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b4569e3bef52b0abab239a5cf3287c71307615ca61be7fc7799d71fdaab33d81" +dependencies = [ + "crossbeam-queue", + "log", + "lru", + "parking_lot", + "pingora-timeout", + "thread_local", + "tokio", +] + +[[package]] +name = "pingora-timeout" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0be182194d34e1b28608eaa49ee0fb86e5b7ab1d21a1d7a2b4d402446fda47e1" +dependencies = [ + "futures", + "once_cell", + "parking_lot", + "pin-project-lite", + "thread_local", + "tokio", +] + [[package]] name = "plotters" version = "0.3.5" @@ -3189,6 +3236,8 @@ dependencies = [ "netns-rs", "nix 0.28.0", "once_cell", + "pin-project-lite", + "pingora-pool", "ppp", "pprof", "prometheus-client", diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index f36e7d770..a04bcdbe0 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -51,7 +51,7 @@ static GLOBAL_CONN_COUNT: AtomicI32 = AtomicI32::new(0); // by flow control throttling. #[derive(Clone)] pub struct WorkloadHBONEPool { - pool_notifier: watch::Sender, + pool_notifier: Arc>, pool_watcher: watch::Receiver, max_streamcount: u16, // this is effectively just a convenience data type - a rwlocked hashmap with keying and LRU drops @@ -78,7 +78,7 @@ impl WorkloadHBONEPool { cfg.pool_max_streams_per_conn ); Self { - pool_notifier: tx, + pool_notifier: Arc::new(tx), pool_watcher: rx, max_streamcount: cfg.pool_max_streams_per_conn, // the number here is simply the number of unique src/dest keys From 58e7ab9a810b052aae7b0fe60bd7200d5c5366ab Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Tue, 16 Apr 2024 00:39:52 -0400 Subject: [PATCH 03/46] lints 2 Signed-off-by: Benjamin Leggett --- src/proxy.rs | 1 + src/proxy/pool.rs | 31 +++++++++++++++---------------- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/proxy.rs b/src/proxy.rs index 10e3191a4..d280fecac 100644 --- a/src/proxy.rs +++ b/src/proxy.rs @@ -110,6 +110,7 @@ pub(super) struct ProxyInputs { proxy_workload_info: Option>, } +#[allow(clippy::too_many_arguments)] impl ProxyInputs { pub fn new( cfg: config::Config, diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index a04bcdbe0..e1ad69cd8 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -109,7 +109,7 @@ impl WorkloadHBONEPool { if existing_conn.is_some() { debug!("using existing conn, connect future will be dropped on the floor"); - return Ok(existing_conn.unwrap()); + Ok(existing_conn.unwrap()) } else { // critical block - this writelocks the entire pool for all tasks/threads // as we check to see if anyone has inserted a sharded mutex for this key. @@ -213,7 +213,7 @@ impl WorkloadHBONEPool { Some(f_conn) => { self.connected_pool.put(&pool_key, f_conn.clone()); let _ = self.pool_notifier.send(true); - return Ok(f_conn); + Ok(f_conn) } None => { @@ -226,7 +226,7 @@ impl WorkloadHBONEPool { ); self.connected_pool.put(&pool_key, r_conn.clone()); let _ = self.pool_notifier.send(true); - return Ok(r_conn); + Ok(r_conn) } } } @@ -249,9 +249,8 @@ impl WorkloadHBONEPool { if e_conn.at_max_streamcount() { debug!("got conn for key {:#?}, but streamcount is maxed", key); None - // Some(Err(Error::WorkloadHBONEPoolConnStreamsMaxed)) } else { - self.connected_pool.put(&pool_key, e_conn.clone()); + self.connected_pool.put(pool_key, e_conn.clone()); let _ = self.pool_notifier.send(true); Some(e_conn) } @@ -408,9 +407,9 @@ mod test { let real_conncount = server_handle.await.unwrap(); assert!(real_conncount == 1, "actual conncount was {real_conncount}"); - assert!(!client1.is_err()); - assert!(!client2.is_err()); - assert!(!client3.is_err()); + assert!(client1.is_ok()); + assert!(client2.is_ok()); + assert!(client3.is_ok()); } #[tokio::test] @@ -450,8 +449,8 @@ mod test { let real_conncount = server_handle.await.unwrap(); assert!(real_conncount == 2, "actual conncount was {real_conncount}"); - assert!(!client1.is_err()); - assert!(!client2.is_err()); // expect this to panic - we used a new key + assert!(client1.is_ok()); + assert!(client2.is_ok()); // expect this to panic - we used a new key } #[tokio::test] @@ -483,8 +482,8 @@ mod test { let real_conncount = server_handle.await.unwrap(); assert!(real_conncount == 2, "actual conncount was {real_conncount}"); - assert!(!client1.is_err()); - assert!(!client2.is_err()); // expect this to panic - same key, but stream limit of 3 + assert!(client1.is_ok()); + assert!(client2.is_ok()); // expect this to panic - same key, but stream limit of 3 } #[tokio::test] @@ -517,8 +516,8 @@ mod test { let real_conncount = server_handle.await.unwrap(); assert!(real_conncount == 2, "actual conncount was {real_conncount}"); - assert!(!client1.is_err()); - assert!(!client2.is_err()); + assert!(client1.is_ok()); + assert!(client2.is_ok()); } #[tokio::test] @@ -644,7 +643,7 @@ mod test { let key1 = WorkloadKey { src_id: Identity::default(), dst_id: vec![Identity::default()], - src: IpAddr::from([127, 0, 0, count.into()]), + src: IpAddr::from([127, 0, 0, count]), dst: server_addr, }; // key1.src = IpAddr::from([127, 0, 0, count]); @@ -847,7 +846,7 @@ mod test { } }; - return conn_count.load(Ordering::Relaxed); + conn_count.load(Ordering::Relaxed) }); (bound_addr, srv_handle) From 5f8e1822e41e4a8ef6d79703058a5001ff6bfccc Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Tue, 16 Apr 2024 02:09:00 -0400 Subject: [PATCH 04/46] Hmm Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 66 ++++++++++++++++++++++++++++------------------- 1 file changed, 40 insertions(+), 26 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index e1ad69cd8..3538700cd 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -51,7 +51,7 @@ static GLOBAL_CONN_COUNT: AtomicI32 = AtomicI32::new(0); // by flow control throttling. #[derive(Clone)] pub struct WorkloadHBONEPool { - pool_notifier: Arc>, + pool_notifier: watch::Sender, pool_watcher: watch::Receiver, max_streamcount: u16, // this is effectively just a convenience data type - a rwlocked hashmap with keying and LRU drops @@ -78,7 +78,7 @@ impl WorkloadHBONEPool { cfg.pool_max_streams_per_conn ); Self { - pool_notifier: Arc::new(tx), + pool_notifier: tx, pool_watcher: rx, max_streamcount: cfg.pool_max_streams_per_conn, // the number here is simply the number of unique src/dest keys @@ -176,18 +176,21 @@ impl WorkloadHBONEPool { // The sharded mutex for this connkey is already locked - someone else must be making a conn // if they are, try to wait for it, but bail if we find one and it's got a maxed streamcount. debug!("something else is creating a conn, wait for it"); - let waiter = self.pool_watcher.changed(); - tokio::pin!(waiter); + // let waiter = self.pool_watcher.changed(); + // tokio::pin!(waiter); loop { - tokio::select! { - _ = &mut waiter => { - debug!("notified a new conn was enpooled, checking for hash {:#?}", hash_key); + match self.pool_watcher.changed().await { + Ok(_) => { + debug!( + "notified a new conn was enpooled, checking for hash {:#?}", + hash_key + ); - let existing_conn = self.connected_pool.get(&hash_key);// .and_then(|e_conn| { + let existing_conn = self.connected_pool.get(&hash_key); // .and_then(|e_conn| { - // debug!("while waiting for new conn, got existing conn for key {:#?}", key); - // }); + // debug!("while waiting for new conn, got existing conn for key {:#?}", key); + // }); match existing_conn { None => { debug!("got nothin"); @@ -203,7 +206,9 @@ impl WorkloadHBONEPool { } } } - + Err(_) => { + break None + } } } } @@ -231,7 +236,6 @@ impl WorkloadHBONEPool { } } } - async fn first_checkout_conn_from_pool( &self, key: &WorkloadKey, @@ -368,7 +372,7 @@ mod test { use tokio::io::AsyncWriteExt; use tokio::net::TcpListener; use tokio::task::{self}; - use tracing::{error, info}; + use tracing::{error, info, Instrument}; use ztunnel::test_helpers::*; @@ -522,7 +526,7 @@ mod test { #[tokio::test] async fn test_pool_100_clients_streamexhaust() { - // crate::telemetry::setup_logging(); + crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; @@ -553,7 +557,8 @@ mod test { break; } } - while let Some(_) = tasks.next().await { + while let Some(Err(res)) = tasks.next().await { + assert!(!res.is_panic(), "CLIENT PANICKED!"); continue; } @@ -561,7 +566,7 @@ mod test { server_drain_signal.drain().await; let real_conncount = server_handle.await.unwrap(); - assert!(real_conncount == 2, "actual conncount was {real_conncount}"); + assert!(real_conncount == 3, "actual conncount was {real_conncount}"); } #[tokio::test] @@ -602,7 +607,8 @@ mod test { break; } } - while let Some(_) = tasks.next().await { + while let Some(Err(res)) = tasks.next().await { + assert!(!res.is_panic(), "CLIENT PANICKED!"); continue; } @@ -654,7 +660,9 @@ mod test { break; } } - while let Some(_) = tasks.next().await { + + while let Some(Err(res)) = tasks.next().await { + assert!(!res.is_panic(), "CLIENT PANICKED!"); continue; } @@ -675,14 +683,14 @@ mod test { ..crate::config::parse_config().unwrap() }; - // crate::telemetry::setup_logging(); + crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), - pool_max_streams_per_conn: 1000, + pool_max_streams_per_conn: 100, ..crate::config::parse_config().unwrap() }; let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); @@ -696,7 +704,7 @@ mod test { dst: server_addr, }; - let client_count = 1000; + let client_count = 100; let mut count = 0u32; let mut tasks = futures::stream::FuturesUnordered::new(); loop { @@ -712,13 +720,14 @@ mod test { key1.src = IpAddr::from([127, 0, 0, 2]); } - tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 1)); + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 100)); if count == client_count { break; } } - while let Some(_) = tasks.next().await { + while let Some(Err(res)) = tasks.next().await { + assert!(!res.is_panic(), "CLIENT PANICKED!"); continue; } @@ -761,11 +770,16 @@ mod test { let mut count = 0u32; loop { count += 1; - if c1.send_request(req()).await.unwrap().status() != 200 { - panic!("nope") + let res = c1.send_request(req()).await; + + if res.is_err() { + panic!("SEND ERR: {:#?} sendcount {count}", res); + } else if res.unwrap().status() != 200 { + panic!("CLIENT RETURNED ERROR") } if count >= req_count { + debug!("CLIENT DONE"); break; } } @@ -777,7 +791,7 @@ mod test { let addr = SocketAddr::from(([127, 0, 0, 1], 0)); let test_cfg = test_config(); async fn hello_world(req: Request) -> Result>, Infallible> { - info!("hello world: received request"); + debug!("hello world: received request"); tokio::task::spawn(async move { match hyper::upgrade::on(req).await { Ok(upgraded) => { From a58565964a2214937fbc8869eb4f34e7163d65d9 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Tue, 16 Apr 2024 09:54:17 -0400 Subject: [PATCH 05/46] Fixup Signed-off-by: Benjamin Leggett --- src/proxy.rs | 3 +++ src/proxy/pool.rs | 8 ++++---- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/proxy.rs b/src/proxy.rs index d280fecac..1b15bf7b7 100644 --- a/src/proxy.rs +++ b/src/proxy.rs @@ -260,6 +260,9 @@ pub enum Error { #[error("connection streams maxed out")] WorkloadHBONEPoolConnStreamsMaxed, + #[error("pool draining")] + WorkloadHBONEPoolDraining, + // #[error("pool: {0}")] // WorkloadHBONEPool(#[from] custom_http2_pool::Error), #[error("{0}")] diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 3538700cd..0e40d308f 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -207,7 +207,7 @@ impl WorkloadHBONEPool { } } Err(_) => { - break None + return Err(Error::WorkloadHBONEPoolDraining) } } } @@ -526,7 +526,7 @@ mod test { #[tokio::test] async fn test_pool_100_clients_streamexhaust() { - crate::telemetry::setup_logging(); + // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; @@ -683,14 +683,14 @@ mod test { ..crate::config::parse_config().unwrap() }; - crate::telemetry::setup_logging(); + // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), - pool_max_streams_per_conn: 100, + pool_max_streams_per_conn: 1000, ..crate::config::parse_config().unwrap() }; let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); From 05e34015a6ce29528ab63becc6f3e39ec6c9a743 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Tue, 16 Apr 2024 10:05:20 -0400 Subject: [PATCH 06/46] Fixup Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 0e40d308f..9665f690c 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -51,7 +51,7 @@ static GLOBAL_CONN_COUNT: AtomicI32 = AtomicI32::new(0); // by flow control throttling. #[derive(Clone)] pub struct WorkloadHBONEPool { - pool_notifier: watch::Sender, + pool_notifier: Arc::>, // This is already impl clone? rustc complains that it isn't, tho pool_watcher: watch::Receiver, max_streamcount: u16, // this is effectively just a convenience data type - a rwlocked hashmap with keying and LRU drops @@ -78,7 +78,7 @@ impl WorkloadHBONEPool { cfg.pool_max_streams_per_conn ); Self { - pool_notifier: tx, + pool_notifier: Arc::new(tx), pool_watcher: rx, max_streamcount: cfg.pool_max_streams_per_conn, // the number here is simply the number of unique src/dest keys From 2a2fa32edd8a8fc3e78849dafcf751b836eaf91b Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Tue, 16 Apr 2024 11:33:59 -0400 Subject: [PATCH 07/46] More comments Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 111 ++++++++++++++++++++++++++++++++++++---------- 1 file changed, 87 insertions(+), 24 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 9665f690c..67607d01f 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -66,6 +66,9 @@ pub struct WorkloadHBONEPool { } impl WorkloadHBONEPool { + // Creates a new pool instance, which should be owned by a single proxied workload. + // The pool will watch the provided drain signal and drain itself when notified. + // Callers should then be safe to drop() the pool instance. pub fn new( cfg: crate::config::Config, socket_factory: Arc, @@ -92,9 +95,14 @@ impl WorkloadHBONEPool { } } + // Obtain a pooled connection. Will prefer to retrieve an existing conn from the pool, but + // if none exist, or the existing conn is maxed out on streamcount, will spawn a new one, + // even if it is to the same dest+port. + // + // If many `connects` request a connection to the same dest at once, all will wait until exactly + // one connection is created, before deciding if they should create more or just use that one. pub async fn connect(&mut self, key: WorkloadKey) -> Result { - // TODO BML this may not be collision resistant - // it may also be slow as shit + // TODO BML this may not be collision resistant/slow. It should be resistant enough for workloads tho. let mut s = DefaultHasher::new(); key.hash(&mut s); let hash_key = s.finish(); @@ -103,6 +111,16 @@ impl WorkloadHBONEPool { GLOBAL_CONN_COUNT.fetch_add(1, Ordering::Relaxed), ); + // First, see if we can naively just check out a connection. + // This should be the common case, except for the first establishment of a new connection/key. + // This will be done under outer readlock (nonexclusive)/inner keyed writelock (exclusive). + // + // It is important that the *initial* check here is authoritative, hence the locks, as + // we must know if this is a connection for a key *nobody* has tried to start yet, + // or if other things have already established conns for this key. + // + // This is so we can backpressure correctly if 1000 tasks all demand a new connection + // to the same key at once, and not eagerly open 1000 tunnel connections. let existing_conn = self .first_checkout_conn_from_pool(&key, hash_key, &pool_key) .await; @@ -111,12 +129,22 @@ impl WorkloadHBONEPool { debug!("using existing conn, connect future will be dropped on the floor"); Ok(existing_conn.unwrap()) } else { + // We couldn't get a conn. This means either nobody has tried to establish any conns for this key yet, + // or they have, but no conns are currently available + // (because someone else has checked all of them out and not put any back yet) + // + // So, we wil writelock the outer lock to see if an inner lock entry exists for our key. + // // critical block - this writelocks the entire pool for all tasks/threads - // as we check to see if anyone has inserted a sharded mutex for this key. - // So we want to hold this for as short as possible a time, and drop it + // as we check to see if anyone has ever inserted a sharded mutex for this key. + // + // If not, we insert one. + // + // We want to hold this for as short as possible a time, and drop it // before we hold it over an await. // - // this is is the only block where we should hold a writelock on the whole mutex map + // this is is the ONLY block where we should hold a writelock on the whole mutex map + // for the rest, a readlock (nonexclusive) is sufficient. { let mut map_write_lock = self.established_conn_writelock.write().await; match map_write_lock.get(&hash_key) { @@ -128,12 +156,15 @@ impl WorkloadHBONEPool { map_write_lock.insert(hash_key, Some(Mutex::new(()))); } }; - drop(map_write_lock); + drop(map_write_lock); // strictly redundant } - // Now we know _someone_ won the race to insert a conn mutex, we don't need a writelock - // on the outer map anymore - so we can just readlock the outer map, - // and get the inner mutex for this connkey that we care about. + + + // If we get here, it means the following are true: + // 1. We have a guaranteed sharded mutex in the outer map for our current key. + // 2. We can now, under readlock(nonexclusive) in the outer map, attempt to + // take the inner writelock for our specific key (exclusive). // // This unblocks other tasks spawning connections against other keys, but blocks other // tasks spawning connections against THIS key - which is what we want. @@ -149,13 +180,18 @@ impl WorkloadHBONEPool { // and we WANT other tasks to go back to sleep if there is an outstanding lock. // So the downsides are actually useful (we WANT task contention - // to block other parallel tasks from trying to spawn a connection if we are already doing so) + // + // BEGIN take outer readlock let map_read_lock = self.established_conn_writelock.read().await; let exist_conn_lock = map_read_lock.get(&hash_key).unwrap(); + // BEGIN take inner writelock let found_conn = match exist_conn_lock.as_ref().unwrap().try_lock() { Ok(_guard) => { - // if we get here, either we won the connlock race and can create one, - // or someone else won, but the streamcount for the one they added is already hit, - // so we should start and insert another. + // If we get here, it means the following are true: + // 1. We did not get a connection for our key. + // 2. We have the exclusive inner writelock to create a new connection for our key. + // + // So, carry on doing that. debug!("appears we need a new conn, retaining connlock"); debug!("nothing else is creating a conn, make one"); let pool_conn = self.spawn_new_pool_conn(key.clone()).await; @@ -171,14 +207,18 @@ impl WorkloadHBONEPool { ); debug!("dropping lock"); Some(client) + // END take inner writelock } Err(_) => { - // The sharded mutex for this connkey is already locked - someone else must be making a conn - // if they are, try to wait for it, but bail if we find one and it's got a maxed streamcount. debug!("something else is creating a conn, wait for it"); - // let waiter = self.pool_watcher.changed(); - // tokio::pin!(waiter); - + // If we get here, it means the following are true: + // 1. At one point, there was a preexisting conn in the pool for this key. + // 2. When we checked, we got nothing for that key. + // 3. We could not get the exclusive inner writelock to add a new one for this key. + // 4. Someone else got the exclusive inner writelock, and is adding a new one for this key. + // + // So, loop and wait for the pool_watcher to tell us a new conn was enpooled, + // so we can pull it out and check it. loop { match self.pool_watcher.changed().await { Ok(_) => { @@ -186,18 +226,17 @@ impl WorkloadHBONEPool { "notified a new conn was enpooled, checking for hash {:#?}", hash_key ); - - let existing_conn = self.connected_pool.get(&hash_key); // .and_then(|e_conn| { - - // debug!("while waiting for new conn, got existing conn for key {:#?}", key); - // }); - match existing_conn { + // The sharded mutex for this connkey is already locked - someone else must be making a conn + // if they are, try to wait for it, but bail if we find one and it's got a maxed streamcount. + let existing_conn = self.connected_pool.get(&hash_key); match existing_conn { None => { - debug!("got nothin"); + debug!("got nothing"); continue; } Some(e_conn) => { debug!("found existing conn after waiting"); + // We found a conn, but it's already maxed out. + // Return None and create another. if e_conn.at_max_streamcount() { debug!("found existing conn for key {:#?}, but streamcount is maxed", key); break None; @@ -207,6 +246,7 @@ impl WorkloadHBONEPool { } } Err(_) => { + // END take outer readlock return Err(Error::WorkloadHBONEPoolDraining) } } @@ -214,13 +254,35 @@ impl WorkloadHBONEPool { } }; + + // If we get here, it means the following are true: + // 1. At one point, there was a preexisting conn in the pool for this key. + // 2. When we checked, we got nothing for that key. + // 3. We could not get the exclusive inner writelock to add a new one + // 4. Someone else got the exclusive inner writelock, and is adding a new one + // 5. We waited until we got that connection for our key, that someone else added. + // + // So now we are down to 2 options: + // + // 1. We got an existing connection, it's usable, we use it. + // 2. We got an existing connection, it's not usable, we start another. + // + // Note that for (2) we do not really need to take the inner writelock again for this key - + // if we checked out a conn inserted by someone else that we cannot use, + // everyone else will implicitly be waiting for us to check one back in, + // since they are in their own loops. match found_conn { + // After waiting, we found an available conn we can use, so no need to start another. + // Clone the underlying client, return a copy, and put the other back in the pool. Some(f_conn) => { self.connected_pool.put(&pool_key, f_conn.clone()); let _ = self.pool_notifier.send(true); Ok(f_conn) } + // After waiting, we found an available conn, but for whatever reason couldn't use it. + // (streamcount maxed, etc) + // Start a new one, clone the underlying client, return a copy, and put the other back in the pool. None => { debug!("spawning new conn for key {:#?} to replace", key); let pool_conn = self.spawn_new_pool_conn(key.clone()).await; @@ -234,6 +296,7 @@ impl WorkloadHBONEPool { Ok(r_conn) } } + // END take outer readlock } } async fn first_checkout_conn_from_pool( From add0dce97377d6360f9fa354bc0513353a7641bd Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Tue, 16 Apr 2024 11:35:37 -0400 Subject: [PATCH 08/46] cleanup Signed-off-by: Benjamin Leggett --- src/proxy.rs | 2 -- src/proxy/inbound.rs | 1 - 2 files changed, 3 deletions(-) diff --git a/src/proxy.rs b/src/proxy.rs index 1b15bf7b7..3a4c28152 100644 --- a/src/proxy.rs +++ b/src/proxy.rs @@ -263,8 +263,6 @@ pub enum Error { #[error("pool draining")] WorkloadHBONEPoolDraining, - // #[error("pool: {0}")] - // WorkloadHBONEPool(#[from] custom_http2_pool::Error), #[error("{0}")] Generic(Box), diff --git a/src/proxy/inbound.rs b/src/proxy/inbound.rs index 78352d2db..379a72326 100644 --- a/src/proxy/inbound.rs +++ b/src/proxy/inbound.rs @@ -117,7 +117,6 @@ impl Inbound { debug!(%conn, "accepted connection"); let enable_original_source = self.pi.cfg.enable_original_source; let serve = crate::hyper_util::http2_server() - // .max_concurrent_streams(100) // TODO BML test .initial_stream_window_size(self.pi.cfg.window_size) .initial_connection_window_size(self.pi.cfg.connection_window_size) .max_frame_size(self.pi.cfg.frame_size) From b290040d361e250540fd44582dfb674249db7494 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Tue, 16 Apr 2024 11:46:11 -0400 Subject: [PATCH 09/46] fixup Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 67607d01f..57710be40 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -51,7 +51,7 @@ static GLOBAL_CONN_COUNT: AtomicI32 = AtomicI32::new(0); // by flow control throttling. #[derive(Clone)] pub struct WorkloadHBONEPool { - pool_notifier: Arc::>, // This is already impl clone? rustc complains that it isn't, tho + pool_notifier: Arc>, // This is already impl clone? rustc complains that it isn't, tho pool_watcher: watch::Receiver, max_streamcount: u16, // this is effectively just a convenience data type - a rwlocked hashmap with keying and LRU drops @@ -159,8 +159,6 @@ impl WorkloadHBONEPool { drop(map_write_lock); // strictly redundant } - - // If we get here, it means the following are true: // 1. We have a guaranteed sharded mutex in the outer map for our current key. // 2. We can now, under readlock(nonexclusive) in the outer map, attempt to @@ -228,7 +226,8 @@ impl WorkloadHBONEPool { ); // The sharded mutex for this connkey is already locked - someone else must be making a conn // if they are, try to wait for it, but bail if we find one and it's got a maxed streamcount. - let existing_conn = self.connected_pool.get(&hash_key); match existing_conn { + let existing_conn = self.connected_pool.get(&hash_key); + match existing_conn { None => { debug!("got nothing"); continue; @@ -247,14 +246,13 @@ impl WorkloadHBONEPool { } Err(_) => { // END take outer readlock - return Err(Error::WorkloadHBONEPoolDraining) + return Err(Error::WorkloadHBONEPoolDraining); } } } } }; - // If we get here, it means the following are true: // 1. At one point, there was a preexisting conn in the pool for this key. // 2. When we checked, we got nothing for that key. @@ -435,7 +433,9 @@ mod test { use tokio::io::AsyncWriteExt; use tokio::net::TcpListener; use tokio::task::{self}; - use tracing::{error, info, Instrument}; + + #[cfg(tokio_unstable)] + use tracing::Instrument; use ztunnel::test_helpers::*; @@ -864,7 +864,7 @@ mod test { wi.write_all(b"waypoint\n").await.unwrap(); tcp::handle_stream(tcp::Mode::ReadWrite, &mut ri, &mut wi).await; } - Err(e) => error!("No upgrade {e}"), + Err(e) => panic!("No upgrade {e}"), } }); Ok::<_, Infallible>(Response::new(http_body_util::Empty::::new())) From 17de2be492871101f861bfb35e7bd835383a0eb5 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Tue, 16 Apr 2024 11:54:53 -0400 Subject: [PATCH 10/46] Clean Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 17 +---------------- 1 file changed, 1 insertion(+), 16 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 57710be40..1cf6a0cef 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -634,11 +634,6 @@ mod test { #[tokio::test] async fn test_pool_100_clients_singleconn() { - let _cfg = crate::config::Config { - local_node: Some("local-node".to_string()), - ..crate::config::parse_config().unwrap() - }; - // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); @@ -684,11 +679,6 @@ mod test { #[tokio::test] async fn test_pool_100_clients_100_srcs() { - let _cfg = crate::config::Config { - local_node: Some("local-node".to_string()), - ..crate::config::parse_config().unwrap() - }; - // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); @@ -741,11 +731,6 @@ mod test { #[tokio::test] async fn test_pool_1000_clients_3_srcs() { - let _cfg = crate::config::Config { - local_node: Some("local-node".to_string()), - ..crate::config::parse_config().unwrap() - }; - // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); @@ -871,7 +856,7 @@ mod test { } let conn_count: Arc = Arc::new(AtomicU32::new(0)); - let _drop_conn_count: Arc = Arc::new(AtomicU32::new(0)); + // let _drop_conn_count: Arc = Arc::new(AtomicU32::new(0)); // We create a TcpListener and bind it to 127.0.0.1:3000 let listener = TcpListener::bind(addr).await.unwrap(); From 6ebfc30f777e2bfd30f2a3d5ff3631607e0e3844 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Tue, 16 Apr 2024 13:03:50 -0400 Subject: [PATCH 11/46] Fix jemalloc Signed-off-by: Benjamin Leggett --- src/admin.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/admin.rs b/src/admin.rs index 35c43e2f2..40075b397 100644 --- a/src/admin.rs +++ b/src/admin.rs @@ -395,7 +395,7 @@ fn change_log_level(reset: bool, level: &str) -> Response> { async fn handle_jemalloc_pprof_heapgen( _req: Request, ) -> anyhow::Result>> { - let mut prof_ctl = jemalloc_pprof::PROF_CTL.as_ref()?.lock().await; + let mut prof_ctl = jemalloc_pprof::PROF_CTL.as_ref().expect("should init").lock().await; if !prof_ctl.activated() { return Ok(Response::builder() .status(hyper::StatusCode::INTERNAL_SERVER_ERROR) @@ -405,7 +405,7 @@ async fn handle_jemalloc_pprof_heapgen( let pprof = prof_ctl.dump_pprof()?; Ok(Response::builder() .status(hyper::StatusCode::OK) - .body(Bytes::from(pprof?).into()) + .body(Bytes::from(pprof).into()) .expect("builder with known status code should not fail")) } From ed2a284dfeaf0ba18cfe1be9e1775143c99875f8 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Tue, 16 Apr 2024 15:47:19 -0400 Subject: [PATCH 12/46] WIP: move out of proxyinfo Signed-off-by: Benjamin Leggett --- src/proxy.rs | 10 ---------- src/proxy/outbound.rs | 19 +++++++++++++++++-- src/proxy/pool.rs | 11 +++++++++++ src/proxy/socks5.rs | 6 ++++++ src/proxyfactory.rs | 6 ------ 5 files changed, 34 insertions(+), 18 deletions(-) diff --git a/src/proxy.rs b/src/proxy.rs index 3a4c28152..080d34ae0 100644 --- a/src/proxy.rs +++ b/src/proxy.rs @@ -105,7 +105,6 @@ pub(super) struct ProxyInputs { hbone_port: u16, pub state: DemandProxyState, metrics: Arc, - pool: pool::WorkloadHBONEPool, socket_factory: Arc, proxy_workload_info: Option>, } @@ -120,7 +119,6 @@ impl ProxyInputs { metrics: Arc, socket_factory: Arc, proxy_workload_info: Option, - pool: pool::WorkloadHBONEPool, ) -> Self { Self { cfg, @@ -128,7 +126,6 @@ impl ProxyInputs { cert_manager, metrics, connection_manager, - pool, hbone_port: 0, socket_factory, proxy_workload_info: proxy_workload_info.map(Arc::new), @@ -147,19 +144,12 @@ impl Proxy { let metrics = Arc::new(metrics); let socket_factory = Arc::new(DefaultSocketFactory); - let pool = pool::WorkloadHBONEPool::new( - cfg.clone(), - socket_factory.clone(), - cert_manager.clone(), - drain.clone(), - ); let pi = ProxyInputs { cfg, state, cert_manager, connection_manager: ConnectionManager::default(), metrics, - pool, hbone_port: 0, socket_factory, proxy_workload_info: None, diff --git a/src/proxy/outbound.rs b/src/proxy/outbound.rs index 8d7baec9f..ecc29a7e5 100644 --- a/src/proxy/outbound.rs +++ b/src/proxy/outbound.rs @@ -78,6 +78,12 @@ impl Outbound { // // So use a drain to nuke tasks that might be stuck sending. let (sub_drain_signal, sub_drain) = drain::channel(); + + let pool = proxy::pool::WorkloadHBONEPool::new( + self.pi.cfg.clone(), + self.pi.socket_factory.clone(), + self.pi.cert_manager.clone(), + sub_drain.clone()); let accept = async move { loop { // Asynchronously wait for an inbound socket. @@ -89,6 +95,7 @@ impl Outbound { let mut oc = OutboundConnection { pi: self.pi.clone(), id: TraceParent::new(), + pool: pool.clone(), }; let span = info_span!("outbound", id=%oc.id); tokio::spawn( @@ -133,6 +140,7 @@ impl Outbound { pub(super) struct OutboundConnection { pub(super) pi: ProxyInputs, pub(super) id: TraceParent, + pub(super) pool: proxy::pool::WorkloadHBONEPool, } impl OutboundConnection { @@ -296,7 +304,11 @@ impl OutboundConnection { dst: req.gateway, }; - let mut connection = self.pi.pool.connect(pool_key.clone()).await?; + debug!("outbound - connection get START"); + let mut connection = self.pool.connect(pool_key.clone()) + .instrument(trace_span!("get pool conn")) + .await?; + debug!("outbound - connection get END"); let mut f = http_types::proxies::Forwarded::new(); f.add_for(remote_addr.to_string()); @@ -318,11 +330,14 @@ impl OutboundConnection { // There are scenarios (upstream hangup, etc) where this "send" will simply get stuck. // As in, stream processing deadlocks, and `send_request` never resolves to anything. // Probably related to https://github.com/hyperium/hyper/issues/3623 - let response = connection.send_request(request).await?; + let response = connection.send_request(request) + .instrument(trace_span!("send pool conn")) + .await?; debug!("outbound - connection send END"); let code = response.status(); if code != 200 { + debug!("outbound - connection send FAIL: {code}"); return Err(Error::HttpStatus(code)); } let upgraded = hyper::upgrade::on(response).await?; diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 1cf6a0cef..ae6dd3a46 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -102,6 +102,7 @@ impl WorkloadHBONEPool { // If many `connects` request a connection to the same dest at once, all will wait until exactly // one connection is created, before deciding if they should create more or just use that one. pub async fn connect(&mut self, key: WorkloadKey) -> Result { + debug!("pool connect START"); // TODO BML this may not be collision resistant/slow. It should be resistant enough for workloads tho. let mut s = DefaultHasher::new(); key.hash(&mut s); @@ -111,6 +112,7 @@ impl WorkloadHBONEPool { GLOBAL_CONN_COUNT.fetch_add(1, Ordering::Relaxed), ); + debug!("pool connect GET EXISTING"); // First, see if we can naively just check out a connection. // This should be the common case, except for the first establishment of a new connection/key. // This will be done under outer readlock (nonexclusive)/inner keyed writelock (exclusive). @@ -125,6 +127,7 @@ impl WorkloadHBONEPool { .first_checkout_conn_from_pool(&key, hash_key, &pool_key) .await; + debug!("pool connect GOT EXISTING"); if existing_conn.is_some() { debug!("using existing conn, connect future will be dropped on the floor"); Ok(existing_conn.unwrap()) @@ -146,7 +149,9 @@ impl WorkloadHBONEPool { // this is is the ONLY block where we should hold a writelock on the whole mutex map // for the rest, a readlock (nonexclusive) is sufficient. { + debug!("pool connect MAP OUTER WRITELOCK START"); let mut map_write_lock = self.established_conn_writelock.write().await; + debug!("pool connect MAP OUTER WRITELOCK END"); match map_write_lock.get(&hash_key) { Some(_) => { debug!("already have conn for key {:#?}", hash_key); @@ -180,9 +185,12 @@ impl WorkloadHBONEPool { // to block other parallel tasks from trying to spawn a connection if we are already doing so) // // BEGIN take outer readlock + debug!("pool connect MAP OUTER READLOCK START"); let map_read_lock = self.established_conn_writelock.read().await; + debug!("pool connect MAP OUTER READLOCK END"); let exist_conn_lock = map_read_lock.get(&hash_key).unwrap(); // BEGIN take inner writelock + debug!("pool connect MAP INNER WRITELOCK START"); let found_conn = match exist_conn_lock.as_ref().unwrap().try_lock() { Ok(_guard) => { // If we get here, it means the following are true: @@ -240,6 +248,7 @@ impl WorkloadHBONEPool { debug!("found existing conn for key {:#?}, but streamcount is maxed", key); break None; } + debug!("pool connect LOOP END"); break Some(e_conn); } } @@ -303,9 +312,11 @@ impl WorkloadHBONEPool { hash_key: u64, pool_key: &pingora_pool::ConnectionMeta, ) -> Option { + debug!("first checkout READLOCK"); let map_read_lock = self.established_conn_writelock.read().await; match map_read_lock.get(&hash_key) { Some(exist_conn_lock) => { + debug!("first checkout INNER WRITELOCK"); let _conn_lock = exist_conn_lock.as_ref().unwrap().lock().await; debug!("getting conn for key {:#?} and hash {:#?}", key, hash_key); diff --git a/src/proxy/socks5.rs b/src/proxy/socks5.rs index d935d5ab4..dac248a7f 100644 --- a/src/proxy/socks5.rs +++ b/src/proxy/socks5.rs @@ -65,12 +65,18 @@ impl Socks5 { let socket = self.listener.accept().await; let inpod = self.pi.cfg.inpod_enabled; let stream_drain = inner_drain.clone(); + let pool = crate::proxy::pool::WorkloadHBONEPool::new( + self.pi.cfg.clone(), + self.pi.socket_factory.clone(), + self.pi.cert_manager.clone(), + stream_drain.clone()); match socket { Ok((stream, remote)) => { info!("accepted outbound connection from {}", remote); let oc = OutboundConnection { pi: self.pi.clone(), id: TraceParent::new(), + pool, }; tokio::spawn(async move { if let Err(err) = handle(oc, stream, stream_drain, inpod).await { diff --git a/src/proxyfactory.rs b/src/proxyfactory.rs index 9a93a5f3d..99295068f 100644 --- a/src/proxyfactory.rs +++ b/src/proxyfactory.rs @@ -100,12 +100,6 @@ impl ProxyFactory { self.proxy_metrics.clone().unwrap(), socket_factory.clone(), proxy_workload_info, - crate::proxy::pool::WorkloadHBONEPool::new( - self.config.clone(), - socket_factory.clone(), - self.cert_manager.clone(), - drain.clone(), - ), ); result.connection_manager = Some(cm); result.proxy = Some(Proxy::from_inputs(pi, drain.clone()).await?); From 5abf00f0426e66e2703256a5d65cdc2426d32254 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Wed, 17 Apr 2024 16:54:17 -0400 Subject: [PATCH 13/46] Evict pooled conns after $INTERVAL Signed-off-by: Benjamin Leggett --- src/config.rs | 9 +++++++ src/proxy/outbound.rs | 14 +++++------ src/proxy/pool.rs | 56 ++++++++++++++++++++++++++++++++++++++----- src/proxy/socks5.rs | 2 ++ 4 files changed, 68 insertions(+), 13 deletions(-) diff --git a/src/config.rs b/src/config.rs index 3cacb53f2..2dd12dfbf 100644 --- a/src/config.rs +++ b/src/config.rs @@ -50,6 +50,7 @@ const SECRET_TTL: &str = "SECRET_TTL"; const FAKE_CA: &str = "FAKE_CA"; const ZTUNNEL_WORKER_THREADS: &str = "ZTUNNEL_WORKER_THREADS"; const POOL_MAX_STREAMS_PER_CONNECTION: &str = "POOL_MAX_STREAMS_PER_CONNECTION"; +const POOL_UNUSED_RELEASE_TIMEOUT: &str = "POOL_UNUSED_RELEASE_TIMEOUT"; const ENABLE_ORIG_SRC: &str = "ENABLE_ORIG_SRC"; const PROXY_CONFIG: &str = "PROXY_CONFIG"; @@ -64,6 +65,7 @@ const DEFAULT_SELFTERM_DEADLINE: Duration = Duration::from_secs(5); const DEFAULT_CLUSTER_ID: &str = "Kubernetes"; const DEFAULT_CLUSTER_DOMAIN: &str = "cluster.local"; const DEFAULT_TTL: Duration = Duration::from_secs(60 * 60 * 24); // 24 hours +const DEFAULT_POOL_RELEASE: Duration = Duration::from_secs(60 * 5); // 5 minutes const DEFAULT_INPOD_MARK: u32 = 1337; @@ -139,6 +141,8 @@ pub struct Config { // default stream queuing. pub pool_max_streams_per_conn: u16, + pub pool_unused_release_timeout: Duration, + pub socks5_addr: Option, pub admin_addr: SocketAddr, pub stats_addr: SocketAddr, @@ -337,6 +341,11 @@ pub fn construct_config(pc: ProxyConfig) -> Result { pool_max_streams_per_conn: parse_default(POOL_MAX_STREAMS_PER_CONNECTION, 250)?, + pool_unused_release_timeout: match parse::(POOL_UNUSED_RELEASE_TIMEOUT)? { + Some(ttl) => duration_str::parse(ttl).unwrap_or(DEFAULT_POOL_RELEASE), + None => DEFAULT_POOL_RELEASE, + }, + window_size: 4 * 1024 * 1024, connection_window_size: 4 * 1024 * 1024, frame_size: 1024 * 1024, diff --git a/src/proxy/outbound.rs b/src/proxy/outbound.rs index ecc29a7e5..e42ece86c 100644 --- a/src/proxy/outbound.rs +++ b/src/proxy/outbound.rs @@ -666,17 +666,17 @@ mod tests { hbone_port: 15008, cfg: cfg.clone(), metrics: test_proxy_metrics(), - pool: pool::WorkloadHBONEPool::new( - cfg, - sock_fact.clone(), - cert_mgr.clone(), - sub_drain, - ), - socket_factory: sock_fact, + socket_factory: sock_fact.clone(), proxy_workload_info: None, connection_manager: ConnectionManager::default(), }, id: TraceParent::new(), + pool: pool::WorkloadHBONEPool::new( + cfg, + sock_fact, + cert_mgr.clone(), + sub_drain, + ), }; let req = outbound diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index ae6dd3a46..0049da0fd 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -29,6 +29,8 @@ use std::sync::atomic::{AtomicI32, AtomicU16, Ordering}; use std::sync::Arc; use tokio::sync::watch; +use tokio::task; + use tokio::sync::{Mutex, RwLock}; use tracing::{debug, error}; @@ -53,6 +55,8 @@ static GLOBAL_CONN_COUNT: AtomicI32 = AtomicI32::new(0); pub struct WorkloadHBONEPool { pool_notifier: Arc>, // This is already impl clone? rustc complains that it isn't, tho pool_watcher: watch::Receiver, + timeout_send: Arc>, // This is already impl clone? rustc complains that it isn't, tho + timeout_recv: watch::Receiver, max_streamcount: u16, // this is effectively just a convenience data type - a rwlocked hashmap with keying and LRU drops // and has no actual hyper/http/connection logic. @@ -63,6 +67,7 @@ pub struct WorkloadHBONEPool { drainer: Watch, // this must be a readlockable list-of-locks, so we can lock per-key, not globally, and avoid holding up all conn attempts established_conn_writelock: Arc>>>>, + close_pollers: Arc>>, } impl WorkloadHBONEPool { @@ -76,6 +81,7 @@ impl WorkloadHBONEPool { drainer: Watch, //when signaled, will stop driving all conns in the pool, effectively draining the pool. ) -> WorkloadHBONEPool { let (tx, rx) = watch::channel(false); + let (timeout_send, timeout_recv) = watch::channel(false); debug!( "constructing pool with {:#?} streams per conn", cfg.pool_max_streams_per_conn @@ -83,6 +89,8 @@ impl WorkloadHBONEPool { Self { pool_notifier: Arc::new(tx), pool_watcher: rx, + timeout_send: Arc::new(timeout_send), + timeout_recv, max_streamcount: cfg.pool_max_streams_per_conn, // the number here is simply the number of unique src/dest keys // the pool is expected to track before the inner hashmap resizes. @@ -92,6 +100,7 @@ impl WorkloadHBONEPool { cert_manager, drainer, established_conn_writelock: Arc::new(RwLock::new(HashMap::new())), + close_pollers: Arc::new(futures::stream::FuturesUnordered::new()), } } @@ -282,8 +291,7 @@ impl WorkloadHBONEPool { // After waiting, we found an available conn we can use, so no need to start another. // Clone the underlying client, return a copy, and put the other back in the pool. Some(f_conn) => { - self.connected_pool.put(&pool_key, f_conn.clone()); - let _ = self.pool_notifier.send(true); + self.checkin_conn(f_conn.clone(), pool_key.clone()); Ok(f_conn) } @@ -298,8 +306,8 @@ impl WorkloadHBONEPool { Arc::new(AtomicU16::new(0)), self.max_streamcount, ); - self.connected_pool.put(&pool_key, r_conn.clone()); - let _ = self.pool_notifier.send(true); + + self.checkin_conn(r_conn.clone(), pool_key.clone()); Ok(r_conn) } } @@ -326,8 +334,7 @@ impl WorkloadHBONEPool { debug!("got conn for key {:#?}, but streamcount is maxed", key); None } else { - self.connected_pool.put(pool_key, e_conn.clone()); - let _ = self.pool_notifier.send(true); + self.checkin_conn(e_conn.clone(), pool_key.clone()); Some(e_conn) } }) @@ -336,6 +343,37 @@ impl WorkloadHBONEPool { } } + // This simply puts the connection back into the inner pool, + // and sets up a timed popper, which will resolve + // - when this reference is popped back out of the inner pool (doing nothing) + // - when this reference is evicted from the inner pool (doing nothing) + // - when the timeout_idler is drained (will pop) + // - when the timeout is hit (will pop) + // + // Idle poppers are safe to invoke if the conn they are popping is already gone + // from the inner queue, so we will start one for every insert, let them run or terminate on their own, + // and poll them to completion on shutdown. + // + // Note that "idle" in the context of this pool means "no one has asked for it or dropped it in X time, so prune it". + // + // Pruning the idle connection from the pool does not close it - it simply ensures the pool stops holding a ref. + // hyper self-closes client conns when all refs are dropped and streamcount is 0, so pool consumers must + // drop their checked out conns and/or terminate their streams as well. + // + // Note that this simply removes the client ref from this pool - if other things hold client/streamrefs refs, + // they must also drop those before the underlying connection is fully closed. + fn checkin_conn(&self, conn: Client, pool_key: pingora_pool::ConnectionMeta) { + let (evict, pickup) = self.connected_pool.put(&pool_key, conn); + let rx = self.timeout_recv.clone(); + let pool_ref = self.connected_pool.clone(); + let pool_key_ref = pool_key.clone(); + let release_timeout = self.cfg.pool_unused_release_timeout; + self.close_pollers.push(tokio::spawn(async move { + pool_ref.idle_timeout(&pool_key_ref, release_timeout, evict, rx, pickup).await; + })); + let _ = self.pool_notifier.send(true); + } + async fn spawn_new_pool_conn( &self, key: WorkloadKey, @@ -413,6 +451,12 @@ impl Client { } } +impl Drop for Client { + fn drop(&mut self) { + println!("Dropping Client!"); + } +} + #[derive(PartialEq, Eq, Hash, Clone, Debug)] pub struct WorkloadKey { pub src_id: Identity, diff --git a/src/proxy/socks5.rs b/src/proxy/socks5.rs index dac248a7f..04c1afe31 100644 --- a/src/proxy/socks5.rs +++ b/src/proxy/socks5.rs @@ -65,6 +65,8 @@ impl Socks5 { let socket = self.listener.accept().await; let inpod = self.pi.cfg.inpod_enabled; let stream_drain = inner_drain.clone(); + // TODO creating a new HBONE pool for SOCKS5 here may not be ideal, + // but ProxyInfo is overloaded and only `outbound` should ever use the pool. let pool = crate::proxy::pool::WorkloadHBONEPool::new( self.pi.cfg.clone(), self.pi.socket_factory.clone(), From 86d458c85fcf898659e44a6157c591fbf742749a Mon Sep 17 00:00:00 2001 From: Ben Leggett <854255+bleggett@users.noreply.github.com> Date: Wed, 17 Apr 2024 17:57:18 -0400 Subject: [PATCH 14/46] Update src/proxy/pool.rs Co-authored-by: Ian Rudie --- src/proxy/pool.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 0049da0fd..006e88f8f 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -108,7 +108,7 @@ impl WorkloadHBONEPool { // if none exist, or the existing conn is maxed out on streamcount, will spawn a new one, // even if it is to the same dest+port. // - // If many `connects` request a connection to the same dest at once, all will wait until exactly + // If many `connects` request a connection to the same dest at once, all will wait until exactly // one connection is created, before deciding if they should create more or just use that one. pub async fn connect(&mut self, key: WorkloadKey) -> Result { debug!("pool connect START"); From 417ee878d75b44580633505f80847290189e8eb8 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Wed, 17 Apr 2024 18:05:14 -0400 Subject: [PATCH 15/46] Evict pooled conns after $INTERVAL Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 006e88f8f..ce6b4d54b 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -70,6 +70,16 @@ pub struct WorkloadHBONEPool { close_pollers: Arc>>, } +impl Drop for WorkloadHBONEPool { + fn drop(&mut self) { + println!("pool dropping, cancelling all outstanding pool eviction timeout spawns"); + let _ = self.timeout_send.send(true); + // No need to wait for all `close_pollers` to resolve, + // since this is a drop - the recievers will either get the notification, or + // return an error if their sender drops first - either way they will resolve. + } +} + impl WorkloadHBONEPool { // Creates a new pool instance, which should be owned by a single proxied workload. // The pool will watch the provided drain signal and drain itself when notified. From 996ac4b0d33316350353f1e4f8a9a13f8aec88bf Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Wed, 17 Apr 2024 19:38:49 -0400 Subject: [PATCH 16/46] For now, just do the foolproof collision check Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 80 +++++++++++++++++++++++++++++++++-------------- 1 file changed, 57 insertions(+), 23 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index ce6b4d54b..666a3eaba 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -60,7 +60,7 @@ pub struct WorkloadHBONEPool { max_streamcount: u16, // this is effectively just a convenience data type - a rwlocked hashmap with keying and LRU drops // and has no actual hyper/http/connection logic. - connected_pool: Arc>, + connected_pool: Arc>, cfg: config::Config, socket_factory: Arc, cert_manager: Arc, @@ -120,7 +120,7 @@ impl WorkloadHBONEPool { // // If many `connects` request a connection to the same dest at once, all will wait until exactly // one connection is created, before deciding if they should create more or just use that one. - pub async fn connect(&mut self, key: WorkloadKey) -> Result { + pub async fn connect(&mut self, key: WorkloadKey) -> Result { debug!("pool connect START"); // TODO BML this may not be collision resistant/slow. It should be resistant enough for workloads tho. let mut s = DefaultHasher::new(); @@ -220,11 +220,12 @@ impl WorkloadHBONEPool { debug!("appears we need a new conn, retaining connlock"); debug!("nothing else is creating a conn, make one"); let pool_conn = self.spawn_new_pool_conn(key.clone()).await; - let client = Client( - pool_conn?, - Arc::new(AtomicU16::new(0)), - self.max_streamcount, - ); + let client = ConnClient{ + sender: pool_conn?, + stream_count: Arc::new(AtomicU16::new(0)), + stream_count_max: self.max_streamcount, + wl_key: key.clone(), + }; debug!( "starting new conn for key {:#?} with pk {:#?}", @@ -311,25 +312,41 @@ impl WorkloadHBONEPool { None => { debug!("spawning new conn for key {:#?} to replace", key); let pool_conn = self.spawn_new_pool_conn(key.clone()).await; - let r_conn = Client( - pool_conn?, - Arc::new(AtomicU16::new(0)), - self.max_streamcount, - ); - + let r_conn = ConnClient{ + sender: pool_conn?, + stream_count: Arc::new(AtomicU16::new(0)), + stream_count_max: self.max_streamcount, + wl_key: key.clone(), + }; self.checkin_conn(r_conn.clone(), pool_key.clone()); Ok(r_conn) } } // END take outer readlock - } + }.and_then(|conn| { + // Finally, we either have a conn or an error. + // Just for safety's sake, since we are using a hash thanks to pingora supporting arbitrary Eq, Hash + // types, do a deep equality test before returning the conn, returning an error if the conn's key does + // not equal the provided key + // + // this is a final safety check for collisions, we will throw up our hands and refuse to return the conn + match conn.is_for_workload(key) { + Ok(()) => { + return Ok(conn) + } + Err(e) => { + return Err(e) + } + } + }) + } async fn first_checkout_conn_from_pool( &self, key: &WorkloadKey, hash_key: u64, pool_key: &pingora_pool::ConnectionMeta, - ) -> Option { + ) -> Option { debug!("first checkout READLOCK"); let map_read_lock = self.established_conn_writelock.read().await; match map_read_lock.get(&hash_key) { @@ -372,7 +389,7 @@ impl WorkloadHBONEPool { // // Note that this simply removes the client ref from this pool - if other things hold client/streamrefs refs, // they must also drop those before the underlying connection is fully closed. - fn checkin_conn(&self, conn: Client, pool_key: pingora_pool::ConnectionMeta) { + fn checkin_conn(&self, conn: ConnClient, pool_key: pingora_pool::ConnectionMeta) { let (evict, pickup) = self.connected_pool.put(&pool_key, conn); let rx = self.timeout_recv.clone(); let pool_ref = self.connected_pool.clone(); @@ -439,13 +456,21 @@ impl WorkloadHBONEPool { } #[derive(Debug, Clone)] -pub struct Client(http2::SendRequest>, Arc, u16); +// A sort of faux-client, that represents a single checked-out 'request sender' which might +// send requests over some underlying stream using some underlying http/2 client +pub struct ConnClient { + sender: http2::SendRequest>, + stream_count: Arc, // the current streamcount for this client conn. + stream_count_max: u16, // the max streamcount associated with this client. + // A WL key may have many clients, but every client has no more than one WL key + wl_key: WorkloadKey, // the WL key associated with this client. +} -impl Client { +impl ConnClient { pub fn at_max_streamcount(&self) -> bool { - let curr_count = self.1.load(Ordering::Relaxed); + let curr_count = self.stream_count.load(Ordering::Relaxed); debug!("checking streamcount: {curr_count}"); - if curr_count >= self.2 { + if curr_count >= self.stream_count_max { return true; } false @@ -456,12 +481,21 @@ impl Client { req: Request>, ) -> impl Future>> { // TODO should we enforce streamcount per-sent-request? This would be slow. - self.1.fetch_add(1, Ordering::Relaxed); - self.0.send_request(req) + self.stream_count.fetch_add(1, Ordering::Relaxed); + self.sender.send_request(req) + } + + pub fn is_for_workload(&self, wl_key: WorkloadKey) -> Result<(), crate::proxy::Error> { + if !(self.wl_key == wl_key) { + return Err(crate::proxy::Error::Generic("fetched connection does not match workload key!".into())) + } else { + return Ok(()) + } } + } -impl Drop for Client { +impl Drop for ConnClient { fn drop(&mut self) { println!("Dropping Client!"); } From c413650fae8ebfc80e188084f9447639fe6cf87d Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Wed, 17 Apr 2024 19:47:45 -0400 Subject: [PATCH 17/46] Don't be silly Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 666a3eaba..f469ee9b0 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -104,7 +104,7 @@ impl WorkloadHBONEPool { max_streamcount: cfg.pool_max_streams_per_conn, // the number here is simply the number of unique src/dest keys // the pool is expected to track before the inner hashmap resizes. - connected_pool: Arc::new(pingora_pool::ConnectionPool::new(50000)), + connected_pool: Arc::new(pingora_pool::ConnectionPool::new(500)), cfg, socket_factory, cert_manager, From cf6b7c18e6af19f70c69bbb3c8c5afb7192175b1 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Wed, 17 Apr 2024 19:53:28 -0400 Subject: [PATCH 18/46] Naming, review comments Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 35 +++++++++++++++++------------------ 1 file changed, 17 insertions(+), 18 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index f469ee9b0..b8c9f632b 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -120,11 +120,11 @@ impl WorkloadHBONEPool { // // If many `connects` request a connection to the same dest at once, all will wait until exactly // one connection is created, before deciding if they should create more or just use that one. - pub async fn connect(&mut self, key: WorkloadKey) -> Result { + pub async fn connect(&mut self, workload_key: WorkloadKey) -> Result { debug!("pool connect START"); // TODO BML this may not be collision resistant/slow. It should be resistant enough for workloads tho. let mut s = DefaultHasher::new(); - key.hash(&mut s); + workload_key.hash(&mut s); let hash_key = s.finish(); let pool_key = pingora_pool::ConnectionMeta::new( hash_key, @@ -143,7 +143,7 @@ impl WorkloadHBONEPool { // This is so we can backpressure correctly if 1000 tasks all demand a new connection // to the same key at once, and not eagerly open 1000 tunnel connections. let existing_conn = self - .first_checkout_conn_from_pool(&key, hash_key, &pool_key) + .first_checkout_conn_from_pool(&workload_key, &pool_key) .await; debug!("pool connect GOT EXISTING"); @@ -219,17 +219,17 @@ impl WorkloadHBONEPool { // So, carry on doing that. debug!("appears we need a new conn, retaining connlock"); debug!("nothing else is creating a conn, make one"); - let pool_conn = self.spawn_new_pool_conn(key.clone()).await; + let pool_conn = self.spawn_new_pool_conn(workload_key.clone()).await; let client = ConnClient{ sender: pool_conn?, stream_count: Arc::new(AtomicU16::new(0)), stream_count_max: self.max_streamcount, - wl_key: key.clone(), + wl_key: workload_key.clone(), }; debug!( "starting new conn for key {:#?} with pk {:#?}", - key, pool_key + workload_key, pool_key ); debug!("dropping lock"); Some(client) @@ -265,7 +265,7 @@ impl WorkloadHBONEPool { // We found a conn, but it's already maxed out. // Return None and create another. if e_conn.at_max_streamcount() { - debug!("found existing conn for key {:#?}, but streamcount is maxed", key); + debug!("found existing conn for key {:#?}, but streamcount is maxed", workload_key); break None; } debug!("pool connect LOOP END"); @@ -310,13 +310,13 @@ impl WorkloadHBONEPool { // (streamcount maxed, etc) // Start a new one, clone the underlying client, return a copy, and put the other back in the pool. None => { - debug!("spawning new conn for key {:#?} to replace", key); - let pool_conn = self.spawn_new_pool_conn(key.clone()).await; + debug!("spawning new conn for key {:#?} to replace", workload_key); + let pool_conn = self.spawn_new_pool_conn(workload_key.clone()).await; let r_conn = ConnClient{ sender: pool_conn?, stream_count: Arc::new(AtomicU16::new(0)), stream_count_max: self.max_streamcount, - wl_key: key.clone(), + wl_key: workload_key.clone(), }; self.checkin_conn(r_conn.clone(), pool_key.clone()); Ok(r_conn) @@ -330,7 +330,7 @@ impl WorkloadHBONEPool { // not equal the provided key // // this is a final safety check for collisions, we will throw up our hands and refuse to return the conn - match conn.is_for_workload(key) { + match conn.is_for_workload(workload_key) { Ok(()) => { return Ok(conn) } @@ -343,22 +343,21 @@ impl WorkloadHBONEPool { } async fn first_checkout_conn_from_pool( &self, - key: &WorkloadKey, - hash_key: u64, + workload_key: &WorkloadKey, pool_key: &pingora_pool::ConnectionMeta, ) -> Option { debug!("first checkout READLOCK"); let map_read_lock = self.established_conn_writelock.read().await; - match map_read_lock.get(&hash_key) { + match map_read_lock.get(&pool_key.key) { Some(exist_conn_lock) => { debug!("first checkout INNER WRITELOCK"); let _conn_lock = exist_conn_lock.as_ref().unwrap().lock().await; - debug!("getting conn for key {:#?} and hash {:#?}", key, hash_key); - self.connected_pool.get(&hash_key).and_then(|e_conn| { - debug!("got existing conn for key {:#?}", key); + debug!("getting conn for key {:#?} and hash {:#?}", workload_key, pool_key.key); + self.connected_pool.get(&pool_key.key).and_then(|e_conn| { + debug!("got existing conn for key {:#?}", workload_key); if e_conn.at_max_streamcount() { - debug!("got conn for key {:#?}, but streamcount is maxed", key); + debug!("got conn for key {:#?}, but streamcount is maxed", workload_key); None } else { self.checkin_conn(e_conn.clone(), pool_key.clone()); From 036a2e626017bc62ce34bb47e349974da4fe6e5d Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Thu, 18 Apr 2024 13:41:49 -0400 Subject: [PATCH 19/46] Tidy Arcs+drains Signed-off-by: Benjamin Leggett --- src/proxy/outbound.rs | 7 +- src/proxy/pool.rs | 207 +++++++++++++++++++++++------------------- src/proxy/socks5.rs | 3 +- 3 files changed, 115 insertions(+), 102 deletions(-) diff --git a/src/proxy/outbound.rs b/src/proxy/outbound.rs index e42ece86c..8b3a435fa 100644 --- a/src/proxy/outbound.rs +++ b/src/proxy/outbound.rs @@ -82,8 +82,7 @@ impl Outbound { let pool = proxy::pool::WorkloadHBONEPool::new( self.pi.cfg.clone(), self.pi.socket_factory.clone(), - self.pi.cert_manager.clone(), - sub_drain.clone()); + self.pi.cert_manager.clone()); let accept = async move { loop { // Asynchronously wait for an inbound socket. @@ -674,9 +673,7 @@ mod tests { pool: pool::WorkloadHBONEPool::new( cfg, sock_fact, - cert_mgr.clone(), - sub_drain, - ), + cert_mgr.clone()), }; let req = outbound diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index b8c9f632b..e1f9985aa 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -15,6 +15,7 @@ use super::{Error, SocketFactory}; use bytes::Bytes; use drain::Watch; +use std::time::Duration; use http_body_util::Empty; use hyper::body::Incoming; use hyper::client::conn::http2; @@ -53,27 +54,71 @@ static GLOBAL_CONN_COUNT: AtomicI32 = AtomicI32::new(0); // by flow control throttling. #[derive(Clone)] pub struct WorkloadHBONEPool { - pool_notifier: Arc>, // This is already impl clone? rustc complains that it isn't, tho - pool_watcher: watch::Receiver, - timeout_send: Arc>, // This is already impl clone? rustc complains that it isn't, tho - timeout_recv: watch::Receiver, - max_streamcount: u16, // this is effectively just a convenience data type - a rwlocked hashmap with keying and LRU drops // and has no actual hyper/http/connection logic. - connected_pool: Arc>, cfg: config::Config, socket_factory: Arc, cert_manager: Arc, - drainer: Watch, + state: Arc, + pool_watcher: watch::Receiver, + max_streamcount: u16, +} + +struct PoolState { + pool_notifier: watch::Sender, // This is already impl clone? rustc complains that it isn't, tho + timeout_tx: watch::Sender, // This is already impl clone? rustc complains that it isn't, tho + timeout_rx: watch::Receiver, + // this is effectively just a convenience data type - a rwlocked hashmap with keying and LRU drops + // and has no actual hyper/http/connection logic. + connected_pool: Arc>, // this must be a readlockable list-of-locks, so we can lock per-key, not globally, and avoid holding up all conn attempts - established_conn_writelock: Arc>>>>, - close_pollers: Arc>>, + established_conn_writelock: RwLock>>>, + close_pollers: futures::stream::FuturesUnordered>, + pool_unused_release_timeout: Duration, } -impl Drop for WorkloadHBONEPool { +impl PoolState { + // This simply puts the connection back into the inner pool, + // and sets up a timed popper, which will resolve + // - when this reference is popped back out of the inner pool (doing nothing) + // - when this reference is evicted from the inner pool (doing nothing) + // - when the timeout_idler is drained (will pop) + // - when the timeout is hit (will pop) + // + // Idle poppers are safe to invoke if the conn they are popping is already gone + // from the inner queue, so we will start one for every insert, let them run or terminate on their own, + // and poll them to completion on shutdown. + // + // Note that "idle" in the context of this pool means "no one has asked for it or dropped it in X time, so prune it". + // + // Pruning the idle connection from the pool does not close it - it simply ensures the pool stops holding a ref. + // hyper self-closes client conns when all refs are dropped and streamcount is 0, so pool consumers must + // drop their checked out conns and/or terminate their streams as well. + // + // Note that this simply removes the client ref from this pool - if other things hold client/streamrefs refs, + // they must also drop those before the underlying connection is fully closed. + fn checkin_conn(&self, conn: ConnClient, pool_key: pingora_pool::ConnectionMeta) { + let (evict, pickup) = self.connected_pool.put(&pool_key, conn); + let rx = self.timeout_rx.clone(); + let pool_ref = self.connected_pool.clone(); + let pool_key_ref = pool_key.clone(); + let release_timeout = self.pool_unused_release_timeout; + self.close_pollers.push(tokio::spawn(async move { + pool_ref.idle_timeout(&pool_key_ref, release_timeout, evict, rx, pickup).await; + debug!("connection {:#?} was removed/checked out/timed out of the pool", pool_key_ref) + })); + let _ = self.pool_notifier.send(true); + } + +} +impl Drop for PoolState { fn drop(&mut self) { - println!("pool dropping, cancelling all outstanding pool eviction timeout spawns"); - let _ = self.timeout_send.send(true); + // let sc = Arc::strong_count(&self.timeout_tx); + // debug!("pool dropping, strong count is {sc}"); + // if sc == 1 { + debug!("poolstate dropping, cancelling all outstanding pool eviction timeout spawns"); + let _ = self.timeout_tx.send(true); + // } // No need to wait for all `close_pollers` to resolve, // since this is a drop - the recievers will either get the notification, or // return an error if their sender drops first - either way they will resolve. @@ -88,29 +133,35 @@ impl WorkloadHBONEPool { cfg: crate::config::Config, socket_factory: Arc, cert_manager: Arc, - drainer: Watch, //when signaled, will stop driving all conns in the pool, effectively draining the pool. + // drainer: Watch, //when signaled, will stop driving all conns in the pool, effectively draining the pool. ) -> WorkloadHBONEPool { - let (tx, rx) = watch::channel(false); + let (timeout_tx, timeout_rx) = watch::channel(false); let (timeout_send, timeout_recv) = watch::channel(false); + let (server_drain_signal, server_drain) = drain::channel(); + let max_count = cfg.pool_max_streams_per_conn; + let pool_duration = cfg.pool_unused_release_timeout; debug!( "constructing pool with {:#?} streams per conn", - cfg.pool_max_streams_per_conn + max_count ); Self { - pool_notifier: Arc::new(tx), - pool_watcher: rx, - timeout_send: Arc::new(timeout_send), - timeout_recv, - max_streamcount: cfg.pool_max_streams_per_conn, - // the number here is simply the number of unique src/dest keys - // the pool is expected to track before the inner hashmap resizes. - connected_pool: Arc::new(pingora_pool::ConnectionPool::new(500)), + state: Arc::new(PoolState{ + pool_notifier: timeout_tx, + timeout_tx: timeout_send, + timeout_rx: timeout_recv, + // the number here is simply the number of unique src/dest keys + // the pool is expected to track before the inner hashmap resizes. + connected_pool: Arc::new(pingora_pool::ConnectionPool::new(500)), + established_conn_writelock: RwLock::new(HashMap::new()), + close_pollers: futures::stream::FuturesUnordered::new(), + pool_unused_release_timeout: pool_duration, + }), cfg, socket_factory, cert_manager, - drainer, - established_conn_writelock: Arc::new(RwLock::new(HashMap::new())), - close_pollers: Arc::new(futures::stream::FuturesUnordered::new()), + pool_watcher: timeout_rx, + max_streamcount: max_count, + // drainer, } } @@ -169,7 +220,7 @@ impl WorkloadHBONEPool { // for the rest, a readlock (nonexclusive) is sufficient. { debug!("pool connect MAP OUTER WRITELOCK START"); - let mut map_write_lock = self.established_conn_writelock.write().await; + let mut map_write_lock = self.state.established_conn_writelock.write().await; debug!("pool connect MAP OUTER WRITELOCK END"); match map_write_lock.get(&hash_key) { Some(_) => { @@ -205,7 +256,7 @@ impl WorkloadHBONEPool { // // BEGIN take outer readlock debug!("pool connect MAP OUTER READLOCK START"); - let map_read_lock = self.established_conn_writelock.read().await; + let map_read_lock = self.state.established_conn_writelock.read().await; debug!("pool connect MAP OUTER READLOCK END"); let exist_conn_lock = map_read_lock.get(&hash_key).unwrap(); // BEGIN take inner writelock @@ -254,7 +305,7 @@ impl WorkloadHBONEPool { ); // The sharded mutex for this connkey is already locked - someone else must be making a conn // if they are, try to wait for it, but bail if we find one and it's got a maxed streamcount. - let existing_conn = self.connected_pool.get(&hash_key); + let existing_conn = self.state.connected_pool.get(&hash_key); match existing_conn { None => { debug!("got nothing"); @@ -302,7 +353,7 @@ impl WorkloadHBONEPool { // After waiting, we found an available conn we can use, so no need to start another. // Clone the underlying client, return a copy, and put the other back in the pool. Some(f_conn) => { - self.checkin_conn(f_conn.clone(), pool_key.clone()); + self.state.checkin_conn(f_conn.clone(), pool_key.clone()); Ok(f_conn) } @@ -318,7 +369,7 @@ impl WorkloadHBONEPool { stream_count_max: self.max_streamcount, wl_key: workload_key.clone(), }; - self.checkin_conn(r_conn.clone(), pool_key.clone()); + self.state.checkin_conn(r_conn.clone(), pool_key.clone()); Ok(r_conn) } } @@ -347,20 +398,20 @@ impl WorkloadHBONEPool { pool_key: &pingora_pool::ConnectionMeta, ) -> Option { debug!("first checkout READLOCK"); - let map_read_lock = self.established_conn_writelock.read().await; + let map_read_lock = self.state.established_conn_writelock.read().await; match map_read_lock.get(&pool_key.key) { Some(exist_conn_lock) => { debug!("first checkout INNER WRITELOCK"); let _conn_lock = exist_conn_lock.as_ref().unwrap().lock().await; debug!("getting conn for key {:#?} and hash {:#?}", workload_key, pool_key.key); - self.connected_pool.get(&pool_key.key).and_then(|e_conn| { + self.state.connected_pool.get(&pool_key.key).and_then(|e_conn| { debug!("got existing conn for key {:#?}", workload_key); if e_conn.at_max_streamcount() { debug!("got conn for key {:#?}, but streamcount is maxed", workload_key); None } else { - self.checkin_conn(e_conn.clone(), pool_key.clone()); + self.state.checkin_conn(e_conn.clone(), pool_key.clone()); Some(e_conn) } }) @@ -369,37 +420,6 @@ impl WorkloadHBONEPool { } } - // This simply puts the connection back into the inner pool, - // and sets up a timed popper, which will resolve - // - when this reference is popped back out of the inner pool (doing nothing) - // - when this reference is evicted from the inner pool (doing nothing) - // - when the timeout_idler is drained (will pop) - // - when the timeout is hit (will pop) - // - // Idle poppers are safe to invoke if the conn they are popping is already gone - // from the inner queue, so we will start one for every insert, let them run or terminate on their own, - // and poll them to completion on shutdown. - // - // Note that "idle" in the context of this pool means "no one has asked for it or dropped it in X time, so prune it". - // - // Pruning the idle connection from the pool does not close it - it simply ensures the pool stops holding a ref. - // hyper self-closes client conns when all refs are dropped and streamcount is 0, so pool consumers must - // drop their checked out conns and/or terminate their streams as well. - // - // Note that this simply removes the client ref from this pool - if other things hold client/streamrefs refs, - // they must also drop those before the underlying connection is fully closed. - fn checkin_conn(&self, conn: ConnClient, pool_key: pingora_pool::ConnectionMeta) { - let (evict, pickup) = self.connected_pool.put(&pool_key, conn); - let rx = self.timeout_recv.clone(); - let pool_ref = self.connected_pool.clone(); - let pool_key_ref = pool_key.clone(); - let release_timeout = self.cfg.pool_unused_release_timeout; - self.close_pollers.push(tokio::spawn(async move { - pool_ref.idle_timeout(&pool_key_ref, release_timeout, evict, rx, pickup).await; - })); - let _ = self.pool_notifier.send(true); - } - async fn spawn_new_pool_conn( &self, key: WorkloadKey, @@ -430,12 +450,12 @@ impl WorkloadHBONEPool { // spawn a task to poll the connection and drive the HTTP state // if we got a drain for that connection, respect it in a race // it is important to have a drain here, or this connection will never terminate - let driver_drain = self.drainer.clone(); + let mut driver_drain = self.state.timeout_rx.clone(); tokio::spawn(async move { debug!("starting a connection driver for {:?}", clone_key); tokio::select! { - _ = driver_drain.signaled() => { - debug!("draining outer HBONE connection"); + _ = driver_drain.changed() => { + debug!("draining outer HBONE connection {:?}", clone_key); } res = connection=> { match res { @@ -494,12 +514,6 @@ impl ConnClient { } -impl Drop for ConnClient { - fn drop(&mut self) { - println!("Dropping Client!"); - } -} - #[derive(PartialEq, Eq, Hash, Clone, Debug)] pub struct WorkloadKey { pub src_id: Identity, @@ -541,11 +555,11 @@ mod test { #[tokio::test] async fn test_pool_reuses_conn_for_same_key() { - // crate::telemetry::setup_logging(); + crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + let (server_addr, server_handle) = spawn_server(server_drain).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -555,7 +569,7 @@ mod test { let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); - let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr); let key1 = WorkloadKey { src_id: Identity::default(), @@ -567,8 +581,12 @@ mod test { let client2 = spawn_client(pool.clone(), key1.clone(), server_addr, 2).await; let client3 = spawn_client(pool.clone(), key1, server_addr, 2).await; - drop(pool); + assert!(client1.is_ok()); + assert!(client2.is_ok()); + assert!(client3.is_ok()); + server_drain_signal.drain().await; + drop(pool); let real_conncount = server_handle.await.unwrap(); assert!(real_conncount == 1, "actual conncount was {real_conncount}"); @@ -580,7 +598,7 @@ mod test { #[tokio::test] async fn test_pool_does_not_reuse_conn_for_diff_key() { let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + let (server_addr, server_handle) = spawn_server(server_drain).await; // crate::telemetry::setup_logging(); @@ -591,7 +609,7 @@ mod test { }; let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); - let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr); let key1 = WorkloadKey { src_id: Identity::default(), @@ -609,8 +627,8 @@ mod test { let client1 = spawn_client(pool.clone(), key1, server_addr, 2).await; let client2 = spawn_client(pool.clone(), key2, server_addr, 2).await; - drop(pool); server_drain_signal.drain().await; + drop(pool); let real_conncount = server_handle.await.unwrap(); assert!(real_conncount == 2, "actual conncount was {real_conncount}"); @@ -621,7 +639,7 @@ mod test { #[tokio::test] async fn test_pool_respects_per_conn_stream_limit() { let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + let (server_addr, server_handle) = spawn_server(server_drain).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -630,7 +648,7 @@ mod test { }; let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); - let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr); let key1 = WorkloadKey { src_id: Identity::default(), @@ -641,8 +659,8 @@ mod test { let client1 = spawn_client(pool.clone(), key1.clone(), server_addr, 4).await; let client2 = spawn_client(pool.clone(), key1, server_addr, 2).await; - drop(pool); server_drain_signal.drain().await; + drop(pool); let real_conncount = server_handle.await.unwrap(); assert!(real_conncount == 2, "actual conncount was {real_conncount}"); @@ -654,7 +672,7 @@ mod test { #[tokio::test] async fn test_pool_handles_many_conns_per_key() { let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + let (server_addr, server_handle) = spawn_server(server_drain).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -664,7 +682,7 @@ mod test { let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); - let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr); let key1 = WorkloadKey { src_id: Identity::default(), @@ -690,7 +708,7 @@ mod test { // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + let (server_addr, server_handle) = spawn_server(server_drain).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -699,7 +717,7 @@ mod test { }; let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); - let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr); let key1 = WorkloadKey { src_id: Identity::default(), @@ -724,7 +742,6 @@ mod test { } drop(pool); - server_drain_signal.drain().await; let real_conncount = server_handle.await.unwrap(); assert!(real_conncount == 3, "actual conncount was {real_conncount}"); @@ -735,7 +752,7 @@ mod test { // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + let (server_addr, server_handle) = spawn_server(server_drain).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -744,7 +761,7 @@ mod test { }; let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); - let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr); let key1 = WorkloadKey { src_id: Identity::default(), @@ -780,7 +797,7 @@ mod test { // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + let (server_addr, server_handle) = spawn_server(server_drain).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -789,7 +806,7 @@ mod test { }; let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); - let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr); let client_count = 100; let mut count = 0u8; @@ -832,7 +849,7 @@ mod test { // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain.clone()).await; + let (server_addr, server_handle) = spawn_server(server_drain).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -841,7 +858,7 @@ mod test { }; let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); - let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr, server_drain); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr); let mut key1 = WorkloadKey { src_id: Identity::default(), diff --git a/src/proxy/socks5.rs b/src/proxy/socks5.rs index 04c1afe31..dc8acace2 100644 --- a/src/proxy/socks5.rs +++ b/src/proxy/socks5.rs @@ -70,8 +70,7 @@ impl Socks5 { let pool = crate::proxy::pool::WorkloadHBONEPool::new( self.pi.cfg.clone(), self.pi.socket_factory.clone(), - self.pi.cert_manager.clone(), - stream_drain.clone()); + self.pi.cert_manager.clone()); match socket { Ok((stream, remote)) => { info!("accepted outbound connection from {}", remote); From b04e5e97b84372b9b7bf14f2d6e28fc201c288b3 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Thu, 18 Apr 2024 14:34:01 -0400 Subject: [PATCH 20/46] Cleanups Signed-off-by: Benjamin Leggett --- src/proxy/outbound.rs | 1 - src/proxy/pool.rs | 86 +++++++++++++++++++------------------------ 2 files changed, 38 insertions(+), 49 deletions(-) diff --git a/src/proxy/outbound.rs b/src/proxy/outbound.rs index 8b3a435fa..c837b6540 100644 --- a/src/proxy/outbound.rs +++ b/src/proxy/outbound.rs @@ -657,7 +657,6 @@ mod tests { let sock_fact = std::sync::Arc::new(crate::proxy::DefaultSocketFactory); let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); - let (_, sub_drain) = drain::channel(); let outbound = OutboundConnection { pi: ProxyInputs { cert_manager: identity::mock::new_secret_manager(Duration::from_secs(10)), diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index e1f9985aa..752822a4b 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -14,7 +14,6 @@ use super::{Error, SocketFactory}; use bytes::Bytes; -use drain::Watch; use std::time::Duration; use http_body_util::Empty; use hyper::body::Incoming; @@ -110,18 +109,40 @@ impl PoolState { let _ = self.pool_notifier.send(true); } + async fn first_checkout_conn_from_pool( + &self, + workload_key: &WorkloadKey, + pool_key: &pingora_pool::ConnectionMeta, + ) -> Option { + debug!("first checkout READLOCK"); + let map_read_lock = self.established_conn_writelock.read().await; + match map_read_lock.get(&pool_key.key) { + Some(exist_conn_lock) => { + debug!("first checkout INNER WRITELOCK"); + let _conn_lock = exist_conn_lock.as_ref().unwrap().lock().await; + + debug!("getting conn for key {:#?} and hash {:#?}", workload_key, pool_key.key); + self.connected_pool.get(&pool_key.key).and_then(|e_conn| { + debug!("got existing conn for key {:#?}", workload_key); + if e_conn.at_max_streamcount() { + debug!("got conn for key {:#?}, but streamcount is maxed", workload_key); + None + } else { + self.checkin_conn(e_conn.clone(), pool_key.clone()); + Some(e_conn) + } + }) + } + None => None, + } + } + } + impl Drop for PoolState { fn drop(&mut self) { - // let sc = Arc::strong_count(&self.timeout_tx); - // debug!("pool dropping, strong count is {sc}"); - // if sc == 1 { - debug!("poolstate dropping, cancelling all outstanding pool eviction timeout spawns"); - let _ = self.timeout_tx.send(true); - // } - // No need to wait for all `close_pollers` to resolve, - // since this is a drop - the recievers will either get the notification, or - // return an error if their sender drops first - either way they will resolve. + debug!("poolstate dropping, cancelling all outstanding pool eviction timeout spawns"); + let _ = self.timeout_tx.send(true); } } @@ -133,11 +154,9 @@ impl WorkloadHBONEPool { cfg: crate::config::Config, socket_factory: Arc, cert_manager: Arc, - // drainer: Watch, //when signaled, will stop driving all conns in the pool, effectively draining the pool. ) -> WorkloadHBONEPool { let (timeout_tx, timeout_rx) = watch::channel(false); let (timeout_send, timeout_recv) = watch::channel(false); - let (server_drain_signal, server_drain) = drain::channel(); let max_count = cfg.pool_max_streams_per_conn; let pool_duration = cfg.pool_unused_release_timeout; debug!( @@ -161,7 +180,6 @@ impl WorkloadHBONEPool { cert_manager, pool_watcher: timeout_rx, max_streamcount: max_count, - // drainer, } } @@ -193,9 +211,7 @@ impl WorkloadHBONEPool { // // This is so we can backpressure correctly if 1000 tasks all demand a new connection // to the same key at once, and not eagerly open 1000 tunnel connections. - let existing_conn = self - .first_checkout_conn_from_pool(&workload_key, &pool_key) - .await; + let existing_conn = self.state.first_checkout_conn_from_pool(&workload_key, &pool_key).await; debug!("pool connect GOT EXISTING"); if existing_conn.is_some() { @@ -383,42 +399,15 @@ impl WorkloadHBONEPool { // this is a final safety check for collisions, we will throw up our hands and refuse to return the conn match conn.is_for_workload(workload_key) { Ok(()) => { - return Ok(conn) + Ok(conn) } Err(e) => { - return Err(e) + Err(e) } } }) } - async fn first_checkout_conn_from_pool( - &self, - workload_key: &WorkloadKey, - pool_key: &pingora_pool::ConnectionMeta, - ) -> Option { - debug!("first checkout READLOCK"); - let map_read_lock = self.state.established_conn_writelock.read().await; - match map_read_lock.get(&pool_key.key) { - Some(exist_conn_lock) => { - debug!("first checkout INNER WRITELOCK"); - let _conn_lock = exist_conn_lock.as_ref().unwrap().lock().await; - - debug!("getting conn for key {:#?} and hash {:#?}", workload_key, pool_key.key); - self.state.connected_pool.get(&pool_key.key).and_then(|e_conn| { - debug!("got existing conn for key {:#?}", workload_key); - if e_conn.at_max_streamcount() { - debug!("got conn for key {:#?}, but streamcount is maxed", workload_key); - None - } else { - self.state.checkin_conn(e_conn.clone(), pool_key.clone()); - Some(e_conn) - } - }) - } - None => None, - } - } async fn spawn_new_pool_conn( &self, @@ -472,6 +461,7 @@ impl WorkloadHBONEPool { Ok(request_sender) } + } #[derive(Debug, Clone)] @@ -506,9 +496,9 @@ impl ConnClient { pub fn is_for_workload(&self, wl_key: WorkloadKey) -> Result<(), crate::proxy::Error> { if !(self.wl_key == wl_key) { - return Err(crate::proxy::Error::Generic("fetched connection does not match workload key!".into())) + Err(crate::proxy::Error::Generic("fetched connection does not match workload key!".into())) } else { - return Ok(()) + Ok(()) } } @@ -555,7 +545,7 @@ mod test { #[tokio::test] async fn test_pool_reuses_conn_for_same_key() { - crate::telemetry::setup_logging(); + // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); From 53e790abdadfe91dc416343017e895832f61cc1a Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Thu, 18 Apr 2024 14:48:53 -0400 Subject: [PATCH 21/46] Format Signed-off-by: Benjamin Leggett --- src/admin.rs | 6 +++++- src/proxy/outbound.rs | 45 +++++++++++++++++-------------------------- src/proxy/pool.rs | 43 +++++++++++++++++++++++++---------------- src/proxy/socks5.rs | 7 ++++--- 4 files changed, 53 insertions(+), 48 deletions(-) diff --git a/src/admin.rs b/src/admin.rs index 40075b397..686709927 100644 --- a/src/admin.rs +++ b/src/admin.rs @@ -395,7 +395,11 @@ fn change_log_level(reset: bool, level: &str) -> Response> { async fn handle_jemalloc_pprof_heapgen( _req: Request, ) -> anyhow::Result>> { - let mut prof_ctl = jemalloc_pprof::PROF_CTL.as_ref().expect("should init").lock().await; + let mut prof_ctl = jemalloc_pprof::PROF_CTL + .as_ref() + .expect("should init") + .lock() + .await; if !prof_ctl.activated() { return Ok(Response::builder() .status(hyper::StatusCode::INTERNAL_SERVER_ERROR) diff --git a/src/proxy/outbound.rs b/src/proxy/outbound.rs index c837b6540..86583531b 100644 --- a/src/proxy/outbound.rs +++ b/src/proxy/outbound.rs @@ -80,9 +80,10 @@ impl Outbound { let (sub_drain_signal, sub_drain) = drain::channel(); let pool = proxy::pool::WorkloadHBONEPool::new( - self.pi.cfg.clone(), - self.pi.socket_factory.clone(), - self.pi.cert_manager.clone()); + self.pi.cfg.clone(), + self.pi.socket_factory.clone(), + self.pi.cert_manager.clone(), + ); let accept = async move { loop { // Asynchronously wait for an inbound socket. @@ -147,13 +148,8 @@ impl OutboundConnection { let source_addr = socket::to_canonical(source_stream.peer_addr().expect("must receive peer addr")); let dst_addr = socket::orig_dst_addr_or_default(&source_stream); - self.proxy_to( - source_stream, - source_addr, - dst_addr, - false, - ) - .await; + self.proxy_to(source_stream, source_addr, dst_addr, false) + .await; } // this is a cancellable outbound proxy. If `out_drain` is a Watch drain, will resolve @@ -251,13 +247,8 @@ impl OutboundConnection { let res = match req.protocol { Protocol::HBONE => { - self.proxy_to_hbone( - &mut source_stream, - source_addr, - &req, - &result_tracker, - ) - .await + self.proxy_to_hbone(&mut source_stream, source_addr, &req, &result_tracker) + .await } Protocol::TCP => { self.proxy_to_tcp(&mut source_stream, &req, &result_tracker) @@ -304,9 +295,11 @@ impl OutboundConnection { }; debug!("outbound - connection get START"); - let mut connection = self.pool.connect(pool_key.clone()) - .instrument(trace_span!("get pool conn")) - .await?; + let mut connection = self + .pool + .connect(pool_key.clone()) + .instrument(trace_span!("get pool conn")) + .await?; debug!("outbound - connection get END"); let mut f = http_types::proxies::Forwarded::new(); @@ -329,9 +322,10 @@ impl OutboundConnection { // There are scenarios (upstream hangup, etc) where this "send" will simply get stuck. // As in, stream processing deadlocks, and `send_request` never resolves to anything. // Probably related to https://github.com/hyperium/hyper/issues/3623 - let response = connection.send_request(request) - .instrument(trace_span!("send pool conn")) - .await?; + let response = connection + .send_request(request) + .instrument(trace_span!("send pool conn")) + .await?; debug!("outbound - connection send END"); let code = response.status(); @@ -669,10 +663,7 @@ mod tests { connection_manager: ConnectionManager::default(), }, id: TraceParent::new(), - pool: pool::WorkloadHBONEPool::new( - cfg, - sock_fact, - cert_mgr.clone()), + pool: pool::WorkloadHBONEPool::new(cfg, sock_fact, cert_mgr.clone()), }; let req = outbound diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 752822a4b..dd2565e37 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -14,11 +14,11 @@ use super::{Error, SocketFactory}; use bytes::Bytes; -use std::time::Duration; use http_body_util::Empty; use hyper::body::Incoming; use hyper::client::conn::http2; use hyper::http::{Request, Response}; +use std::time::Duration; use std::collections::hash_map::DefaultHasher; use std::future::Future; @@ -103,8 +103,13 @@ impl PoolState { let pool_key_ref = pool_key.clone(); let release_timeout = self.pool_unused_release_timeout; self.close_pollers.push(tokio::spawn(async move { - pool_ref.idle_timeout(&pool_key_ref, release_timeout, evict, rx, pickup).await; - debug!("connection {:#?} was removed/checked out/timed out of the pool", pool_key_ref) + pool_ref + .idle_timeout(&pool_key_ref, release_timeout, evict, rx, pickup) + .await; + debug!( + "connection {:#?} was removed/checked out/timed out of the pool", + pool_key_ref + ) })); let _ = self.pool_notifier.send(true); } @@ -121,11 +126,17 @@ impl PoolState { debug!("first checkout INNER WRITELOCK"); let _conn_lock = exist_conn_lock.as_ref().unwrap().lock().await; - debug!("getting conn for key {:#?} and hash {:#?}", workload_key, pool_key.key); + debug!( + "getting conn for key {:#?} and hash {:#?}", + workload_key, pool_key.key + ); self.connected_pool.get(&pool_key.key).and_then(|e_conn| { debug!("got existing conn for key {:#?}", workload_key); if e_conn.at_max_streamcount() { - debug!("got conn for key {:#?}, but streamcount is maxed", workload_key); + debug!( + "got conn for key {:#?}, but streamcount is maxed", + workload_key + ); None } else { self.checkin_conn(e_conn.clone(), pool_key.clone()); @@ -136,7 +147,6 @@ impl PoolState { None => None, } } - } impl Drop for PoolState { @@ -159,12 +169,9 @@ impl WorkloadHBONEPool { let (timeout_send, timeout_recv) = watch::channel(false); let max_count = cfg.pool_max_streams_per_conn; let pool_duration = cfg.pool_unused_release_timeout; - debug!( - "constructing pool with {:#?} streams per conn", - max_count - ); + debug!("constructing pool with {:#?} streams per conn", max_count); Self { - state: Arc::new(PoolState{ + state: Arc::new(PoolState { pool_notifier: timeout_tx, timeout_tx: timeout_send, timeout_rx: timeout_recv, @@ -211,7 +218,10 @@ impl WorkloadHBONEPool { // // This is so we can backpressure correctly if 1000 tasks all demand a new connection // to the same key at once, and not eagerly open 1000 tunnel connections. - let existing_conn = self.state.first_checkout_conn_from_pool(&workload_key, &pool_key).await; + let existing_conn = self + .state + .first_checkout_conn_from_pool(&workload_key, &pool_key) + .await; debug!("pool connect GOT EXISTING"); if existing_conn.is_some() { @@ -406,7 +416,6 @@ impl WorkloadHBONEPool { } } }) - } async fn spawn_new_pool_conn( @@ -461,7 +470,6 @@ impl WorkloadHBONEPool { Ok(request_sender) } - } #[derive(Debug, Clone)] @@ -470,7 +478,7 @@ impl WorkloadHBONEPool { pub struct ConnClient { sender: http2::SendRequest>, stream_count: Arc, // the current streamcount for this client conn. - stream_count_max: u16, // the max streamcount associated with this client. + stream_count_max: u16, // the max streamcount associated with this client. // A WL key may have many clients, but every client has no more than one WL key wl_key: WorkloadKey, // the WL key associated with this client. } @@ -496,12 +504,13 @@ impl ConnClient { pub fn is_for_workload(&self, wl_key: WorkloadKey) -> Result<(), crate::proxy::Error> { if !(self.wl_key == wl_key) { - Err(crate::proxy::Error::Generic("fetched connection does not match workload key!".into())) + Err(crate::proxy::Error::Generic( + "fetched connection does not match workload key!".into(), + )) } else { Ok(()) } } - } #[derive(PartialEq, Eq, Hash, Clone, Debug)] diff --git a/src/proxy/socks5.rs b/src/proxy/socks5.rs index dc8acace2..0346a7e2b 100644 --- a/src/proxy/socks5.rs +++ b/src/proxy/socks5.rs @@ -68,9 +68,10 @@ impl Socks5 { // TODO creating a new HBONE pool for SOCKS5 here may not be ideal, // but ProxyInfo is overloaded and only `outbound` should ever use the pool. let pool = crate::proxy::pool::WorkloadHBONEPool::new( - self.pi.cfg.clone(), - self.pi.socket_factory.clone(), - self.pi.cert_manager.clone()); + self.pi.cfg.clone(), + self.pi.socket_factory.clone(), + self.pi.cert_manager.clone(), + ); match socket { Ok((stream, remote)) => { info!("accepted outbound connection from {}", remote); From 6be0b55da727842ba494210cac2e32089416587f Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Thu, 18 Apr 2024 15:51:50 -0400 Subject: [PATCH 22/46] Use the fancy lockless outer map, drop realm-io Signed-off-by: Benjamin Leggett --- Cargo.lock | 53 +++++++++++++++++++++++++++++ Cargo.toml | 1 + fuzz/Cargo.lock | 53 +++++++++++++++++++++++++++++ src/proxy/pool.rs | 87 ++++++++++++++++++++++++++++++++--------------- 4 files changed, 166 insertions(+), 28 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8a72c3da4..138accc7e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -34,6 +34,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" dependencies = [ "cfg-if", + "const-random", "once_cell", "version_check", "zerocopy", @@ -482,6 +483,26 @@ version = "0.9.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2459377285ad874054d797f3ccebf984978aa39129f6eafde5cdc8315b612f8" +[[package]] +name = "const-random" +version = "0.1.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87e00182fe74b066627d63b85fd550ac2998d4b0bd86bfed477a0ae4c7c71359" +dependencies = [ + "const-random-macro", +] + +[[package]] +name = "const-random-macro" +version = "0.1.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" +dependencies = [ + "getrandom 0.2.12", + "once_cell", + "tiny-keccak", +] + [[package]] name = "core-foundation" version = "0.9.4" @@ -796,6 +817,18 @@ dependencies = [ "miniz_oxide", ] +[[package]] +name = "flurry" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7874ce5eeafa5e546227f7c62911e586387bf03d6c9a45ac78aa1c3bc2fedb61" +dependencies = [ + "ahash", + "num_cpus", + "parking_lot", + "seize", +] + [[package]] name = "fnv" version = "1.0.7" @@ -2621,6 +2654,16 @@ dependencies = [ "libc", ] +[[package]] +name = "seize" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0e5739de653b129b0a59da381599cf17caf24bc586f6a797c52d3d6147c5b85a" +dependencies = [ + "num_cpus", + "once_cell", +] + [[package]] name = "semver" version = "1.0.22" @@ -2973,6 +3016,15 @@ dependencies = [ "time-core", ] +[[package]] +name = "tiny-keccak" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c9d3793400a45f954c52e73d068316d76b6f4e36977e3fcebb13a2721e80237" +dependencies = [ + "crunchy", +] + [[package]] name = "tinytemplate" version = "1.2.1" @@ -3677,6 +3729,7 @@ dependencies = [ "diff", "drain", "duration-str", + "flurry", "futures", "futures-core", "futures-util", diff --git a/Cargo.toml b/Cargo.toml index e2028a336..d33ef5be4 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -99,6 +99,7 @@ tracing-log = "0.2" backoff = "0.4.0" pin-project-lite = "0.2" pingora-pool = "0.1.0" +flurry = "0.5.0" [target.'cfg(target_os = "linux")'.dependencies] netns-rs = "0.1" diff --git a/fuzz/Cargo.lock b/fuzz/Cargo.lock index c63703336..e02024e91 100644 --- a/fuzz/Cargo.lock +++ b/fuzz/Cargo.lock @@ -24,6 +24,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "42cd52102d3df161c77a887b608d7a4897d7cc112886a9537b738a887a03aaff" dependencies = [ "cfg-if", + "const-random", "once_cell", "version_check", "zerocopy", @@ -364,6 +365,26 @@ dependencies = [ "crossbeam-utils", ] +[[package]] +name = "const-random" +version = "0.1.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87e00182fe74b066627d63b85fd550ac2998d4b0bd86bfed477a0ae4c7c71359" +dependencies = [ + "const-random-macro", +] + +[[package]] +name = "const-random-macro" +version = "0.1.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" +dependencies = [ + "getrandom 0.2.12", + "once_cell", + "tiny-keccak", +] + [[package]] name = "core-foundation" version = "0.9.4" @@ -622,6 +643,18 @@ version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" +[[package]] +name = "flurry" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7874ce5eeafa5e546227f7c62911e586387bf03d6c9a45ac78aa1c3bc2fedb61" +dependencies = [ + "ahash", + "num_cpus", + "parking_lot", + "seize", +] + [[package]] name = "fnv" version = "1.0.7" @@ -2254,6 +2287,16 @@ dependencies = [ "libc", ] +[[package]] +name = "seize" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0e5739de653b129b0a59da381599cf17caf24bc586f6a797c52d3d6147c5b85a" +dependencies = [ + "num_cpus", + "once_cell", +] + [[package]] name = "semver" version = "1.0.21" @@ -2516,6 +2559,15 @@ dependencies = [ "time-core", ] +[[package]] +name = "tiny-keccak" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c9d3793400a45f954c52e73d068316d76b6f4e36977e3fcebb13a2721e80237" +dependencies = [ + "crunchy", +] + [[package]] name = "tinytemplate" version = "1.2.1" @@ -3212,6 +3264,7 @@ dependencies = [ "chrono", "drain", "duration-str", + "flurry", "futures", "futures-core", "futures-util", diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index dd2565e37..4f8c278e5 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -31,13 +31,13 @@ use std::sync::Arc; use tokio::sync::watch; use tokio::task; -use tokio::sync::{Mutex, RwLock}; +use tokio::sync::Mutex; use tracing::{debug, error}; use crate::config; use crate::identity::{Identity, SecretManager}; -use std::collections::HashMap; +use flurry::HashMap; use pingora_pool; @@ -71,7 +71,7 @@ struct PoolState { // and has no actual hyper/http/connection logic. connected_pool: Arc>, // this must be a readlockable list-of-locks, so we can lock per-key, not globally, and avoid holding up all conn attempts - established_conn_writelock: RwLock>>>, + established_conn_writelock: HashMap>>>, close_pollers: futures::stream::FuturesUnordered>, pool_unused_release_timeout: Duration, } @@ -119,12 +119,34 @@ impl PoolState { workload_key: &WorkloadKey, pool_key: &pingora_pool::ConnectionMeta, ) -> Option { - debug!("first checkout READLOCK"); - let map_read_lock = self.established_conn_writelock.read().await; - match map_read_lock.get(&pool_key.key) { + debug!("first checkout READGUARD"); + + let found_conn = { + // BEGIN take outer readlock + debug!("pool connect MAP OUTER READ/WRITE GUARD"); + let guard = self.established_conn_writelock.guard(); + debug!("pool connect MAP OUTER READ/WRITE START"); + + //OLD + // debug!("pool connect MAP OUTER READLOCK START"); + // let map_read_lock = self.state.established_conn_writelock.read().await; + // debug!("pool connect MAP OUTER READLOCK END"); + let exist_conn_lock = self.established_conn_writelock.get(&pool_key.key, &guard); + // BEGIN take inner writelock + debug!("pool connect MAP INNER WRITELOCK START"); + match exist_conn_lock { + Some(e_conn_lock) => e_conn_lock.clone(), + None => None, + } + // exist_conn_lock.as_ref().unwrap().clone() + }; + // let guard = self.established_conn_writelock.guard(); + // debug!("pool connect MAP OUTER READGUARD GOT"); + match found_conn { + // match map_read_lock.get(&pool_key.key) { Some(exist_conn_lock) => { debug!("first checkout INNER WRITELOCK"); - let _conn_lock = exist_conn_lock.as_ref().unwrap().lock().await; + let _conn_lock = exist_conn_lock.as_ref().lock().await; debug!( "getting conn for key {:#?} and hash {:#?}", @@ -178,7 +200,7 @@ impl WorkloadHBONEPool { // the number here is simply the number of unique src/dest keys // the pool is expected to track before the inner hashmap resizes. connected_pool: Arc::new(pingora_pool::ConnectionPool::new(500)), - established_conn_writelock: RwLock::new(HashMap::new()), + established_conn_writelock: HashMap::new(), close_pollers: futures::stream::FuturesUnordered::new(), pool_unused_release_timeout: pool_duration, }), @@ -245,19 +267,17 @@ impl WorkloadHBONEPool { // this is is the ONLY block where we should hold a writelock on the whole mutex map // for the rest, a readlock (nonexclusive) is sufficient. { - debug!("pool connect MAP OUTER WRITELOCK START"); - let mut map_write_lock = self.state.established_conn_writelock.write().await; - debug!("pool connect MAP OUTER WRITELOCK END"); - match map_write_lock.get(&hash_key) { - Some(_) => { - debug!("already have conn for key {:#?}", hash_key); - } - None => { + debug!("pool connect MAP OUTER READ/WRITE GUARD"); + let guard = self.state.established_conn_writelock.guard(); + debug!("pool connect MAP OUTER READ/WRITE START"); + match self.state.established_conn_writelock.try_insert(hash_key, Some(Arc::new(Mutex::new(()))), &guard) { + Ok(_) => { debug!("inserting conn mutex for key {:#?}", hash_key); - map_write_lock.insert(hash_key, Some(Mutex::new(()))); } - }; - drop(map_write_lock); // strictly redundant + Err(_) => { + debug!("already have conn for key {:#?}", hash_key); + } + } } // If we get here, it means the following are true: @@ -280,14 +300,25 @@ impl WorkloadHBONEPool { // So the downsides are actually useful (we WANT task contention - // to block other parallel tasks from trying to spawn a connection if we are already doing so) // - // BEGIN take outer readlock - debug!("pool connect MAP OUTER READLOCK START"); - let map_read_lock = self.state.established_conn_writelock.read().await; - debug!("pool connect MAP OUTER READLOCK END"); - let exist_conn_lock = map_read_lock.get(&hash_key).unwrap(); - // BEGIN take inner writelock - debug!("pool connect MAP INNER WRITELOCK START"); - let found_conn = match exist_conn_lock.as_ref().unwrap().try_lock() { + let found_conn = { + // BEGIN take outer readlock + debug!("pool connect MAP OUTER READ/WRITE GUARD"); + let guard = self.state.established_conn_writelock.guard(); + debug!("pool connect MAP OUTER READ/WRITE START"); + + + //OLD + // debug!("pool connect MAP OUTER READLOCK START"); + // let map_read_lock = self.state.established_conn_writelock.read().await; + // debug!("pool connect MAP OUTER READLOCK END"); + let exist_conn_lock = self.state.established_conn_writelock.get(&hash_key, &guard).unwrap(); + // BEGIN take inner writelock + debug!("pool connect MAP INNER WRITELOCK START"); + exist_conn_lock.as_ref().unwrap().clone() + }; + // drop(guard); + + let got_conn = match found_conn.try_lock() { Ok(_guard) => { // If we get here, it means the following are true: // 1. We did not get a connection for our key. @@ -375,7 +406,7 @@ impl WorkloadHBONEPool { // if we checked out a conn inserted by someone else that we cannot use, // everyone else will implicitly be waiting for us to check one back in, // since they are in their own loops. - match found_conn { + match got_conn { // After waiting, we found an available conn we can use, so no need to start another. // Clone the underlying client, return a copy, and put the other back in the pool. Some(f_conn) => { From b461cf3b946148229830f5b72419e2f75717b9cd Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Thu, 18 Apr 2024 16:18:30 -0400 Subject: [PATCH 23/46] Cleanup comments Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 119 +++++++++++++++++++++------------------------- 1 file changed, 53 insertions(+), 66 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 4f8c278e5..2700b0d25 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -32,7 +32,7 @@ use tokio::sync::watch; use tokio::task; use tokio::sync::Mutex; -use tracing::{debug, error}; +use tracing::{debug, error, trace}; use crate::config; use crate::identity::{Identity, SecretManager}; @@ -41,6 +41,8 @@ use flurry::HashMap; use pingora_pool; +// This is merely a counter to track the overall number of conns this pool spawns +// to ensure we get unique poolkeys-per-new-conn, it is not a limit static GLOBAL_CONN_COUNT: AtomicI32 = AtomicI32::new(0); // A relatively nonstandard HTTP/2 connection pool designed to allow multiplexing proxied workload connections @@ -53,8 +55,6 @@ static GLOBAL_CONN_COUNT: AtomicI32 = AtomicI32::new(0); // by flow control throttling. #[derive(Clone)] pub struct WorkloadHBONEPool { - // this is effectively just a convenience data type - a rwlocked hashmap with keying and LRU drops - // and has no actual hyper/http/connection logic. cfg: config::Config, socket_factory: Arc, cert_manager: Arc, @@ -63,6 +63,7 @@ pub struct WorkloadHBONEPool { max_streamcount: u16, } +// PoolState is effectively the gnarly inner state stuff that needs thread/task sync, and should be wrapped in a Mutex. struct PoolState { pool_notifier: watch::Sender, // This is already impl clone? rustc complains that it isn't, tho timeout_tx: watch::Sender, // This is already impl clone? rustc complains that it isn't, tho @@ -70,7 +71,7 @@ struct PoolState { // this is effectively just a convenience data type - a rwlocked hashmap with keying and LRU drops // and has no actual hyper/http/connection logic. connected_pool: Arc>, - // this must be a readlockable list-of-locks, so we can lock per-key, not globally, and avoid holding up all conn attempts + // this must be an atomic/concurrent-safe list-of-locks, so we can lock per-key, not globally, and avoid holding up all conn attempts established_conn_writelock: HashMap>>>, close_pollers: futures::stream::FuturesUnordered>, pool_unused_release_timeout: Duration, @@ -86,7 +87,8 @@ impl PoolState { // // Idle poppers are safe to invoke if the conn they are popping is already gone // from the inner queue, so we will start one for every insert, let them run or terminate on their own, - // and poll them to completion on shutdown. + // and poll them to completion on shutdown - any duplicates from repeated checkouts/checkins of the same conn + // will simply resolve as a no-op in order. // // Note that "idle" in the context of this pool means "no one has asked for it or dropped it in X time, so prune it". // @@ -114,6 +116,19 @@ impl PoolState { let _ = self.pool_notifier.send(true); } + // Does an initial, naive check to see if a conn exists for this key. + // + // If it does, WRITELOCK the mutex for that key, clone (or create), check in the clone, + // and return the other reference for use. + // + // Otherwise, return None. + // + // It is important that the *initial* check here is authoritative, hence the locks, as + // we must know if this is a connection for a key *nobody* has tried to start yet, + // or if other things have already established conns for this key. + // + // This is so we can backpressure correctly if 1000 tasks all demand a new connection + // to the same key at once, and not eagerly open 1000 tunnel connections. async fn first_checkout_conn_from_pool( &self, workload_key: &WorkloadKey, @@ -122,38 +137,30 @@ impl PoolState { debug!("first checkout READGUARD"); let found_conn = { - // BEGIN take outer readlock - debug!("pool connect MAP OUTER READ/WRITE GUARD"); + trace!("pool connect outer map - take guard"); let guard = self.established_conn_writelock.guard(); - debug!("pool connect MAP OUTER READ/WRITE START"); - //OLD - // debug!("pool connect MAP OUTER READLOCK START"); - // let map_read_lock = self.state.established_conn_writelock.read().await; - // debug!("pool connect MAP OUTER READLOCK END"); + trace!("pool connect outer map - check for keyed mutex"); let exist_conn_lock = self.established_conn_writelock.get(&pool_key.key, &guard); - // BEGIN take inner writelock - debug!("pool connect MAP INNER WRITELOCK START"); match exist_conn_lock { Some(e_conn_lock) => e_conn_lock.clone(), None => None, } - // exist_conn_lock.as_ref().unwrap().clone() }; - // let guard = self.established_conn_writelock.guard(); - // debug!("pool connect MAP OUTER READGUARD GOT"); match found_conn { - // match map_read_lock.get(&pool_key.key) { Some(exist_conn_lock) => { - debug!("first checkout INNER WRITELOCK"); + debug!("first checkout - found mutex for key, waiting for writelock"); let _conn_lock = exist_conn_lock.as_ref().lock().await; debug!( - "getting conn for key {:#?} and hash {:#?}", + "first checkout - got writelock for conn with key {:#?} and hash {:#?}", workload_key, pool_key.key ); self.connected_pool.get(&pool_key.key).and_then(|e_conn| { - debug!("got existing conn for key {:#?}", workload_key); + trace!( + "first checkout - inner pool - got existing conn for key {:#?}", + workload_key + ); if e_conn.at_max_streamcount() { debug!( "got conn for key {:#?}, but streamcount is maxed", @@ -171,9 +178,11 @@ impl PoolState { } } +// When the Arc-wrapped PoolState is finally dropped, trigger the drain, +// which will terminate all connection driver spawns, as well as cancel all outstanding eviction timeout spawns impl Drop for PoolState { fn drop(&mut self) { - debug!("poolstate dropping, cancelling all outstanding pool eviction timeout spawns"); + debug!("poolstate dropping, stopping all connection drivers and cancelling all outstanding eviction timeout spawns"); let _ = self.timeout_tx.send(true); } } @@ -221,6 +230,7 @@ impl WorkloadHBONEPool { pub async fn connect(&mut self, workload_key: WorkloadKey) -> Result { debug!("pool connect START"); // TODO BML this may not be collision resistant/slow. It should be resistant enough for workloads tho. + // We are doing a deep-equals check at the end to mitigate any collisions, will see about bumping Pingora let mut s = DefaultHasher::new(); workload_key.hash(&mut s); let hash_key = s.finish(); @@ -228,8 +238,7 @@ impl WorkloadHBONEPool { hash_key, GLOBAL_CONN_COUNT.fetch_add(1, Ordering::Relaxed), ); - - debug!("pool connect GET EXISTING"); + debug!("initial attempt - try to get existing conn from pool"); // First, see if we can naively just check out a connection. // This should be the common case, except for the first establishment of a new connection/key. // This will be done under outer readlock (nonexclusive)/inner keyed writelock (exclusive). @@ -245,37 +254,27 @@ impl WorkloadHBONEPool { .first_checkout_conn_from_pool(&workload_key, &pool_key) .await; - debug!("pool connect GOT EXISTING"); if existing_conn.is_some() { - debug!("using existing conn, connect future will be dropped on the floor"); + debug!("initial attempt - found existing conn, done"); Ok(existing_conn.unwrap()) } else { // We couldn't get a conn. This means either nobody has tried to establish any conns for this key yet, // or they have, but no conns are currently available // (because someone else has checked all of them out and not put any back yet) // - // So, we wil writelock the outer lock to see if an inner lock entry exists for our key. - // - // critical block - this writelocks the entire pool for all tasks/threads - // as we check to see if anyone has ever inserted a sharded mutex for this key. + // So, we will take a nonexclusive readlock on the lockmap, to see if an inner lock + // exists for our key. // // If not, we insert one. - // - // We want to hold this for as short as possible a time, and drop it - // before we hold it over an await. - // - // this is is the ONLY block where we should hold a writelock on the whole mutex map - // for the rest, a readlock (nonexclusive) is sufficient. { - debug!("pool connect MAP OUTER READ/WRITE GUARD"); + debug!("didn't find a connection for key {:#?}, making sure lockmap has entry", hash_key); let guard = self.state.established_conn_writelock.guard(); - debug!("pool connect MAP OUTER READ/WRITE START"); match self.state.established_conn_writelock.try_insert(hash_key, Some(Arc::new(Mutex::new(()))), &guard) { Ok(_) => { - debug!("inserting conn mutex for key {:#?}", hash_key); + debug!("inserting conn mutex for key {:#?} into lockmap", hash_key); } Err(_) => { - debug!("already have conn for key {:#?}", hash_key); + debug!("already have conn for key {:#?} in lockmap", hash_key); } } } @@ -285,7 +284,7 @@ impl WorkloadHBONEPool { // 2. We can now, under readlock(nonexclusive) in the outer map, attempt to // take the inner writelock for our specific key (exclusive). // - // This unblocks other tasks spawning connections against other keys, but blocks other + // This doesn't block other tasks spawning connections against other keys, but DOES block other // tasks spawning connections against THIS key - which is what we want. // NOTE: This inner, key-specific mutex is a tokio::async::Mutex, and not a stdlib sync mutex. @@ -296,37 +295,29 @@ impl WorkloadHBONEPool { // hold it over does not resolve. // // HOWEVER. Here we know this connection will either establish or timeout - // and we WANT other tasks to go back to sleep if there is an outstanding lock. - // So the downsides are actually useful (we WANT task contention - - // to block other parallel tasks from trying to spawn a connection if we are already doing so) + // and we WANT other tasks to go back to sleep if a task is already trying to create a new connection for this key. // - let found_conn = { - // BEGIN take outer readlock - debug!("pool connect MAP OUTER READ/WRITE GUARD"); + // So the downsides are actually useful (we WANT task contention - + // to block other parallel tasks from trying to spawn a connection for this key if we are already doing so) + let inner_conn_lock = { + trace!("fallback attempt - getting keyed lock out of lockmap"); let guard = self.state.established_conn_writelock.guard(); - debug!("pool connect MAP OUTER READ/WRITE START"); - - //OLD - // debug!("pool connect MAP OUTER READLOCK START"); - // let map_read_lock = self.state.established_conn_writelock.read().await; - // debug!("pool connect MAP OUTER READLOCK END"); let exist_conn_lock = self.state.established_conn_writelock.get(&hash_key, &guard).unwrap(); - // BEGIN take inner writelock - debug!("pool connect MAP INNER WRITELOCK START"); + trace!("fallback attempt - got keyed lock out of lockmap"); exist_conn_lock.as_ref().unwrap().clone() }; - // drop(guard); - let got_conn = match found_conn.try_lock() { + debug!("appears we need a new conn, attempting to win connlock for wl key {:#?}", workload_key); + let got_conn = match inner_conn_lock.try_lock() { Ok(_guard) => { + // BEGIN take inner writelock // If we get here, it means the following are true: // 1. We did not get a connection for our key. // 2. We have the exclusive inner writelock to create a new connection for our key. // // So, carry on doing that. - debug!("appears we need a new conn, retaining connlock"); - debug!("nothing else is creating a conn, make one"); + debug!("nothing else is creating a conn and we won the lock, make one"); let pool_conn = self.spawn_new_pool_conn(workload_key.clone()).await; let client = ConnClient{ sender: pool_conn?, @@ -339,12 +330,11 @@ impl WorkloadHBONEPool { "starting new conn for key {:#?} with pk {:#?}", workload_key, pool_key ); - debug!("dropping lock"); Some(client) // END take inner writelock } Err(_) => { - debug!("something else is creating a conn, wait for it"); + debug!("we didnt' win the lock, something else is creating a conn, wait for it"); // If we get here, it means the following are true: // 1. At one point, there was a preexisting conn in the pool for this key. // 2. When we checked, we got nothing for that key. @@ -356,7 +346,7 @@ impl WorkloadHBONEPool { loop { match self.pool_watcher.changed().await { Ok(_) => { - debug!( + trace!( "notified a new conn was enpooled, checking for hash {:#?}", hash_key ); @@ -365,7 +355,7 @@ impl WorkloadHBONEPool { let existing_conn = self.state.connected_pool.get(&hash_key); match existing_conn { None => { - debug!("got nothing"); + trace!("woke up on pool notification, but didn't find a conn for {:#?} yet", hash_key); continue; } Some(e_conn) => { @@ -376,13 +366,11 @@ impl WorkloadHBONEPool { debug!("found existing conn for key {:#?}, but streamcount is maxed", workload_key); break None; } - debug!("pool connect LOOP END"); break Some(e_conn); } } } Err(_) => { - // END take outer readlock return Err(Error::WorkloadHBONEPoolDraining); } } @@ -430,7 +418,6 @@ impl WorkloadHBONEPool { Ok(r_conn) } } - // END take outer readlock }.and_then(|conn| { // Finally, we either have a conn or an error. // Just for safety's sake, since we are using a hash thanks to pingora supporting arbitrary Eq, Hash From d4b4d30884496ddbdf1f2e15c7d982ae02ad44ed Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Thu, 18 Apr 2024 16:52:13 -0400 Subject: [PATCH 24/46] Fix outdent (review comment) Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 324 ++++++++++++++++++++++++---------------------- 1 file changed, 168 insertions(+), 156 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 2700b0d25..0fbe29101 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -254,171 +254,187 @@ impl WorkloadHBONEPool { .first_checkout_conn_from_pool(&workload_key, &pool_key) .await; + // Early return, no need to do anything else if existing_conn.is_some() { debug!("initial attempt - found existing conn, done"); - Ok(existing_conn.unwrap()) - } else { - // We couldn't get a conn. This means either nobody has tried to establish any conns for this key yet, - // or they have, but no conns are currently available - // (because someone else has checked all of them out and not put any back yet) - // - // So, we will take a nonexclusive readlock on the lockmap, to see if an inner lock - // exists for our key. - // - // If not, we insert one. - { - debug!("didn't find a connection for key {:#?}, making sure lockmap has entry", hash_key); - let guard = self.state.established_conn_writelock.guard(); - match self.state.established_conn_writelock.try_insert(hash_key, Some(Arc::new(Mutex::new(()))), &guard) { - Ok(_) => { - debug!("inserting conn mutex for key {:#?} into lockmap", hash_key); - } - Err(_) => { - debug!("already have conn for key {:#?} in lockmap", hash_key); - } + return Ok(existing_conn.unwrap()); + } + + // We couldn't get a conn. This means either nobody has tried to establish any conns for this key yet, + // or they have, but no conns are currently available + // (because someone else has checked all of them out and not put any back yet) + // + // So, we will take a nonexclusive readlock on the lockmap, to see if an inner lock + // exists for our key. + // + // If not, we insert one. + { + debug!( + "didn't find a connection for key {:#?}, making sure lockmap has entry", + hash_key + ); + let guard = self.state.established_conn_writelock.guard(); + match self.state.established_conn_writelock.try_insert( + hash_key, + Some(Arc::new(Mutex::new(()))), + &guard, + ) { + Ok(_) => { + debug!("inserting conn mutex for key {:#?} into lockmap", hash_key); + } + Err(_) => { + debug!("already have conn for key {:#?} in lockmap", hash_key); } } + } - // If we get here, it means the following are true: - // 1. We have a guaranteed sharded mutex in the outer map for our current key. - // 2. We can now, under readlock(nonexclusive) in the outer map, attempt to - // take the inner writelock for our specific key (exclusive). - // - // This doesn't block other tasks spawning connections against other keys, but DOES block other - // tasks spawning connections against THIS key - which is what we want. - - // NOTE: This inner, key-specific mutex is a tokio::async::Mutex, and not a stdlib sync mutex. - // these differ from the stdlib sync mutex in that they are (slightly) slower - // (they effectively sleep the current task) and they can be held over an await. - // The tokio docs (rightly) advise you to not use these, - // because holding a lock over an await is a great way to create deadlocks if the await you - // hold it over does not resolve. - // - // HOWEVER. Here we know this connection will either establish or timeout - // and we WANT other tasks to go back to sleep if a task is already trying to create a new connection for this key. - // - // So the downsides are actually useful (we WANT task contention - - // to block other parallel tasks from trying to spawn a connection for this key if we are already doing so) - let inner_conn_lock = { - trace!("fallback attempt - getting keyed lock out of lockmap"); - let guard = self.state.established_conn_writelock.guard(); - - let exist_conn_lock = self.state.established_conn_writelock.get(&hash_key, &guard).unwrap(); - trace!("fallback attempt - got keyed lock out of lockmap"); - exist_conn_lock.as_ref().unwrap().clone() - }; + // If we get here, it means the following are true: + // 1. We have a guaranteed sharded mutex in the outer map for our current key. + // 2. We can now, under readlock(nonexclusive) in the outer map, attempt to + // take the inner writelock for our specific key (exclusive). + // + // This doesn't block other tasks spawning connections against other keys, but DOES block other + // tasks spawning connections against THIS key - which is what we want. + + // NOTE: This inner, key-specific mutex is a tokio::async::Mutex, and not a stdlib sync mutex. + // these differ from the stdlib sync mutex in that they are (slightly) slower + // (they effectively sleep the current task) and they can be held over an await. + // The tokio docs (rightly) advise you to not use these, + // because holding a lock over an await is a great way to create deadlocks if the await you + // hold it over does not resolve. + // + // HOWEVER. Here we know this connection will either establish or timeout + // and we WANT other tasks to go back to sleep if a task is already trying to create a new connection for this key. + // + // So the downsides are actually useful (we WANT task contention - + // to block other parallel tasks from trying to spawn a connection for this key if we are already doing so) + let inner_conn_lock = { + trace!("fallback attempt - getting keyed lock out of lockmap"); + let guard = self.state.established_conn_writelock.guard(); + + let exist_conn_lock = self + .state + .established_conn_writelock + .get(&hash_key, &guard) + .unwrap(); + trace!("fallback attempt - got keyed lock out of lockmap"); + exist_conn_lock.as_ref().unwrap().clone() + }; - debug!("appears we need a new conn, attempting to win connlock for wl key {:#?}", workload_key); - let got_conn = match inner_conn_lock.try_lock() { - Ok(_guard) => { - // BEGIN take inner writelock - // If we get here, it means the following are true: - // 1. We did not get a connection for our key. - // 2. We have the exclusive inner writelock to create a new connection for our key. - // - // So, carry on doing that. - debug!("nothing else is creating a conn and we won the lock, make one"); - let pool_conn = self.spawn_new_pool_conn(workload_key.clone()).await; - let client = ConnClient{ - sender: pool_conn?, - stream_count: Arc::new(AtomicU16::new(0)), - stream_count_max: self.max_streamcount, - wl_key: workload_key.clone(), - }; - - debug!( - "starting new conn for key {:#?} with pk {:#?}", - workload_key, pool_key - ); - Some(client) - // END take inner writelock - } - Err(_) => { - debug!("we didnt' win the lock, something else is creating a conn, wait for it"); - // If we get here, it means the following are true: - // 1. At one point, there was a preexisting conn in the pool for this key. - // 2. When we checked, we got nothing for that key. - // 3. We could not get the exclusive inner writelock to add a new one for this key. - // 4. Someone else got the exclusive inner writelock, and is adding a new one for this key. - // - // So, loop and wait for the pool_watcher to tell us a new conn was enpooled, - // so we can pull it out and check it. - loop { - match self.pool_watcher.changed().await { - Ok(_) => { - trace!( - "notified a new conn was enpooled, checking for hash {:#?}", - hash_key - ); - // The sharded mutex for this connkey is already locked - someone else must be making a conn - // if they are, try to wait for it, but bail if we find one and it's got a maxed streamcount. - let existing_conn = self.state.connected_pool.get(&hash_key); - match existing_conn { - None => { - trace!("woke up on pool notification, but didn't find a conn for {:#?} yet", hash_key); - continue; - } - Some(e_conn) => { - debug!("found existing conn after waiting"); - // We found a conn, but it's already maxed out. - // Return None and create another. - if e_conn.at_max_streamcount() { - debug!("found existing conn for key {:#?}, but streamcount is maxed", workload_key); - break None; - } - break Some(e_conn); + debug!( + "appears we need a new conn, attempting to win connlock for wl key {:#?}", + workload_key + ); + let got_conn = match inner_conn_lock.try_lock() { + Ok(_guard) => { + // BEGIN take inner writelock + // If we get here, it means the following are true: + // 1. We did not get a connection for our key. + // 2. We have the exclusive inner writelock to create a new connection for our key. + // + // So, carry on doing that. + debug!("nothing else is creating a conn and we won the lock, make one"); + let pool_conn = self.spawn_new_pool_conn(workload_key.clone()).await; + let client = ConnClient { + sender: pool_conn?, + stream_count: Arc::new(AtomicU16::new(0)), + stream_count_max: self.max_streamcount, + wl_key: workload_key.clone(), + }; + + debug!( + "starting new conn for key {:#?} with pk {:#?}", + workload_key, pool_key + ); + Some(client) + // END take inner writelock + } + Err(_) => { + debug!("we didnt' win the lock, something else is creating a conn, wait for it"); + // If we get here, it means the following are true: + // 1. At one point, there was a preexisting conn in the pool for this key. + // 2. When we checked, we got nothing for that key. + // 3. We could not get the exclusive inner writelock to add a new one for this key. + // 4. Someone else got the exclusive inner writelock, and is adding a new one for this key. + // + // So, loop and wait for the pool_watcher to tell us a new conn was enpooled, + // so we can pull it out and check it. + loop { + match self.pool_watcher.changed().await { + Ok(_) => { + trace!( + "notified a new conn was enpooled, checking for hash {:#?}", + hash_key + ); + // The sharded mutex for this connkey is already locked - someone else must be making a conn + // if they are, try to wait for it, but bail if we find one and it's got a maxed streamcount. + let existing_conn = self.state.connected_pool.get(&hash_key); + match existing_conn { + None => { + trace!("woke up on pool notification, but didn't find a conn for {:#?} yet", hash_key); + continue; + } + Some(e_conn) => { + debug!("found existing conn after waiting"); + // We found a conn, but it's already maxed out. + // Return None and create another. + if e_conn.at_max_streamcount() { + debug!("found existing conn for key {:#?}, but streamcount is maxed", workload_key); + break None; } + break Some(e_conn); } } - Err(_) => { - return Err(Error::WorkloadHBONEPoolDraining); - } + } + Err(_) => { + return Err(Error::WorkloadHBONEPoolDraining); } } } - }; + } + }; - // If we get here, it means the following are true: - // 1. At one point, there was a preexisting conn in the pool for this key. - // 2. When we checked, we got nothing for that key. - // 3. We could not get the exclusive inner writelock to add a new one - // 4. Someone else got the exclusive inner writelock, and is adding a new one - // 5. We waited until we got that connection for our key, that someone else added. - // - // So now we are down to 2 options: - // - // 1. We got an existing connection, it's usable, we use it. - // 2. We got an existing connection, it's not usable, we start another. - // - // Note that for (2) we do not really need to take the inner writelock again for this key - - // if we checked out a conn inserted by someone else that we cannot use, - // everyone else will implicitly be waiting for us to check one back in, - // since they are in their own loops. - match got_conn { - // After waiting, we found an available conn we can use, so no need to start another. - // Clone the underlying client, return a copy, and put the other back in the pool. - Some(f_conn) => { - self.state.checkin_conn(f_conn.clone(), pool_key.clone()); - Ok(f_conn) - } + // If we get here, it means the following are true: + // 1. At one point, there was a preexisting conn in the pool for this key. + // 2. When we checked, we got nothing for that key. + // 3. We could not get the exclusive inner writelock to add a new one + // 4. Someone else got the exclusive inner writelock, and is adding a new one + // 5. We waited until we got that connection for our key, that someone else added. + // + // So now we are down to 2 options: + // + // 1. We got an existing connection, it's usable, we use it. + // 2. We got an existing connection, it's not usable, we start another. + // + // Note that for (2) we do not really need to take the inner writelock again for this key - + // if we checked out a conn inserted by someone else that we cannot use, + // everyone else will implicitly be waiting for us to check one back in, + // since they are in their own loops. + match got_conn { + // After waiting, we found an available conn we can use, so no need to start another. + // Clone the underlying client, return a copy, and put the other back in the pool. + Some(f_conn) => { + self.state.checkin_conn(f_conn.clone(), pool_key.clone()); + Ok(f_conn) + } - // After waiting, we found an available conn, but for whatever reason couldn't use it. - // (streamcount maxed, etc) - // Start a new one, clone the underlying client, return a copy, and put the other back in the pool. - None => { - debug!("spawning new conn for key {:#?} to replace", workload_key); - let pool_conn = self.spawn_new_pool_conn(workload_key.clone()).await; - let r_conn = ConnClient{ - sender: pool_conn?, - stream_count: Arc::new(AtomicU16::new(0)), - stream_count_max: self.max_streamcount, - wl_key: workload_key.clone(), - }; - self.state.checkin_conn(r_conn.clone(), pool_key.clone()); - Ok(r_conn) - } + // After waiting, we found an available conn, but for whatever reason couldn't use it. + // (streamcount maxed, etc) + // Start a new one, clone the underlying client, return a copy, and put the other back in the pool. + None => { + debug!("spawning new conn for key {:#?} to replace", workload_key); + let pool_conn = self.spawn_new_pool_conn(workload_key.clone()).await; + let r_conn = ConnClient { + sender: pool_conn?, + stream_count: Arc::new(AtomicU16::new(0)), + stream_count_max: self.max_streamcount, + wl_key: workload_key.clone(), + }; + self.state.checkin_conn(r_conn.clone(), pool_key.clone()); + Ok(r_conn) } - }.and_then(|conn| { + } + .and_then(|conn| { // Finally, we either have a conn or an error. // Just for safety's sake, since we are using a hash thanks to pingora supporting arbitrary Eq, Hash // types, do a deep equality test before returning the conn, returning an error if the conn's key does @@ -426,12 +442,8 @@ impl WorkloadHBONEPool { // // this is a final safety check for collisions, we will throw up our hands and refuse to return the conn match conn.is_for_workload(workload_key) { - Ok(()) => { - Ok(conn) - } - Err(e) => { - Err(e) - } + Ok(()) => Ok(conn), + Err(e) => Err(e), } }) } From fe5ea2c12e86994027e04e22248de35312b408fd Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Fri, 19 Apr 2024 12:54:28 -0400 Subject: [PATCH 25/46] Fixups/review comments Signed-off-by: Benjamin Leggett --- src/config.rs | 16 +++++---- src/proxy/pool.rs | 87 ++++++++++++++++++++++++++++++----------------- 2 files changed, 65 insertions(+), 38 deletions(-) diff --git a/src/config.rs b/src/config.rs index 2dd12dfbf..cfa2e1d27 100644 --- a/src/config.rs +++ b/src/config.rs @@ -65,7 +65,8 @@ const DEFAULT_SELFTERM_DEADLINE: Duration = Duration::from_secs(5); const DEFAULT_CLUSTER_ID: &str = "Kubernetes"; const DEFAULT_CLUSTER_DOMAIN: &str = "cluster.local"; const DEFAULT_TTL: Duration = Duration::from_secs(60 * 60 * 24); // 24 hours -const DEFAULT_POOL_RELEASE: Duration = Duration::from_secs(60 * 5); // 5 minutes +const DEFAULT_POOL_UNUSED_RELEASE_TIMEOUT: Duration = Duration::from_secs(60 * 5); // 5 minutes +const DEFAULT_POOL_MAX_STREAMS_PER_CONNECTION: u16 = 100; //Go: 100, Hyper: 200, Envoy: 2147483647 (lol), Spec recommended minimum 100 const DEFAULT_INPOD_MARK: u32 = 1337; @@ -134,8 +135,8 @@ pub struct Config { // This can be used to effect flow control for "connection storms" when workload clients // (such as loadgen clients) open many connections all at once. // - // Note that this will only be checked and inner conns rebalanced accordingly when a new connection - // is requested from the pool, and not on every stream queue on that connection. + // Note that this will only be checked when a *new* connection + // is requested from the pool, and not on every *stream* queued on that connection. // So if you request a single connection from a pool configured wiht a max streamcount of 200, // and queue 500 streams on it, you will still exceed this limit and are at the mercy of hyper's // default stream queuing. @@ -339,11 +340,14 @@ pub fn construct_config(pc: ProxyConfig) -> Result { .get(DNS_CAPTURE_METADATA) .map_or(false, |value| value.to_lowercase() == "true"), - pool_max_streams_per_conn: parse_default(POOL_MAX_STREAMS_PER_CONNECTION, 250)?, + pool_max_streams_per_conn: parse_default( + POOL_MAX_STREAMS_PER_CONNECTION, + DEFAULT_POOL_MAX_STREAMS_PER_CONNECTION, + )?, pool_unused_release_timeout: match parse::(POOL_UNUSED_RELEASE_TIMEOUT)? { - Some(ttl) => duration_str::parse(ttl).unwrap_or(DEFAULT_POOL_RELEASE), - None => DEFAULT_POOL_RELEASE, + Some(ttl) => duration_str::parse(ttl).unwrap_or(DEFAULT_POOL_UNUSED_RELEASE_TIMEOUT), + None => DEFAULT_POOL_UNUSED_RELEASE_TIMEOUT, }, window_size: 4 * 1024 * 1024, diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 0fbe29101..23ed09b94 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -41,10 +41,6 @@ use flurry::HashMap; use pingora_pool; -// This is merely a counter to track the overall number of conns this pool spawns -// to ensure we get unique poolkeys-per-new-conn, it is not a limit -static GLOBAL_CONN_COUNT: AtomicI32 = AtomicI32::new(0); - // A relatively nonstandard HTTP/2 connection pool designed to allow multiplexing proxied workload connections // over a (smaller) number of HTTP/2 mTLS tunnels. // @@ -75,6 +71,9 @@ struct PoolState { established_conn_writelock: HashMap>>>, close_pollers: futures::stream::FuturesUnordered>, pool_unused_release_timeout: Duration, + // This is merely a counter to track the overall number of conns this pool spawns + // to ensure we get unique poolkeys-per-new-conn, it is not a limit + pool_global_conn_count: AtomicI32, } impl PoolState { @@ -116,6 +115,37 @@ impl PoolState { let _ = self.pool_notifier.send(true); } + // Since we are using a hash key to do lookup on the inner pingora pool, do a get guard + // to make sure what we pull out actually deep-equals the workload_key, to avoid *sigh* crossing the streams. + fn guarded_get( + &self, + hash_key: &u64, + workload_key: &WorkloadKey, + ) -> Result, Error> { + match self.connected_pool.get(hash_key) { + None => Ok(None), + Some(conn) => match Self::enforce_key_integrity(conn, workload_key) { + Err(e) => Err(e), + Ok(conn) => Ok(Some(conn)), + }, + } + } + + // Just for safety's sake, since we are using a hash thanks to pingora supporting arbitrary Eq, Hash + // types, do a deep equality test before returning the conn, returning an error if the conn's key does + // not equal the provided key + // + // this is a final safety check for collisions, we will throw up our hands and refuse to return the conn + fn enforce_key_integrity( + conn: ConnClient, + expected_key: &WorkloadKey, + ) -> Result { + match conn.is_for_workload(expected_key) { + Ok(()) => Ok(conn), + Err(e) => Err(e), + } + } + // Does an initial, naive check to see if a conn exists for this key. // // If it does, WRITELOCK the mutex for that key, clone (or create), check in the clone, @@ -133,7 +163,7 @@ impl PoolState { &self, workload_key: &WorkloadKey, pool_key: &pingora_pool::ConnectionMeta, - ) -> Option { + ) -> Result, Error> { debug!("first checkout READGUARD"); let found_conn = { @@ -142,10 +172,7 @@ impl PoolState { trace!("pool connect outer map - check for keyed mutex"); let exist_conn_lock = self.established_conn_writelock.get(&pool_key.key, &guard); - match exist_conn_lock { - Some(e_conn_lock) => e_conn_lock.clone(), - None => None, - } + exist_conn_lock.and_then(|e_conn_lock| e_conn_lock.clone()) }; match found_conn { Some(exist_conn_lock) => { @@ -156,7 +183,8 @@ impl PoolState { "first checkout - got writelock for conn with key {:#?} and hash {:#?}", workload_key, pool_key.key ); - self.connected_pool.get(&pool_key.key).and_then(|e_conn| { + let got = self.guarded_get(&pool_key.key, workload_key)?; + Ok(got.and_then(|e_conn| { trace!( "first checkout - inner pool - got existing conn for key {:#?}", workload_key @@ -171,9 +199,9 @@ impl PoolState { self.checkin_conn(e_conn.clone(), pool_key.clone()); Some(e_conn) } - }) + })) } - None => None, + None => Ok(None), } } } @@ -200,7 +228,10 @@ impl WorkloadHBONEPool { let (timeout_send, timeout_recv) = watch::channel(false); let max_count = cfg.pool_max_streams_per_conn; let pool_duration = cfg.pool_unused_release_timeout; + // This is merely a counter to track the overall number of conns this pool spawns + // to ensure we get unique poolkeys-per-new-conn, it is not a limit debug!("constructing pool with {:#?} streams per conn", max_count); + Self { state: Arc::new(PoolState { pool_notifier: timeout_tx, @@ -212,6 +243,7 @@ impl WorkloadHBONEPool { established_conn_writelock: HashMap::new(), close_pollers: futures::stream::FuturesUnordered::new(), pool_unused_release_timeout: pool_duration, + pool_global_conn_count: AtomicI32::new(0), }), cfg, socket_factory, @@ -236,7 +268,9 @@ impl WorkloadHBONEPool { let hash_key = s.finish(); let pool_key = pingora_pool::ConnectionMeta::new( hash_key, - GLOBAL_CONN_COUNT.fetch_add(1, Ordering::Relaxed), + self.state + .pool_global_conn_count + .fetch_add(1, Ordering::Relaxed), ); debug!("initial attempt - try to get existing conn from pool"); // First, see if we can naively just check out a connection. @@ -252,7 +286,7 @@ impl WorkloadHBONEPool { let existing_conn = self .state .first_checkout_conn_from_pool(&workload_key, &pool_key) - .await; + .await?; // Early return, no need to do anything else if existing_conn.is_some() { @@ -350,7 +384,7 @@ impl WorkloadHBONEPool { // END take inner writelock } Err(_) => { - debug!("we didnt' win the lock, something else is creating a conn, wait for it"); + debug!("we didn't win the lock, something else is creating a conn, wait for it"); // If we get here, it means the following are true: // 1. At one point, there was a preexisting conn in the pool for this key. // 2. When we checked, we got nothing for that key. @@ -366,9 +400,10 @@ impl WorkloadHBONEPool { "notified a new conn was enpooled, checking for hash {:#?}", hash_key ); - // The sharded mutex for this connkey is already locked - someone else must be making a conn - // if they are, try to wait for it, but bail if we find one and it's got a maxed streamcount. - let existing_conn = self.state.connected_pool.get(&hash_key); + + // Notifier fired, try and get a conn out for our key. + // If we do, make sure it's not maxed out on streams. + let existing_conn = self.state.guarded_get(&hash_key, &workload_key)?; match existing_conn { None => { trace!("woke up on pool notification, but didn't find a conn for {:#?} yet", hash_key); @@ -434,18 +469,6 @@ impl WorkloadHBONEPool { Ok(r_conn) } } - .and_then(|conn| { - // Finally, we either have a conn or an error. - // Just for safety's sake, since we are using a hash thanks to pingora supporting arbitrary Eq, Hash - // types, do a deep equality test before returning the conn, returning an error if the conn's key does - // not equal the provided key - // - // this is a final safety check for collisions, we will throw up our hands and refuse to return the conn - match conn.is_for_workload(workload_key) { - Ok(()) => Ok(conn), - Err(e) => Err(e), - } - }) } async fn spawn_new_pool_conn( @@ -532,8 +555,8 @@ impl ConnClient { self.sender.send_request(req) } - pub fn is_for_workload(&self, wl_key: WorkloadKey) -> Result<(), crate::proxy::Error> { - if !(self.wl_key == wl_key) { + pub fn is_for_workload(&self, wl_key: &WorkloadKey) -> Result<(), crate::proxy::Error> { + if !(self.wl_key == *wl_key) { Err(crate::proxy::Error::Generic( "fetched connection does not match workload key!".into(), )) From 6ff22cc147ecf51b18f3cebf443289c75b443994 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Mon, 22 Apr 2024 12:20:13 -0400 Subject: [PATCH 26/46] resync Signed-off-by: Benjamin Leggett --- Cargo.lock | 2 +- src/config.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 138accc7e..bfc0882f9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -498,7 +498,7 @@ version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" dependencies = [ - "getrandom 0.2.12", + "getrandom 0.2.14", "once_cell", "tiny-keccak", ] diff --git a/src/config.rs b/src/config.rs index cfa2e1d27..216739743 100644 --- a/src/config.rs +++ b/src/config.rs @@ -371,7 +371,7 @@ pub fn construct_config(pc: ProxyConfig) -> Result { DEFAULT_READINESS_PORT, // There is no config for this in ProxyConfig currently ), - socks5_addr: SocketAddr::new(IpAddr::V4(Ipv4Addr::LOCALHOST), 15080), + socks5_addr, inbound_addr: SocketAddr::new(IpAddr::V6(Ipv6Addr::UNSPECIFIED), 15008), inbound_plaintext_addr: SocketAddr::new(IpAddr::V6(Ipv6Addr::UNSPECIFIED), 15006), outbound_addr: SocketAddr::new(IpAddr::V6(Ipv6Addr::UNSPECIFIED), 15001), From 2025a0f5ee73f308eb06b2aafb7c45c780494d26 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Mon, 22 Apr 2024 13:50:15 -0400 Subject: [PATCH 27/46] Droptests Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 217 +++++++++++++++++++++++++++++++++++++++++----- 1 file changed, 195 insertions(+), 22 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 23ed09b94..5ad977b26 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -594,6 +594,7 @@ mod test { use hyper::{Request, Response}; use std::sync::atomic::AtomicU32; use std::time::Duration; + use tokio::time::sleep; use tokio::io::AsyncWriteExt; use tokio::net::TcpListener; use tokio::task::{self}; @@ -611,7 +612,9 @@ mod test { let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain).await; + let conn_counter: Arc = Arc::new(AtomicU32::new(0)); + let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); + let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -639,7 +642,8 @@ mod test { server_drain_signal.drain().await; drop(pool); - let real_conncount = server_handle.await.unwrap(); + server_handle.await.unwrap(); + let real_conncount = conn_counter.load(Ordering::Relaxed); assert!(real_conncount == 1, "actual conncount was {real_conncount}"); assert!(client1.is_ok()); @@ -650,7 +654,10 @@ mod test { #[tokio::test] async fn test_pool_does_not_reuse_conn_for_diff_key() { let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain).await; + + let conn_counter: Arc = Arc::new(AtomicU32::new(0)); + let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); + let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; // crate::telemetry::setup_logging(); @@ -681,7 +688,10 @@ mod test { server_drain_signal.drain().await; drop(pool); - let real_conncount = server_handle.await.unwrap(); + + server_handle.await.unwrap(); + + let real_conncount = conn_counter.load(Ordering::Relaxed); assert!(real_conncount == 2, "actual conncount was {real_conncount}"); assert!(client1.is_ok()); @@ -691,7 +701,10 @@ mod test { #[tokio::test] async fn test_pool_respects_per_conn_stream_limit() { let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain).await; + + let conn_counter: Arc = Arc::new(AtomicU32::new(0)); + let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); + let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -714,7 +727,9 @@ mod test { server_drain_signal.drain().await; drop(pool); - let real_conncount = server_handle.await.unwrap(); + server_handle.await.unwrap(); + + let real_conncount = conn_counter.load(Ordering::Relaxed); assert!(real_conncount == 2, "actual conncount was {real_conncount}"); assert!(client1.is_ok()); @@ -724,7 +739,10 @@ mod test { #[tokio::test] async fn test_pool_handles_many_conns_per_key() { let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain).await; + + let conn_counter: Arc = Arc::new(AtomicU32::new(0)); + let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); + let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -748,7 +766,9 @@ mod test { drop(pool); server_drain_signal.drain().await; - let real_conncount = server_handle.await.unwrap(); + server_handle.await.unwrap(); + + let real_conncount = conn_counter.load(Ordering::Relaxed); assert!(real_conncount == 2, "actual conncount was {real_conncount}"); assert!(client1.is_ok()); @@ -760,7 +780,10 @@ mod test { // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain).await; + + let conn_counter: Arc = Arc::new(AtomicU32::new(0)); + let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); + let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -795,7 +818,8 @@ mod test { drop(pool); server_drain_signal.drain().await; - let real_conncount = server_handle.await.unwrap(); + + let real_conncount = conn_counter.load(Ordering::Relaxed); assert!(real_conncount == 3, "actual conncount was {real_conncount}"); } @@ -804,7 +828,10 @@ mod test { // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain).await; + + let conn_counter: Arc = Arc::new(AtomicU32::new(0)); + let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); + let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -840,7 +867,9 @@ mod test { drop(pool); server_drain_signal.drain().await; - let real_conncount = server_handle.await.unwrap(); + server_handle.await.unwrap(); + + let real_conncount = conn_counter.load(Ordering::Relaxed); assert!(real_conncount == 1, "actual conncount was {real_conncount}"); } @@ -849,7 +878,10 @@ mod test { // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain).await; + + let conn_counter: Arc = Arc::new(AtomicU32::new(0)); + let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); + let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -889,7 +921,9 @@ mod test { drop(pool); server_drain_signal.drain().await; - let real_conncount = server_handle.await.unwrap(); + server_handle.await.unwrap(); + + let real_conncount = conn_counter.load(Ordering::Relaxed); assert!( real_conncount == 100, "actual conncount was {real_conncount}" @@ -901,7 +935,10 @@ mod test { // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); - let (server_addr, server_handle) = spawn_server(server_drain).await; + + let conn_counter: Arc = Arc::new(AtomicU32::new(0)); + let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); + let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -948,9 +985,147 @@ mod test { drop(pool); + server_handle.await.unwrap(); + + let real_conncount = conn_counter.load(Ordering::Relaxed); + assert!(real_conncount == 3, "actual conncount was {real_conncount}"); + } + + #[tokio::test] + async fn test_pool_1000_clients_3_srcs_drops_after_timeout() { + // crate::telemetry::setup_logging(); + + let (server_drain_signal, server_drain) = drain::channel(); + + let conn_counter: Arc = Arc::new(AtomicU32::new(0)); + let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); + let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; + + let cfg = crate::config::Config { + local_node: Some("local-node".to_string()), + pool_max_streams_per_conn: 1000, + pool_unused_release_timeout: Duration::from_secs(5), + ..crate::config::parse_config().unwrap() + }; + let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); + let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr); + + let mut key1 = WorkloadKey { + src_id: Identity::default(), + dst_id: vec![Identity::default()], + src: IpAddr::from([127, 0, 0, 1]), + dst: server_addr, + }; + + let client_count = 100; + let mut count = 0u32; + let mut tasks = futures::stream::FuturesUnordered::new(); + loop { + count += 1; + if count % 2 == 0 { + debug!("using key 2"); + key1.src = IpAddr::from([127, 0, 0, 4]); + } else if count % 3 == 0 { + debug!("using key 3"); + key1.src = IpAddr::from([127, 0, 0, 6]); + } else { + debug!("using key 1"); + key1.src = IpAddr::from([127, 0, 0, 2]); + } + + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 100)); + + if count == client_count { + break; + } + } + while let Some(Err(res)) = tasks.next().await { + assert!(!res.is_panic(), "CLIENT PANICKED!"); + continue; + } + + let before_conncount = conn_counter.load(Ordering::Relaxed); + let before_dropcount = conn_drop_counter.load(Ordering::Relaxed); + assert!(before_conncount == 3, "actual before conncount was {before_conncount}"); + assert!(before_dropcount == 0, "actual before dropcount was {before_dropcount}"); + + drop(pool); + // Attempt to wait long enough for pool conns to timeout+drop + sleep(Duration::from_secs(6)).await; + + let real_conncount = conn_counter.load(Ordering::Relaxed); + let real_dropcount = conn_drop_counter.load(Ordering::Relaxed); + assert!(real_conncount == 3, "actual conncount was {real_conncount}"); + assert!(real_dropcount == 3, "actual dropcount was {real_dropcount}"); + server_drain_signal.drain().await; - let real_conncount = server_handle.await.unwrap(); + server_handle.await.unwrap(); + } + + #[tokio::test] + async fn test_pool_1000_clients_3_srcs_does_not_drop_before_timeout() { + // crate::telemetry::setup_logging(); + + let (server_drain_signal, server_drain) = drain::channel(); + let conn_counter: Arc = Arc::new(AtomicU32::new(0)); + let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); + let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; + + let cfg = crate::config::Config { + local_node: Some("local-node".to_string()), + pool_max_streams_per_conn: 1000, + pool_unused_release_timeout: Duration::from_secs(5), + ..crate::config::parse_config().unwrap() + }; + let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); + let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr); + + let mut key1 = WorkloadKey { + src_id: Identity::default(), + dst_id: vec![Identity::default()], + src: IpAddr::from([127, 0, 0, 1]), + dst: server_addr, + }; + + let client_count = 100; + let mut count = 0u32; + let mut tasks = futures::stream::FuturesUnordered::new(); + loop { + count += 1; + if count % 2 == 0 { + debug!("using key 2"); + key1.src = IpAddr::from([127, 0, 0, 4]); + } else if count % 3 == 0 { + debug!("using key 3"); + key1.src = IpAddr::from([127, 0, 0, 6]); + } else { + debug!("using key 1"); + key1.src = IpAddr::from([127, 0, 0, 2]); + } + + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 100)); + + if count == client_count { + break; + } + } + while let Some(Err(res)) = tasks.next().await { + assert!(!res.is_panic(), "CLIENT PANICKED!"); + continue; + } + + drop(pool); + // Attempt to wait long enough for pool conns to timeout+drop + sleep(Duration::from_secs(6)).await; + + server_drain_signal.drain().await; + server_handle.await.unwrap(); + let real_conncount = conn_counter.load(Ordering::Relaxed); + let real_dropcount = conn_drop_counter.load(Ordering::Relaxed); assert!(real_conncount == 3, "actual conncount was {real_conncount}"); + assert!(real_dropcount == 3, "actual dropcount was {real_dropcount}"); } fn spawn_client( @@ -1001,7 +1176,7 @@ mod test { }) } - async fn spawn_server(stop: Watch) -> (SocketAddr, task::JoinHandle) { + async fn spawn_server(stop: Watch, conn_count: Arc, conn_drop_count: Arc) -> (SocketAddr, task::JoinHandle<()>) { // We'll bind to 127.0.0.1:3000 let addr = SocketAddr::from(([127, 0, 0, 1], 0)); let test_cfg = test_config(); @@ -1022,9 +1197,6 @@ mod test { Ok::<_, Infallible>(Response::new(http_body_util::Empty::::new())) } - let conn_count: Arc = Arc::new(AtomicU32::new(0)); - // let _drop_conn_count: Arc = Arc::new(AtomicU32::new(0)); - // We create a TcpListener and bind it to 127.0.0.1:3000 let listener = TcpListener::bind(addr).await.unwrap(); let bound_addr = listener.local_addr().unwrap(); @@ -1041,10 +1213,12 @@ mod test { // We start a loop to continuously accept incoming connections // and also count them let movable_count = conn_count.clone(); + let movable_drop_count = conn_drop_count.clone(); let accept = async move { loop { let stream = tls_stream.next().await.unwrap(); movable_count.fetch_add(1, Ordering::Relaxed); + let dcount = movable_drop_count.clone(); debug!("bump serverconn"); // Spawn a tokio task to serve multiple connections concurrently @@ -1065,6 +1239,7 @@ mod test { { println!("Error serving connection: {:?}", err); } + dcount.fetch_add(1, Ordering::Relaxed); }); } }; @@ -1074,8 +1249,6 @@ mod test { debug!("GOT STOP SERVER"); } }; - - conn_count.load(Ordering::Relaxed) }); (bound_addr, srv_handle) From 4a0dc47179995b76c5052213197fda6b4c9cfedd Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Mon, 22 Apr 2024 13:57:36 -0400 Subject: [PATCH 28/46] fix testhang Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 200 ++++++++++++++++++++++++++++------------------ 1 file changed, 122 insertions(+), 78 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 5ad977b26..45c05363a 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -594,10 +594,10 @@ mod test { use hyper::{Request, Response}; use std::sync::atomic::AtomicU32; use std::time::Duration; - use tokio::time::sleep; use tokio::io::AsyncWriteExt; use tokio::net::TcpListener; use tokio::task::{self}; + use tokio::time::sleep; #[cfg(tokio_unstable)] use tracing::Instrument; @@ -614,7 +614,12 @@ mod test { let conn_counter: Arc = Arc::new(AtomicU32::new(0)); let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); - let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; + let (server_addr, server_handle) = spawn_server( + server_drain, + conn_counter.clone(), + conn_drop_counter.clone(), + ) + .await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -657,7 +662,12 @@ mod test { let conn_counter: Arc = Arc::new(AtomicU32::new(0)); let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); - let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; + let (server_addr, server_handle) = spawn_server( + server_drain, + conn_counter.clone(), + conn_drop_counter.clone(), + ) + .await; // crate::telemetry::setup_logging(); @@ -704,7 +714,12 @@ mod test { let conn_counter: Arc = Arc::new(AtomicU32::new(0)); let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); - let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; + let (server_addr, server_handle) = spawn_server( + server_drain, + conn_counter.clone(), + conn_drop_counter.clone(), + ) + .await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -742,7 +757,12 @@ mod test { let conn_counter: Arc = Arc::new(AtomicU32::new(0)); let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); - let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; + let (server_addr, server_handle) = spawn_server( + server_drain, + conn_counter.clone(), + conn_drop_counter.clone(), + ) + .await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -783,7 +803,12 @@ mod test { let conn_counter: Arc = Arc::new(AtomicU32::new(0)); let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); - let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; + let (server_addr, server_handle) = spawn_server( + server_drain, + conn_counter.clone(), + conn_drop_counter.clone(), + ) + .await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -831,7 +856,12 @@ mod test { let conn_counter: Arc = Arc::new(AtomicU32::new(0)); let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); - let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; + let (server_addr, server_handle) = spawn_server( + server_drain, + conn_counter.clone(), + conn_drop_counter.clone(), + ) + .await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -881,7 +911,12 @@ mod test { let conn_counter: Arc = Arc::new(AtomicU32::new(0)); let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); - let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; + let (server_addr, server_handle) = spawn_server( + server_drain, + conn_counter.clone(), + conn_drop_counter.clone(), + ) + .await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -938,7 +973,12 @@ mod test { let conn_counter: Arc = Arc::new(AtomicU32::new(0)); let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); - let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; + let (server_addr, server_handle) = spawn_server( + server_drain, + conn_counter.clone(), + conn_drop_counter.clone(), + ) + .await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -985,6 +1025,7 @@ mod test { drop(pool); + server_drain_signal.drain().await; server_handle.await.unwrap(); let real_conncount = conn_counter.load(Ordering::Relaxed); @@ -999,7 +1040,12 @@ mod test { let conn_counter: Arc = Arc::new(AtomicU32::new(0)); let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); - let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; + let (server_addr, server_handle) = spawn_server( + server_drain, + conn_counter.clone(), + conn_drop_counter.clone(), + ) + .await; let cfg = crate::config::Config { local_node: Some("local-node".to_string()), @@ -1047,8 +1093,14 @@ mod test { let before_conncount = conn_counter.load(Ordering::Relaxed); let before_dropcount = conn_drop_counter.load(Ordering::Relaxed); - assert!(before_conncount == 3, "actual before conncount was {before_conncount}"); - assert!(before_dropcount == 0, "actual before dropcount was {before_dropcount}"); + assert!( + before_conncount == 3, + "actual before conncount was {before_conncount}" + ); + assert!( + before_dropcount == 0, + "actual before dropcount was {before_dropcount}" + ); drop(pool); // Attempt to wait long enough for pool conns to timeout+drop @@ -1063,71 +1115,6 @@ mod test { server_handle.await.unwrap(); } - #[tokio::test] - async fn test_pool_1000_clients_3_srcs_does_not_drop_before_timeout() { - // crate::telemetry::setup_logging(); - - let (server_drain_signal, server_drain) = drain::channel(); - let conn_counter: Arc = Arc::new(AtomicU32::new(0)); - let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); - let (server_addr, server_handle) = spawn_server(server_drain, conn_counter.clone(), conn_drop_counter.clone()).await; - - let cfg = crate::config::Config { - local_node: Some("local-node".to_string()), - pool_max_streams_per_conn: 1000, - pool_unused_release_timeout: Duration::from_secs(5), - ..crate::config::parse_config().unwrap() - }; - let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); - let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); - let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr); - - let mut key1 = WorkloadKey { - src_id: Identity::default(), - dst_id: vec![Identity::default()], - src: IpAddr::from([127, 0, 0, 1]), - dst: server_addr, - }; - - let client_count = 100; - let mut count = 0u32; - let mut tasks = futures::stream::FuturesUnordered::new(); - loop { - count += 1; - if count % 2 == 0 { - debug!("using key 2"); - key1.src = IpAddr::from([127, 0, 0, 4]); - } else if count % 3 == 0 { - debug!("using key 3"); - key1.src = IpAddr::from([127, 0, 0, 6]); - } else { - debug!("using key 1"); - key1.src = IpAddr::from([127, 0, 0, 2]); - } - - tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 100)); - - if count == client_count { - break; - } - } - while let Some(Err(res)) = tasks.next().await { - assert!(!res.is_panic(), "CLIENT PANICKED!"); - continue; - } - - drop(pool); - // Attempt to wait long enough for pool conns to timeout+drop - sleep(Duration::from_secs(6)).await; - - server_drain_signal.drain().await; - server_handle.await.unwrap(); - let real_conncount = conn_counter.load(Ordering::Relaxed); - let real_dropcount = conn_drop_counter.load(Ordering::Relaxed); - assert!(real_conncount == 3, "actual conncount was {real_conncount}"); - assert!(real_dropcount == 3, "actual dropcount was {real_dropcount}"); - } - fn spawn_client( mut pool: WorkloadHBONEPool, key: WorkloadKey, @@ -1176,7 +1163,64 @@ mod test { }) } - async fn spawn_server(stop: Watch, conn_count: Arc, conn_drop_count: Arc) -> (SocketAddr, task::JoinHandle<()>) { + fn spawn_persistent_client( + mut pool: WorkloadHBONEPool, + key: WorkloadKey, + remote_addr: SocketAddr, + stop: Watch, + ) -> task::JoinHandle<()> { + tokio::spawn(async move { + let req = || { + hyper::Request::builder() + .uri(format!("{remote_addr}")) + .method(hyper::Method::CONNECT) + .version(hyper::Version::HTTP_2) + .body(Empty::::new()) + .unwrap() + }; + + let start = Instant::now(); + + let mut c1 = pool + .connect(key.clone()) + // needs tokio_unstable, but useful + // .instrument(tracing::debug_span!("client_tid", tid=%tokio::task::id())) + .await + .unwrap(); + debug!( + "client spent {}ms waiting for conn", + start.elapsed().as_millis() + ); + + let mut count = 0u32; + // send forever, once we get a conn, until someone signals us to stop + let send_loop = async move { + loop { + count += 1; + let res = c1.send_request(req()).await; + + if res.is_err() { + panic!("SEND ERR: {:#?} sendcount {count}", res); + } else if res.unwrap().status() != 200 { + panic!("CLIENT RETURNED ERROR") + } + } + }; + + tokio::select! { + _ = send_loop => {} + _ = stop.signaled() => { + debug!("GOT STOP PERSISTENT CLIENT"); + } + }; + }) + } + + async fn spawn_server( + stop: Watch, + conn_count: Arc, + conn_drop_count: Arc, + ) -> (SocketAddr, task::JoinHandle<()>) { // We'll bind to 127.0.0.1:3000 let addr = SocketAddr::from(([127, 0, 0, 1], 0)); let test_cfg = test_config(); From 852100c4da557f957d5184980122f76cfa7ef3ab Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Mon, 22 Apr 2024 14:13:02 -0400 Subject: [PATCH 29/46] add smarter evict test Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 86 ++++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 85 insertions(+), 1 deletion(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 45c05363a..2cc9ab4db 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -1102,7 +1102,6 @@ mod test { "actual before dropcount was {before_dropcount}" ); - drop(pool); // Attempt to wait long enough for pool conns to timeout+drop sleep(Duration::from_secs(6)).await; @@ -1113,6 +1112,91 @@ mod test { server_drain_signal.drain().await; server_handle.await.unwrap(); + drop(pool); + } + + #[tokio::test] + async fn test_pool_100_clients_evicts_but_does_not_close_active_conn() { + // crate::telemetry::setup_logging(); + + let (server_drain_signal, server_drain) = drain::channel(); + + let conn_counter: Arc = Arc::new(AtomicU32::new(0)); + let conn_drop_counter: Arc = Arc::new(AtomicU32::new(0)); + let (server_addr, server_handle) = spawn_server( + server_drain, + conn_counter.clone(), + conn_drop_counter.clone(), + ) + .await; + + let cfg = crate::config::Config { + local_node: Some("local-node".to_string()), + pool_max_streams_per_conn: 50, + ..crate::config::parse_config().unwrap() + }; + let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); + let cert_mgr = identity::mock::new_secret_manager(Duration::from_secs(10)); + let pool = WorkloadHBONEPool::new(cfg.clone(), sock_fact, cert_mgr); + + let key1 = WorkloadKey { + src_id: Identity::default(), + dst_id: vec![Identity::default()], + src: IpAddr::from([127, 0, 0, 2]), + dst: server_addr, + }; + let client_count = 100; + let mut count = 0u32; + let mut tasks = futures::stream::FuturesUnordered::new(); + loop { + count += 1; + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 100)); + + if count == client_count { + break; + } + } + + let (client_stop_signal, client_stop) = drain::channel(); + let persist_res = spawn_persistent_client(pool.clone(), key1.clone(), server_addr, client_stop); + + //loop thru the nonpersistent results + while let Some(Err(res)) = tasks.next().await { + assert!(!res.is_panic(), "CLIENT PANICKED!"); + continue; + } + + let before_conncount = conn_counter.load(Ordering::Relaxed); + let before_dropcount = conn_drop_counter.load(Ordering::Relaxed); + assert!( + before_conncount == 3, + "actual before conncount was {before_conncount}" + ); + assert!( + before_dropcount == 0, + "actual before dropcount was {before_dropcount}" + ); + + + // Attempt to wait long enough for pool conns to timeout+drop + sleep(Duration::from_secs(6)).await; + + let real_conncount = conn_counter.load(Ordering::Relaxed); + assert!(real_conncount == 3, "actual conncount was {real_conncount}"); + // At this point, we should still have one conn that hasn't been dropped (even though the pool has) + // because we haven't ended the + let real_dropcount = conn_drop_counter.load(Ordering::Relaxed); + assert!( + real_dropcount == 2, + "actual dropcount was {real_dropcount}" + ); + server_drain_signal.drain().await; + client_stop_signal.drain().await; + + assert!(!persist_res.await.is_err(), "PERSIST CLIENT ERROR"); + + drop(pool); + } fn spawn_client( From 45f23ac8e0c5acb9867fa5d5ce0f0d5ad01df519 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Mon, 22 Apr 2024 14:22:37 -0400 Subject: [PATCH 30/46] Interesting failure Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 2cc9ab4db..f800e2bdd 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -843,6 +843,7 @@ mod test { drop(pool); server_drain_signal.drain().await; + server_handle.await.unwrap(); let real_conncount = conn_counter.load(Ordering::Relaxed); assert!(real_conncount == 3, "actual conncount was {real_conncount}"); @@ -1184,17 +1185,27 @@ mod test { let real_conncount = conn_counter.load(Ordering::Relaxed); assert!(real_conncount == 3, "actual conncount was {real_conncount}"); // At this point, we should still have one conn that hasn't been dropped (even though the pool has) - // because we haven't ended the + // because we haven't ended the persistent client let real_dropcount = conn_drop_counter.load(Ordering::Relaxed); assert!( real_dropcount == 2, "actual dropcount was {real_dropcount}" ); - server_drain_signal.drain().await; client_stop_signal.drain().await; - assert!(!persist_res.await.is_err(), "PERSIST CLIENT ERROR"); + sleep(Duration::from_secs(6)).await; + + let after_conncount = conn_counter.load(Ordering::Relaxed); + assert!(after_conncount == 3, "after conncount was {after_conncount}"); + let after_dropcount = conn_drop_counter.load(Ordering::Relaxed); + assert!( + after_dropcount == 3, + "after dropcount was {after_dropcount}" + ); + server_drain_signal.drain().await; + server_handle.await.unwrap(); + drop(pool); } From 9ec19b26e66321ca6422e0318847c28e9cff3ddc Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Mon, 22 Apr 2024 14:58:53 -0400 Subject: [PATCH 31/46] No, it's not Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index f800e2bdd..736ce5552 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -1051,7 +1051,7 @@ mod test { let cfg = crate::config::Config { local_node: Some("local-node".to_string()), pool_max_streams_per_conn: 1000, - pool_unused_release_timeout: Duration::from_secs(5), + pool_unused_release_timeout: Duration::from_secs(2), ..crate::config::parse_config().unwrap() }; let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); @@ -1104,7 +1104,7 @@ mod test { ); // Attempt to wait long enough for pool conns to timeout+drop - sleep(Duration::from_secs(6)).await; + sleep(Duration::from_secs(2)).await; let real_conncount = conn_counter.load(Ordering::Relaxed); let real_dropcount = conn_drop_counter.load(Ordering::Relaxed); @@ -1134,6 +1134,7 @@ mod test { let cfg = crate::config::Config { local_node: Some("local-node".to_string()), pool_max_streams_per_conn: 50, + pool_unused_release_timeout: Duration::from_secs(2), ..crate::config::parse_config().unwrap() }; let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); @@ -1161,7 +1162,7 @@ mod test { let (client_stop_signal, client_stop) = drain::channel(); let persist_res = spawn_persistent_client(pool.clone(), key1.clone(), server_addr, client_stop); - //loop thru the nonpersistent results + //loop thru the nonpersistent clients and wait for them to finish while let Some(Err(res)) = tasks.next().await { assert!(!res.is_panic(), "CLIENT PANICKED!"); continue; @@ -1179,12 +1180,12 @@ mod test { ); - // Attempt to wait long enough for pool conns to timeout+drop - sleep(Duration::from_secs(6)).await; + // Attempt to wait long enough for pool conns to timeout+evict + sleep(Duration::from_secs(2)).await; let real_conncount = conn_counter.load(Ordering::Relaxed); assert!(real_conncount == 3, "actual conncount was {real_conncount}"); - // At this point, we should still have one conn that hasn't been dropped (even though the pool has) + // At this point, we should still have one conn that hasn't been dropped // because we haven't ended the persistent client let real_dropcount = conn_drop_counter.load(Ordering::Relaxed); assert!( @@ -1194,7 +1195,7 @@ mod test { client_stop_signal.drain().await; assert!(!persist_res.await.is_err(), "PERSIST CLIENT ERROR"); - sleep(Duration::from_secs(6)).await; + sleep(Duration::from_secs(2)).await; let after_conncount = conn_counter.load(Ordering::Relaxed); assert!(after_conncount == 3, "after conncount was {after_conncount}"); @@ -1293,12 +1294,12 @@ mod test { loop { count += 1; let res = c1.send_request(req()).await; - if res.is_err() { panic!("SEND ERR: {:#?} sendcount {count}", res); } else if res.unwrap().status() != 200 { panic!("CLIENT RETURNED ERROR") } + } }; From c2f90150db704b965e8f04d94ce42dde8d3d7939 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Mon, 22 Apr 2024 17:46:14 -0400 Subject: [PATCH 32/46] Make this a bit simpler Signed-off-by: Benjamin Leggett --- src/proxy/outbound.rs | 2 +- src/proxy/pool.rs | 288 ++++++++++++++++++++---------------------- 2 files changed, 138 insertions(+), 152 deletions(-) diff --git a/src/proxy/outbound.rs b/src/proxy/outbound.rs index 86583531b..911b318f4 100644 --- a/src/proxy/outbound.rs +++ b/src/proxy/outbound.rs @@ -171,7 +171,7 @@ impl OutboundConnection { Some(drain) => { tokio::select! { _ = drain.signaled() => { - info!("socks drain signaled"); + info!("drain signaled"); } res = self.proxy_to(stream, remote_addr, orig_dst_addr, block_passthrough) => res } diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 736ce5552..a2bfcb727 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -37,7 +37,7 @@ use tracing::{debug, error, trace}; use crate::config; use crate::identity::{Identity, SecretManager}; -use flurry::HashMap; +use flurry; use pingora_pool; @@ -51,12 +51,8 @@ use pingora_pool; // by flow control throttling. #[derive(Clone)] pub struct WorkloadHBONEPool { - cfg: config::Config, - socket_factory: Arc, - cert_manager: Arc, state: Arc, pool_watcher: watch::Receiver, - max_streamcount: u16, } // PoolState is effectively the gnarly inner state stuff that needs thread/task sync, and should be wrapped in a Mutex. @@ -68,12 +64,17 @@ struct PoolState { // and has no actual hyper/http/connection logic. connected_pool: Arc>, // this must be an atomic/concurrent-safe list-of-locks, so we can lock per-key, not globally, and avoid holding up all conn attempts - established_conn_writelock: HashMap>>>, + established_conn_writelock: flurry::HashMap>>>, close_pollers: futures::stream::FuturesUnordered>, pool_unused_release_timeout: Duration, // This is merely a counter to track the overall number of conns this pool spawns // to ensure we get unique poolkeys-per-new-conn, it is not a limit pool_global_conn_count: AtomicI32, + // TODO move this stuff out + cfg: config::Config, + socket_factory: Arc, + cert_manager: Arc, + max_streamcount: u16, } impl PoolState { @@ -104,6 +105,7 @@ impl PoolState { let pool_key_ref = pool_key.clone(); let release_timeout = self.pool_unused_release_timeout; self.close_pollers.push(tokio::spawn(async move { + debug!("starting an idle timeout for connection {:#?}", pool_key_ref); pool_ref .idle_timeout(&pool_key_ref, release_timeout, evict, rx, pickup) .await; @@ -131,6 +133,60 @@ impl PoolState { } } + async fn spawn_new_pool_conn( + &self, + key: WorkloadKey, + ) -> Result>, Error> { + debug!("spawning new pool conn for key {:#?}", key); + let clone_key = key.clone(); + let mut c_builder = http2::Builder::new(crate::hyper_util::TokioExecutor); + let builder = c_builder + .initial_stream_window_size(self.cfg.window_size) + .max_frame_size(self.cfg.frame_size) + .initial_connection_window_size(self.cfg.connection_window_size); + + let local = self + .cfg + .enable_original_source + .unwrap_or_default() + .then_some(key.src); + let cert = self.cert_manager.fetch_certificate(&key.src_id).await?; + let connector = cert.outbound_connector(key.dst_id)?; + let tcp_stream = + super::freebind_connect(local, key.dst, self.socket_factory.as_ref()).await?; + tcp_stream.set_nodelay(true)?; // TODO: this is backwards of expectations + let tls_stream = connector.connect(tcp_stream).await?; + let (request_sender, connection) = builder + .handshake(::hyper_util::rt::TokioIo::new(tls_stream)) + .await + .map_err(Error::HttpHandshake)?; + + // spawn a task to poll the connection and drive the HTTP state + // if we got a drain for that connection, respect it in a race + // it is important to have a drain here, or this connection will never terminate + let mut driver_drain = self.timeout_rx.clone(); + tokio::spawn(async move { + debug!("starting a connection driver for {:?}", clone_key); + tokio::select! { + _ = driver_drain.changed() => { + debug!("draining outer HBONE connection {:?}", clone_key); + } + res = connection=> { + match res { + Err(e) => { + error!("Error in HBONE connection handshake: {:?}", e); + } + Ok(_) => { + debug!("done with HBONE connection handshake: {:?}", res); + } + } + } + } + }); + + Ok(request_sender) + } + // Just for safety's sake, since we are using a hash thanks to pingora supporting arbitrary Eq, Hash // types, do a deep equality test before returning the conn, returning an error if the conn's key does // not equal the provided key @@ -159,7 +215,7 @@ impl PoolState { // // This is so we can backpressure correctly if 1000 tasks all demand a new connection // to the same key at once, and not eagerly open 1000 tunnel connections. - async fn first_checkout_conn_from_pool( + async fn maybe_checkout_conn_from_pool_under_writelock( &self, workload_key: &WorkloadKey, pool_key: &pingora_pool::ConnectionMeta, @@ -183,23 +239,39 @@ impl PoolState { "first checkout - got writelock for conn with key {:#?} and hash {:#?}", workload_key, pool_key.key ); - let got = self.guarded_get(&pool_key.key, workload_key)?; - Ok(got.and_then(|e_conn| { - trace!( - "first checkout - inner pool - got existing conn for key {:#?}", - workload_key - ); - if e_conn.at_max_streamcount() { - debug!( - "got conn for key {:#?}, but streamcount is maxed", + let result = match self.guarded_get(&pool_key.key, workload_key)? { + Some(e_conn) => { + trace!( + "first checkout - inner pool - got existing conn for key {:#?}", workload_key ); - None - } else { - self.checkin_conn(e_conn.clone(), pool_key.clone()); - Some(e_conn) + if e_conn.at_max_streamcount() { + debug!( + "got conn for key {:#?}, but streamcount is maxed", + workload_key + ); + + debug!("spawning new conn for wl key {:#?} to replace using pool key {:#?}", workload_key, pool_key); + let pool_conn = self.spawn_new_pool_conn(workload_key.clone()).await; + let r_conn = ConnClient { + sender: pool_conn?, + stream_count: Arc::new(AtomicU16::new(0)), + stream_count_max: self.max_streamcount, + wl_key: workload_key.clone(), + }; + self.checkin_conn(r_conn.clone(), pool_key.clone()); + // None + Some(r_conn) + } else { + debug!("checking existing conn for key {:#?} back in", pool_key); + self.checkin_conn(e_conn.clone(), pool_key.clone()); + Some(e_conn) + } } - })) + None => None, + }; + + Ok(result) } None => Ok(None), } @@ -240,16 +312,16 @@ impl WorkloadHBONEPool { // the number here is simply the number of unique src/dest keys // the pool is expected to track before the inner hashmap resizes. connected_pool: Arc::new(pingora_pool::ConnectionPool::new(500)), - established_conn_writelock: HashMap::new(), + established_conn_writelock: flurry::HashMap::new(), close_pollers: futures::stream::FuturesUnordered::new(), pool_unused_release_timeout: pool_duration, pool_global_conn_count: AtomicI32::new(0), + cfg, + socket_factory, + cert_manager, + max_streamcount: max_count, }), - cfg, - socket_factory, - cert_manager, pool_watcher: timeout_rx, - max_streamcount: max_count, } } @@ -270,7 +342,7 @@ impl WorkloadHBONEPool { hash_key, self.state .pool_global_conn_count - .fetch_add(1, Ordering::Relaxed), + .fetch_add(1, Ordering::SeqCst), ); debug!("initial attempt - try to get existing conn from pool"); // First, see if we can naively just check out a connection. @@ -285,7 +357,7 @@ impl WorkloadHBONEPool { // to the same key at once, and not eagerly open 1000 tunnel connections. let existing_conn = self .state - .first_checkout_conn_from_pool(&workload_key, &pool_key) + .maybe_checkout_conn_from_pool_under_writelock(&workload_key, &pool_key) .await?; // Early return, no need to do anything else @@ -356,10 +428,10 @@ impl WorkloadHBONEPool { }; debug!( - "appears we need a new conn, attempting to win connlock for wl key {:#?}", + "appears we need a new conn for this key, attempting to win connlock for wl key {:#?}", workload_key ); - let got_conn = match inner_conn_lock.try_lock() { + let res = match inner_conn_lock.try_lock() { Ok(_guard) => { // BEGIN take inner writelock // If we get here, it means the following are true: @@ -368,11 +440,11 @@ impl WorkloadHBONEPool { // // So, carry on doing that. debug!("nothing else is creating a conn and we won the lock, make one"); - let pool_conn = self.spawn_new_pool_conn(workload_key.clone()).await; + let pool_conn = self.state.spawn_new_pool_conn(workload_key.clone()).await; let client = ConnClient { sender: pool_conn?, stream_count: Arc::new(AtomicU16::new(0)), - stream_count_max: self.max_streamcount, + stream_count_max: self.state.max_streamcount, wl_key: workload_key.clone(), }; @@ -380,7 +452,10 @@ impl WorkloadHBONEPool { "starting new conn for key {:#?} with pk {:#?}", workload_key, pool_key ); - Some(client) + + debug!("checking new conn for key {:#?} into pool", pool_key); + self.state.checkin_conn(client.clone(), pool_key.clone()); + client // END take inner writelock } Err(_) => { @@ -400,10 +475,11 @@ impl WorkloadHBONEPool { "notified a new conn was enpooled, checking for hash {:#?}", hash_key ); - // Notifier fired, try and get a conn out for our key. - // If we do, make sure it's not maxed out on streams. - let existing_conn = self.state.guarded_get(&hash_key, &workload_key)?; + let existing_conn = self + .state + .maybe_checkout_conn_from_pool_under_writelock(&workload_key, &pool_key) + .await?; match existing_conn { None => { trace!("woke up on pool notification, but didn't find a conn for {:#?} yet", hash_key); @@ -411,13 +487,7 @@ impl WorkloadHBONEPool { } Some(e_conn) => { debug!("found existing conn after waiting"); - // We found a conn, but it's already maxed out. - // Return None and create another. - if e_conn.at_max_streamcount() { - debug!("found existing conn for key {:#?}, but streamcount is maxed", workload_key); - break None; - } - break Some(e_conn); + break e_conn; } } } @@ -429,99 +499,7 @@ impl WorkloadHBONEPool { } }; - // If we get here, it means the following are true: - // 1. At one point, there was a preexisting conn in the pool for this key. - // 2. When we checked, we got nothing for that key. - // 3. We could not get the exclusive inner writelock to add a new one - // 4. Someone else got the exclusive inner writelock, and is adding a new one - // 5. We waited until we got that connection for our key, that someone else added. - // - // So now we are down to 2 options: - // - // 1. We got an existing connection, it's usable, we use it. - // 2. We got an existing connection, it's not usable, we start another. - // - // Note that for (2) we do not really need to take the inner writelock again for this key - - // if we checked out a conn inserted by someone else that we cannot use, - // everyone else will implicitly be waiting for us to check one back in, - // since they are in their own loops. - match got_conn { - // After waiting, we found an available conn we can use, so no need to start another. - // Clone the underlying client, return a copy, and put the other back in the pool. - Some(f_conn) => { - self.state.checkin_conn(f_conn.clone(), pool_key.clone()); - Ok(f_conn) - } - - // After waiting, we found an available conn, but for whatever reason couldn't use it. - // (streamcount maxed, etc) - // Start a new one, clone the underlying client, return a copy, and put the other back in the pool. - None => { - debug!("spawning new conn for key {:#?} to replace", workload_key); - let pool_conn = self.spawn_new_pool_conn(workload_key.clone()).await; - let r_conn = ConnClient { - sender: pool_conn?, - stream_count: Arc::new(AtomicU16::new(0)), - stream_count_max: self.max_streamcount, - wl_key: workload_key.clone(), - }; - self.state.checkin_conn(r_conn.clone(), pool_key.clone()); - Ok(r_conn) - } - } - } - - async fn spawn_new_pool_conn( - &self, - key: WorkloadKey, - ) -> Result>, Error> { - let clone_key = key.clone(); - let mut c_builder = http2::Builder::new(crate::hyper_util::TokioExecutor); - let builder = c_builder - .initial_stream_window_size(self.cfg.window_size) - .max_frame_size(self.cfg.frame_size) - .initial_connection_window_size(self.cfg.connection_window_size); - - let local = self - .cfg - .enable_original_source - .unwrap_or_default() - .then_some(key.src); - let cert = self.cert_manager.fetch_certificate(&key.src_id).await?; - let connector = cert.outbound_connector(key.dst_id)?; - let tcp_stream = - super::freebind_connect(local, key.dst, self.socket_factory.as_ref()).await?; - tcp_stream.set_nodelay(true)?; // TODO: this is backwards of expectations - let tls_stream = connector.connect(tcp_stream).await?; - let (request_sender, connection) = builder - .handshake(::hyper_util::rt::TokioIo::new(tls_stream)) - .await - .map_err(Error::HttpHandshake)?; - - // spawn a task to poll the connection and drive the HTTP state - // if we got a drain for that connection, respect it in a race - // it is important to have a drain here, or this connection will never terminate - let mut driver_drain = self.state.timeout_rx.clone(); - tokio::spawn(async move { - debug!("starting a connection driver for {:?}", clone_key); - tokio::select! { - _ = driver_drain.changed() => { - debug!("draining outer HBONE connection {:?}", clone_key); - } - res = connection=> { - match res { - Err(e) => { - error!("Error in HBONE connection handshake: {:?}", e); - } - Ok(_) => { - debug!("done with HBONE connection handshake: {:?}", res); - } - } - } - } - }); - - Ok(request_sender) + Ok(res) } } @@ -566,6 +544,16 @@ impl ConnClient { } } +// This is currently only for debugging +impl Drop for ConnClient { + fn drop(&mut self) { + debug!( + "dropping ConnClient for key {:#?} with streamcount: {:?} / {:?}", + self.wl_key, self.stream_count, self.stream_count_max + ) + } +} + #[derive(PartialEq, Eq, Hash, Clone, Debug)] pub struct WorkloadKey { pub src_id: Identity, @@ -846,7 +834,7 @@ mod test { server_handle.await.unwrap(); let real_conncount = conn_counter.load(Ordering::Relaxed); - assert!(real_conncount == 3, "actual conncount was {real_conncount}"); + assert!(real_conncount == 4, "actual conncount was {real_conncount}"); } #[tokio::test] @@ -1103,7 +1091,7 @@ mod test { "actual before dropcount was {before_dropcount}" ); - // Attempt to wait long enough for pool conns to timeout+drop + // Attempt to wait long enough for pool conns to timeout+evict sleep(Duration::from_secs(2)).await; let real_conncount = conn_counter.load(Ordering::Relaxed); @@ -1160,7 +1148,8 @@ mod test { } let (client_stop_signal, client_stop) = drain::channel(); - let persist_res = spawn_persistent_client(pool.clone(), key1.clone(), server_addr, client_stop); + let persist_res = + spawn_persistent_client(pool.clone(), key1.clone(), server_addr, client_stop); //loop thru the nonpersistent clients and wait for them to finish while let Some(Err(res)) = tasks.next().await { @@ -1171,7 +1160,7 @@ mod test { let before_conncount = conn_counter.load(Ordering::Relaxed); let before_dropcount = conn_drop_counter.load(Ordering::Relaxed); assert!( - before_conncount == 3, + before_conncount == 4, "actual before conncount was {before_conncount}" ); assert!( @@ -1179,36 +1168,34 @@ mod test { "actual before dropcount was {before_dropcount}" ); - // Attempt to wait long enough for pool conns to timeout+evict sleep(Duration::from_secs(2)).await; let real_conncount = conn_counter.load(Ordering::Relaxed); - assert!(real_conncount == 3, "actual conncount was {real_conncount}"); + assert!(real_conncount == 4, "actual conncount was {real_conncount}"); // At this point, we should still have one conn that hasn't been dropped // because we haven't ended the persistent client let real_dropcount = conn_drop_counter.load(Ordering::Relaxed); - assert!( - real_dropcount == 2, - "actual dropcount was {real_dropcount}" - ); + assert!(real_dropcount == 3, "actual dropcount was {real_dropcount}"); client_stop_signal.drain().await; - assert!(!persist_res.await.is_err(), "PERSIST CLIENT ERROR"); + assert!(persist_res.await.is_ok(), "PERSIST CLIENT ERROR"); sleep(Duration::from_secs(2)).await; let after_conncount = conn_counter.load(Ordering::Relaxed); - assert!(after_conncount == 3, "after conncount was {after_conncount}"); + assert!( + after_conncount == 4, + "after conncount was {after_conncount}" + ); let after_dropcount = conn_drop_counter.load(Ordering::Relaxed); assert!( - after_dropcount == 3, + after_dropcount == 4, "after dropcount was {after_dropcount}" ); server_drain_signal.drain().await; server_handle.await.unwrap(); drop(pool); - } fn spawn_client( @@ -1299,7 +1286,6 @@ mod test { } else if res.unwrap().status() != 200 { panic!("CLIENT RETURNED ERROR") } - } }; From bde5b459652ad486dad777926684807446f51e03 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Mon, 22 Apr 2024 19:17:38 -0400 Subject: [PATCH 33/46] Separate out the connspawner Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 264 ++++++++++++++++++++++++++-------------------- 1 file changed, 150 insertions(+), 114 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index a2bfcb727..1f03a6fe7 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -59,7 +59,6 @@ pub struct WorkloadHBONEPool { struct PoolState { pool_notifier: watch::Sender, // This is already impl clone? rustc complains that it isn't, tho timeout_tx: watch::Sender, // This is already impl clone? rustc complains that it isn't, tho - timeout_rx: watch::Receiver, // this is effectively just a convenience data type - a rwlocked hashmap with keying and LRU drops // and has no actual hyper/http/connection logic. connected_pool: Arc>, @@ -70,11 +69,72 @@ struct PoolState { // This is merely a counter to track the overall number of conns this pool spawns // to ensure we get unique poolkeys-per-new-conn, it is not a limit pool_global_conn_count: AtomicI32, - // TODO move this stuff out + max_streamcount: u16, + spawner: ConnSpawner, +} + +struct ConnSpawner { cfg: config::Config, socket_factory: Arc, cert_manager: Arc, - max_streamcount: u16, + timeout_rx: watch::Receiver, +} + +// Does nothing but spawn new conns when asked +impl ConnSpawner { + async fn new_pool_conn( + &self, + key: WorkloadKey, + ) -> Result>, Error> { + debug!("spawning new pool conn for key {:#?}", key); + let clone_key = key.clone(); + let mut c_builder = http2::Builder::new(crate::hyper_util::TokioExecutor); + let builder = c_builder + .initial_stream_window_size(self.cfg.window_size) + .max_frame_size(self.cfg.frame_size) + .initial_connection_window_size(self.cfg.connection_window_size); + + let local = self + .cfg + .enable_original_source + .unwrap_or_default() + .then_some(key.src); + let cert = self.cert_manager.fetch_certificate(&key.src_id).await?; + let connector = cert.outbound_connector(key.dst_id)?; + let tcp_stream = + super::freebind_connect(local, key.dst, self.socket_factory.as_ref()).await?; + tcp_stream.set_nodelay(true)?; // TODO: this is backwards of expectations + let tls_stream = connector.connect(tcp_stream).await?; + let (request_sender, connection) = builder + .handshake(::hyper_util::rt::TokioIo::new(tls_stream)) + .await + .map_err(Error::HttpHandshake)?; + + // spawn a task to poll the connection and drive the HTTP state + // if we got a drain for that connection, respect it in a race + // it is important to have a drain here, or this connection will never terminate + let mut driver_drain = self.timeout_rx.clone(); + tokio::spawn(async move { + debug!("starting a connection driver for {:?}", clone_key); + tokio::select! { + _ = driver_drain.changed() => { + debug!("draining outer HBONE connection {:?}", clone_key); + } + res = connection=> { + match res { + Err(e) => { + error!("Error in HBONE connection handshake: {:?}", e); + } + Ok(_) => { + debug!("done with HBONE connection handshake: {:?}", res); + } + } + } + } + }); + + Ok(request_sender) + } } impl PoolState { @@ -100,12 +160,15 @@ impl PoolState { // they must also drop those before the underlying connection is fully closed. fn checkin_conn(&self, conn: ConnClient, pool_key: pingora_pool::ConnectionMeta) { let (evict, pickup) = self.connected_pool.put(&pool_key, conn); - let rx = self.timeout_rx.clone(); + let rx = self.spawner.timeout_rx.clone(); let pool_ref = self.connected_pool.clone(); let pool_key_ref = pool_key.clone(); let release_timeout = self.pool_unused_release_timeout; self.close_pollers.push(tokio::spawn(async move { - debug!("starting an idle timeout for connection {:#?}", pool_key_ref); + debug!( + "starting an idle timeout for connection {:#?}", + pool_key_ref + ); pool_ref .idle_timeout(&pool_key_ref, release_timeout, evict, rx, pickup) .await; @@ -133,60 +196,6 @@ impl PoolState { } } - async fn spawn_new_pool_conn( - &self, - key: WorkloadKey, - ) -> Result>, Error> { - debug!("spawning new pool conn for key {:#?}", key); - let clone_key = key.clone(); - let mut c_builder = http2::Builder::new(crate::hyper_util::TokioExecutor); - let builder = c_builder - .initial_stream_window_size(self.cfg.window_size) - .max_frame_size(self.cfg.frame_size) - .initial_connection_window_size(self.cfg.connection_window_size); - - let local = self - .cfg - .enable_original_source - .unwrap_or_default() - .then_some(key.src); - let cert = self.cert_manager.fetch_certificate(&key.src_id).await?; - let connector = cert.outbound_connector(key.dst_id)?; - let tcp_stream = - super::freebind_connect(local, key.dst, self.socket_factory.as_ref()).await?; - tcp_stream.set_nodelay(true)?; // TODO: this is backwards of expectations - let tls_stream = connector.connect(tcp_stream).await?; - let (request_sender, connection) = builder - .handshake(::hyper_util::rt::TokioIo::new(tls_stream)) - .await - .map_err(Error::HttpHandshake)?; - - // spawn a task to poll the connection and drive the HTTP state - // if we got a drain for that connection, respect it in a race - // it is important to have a drain here, or this connection will never terminate - let mut driver_drain = self.timeout_rx.clone(); - tokio::spawn(async move { - debug!("starting a connection driver for {:?}", clone_key); - tokio::select! { - _ = driver_drain.changed() => { - debug!("draining outer HBONE connection {:?}", clone_key); - } - res = connection=> { - match res { - Err(e) => { - error!("Error in HBONE connection handshake: {:?}", e); - } - Ok(_) => { - debug!("done with HBONE connection handshake: {:?}", res); - } - } - } - } - }); - - Ok(request_sender) - } - // Just for safety's sake, since we are using a hash thanks to pingora supporting arbitrary Eq, Hash // types, do a deep equality test before returning the conn, returning an error if the conn's key does // not equal the provided key @@ -202,6 +211,65 @@ impl PoolState { } } + async fn start_conn_if_win_writelock( + &self, + workload_key: &WorkloadKey, + pool_key: &pingora_pool::ConnectionMeta, + ) -> Option { + let inner_conn_lock = { + trace!("getting keyed lock out of lockmap"); + let guard = self.established_conn_writelock.guard(); + + let exist_conn_lock = self + .established_conn_writelock + .get(&pool_key.key, &guard) + .unwrap(); + trace!("got keyed lock out of lockmap"); + exist_conn_lock.as_ref().unwrap().clone() + }; + + debug!("attempting to win connlock for wl key {:#?}", workload_key); + + let inner_lock = inner_conn_lock.try_lock(); + match inner_lock { + Ok(_guard) => { + // BEGIN take inner writelock + debug!("nothing else is creating a conn and we won the lock, make one"); + match self.spawner.new_pool_conn(workload_key.clone()).await { + Ok(pool_conn) => { + let client = ConnClient { + sender: pool_conn, + stream_count: Arc::new(AtomicU16::new(0)), + stream_count_max: self.max_streamcount, + wl_key: workload_key.clone(), + }; + + debug!( + "starting new conn for key {:#?} with pk {:#?}", + workload_key, pool_key + ); + + debug!("checking new conn for key {:#?} into pool", pool_key); + self.checkin_conn(client.clone(), pool_key.clone()); + Some(client) + } + Err(e) => { + error!("could not spawn new conn, got {e}"); + None + } + } + // END take inner writelock + } + Err(_) => { + debug!( + "did not win connlock for wl key {:#?}, something else has it", + workload_key + ); + None + } + } + } + // Does an initial, naive check to see if a conn exists for this key. // // If it does, WRITELOCK the mutex for that key, clone (or create), check in the clone, @@ -215,7 +283,7 @@ impl PoolState { // // This is so we can backpressure correctly if 1000 tasks all demand a new connection // to the same key at once, and not eagerly open 1000 tunnel connections. - async fn maybe_checkout_conn_from_pool_under_writelock( + async fn checkout_conn_under_writelock( &self, workload_key: &WorkloadKey, pool_key: &pingora_pool::ConnectionMeta, @@ -252,7 +320,7 @@ impl PoolState { ); debug!("spawning new conn for wl key {:#?} to replace using pool key {:#?}", workload_key, pool_key); - let pool_conn = self.spawn_new_pool_conn(workload_key.clone()).await; + let pool_conn = self.spawner.new_pool_conn(workload_key.clone()).await; let r_conn = ConnClient { sender: pool_conn?, stream_count: Arc::new(AtomicU16::new(0)), @@ -300,6 +368,14 @@ impl WorkloadHBONEPool { let (timeout_send, timeout_recv) = watch::channel(false); let max_count = cfg.pool_max_streams_per_conn; let pool_duration = cfg.pool_unused_release_timeout; + + let spawner = ConnSpawner { + cfg, + socket_factory, + cert_manager, + timeout_rx: timeout_recv.clone(), + }; + // This is merely a counter to track the overall number of conns this pool spawns // to ensure we get unique poolkeys-per-new-conn, it is not a limit debug!("constructing pool with {:#?} streams per conn", max_count); @@ -308,7 +384,7 @@ impl WorkloadHBONEPool { state: Arc::new(PoolState { pool_notifier: timeout_tx, timeout_tx: timeout_send, - timeout_rx: timeout_recv, + // timeout_rx: timeout_recv, // the number here is simply the number of unique src/dest keys // the pool is expected to track before the inner hashmap resizes. connected_pool: Arc::new(pingora_pool::ConnectionPool::new(500)), @@ -316,10 +392,8 @@ impl WorkloadHBONEPool { close_pollers: futures::stream::FuturesUnordered::new(), pool_unused_release_timeout: pool_duration, pool_global_conn_count: AtomicI32::new(0), - cfg, - socket_factory, - cert_manager, max_streamcount: max_count, + spawner, }), pool_watcher: timeout_rx, } @@ -357,7 +431,7 @@ impl WorkloadHBONEPool { // to the same key at once, and not eagerly open 1000 tunnel connections. let existing_conn = self .state - .maybe_checkout_conn_from_pool_under_writelock(&workload_key, &pool_key) + .checkout_conn_under_writelock(&workload_key, &pool_key) .await?; // Early return, no need to do anything else @@ -402,7 +476,7 @@ impl WorkloadHBONEPool { // This doesn't block other tasks spawning connections against other keys, but DOES block other // tasks spawning connections against THIS key - which is what we want. - // NOTE: This inner, key-specific mutex is a tokio::async::Mutex, and not a stdlib sync mutex. + // NOTE: The inner, key-specific mutex is a tokio::async::Mutex, and not a stdlib sync mutex. // these differ from the stdlib sync mutex in that they are (slightly) slower // (they effectively sleep the current task) and they can be held over an await. // The tokio docs (rightly) advise you to not use these, @@ -414,51 +488,14 @@ impl WorkloadHBONEPool { // // So the downsides are actually useful (we WANT task contention - // to block other parallel tasks from trying to spawn a connection for this key if we are already doing so) - let inner_conn_lock = { - trace!("fallback attempt - getting keyed lock out of lockmap"); - let guard = self.state.established_conn_writelock.guard(); - - let exist_conn_lock = self - .state - .established_conn_writelock - .get(&hash_key, &guard) - .unwrap(); - trace!("fallback attempt - got keyed lock out of lockmap"); - exist_conn_lock.as_ref().unwrap().clone() - }; - - debug!( - "appears we need a new conn for this key, attempting to win connlock for wl key {:#?}", - workload_key - ); - let res = match inner_conn_lock.try_lock() { - Ok(_guard) => { - // BEGIN take inner writelock - // If we get here, it means the following are true: - // 1. We did not get a connection for our key. - // 2. We have the exclusive inner writelock to create a new connection for our key. - // - // So, carry on doing that. - debug!("nothing else is creating a conn and we won the lock, make one"); - let pool_conn = self.state.spawn_new_pool_conn(workload_key.clone()).await; - let client = ConnClient { - sender: pool_conn?, - stream_count: Arc::new(AtomicU16::new(0)), - stream_count_max: self.state.max_streamcount, - wl_key: workload_key.clone(), - }; - - debug!( - "starting new conn for key {:#?} with pk {:#?}", - workload_key, pool_key - ); - - debug!("checking new conn for key {:#?} into pool", pool_key); - self.state.checkin_conn(client.clone(), pool_key.clone()); - client - // END take inner writelock - } - Err(_) => { + trace!("fallback attempt - trying win win connlock"); + let res = match self + .state + .start_conn_if_win_writelock(&workload_key, &pool_key) + .await + { + Some(client) => client, + None => { debug!("we didn't win the lock, something else is creating a conn, wait for it"); // If we get here, it means the following are true: // 1. At one point, there was a preexisting conn in the pool for this key. @@ -478,7 +515,7 @@ impl WorkloadHBONEPool { // Notifier fired, try and get a conn out for our key. let existing_conn = self .state - .maybe_checkout_conn_from_pool_under_writelock(&workload_key, &pool_key) + .checkout_conn_under_writelock(&workload_key, &pool_key) .await?; match existing_conn { None => { @@ -498,7 +535,6 @@ impl WorkloadHBONEPool { } } }; - Ok(res) } } From 22840aed43d947f88ab1f612f9a644154f979662 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Mon, 22 Apr 2024 19:26:10 -0400 Subject: [PATCH 34/46] Tidy logging a bit Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 41 +++++++++++++++-------------------------- 1 file changed, 15 insertions(+), 26 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 1f03a6fe7..c7dba91a6 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -228,7 +228,7 @@ impl PoolState { exist_conn_lock.as_ref().unwrap().clone() }; - debug!("attempting to win connlock for wl key {:#?}", workload_key); + trace!("attempting to win connlock for wl key {:#?}", workload_key); let inner_lock = inner_conn_lock.try_lock(); match inner_lock { @@ -245,11 +245,9 @@ impl PoolState { }; debug!( - "starting new conn for key {:#?} with pk {:#?}", + "checking in new conn for key {:#?} with pk {:#?}", workload_key, pool_key ); - - debug!("checking new conn for key {:#?} into pool", pool_key); self.checkin_conn(client.clone(), pool_key.clone()); Some(client) } @@ -288,8 +286,6 @@ impl PoolState { workload_key: &WorkloadKey, pool_key: &pingora_pool::ConnectionMeta, ) -> Result, Error> { - debug!("first checkout READGUARD"); - let found_conn = { trace!("pool connect outer map - take guard"); let guard = self.established_conn_writelock.guard(); @@ -300,26 +296,19 @@ impl PoolState { }; match found_conn { Some(exist_conn_lock) => { - debug!("first checkout - found mutex for key, waiting for writelock"); + debug!("checkout - found mutex for key, waiting for writelock"); let _conn_lock = exist_conn_lock.as_ref().lock().await; - debug!( - "first checkout - got writelock for conn with key {:#?} and hash {:#?}", - workload_key, pool_key.key + trace!( + "checkout - got writelock for conn with key {:#?} and hash {:#?}", + workload_key, + pool_key.key ); let result = match self.guarded_get(&pool_key.key, workload_key)? { Some(e_conn) => { - trace!( - "first checkout - inner pool - got existing conn for key {:#?}", - workload_key - ); + trace!("checkout - got existing conn for key {:#?}", workload_key); if e_conn.at_max_streamcount() { - debug!( - "got conn for key {:#?}, but streamcount is maxed", - workload_key - ); - - debug!("spawning new conn for wl key {:#?} to replace using pool key {:#?}", workload_key, pool_key); + debug!("got conn for wl key {:#?}, but streamcount is maxed, spawning new conn to replace using pool key {:#?}", workload_key, pool_key); let pool_conn = self.spawner.new_pool_conn(workload_key.clone()).await; let r_conn = ConnClient { sender: pool_conn?, @@ -328,7 +317,6 @@ impl PoolState { wl_key: workload_key.clone(), }; self.checkin_conn(r_conn.clone(), pool_key.clone()); - // None Some(r_conn) } else { debug!("checking existing conn for key {:#?} back in", pool_key); @@ -406,7 +394,7 @@ impl WorkloadHBONEPool { // If many `connects` request a connection to the same dest at once, all will wait until exactly // one connection is created, before deciding if they should create more or just use that one. pub async fn connect(&mut self, workload_key: WorkloadKey) -> Result { - debug!("pool connect START"); + trace!("pool connect START"); // TODO BML this may not be collision resistant/slow. It should be resistant enough for workloads tho. // We are doing a deep-equals check at the end to mitigate any collisions, will see about bumping Pingora let mut s = DefaultHasher::new(); @@ -418,7 +406,6 @@ impl WorkloadHBONEPool { .pool_global_conn_count .fetch_add(1, Ordering::SeqCst), ); - debug!("initial attempt - try to get existing conn from pool"); // First, see if we can naively just check out a connection. // This should be the common case, except for the first establishment of a new connection/key. // This will be done under outer readlock (nonexclusive)/inner keyed writelock (exclusive). @@ -553,7 +540,7 @@ pub struct ConnClient { impl ConnClient { pub fn at_max_streamcount(&self) -> bool { let curr_count = self.stream_count.load(Ordering::Relaxed); - debug!("checking streamcount: {curr_count}"); + trace!("checking streamcount: {curr_count}"); if curr_count >= self.stream_count_max { return true; } @@ -583,9 +570,11 @@ impl ConnClient { // This is currently only for debugging impl Drop for ConnClient { fn drop(&mut self) { - debug!( + trace!( "dropping ConnClient for key {:#?} with streamcount: {:?} / {:?}", - self.wl_key, self.stream_count, self.stream_count_max + self.wl_key, + self.stream_count, + self.stream_count_max ) } } From f1b102a7242fb4b48874b109e84ce59ca7ffffc1 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Tue, 23 Apr 2024 13:26:37 -0400 Subject: [PATCH 35/46] Add serverside keepalive Signed-off-by: Benjamin Leggett --- src/proxy/inbound.rs | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/proxy/inbound.rs b/src/proxy/inbound.rs index 379a72326..4e60e3ec7 100644 --- a/src/proxy/inbound.rs +++ b/src/proxy/inbound.rs @@ -17,7 +17,7 @@ use std::fmt; use std::fmt::{Display, Formatter}; use std::net::SocketAddr; use std::sync::Arc; -use std::time::Instant; +use std::time::{Instant, Duration}; use bytes::Bytes; use drain::Watch; @@ -119,6 +119,11 @@ impl Inbound { let serve = crate::hyper_util::http2_server() .initial_stream_window_size(self.pi.cfg.window_size) .initial_connection_window_size(self.pi.cfg.connection_window_size) + // well behaved clients should close connections. + // not all clients are well-behaved. This will prune + // connections when the client is not responding, to keep + // us from holding many stale conns from deceased clients + .keep_alive_interval(Some(Duration::from_secs(10))) .max_frame_size(self.pi.cfg.frame_size) // 64KB max; default is 16MB driven from Golang's defaults // Since we know we are going to recieve a bounded set of headers, more is overkill. From 795b4aeae7272c4a54b913320ca7bf51e3aee5e4 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Tue, 23 Apr 2024 13:48:05 -0400 Subject: [PATCH 36/46] fixup Signed-off-by: Benjamin Leggett --- src/proxy/inbound.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/proxy/inbound.rs b/src/proxy/inbound.rs index 4e60e3ec7..b3d511723 100644 --- a/src/proxy/inbound.rs +++ b/src/proxy/inbound.rs @@ -17,7 +17,7 @@ use std::fmt; use std::fmt::{Display, Formatter}; use std::net::SocketAddr; use std::sync::Arc; -use std::time::{Instant, Duration}; +use std::time::{Duration, Instant}; use bytes::Bytes; use drain::Watch; From e16368aac79d216a6ba9a76b4282c5cc429d94fa Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Wed, 24 Apr 2024 14:58:51 -0400 Subject: [PATCH 37/46] Just for kicks Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 20 +++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index c7dba91a6..ea7848dd6 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -526,7 +526,7 @@ impl WorkloadHBONEPool { } } -#[derive(Debug, Clone)] +#[derive(Debug)] // A sort of faux-client, that represents a single checked-out 'request sender' which might // send requests over some underlying stream using some underlying http/2 client pub struct ConnClient { @@ -579,6 +579,24 @@ impl Drop for ConnClient { } } +// This is currently only for debugging +impl Clone for ConnClient { + fn clone(&self) -> Self { + trace!( + "cloning ConnClient for key {:#?} with streamcount: {:?} / {:?}", + self.wl_key, + self.stream_count, + self.stream_count_max + ); + ConnClient { + sender: self.sender.clone(), + stream_count: self.stream_count.clone(), + stream_count_max: self.stream_count_max, + wl_key: self.wl_key.clone(), + } + } +} + #[derive(PartialEq, Eq, Hash, Clone, Debug)] pub struct WorkloadKey { pub src_id: Identity, From 969b99aa0ffc537ac0837c705c824522806a5773 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Thu, 25 Apr 2024 13:10:06 -0400 Subject: [PATCH 38/46] D'oh - use mthread runtime for tests Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 23 +++++++++++------------ 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index ea7848dd6..c2a429411 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -637,7 +637,7 @@ mod test { use super::*; - #[tokio::test] + #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn test_pool_reuses_conn_for_same_key() { // crate::telemetry::setup_logging(); @@ -687,7 +687,7 @@ mod test { assert!(client3.is_ok()); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn test_pool_does_not_reuse_conn_for_diff_key() { let (server_drain_signal, server_drain) = drain::channel(); @@ -739,7 +739,7 @@ mod test { assert!(client2.is_ok()); // expect this to panic - we used a new key } - #[tokio::test] + #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn test_pool_respects_per_conn_stream_limit() { let (server_drain_signal, server_drain) = drain::channel(); @@ -782,7 +782,7 @@ mod test { assert!(client2.is_ok()); // expect this to panic - same key, but stream limit of 3 } - #[tokio::test] + #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn test_pool_handles_many_conns_per_key() { let (server_drain_signal, server_drain) = drain::channel(); @@ -826,9 +826,9 @@ mod test { assert!(client2.is_ok()); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn test_pool_100_clients_streamexhaust() { - // crate::telemetry::setup_logging(); + crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); @@ -872,15 +872,15 @@ mod test { continue; } - drop(pool); server_drain_signal.drain().await; server_handle.await.unwrap(); + drop(pool); let real_conncount = conn_counter.load(Ordering::Relaxed); assert!(real_conncount == 4, "actual conncount was {real_conncount}"); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn test_pool_100_clients_singleconn() { // crate::telemetry::setup_logging(); @@ -935,7 +935,7 @@ mod test { assert!(real_conncount == 1, "actual conncount was {real_conncount}"); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn test_pool_100_clients_100_srcs() { // crate::telemetry::setup_logging(); @@ -997,7 +997,7 @@ mod test { ); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn test_pool_1000_clients_3_srcs() { // crate::telemetry::setup_logging(); @@ -1356,8 +1356,7 @@ mod test { Ok(upgraded) => { let (mut ri, mut wi) = tokio::io::split(hyper_util::rt::TokioIo::new(upgraded)); - // Signal we are the waypoint so tests can validate this - wi.write_all(b"waypoint\n").await.unwrap(); + wi.write_all(b"hbone\n").await.unwrap(); tcp::handle_stream(tcp::Mode::ReadWrite, &mut ri, &mut wi).await; } Err(e) => panic!("No upgrade {e}"), From bdb1c86d96e4e940ca38234b01ba55def23f400e Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Thu, 25 Apr 2024 16:26:17 -0400 Subject: [PATCH 39/46] Fix none race Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index c2a429411..6af4064a0 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -211,6 +211,7 @@ impl PoolState { } } + async fn start_conn_if_win_writelock( &self, workload_key: &WorkloadKey, @@ -324,7 +325,18 @@ impl PoolState { Some(e_conn) } } - None => None, + None => { + trace!("checkout - no existing conn for key {:#?}, adding one", workload_key); + let pool_conn = self.spawner.new_pool_conn(workload_key.clone()).await; + let r_conn = ConnClient { + sender: pool_conn?, + stream_count: Arc::new(AtomicU16::new(0)), + stream_count_max: self.max_streamcount, + wl_key: workload_key.clone(), + }; + self.checkin_conn(r_conn.clone(), pool_key.clone()); + Some(r_conn) + }, }; Ok(result) @@ -427,9 +439,7 @@ impl WorkloadHBONEPool { return Ok(existing_conn.unwrap()); } - // We couldn't get a conn. This means either nobody has tried to establish any conns for this key yet, - // or they have, but no conns are currently available - // (because someone else has checked all of them out and not put any back yet) + // We couldn't get a conn. This means nobody has tried to establish any conns for this key yet, // // So, we will take a nonexclusive readlock on the lockmap, to see if an inner lock // exists for our key. @@ -828,7 +838,7 @@ mod test { #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn test_pool_100_clients_streamexhaust() { - crate::telemetry::setup_logging(); + // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); From bd2ca75e85b2df2d09f69a657e2d5cb1a16ac663 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Thu, 25 Apr 2024 16:31:10 -0400 Subject: [PATCH 40/46] Propagate connection establish errors Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 32 +++++++++++++++++++++++--------- 1 file changed, 23 insertions(+), 9 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 6af4064a0..a842f0750 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -101,15 +101,19 @@ impl ConnSpawner { .then_some(key.src); let cert = self.cert_manager.fetch_certificate(&key.src_id).await?; let connector = cert.outbound_connector(key.dst_id)?; + debug!("got connector for key"); let tcp_stream = super::freebind_connect(local, key.dst, self.socket_factory.as_ref()).await?; + debug!("got stream for key"); tcp_stream.set_nodelay(true)?; // TODO: this is backwards of expectations let tls_stream = connector.connect(tcp_stream).await?; + debug!("connector connected, handshaking"); let (request_sender, connection) = builder .handshake(::hyper_util::rt::TokioIo::new(tls_stream)) .await .map_err(Error::HttpHandshake)?; + debug!("got TLS stream for key"); // spawn a task to poll the connection and drive the HTTP state // if we got a drain for that connection, respect it in a race // it is important to have a drain here, or this connection will never terminate @@ -216,7 +220,7 @@ impl PoolState { &self, workload_key: &WorkloadKey, pool_key: &pingora_pool::ConnectionMeta, - ) -> Option { + ) -> Result, Error> { let inner_conn_lock = { trace!("getting keyed lock out of lockmap"); let guard = self.established_conn_writelock.guard(); @@ -250,11 +254,11 @@ impl PoolState { workload_key, pool_key ); self.checkin_conn(client.clone(), pool_key.clone()); - Some(client) + Ok(Some(client)) } Err(e) => { error!("could not spawn new conn, got {e}"); - None + return Err(e) } } // END take inner writelock @@ -264,7 +268,7 @@ impl PoolState { "did not win connlock for wl key {:#?}, something else has it", workload_key ); - None + Ok(None) } } } @@ -489,7 +493,7 @@ impl WorkloadHBONEPool { let res = match self .state .start_conn_if_win_writelock(&workload_key, &pool_key) - .await + .await? { Some(client) => client, None => { @@ -639,6 +643,7 @@ mod test { use tokio::net::TcpListener; use tokio::task::{self}; use tokio::time::sleep; + use http_body_util::BodyExt; #[cfg(tokio_unstable)] use tracing::Instrument; @@ -886,8 +891,8 @@ mod test { server_handle.await.unwrap(); drop(pool); - let real_conncount = conn_counter.load(Ordering::Relaxed); - assert!(real_conncount == 4, "actual conncount was {real_conncount}"); + let real_conncount = conn_counter.load(Ordering::SeqCst); + assert!(real_conncount == 2, "actual conncount was {real_conncount}"); } #[tokio::test(flavor = "multi_thread", worker_threads = 2)] @@ -1274,7 +1279,7 @@ mod test { // needs tokio_unstable, but useful // .instrument(tracing::debug_span!("client_tid", tid=%tokio::task::id())) .await - .unwrap(); + .expect("connect should succeed"); debug!( "client spent {}ms waiting for conn", start.elapsed().as_millis() @@ -1287,10 +1292,18 @@ mod test { if res.is_err() { panic!("SEND ERR: {:#?} sendcount {count}", res); - } else if res.unwrap().status() != 200 { + } else if !res.is_ok() { panic!("CLIENT RETURNED ERROR") } + let mut okres = res.unwrap(); + const HBONE_MESSAGE: &[u8] = b"hbone\n"; + while let Some(next) = okres.frame().await { + let frame = next.expect("better have a resp body"); + if let Some(chunk) = frame.data_ref() { + assert_eq!(HBONE_MESSAGE, chunk); + } + } if count >= req_count { debug!("CLIENT DONE"); break; @@ -1366,6 +1379,7 @@ mod test { Ok(upgraded) => { let (mut ri, mut wi) = tokio::io::split(hyper_util::rt::TokioIo::new(upgraded)); + // wi.write_all(b"hbone\n").await.unwrap(); wi.write_all(b"hbone\n").await.unwrap(); tcp::handle_stream(tcp::Mode::ReadWrite, &mut ri, &mut wi).await; } From fb872fd2eda5fc8da39c93452aaadc3a85258ae4 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Thu, 25 Apr 2024 16:44:29 -0400 Subject: [PATCH 41/46] Cleanup Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 58 ++++++++++++++++++++++------------------------- 1 file changed, 27 insertions(+), 31 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index a842f0750..8311abc71 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -215,7 +215,6 @@ impl PoolState { } } - async fn start_conn_if_win_writelock( &self, workload_key: &WorkloadKey, @@ -240,27 +239,20 @@ impl PoolState { Ok(_guard) => { // BEGIN take inner writelock debug!("nothing else is creating a conn and we won the lock, make one"); - match self.spawner.new_pool_conn(workload_key.clone()).await { - Ok(pool_conn) => { - let client = ConnClient { - sender: pool_conn, - stream_count: Arc::new(AtomicU16::new(0)), - stream_count_max: self.max_streamcount, - wl_key: workload_key.clone(), - }; + let pool_conn = self.spawner.new_pool_conn(workload_key.clone()).await?; + let client = ConnClient { + sender: pool_conn, + stream_count: Arc::new(AtomicU16::new(0)), + stream_count_max: self.max_streamcount, + wl_key: workload_key.clone(), + }; - debug!( - "checking in new conn for key {:#?} with pk {:#?}", - workload_key, pool_key - ); - self.checkin_conn(client.clone(), pool_key.clone()); - Ok(Some(client)) - } - Err(e) => { - error!("could not spawn new conn, got {e}"); - return Err(e) - } - } + debug!( + "checking in new conn for key {:#?} with pk {:#?}", + workload_key, pool_key + ); + self.checkin_conn(client.clone(), pool_key.clone()); + Ok(Some(client)) // END take inner writelock } Err(_) => { @@ -314,9 +306,10 @@ impl PoolState { trace!("checkout - got existing conn for key {:#?}", workload_key); if e_conn.at_max_streamcount() { debug!("got conn for wl key {:#?}, but streamcount is maxed, spawning new conn to replace using pool key {:#?}", workload_key, pool_key); - let pool_conn = self.spawner.new_pool_conn(workload_key.clone()).await; + let pool_conn = + self.spawner.new_pool_conn(workload_key.clone()).await?; let r_conn = ConnClient { - sender: pool_conn?, + sender: pool_conn, stream_count: Arc::new(AtomicU16::new(0)), stream_count_max: self.max_streamcount, wl_key: workload_key.clone(), @@ -330,17 +323,20 @@ impl PoolState { } } None => { - trace!("checkout - no existing conn for key {:#?}, adding one", workload_key); - let pool_conn = self.spawner.new_pool_conn(workload_key.clone()).await; + trace!( + "checkout - no existing conn for key {:#?}, adding one", + workload_key + ); + let pool_conn = self.spawner.new_pool_conn(workload_key.clone()).await?; let r_conn = ConnClient { - sender: pool_conn?, + sender: pool_conn, stream_count: Arc::new(AtomicU16::new(0)), stream_count_max: self.max_streamcount, wl_key: workload_key.clone(), }; self.checkin_conn(r_conn.clone(), pool_key.clone()); Some(r_conn) - }, + } }; Ok(result) @@ -635,6 +631,7 @@ mod test { use futures_util::StreamExt; use hyper::body::Incoming; + use http_body_util::BodyExt; use hyper::service::service_fn; use hyper::{Request, Response}; use std::sync::atomic::AtomicU32; @@ -643,7 +640,6 @@ mod test { use tokio::net::TcpListener; use tokio::task::{self}; use tokio::time::sleep; - use http_body_util::BodyExt; #[cfg(tokio_unstable)] use tracing::Instrument; @@ -843,7 +839,7 @@ mod test { #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn test_pool_100_clients_streamexhaust() { - // crate::telemetry::setup_logging(); + crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); @@ -876,7 +872,7 @@ mod test { let mut tasks = futures::stream::FuturesUnordered::new(); loop { count += 1; - tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 100)); + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 1)); if count == client_count { break; @@ -1380,7 +1376,7 @@ mod test { let (mut ri, mut wi) = tokio::io::split(hyper_util::rt::TokioIo::new(upgraded)); // wi.write_all(b"hbone\n").await.unwrap(); - wi.write_all(b"hbone\n").await.unwrap(); + wi.write_all(b"hbone\n").await; tcp::handle_stream(tcp::Mode::ReadWrite, &mut ri, &mut wi).await; } Err(e) => panic!("No upgrade {e}"), From 3105377b645e6c43d4a509ac13cdb89e8663c2ef Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Thu, 25 Apr 2024 20:17:04 -0400 Subject: [PATCH 42/46] Work around local test server getting overloaded Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 43 +++++++++++++++++++++++++++++-------------- 1 file changed, 29 insertions(+), 14 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 8311abc71..6575e2955 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -839,7 +839,7 @@ mod test { #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn test_pool_100_clients_streamexhaust() { - crate::telemetry::setup_logging(); + // crate::telemetry::setup_logging(); let (server_drain_signal, server_drain) = drain::channel(); @@ -854,7 +854,7 @@ mod test { let cfg = crate::config::Config { local_node: Some("local-node".to_string()), - pool_max_streams_per_conn: 50, + pool_max_streams_per_conn: 25, ..crate::config::parse_config().unwrap() }; let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); @@ -867,17 +867,33 @@ mod test { src: IpAddr::from([127, 0, 0, 2]), dst: server_addr, }; - let client_count = 100; + let client_count = 50; let mut count = 0u32; let mut tasks = futures::stream::FuturesUnordered::new(); loop { count += 1; tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 1)); - if count == client_count { break; } } + + // TODO we spawn clients too fast (and they have little to do) and they actually break the + // local "fake" test server, causing it to start returning "conn refused/peer refused the connection" + // when the pool tries to create new connections for that caller + // + // (the pool will just pass that conn refused back to the caller) + // + // In the real world this is fine, since we aren't hitting a local server, + // servers can refuse connections - in synthetic tests it leads to flakes. + // + // It is worth considering if the pool should throttle how frequently it allows itself to create + // connections to real upstreams (e.g. "I created a conn for this key 10ms ago and you've already burned through + // your streamcount, chill out, you're gonna overload the dest") + // + // For now, streamcount is an inexact flow control for this. + sleep(Duration::from_millis(500)).await; + while let Some(Err(res)) = tasks.next().await { assert!(!res.is_panic(), "CLIENT PANICKED!"); continue; @@ -926,7 +942,7 @@ mod test { let mut tasks = futures::stream::FuturesUnordered::new(); loop { count += 1; - tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 100)); + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 1)); if count == client_count { break; @@ -982,9 +998,8 @@ mod test { src: IpAddr::from([127, 0, 0, count]), dst: server_addr, }; - // key1.src = IpAddr::from([127, 0, 0, count]); - tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 100)); + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 20)); if count == client_count { break; @@ -1055,7 +1070,7 @@ mod test { key1.src = IpAddr::from([127, 0, 0, 2]); } - tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 100)); + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 50)); if count == client_count { break; @@ -1093,7 +1108,7 @@ mod test { let cfg = crate::config::Config { local_node: Some("local-node".to_string()), pool_max_streams_per_conn: 1000, - pool_unused_release_timeout: Duration::from_secs(2), + pool_unused_release_timeout: Duration::from_secs(1), ..crate::config::parse_config().unwrap() }; let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); @@ -1123,7 +1138,7 @@ mod test { key1.src = IpAddr::from([127, 0, 0, 2]); } - tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 100)); + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 50)); if count == client_count { break; @@ -1146,7 +1161,7 @@ mod test { ); // Attempt to wait long enough for pool conns to timeout+evict - sleep(Duration::from_secs(2)).await; + sleep(Duration::from_secs(1)).await; let real_conncount = conn_counter.load(Ordering::Relaxed); let real_dropcount = conn_drop_counter.load(Ordering::Relaxed); @@ -1194,7 +1209,7 @@ mod test { let mut tasks = futures::stream::FuturesUnordered::new(); loop { count += 1; - tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 100)); + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 50)); if count == client_count { break; @@ -1304,6 +1319,7 @@ mod test { debug!("CLIENT DONE"); break; } + } }) } @@ -1375,8 +1391,7 @@ mod test { Ok(upgraded) => { let (mut ri, mut wi) = tokio::io::split(hyper_util::rt::TokioIo::new(upgraded)); - // wi.write_all(b"hbone\n").await.unwrap(); - wi.write_all(b"hbone\n").await; + wi.write_all(b"hbone\n").await.unwrap(); tcp::handle_stream(tcp::Mode::ReadWrite, &mut ri, &mut wi).await; } Err(e) => panic!("No upgrade {e}"), From e713b17282968e527d037dfee85e4133d6b2a36f Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Thu, 25 Apr 2024 20:25:54 -0400 Subject: [PATCH 43/46] Move the rest to multi_thread, chill out on iterations, work around test rig flakes Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 6575e2955..b57b7e41a 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -1090,7 +1090,7 @@ mod test { assert!(real_conncount == 3, "actual conncount was {real_conncount}"); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn test_pool_1000_clients_3_srcs_drops_after_timeout() { // crate::telemetry::setup_logging(); @@ -1173,7 +1173,7 @@ mod test { drop(pool); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn test_pool_100_clients_evicts_but_does_not_close_active_conn() { // crate::telemetry::setup_logging(); @@ -1209,7 +1209,7 @@ mod test { let mut tasks = futures::stream::FuturesUnordered::new(); loop { count += 1; - tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 50)); + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 25)); if count == client_count { break; @@ -1229,7 +1229,7 @@ mod test { let before_conncount = conn_counter.load(Ordering::Relaxed); let before_dropcount = conn_drop_counter.load(Ordering::Relaxed); assert!( - before_conncount == 4, + before_conncount == 3, "actual before conncount was {before_conncount}" ); assert!( @@ -1241,11 +1241,11 @@ mod test { sleep(Duration::from_secs(2)).await; let real_conncount = conn_counter.load(Ordering::Relaxed); - assert!(real_conncount == 4, "actual conncount was {real_conncount}"); + assert!(real_conncount == 3, "actual conncount was {real_conncount}"); // At this point, we should still have one conn that hasn't been dropped // because we haven't ended the persistent client let real_dropcount = conn_drop_counter.load(Ordering::Relaxed); - assert!(real_dropcount == 3, "actual dropcount was {real_dropcount}"); + assert!(real_dropcount == 2, "actual dropcount was {real_dropcount}"); client_stop_signal.drain().await; assert!(persist_res.await.is_ok(), "PERSIST CLIENT ERROR"); @@ -1253,12 +1253,12 @@ mod test { let after_conncount = conn_counter.load(Ordering::Relaxed); assert!( - after_conncount == 4, + after_conncount == 3, "after conncount was {after_conncount}" ); let after_dropcount = conn_drop_counter.load(Ordering::Relaxed); assert!( - after_dropcount == 4, + after_dropcount == 3, "after dropcount was {after_dropcount}" ); server_drain_signal.drain().await; From 4495fc28aeaa11a21b4c65e67a30345d2e88fb5a Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Thu, 25 Apr 2024 20:41:56 -0400 Subject: [PATCH 44/46] Tidy comments Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 68 ++++++++++++++++++++++++++--------------------- 1 file changed, 38 insertions(+), 30 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index b57b7e41a..25bad8e2f 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -101,19 +101,16 @@ impl ConnSpawner { .then_some(key.src); let cert = self.cert_manager.fetch_certificate(&key.src_id).await?; let connector = cert.outbound_connector(key.dst_id)?; - debug!("got connector for key"); let tcp_stream = super::freebind_connect(local, key.dst, self.socket_factory.as_ref()).await?; - debug!("got stream for key"); tcp_stream.set_nodelay(true)?; // TODO: this is backwards of expectations let tls_stream = connector.connect(tcp_stream).await?; - debug!("connector connected, handshaking"); + trace!("connector connected, handshaking"); let (request_sender, connection) = builder .handshake(::hyper_util::rt::TokioIo::new(tls_stream)) .await .map_err(Error::HttpHandshake)?; - debug!("got TLS stream for key"); // spawn a task to poll the connection and drive the HTTP state // if we got a drain for that connection, respect it in a race // it is important to have a drain here, or this connection will never terminate @@ -200,7 +197,7 @@ impl PoolState { } } - // Just for safety's sake, since we are using a hash thanks to pingora supporting arbitrary Eq, Hash + // Just for safety's sake, since we are using a hash thanks to pingora NOT supporting arbitrary Eq, Hash // types, do a deep equality test before returning the conn, returning an error if the conn's key does // not equal the provided key // @@ -215,6 +212,23 @@ impl PoolState { } } + // 1. Tries to get a writelock. + // 2. If successful, hold it, spawn a new connection, check it in, return a clone of it. + // 3. If not successful, return nothing. + // + // This is useful if we want to race someone else to the writelock to spawn a connection, + // and expect the losers to queue up and wait for the (singular) winner of the writelock + // + // This function should ALWAYS return a connection if it wins the writelock for the provided key. + // This function should NEVER return a connection if it does not win the writelock for the provided key. + // + // It is important that the *initial* check here is authoritative, hence the locks, as + // we must know if this is a connection for a key *nobody* has tried to start yet + // (i.e. no writelock for our key in the outer map) + // or if other things have already established conns for this key (writelock for our key in the outer map). + // + // This is so we can backpressure correctly if 1000 tasks all demand a new connection + // to the same key at once, and not eagerly open 1000 tunnel connections. async fn start_conn_if_win_writelock( &self, workload_key: &WorkloadKey, @@ -265,16 +279,20 @@ impl PoolState { } } - // Does an initial, naive check to see if a conn exists for this key. + // Does an initial, naive check to see if we have a writelock inserted into the map for this key // - // If it does, WRITELOCK the mutex for that key, clone (or create), check in the clone, - // and return the other reference for use. + // If we do, take the writelock for that key, clone (or create) a connection, check it back in, + // and return a cloned ref, then drop the writelock. // // Otherwise, return None. // + // This function should ALWAYS return a connection if a writelock exists for the provided key. + // This function should NEVER return a connection if no writelock exists for the provided key. + // // It is important that the *initial* check here is authoritative, hence the locks, as - // we must know if this is a connection for a key *nobody* has tried to start yet, - // or if other things have already established conns for this key. + // we must know if this is a connection for a key *nobody* has tried to start yet + // (i.e. no writelock for our key in the outer map) + // or if other things have already established conns for this key (writelock for our key in the outer map). // // This is so we can backpressure correctly if 1000 tasks all demand a new connection // to the same key at once, and not eagerly open 1000 tunnel connections. @@ -407,7 +425,7 @@ impl WorkloadHBONEPool { // one connection is created, before deciding if they should create more or just use that one. pub async fn connect(&mut self, workload_key: WorkloadKey) -> Result { trace!("pool connect START"); - // TODO BML this may not be collision resistant/slow. It should be resistant enough for workloads tho. + // TODO BML this may not be collision resistant, or a fast hash. It should be resistant enough for workloads tho. // We are doing a deep-equals check at the end to mitigate any collisions, will see about bumping Pingora let mut s = DefaultHasher::new(); workload_key.hash(&mut s); @@ -418,16 +436,9 @@ impl WorkloadHBONEPool { .pool_global_conn_count .fetch_add(1, Ordering::SeqCst), ); - // First, see if we can naively just check out a connection. + // First, see if we can naively take an inner lock for our specific key, and get a connection. // This should be the common case, except for the first establishment of a new connection/key. // This will be done under outer readlock (nonexclusive)/inner keyed writelock (exclusive). - // - // It is important that the *initial* check here is authoritative, hence the locks, as - // we must know if this is a connection for a key *nobody* has tried to start yet, - // or if other things have already established conns for this key. - // - // This is so we can backpressure correctly if 1000 tasks all demand a new connection - // to the same key at once, and not eagerly open 1000 tunnel connections. let existing_conn = self .state .checkout_conn_under_writelock(&workload_key, &pool_key) @@ -439,12 +450,10 @@ impl WorkloadHBONEPool { return Ok(existing_conn.unwrap()); } - // We couldn't get a conn. This means nobody has tried to establish any conns for this key yet, - // - // So, we will take a nonexclusive readlock on the lockmap, to see if an inner lock - // exists for our key. + // We couldn't get a writelock for this key. This means nobody has tried to establish any conns for this key yet, + // So, we will take a nonexclusive readlock on the outer lockmap, and attempt to insert one. // - // If not, we insert one. + // (if multiple threads try to insert one, only one will succeed.) { debug!( "didn't find a connection for key {:#?}, making sure lockmap has entry", @@ -466,7 +475,7 @@ impl WorkloadHBONEPool { } // If we get here, it means the following are true: - // 1. We have a guaranteed sharded mutex in the outer map for our current key. + // 1. We have a guaranteed sharded mutex in the outer map for our current key // 2. We can now, under readlock(nonexclusive) in the outer map, attempt to // take the inner writelock for our specific key (exclusive). // @@ -480,7 +489,7 @@ impl WorkloadHBONEPool { // because holding a lock over an await is a great way to create deadlocks if the await you // hold it over does not resolve. // - // HOWEVER. Here we know this connection will either establish or timeout + // HOWEVER. Here we know this connection will either establish or timeout (or fail with error) // and we WANT other tasks to go back to sleep if a task is already trying to create a new connection for this key. // // So the downsides are actually useful (we WANT task contention - @@ -495,10 +504,9 @@ impl WorkloadHBONEPool { None => { debug!("we didn't win the lock, something else is creating a conn, wait for it"); // If we get here, it means the following are true: - // 1. At one point, there was a preexisting conn in the pool for this key. - // 2. When we checked, we got nothing for that key. - // 3. We could not get the exclusive inner writelock to add a new one for this key. - // 4. Someone else got the exclusive inner writelock, and is adding a new one for this key. + // 1. We have a writelock in the outer map for this key (either we inserted, or someone beat us to it - but it's there) + // 2. We could not get the exclusive inner writelock to add a new conn for this key. + // 3. Someone else got the exclusive inner writelock, and is adding a new conn for this key. // // So, loop and wait for the pool_watcher to tell us a new conn was enpooled, // so we can pull it out and check it. From 7e794b608b005c5a867f923f5a37a12ac456abf1 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Thu, 25 Apr 2024 21:26:26 -0400 Subject: [PATCH 45/46] lints Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 88 +++++++++++++++++++++++------------------------ 1 file changed, 44 insertions(+), 44 deletions(-) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 25bad8e2f..7e6813520 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -311,7 +311,10 @@ impl PoolState { }; match found_conn { Some(exist_conn_lock) => { - debug!("checkout - found mutex for key, waiting for writelock"); + debug!( + "checkout - found mutex for pool key {:#?}, waiting for writelock", + pool_key + ); let _conn_lock = exist_conn_lock.as_ref().lock().await; trace!( @@ -639,7 +642,6 @@ mod test { use futures_util::StreamExt; use hyper::body::Incoming; - use http_body_util::BodyExt; use hyper::service::service_fn; use hyper::{Request, Response}; use std::sync::atomic::AtomicU32; @@ -1164,7 +1166,7 @@ mod test { "actual before conncount was {before_conncount}" ); assert!( - before_dropcount == 0, + before_dropcount != 3, "actual before dropcount was {before_dropcount}" ); @@ -1199,7 +1201,7 @@ mod test { let cfg = crate::config::Config { local_node: Some("local-node".to_string()), pool_max_streams_per_conn: 50, - pool_unused_release_timeout: Duration::from_secs(2), + pool_unused_release_timeout: Duration::from_secs(1), ..crate::config::parse_config().unwrap() }; let sock_fact = Arc::new(crate::proxy::DefaultSocketFactory); @@ -1217,47 +1219,58 @@ mod test { let mut tasks = futures::stream::FuturesUnordered::new(); loop { count += 1; - tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 25)); + tasks.push(spawn_client(pool.clone(), key1.clone(), server_addr, 1)); if count == client_count { break; } } - let (client_stop_signal, client_stop) = drain::channel(); - let persist_res = - spawn_persistent_client(pool.clone(), key1.clone(), server_addr, client_stop); - + // TODO we spawn clients too fast (and they have little to do) and they actually break the + // local "fake" test server, causing it to start returning "conn refused/peer refused the connection" + // when the pool tries to create new connections for that caller + // + // (the pool will just pass that conn refused back to the caller) + // + // In the real world this is fine, since we aren't hitting a local server, + // servers can refuse connections - in synthetic tests it leads to flakes. + // + // It is worth considering if the pool should throttle how frequently it allows itself to create + // connections to real upstreams (e.g. "I created a conn for this key 10ms ago and you've already burned through + // your streamcount, chill out, you're gonna overload the dest") + // + // For now, streamcount is an inexact flow control for this. + sleep(Duration::from_millis(500)).await; //loop thru the nonpersistent clients and wait for them to finish while let Some(Err(res)) = tasks.next().await { assert!(!res.is_panic(), "CLIENT PANICKED!"); continue; } + let (client_stop_signal, client_stop) = drain::channel(); + let persist_res = + spawn_persistent_client(pool.clone(), key1.clone(), server_addr, client_stop); + + //Attempt to wait a bit more, to ensure the connections NOT held open by our persistent client are dropped. + sleep(Duration::from_secs(1)).await; let before_conncount = conn_counter.load(Ordering::Relaxed); let before_dropcount = conn_drop_counter.load(Ordering::Relaxed); assert!( before_conncount == 3, "actual before conncount was {before_conncount}" ); + // At this point, we should still have one conn that hasn't been dropped + // because we haven't ended the persistent client assert!( - before_dropcount == 0, + before_dropcount == 2, "actual before dropcount was {before_dropcount}" ); - // Attempt to wait long enough for pool conns to timeout+evict - sleep(Duration::from_secs(2)).await; - - let real_conncount = conn_counter.load(Ordering::Relaxed); - assert!(real_conncount == 3, "actual conncount was {real_conncount}"); - // At this point, we should still have one conn that hasn't been dropped - // because we haven't ended the persistent client - let real_dropcount = conn_drop_counter.load(Ordering::Relaxed); - assert!(real_dropcount == 2, "actual dropcount was {real_dropcount}"); client_stop_signal.drain().await; assert!(persist_res.await.is_ok(), "PERSIST CLIENT ERROR"); - sleep(Duration::from_secs(2)).await; + //Attempt to wait a bit more, to ensure the connections held open by our persistent client is dropped. + sleep(Duration::from_secs(1)).await; let after_conncount = conn_counter.load(Ordering::Relaxed); assert!( @@ -1311,23 +1324,12 @@ mod test { if res.is_err() { panic!("SEND ERR: {:#?} sendcount {count}", res); - } else if !res.is_ok() { - panic!("CLIENT RETURNED ERROR") } - let mut okres = res.unwrap(); - const HBONE_MESSAGE: &[u8] = b"hbone\n"; - while let Some(next) = okres.frame().await { - let frame = next.expect("better have a resp body"); - if let Some(chunk) = frame.data_ref() { - assert_eq!(HBONE_MESSAGE, chunk); - } - } if count >= req_count { debug!("CLIENT DONE"); break; } - } }) } @@ -1361,17 +1363,16 @@ mod test { start.elapsed().as_millis() ); - let mut count = 0u32; - // send forever, once we get a conn, until someone signals us to stop let send_loop = async move { + //send once, then hold the conn open until signaled + let res = c1.send_request(req()).await; + if res.is_err() { + panic!("SEND ERR: {:#?}", res); + } loop { - count += 1; - let res = c1.send_request(req()).await; - if res.is_err() { - panic!("SEND ERR: {:#?} sendcount {count}", res); - } else if res.unwrap().status() != 200 { - panic!("CLIENT RETURNED ERROR") - } + debug!("persistent client yielding"); + sleep(Duration::from_millis(1)).await; //yield may be enough + tokio::task::yield_now().await; } }; @@ -1397,10 +1398,9 @@ mod test { tokio::task::spawn(async move { match hyper::upgrade::on(req).await { Ok(upgraded) => { - let (mut ri, mut wi) = - tokio::io::split(hyper_util::rt::TokioIo::new(upgraded)); - wi.write_all(b"hbone\n").await.unwrap(); - tcp::handle_stream(tcp::Mode::ReadWrite, &mut ri, &mut wi).await; + let mut io = hyper_util::rt::TokioIo::new(upgraded); + io.write_all(b"poolsrv\n").await.unwrap(); + tcp::handle_stream(tcp::Mode::ReadWrite, &mut io).await; } Err(e) => panic!("No upgrade {e}"), } From 8737cd762201ea32c9187bd1094ebac2bb513f87 Mon Sep 17 00:00:00 2001 From: Benjamin Leggett Date: Fri, 26 Apr 2024 10:44:54 -0400 Subject: [PATCH 46/46] Clarify comment Signed-off-by: Benjamin Leggett --- src/proxy/pool.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/proxy/pool.rs b/src/proxy/pool.rs index 7e6813520..502a479ae 100644 --- a/src/proxy/pool.rs +++ b/src/proxy/pool.rs @@ -221,6 +221,7 @@ impl PoolState { // // This function should ALWAYS return a connection if it wins the writelock for the provided key. // This function should NEVER return a connection if it does not win the writelock for the provided key. + // This function should ALWAYS propagate Error results to the caller // // It is important that the *initial* check here is authoritative, hence the locks, as // we must know if this is a connection for a key *nobody* has tried to start yet @@ -288,6 +289,7 @@ impl PoolState { // // This function should ALWAYS return a connection if a writelock exists for the provided key. // This function should NEVER return a connection if no writelock exists for the provided key. + // This function should ALWAYS propagate Error results to the caller // // It is important that the *initial* check here is authoritative, hence the locks, as // we must know if this is a connection for a key *nobody* has tried to start yet