mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-17 02:12:56 +00:00
proxy: hyper1 for only proxy (#7073)
## Problem
hyper1 offers control over the HTTP connection that hyper0_14 does not.
We're blocked on switching all services to hyper1 because of how we use
tonic, but no reason we can't switch proxy over.
## Summary of changes
1. hyper0.14 -> hyper1
1. self managed server
2. Remove the `WithConnectionGuard` wrapper from `protocol2`
2. Remove TLS listener as it's no longer necessary
3. include first session ID in connection startup logs
This commit is contained in:
@@ -4,42 +4,48 @@
|
||||
|
||||
mod backend;
|
||||
mod conn_pool;
|
||||
mod http_util;
|
||||
mod json;
|
||||
mod sql_over_http;
|
||||
pub mod tls_listener;
|
||||
mod websocket;
|
||||
|
||||
use atomic_take::AtomicTake;
|
||||
use bytes::Bytes;
|
||||
pub use conn_pool::GlobalConnPoolOptions;
|
||||
|
||||
use anyhow::bail;
|
||||
use hyper::StatusCode;
|
||||
use metrics::IntCounterPairGuard;
|
||||
use anyhow::Context;
|
||||
use futures::future::{select, Either};
|
||||
use futures::TryFutureExt;
|
||||
use http::{Method, Response, StatusCode};
|
||||
use http_body_util::Full;
|
||||
use hyper1::body::Incoming;
|
||||
use hyper_util::rt::TokioExecutor;
|
||||
use hyper_util::server::conn::auto::Builder;
|
||||
use rand::rngs::StdRng;
|
||||
use rand::SeedableRng;
|
||||
pub use reqwest_middleware::{ClientWithMiddleware, Error};
|
||||
pub use reqwest_retry::{policies::ExponentialBackoff, RetryTransientMiddleware};
|
||||
use tokio::time::timeout;
|
||||
use tokio_rustls::TlsAcceptor;
|
||||
use tokio_util::task::TaskTracker;
|
||||
use tracing::instrument::Instrumented;
|
||||
|
||||
use crate::cancellation::CancellationHandlerMain;
|
||||
use crate::config::ProxyConfig;
|
||||
use crate::context::RequestMonitoring;
|
||||
use crate::protocol2::{ProxyProtocolAccept, WithClientIp, WithConnectionGuard};
|
||||
use crate::metrics::{NUM_CLIENT_CONNECTION_GAUGE, TLS_HANDSHAKE_FAILURES};
|
||||
use crate::protocol2::WithClientIp;
|
||||
use crate::proxy::run_until_cancelled;
|
||||
use crate::rate_limiter::EndpointRateLimiter;
|
||||
use crate::serverless::backend::PoolingBackend;
|
||||
use hyper::{
|
||||
server::conn::{AddrIncoming, AddrStream},
|
||||
Body, Method, Request, Response,
|
||||
};
|
||||
use crate::serverless::http_util::{api_error_into_response, json_response};
|
||||
|
||||
use std::net::IpAddr;
|
||||
use std::net::{IpAddr, SocketAddr};
|
||||
use std::pin::pin;
|
||||
use std::sync::Arc;
|
||||
use std::task::Poll;
|
||||
use tls_listener::TlsListener;
|
||||
use tokio::net::TcpListener;
|
||||
use tokio_util::sync::{CancellationToken, DropGuard};
|
||||
use tokio::net::{TcpListener, TcpStream};
|
||||
use tokio_util::sync::CancellationToken;
|
||||
use tracing::{error, info, warn, Instrument};
|
||||
use utils::http::{error::ApiError, json::json_response};
|
||||
use utils::http::error::ApiError;
|
||||
|
||||
pub const SERVERLESS_DRIVER_SNI: &str = "api";
|
||||
|
||||
@@ -91,161 +97,174 @@ pub async fn task_main(
|
||||
tls_server_config.alpn_protocols = vec![b"h2".to_vec(), b"http/1.1".to_vec()];
|
||||
let tls_acceptor: tokio_rustls::TlsAcceptor = Arc::new(tls_server_config).into();
|
||||
|
||||
let mut addr_incoming = AddrIncoming::from_listener(ws_listener)?;
|
||||
let _ = addr_incoming.set_nodelay(true);
|
||||
let addr_incoming = ProxyProtocolAccept {
|
||||
incoming: addr_incoming,
|
||||
protocol: "http",
|
||||
};
|
||||
let connections = tokio_util::task::task_tracker::TaskTracker::new();
|
||||
connections.close(); // allows `connections.wait to complete`
|
||||
|
||||
let ws_connections = tokio_util::task::task_tracker::TaskTracker::new();
|
||||
ws_connections.close(); // allows `ws_connections.wait to complete`
|
||||
let server = Builder::new(hyper_util::rt::TokioExecutor::new());
|
||||
|
||||
let tls_listener = TlsListener::new(tls_acceptor, addr_incoming, config.handshake_timeout);
|
||||
while let Some(res) = run_until_cancelled(ws_listener.accept(), &cancellation_token).await {
|
||||
let (conn, peer_addr) = res.context("could not accept TCP stream")?;
|
||||
if let Err(e) = conn.set_nodelay(true) {
|
||||
tracing::error!("could not set nodelay: {e}");
|
||||
continue;
|
||||
}
|
||||
let conn_id = uuid::Uuid::new_v4();
|
||||
let http_conn_span = tracing::info_span!("http_conn", ?conn_id);
|
||||
|
||||
let make_svc = hyper::service::make_service_fn(
|
||||
|stream: &tokio_rustls::server::TlsStream<
|
||||
WithConnectionGuard<WithClientIp<AddrStream>>,
|
||||
>| {
|
||||
let (conn, _) = stream.get_ref();
|
||||
connections.spawn(
|
||||
connection_handler(
|
||||
config,
|
||||
backend.clone(),
|
||||
connections.clone(),
|
||||
cancellation_handler.clone(),
|
||||
endpoint_rate_limiter.clone(),
|
||||
cancellation_token.clone(),
|
||||
server.clone(),
|
||||
tls_acceptor.clone(),
|
||||
conn,
|
||||
peer_addr,
|
||||
)
|
||||
.instrument(http_conn_span),
|
||||
);
|
||||
}
|
||||
|
||||
// this is jank. should dissapear with hyper 1.0 migration.
|
||||
let gauge = conn
|
||||
.gauge
|
||||
.lock()
|
||||
.expect("lock should not be poisoned")
|
||||
.take()
|
||||
.expect("gauge should be set on connection start");
|
||||
|
||||
// Cancel all current inflight HTTP requests if the HTTP connection is closed.
|
||||
let http_cancellation_token = CancellationToken::new();
|
||||
let cancel_connection = http_cancellation_token.clone().drop_guard();
|
||||
|
||||
let span = conn.span.clone();
|
||||
let client_addr = conn.inner.client_addr();
|
||||
let remote_addr = conn.inner.inner.remote_addr();
|
||||
let backend = backend.clone();
|
||||
let ws_connections = ws_connections.clone();
|
||||
let endpoint_rate_limiter = endpoint_rate_limiter.clone();
|
||||
let cancellation_handler = cancellation_handler.clone();
|
||||
async move {
|
||||
let peer_addr = match client_addr {
|
||||
Some(addr) => addr,
|
||||
None if config.require_client_ip => bail!("missing required client ip"),
|
||||
None => remote_addr,
|
||||
};
|
||||
Ok(MetricService::new(
|
||||
hyper::service::service_fn(move |req: Request<Body>| {
|
||||
let backend = backend.clone();
|
||||
let ws_connections2 = ws_connections.clone();
|
||||
let endpoint_rate_limiter = endpoint_rate_limiter.clone();
|
||||
let cancellation_handler = cancellation_handler.clone();
|
||||
let http_cancellation_token = http_cancellation_token.child_token();
|
||||
|
||||
// `request_handler` is not cancel safe. It expects to be cancelled only at specific times.
|
||||
// By spawning the future, we ensure it never gets cancelled until it decides to.
|
||||
ws_connections.spawn(
|
||||
async move {
|
||||
// Cancel the current inflight HTTP request if the requets stream is closed.
|
||||
// This is slightly different to `_cancel_connection` in that
|
||||
// h2 can cancel individual requests with a `RST_STREAM`.
|
||||
let _cancel_session = http_cancellation_token.clone().drop_guard();
|
||||
|
||||
let res = request_handler(
|
||||
req,
|
||||
config,
|
||||
backend,
|
||||
ws_connections2,
|
||||
cancellation_handler,
|
||||
peer_addr.ip(),
|
||||
endpoint_rate_limiter,
|
||||
http_cancellation_token,
|
||||
)
|
||||
.await
|
||||
.map_or_else(|e| e.into_response(), |r| r);
|
||||
|
||||
_cancel_session.disarm();
|
||||
|
||||
res
|
||||
}
|
||||
.in_current_span(),
|
||||
)
|
||||
}),
|
||||
gauge,
|
||||
cancel_connection,
|
||||
span,
|
||||
))
|
||||
}
|
||||
},
|
||||
);
|
||||
|
||||
hyper::Server::builder(tls_listener)
|
||||
.serve(make_svc)
|
||||
.with_graceful_shutdown(cancellation_token.cancelled())
|
||||
.await?;
|
||||
|
||||
// await websocket connections
|
||||
ws_connections.wait().await;
|
||||
connections.wait().await;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
struct MetricService<S> {
|
||||
inner: S,
|
||||
_gauge: IntCounterPairGuard,
|
||||
_cancel: DropGuard,
|
||||
span: tracing::Span,
|
||||
}
|
||||
/// Handles the TCP lifecycle.
|
||||
///
|
||||
/// 1. Parses PROXY protocol V2
|
||||
/// 2. Handles TLS handshake
|
||||
/// 3. Handles HTTP connection
|
||||
/// 1. With graceful shutdowns
|
||||
/// 2. With graceful request cancellation with connection failure
|
||||
/// 3. With websocket upgrade support.
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
async fn connection_handler(
|
||||
config: &'static ProxyConfig,
|
||||
backend: Arc<PoolingBackend>,
|
||||
connections: TaskTracker,
|
||||
cancellation_handler: Arc<CancellationHandlerMain>,
|
||||
endpoint_rate_limiter: Arc<EndpointRateLimiter>,
|
||||
cancellation_token: CancellationToken,
|
||||
server: Builder<TokioExecutor>,
|
||||
tls_acceptor: TlsAcceptor,
|
||||
conn: TcpStream,
|
||||
peer_addr: SocketAddr,
|
||||
) {
|
||||
let session_id = uuid::Uuid::new_v4();
|
||||
|
||||
impl<S> MetricService<S> {
|
||||
fn new(
|
||||
inner: S,
|
||||
_gauge: IntCounterPairGuard,
|
||||
_cancel: DropGuard,
|
||||
span: tracing::Span,
|
||||
) -> MetricService<S> {
|
||||
MetricService {
|
||||
inner,
|
||||
_gauge,
|
||||
_cancel,
|
||||
span,
|
||||
let _gauge = NUM_CLIENT_CONNECTION_GAUGE
|
||||
.with_label_values(&["http"])
|
||||
.guard();
|
||||
|
||||
// handle PROXY protocol
|
||||
let mut conn = WithClientIp::new(conn);
|
||||
let peer = match conn.wait_for_addr().await {
|
||||
Ok(peer) => peer,
|
||||
Err(e) => {
|
||||
tracing::error!(?session_id, %peer_addr, "failed to accept TCP connection: invalid PROXY protocol V2 header: {e:#}");
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
impl<S, ReqBody> hyper::service::Service<Request<ReqBody>> for MetricService<S>
|
||||
where
|
||||
S: hyper::service::Service<Request<ReqBody>>,
|
||||
{
|
||||
type Response = S::Response;
|
||||
type Error = S::Error;
|
||||
type Future = Instrumented<S::Future>;
|
||||
let peer_addr = peer.unwrap_or(peer_addr).ip();
|
||||
info!(?session_id, %peer_addr, "accepted new TCP connection");
|
||||
|
||||
fn poll_ready(&mut self, cx: &mut std::task::Context<'_>) -> Poll<Result<(), Self::Error>> {
|
||||
self.inner.poll_ready(cx)
|
||||
}
|
||||
// try upgrade to TLS, but with a timeout.
|
||||
let conn = match timeout(config.handshake_timeout, tls_acceptor.accept(conn)).await {
|
||||
Ok(Ok(conn)) => {
|
||||
info!(?session_id, %peer_addr, "accepted new TLS connection");
|
||||
conn
|
||||
}
|
||||
// The handshake failed
|
||||
Ok(Err(e)) => {
|
||||
TLS_HANDSHAKE_FAILURES.inc();
|
||||
warn!(?session_id, %peer_addr, "failed to accept TLS connection: {e:?}");
|
||||
return;
|
||||
}
|
||||
// The handshake timed out
|
||||
Err(e) => {
|
||||
TLS_HANDSHAKE_FAILURES.inc();
|
||||
warn!(?session_id, %peer_addr, "failed to accept TLS connection: {e:?}");
|
||||
return;
|
||||
}
|
||||
};
|
||||
|
||||
fn call(&mut self, req: Request<ReqBody>) -> Self::Future {
|
||||
self.span
|
||||
.in_scope(|| self.inner.call(req))
|
||||
.instrument(self.span.clone())
|
||||
let session_id = AtomicTake::new(session_id);
|
||||
|
||||
// Cancel all current inflight HTTP requests if the HTTP connection is closed.
|
||||
let http_cancellation_token = CancellationToken::new();
|
||||
let _cancel_connection = http_cancellation_token.clone().drop_guard();
|
||||
|
||||
let conn = server.serve_connection_with_upgrades(
|
||||
hyper_util::rt::TokioIo::new(conn),
|
||||
hyper1::service::service_fn(move |req: hyper1::Request<Incoming>| {
|
||||
// First HTTP request shares the same session ID
|
||||
let session_id = session_id.take().unwrap_or_else(uuid::Uuid::new_v4);
|
||||
|
||||
// Cancel the current inflight HTTP request if the requets stream is closed.
|
||||
// This is slightly different to `_cancel_connection` in that
|
||||
// h2 can cancel individual requests with a `RST_STREAM`.
|
||||
let http_request_token = http_cancellation_token.child_token();
|
||||
let cancel_request = http_request_token.clone().drop_guard();
|
||||
|
||||
// `request_handler` is not cancel safe. It expects to be cancelled only at specific times.
|
||||
// By spawning the future, we ensure it never gets cancelled until it decides to.
|
||||
let handler = connections.spawn(
|
||||
request_handler(
|
||||
req,
|
||||
config,
|
||||
backend.clone(),
|
||||
connections.clone(),
|
||||
cancellation_handler.clone(),
|
||||
session_id,
|
||||
peer_addr,
|
||||
endpoint_rate_limiter.clone(),
|
||||
http_request_token,
|
||||
)
|
||||
.in_current_span()
|
||||
.map_ok_or_else(api_error_into_response, |r| r),
|
||||
);
|
||||
|
||||
async move {
|
||||
let res = handler.await;
|
||||
cancel_request.disarm();
|
||||
res
|
||||
}
|
||||
}),
|
||||
);
|
||||
|
||||
// On cancellation, trigger the HTTP connection handler to shut down.
|
||||
let res = match select(pin!(cancellation_token.cancelled()), pin!(conn)).await {
|
||||
Either::Left((_cancelled, mut conn)) => {
|
||||
conn.as_mut().graceful_shutdown();
|
||||
conn.await
|
||||
}
|
||||
Either::Right((res, _)) => res,
|
||||
};
|
||||
|
||||
match res {
|
||||
Ok(()) => tracing::info!(%peer_addr, "HTTP connection closed"),
|
||||
Err(e) => tracing::warn!(%peer_addr, "HTTP connection error {e}"),
|
||||
}
|
||||
}
|
||||
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
async fn request_handler(
|
||||
mut request: Request<Body>,
|
||||
mut request: hyper1::Request<Incoming>,
|
||||
config: &'static ProxyConfig,
|
||||
backend: Arc<PoolingBackend>,
|
||||
ws_connections: TaskTracker,
|
||||
cancellation_handler: Arc<CancellationHandlerMain>,
|
||||
session_id: uuid::Uuid,
|
||||
peer_addr: IpAddr,
|
||||
endpoint_rate_limiter: Arc<EndpointRateLimiter>,
|
||||
// used to cancel in-flight HTTP requests. not used to cancel websockets
|
||||
http_cancellation_token: CancellationToken,
|
||||
) -> Result<Response<Body>, ApiError> {
|
||||
let session_id = uuid::Uuid::new_v4();
|
||||
|
||||
) -> Result<Response<Full<Bytes>>, ApiError> {
|
||||
let host = request
|
||||
.headers()
|
||||
.get("host")
|
||||
@@ -282,14 +301,14 @@ async fn request_handler(
|
||||
|
||||
// Return the response so the spawned future can continue.
|
||||
Ok(response)
|
||||
} else if request.uri().path() == "/sql" && request.method() == Method::POST {
|
||||
} else if request.uri().path() == "/sql" && *request.method() == Method::POST {
|
||||
let ctx = RequestMonitoring::new(session_id, peer_addr, "http", &config.region);
|
||||
let span = ctx.span.clone();
|
||||
|
||||
sql_over_http::handle(config, ctx, request, backend, http_cancellation_token)
|
||||
.instrument(span)
|
||||
.await
|
||||
} else if request.uri().path() == "/sql" && request.method() == Method::OPTIONS {
|
||||
} else if request.uri().path() == "/sql" && *request.method() == Method::OPTIONS {
|
||||
Response::builder()
|
||||
.header("Allow", "OPTIONS, POST")
|
||||
.header("Access-Control-Allow-Origin", "*")
|
||||
@@ -299,7 +318,7 @@ async fn request_handler(
|
||||
)
|
||||
.header("Access-Control-Max-Age", "86400" /* 24 hours */)
|
||||
.status(StatusCode::OK) // 204 is also valid, but see: https://developer.mozilla.org/en-US/docs/Web/HTTP/Methods/OPTIONS#status_code
|
||||
.body(Body::empty())
|
||||
.body(Full::new(Bytes::new()))
|
||||
.map_err(|e| ApiError::InternalServerError(e.into()))
|
||||
} else {
|
||||
json_response(StatusCode::BAD_REQUEST, "query is not supported")
|
||||
|
||||
Reference in New Issue
Block a user