From caf8840d9ec474236ee6779d370c48ae2d5b2f2c Mon Sep 17 00:00:00 2001 From: Nikhar Saxena Date: Wed, 12 Nov 2025 14:22:48 -0800 Subject: [PATCH 1/7] feat(ingest): Implement first version of project config handler Implemented the basic version of relay project config handler which splits the project keys across multiple cells. Right now it does a round robin split. Will change to lookup into locator later. --- Cargo.lock | 2 + ingest-router/Cargo.toml | 1 + ingest-router/src/errors.rs | 15 +- ingest-router/src/lib.rs | 48 +- .../src/relay_project_config_handler.rs | 982 ++++++++++++++++++ ingest-router/src/router.rs | 88 +- shared/Cargo.toml | 1 + 7 files changed, 1081 insertions(+), 56 deletions(-) create mode 100644 ingest-router/src/relay_project_config_handler.rs diff --git a/Cargo.lock b/Cargo.lock index e03da3a..8591b8f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1082,6 +1082,7 @@ dependencies = [ "hyper", "hyper-util", "serde", + "serde_json", "serde_yaml", "shared", "thiserror 2.0.17", @@ -2040,6 +2041,7 @@ dependencies = [ "hyper-util", "tokio", "tracing", + "url", ] [[package]] diff --git a/ingest-router/Cargo.toml b/ingest-router/Cargo.toml index a47f115..abe2cec 100644 --- a/ingest-router/Cargo.toml +++ b/ingest-router/Cargo.toml @@ -9,6 +9,7 @@ http-body-util = { workspace = true } hyper = { workspace = true } hyper-util = { workspace = true } serde = { workspace = true } +serde_json = "1.0" shared = { path = "../shared" } thiserror = { workspace = true } tokio = { workspace = true } diff --git a/ingest-router/src/errors.rs b/ingest-router/src/errors.rs index 13b2a40..ad05775 100644 --- a/ingest-router/src/errors.rs +++ b/ingest-router/src/errors.rs @@ -9,26 +9,17 @@ pub enum IngestRouterError { #[error("Failed to read response body: {0}")] ResponseBodyError(String), - #[error("No route matched for request")] - NoRouteMatched, - - #[error("Upstream not found: {0}")] - UpstreamNotFound(String), - #[error("Upstream request failed for {0}: {1}")] UpstreamRequestFailed(String, String), #[error("Upstream timeout for {0}")] UpstreamTimeout(String), - #[error("Response serialization error: {0}")] + #[error("Failed to serialize response: {0}")] ResponseSerializationError(String), - #[error("Hyper error: {0}")] - HyperError(String), - - #[error("HTTP client error: {0}")] - HttpClientError(String), + #[error("Failed to build response: {0}")] + ResponseBuildError(String), #[error("Internal error: {0}")] InternalError(String), diff --git a/ingest-router/src/lib.rs b/ingest-router/src/lib.rs index 84fb094..bf9b8af 100644 --- a/ingest-router/src/lib.rs +++ b/ingest-router/src/lib.rs @@ -2,43 +2,57 @@ pub mod config; pub mod errors; pub mod http; pub mod locale; +pub mod relay_project_config_handler; pub mod router; -use http_body_util::{BodyExt, Full, combinators::BoxBody}; +use http_body_util::combinators::BoxBody; use hyper::body::Bytes; use hyper::body::Incoming; use hyper::service::Service; use hyper::{Request, Response}; use shared::http::run_http_service; +use std::collections::HashMap; use std::pin::Pin; +use crate::config::{Config, CellConfig, Route}; +use crate::router::Router; +use crate::relay_project_config_handler::RelayProjectConfigsHandler; +use crate::errors::IngestRouterError; -#[derive(thiserror::Error, Debug)] -pub enum IngestRouterError { - #[error("io error: {0}")] - Io(#[from] std::io::Error), -} - -pub async fn run(config: config::Config) -> Result<(), IngestRouterError> { - let router_service = IngestRouterService {}; +pub async fn run(config: Config) -> Result<(), errors::IngestRouterError> { + let router_service = IngestRouterService::new(config.routes.clone(), config.locales.clone()); let router_task = run_http_service(&config.listener.host, config.listener.port, router_service); router_task.await?; Ok(()) } -struct IngestRouterService {} +#[derive(Clone)] +struct IngestRouterService { + router: Router, +} + +impl IngestRouterService { + fn new( + routes: Vec, + locales: HashMap< + String, + HashMap, + >, + ) -> Self { + let handler = RelayProjectConfigsHandler::new(locales); + Self { + router: Router::new(routes, handler), + } + } +} impl Service> for IngestRouterService { - type Response = Response>; + type Response = Response>; type Error = IngestRouterError; type Future = Pin> + Send + 'static>>; fn call(&self, req: Request) -> Self::Future { - println!("Received request: {req:?}"); - Box::pin(async move { - Ok(Response::new( - Full::new("ok\n".into()).map_err(|e| match e {}).boxed(), - )) - }) + let router = self.router.clone(); + Box::pin(async move { router.route(req).await }) } } diff --git a/ingest-router/src/relay_project_config_handler.rs b/ingest-router/src/relay_project_config_handler.rs new file mode 100644 index 0000000..d1636ab --- /dev/null +++ b/ingest-router/src/relay_project_config_handler.rs @@ -0,0 +1,982 @@ +//! Relay Project Configuration Handler +//! +//! This module implements the split-merge strategy for handling Sentry's +//! `/api/0/relays/projectconfigs/` endpoint across multiple upstream Sentry instances +//! in a multi-cell architecture. +//! +//! # Protocol Overview +//! +//! The Relay Project Configs endpoint (version 3) is used by Relay instances to fetch +//! project configurations needed to process events. This implementation acts as a proxy +//! that: +//! 1. Splits requests across multiple upstream Sentry cells based on project ownership +//! 2. Fans out requests in parallel to multiple upstreams +//! 3. Merges responses back into a single v3-compliant response +//! 4. Passes through all config data unchanged +//! +//! ## Endpoint Details +//! +//! - **Path**: `/api/0/relays/projectconfigs/` +//! - **Method**: `POST` +//! - **Protocol Version**: 3 (current) +//! - **Authentication**: RelayAuthentication (X-Sentry-Relay-Id, X-Sentry-Relay-Signature) +//! +//! # Request Format (Version 3) +//! +//! ```json +//! { +//! "publicKeys": ["key1", "key2", "key3"], +//! "noCache": false, +//! "global": true +//! } +//! ``` +//! +//! ## Request Fields +//! +//! - **`publicKeys`** (required): Array of project public keys (DSN keys) to fetch configs for +//! - **`noCache`** (optional): If `true`, bypass caching and compute fresh configs (downgrades to v2 behavior) +//! - **`global`** (optional): If `true`, include global relay configuration in response +//! +//! # Response Format (Version 3) +//! +//! ```json +//! { +//! "configs": { +//! "key1": { +//! "disabled": false, +//! "slug": "project-slug", +//! "publicKeys": [...], +//! "config": {...}, +//! "organizationId": 1, +//! "projectId": 42 +//! } +//! }, +//! "pending": ["key2", "key3"], +//! "global": { +//! "measurements": {...}, +//! "options": {...} +//! }, +//! "global_status": "ready" +//! } +//! ``` +//! +//! ## Response Fields +//! +//! - **`configs`**: Map of public keys to their project configurations +//! - Configs are passed through unchanged from upstream Sentry instances +//! - They will add the relevant processing relay URL in the response +//! - **`pending`**: Array of public keys for which configs are being computed asynchronously +//! - Relay should retry the request later to fetch these configs +//! - Also used when upstreams fail/timeout (graceful degradation) +//! - **`global`**: Global relay configuration (only if `global: true` in request) +//! - **`global_status`**: Status of global config (always "ready" when present) +//! +//! # Implementation Details +//! +//! ## Request Splitting Strategy +//! +//! When a request arrives with multiple public keys: +//! +//! 1. **Route each key to its owning cell** +//! - Currently: Round-robin distribution (TODO: replace with control plane lookup) +//! - TODO: Query locator service for each public key to get cell name +//! +//! 2. **Group keys by target upstream** +//! - Keys routed to the same cell are batched into one request +//! +//! 3. **Handle global config flag** +//! - First upstream gets `global: true` (or original value) +//! - All other upstreams get `global: false` +//! - Prevents complex global config merging (only one upstream returns it) +//! - TODO: Add capability to send to both but return only from first. This would +//! enable a failover mechanism. +//! +//! ## Response Merging Strategy +//! +//! Responses from multiple upstreams are merged as follows: +//! +//! ### Configs (HashMap merge) +//! - Merge all `configs` HashMaps from all upstreams +//! - Configs are passed through unchanged (no modifications) +//! - relay_url is expected to be added in the upstream response +//! +//! ### Pending (Array concatenation) +//! - Concatenate all `pending` arrays from all upstream responses +//! - Include keys from failed/timed-out upstreams +//! - Relay will retry these keys in a subsequent request +//! +//! ### Extra fields (HashMap merge) +//! - Merge `extra` fields (includes `global`, `global_status`, future fields) +//! - No conflicts expected (only first upstream has global config) +//! - Forward compatibility: new fields are automatically preserved +//! +//! ## Error Handling +//! +//! ### Partial Failures (Graceful Degradation) +//! - If some upstreams succeed: Return successful configs + pending for failed keys +//! - Failed keys are added to `pending` array (v3 protocol) +//! - Logged but does not block response +//! +//! ### Total Failure +//! - If all upstreams fail: Check if any keys were added to pending +//! - If pending is not empty: Return 200 OK with pending array (relay will retry) +//! - If pending is empty: Return 500 error (no recoverable state) +//! +//! ### Upstream Failure Scenarios +//! - **Timeout**: All keys from that upstream → pending +//! - **Connection error**: All keys from that upstream → pending +//! - **Parse error**: All keys from that upstream → pending +//! - **Task panic**: Logged error (extreme edge case, keys lost) +//! +//! ## Request Flow +//! +//! ### Success Scenario +//! +//! ```text +//! ┌─────────────┐ +//! │ Relay │ +//! └──────┬──────┘ +//! │ +//! │ POST /api/0/relays/projectconfigs/ +//! │ {publicKeys: [A,B,C,D,E,F]} +//! │ +//! ▼ +//! ┌──────────────────────────────────────┐ +//! │ Ingest Router (this handler) │ +//! │ │ +//! │ 1. Parse request │ +//! │ 2. Split keys by cell: │ +//! │ • US1 → [A,C,E] │ +//! │ • US2 → [B,D,F] │ +//! └───┬──────────────────────────┬───────┘ +//! │ │ +//! │ {publicKeys: [A,C,E], │ {publicKeys: [B,D,F], +//! │ global: true} │ global: false} +//! │ │ +//! ▼ ▼ +//! ┌──────────┐ ┌──────────┐ +//! │Cell US1 │ │Cell US2 │ +//! │(Sentry) │ │(Sentry) │ +//! └────┬─────┘ └─────┬────┘ +//! │ │ +//! │ {configs: {A,C,E}} │ {configs: {B,D,F}} +//! │ │ +//! └──────────┬───────────────┘ +//! ▼ +//! ┌──────────────────────────────────────┐ +//! │ Ingest Router (this handler) │ +//! │ │ +//! │ 3. Merge responses: │ +//! │ • Combine all configs │ +//! │ • Merge pending arrays │ +//! │ • Merge extra fields (global) │ +//! └──────────────┬───────────────────────┘ +//! │ +//! │ {configs: {A,B,C,D,E,F}, +//! │ global: {...}} +//! │ +//! ▼ +//! ┌─────────────┐ +//! │ Relay │ +//! └─────────────┘ +//! ``` +//! +//! ### Failure Scenario (Graceful Degradation) +//! +//! When an upstream fails or times out, keys are added to the `pending` array: +//! +//! ```text +//! ┌─────────────┐ +//! │ Relay │ +//! └──────┬──────┘ +//! │ +//! │ POST {publicKeys: [A,B,C,D,E,F]} +//! │ +//! ▼ +//! ┌──────────────────────────────────────┐ +//! │ Ingest Router (this handler) │ +//! │ Split: US1→[A,C,E], US2→[B,D,F] │ +//! └───┬──────────────────────────┬───────┘ +//! │ │ +//! ▼ ▼ +//! ┌──────────┐ ┌──────────┐ +//! │Cell US1 │ │Cell US2 │ +//! └────┬─────┘ └─────┬────┘ +//! │ │ +//! │ ✓ Success │ ✗ Timeout/Error +//! │ {configs: {A,C,E}} │ +//! │ │ +//! └──────────┬───────────────┘ +//! ▼ +//! ┌──────────────────────────────────────┐ +//! │ Ingest Router (this handler) │ +//! │ │ +//! │ • Collect successful: {A,C,E} │ +//! │ • Add failed to pending: [B,D,F] │ +//! └──────────────┬───────────────────────┘ +//! │ +//! │ {configs: {A,C,E}, +//! │ pending: [B,D,F]} +//! │ +//! ▼ +//! ┌─────────────┐ +//! │ Relay │ (will retry pending) +//! └─────────────┘ +//! ``` +//! +//! # Examples +//! +//! ## Example 1: All upstreams succeed +//! +//! **Request**: +//! ```json +//! {"publicKeys": ["key1", "key2"]} +//! ``` +//! +//! **Response**: +//! ```json +//! { +//! "configs": { +//! "key1": {"disabled": false, "slug": "project-us1", ...}, +//! "key2": {"disabled": false, "slug": "project-us2", ...} +//! } +//! } +//! ``` +//! +//! ## Example 2: One upstream fails +//! +//! **Request**: +//! ```json +//! {"publicKeys": ["key1", "key2", "key3"]} +//! ``` +//! +//! **Response** (if upstream with key2,key3 times out): +//! ```json +//! { +//! "configs": { +//! "key1": {"disabled": false, "slug": "project-us1", ...} +//! }, +//! "pending": ["key2", "key3"] +//! } +//! ``` +//! +//! ## Example 3: Request with global config +//! +//! **Request**: +//! ```json +//! {"publicKeys": ["key1", "key2"], "global": true} +//! ``` +//! +//! **Splitting**: +//! - Request to US1: `{"publicKeys": ["key1"], "global": true}` +//! - Request to US2: `{"publicKeys": ["key2"], "global": false}` +//! +//! **Response**: +//! ```json +//! { +//! "configs": {...}, +//! "global": {"measurements": {...}}, +//! "global_status": "ready" +//! } +//! ``` +//! +//! # See Also +//! +//! - [`RelayProjectConfigsHandler`] - Main handler struct for processing requests + +use crate::config::CellConfig; +use crate::errors::IngestRouterError; +use crate::http::send_to_upstream; +use crate::locale::{Locales, Upstream}; +use http_body_util::{BodyExt, Full, combinators::BoxBody}; +use hyper::body::{Body, Bytes}; +use hyper::header::CONTENT_TYPE; +use hyper::{Request, Response, StatusCode}; +use hyper_util::client::legacy::Client; +use hyper_util::client::legacy::connect::HttpConnector; +use hyper_util::rt::TokioExecutor; +use serde::{Deserialize, Serialize}; +use serde_json::Value as JsonValue; +use std::collections::HashMap; +use std::time::Duration; +use tokio::task::JoinHandle; +use tokio::time::timeout; + +/// Request format for the relay project configs endpoint. +/// +/// See module-level docs for full protocol details. +#[derive(Debug, Clone, Serialize, Deserialize)] +struct RelayProjectConfigsRequest { + /// DSN public keys to fetch configs for. + #[serde(rename = "publicKeys")] + pub public_keys: Vec, + + /// Whether to include global config (optional). + /// + /// first upstream gets original value, others get `Some(false)`. + #[serde(skip_serializing_if = "Option::is_none")] + pub global: Option, + + /// Other fields (`noCache`, future fields) for forward compatibility. + #[serde(flatten)] + pub extra: HashMap, +} + +impl RelayProjectConfigsRequest { + fn from_bytes(bytes: &Bytes) -> Result { + serde_json::from_slice(bytes) + } + + fn to_bytes(&self) -> Result { + let json = serde_json::to_vec(self)?; + Ok(Bytes::from(json)) + } +} + +/// Response format for the relay project configs endpoint. +/// +/// See module-level docs for merge strategy and field details. +#[derive(Debug, Clone, Serialize, Deserialize)] +struct RelayProjectConfigsResponse { + /// Project configs (HashMap merged from all upstreams). + pub configs: HashMap, + + /// Keys being computed async or from failed upstreams (concatenated). + #[serde(skip_serializing_if = "Option::is_none")] + pub pending: Option>, + + /// Other fields (`global`, `global_status`, future fields). + #[serde(flatten)] + pub extra: HashMap, +} + +impl RelayProjectConfigsResponse { + fn from_bytes(bytes: &Bytes) -> Result { + serde_json::from_slice(bytes) + } +} + +/// Result from an upstream task request. +struct UpstreamTaskResult { + /// The upstream URL that was contacted. + sentry_url: String, + /// The public keys that were requested from this upstream. + public_keys: Vec, + /// The result of the upstream request. + result: Result, IngestRouterError>, +} + +/// Merged results from all upstream tasks. +struct MergedResults { + /// All configs merged from successful upstreams. + configs: HashMap, + /// All pending keys (from failed upstreams or upstream pending arrays). + pending: Vec, + /// Extra fields (global config, status, etc.). + extra: HashMap, +} + +/// Handler for Relay Project Configs endpoint. +/// +/// See module-level docs for complete protocol details, implementation strategy, +/// and request/response flow diagrams. +pub struct RelayProjectConfigsHandler { + /// HTTP client for sending requests to upstream Sentry instances. + client: Client>, + + /// Locales mapping for locale-based upstream lookups. + /// + /// Maps locale → cell name → upstream (relay URL + sentry URL). + locales: Locales, +} + +impl RelayProjectConfigsHandler { + pub fn new(locales_config: HashMap>) -> Self { + let connector = HttpConnector::new(); + let client = Client::builder(TokioExecutor::new()).build(connector); + + // Build locales from config + let locales = Locales::new(locales_config); + + Self { client, locales } + } + + pub async fn handle( + &self, + locale: &str, + request: Request, + ) -> Result>, IngestRouterError> + where + B: Body + Send + 'static, + B::Data: Send, + B::Error: std::error::Error + Send + Sync + 'static, + { + // Get cells for this locale + let cells = self.locales.get_cells(locale).ok_or_else(|| { + IngestRouterError::InternalError(format!( + "No targets configured for locale: {}", + locale + )) + })?; + + // Buffer the request body. We need to full body in order to do request massaging. + let (parts, body) = request.into_parts(); + let body_bytes = body + .collect() + .await + .map(|collected| collected.to_bytes()) + .map_err(|e| IngestRouterError::RequestBodyError(e.to_string()))?; + let base_request = Request::from_parts(parts, ()); + + // Process the request + self.handle_with_targets(&cells.cell_to_upstreams, base_request, body_bytes) + .await + } + + /// Internal method that orchestrates the split-merge flow. + /// + /// High-level steps: + /// 1. Parse and split the request across upstreams + /// 2. Spawn parallel tasks to fan out requests + /// 3. Collect and merge results from all upstreams + /// 4. Build and return the merged response + async fn handle_with_targets( + &self, + cell_upstreams: &HashMap, + base_request: Request<()>, + body_bytes: Bytes, + ) -> Result>, IngestRouterError> { + // Parse the request + let request_data = RelayProjectConfigsRequest::from_bytes(&body_bytes).map_err(|e| { + IngestRouterError::RequestBodyError(format!("Failed to parse request: {e}")) + })?; + + // Split publicKeys across upstreams + let split_requests = self.split_keys_by_upstream(&request_data, cell_upstreams); + + // Spawn tasks to fan out requests in parallel + let tasks = self.spawn_upstream_tasks(split_requests, &base_request)?; + + // Collect and merge results from all tasks + let results = self.collect_task_results(tasks).await; + + // Only return an error if we have no configs AND no pending AND we had keys to request + // Having keys in pending is a valid v3 response (relay will retry later) + if results.configs.is_empty() + && results.pending.is_empty() + && !request_data.public_keys.is_empty() + { + return Err(IngestRouterError::InternalError( + "All upstream requests failed with no recoverable state".to_string(), + )); + } + + // Build merged response in the relay format + self.build_merged_response(results.configs, results.pending, results.extra) + } + + /// Splits public keys across multiple upstream cells. + /// + /// Current: Round-robin stub. TODO: Replace with locator service lookup. + /// See module-level docs for complete splitting strategy and global config handling. + fn split_keys_by_upstream( + &self, + request: &RelayProjectConfigsRequest, + cell_upstreams: &HashMap, + ) -> Vec<(Upstream, RelayProjectConfigsRequest)> { + if cell_upstreams.is_empty() { + return Vec::new(); + } + + // For now, convert HashMap to Vec for round-robin stub + // In the future, we'll use cell_upstreams.get(cell_name) directly + let upstreams: Vec<&Upstream> = cell_upstreams.values().collect(); + + let mut split: HashMap> = HashMap::new(); + + // Round-robin stub: distribute publicKeys evenly across upstreams + // TODO: Replace with control plane lookup per key + for (index, public_key) in request.public_keys.iter().enumerate() { + let upstream_index = index % upstreams.len(); + + split + .entry(upstream_index) + .or_default() + .push(public_key.clone()); + } + + // Build a request for each upstream with its assigned publicKeys + split + .into_iter() + .enumerate() + .map(|(idx, (upstream_index, public_keys))| { + let upstream = upstreams[upstream_index].clone(); + + // For v3 protocol: set global=true for only the first upstream + // This avoids needing to merge global configs from multiple responses + let global = if idx == 0 { + // First upstream: keep global flag as-is from original request + request.global + } else { + // Other upstreams: explicitly set global=false + Some(false) + }; + + let split_request = RelayProjectConfigsRequest { + public_keys, + global, + extra: request.extra.clone(), + }; + (upstream, split_request) + }) + .collect() + } + + /// Spawn async tasks to send requests to all upstreams in parallel. + /// + /// Each task sends the split request to its designated upstream and returns + /// an `UpstreamTaskResult` containing the response or error. + fn spawn_upstream_tasks( + &self, + split_requests: Vec<(Upstream, RelayProjectConfigsRequest)>, + base_request: &Request<()>, + ) -> Result>, IngestRouterError> { + let mut tasks = Vec::new(); + + for (upstream, split_request) in split_requests { + // Track public keys for this upstream (needed if request fails) + let public_keys = split_request.public_keys.clone(); + + // Serialize the split request body + let request_body = split_request.to_bytes().map_err(|e| { + IngestRouterError::InternalError(format!("Failed to serialize request: {e}")) + })?; + + // Build request with headers from original request + let mut req_builder = Request::builder() + .method(base_request.method()) + .uri(base_request.uri().clone()) + .version(base_request.version()); + + for (name, value) in base_request.headers() { + req_builder = req_builder.header(name, value); + } + + let request = req_builder + .body(Full::new(request_body)) + .expect("Failed to build request"); + + let client = self.client.clone(); + let sentry_url = upstream.sentry_url.clone(); + + // Spawn a task for each upstream request + let task = tokio::spawn(async move { + let sentry_url_str = sentry_url.to_string(); + let result: Result, IngestRouterError> = + send_to_upstream(&client, &sentry_url, request, 30).await; + + UpstreamTaskResult { + sentry_url: sentry_url_str, + public_keys, + result, + } + }); + + tasks.push(task); + } + + Ok(tasks) + } + + /// Collect results from all spawned tasks and merge them. + /// + /// Handles timeouts, task panics, and HTTP failures gracefully. + /// Failed keys are added to the pending array for retry. + async fn collect_task_results( + &self, + tasks: Vec>, + ) -> MergedResults { + let mut merged_configs = HashMap::new(); + let mut merged_pending = Vec::new(); + let mut merged_extra = HashMap::new(); + + for task in tasks { + let task_result = timeout(Duration::from_secs(30), task).await; + + // Handle timeout + let Ok(join_result) = task_result else { + tracing::error!("Task timed out after 30 seconds"); + continue; + }; + + // Handle task panic + let Ok(upstream_result) = join_result else { + if let Err(e) = join_result { + tracing::error!("Task panicked: {e}"); + } + continue; + }; + + // Process the upstream result + self.process_upstream_result( + upstream_result, + &mut merged_configs, + &mut merged_pending, + &mut merged_extra, + ); + } + + MergedResults { + configs: merged_configs, + pending: merged_pending, + extra: merged_extra, + } + } + + /// Process the result from a single upstream task. + /// + /// Handles both successful and failed upstream responses, merging successful + /// configs and adding failed keys to the pending array. + fn process_upstream_result( + &self, + upstream_result: UpstreamTaskResult, + merged_configs: &mut HashMap, + merged_pending: &mut Vec, + merged_extra: &mut HashMap, + ) { + let UpstreamTaskResult { + sentry_url, + public_keys, + result, + } = upstream_result; + + // Handle HTTP request failure + let Ok(response) = result else { + tracing::error!( + sentry_url = %sentry_url, + error = %result.unwrap_err(), + "Request to upstream failed" + ); + // Add all keys from this upstream to pending (v3 protocol) + merged_pending.extend(public_keys); + return; + }; + + // Parse response body + let body = response.into_body(); + match RelayProjectConfigsResponse::from_bytes(&body) { + Ok(response_data) => { + // Merge configs from this upstream + merged_configs.extend(response_data.configs); + + // Merge pending arrays (v3 protocol) + if let Some(pending) = response_data.pending { + merged_pending.extend(pending); + } + + // Merge extra fields (global, global_status, future fields) + merged_extra.extend(response_data.extra); + } + Err(e) => { + tracing::error!( + sentry_url = %sentry_url, + error = %e, + "Failed to parse response from upstream" + ); + // Add all keys from this upstream to pending + merged_pending.extend(public_keys); + } + } + } + + /// Build a merged response from collected configs in relay format + fn build_merged_response( + &self, + merged_configs: HashMap, + merged_pending: Vec, + merged_extra: HashMap, + ) -> Result>, IngestRouterError> { + // Wrap in relay response format + let response = RelayProjectConfigsResponse { + configs: merged_configs, + pending: if merged_pending.is_empty() { + None + } else { + Some(merged_pending) + }, + extra: merged_extra, + }; + + let merged_json = serde_json::to_vec(&response) + .map_err(|e| IngestRouterError::ResponseSerializationError(e.to_string()))?; + + Response::builder() + .status(StatusCode::OK) + .header(CONTENT_TYPE, "application/json") + .body( + Full::new(Bytes::from(merged_json)) + .map_err(|e| match e {}) + .boxed(), + ) + .map_err(|e| IngestRouterError::ResponseBuildError(e.to_string())) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use url::Url; + + #[tokio::test] + async fn test_build_merged_response() { + let handler = RelayProjectConfigsHandler::new(HashMap::new()); + + // Test 1: Empty response + let response = handler + .build_merged_response(HashMap::new(), Vec::new(), HashMap::new()) + .unwrap(); + assert_eq!(response.status(), StatusCode::OK); + let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); + let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); + assert_eq!(parsed, serde_json::json!({"configs": {}})); + + // Test 2: Multiple configs with all fields preserved (pass-through verification) + let mut configs = HashMap::new(); + configs.insert( + "project1".to_string(), + serde_json::json!({ + "disabled": false, + "slug": "test-project", + "organizationId": 42, + "projectId": 100, + "customField": "customValue" + }), + ); + configs.insert( + "project2".to_string(), + serde_json::json!({"config": "value2"}), + ); + + let response = handler + .build_merged_response(configs, Vec::new(), HashMap::new()) + .unwrap(); + let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); + let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); + + // Verify multiple configs present + assert!(parsed["configs"].get("project1").is_some()); + assert!(parsed["configs"].get("project2").is_some()); + + // Verify all fields preserved exactly as-is (pass-through) + assert_eq!(parsed["configs"]["project1"]["disabled"], false); + assert_eq!(parsed["configs"]["project1"]["slug"], "test-project"); + assert_eq!(parsed["configs"]["project1"]["organizationId"], 42); + assert_eq!(parsed["configs"]["project1"]["projectId"], 100); + assert_eq!(parsed["configs"]["project1"]["customField"], "customValue"); + } + + #[tokio::test] + async fn test_all_upstreams_fail_returns_error() { + // Set up handler with invalid upstreams + let mut locales = HashMap::new(); + locales.insert( + "us".to_string(), + HashMap::from([( + "us-cell-1".to_string(), + CellConfig { + relay_url: Url::parse("http://invalid-relay.example.com:8080").unwrap(), + sentry_url: Url::parse("http://invalid-sentry.example.com:8080").unwrap(), + }, + )]), + ); + + let handler = RelayProjectConfigsHandler::new(locales); + + // Create a request with public keys + let request_body = serde_json::json!({ + "publicKeys": ["test-key-1", "test-key-2"] + }); + let body_bytes = Bytes::from(serde_json::to_vec(&request_body).unwrap()); + + // Create empty cell_targets to simulate all upstreams failing + let empty_targets = HashMap::new(); + + // Create a base request + let request = Request::builder().method("POST").uri("/").body(()).unwrap(); + + // This should return an error since no upstreams will succeed (empty targets) + let result = handler + .handle_with_targets(&empty_targets, request, body_bytes) + .await; + + assert!(result.is_err()); + match result { + Err(IngestRouterError::InternalError(msg)) => { + assert_eq!( + msg, + "All upstream requests failed with no recoverable state" + ); + } + _ => panic!("Expected InternalError"), + } + } + + #[tokio::test] + async fn test_v3_protocol_fields() { + let handler = RelayProjectConfigsHandler::new(HashMap::new()); + + // Test 1: Pending array with values + let pending = vec!["key1".to_string(), "key2".to_string(), "key3".to_string()]; + let response = handler + .build_merged_response(HashMap::new(), pending, HashMap::new()) + .unwrap(); + let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); + let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); + assert_eq!(parsed["pending"].as_array().unwrap().len(), 3); + assert_eq!(parsed["pending"][0], "key1"); + + // Test 2: Empty pending omitted (skip_serializing_if) + let response = handler + .build_merged_response(HashMap::new(), Vec::new(), HashMap::new()) + .unwrap(); + let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); + let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); + assert!(parsed.get("pending").is_none()); + + // Test 3: Global config and extra fields (forward compatibility) + let mut extra = HashMap::new(); + extra.insert( + "global".to_string(), + serde_json::json!({"measurements": {"maxCustomMeasurements": 10}}), + ); + extra.insert("global_status".to_string(), serde_json::json!("ready")); + extra.insert( + "futureFeature".to_string(), + serde_json::json!({"enabled": true}), + ); + + let response = handler + .build_merged_response(HashMap::new(), Vec::new(), extra) + .unwrap(); + let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); + let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); + + // Verify global config + assert_eq!( + parsed["global"]["measurements"]["maxCustomMeasurements"], + 10 + ); + assert_eq!(parsed["global_status"], "ready"); + // Verify future/extra fields preserved + assert_eq!(parsed["futureFeature"]["enabled"], true); + } + + #[tokio::test] + async fn test_v3_upstream_failure_adds_keys_to_pending() { + // Set up handler with invalid upstream that will fail to connect + let mut locales = HashMap::new(); + locales.insert( + "us".to_string(), + HashMap::from([( + "us-cell-1".to_string(), + CellConfig { + relay_url: Url::parse("http://localhost:1").unwrap(), // Invalid port + sentry_url: Url::parse("http://localhost:1").unwrap(), // Will fail to connect + }, + )]), + ); + + let handler = RelayProjectConfigsHandler::new(locales.clone()); + + // Create a request with public keys + let request_body = serde_json::json!({ + "publicKeys": ["test-key-1", "test-key-2", "test-key-3"] + }); + let body_bytes = Bytes::from(serde_json::to_vec(&request_body).unwrap()); + + // Get upstreams + let locales = Locales::new(locales); + let cell_upstreams = &locales.get_cells("us").unwrap().cell_to_upstreams; + + // Create a base request + let request = Request::builder().method("POST").uri("/").body(()).unwrap(); + + // When upstream fails, all its keys should be added to pending + let result = handler + .handle_with_targets(cell_upstreams, request, body_bytes) + .await; + + // Should succeed (v3 protocol - returning pending is valid) + assert!(result.is_ok()); + let response = result.unwrap(); + assert_eq!(response.status(), StatusCode::OK); + + // Parse response + let body = response.into_body(); + let body_bytes = body.collect().await.unwrap().to_bytes(); + let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); + + // Verify configs is empty (upstream failed) + assert!(parsed["configs"].as_object().unwrap().is_empty()); + + // Verify all keys are in pending + assert!(parsed.get("pending").is_some()); + let pending = parsed["pending"].as_array().unwrap(); + assert_eq!(pending.len(), 3); + assert!(pending.contains(&serde_json::json!("test-key-1"))); + assert!(pending.contains(&serde_json::json!("test-key-2"))); + assert!(pending.contains(&serde_json::json!("test-key-3"))); + } + + #[test] + fn test_global_field_handling() { + // Test 1: Split behavior - first upstream gets true, others get false + let mut locales = HashMap::new(); + locales.insert( + "us".to_string(), + HashMap::from([ + ( + "us-cell-1".to_string(), + CellConfig { + relay_url: Url::parse("http://us1-relay:8080").unwrap(), + sentry_url: Url::parse("http://us1-sentry:8080").unwrap(), + }, + ), + ( + "us-cell-2".to_string(), + CellConfig { + relay_url: Url::parse("http://us2-relay:8080").unwrap(), + sentry_url: Url::parse("http://us2-sentry:8080").unwrap(), + }, + ), + ]), + ); + + let handler = RelayProjectConfigsHandler::new(locales.clone()); + let request = RelayProjectConfigsRequest { + public_keys: vec!["key1".to_string(), "key2".to_string()], + global: Some(true), + extra: HashMap::new(), + }; + + let locales_obj = Locales::new(locales); + let cell_upstreams = &locales_obj.get_cells("us").unwrap().cell_to_upstreams; + let splits = handler.split_keys_by_upstream(&request, cell_upstreams); + + assert_eq!(splits.len(), 2); + assert_eq!(splits[0].1.global, Some(true)); // First gets true + assert_eq!(splits[1].1.global, Some(false)); // Others get false + + // Test 2: Serialization - global omitted when None + let request = RelayProjectConfigsRequest { + public_keys: vec!["key1".to_string()], + global: None, + extra: HashMap::new(), + }; + + let json = serde_json::to_string(&request).unwrap(); + let parsed: JsonValue = serde_json::from_str(&json).unwrap(); + assert!(parsed.get("global").is_none()); + assert!(parsed.get("publicKeys").is_some()); + } +} diff --git a/ingest-router/src/router.rs b/ingest-router/src/router.rs index 015c4a5..3488c92 100644 --- a/ingest-router/src/router.rs +++ b/ingest-router/src/router.rs @@ -1,5 +1,6 @@ use crate::config::{HandlerAction, Route}; use crate::errors::IngestRouterError; +use crate::relay_project_config_handler::RelayProjectConfigsHandler; use http_body_util::{BodyExt, Full, combinators::BoxBody}; use hyper::body::Bytes; use hyper::{Request, Response, StatusCode}; @@ -9,13 +10,15 @@ use std::sync::Arc; #[derive(Clone)] pub struct Router { routes: Arc>, + handler: Arc, } impl Router { - /// Creates a new router with the given routes - pub fn new(routes: Vec) -> Self { + /// Creates a new router with the given routes and handler + pub fn new(routes: Vec, handler: RelayProjectConfigsHandler) -> Self { Self { routes: Arc::new(routes), + handler: Arc::new(handler), } } @@ -26,6 +29,8 @@ impl Router { ) -> Result>, IngestRouterError> where B: hyper::body::Body + Send + 'static, + B::Data: Send, + B::Error: std::error::Error + Send + Sync + 'static, { // Find a matching route match self.find_matching_route(&req) { @@ -90,37 +95,22 @@ impl Router { true } - /// Handles a matched action (placeholder for now) + /// Handles a matched action by calling the appropriate handler async fn handle_action( &self, - _req: Request, + req: Request, action: &HandlerAction, ) -> Result>, IngestRouterError> where B: hyper::body::Body + Send + 'static, + B::Data: Send, + B::Error: std::error::Error + Send + Sync + 'static, { - // TODO: Implement actual handler logic - // This will need to be passed to a separate Handler interface that has access to - // locale_to_cells mapping and upstreams - - // For now, just return a debug response showing which handler would be called - let response_body = match action { + match action { HandlerAction::RelayProjectConfigs(args) => { - format!( - "Route matched!\nHandler: RelayProjectConfigs\nLocale: {}\n", - args.locale - ) + self.handler.handle(&args.locale, req).await } - }; - - Response::builder() - .status(StatusCode::OK) - .body( - Full::new(response_body.into()) - .map_err(|e| match e {}) - .boxed(), - ) - .map_err(|e| IngestRouterError::InternalError(format!("Failed to build response: {e}"))) + } } /// Handles an unmatched request @@ -142,13 +132,50 @@ impl Router { mod tests { use super::*; use crate::config::{HttpMethod, Match, RelayProjectConfigsArgs, Route}; - use http_body_util::Empty; use hyper::body::Bytes; use hyper::header::HOST; use hyper::{Method, Request}; fn test_router(routes: Vec) -> Router { - Router::new(routes) + use crate::config::CellConfig; + use std::collections::HashMap; + use url::Url; + + // Create test locales + let mut locales = HashMap::new(); + locales.insert( + "us".to_string(), + HashMap::from([( + "us-cell-1".to_string(), + CellConfig { + relay_url: Url::parse("http://us-relay.example.com:8080").unwrap(), + sentry_url: Url::parse("http://us-sentry.example.com:8080").unwrap(), + }, + )]), + ); + locales.insert( + "de".to_string(), + HashMap::from([( + "de-cell-1".to_string(), + CellConfig { + relay_url: Url::parse("http://de-relay.example.com:8080").unwrap(), + sentry_url: Url::parse("http://de-sentry.example.com:8080").unwrap(), + }, + )]), + ); + locales.insert( + "local".to_string(), + HashMap::from([( + "local-cell".to_string(), + CellConfig { + relay_url: Url::parse("http://local-relay.example.com:8080").unwrap(), + sentry_url: Url::parse("http://local-sentry.example.com:8080").unwrap(), + }, + )]), + ); + + let handler = RelayProjectConfigsHandler::new(locales); + Router::new(routes, handler) } fn test_request( @@ -156,13 +183,20 @@ mod tests { path: &str, host: Option<&str>, ) -> Request> { + // Create a valid relay project configs request body with empty publicKeys + // so we don't need to contact upstreams in routing tests + let request_body = serde_json::json!({ + "publicKeys": [] + }); + let body_str = serde_json::to_string(&request_body).unwrap(); + let mut builder = Request::builder().method(method).uri(path); if let Some(h) = host { builder = builder.header(HOST, h); } builder .body( - Empty::::new() + http_body_util::Full::new(Bytes::from(body_str)) .map_err(|never| match never {}) .boxed(), ) diff --git a/shared/Cargo.toml b/shared/Cargo.toml index 7f786ac..406a197 100644 --- a/shared/Cargo.toml +++ b/shared/Cargo.toml @@ -10,3 +10,4 @@ hyper = { workspace = true } hyper-util = { workspace = true } tokio = { workspace = true } tracing = { workspace = true } +url = { workspace = true } From 01b0a642b39a6bcc3ad8fde5978c648810697366 Mon Sep 17 00:00:00 2001 From: Nikhar Saxena Date: Thu, 20 Nov 2025 11:53:51 -0800 Subject: [PATCH 2/7] Fix lint --- ingest-router/src/lib.rs | 16 +++++----------- .../src/relay_project_config_handler.rs | 2 +- 2 files changed, 6 insertions(+), 12 deletions(-) diff --git a/ingest-router/src/lib.rs b/ingest-router/src/lib.rs index bf9b8af..13cf432 100644 --- a/ingest-router/src/lib.rs +++ b/ingest-router/src/lib.rs @@ -5,6 +5,10 @@ pub mod locale; pub mod relay_project_config_handler; pub mod router; +use crate::config::{CellConfig, Config, Route}; +use crate::errors::IngestRouterError; +use crate::relay_project_config_handler::RelayProjectConfigsHandler; +use crate::router::Router; use http_body_util::combinators::BoxBody; use hyper::body::Bytes; use hyper::body::Incoming; @@ -13,10 +17,6 @@ use hyper::{Request, Response}; use shared::http::run_http_service; use std::collections::HashMap; use std::pin::Pin; -use crate::config::{Config, CellConfig, Route}; -use crate::router::Router; -use crate::relay_project_config_handler::RelayProjectConfigsHandler; -use crate::errors::IngestRouterError; pub async fn run(config: Config) -> Result<(), errors::IngestRouterError> { let router_service = IngestRouterService::new(config.routes.clone(), config.locales.clone()); @@ -31,13 +31,7 @@ struct IngestRouterService { } impl IngestRouterService { - fn new( - routes: Vec, - locales: HashMap< - String, - HashMap, - >, - ) -> Self { + fn new(routes: Vec, locales: HashMap>) -> Self { let handler = RelayProjectConfigsHandler::new(locales); Self { router: Router::new(routes, handler), diff --git a/ingest-router/src/relay_project_config_handler.rs b/ingest-router/src/relay_project_config_handler.rs index d1636ab..784df25 100644 --- a/ingest-router/src/relay_project_config_handler.rs +++ b/ingest-router/src/relay_project_config_handler.rs @@ -602,7 +602,7 @@ impl RelayProjectConfigsHandler { for task in tasks { let task_result = timeout(Duration::from_secs(30), task).await; - + // Handle timeout let Ok(join_result) = task_result else { tracing::error!("Task timed out after 30 seconds"); From 5055556eb35f786a29e48527bbf5cf9b9c81d487 Mon Sep 17 00:00:00 2001 From: Nikhar Saxena Date: Thu, 20 Nov 2025 12:03:22 -0800 Subject: [PATCH 3/7] Address Seer comments --- .../src/relay_project_config_handler.rs | 59 ++++++++++++++----- 1 file changed, 44 insertions(+), 15 deletions(-) diff --git a/ingest-router/src/relay_project_config_handler.rs b/ingest-router/src/relay_project_config_handler.rs index 784df25..b58ac17 100644 --- a/ingest-router/src/relay_project_config_handler.rs +++ b/ingest-router/src/relay_project_config_handler.rs @@ -376,6 +376,17 @@ struct MergedResults { extra: HashMap, } +/// Task handle paired with its public keys for graceful failure handling. +/// +/// The public keys are tracked outside the task so they can be added to the +/// pending array if the task times out or panics, maintaining v3 protocol compliance. +struct TaskWithKeys { + /// The spawned task handle. + handle: JoinHandle, + /// Public keys requested from this upstream. + public_keys: Vec, +} + /// Handler for Relay Project Configs endpoint. /// /// See module-level docs for complete protocol details, implementation strategy, @@ -506,16 +517,19 @@ impl RelayProjectConfigsHandler { } // Build a request for each upstream with its assigned publicKeys - split + // Sort by upstream_index to ensure deterministic ordering + let mut sorted_split: Vec<_> = split.into_iter().collect(); + sorted_split.sort_by_key(|(upstream_index, _)| *upstream_index); + + sorted_split .into_iter() - .enumerate() - .map(|(idx, (upstream_index, public_keys))| { + .map(|(upstream_index, public_keys)| { let upstream = upstreams[upstream_index].clone(); - // For v3 protocol: set global=true for only the first upstream + // For v3 protocol: set global=true for only the first upstream (index 0) // This avoids needing to merge global configs from multiple responses - let global = if idx == 0 { - // First upstream: keep global flag as-is from original request + let global = if upstream_index == 0 { + // First upstream (index 0): keep global flag as-is from original request request.global } else { // Other upstreams: explicitly set global=false @@ -536,11 +550,14 @@ impl RelayProjectConfigsHandler { /// /// Each task sends the split request to its designated upstream and returns /// an `UpstreamTaskResult` containing the response or error. + /// + /// Returns tuples of (JoinHandle, public_keys) so keys can be added to pending + /// if the task times out or panics. fn spawn_upstream_tasks( &self, split_requests: Vec<(Upstream, RelayProjectConfigsRequest)>, base_request: &Request<()>, - ) -> Result>, IngestRouterError> { + ) -> Result, IngestRouterError> { let mut tasks = Vec::new(); for (upstream, split_request) in split_requests { @@ -569,6 +586,9 @@ impl RelayProjectConfigsHandler { let client = self.client.clone(); let sentry_url = upstream.sentry_url.clone(); + // Clone keys so we can track them even if task times out + let public_keys_for_task = public_keys.clone(); + // Spawn a task for each upstream request let task = tokio::spawn(async move { let sentry_url_str = sentry_url.to_string(); @@ -577,12 +597,16 @@ impl RelayProjectConfigsHandler { UpstreamTaskResult { sentry_url: sentry_url_str, - public_keys, + public_keys: public_keys_for_task, result, } }); - tasks.push(task); + // Store both task handle and keys (keys needed if task times out) + tasks.push(TaskWithKeys { + handle: task, + public_keys, + }); } Ok(tasks) @@ -592,20 +616,23 @@ impl RelayProjectConfigsHandler { /// /// Handles timeouts, task panics, and HTTP failures gracefully. /// Failed keys are added to the pending array for retry. - async fn collect_task_results( - &self, - tasks: Vec>, - ) -> MergedResults { + async fn collect_task_results(&self, tasks: Vec) -> MergedResults { let mut merged_configs = HashMap::new(); let mut merged_pending = Vec::new(); let mut merged_extra = HashMap::new(); - for task in tasks { - let task_result = timeout(Duration::from_secs(30), task).await; + for task_with_keys in tasks { + let TaskWithKeys { + handle, + public_keys, + } = task_with_keys; + let task_result = timeout(Duration::from_secs(30), handle).await; // Handle timeout let Ok(join_result) = task_result else { tracing::error!("Task timed out after 30 seconds"); + // Add all keys from this upstream to pending (v3 protocol) + merged_pending.extend(public_keys); continue; }; @@ -614,6 +641,8 @@ impl RelayProjectConfigsHandler { if let Err(e) = join_result { tracing::error!("Task panicked: {e}"); } + // Add all keys from this upstream to pending (v3 protocol) + merged_pending.extend(public_keys); continue; }; From 3bca55c590cf803ad1544ca831a9ed87019d6c41 Mon Sep 17 00:00:00 2001 From: Nikhar Saxena Date: Thu, 20 Nov 2025 12:07:22 -0800 Subject: [PATCH 4/7] Call abort on task failure --- ingest-router/src/relay_project_config_handler.rs | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/ingest-router/src/relay_project_config_handler.rs b/ingest-router/src/relay_project_config_handler.rs index b58ac17..681485b 100644 --- a/ingest-router/src/relay_project_config_handler.rs +++ b/ingest-router/src/relay_project_config_handler.rs @@ -623,14 +623,16 @@ impl RelayProjectConfigsHandler { for task_with_keys in tasks { let TaskWithKeys { - handle, + mut handle, public_keys, } = task_with_keys; - let task_result = timeout(Duration::from_secs(30), handle).await; + let task_result = timeout(Duration::from_secs(30), &mut handle).await; // Handle timeout let Ok(join_result) = task_result else { tracing::error!("Task timed out after 30 seconds"); + // Abort the timed-out task to prevent it from continuing in background + handle.abort(); // Add all keys from this upstream to pending (v3 protocol) merged_pending.extend(public_keys); continue; From 607a63eff78d2d3bbea1dfefc703073f57f678f9 Mon Sep 17 00:00:00 2001 From: Nikhar Saxena Date: Tue, 25 Nov 2025 11:24:45 -0800 Subject: [PATCH 5/7] Address review comments --- example_config_ingest_router.yaml | 16 +- ingest-router/src/config.rs | 162 +++- ingest-router/src/errors.rs | 65 ++ ingest-router/src/lib.rs | 16 +- ingest-router/src/locale.rs | 67 +- .../src/relay_project_config_handler.rs | 796 +++++++++++------- ingest-router/src/router.rs | 74 +- 7 files changed, 792 insertions(+), 404 deletions(-) diff --git a/example_config_ingest_router.yaml b/example_config_ingest_router.yaml index dabe4f6..7f44928 100644 --- a/example_config_ingest_router.yaml +++ b/example_config_ingest_router.yaml @@ -7,17 +7,27 @@ ingest_router: port: 3001 locales: us: - us1: + - name: us1 sentry_url: "http://10.0.0.1:8080" relay_url: "http://10.0.0.1:8090" - us2: + - name: us2 sentry_url: "http://10.0.0.2:8080" relay_url: "http://10.0.0.2:8090" de: - de: + - name: de sentry_url: "http://10.0.0.3:8080" relay_url: "http://10.0.0.3:8090" + # Timeout configuration for relay handlers (optional, defaults shown) + # Two-layer timeout strategy: + # - HTTP timeout: Per-request timeout for individual HTTP calls (fixed) + # - Task timeout: Global cutoff - once first succeeds, ALL remaining tasks have + # task_subsequent_timeout_secs total to complete (prevents slow cells from blocking) + # relay_timeouts: + # http_timeout_secs: 15 # HTTP request timeout + # task_initial_timeout_secs: 20 # Wait for first upstream (must be >= http_timeout) + # task_subsequent_timeout_secs: 5 # Global deadline after first success + routes: - match: host: us.sentry.io diff --git a/ingest-router/src/config.rs b/ingest-router/src/config.rs index 14b85c3..a43c5d3 100644 --- a/ingest-router/src/config.rs +++ b/ingest-router/src/config.rs @@ -22,6 +22,9 @@ pub enum ValidationError { #[error("Locale '{0}' has no valid cells (none of its cells match any upstream)")] LocaleHasNoValidCells(String), + + #[error("Invalid timeout configuration: {0}")] + InvalidTimeouts(String), } /// HTTP methods supported for route matching @@ -59,10 +62,63 @@ pub struct RelayProjectConfigsArgs { pub locale: String, } +/// Timeout configuration for relay project configs handler +#[derive(Clone, Debug, Deserialize, PartialEq)] +#[serde(default)] +pub struct RelayTimeouts { + /// HTTP request timeout for individual upstream calls (seconds). + /// This is the maximum time a single HTTP request can take. + /// Default: 15 seconds + pub http_timeout_secs: u16, + + /// Task timeout when waiting for the first upstream to respond (seconds). + /// Must be >= http_timeout_secs to allow HTTP requests to complete. + /// Default: 20 seconds + pub task_initial_timeout_secs: u16, + + /// Deadline for all remaining tasks after first success (seconds). + /// Aggressively cuts off slow upstreams once we have good data. + /// Default: 5 seconds + pub task_subsequent_timeout_secs: u16, +} + +impl Default for RelayTimeouts { + fn default() -> Self { + Self { + http_timeout_secs: 15, + task_initial_timeout_secs: 20, + task_subsequent_timeout_secs: 5, + } + } +} + +impl RelayTimeouts { + /// Validates the timeout configuration + pub fn validate(&self) -> Result<(), ValidationError> { + // Initial task timeout must be >= HTTP timeout to allow requests to complete + if self.task_initial_timeout_secs < self.http_timeout_secs { + return Err(ValidationError::InvalidTimeouts( + "task_initial_timeout_secs must be >= http_timeout_secs".to_string(), + )); + } + + // Subsequent task timeout should be > 0 + if self.task_subsequent_timeout_secs == 0 { + return Err(ValidationError::InvalidTimeouts( + "task_subsequent_timeout_secs must be > 0".to_string(), + )); + } + + Ok(()) + } +} + /// Cell/upstream configuration /// Note: The cell name is the HashMap key in Config.locales #[derive(Clone, Debug, Deserialize, PartialEq)] pub struct CellConfig { + /// Name/identifier of the cell + pub name: String, /// URL of the Sentry upstream server pub sentry_url: Url, /// URL of the Relay upstream server @@ -80,12 +136,14 @@ pub struct Config { pub admin_listener: AdminListener, /// Maps locale identifiers to their cells (cell name -> cell config) /// - /// Note: Uses String keys instead of an enum to allow flexible, - /// deployment-specific locale configuration without code changes. - /// Different deployments may use different locale identifiers. - pub locales: HashMap>, + /// Cells are stored as a Vec to maintain priority order - the first cell + /// in the list has highest priority for global config responses. + pub locales: HashMap>, /// Request routing rules pub routes: Vec, + /// Timeout configuration for relay handlers + #[serde(default)] + pub relay_timeouts: RelayTimeouts, } impl Config { @@ -95,6 +153,9 @@ impl Config { self.listener.validate()?; self.admin_listener.validate()?; + // Validate timeouts + self.relay_timeouts.validate()?; + // Validate locales and cells for (locale, cells) in &self.locales { // Check that locale has at least one cell @@ -102,11 +163,15 @@ impl Config { return Err(ValidationError::LocaleHasNoValidCells(locale.clone())); } - // Check for empty cell names (HashMap keys) - for cell_name in cells.keys() { - if cell_name.is_empty() { + // Check for empty cell names and collect for duplicate checking + let mut seen_names = HashSet::new(); + for cell in cells { + if cell.name.is_empty() { return Err(ValidationError::EmptyUpstreamName); } + if !seen_names.insert(&cell.name) { + return Err(ValidationError::DuplicateUpstream(cell.name.clone())); + } } } @@ -226,16 +291,16 @@ admin_listener: port: 3001 locales: us: - us1: - sentry_url: "http://127.0.0.1:8080" - relay_url: "http://127.0.0.1:8090" - us2: - sentry_url: "http://10.0.0.2:8080" - relay_url: "http://10.0.0.2:8090" + - name: us1 + sentry_url: "http://127.0.0.1:8080" + relay_url: "http://127.0.0.1:8090" + - name: us2 + sentry_url: "http://10.0.0.2:8080" + relay_url: "http://10.0.0.2:8090" de: - de1: - sentry_url: "http://10.0.0.3:8080" - relay_url: "http://10.0.0.3:8090" + - name: de1 + sentry_url: "http://10.0.0.3:8080" + relay_url: "http://10.0.0.3:8090" routes: - match: host: us.sentry.io @@ -261,6 +326,8 @@ routes: assert_eq!(config.locales.len(), 2); assert_eq!(config.locales.get("us").unwrap().len(), 2); assert_eq!(config.locales.get("de").unwrap().len(), 1); + assert_eq!(config.locales.get("us").unwrap()[0].name, "us1"); + assert_eq!(config.locales.get("us").unwrap()[1].name, "us2"); assert_eq!(config.routes.len(), 2); assert_eq!(config.routes[0].r#match.method, Some(HttpMethod::Post)); assert_eq!(config.routes[1].r#match.host, None); @@ -285,14 +352,13 @@ routes: }, locales: HashMap::from([( "us".to_string(), - HashMap::from([( - "us1".to_string(), - CellConfig { - sentry_url: Url::parse("http://127.0.0.1:8080").unwrap(), - relay_url: Url::parse("http://127.0.0.1:8090").unwrap(), - }, - )]), + vec![CellConfig { + name: "us1".to_string(), + sentry_url: Url::parse("http://127.0.0.1:8080").unwrap(), + relay_url: Url::parse("http://127.0.0.1:8090").unwrap(), + }], )]), + relay_timeouts: RelayTimeouts::default(), routes: vec![Route { r#match: Match { path: Some("/api/".to_string()), @@ -315,18 +381,28 @@ routes: // Test empty cell name let mut config = base_config.clone(); - config.locales.get_mut("us").unwrap().insert( - "".to_string(), - CellConfig { - sentry_url: Url::parse("http://10.0.0.2:8080").unwrap(), - relay_url: Url::parse("http://10.0.0.2:8090").unwrap(), - }, - ); + config.locales.get_mut("us").unwrap().push(CellConfig { + name: "".to_string(), + sentry_url: Url::parse("http://10.0.0.2:8080").unwrap(), + relay_url: Url::parse("http://10.0.0.2:8090").unwrap(), + }); assert!(matches!( config.validate().unwrap_err(), ValidationError::EmptyUpstreamName )); + // Test duplicate cell name + let mut config = base_config.clone(); + config.locales.get_mut("us").unwrap().push(CellConfig { + name: "us1".to_string(), + sentry_url: Url::parse("http://10.0.0.2:8080").unwrap(), + relay_url: Url::parse("http://10.0.0.2:8090").unwrap(), + }); + assert!(matches!( + config.validate().unwrap_err(), + ValidationError::DuplicateUpstream(_) + )); + // Test unknown locale in action let mut config = base_config.clone(); config.routes[0].action = HandlerAction::RelayProjectConfigs(RelayProjectConfigsArgs { @@ -341,11 +417,35 @@ routes: let mut config = base_config.clone(); config .locales - .insert("invalid_locale".to_string(), HashMap::new()); + .insert("invalid_locale".to_string(), Vec::new()); assert!(matches!( config.validate().unwrap_err(), ValidationError::LocaleHasNoValidCells(_) )); + + // Test invalid timeouts: task_initial < http + let mut config = base_config.clone(); + config.relay_timeouts = RelayTimeouts { + http_timeout_secs: 20, + task_initial_timeout_secs: 15, // Less than HTTP timeout + task_subsequent_timeout_secs: 5, + }; + assert!(matches!( + config.validate().unwrap_err(), + ValidationError::InvalidTimeouts(_) + )); + + // Test invalid timeouts: task_subsequent = 0 + let mut config = base_config.clone(); + config.relay_timeouts = RelayTimeouts { + http_timeout_secs: 15, + task_initial_timeout_secs: 20, + task_subsequent_timeout_secs: 0, // Zero timeout + }; + assert!(matches!( + config.validate().unwrap_err(), + ValidationError::InvalidTimeouts(_) + )); } #[test] diff --git a/ingest-router/src/errors.rs b/ingest-router/src/errors.rs index ad05775..fb9dda7 100644 --- a/ingest-router/src/errors.rs +++ b/ingest-router/src/errors.rs @@ -1,3 +1,6 @@ +use http_body_util::{BodyExt, Full, combinators::BoxBody}; +use hyper::body::Bytes; +use hyper::{Response, StatusCode}; use thiserror::Error; /// Errors that can occur during ingest-router operations @@ -24,6 +27,68 @@ pub enum IngestRouterError { #[error("Internal error: {0}")] InternalError(String), + #[error("Service unavailable: {0}")] + ServiceUnavailable(String), + #[error("IO error: {0}")] Io(#[from] std::io::Error), } + +impl IngestRouterError { + /// Returns the appropriate HTTP status code for this error + pub fn status_code(&self) -> StatusCode { + match self { + IngestRouterError::RequestBodyError(_) => StatusCode::BAD_REQUEST, + IngestRouterError::ResponseBodyError(_) => StatusCode::BAD_GATEWAY, + IngestRouterError::UpstreamRequestFailed(_, _) => StatusCode::BAD_GATEWAY, + IngestRouterError::UpstreamTimeout(_) => StatusCode::GATEWAY_TIMEOUT, + IngestRouterError::ResponseSerializationError(_) => StatusCode::INTERNAL_SERVER_ERROR, + IngestRouterError::ResponseBuildError(_) => StatusCode::INTERNAL_SERVER_ERROR, + IngestRouterError::InternalError(_) => StatusCode::INTERNAL_SERVER_ERROR, + IngestRouterError::ServiceUnavailable(_) => StatusCode::SERVICE_UNAVAILABLE, + IngestRouterError::Io(_) => StatusCode::INTERNAL_SERVER_ERROR, + } + } + + /// Converts this error into an HTTP response + pub fn into_response(self) -> Response> { + let status = self.status_code(); + let body = format!("{}\n", self); + + Response::builder() + .status(status) + .body(Full::new(Bytes::from(body)).map_err(|e| match e {}).boxed()) + .unwrap_or_else(|_| { + // Fallback if response building fails + Response::builder() + .status(StatusCode::INTERNAL_SERVER_ERROR) + .body( + Full::new(Bytes::from("Internal server error\n")) + .map_err(|e| match e {}) + .boxed(), + ) + .unwrap() + }) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[tokio::test] + async fn test_error_into_response() { + use http_body_util::BodyExt; + + // Test that errors convert to proper HTTP responses + let error = IngestRouterError::ServiceUnavailable("Test unavailable".to_string()); + let response = error.into_response(); + + assert_eq!(response.status(), StatusCode::SERVICE_UNAVAILABLE); + + // Verify body contains error message + let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); + let body_str = String::from_utf8(body_bytes.to_vec()).unwrap(); + assert!(body_str.contains("Test unavailable")); + } +} diff --git a/ingest-router/src/lib.rs b/ingest-router/src/lib.rs index 13cf432..3dc644f 100644 --- a/ingest-router/src/lib.rs +++ b/ingest-router/src/lib.rs @@ -5,7 +5,7 @@ pub mod locale; pub mod relay_project_config_handler; pub mod router; -use crate::config::{CellConfig, Config, Route}; +use crate::config::{CellConfig, Config, RelayTimeouts, Route}; use crate::errors::IngestRouterError; use crate::relay_project_config_handler::RelayProjectConfigsHandler; use crate::router::Router; @@ -19,7 +19,11 @@ use std::collections::HashMap; use std::pin::Pin; pub async fn run(config: Config) -> Result<(), errors::IngestRouterError> { - let router_service = IngestRouterService::new(config.routes.clone(), config.locales.clone()); + let router_service = IngestRouterService::new( + config.routes.clone(), + config.locales.clone(), + config.relay_timeouts.clone(), + ); let router_task = run_http_service(&config.listener.host, config.listener.port, router_service); router_task.await?; Ok(()) @@ -31,8 +35,12 @@ struct IngestRouterService { } impl IngestRouterService { - fn new(routes: Vec, locales: HashMap>) -> Self { - let handler = RelayProjectConfigsHandler::new(locales); + fn new( + routes: Vec, + locales: HashMap>, + relay_timeouts: RelayTimeouts, + ) -> Self { + let handler = RelayProjectConfigsHandler::new(locales, relay_timeouts); Self { router: Router::new(routes, handler), } diff --git a/ingest-router/src/locale.rs b/ingest-router/src/locale.rs index c50130f..6f16f7b 100644 --- a/ingest-router/src/locale.rs +++ b/ingest-router/src/locale.rs @@ -49,17 +49,28 @@ impl From for Upstream { /// Collection of upstreams grouped by cell name #[derive(Clone, Debug)] pub struct Cells { + /// Prioritized list of cell names (first = highest priority) + pub cell_list: Vec, pub cell_to_upstreams: HashMap, } impl Cells { /// Build cells from cell configurations - fn from_config(cell_configs: HashMap) -> Self { + fn from_config(cell_configs: Vec) -> Self { + let mut cell_list = Vec::new(); + let mut cell_to_upstreams = HashMap::new(); + + for config in cell_configs { + let name = config.name.clone(); + let upstream = Upstream::from(config); + + cell_list.push(name.clone()); + cell_to_upstreams.insert(name, upstream); + } + Self { - cell_to_upstreams: cell_configs - .into_iter() - .map(|(name, config)| (name, config.into())) - .collect(), + cell_list, + cell_to_upstreams, } } } @@ -73,7 +84,7 @@ pub struct Locales { impl Locales { /// Build locale mappings from configuration - pub fn new(locales: HashMap>) -> Self { + pub fn new(locales: HashMap>) -> Self { // Build locale -> cells mapping let locale_to_cells = locales .into_iter() @@ -98,6 +109,7 @@ mod tests { fn cell_config(sentry_url: &str, relay_url: &str) -> CellConfig { CellConfig { + name: String::new(), sentry_url: Url::parse(sentry_url).unwrap(), relay_url: Url::parse(relay_url).unwrap(), } @@ -108,32 +120,35 @@ mod tests { let mut locales_config = HashMap::new(); locales_config.insert( "us".to_string(), - HashMap::from([ - ( - "us1".to_string(), - cell_config( + vec![ + { + let mut config = cell_config( "http://us1-sentry.example.com", "http://us1-relay.example.com", - ), - ), - ( - "us2".to_string(), - cell_config( + ); + config.name = "us1".to_string(); + config + }, + { + let mut config = cell_config( "http://us2-sentry.example.com", "http://us2-relay.example.com", - ), - ), - ]), + ); + config.name = "us2".to_string(); + config + }, + ], ); locales_config.insert( "de".to_string(), - HashMap::from([( - "de".to_string(), - cell_config( + vec![{ + let mut config = cell_config( "http://de-sentry.example.com", "http://de-relay.example.com", - ), - )]), + ); + config.name = "de".to_string(); + config + }], ); let locales = Locales::new(locales_config); @@ -141,13 +156,19 @@ mod tests { // Verify US locale has 2 cells let us_cells = locales.get_cells("us").unwrap(); assert_eq!(us_cells.cell_to_upstreams.len(), 2); + assert_eq!(us_cells.cell_list.len(), 2); assert!(us_cells.cell_to_upstreams.contains_key("us1")); assert!(us_cells.cell_to_upstreams.contains_key("us2")); + // Verify priority order (us1 is first) + assert_eq!(us_cells.cell_list[0], "us1"); + assert_eq!(us_cells.cell_list[1], "us2"); // Verify DE locale has 1 cell let de_cells = locales.get_cells("de").unwrap(); assert_eq!(de_cells.cell_to_upstreams.len(), 1); + assert_eq!(de_cells.cell_list.len(), 1); assert!(de_cells.cell_to_upstreams.contains_key("de")); + assert_eq!(de_cells.cell_list[0], "de"); // Verify unknown locale returns None assert!(locales.get_cells("unknown").is_none()); diff --git a/ingest-router/src/relay_project_config_handler.rs b/ingest-router/src/relay_project_config_handler.rs index 681485b..7f5033c 100644 --- a/ingest-router/src/relay_project_config_handler.rs +++ b/ingest-router/src/relay_project_config_handler.rs @@ -7,18 +7,19 @@ //! # Protocol Overview //! //! The Relay Project Configs endpoint (version 3) is used by Relay instances to fetch -//! project configurations needed to process events. This implementation acts as a proxy -//! that: +//! project configurations needed to process events. This implementation acts as an +//! aggregation layer that: //! 1. Splits requests across multiple upstream Sentry cells based on project ownership //! 2. Fans out requests in parallel to multiple upstreams //! 3. Merges responses back into a single v3-compliant response //! 4. Passes through all config data unchanged //! //! ## Endpoint Details +//! The endpoint implementation is at https://github.com/getsentry/sentry/blob/master/src/sentry/api/endpoints/relay/project_configs.py //! //! - **Path**: `/api/0/relays/projectconfigs/` //! - **Method**: `POST` -//! - **Protocol Version**: 3 (current) +//! - **Protocol Version**: 3 //! - **Authentication**: RelayAuthentication (X-Sentry-Relay-Id, X-Sentry-Relay-Signature) //! //! # Request Format (Version 3) @@ -85,11 +86,10 @@ //! - Keys routed to the same cell are batched into one request //! //! 3. **Handle global config flag** -//! - First upstream gets `global: true` (or original value) -//! - All other upstreams get `global: false` -//! - Prevents complex global config merging (only one upstream returns it) -//! - TODO: Add capability to send to both but return only from first. This would -//! enable a failover mechanism. +//! - All upstreams receive the same `global` flag value as the original request +//! - Global config is selected from the highest priority cell that returns it +//! - Priority is determined by cell order in configuration (first = highest priority) +//! - Enables failover: if highest priority cell fails, next cell's global config is used //! //! ## Response Merging Strategy //! @@ -105,9 +105,9 @@ //! - Include keys from failed/timed-out upstreams //! - Relay will retry these keys in a subsequent request //! -//! ### Extra fields (HashMap merge) -//! - Merge `extra` fields (includes `global`, `global_status`, future fields) -//! - No conflicts expected (only first upstream has global config) +//! ### Extra fields (Priority-based selection) +//! - Select `extra` fields from highest priority cell that responds successfully +//! - Priority determined by cell order in configuration (first = highest) //! - Forward compatibility: new fields are automatically preserved //! //! ## Error Handling @@ -120,7 +120,7 @@ //! ### Total Failure //! - If all upstreams fail: Check if any keys were added to pending //! - If pending is not empty: Return 200 OK with pending array (relay will retry) -//! - If pending is empty: Return 500 error (no recoverable state) +//! - If pending is empty: Return 503 error (no recoverable state) //! //! ### Upstream Failure Scenarios //! - **Timeout**: All keys from that upstream → pending @@ -151,7 +151,7 @@ //! └───┬──────────────────────────┬───────┘ //! │ │ //! │ {publicKeys: [A,C,E], │ {publicKeys: [B,D,F], -//! │ global: true} │ global: false} +//! │ global: true} │ global: true} //! │ │ //! ▼ ▼ //! ┌──────────┐ ┌──────────┐ @@ -169,7 +169,7 @@ //! │ 3. Merge responses: │ //! │ • Combine all configs │ //! │ • Merge pending arrays │ -//! │ • Merge extra fields (global) │ +//! │ • Select others from priority │ //! └──────────────┬───────────────────────┘ //! │ //! │ {configs: {A,B,C,D,E,F}, @@ -269,7 +269,9 @@ //! //! **Splitting**: //! - Request to US1: `{"publicKeys": ["key1"], "global": true}` -//! - Request to US2: `{"publicKeys": ["key2"], "global": false}` +//! - Request to US2: `{"publicKeys": ["key2"], "global": true}` +//! +//! (US1 has higher priority, so its global config will be used in the final response) //! //! **Response**: //! ```json @@ -284,13 +286,13 @@ //! //! - [`RelayProjectConfigsHandler`] - Main handler struct for processing requests -use crate::config::CellConfig; +use crate::config::{CellConfig, RelayTimeouts}; use crate::errors::IngestRouterError; use crate::http::send_to_upstream; -use crate::locale::{Locales, Upstream}; +use crate::locale::{Cells, Locales, Upstream}; use http_body_util::{BodyExt, Full, combinators::BoxBody}; use hyper::body::{Body, Bytes}; -use hyper::header::CONTENT_TYPE; +use hyper::header::{CONTENT_LENGTH, CONTENT_TYPE, HeaderMap}; use hyper::{Request, Response, StatusCode}; use hyper_util::client::legacy::Client; use hyper_util::client::legacy::connect::HttpConnector; @@ -300,7 +302,7 @@ use serde_json::Value as JsonValue; use std::collections::HashMap; use std::time::Duration; use tokio::task::JoinHandle; -use tokio::time::timeout; +use tokio::time::{Instant, timeout}; /// Request format for the relay project configs endpoint. /// @@ -311,13 +313,8 @@ struct RelayProjectConfigsRequest { #[serde(rename = "publicKeys")] pub public_keys: Vec, - /// Whether to include global config (optional). - /// - /// first upstream gets original value, others get `Some(false)`. - #[serde(skip_serializing_if = "Option::is_none")] - pub global: Option, - - /// Other fields (`noCache`, future fields) for forward compatibility. + /// Other fields (`global`, `noCache`, future fields) for forward compatibility. + /// All fields are passed through as-is to upstreams. #[serde(flatten)] pub extra: HashMap, } @@ -358,6 +355,8 @@ impl RelayProjectConfigsResponse { /// Result from an upstream task request. struct UpstreamTaskResult { + /// The cell name this result is from. + cell_name: String, /// The upstream URL that was contacted. sentry_url: String, /// The public keys that were requested from this upstream. @@ -374,6 +373,8 @@ struct MergedResults { pending: Vec, /// Extra fields (global config, status, etc.). extra: HashMap, + /// Headers from upstream + headers: HeaderMap, } /// Task handle paired with its public keys for graceful failure handling. @@ -387,181 +388,53 @@ struct TaskWithKeys { public_keys: Vec, } -/// Handler for Relay Project Configs endpoint. +/// A request split for a specific upstream cell. +struct SplitRequest { + /// The cell name this request is for + cell_name: String, + /// The upstream to send the request to + upstream: Upstream, + /// The request body to send + request: RelayProjectConfigsRequest, +} + +/// Executor for spawning and collecting upstream tasks. /// -/// See module-level docs for complete protocol details, implementation strategy, -/// and request/response flow diagrams. -pub struct RelayProjectConfigsHandler { +/// Encapsulates the parallel task execution logic for fanning out requests +/// to multiple upstream Sentry instances and collecting/merging their results. +struct UpstreamTaskExecutor { /// HTTP client for sending requests to upstream Sentry instances. client: Client>, - /// Locales mapping for locale-based upstream lookups. - /// - /// Maps locale → cell name → upstream (relay URL + sentry URL). - locales: Locales, + /// Timeout configuration for HTTP and task-level timeouts. + timeouts: RelayTimeouts, } -impl RelayProjectConfigsHandler { - pub fn new(locales_config: HashMap>) -> Self { - let connector = HttpConnector::new(); - let client = Client::builder(TokioExecutor::new()).build(connector); - - // Build locales from config - let locales = Locales::new(locales_config); - - Self { client, locales } +impl UpstreamTaskExecutor { + fn new(client: Client>, timeouts: RelayTimeouts) -> Self { + Self { client, timeouts } } - pub async fn handle( - &self, - locale: &str, - request: Request, - ) -> Result>, IngestRouterError> - where - B: Body + Send + 'static, - B::Data: Send, - B::Error: std::error::Error + Send + Sync + 'static, - { - // Get cells for this locale - let cells = self.locales.get_cells(locale).ok_or_else(|| { - IngestRouterError::InternalError(format!( - "No targets configured for locale: {}", - locale - )) - })?; - - // Buffer the request body. We need to full body in order to do request massaging. - let (parts, body) = request.into_parts(); - let body_bytes = body - .collect() - .await - .map(|collected| collected.to_bytes()) - .map_err(|e| IngestRouterError::RequestBodyError(e.to_string()))?; - let base_request = Request::from_parts(parts, ()); - - // Process the request - self.handle_with_targets(&cells.cell_to_upstreams, base_request, body_bytes) - .await - } - - /// Internal method that orchestrates the split-merge flow. + /// Spawns parallel tasks to fan out requests to multiple upstreams. /// - /// High-level steps: - /// 1. Parse and split the request across upstreams - /// 2. Spawn parallel tasks to fan out requests - /// 3. Collect and merge results from all upstreams - /// 4. Build and return the merged response - async fn handle_with_targets( - &self, - cell_upstreams: &HashMap, - base_request: Request<()>, - body_bytes: Bytes, - ) -> Result>, IngestRouterError> { - // Parse the request - let request_data = RelayProjectConfigsRequest::from_bytes(&body_bytes).map_err(|e| { - IngestRouterError::RequestBodyError(format!("Failed to parse request: {e}")) - })?; - - // Split publicKeys across upstreams - let split_requests = self.split_keys_by_upstream(&request_data, cell_upstreams); - - // Spawn tasks to fan out requests in parallel - let tasks = self.spawn_upstream_tasks(split_requests, &base_request)?; - - // Collect and merge results from all tasks - let results = self.collect_task_results(tasks).await; - - // Only return an error if we have no configs AND no pending AND we had keys to request - // Having keys in pending is a valid v3 response (relay will retry later) - if results.configs.is_empty() - && results.pending.is_empty() - && !request_data.public_keys.is_empty() - { - return Err(IngestRouterError::InternalError( - "All upstream requests failed with no recoverable state".to_string(), - )); - } - - // Build merged response in the relay format - self.build_merged_response(results.configs, results.pending, results.extra) - } - - /// Splits public keys across multiple upstream cells. + /// Each task performs an HTTP request to an upstream Sentry instance with + /// the appropriate subset of public keys. /// - /// Current: Round-robin stub. TODO: Replace with locator service lookup. - /// See module-level docs for complete splitting strategy and global config handling. - fn split_keys_by_upstream( + /// Uses two-layer timeout strategy: + /// - HTTP timeout: Applied to individual HTTP requests + /// - Task timeout: Applied at collection level with adaptive strategy + fn spawn_tasks( &self, - request: &RelayProjectConfigsRequest, - cell_upstreams: &HashMap, - ) -> Vec<(Upstream, RelayProjectConfigsRequest)> { - if cell_upstreams.is_empty() { - return Vec::new(); - } - - // For now, convert HashMap to Vec for round-robin stub - // In the future, we'll use cell_upstreams.get(cell_name) directly - let upstreams: Vec<&Upstream> = cell_upstreams.values().collect(); - - let mut split: HashMap> = HashMap::new(); - - // Round-robin stub: distribute publicKeys evenly across upstreams - // TODO: Replace with control plane lookup per key - for (index, public_key) in request.public_keys.iter().enumerate() { - let upstream_index = index % upstreams.len(); - - split - .entry(upstream_index) - .or_default() - .push(public_key.clone()); - } - - // Build a request for each upstream with its assigned publicKeys - // Sort by upstream_index to ensure deterministic ordering - let mut sorted_split: Vec<_> = split.into_iter().collect(); - sorted_split.sort_by_key(|(upstream_index, _)| *upstream_index); - - sorted_split - .into_iter() - .map(|(upstream_index, public_keys)| { - let upstream = upstreams[upstream_index].clone(); - - // For v3 protocol: set global=true for only the first upstream (index 0) - // This avoids needing to merge global configs from multiple responses - let global = if upstream_index == 0 { - // First upstream (index 0): keep global flag as-is from original request - request.global - } else { - // Other upstreams: explicitly set global=false - Some(false) - }; - - let split_request = RelayProjectConfigsRequest { - public_keys, - global, - extra: request.extra.clone(), - }; - (upstream, split_request) - }) - .collect() - } - - /// Spawn async tasks to send requests to all upstreams in parallel. - /// - /// Each task sends the split request to its designated upstream and returns - /// an `UpstreamTaskResult` containing the response or error. - /// - /// Returns tuples of (JoinHandle, public_keys) so keys can be added to pending - /// if the task times out or panics. - fn spawn_upstream_tasks( - &self, - split_requests: Vec<(Upstream, RelayProjectConfigsRequest)>, + split_requests: Vec, base_request: &Request<()>, ) -> Result, IngestRouterError> { let mut tasks = Vec::new(); - for (upstream, split_request) in split_requests { - // Track public keys for this upstream (needed if request fails) + for split in split_requests { + let cell_name = split.cell_name; + let upstream = split.upstream; + let split_request = split.request; + // Track public keys for this upstream let public_keys = split_request.public_keys.clone(); // Serialize the split request body @@ -585,24 +458,27 @@ impl RelayProjectConfigsHandler { let client = self.client.clone(); let sentry_url = upstream.sentry_url.clone(); + let http_timeout_secs = self.timeouts.http_timeout_secs as u64; - // Clone keys so we can track them even if task times out + // Clone data we need to track even if task times out let public_keys_for_task = public_keys.clone(); + let cell_name_for_task = cell_name.clone(); // Spawn a task for each upstream request let task = tokio::spawn(async move { let sentry_url_str = sentry_url.to_string(); let result: Result, IngestRouterError> = - send_to_upstream(&client, &sentry_url, request, 30).await; + send_to_upstream(&client, &sentry_url, request, http_timeout_secs).await; UpstreamTaskResult { + cell_name: cell_name_for_task, sentry_url: sentry_url_str, public_keys: public_keys_for_task, result, } }); - // Store both task handle and keys (keys needed if task times out) + // Store task handle and keys (keys needed if task times out) tasks.push(TaskWithKeys { handle: task, public_keys, @@ -612,25 +488,54 @@ impl RelayProjectConfigsHandler { Ok(tasks) } - /// Collect results from all spawned tasks and merge them. + /// Collects results from all spawned tasks and merges them. + /// + /// Uses adaptive timeouts with cutoff strategy: + /// - Initial: Wait up to task_initial_timeout_secs for first upstream to respond + /// - Subsequent: Once first succeeds, ALL remaining tasks have task_subsequent_timeout_secs + /// total (from first success) to complete. This prevents slow/down cells from blocking + /// progress when we already have good data. /// - /// Handles timeouts, task panics, and HTTP failures gracefully. /// Failed keys are added to the pending array for retry. - async fn collect_task_results(&self, tasks: Vec) -> MergedResults { + /// + /// Global config is selected from the highest priority cell based on cells.cell_list order. + async fn collect_and_merge(&self, tasks: Vec, cells: &Cells) -> MergedResults { let mut merged_configs = HashMap::new(); let mut merged_pending = Vec::new(); - let mut merged_extra = HashMap::new(); + let mut extra_by_cell: HashMap> = HashMap::new(); + let mut headers_by_cell: HashMap = HashMap::new(); + let mut additional_deadline: Option = None; for task_with_keys in tasks { let TaskWithKeys { mut handle, public_keys, } = task_with_keys; - let task_result = timeout(Duration::from_secs(30), &mut handle).await; + + // Adaptive timeout + // - Before first success: Use task_initial_timeout_secs + // - After first success: Use remaining time until deadline + let timeout_duration = if let Some(deadline) = additional_deadline { + // Calculate remaining time to deadline + let now = Instant::now(); + if now >= deadline { + // Deadline already passed, use minimal timeout + Duration::from_millis(1) + } else { + deadline.duration_since(now) + } + } else { + Duration::from_secs(self.timeouts.task_initial_timeout_secs as u64) + }; + + let task_result = timeout(timeout_duration, &mut handle).await; // Handle timeout let Ok(join_result) = task_result else { - tracing::error!("Task timed out after 30 seconds"); + tracing::error!( + "Task timed out after {} seconds", + timeout_duration.as_secs() + ); // Abort the timed-out task to prevent it from continuing in background handle.abort(); // Add all keys from this upstream to pending (v3 protocol) @@ -649,33 +554,63 @@ impl RelayProjectConfigsHandler { }; // Process the upstream result - self.process_upstream_result( + let result_had_configs = upstream_result.result.is_ok(); + let cell_name_from_result = upstream_result.cell_name.clone(); + if let Some((extra, headers)) = self.process_upstream_result( upstream_result, &mut merged_configs, &mut merged_pending, - &mut merged_extra, - ); + ) { + // Store extra fields and headers by cell name for later priority-based selection + extra_by_cell.insert(cell_name_from_result.clone(), extra); + headers_by_cell.insert(cell_name_from_result, headers); + } + + // Set deadline on first success + if result_had_configs && additional_deadline.is_none() { + additional_deadline = Some( + Instant::now() + + Duration::from_secs(self.timeouts.task_subsequent_timeout_secs as u64), + ); + } } + // Select global config and headers from highest priority cell by iterating cells.cell_list + // cell_list is already in priority order (first = highest priority) + let (merged_extra, merged_headers) = cells + .cell_list + .iter() + .find_map(|cell_name| { + extra_by_cell.get(cell_name).cloned().map(|extra| { + let headers = headers_by_cell.get(cell_name).cloned().unwrap_or_default(); + (extra, headers) + }) + }) + .unwrap_or_default(); + MergedResults { configs: merged_configs, pending: merged_pending, extra: merged_extra, + headers: merged_headers, } } - /// Process the result from a single upstream task. + /// Processes the result from a single upstream task. /// /// Handles both successful and failed upstream responses, merging successful /// configs and adding failed keys to the pending array. + /// + /// Returns the extra fields (global config, etc.) and headers if the request succeeded, + /// which allows the caller to select based on cell priority. fn process_upstream_result( &self, upstream_result: UpstreamTaskResult, merged_configs: &mut HashMap, merged_pending: &mut Vec, - merged_extra: &mut HashMap, - ) { + ) -> Option<(HashMap, HeaderMap)> { let UpstreamTaskResult { + cell_name: _, sentry_url, public_keys, result, @@ -690,11 +625,14 @@ impl RelayProjectConfigsHandler { ); // Add all keys from this upstream to pending (v3 protocol) merged_pending.extend(public_keys); - return; + return None; }; + // Extract headers before consuming the response + let (parts, body) = response.into_parts(); + let headers = parts.headers; + // Parse response body - let body = response.into_body(); match RelayProjectConfigsResponse::from_bytes(&body) { Ok(response_data) => { // Merge configs from this upstream @@ -705,8 +643,8 @@ impl RelayProjectConfigsHandler { merged_pending.extend(pending); } - // Merge extra fields (global, global_status, future fields) - merged_extra.extend(response_data.extra); + // Return extra fields and headers for priority-based selection + Some((response_data.extra, headers)) } Err(e) => { tracing::error!( @@ -716,16 +654,178 @@ impl RelayProjectConfigsHandler { ); // Add all keys from this upstream to pending merged_pending.extend(public_keys); + None } } } +} + +/// Handler for Relay Project Configs endpoint. +/// +/// See module-level docs for complete protocol details, implementation strategy, +/// and request/response flow diagrams. +pub struct RelayProjectConfigsHandler { + /// Executor for spawning and collecting upstream tasks. + executor: UpstreamTaskExecutor, + + /// Locales mapping for locale-based upstream lookups. + /// + /// Maps locale → cell name → upstream (relay URL + sentry URL). + locales: Locales, +} + +impl RelayProjectConfigsHandler { + pub fn new(locales_config: HashMap>, timeouts: RelayTimeouts) -> Self { + let connector = HttpConnector::new(); + let client = Client::builder(TokioExecutor::new()).build(connector); + + // Build locales from config + let locales = Locales::new(locales_config); + + // Create executor for task management + let executor = UpstreamTaskExecutor::new(client, timeouts); + + Self { executor, locales } + } - /// Build a merged response from collected configs in relay format + pub async fn handle( + &self, + locale: &str, + request: Request, + ) -> Result>, IngestRouterError> + where + B: Body + Send + 'static, + B::Data: Send, + B::Error: std::error::Error + Send + Sync + 'static, + { + // Get cells for this locale + let cells = self.locales.get_cells(locale).ok_or_else(|| { + IngestRouterError::InternalError(format!( + "No targets configured for locale: {}", + locale + )) + })?; + + // Buffer the request body. We need to full body in order to do request massaging. + let (parts, body) = request.into_parts(); + let body_bytes = body + .collect() + .await + .map(|collected| collected.to_bytes()) + .map_err(|e| IngestRouterError::RequestBodyError(e.to_string()))?; + let base_request = Request::from_parts(parts, ()); + + // Process the request + self.handle_with_targets(cells, base_request, body_bytes) + .await + } + + /// Internal method that orchestrates the split-merge flow. + /// + /// High-level steps: + /// 1. Parse and split the request across upstreams + /// 2. Spawn parallel tasks to fan out requests + /// 3. Collect and merge results from all upstreams + /// 4. Build and return the merged response + async fn handle_with_targets( + &self, + cells: &Cells, + base_request: Request<()>, + body_bytes: Bytes, + ) -> Result>, IngestRouterError> { + // Parse the request + let request_data = RelayProjectConfigsRequest::from_bytes(&body_bytes).map_err(|e| { + IngestRouterError::RequestBodyError(format!("Failed to parse request: {e}")) + })?; + + // Split publicKeys across upstreams + let split_requests = self.split_keys_by_upstream(&request_data, cells); + + // Spawn tasks to fan out requests in parallel + let tasks = self.executor.spawn_tasks(split_requests, &base_request)?; + + // Collect and merge results from all tasks (uses cells.cell_list priority order) + let results = self.executor.collect_and_merge(tasks, cells).await; + + // Only return an error if we have no configs AND no pending AND we had keys to request + // Having keys in pending is a valid v3 response (relay will retry later) + // Return 503 Service Unavailable to indicate temporary unavailability + if results.configs.is_empty() + && results.pending.is_empty() + && !request_data.public_keys.is_empty() + { + return Err(IngestRouterError::ServiceUnavailable( + "All upstream cells are unavailable".to_string(), + )); + } + + // Build merged response in the relay format + self.build_merged_response( + results.configs, + results.pending, + results.extra, + results.headers, + ) + } + + /// Splits public keys across multiple upstream cells. + /// + /// Current: Round-robin stub. TODO: Replace with locator service lookup. + /// See module-level docs for complete splitting strategy and global config handling. + fn split_keys_by_upstream( + &self, + request: &RelayProjectConfigsRequest, + cells: &Cells, + ) -> Vec { + let cell_list = &cells.cell_list; + if cell_list.is_empty() { + return Vec::new(); + } + + let mut split: HashMap> = HashMap::new(); + + // TODO: Replace with control plane lookup per key + for (index, public_key) in request.public_keys.iter().enumerate() { + let cell_name = &cell_list[index % cell_list.len()]; + + split + .entry(cell_name.clone()) + .or_default() + .push(public_key.clone()); + } + + // Build a request for each cell with its assigned publicKeys + split + .into_iter() + .map(|(cell_name, public_keys)| { + let upstream = cells + .cell_to_upstreams + .get(&cell_name) + .expect("Cell name in list must exist in HashMap"); + + // All fields in extra (including global, noCache, etc.) are passed through as-is + SplitRequest { + cell_name, + upstream: upstream.clone(), + request: RelayProjectConfigsRequest { + public_keys, + extra: request.extra.clone(), + }, + } + }) + .collect() + } + + /// Build a merged response from collected configs in relay format. + /// + /// Uses headers from the highest priority cell (same cell used for global config). + /// Filters out hop-by-hop headers and Content-Length (which is recalculated for the new body). fn build_merged_response( &self, merged_configs: HashMap, merged_pending: Vec, merged_extra: HashMap, + mut upstream_headers: HeaderMap, ) -> Result>, IngestRouterError> { // Wrap in relay response format let response = RelayProjectConfigsResponse { @@ -741,9 +841,25 @@ impl RelayProjectConfigsHandler { let merged_json = serde_json::to_vec(&response) .map_err(|e| IngestRouterError::ResponseSerializationError(e.to_string()))?; - Response::builder() - .status(StatusCode::OK) - .header(CONTENT_TYPE, "application/json") + // Filter hop-by-hop headers from upstream (assumes HTTP/1.1) + // These headers are connection-specific and shouldn't be forwarded + shared::http::filter_hop_by_hop(&mut upstream_headers, hyper::Version::HTTP_11); + + // Remove Content-Length since we're creating a new body with different length + upstream_headers.remove(CONTENT_LENGTH); + + // Build response with filtered headers from highest priority cell + let mut builder = Response::builder().status(StatusCode::OK); + + // Copy filtered headers from upstream + for (name, value) in upstream_headers.iter() { + builder = builder.header(name, value); + } + + // Always set/override Content-Type to ensure it's correct + builder = builder.header(CONTENT_TYPE, "application/json"); + + builder .body( Full::new(Bytes::from(merged_json)) .map_err(|e| match e {}) @@ -759,19 +875,19 @@ mod tests { use url::Url; #[tokio::test] - async fn test_build_merged_response() { - let handler = RelayProjectConfigsHandler::new(HashMap::new()); + async fn test_response_building_and_v3_protocol() { + let handler = RelayProjectConfigsHandler::new(HashMap::new(), RelayTimeouts::default()); - // Test 1: Empty response + // Test: Empty response let response = handler - .build_merged_response(HashMap::new(), Vec::new(), HashMap::new()) + .build_merged_response(HashMap::new(), Vec::new(), HashMap::new(), HeaderMap::new()) .unwrap(); assert_eq!(response.status(), StatusCode::OK); let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); assert_eq!(parsed, serde_json::json!({"configs": {}})); - // Test 2: Multiple configs with all fields preserved (pass-through verification) + // Test: Multiple configs with field preservation let mut configs = HashMap::new(); configs.insert( "project1".to_string(), @@ -789,92 +905,35 @@ mod tests { ); let response = handler - .build_merged_response(configs, Vec::new(), HashMap::new()) + .build_merged_response(configs, Vec::new(), HashMap::new(), HeaderMap::new()) .unwrap(); let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); - - // Verify multiple configs present assert!(parsed["configs"].get("project1").is_some()); assert!(parsed["configs"].get("project2").is_some()); - - // Verify all fields preserved exactly as-is (pass-through) assert_eq!(parsed["configs"]["project1"]["disabled"], false); assert_eq!(parsed["configs"]["project1"]["slug"], "test-project"); - assert_eq!(parsed["configs"]["project1"]["organizationId"], 42); - assert_eq!(parsed["configs"]["project1"]["projectId"], 100); assert_eq!(parsed["configs"]["project1"]["customField"], "customValue"); - } - - #[tokio::test] - async fn test_all_upstreams_fail_returns_error() { - // Set up handler with invalid upstreams - let mut locales = HashMap::new(); - locales.insert( - "us".to_string(), - HashMap::from([( - "us-cell-1".to_string(), - CellConfig { - relay_url: Url::parse("http://invalid-relay.example.com:8080").unwrap(), - sentry_url: Url::parse("http://invalid-sentry.example.com:8080").unwrap(), - }, - )]), - ); - - let handler = RelayProjectConfigsHandler::new(locales); - - // Create a request with public keys - let request_body = serde_json::json!({ - "publicKeys": ["test-key-1", "test-key-2"] - }); - let body_bytes = Bytes::from(serde_json::to_vec(&request_body).unwrap()); - - // Create empty cell_targets to simulate all upstreams failing - let empty_targets = HashMap::new(); - - // Create a base request - let request = Request::builder().method("POST").uri("/").body(()).unwrap(); - - // This should return an error since no upstreams will succeed (empty targets) - let result = handler - .handle_with_targets(&empty_targets, request, body_bytes) - .await; - assert!(result.is_err()); - match result { - Err(IngestRouterError::InternalError(msg)) => { - assert_eq!( - msg, - "All upstream requests failed with no recoverable state" - ); - } - _ => panic!("Expected InternalError"), - } - } - - #[tokio::test] - async fn test_v3_protocol_fields() { - let handler = RelayProjectConfigsHandler::new(HashMap::new()); - - // Test 1: Pending array with values + // Test: V3 protocol - Pending array with values let pending = vec!["key1".to_string(), "key2".to_string(), "key3".to_string()]; let response = handler - .build_merged_response(HashMap::new(), pending, HashMap::new()) + .build_merged_response(HashMap::new(), pending, HashMap::new(), HeaderMap::new()) .unwrap(); let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); assert_eq!(parsed["pending"].as_array().unwrap().len(), 3); assert_eq!(parsed["pending"][0], "key1"); - // Test 2: Empty pending omitted (skip_serializing_if) + // Test: V3 protocol - Empty pending omitted let response = handler - .build_merged_response(HashMap::new(), Vec::new(), HashMap::new()) + .build_merged_response(HashMap::new(), Vec::new(), HashMap::new(), HeaderMap::new()) .unwrap(); let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); assert!(parsed.get("pending").is_none()); - // Test 3: Global config and extra fields (forward compatibility) + // Test: V3 protocol - Global config and extra fields let mut extra = HashMap::new(); extra.insert( "global".to_string(), @@ -887,37 +946,79 @@ mod tests { ); let response = handler - .build_merged_response(HashMap::new(), Vec::new(), extra) + .build_merged_response(HashMap::new(), Vec::new(), extra, HeaderMap::new()) .unwrap(); let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); - - // Verify global config assert_eq!( parsed["global"]["measurements"]["maxCustomMeasurements"], 10 ); assert_eq!(parsed["global_status"], "ready"); - // Verify future/extra fields preserved assert_eq!(parsed["futureFeature"]["enabled"], true); } + #[tokio::test] + async fn test_all_upstreams_fail_returns_error() { + // Set up handler with invalid upstreams + let mut locales = HashMap::new(); + locales.insert( + "us".to_string(), + vec![CellConfig { + name: "us-cell-1".to_string(), + relay_url: Url::parse("http://invalid-relay.example.com:8080").unwrap(), + sentry_url: Url::parse("http://invalid-sentry.example.com:8080").unwrap(), + }], + ); + + let handler = RelayProjectConfigsHandler::new(locales, RelayTimeouts::default()); + + // Create a request with public keys + let request_body = serde_json::json!({ + "publicKeys": ["test-key-1", "test-key-2"] + }); + let body_bytes = Bytes::from(serde_json::to_vec(&request_body).unwrap()); + + // Create empty cells to simulate all upstreams failing + let empty_cells = Cells { + cell_list: Vec::new(), + cell_to_upstreams: HashMap::new(), + }; + + // Create a base request + let request = Request::builder().method("POST").uri("/").body(()).unwrap(); + + // This should return an error since no upstreams will succeed (empty targets) + let result = handler + .handle_with_targets(&empty_cells, request, body_bytes) + .await; + + assert!(result.is_err()); + match result { + Err(IngestRouterError::ServiceUnavailable(msg)) => { + assert_eq!(msg, "All upstream cells are unavailable"); + // Verify the error maps to 503 status code + let error = IngestRouterError::ServiceUnavailable(msg); + assert_eq!(error.status_code(), hyper::StatusCode::SERVICE_UNAVAILABLE); + } + _ => panic!("Expected ServiceUnavailable error"), + } + } + #[tokio::test] async fn test_v3_upstream_failure_adds_keys_to_pending() { // Set up handler with invalid upstream that will fail to connect let mut locales = HashMap::new(); locales.insert( "us".to_string(), - HashMap::from([( - "us-cell-1".to_string(), - CellConfig { - relay_url: Url::parse("http://localhost:1").unwrap(), // Invalid port - sentry_url: Url::parse("http://localhost:1").unwrap(), // Will fail to connect - }, - )]), + vec![CellConfig { + name: "us-cell-1".to_string(), + relay_url: Url::parse("http://localhost:1").unwrap(), // Invalid port + sentry_url: Url::parse("http://localhost:1").unwrap(), // Will fail to connect + }], ); - let handler = RelayProjectConfigsHandler::new(locales.clone()); + let handler = RelayProjectConfigsHandler::new(locales.clone(), RelayTimeouts::default()); // Create a request with public keys let request_body = serde_json::json!({ @@ -925,16 +1026,16 @@ mod tests { }); let body_bytes = Bytes::from(serde_json::to_vec(&request_body).unwrap()); - // Get upstreams + // Get cells let locales = Locales::new(locales); - let cell_upstreams = &locales.get_cells("us").unwrap().cell_to_upstreams; + let cells = locales.get_cells("us").unwrap(); // Create a base request let request = Request::builder().method("POST").uri("/").body(()).unwrap(); // When upstream fails, all its keys should be added to pending let result = handler - .handle_with_targets(cell_upstreams, request, body_bytes) + .handle_with_targets(cells, request, body_bytes) .await; // Should succeed (v3 protocol - returning pending is valid) @@ -960,48 +1061,51 @@ mod tests { } #[test] - fn test_global_field_handling() { - // Test 1: Split behavior - first upstream gets true, others get false + fn test_extra_fields_passthrough() { + // Test 1: Split behavior - all upstreams get the same extra fields (including global) let mut locales = HashMap::new(); locales.insert( "us".to_string(), - HashMap::from([ - ( - "us-cell-1".to_string(), - CellConfig { - relay_url: Url::parse("http://us1-relay:8080").unwrap(), - sentry_url: Url::parse("http://us1-sentry:8080").unwrap(), - }, - ), - ( - "us-cell-2".to_string(), - CellConfig { - relay_url: Url::parse("http://us2-relay:8080").unwrap(), - sentry_url: Url::parse("http://us2-sentry:8080").unwrap(), - }, - ), - ]), + vec![ + CellConfig { + name: "us-cell-1".to_string(), + relay_url: Url::parse("http://us1-relay:8080").unwrap(), + sentry_url: Url::parse("http://us1-sentry:8080").unwrap(), + }, + CellConfig { + name: "us-cell-2".to_string(), + relay_url: Url::parse("http://us2-relay:8080").unwrap(), + sentry_url: Url::parse("http://us2-sentry:8080").unwrap(), + }, + ], ); - let handler = RelayProjectConfigsHandler::new(locales.clone()); + let handler = RelayProjectConfigsHandler::new(locales.clone(), RelayTimeouts::default()); + let mut extra = HashMap::new(); + extra.insert("global".to_string(), serde_json::json!(true)); let request = RelayProjectConfigsRequest { public_keys: vec!["key1".to_string(), "key2".to_string()], - global: Some(true), - extra: HashMap::new(), + extra, }; let locales_obj = Locales::new(locales); - let cell_upstreams = &locales_obj.get_cells("us").unwrap().cell_to_upstreams; - let splits = handler.split_keys_by_upstream(&request, cell_upstreams); + let cells = locales_obj.get_cells("us").unwrap(); + let splits = handler.split_keys_by_upstream(&request, cells); assert_eq!(splits.len(), 2); - assert_eq!(splits[0].1.global, Some(true)); // First gets true - assert_eq!(splits[1].1.global, Some(false)); // Others get false + // All upstreams get the same extra fields (including global: true) + assert_eq!( + splits[0].request.extra.get("global"), + Some(&serde_json::json!(true)) + ); + assert_eq!( + splits[1].request.extra.get("global"), + Some(&serde_json::json!(true)) + ); - // Test 2: Serialization - global omitted when None + // Test 2: Serialization - extra fields are included let request = RelayProjectConfigsRequest { public_keys: vec!["key1".to_string()], - global: None, extra: HashMap::new(), }; @@ -1010,4 +1114,58 @@ mod tests { assert!(parsed.get("global").is_none()); assert!(parsed.get("publicKeys").is_some()); } + + #[tokio::test] + async fn test_headers_from_highest_priority_cell() { + use hyper::header::{CACHE_CONTROL, HeaderValue}; + + let mut locales = HashMap::new(); + locales.insert( + "test".to_string(), + vec![CellConfig { + name: "test-cell".to_string(), + relay_url: Url::parse("http://localhost:8090").unwrap(), + sentry_url: Url::parse("http://localhost:8080").unwrap(), + }], + ); + + let handler = RelayProjectConfigsHandler::new(locales, RelayTimeouts::default()); + + // Create headers from upstream (simulating what we'd get from highest priority cell) + let mut upstream_headers = HeaderMap::new(); + upstream_headers.insert(CACHE_CONTROL, HeaderValue::from_static("max-age=300")); + upstream_headers.insert( + "X-Sentry-Rate-Limit-Remaining", + HeaderValue::from_static("100"), + ); + + // Add a hop-by-hop header that should be filtered + upstream_headers.insert( + hyper::header::CONNECTION, + HeaderValue::from_static("keep-alive"), + ); + + let response = handler + .build_merged_response(HashMap::new(), Vec::new(), HashMap::new(), upstream_headers) + .unwrap(); + + // Verify headers are copied + assert_eq!( + response.headers().get(CACHE_CONTROL), + Some(&HeaderValue::from_static("max-age=300")) + ); + assert_eq!( + response.headers().get("X-Sentry-Rate-Limit-Remaining"), + Some(&HeaderValue::from_static("100")) + ); + + // Verify hop-by-hop headers are filtered out + assert!(response.headers().get(hyper::header::CONNECTION).is_none()); + + // Verify Content-Type is always set + assert_eq!( + response.headers().get(CONTENT_TYPE), + Some(&HeaderValue::from_static("application/json")) + ); + } } diff --git a/ingest-router/src/router.rs b/ingest-router/src/router.rs index 3488c92..ed13a9d 100644 --- a/ingest-router/src/router.rs +++ b/ingest-router/src/router.rs @@ -36,7 +36,14 @@ impl Router { match self.find_matching_route(&req) { Some(action) => { tracing::debug!(action = ?action, "Matched route"); - self.handle_action(req, action).await + // Convert errors to HTTP responses with proper status codes + match self.handle_action(req, action).await { + Ok(response) => Ok(response), + Err(e) => { + tracing::error!(error = %e, "Handler error"); + Ok(e.into_response()) + } + } } None => { tracing::warn!( @@ -137,7 +144,7 @@ mod tests { use hyper::{Method, Request}; fn test_router(routes: Vec) -> Router { - use crate::config::CellConfig; + use crate::config::{CellConfig, RelayTimeouts}; use std::collections::HashMap; use url::Url; @@ -145,36 +152,30 @@ mod tests { let mut locales = HashMap::new(); locales.insert( "us".to_string(), - HashMap::from([( - "us-cell-1".to_string(), - CellConfig { - relay_url: Url::parse("http://us-relay.example.com:8080").unwrap(), - sentry_url: Url::parse("http://us-sentry.example.com:8080").unwrap(), - }, - )]), + vec![CellConfig { + name: "us-cell-1".to_string(), + relay_url: Url::parse("http://us-relay.example.com:8080").unwrap(), + sentry_url: Url::parse("http://us-sentry.example.com:8080").unwrap(), + }], ); locales.insert( "de".to_string(), - HashMap::from([( - "de-cell-1".to_string(), - CellConfig { - relay_url: Url::parse("http://de-relay.example.com:8080").unwrap(), - sentry_url: Url::parse("http://de-sentry.example.com:8080").unwrap(), - }, - )]), + vec![CellConfig { + name: "de-cell-1".to_string(), + relay_url: Url::parse("http://de-relay.example.com:8080").unwrap(), + sentry_url: Url::parse("http://de-sentry.example.com:8080").unwrap(), + }], ); locales.insert( "local".to_string(), - HashMap::from([( - "local-cell".to_string(), - CellConfig { - relay_url: Url::parse("http://local-relay.example.com:8080").unwrap(), - sentry_url: Url::parse("http://local-sentry.example.com:8080").unwrap(), - }, - )]), + vec![CellConfig { + name: "local-cell".to_string(), + relay_url: Url::parse("http://local-relay.example.com:8080").unwrap(), + sentry_url: Url::parse("http://local-sentry.example.com:8080").unwrap(), + }], ); - let handler = RelayProjectConfigsHandler::new(locales); + let handler = RelayProjectConfigsHandler::new(locales, RelayTimeouts::default()); Router::new(routes, handler) } @@ -288,4 +289,29 @@ mod tests { let response = router.route(req).await.unwrap(); assert_eq!(response.status(), StatusCode::NOT_FOUND); } + + #[tokio::test] + async fn test_error_converted_to_response() { + use std::collections::HashMap; + + // Create a router with a route that references a non-existent locale + let handler = RelayProjectConfigsHandler::new( + HashMap::new(), // Empty locales + crate::config::RelayTimeouts::default(), + ); + let routes = vec![test_route( + None, + Some("/test".to_string()), + None, + "nonexistent", // This locale doesn't exist + )]; + let router = Router::new(routes, handler); + + // This should trigger an InternalError (locale not found) and return 500 + let req = test_request(Method::POST, "/test", None); + let response = router.route(req).await.unwrap(); + + // Verify the error was converted to a proper HTTP response + assert_eq!(response.status(), StatusCode::INTERNAL_SERVER_ERROR); + } } From eb86374b7552a307013664df2d913930e76ad809 Mon Sep 17 00:00:00 2001 From: Nikhar Saxena Date: Tue, 25 Nov 2025 11:30:45 -0800 Subject: [PATCH 6/7] Make serde_json as workspace depenedency --- Cargo.toml | 1 + ingest-router/Cargo.toml | 2 +- locator/Cargo.toml | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 4accfdb..d48b46b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -14,6 +14,7 @@ reqwest = { version = "0.12.23", features = ["json"] } sentry = { version = "0.45.0", features = ["tracing"] } serde = { version = "1.0.228", features = ["derive"] } serde_yaml = "0.9.34" +serde_json = "1.0.145" tempfile = "3.23.0" thiserror = "2.0.17" tokio = { version = "1.48.0", features = ["full"] } diff --git a/ingest-router/Cargo.toml b/ingest-router/Cargo.toml index abe2cec..edecc50 100644 --- a/ingest-router/Cargo.toml +++ b/ingest-router/Cargo.toml @@ -9,7 +9,7 @@ http-body-util = { workspace = true } hyper = { workspace = true } hyper-util = { workspace = true } serde = { workspace = true } -serde_json = "1.0" +serde_json = { workspace = true } shared = { path = "../shared" } thiserror = { workspace = true } tokio = { workspace = true } diff --git a/locator/Cargo.toml b/locator/Cargo.toml index b0bd4c7..7c8be1a 100644 --- a/locator/Cargo.toml +++ b/locator/Cargo.toml @@ -13,7 +13,7 @@ google-cloud-storage = "1.4.0" metrics = { workspace = true } moka = { version = "0.12.11", features = ["sync"] } serde = { workspace = true } -serde_json = "1.0.145" +serde_json = { workspace = true } reqwest = { workspace = true } thiserror = { workspace = true } tokio = { workspace = true } From 95d4bb7c0e2792788dcff703f3d17bb937b1fdea Mon Sep 17 00:00:00 2001 From: Nikhar Saxena Date: Wed, 26 Nov 2025 13:52:03 -0800 Subject: [PATCH 7/7] Make project handler submodule --- .../src/relay_project_config_handler.rs | 1171 ----------------- .../relay_project_config_handler/merger.rs | 249 ++++ .../src/relay_project_config_handler/mod.rs | 489 +++++++ .../relay_project_config_handler/protocol.rs | 136 ++ .../relay_project_config_handler/splitter.rs | 111 ++ .../task_executor.rs | 606 +++++++++ 6 files changed, 1591 insertions(+), 1171 deletions(-) delete mode 100644 ingest-router/src/relay_project_config_handler.rs create mode 100644 ingest-router/src/relay_project_config_handler/merger.rs create mode 100644 ingest-router/src/relay_project_config_handler/mod.rs create mode 100644 ingest-router/src/relay_project_config_handler/protocol.rs create mode 100644 ingest-router/src/relay_project_config_handler/splitter.rs create mode 100644 ingest-router/src/relay_project_config_handler/task_executor.rs diff --git a/ingest-router/src/relay_project_config_handler.rs b/ingest-router/src/relay_project_config_handler.rs deleted file mode 100644 index 7f5033c..0000000 --- a/ingest-router/src/relay_project_config_handler.rs +++ /dev/null @@ -1,1171 +0,0 @@ -//! Relay Project Configuration Handler -//! -//! This module implements the split-merge strategy for handling Sentry's -//! `/api/0/relays/projectconfigs/` endpoint across multiple upstream Sentry instances -//! in a multi-cell architecture. -//! -//! # Protocol Overview -//! -//! The Relay Project Configs endpoint (version 3) is used by Relay instances to fetch -//! project configurations needed to process events. This implementation acts as an -//! aggregation layer that: -//! 1. Splits requests across multiple upstream Sentry cells based on project ownership -//! 2. Fans out requests in parallel to multiple upstreams -//! 3. Merges responses back into a single v3-compliant response -//! 4. Passes through all config data unchanged -//! -//! ## Endpoint Details -//! The endpoint implementation is at https://github.com/getsentry/sentry/blob/master/src/sentry/api/endpoints/relay/project_configs.py -//! -//! - **Path**: `/api/0/relays/projectconfigs/` -//! - **Method**: `POST` -//! - **Protocol Version**: 3 -//! - **Authentication**: RelayAuthentication (X-Sentry-Relay-Id, X-Sentry-Relay-Signature) -//! -//! # Request Format (Version 3) -//! -//! ```json -//! { -//! "publicKeys": ["key1", "key2", "key3"], -//! "noCache": false, -//! "global": true -//! } -//! ``` -//! -//! ## Request Fields -//! -//! - **`publicKeys`** (required): Array of project public keys (DSN keys) to fetch configs for -//! - **`noCache`** (optional): If `true`, bypass caching and compute fresh configs (downgrades to v2 behavior) -//! - **`global`** (optional): If `true`, include global relay configuration in response -//! -//! # Response Format (Version 3) -//! -//! ```json -//! { -//! "configs": { -//! "key1": { -//! "disabled": false, -//! "slug": "project-slug", -//! "publicKeys": [...], -//! "config": {...}, -//! "organizationId": 1, -//! "projectId": 42 -//! } -//! }, -//! "pending": ["key2", "key3"], -//! "global": { -//! "measurements": {...}, -//! "options": {...} -//! }, -//! "global_status": "ready" -//! } -//! ``` -//! -//! ## Response Fields -//! -//! - **`configs`**: Map of public keys to their project configurations -//! - Configs are passed through unchanged from upstream Sentry instances -//! - They will add the relevant processing relay URL in the response -//! - **`pending`**: Array of public keys for which configs are being computed asynchronously -//! - Relay should retry the request later to fetch these configs -//! - Also used when upstreams fail/timeout (graceful degradation) -//! - **`global`**: Global relay configuration (only if `global: true` in request) -//! - **`global_status`**: Status of global config (always "ready" when present) -//! -//! # Implementation Details -//! -//! ## Request Splitting Strategy -//! -//! When a request arrives with multiple public keys: -//! -//! 1. **Route each key to its owning cell** -//! - Currently: Round-robin distribution (TODO: replace with control plane lookup) -//! - TODO: Query locator service for each public key to get cell name -//! -//! 2. **Group keys by target upstream** -//! - Keys routed to the same cell are batched into one request -//! -//! 3. **Handle global config flag** -//! - All upstreams receive the same `global` flag value as the original request -//! - Global config is selected from the highest priority cell that returns it -//! - Priority is determined by cell order in configuration (first = highest priority) -//! - Enables failover: if highest priority cell fails, next cell's global config is used -//! -//! ## Response Merging Strategy -//! -//! Responses from multiple upstreams are merged as follows: -//! -//! ### Configs (HashMap merge) -//! - Merge all `configs` HashMaps from all upstreams -//! - Configs are passed through unchanged (no modifications) -//! - relay_url is expected to be added in the upstream response -//! -//! ### Pending (Array concatenation) -//! - Concatenate all `pending` arrays from all upstream responses -//! - Include keys from failed/timed-out upstreams -//! - Relay will retry these keys in a subsequent request -//! -//! ### Extra fields (Priority-based selection) -//! - Select `extra` fields from highest priority cell that responds successfully -//! - Priority determined by cell order in configuration (first = highest) -//! - Forward compatibility: new fields are automatically preserved -//! -//! ## Error Handling -//! -//! ### Partial Failures (Graceful Degradation) -//! - If some upstreams succeed: Return successful configs + pending for failed keys -//! - Failed keys are added to `pending` array (v3 protocol) -//! - Logged but does not block response -//! -//! ### Total Failure -//! - If all upstreams fail: Check if any keys were added to pending -//! - If pending is not empty: Return 200 OK with pending array (relay will retry) -//! - If pending is empty: Return 503 error (no recoverable state) -//! -//! ### Upstream Failure Scenarios -//! - **Timeout**: All keys from that upstream → pending -//! - **Connection error**: All keys from that upstream → pending -//! - **Parse error**: All keys from that upstream → pending -//! - **Task panic**: Logged error (extreme edge case, keys lost) -//! -//! ## Request Flow -//! -//! ### Success Scenario -//! -//! ```text -//! ┌─────────────┐ -//! │ Relay │ -//! └──────┬──────┘ -//! │ -//! │ POST /api/0/relays/projectconfigs/ -//! │ {publicKeys: [A,B,C,D,E,F]} -//! │ -//! ▼ -//! ┌──────────────────────────────────────┐ -//! │ Ingest Router (this handler) │ -//! │ │ -//! │ 1. Parse request │ -//! │ 2. Split keys by cell: │ -//! │ • US1 → [A,C,E] │ -//! │ • US2 → [B,D,F] │ -//! └───┬──────────────────────────┬───────┘ -//! │ │ -//! │ {publicKeys: [A,C,E], │ {publicKeys: [B,D,F], -//! │ global: true} │ global: true} -//! │ │ -//! ▼ ▼ -//! ┌──────────┐ ┌──────────┐ -//! │Cell US1 │ │Cell US2 │ -//! │(Sentry) │ │(Sentry) │ -//! └────┬─────┘ └─────┬────┘ -//! │ │ -//! │ {configs: {A,C,E}} │ {configs: {B,D,F}} -//! │ │ -//! └──────────┬───────────────┘ -//! ▼ -//! ┌──────────────────────────────────────┐ -//! │ Ingest Router (this handler) │ -//! │ │ -//! │ 3. Merge responses: │ -//! │ • Combine all configs │ -//! │ • Merge pending arrays │ -//! │ • Select others from priority │ -//! └──────────────┬───────────────────────┘ -//! │ -//! │ {configs: {A,B,C,D,E,F}, -//! │ global: {...}} -//! │ -//! ▼ -//! ┌─────────────┐ -//! │ Relay │ -//! └─────────────┘ -//! ``` -//! -//! ### Failure Scenario (Graceful Degradation) -//! -//! When an upstream fails or times out, keys are added to the `pending` array: -//! -//! ```text -//! ┌─────────────┐ -//! │ Relay │ -//! └──────┬──────┘ -//! │ -//! │ POST {publicKeys: [A,B,C,D,E,F]} -//! │ -//! ▼ -//! ┌──────────────────────────────────────┐ -//! │ Ingest Router (this handler) │ -//! │ Split: US1→[A,C,E], US2→[B,D,F] │ -//! └───┬──────────────────────────┬───────┘ -//! │ │ -//! ▼ ▼ -//! ┌──────────┐ ┌──────────┐ -//! │Cell US1 │ │Cell US2 │ -//! └────┬─────┘ └─────┬────┘ -//! │ │ -//! │ ✓ Success │ ✗ Timeout/Error -//! │ {configs: {A,C,E}} │ -//! │ │ -//! └──────────┬───────────────┘ -//! ▼ -//! ┌──────────────────────────────────────┐ -//! │ Ingest Router (this handler) │ -//! │ │ -//! │ • Collect successful: {A,C,E} │ -//! │ • Add failed to pending: [B,D,F] │ -//! └──────────────┬───────────────────────┘ -//! │ -//! │ {configs: {A,C,E}, -//! │ pending: [B,D,F]} -//! │ -//! ▼ -//! ┌─────────────┐ -//! │ Relay │ (will retry pending) -//! └─────────────┘ -//! ``` -//! -//! # Examples -//! -//! ## Example 1: All upstreams succeed -//! -//! **Request**: -//! ```json -//! {"publicKeys": ["key1", "key2"]} -//! ``` -//! -//! **Response**: -//! ```json -//! { -//! "configs": { -//! "key1": {"disabled": false, "slug": "project-us1", ...}, -//! "key2": {"disabled": false, "slug": "project-us2", ...} -//! } -//! } -//! ``` -//! -//! ## Example 2: One upstream fails -//! -//! **Request**: -//! ```json -//! {"publicKeys": ["key1", "key2", "key3"]} -//! ``` -//! -//! **Response** (if upstream with key2,key3 times out): -//! ```json -//! { -//! "configs": { -//! "key1": {"disabled": false, "slug": "project-us1", ...} -//! }, -//! "pending": ["key2", "key3"] -//! } -//! ``` -//! -//! ## Example 3: Request with global config -//! -//! **Request**: -//! ```json -//! {"publicKeys": ["key1", "key2"], "global": true} -//! ``` -//! -//! **Splitting**: -//! - Request to US1: `{"publicKeys": ["key1"], "global": true}` -//! - Request to US2: `{"publicKeys": ["key2"], "global": true}` -//! -//! (US1 has higher priority, so its global config will be used in the final response) -//! -//! **Response**: -//! ```json -//! { -//! "configs": {...}, -//! "global": {"measurements": {...}}, -//! "global_status": "ready" -//! } -//! ``` -//! -//! # See Also -//! -//! - [`RelayProjectConfigsHandler`] - Main handler struct for processing requests - -use crate::config::{CellConfig, RelayTimeouts}; -use crate::errors::IngestRouterError; -use crate::http::send_to_upstream; -use crate::locale::{Cells, Locales, Upstream}; -use http_body_util::{BodyExt, Full, combinators::BoxBody}; -use hyper::body::{Body, Bytes}; -use hyper::header::{CONTENT_LENGTH, CONTENT_TYPE, HeaderMap}; -use hyper::{Request, Response, StatusCode}; -use hyper_util::client::legacy::Client; -use hyper_util::client::legacy::connect::HttpConnector; -use hyper_util::rt::TokioExecutor; -use serde::{Deserialize, Serialize}; -use serde_json::Value as JsonValue; -use std::collections::HashMap; -use std::time::Duration; -use tokio::task::JoinHandle; -use tokio::time::{Instant, timeout}; - -/// Request format for the relay project configs endpoint. -/// -/// See module-level docs for full protocol details. -#[derive(Debug, Clone, Serialize, Deserialize)] -struct RelayProjectConfigsRequest { - /// DSN public keys to fetch configs for. - #[serde(rename = "publicKeys")] - pub public_keys: Vec, - - /// Other fields (`global`, `noCache`, future fields) for forward compatibility. - /// All fields are passed through as-is to upstreams. - #[serde(flatten)] - pub extra: HashMap, -} - -impl RelayProjectConfigsRequest { - fn from_bytes(bytes: &Bytes) -> Result { - serde_json::from_slice(bytes) - } - - fn to_bytes(&self) -> Result { - let json = serde_json::to_vec(self)?; - Ok(Bytes::from(json)) - } -} - -/// Response format for the relay project configs endpoint. -/// -/// See module-level docs for merge strategy and field details. -#[derive(Debug, Clone, Serialize, Deserialize)] -struct RelayProjectConfigsResponse { - /// Project configs (HashMap merged from all upstreams). - pub configs: HashMap, - - /// Keys being computed async or from failed upstreams (concatenated). - #[serde(skip_serializing_if = "Option::is_none")] - pub pending: Option>, - - /// Other fields (`global`, `global_status`, future fields). - #[serde(flatten)] - pub extra: HashMap, -} - -impl RelayProjectConfigsResponse { - fn from_bytes(bytes: &Bytes) -> Result { - serde_json::from_slice(bytes) - } -} - -/// Result from an upstream task request. -struct UpstreamTaskResult { - /// The cell name this result is from. - cell_name: String, - /// The upstream URL that was contacted. - sentry_url: String, - /// The public keys that were requested from this upstream. - public_keys: Vec, - /// The result of the upstream request. - result: Result, IngestRouterError>, -} - -/// Merged results from all upstream tasks. -struct MergedResults { - /// All configs merged from successful upstreams. - configs: HashMap, - /// All pending keys (from failed upstreams or upstream pending arrays). - pending: Vec, - /// Extra fields (global config, status, etc.). - extra: HashMap, - /// Headers from upstream - headers: HeaderMap, -} - -/// Task handle paired with its public keys for graceful failure handling. -/// -/// The public keys are tracked outside the task so they can be added to the -/// pending array if the task times out or panics, maintaining v3 protocol compliance. -struct TaskWithKeys { - /// The spawned task handle. - handle: JoinHandle, - /// Public keys requested from this upstream. - public_keys: Vec, -} - -/// A request split for a specific upstream cell. -struct SplitRequest { - /// The cell name this request is for - cell_name: String, - /// The upstream to send the request to - upstream: Upstream, - /// The request body to send - request: RelayProjectConfigsRequest, -} - -/// Executor for spawning and collecting upstream tasks. -/// -/// Encapsulates the parallel task execution logic for fanning out requests -/// to multiple upstream Sentry instances and collecting/merging their results. -struct UpstreamTaskExecutor { - /// HTTP client for sending requests to upstream Sentry instances. - client: Client>, - - /// Timeout configuration for HTTP and task-level timeouts. - timeouts: RelayTimeouts, -} - -impl UpstreamTaskExecutor { - fn new(client: Client>, timeouts: RelayTimeouts) -> Self { - Self { client, timeouts } - } - - /// Spawns parallel tasks to fan out requests to multiple upstreams. - /// - /// Each task performs an HTTP request to an upstream Sentry instance with - /// the appropriate subset of public keys. - /// - /// Uses two-layer timeout strategy: - /// - HTTP timeout: Applied to individual HTTP requests - /// - Task timeout: Applied at collection level with adaptive strategy - fn spawn_tasks( - &self, - split_requests: Vec, - base_request: &Request<()>, - ) -> Result, IngestRouterError> { - let mut tasks = Vec::new(); - - for split in split_requests { - let cell_name = split.cell_name; - let upstream = split.upstream; - let split_request = split.request; - // Track public keys for this upstream - let public_keys = split_request.public_keys.clone(); - - // Serialize the split request body - let request_body = split_request.to_bytes().map_err(|e| { - IngestRouterError::InternalError(format!("Failed to serialize request: {e}")) - })?; - - // Build request with headers from original request - let mut req_builder = Request::builder() - .method(base_request.method()) - .uri(base_request.uri().clone()) - .version(base_request.version()); - - for (name, value) in base_request.headers() { - req_builder = req_builder.header(name, value); - } - - let request = req_builder - .body(Full::new(request_body)) - .expect("Failed to build request"); - - let client = self.client.clone(); - let sentry_url = upstream.sentry_url.clone(); - let http_timeout_secs = self.timeouts.http_timeout_secs as u64; - - // Clone data we need to track even if task times out - let public_keys_for_task = public_keys.clone(); - let cell_name_for_task = cell_name.clone(); - - // Spawn a task for each upstream request - let task = tokio::spawn(async move { - let sentry_url_str = sentry_url.to_string(); - let result: Result, IngestRouterError> = - send_to_upstream(&client, &sentry_url, request, http_timeout_secs).await; - - UpstreamTaskResult { - cell_name: cell_name_for_task, - sentry_url: sentry_url_str, - public_keys: public_keys_for_task, - result, - } - }); - - // Store task handle and keys (keys needed if task times out) - tasks.push(TaskWithKeys { - handle: task, - public_keys, - }); - } - - Ok(tasks) - } - - /// Collects results from all spawned tasks and merges them. - /// - /// Uses adaptive timeouts with cutoff strategy: - /// - Initial: Wait up to task_initial_timeout_secs for first upstream to respond - /// - Subsequent: Once first succeeds, ALL remaining tasks have task_subsequent_timeout_secs - /// total (from first success) to complete. This prevents slow/down cells from blocking - /// progress when we already have good data. - /// - /// Failed keys are added to the pending array for retry. - /// - /// Global config is selected from the highest priority cell based on cells.cell_list order. - async fn collect_and_merge(&self, tasks: Vec, cells: &Cells) -> MergedResults { - let mut merged_configs = HashMap::new(); - let mut merged_pending = Vec::new(); - let mut extra_by_cell: HashMap> = HashMap::new(); - let mut headers_by_cell: HashMap = HashMap::new(); - let mut additional_deadline: Option = None; - - for task_with_keys in tasks { - let TaskWithKeys { - mut handle, - public_keys, - } = task_with_keys; - - // Adaptive timeout - // - Before first success: Use task_initial_timeout_secs - // - After first success: Use remaining time until deadline - let timeout_duration = if let Some(deadline) = additional_deadline { - // Calculate remaining time to deadline - let now = Instant::now(); - if now >= deadline { - // Deadline already passed, use minimal timeout - Duration::from_millis(1) - } else { - deadline.duration_since(now) - } - } else { - Duration::from_secs(self.timeouts.task_initial_timeout_secs as u64) - }; - - let task_result = timeout(timeout_duration, &mut handle).await; - - // Handle timeout - let Ok(join_result) = task_result else { - tracing::error!( - "Task timed out after {} seconds", - timeout_duration.as_secs() - ); - // Abort the timed-out task to prevent it from continuing in background - handle.abort(); - // Add all keys from this upstream to pending (v3 protocol) - merged_pending.extend(public_keys); - continue; - }; - - // Handle task panic - let Ok(upstream_result) = join_result else { - if let Err(e) = join_result { - tracing::error!("Task panicked: {e}"); - } - // Add all keys from this upstream to pending (v3 protocol) - merged_pending.extend(public_keys); - continue; - }; - - // Process the upstream result - let result_had_configs = upstream_result.result.is_ok(); - let cell_name_from_result = upstream_result.cell_name.clone(); - if let Some((extra, headers)) = self.process_upstream_result( - upstream_result, - &mut merged_configs, - &mut merged_pending, - ) { - // Store extra fields and headers by cell name for later priority-based selection - extra_by_cell.insert(cell_name_from_result.clone(), extra); - headers_by_cell.insert(cell_name_from_result, headers); - } - - // Set deadline on first success - if result_had_configs && additional_deadline.is_none() { - additional_deadline = Some( - Instant::now() - + Duration::from_secs(self.timeouts.task_subsequent_timeout_secs as u64), - ); - } - } - - // Select global config and headers from highest priority cell by iterating cells.cell_list - // cell_list is already in priority order (first = highest priority) - let (merged_extra, merged_headers) = cells - .cell_list - .iter() - .find_map(|cell_name| { - extra_by_cell.get(cell_name).cloned().map(|extra| { - let headers = headers_by_cell.get(cell_name).cloned().unwrap_or_default(); - (extra, headers) - }) - }) - .unwrap_or_default(); - - MergedResults { - configs: merged_configs, - pending: merged_pending, - extra: merged_extra, - headers: merged_headers, - } - } - - /// Processes the result from a single upstream task. - /// - /// Handles both successful and failed upstream responses, merging successful - /// configs and adding failed keys to the pending array. - /// - /// Returns the extra fields (global config, etc.) and headers if the request succeeded, - /// which allows the caller to select based on cell priority. - fn process_upstream_result( - &self, - upstream_result: UpstreamTaskResult, - merged_configs: &mut HashMap, - merged_pending: &mut Vec, - ) -> Option<(HashMap, HeaderMap)> { - let UpstreamTaskResult { - cell_name: _, - sentry_url, - public_keys, - result, - } = upstream_result; - - // Handle HTTP request failure - let Ok(response) = result else { - tracing::error!( - sentry_url = %sentry_url, - error = %result.unwrap_err(), - "Request to upstream failed" - ); - // Add all keys from this upstream to pending (v3 protocol) - merged_pending.extend(public_keys); - return None; - }; - - // Extract headers before consuming the response - let (parts, body) = response.into_parts(); - let headers = parts.headers; - - // Parse response body - match RelayProjectConfigsResponse::from_bytes(&body) { - Ok(response_data) => { - // Merge configs from this upstream - merged_configs.extend(response_data.configs); - - // Merge pending arrays (v3 protocol) - if let Some(pending) = response_data.pending { - merged_pending.extend(pending); - } - - // Return extra fields and headers for priority-based selection - Some((response_data.extra, headers)) - } - Err(e) => { - tracing::error!( - sentry_url = %sentry_url, - error = %e, - "Failed to parse response from upstream" - ); - // Add all keys from this upstream to pending - merged_pending.extend(public_keys); - None - } - } - } -} - -/// Handler for Relay Project Configs endpoint. -/// -/// See module-level docs for complete protocol details, implementation strategy, -/// and request/response flow diagrams. -pub struct RelayProjectConfigsHandler { - /// Executor for spawning and collecting upstream tasks. - executor: UpstreamTaskExecutor, - - /// Locales mapping for locale-based upstream lookups. - /// - /// Maps locale → cell name → upstream (relay URL + sentry URL). - locales: Locales, -} - -impl RelayProjectConfigsHandler { - pub fn new(locales_config: HashMap>, timeouts: RelayTimeouts) -> Self { - let connector = HttpConnector::new(); - let client = Client::builder(TokioExecutor::new()).build(connector); - - // Build locales from config - let locales = Locales::new(locales_config); - - // Create executor for task management - let executor = UpstreamTaskExecutor::new(client, timeouts); - - Self { executor, locales } - } - - pub async fn handle( - &self, - locale: &str, - request: Request, - ) -> Result>, IngestRouterError> - where - B: Body + Send + 'static, - B::Data: Send, - B::Error: std::error::Error + Send + Sync + 'static, - { - // Get cells for this locale - let cells = self.locales.get_cells(locale).ok_or_else(|| { - IngestRouterError::InternalError(format!( - "No targets configured for locale: {}", - locale - )) - })?; - - // Buffer the request body. We need to full body in order to do request massaging. - let (parts, body) = request.into_parts(); - let body_bytes = body - .collect() - .await - .map(|collected| collected.to_bytes()) - .map_err(|e| IngestRouterError::RequestBodyError(e.to_string()))?; - let base_request = Request::from_parts(parts, ()); - - // Process the request - self.handle_with_targets(cells, base_request, body_bytes) - .await - } - - /// Internal method that orchestrates the split-merge flow. - /// - /// High-level steps: - /// 1. Parse and split the request across upstreams - /// 2. Spawn parallel tasks to fan out requests - /// 3. Collect and merge results from all upstreams - /// 4. Build and return the merged response - async fn handle_with_targets( - &self, - cells: &Cells, - base_request: Request<()>, - body_bytes: Bytes, - ) -> Result>, IngestRouterError> { - // Parse the request - let request_data = RelayProjectConfigsRequest::from_bytes(&body_bytes).map_err(|e| { - IngestRouterError::RequestBodyError(format!("Failed to parse request: {e}")) - })?; - - // Split publicKeys across upstreams - let split_requests = self.split_keys_by_upstream(&request_data, cells); - - // Spawn tasks to fan out requests in parallel - let tasks = self.executor.spawn_tasks(split_requests, &base_request)?; - - // Collect and merge results from all tasks (uses cells.cell_list priority order) - let results = self.executor.collect_and_merge(tasks, cells).await; - - // Only return an error if we have no configs AND no pending AND we had keys to request - // Having keys in pending is a valid v3 response (relay will retry later) - // Return 503 Service Unavailable to indicate temporary unavailability - if results.configs.is_empty() - && results.pending.is_empty() - && !request_data.public_keys.is_empty() - { - return Err(IngestRouterError::ServiceUnavailable( - "All upstream cells are unavailable".to_string(), - )); - } - - // Build merged response in the relay format - self.build_merged_response( - results.configs, - results.pending, - results.extra, - results.headers, - ) - } - - /// Splits public keys across multiple upstream cells. - /// - /// Current: Round-robin stub. TODO: Replace with locator service lookup. - /// See module-level docs for complete splitting strategy and global config handling. - fn split_keys_by_upstream( - &self, - request: &RelayProjectConfigsRequest, - cells: &Cells, - ) -> Vec { - let cell_list = &cells.cell_list; - if cell_list.is_empty() { - return Vec::new(); - } - - let mut split: HashMap> = HashMap::new(); - - // TODO: Replace with control plane lookup per key - for (index, public_key) in request.public_keys.iter().enumerate() { - let cell_name = &cell_list[index % cell_list.len()]; - - split - .entry(cell_name.clone()) - .or_default() - .push(public_key.clone()); - } - - // Build a request for each cell with its assigned publicKeys - split - .into_iter() - .map(|(cell_name, public_keys)| { - let upstream = cells - .cell_to_upstreams - .get(&cell_name) - .expect("Cell name in list must exist in HashMap"); - - // All fields in extra (including global, noCache, etc.) are passed through as-is - SplitRequest { - cell_name, - upstream: upstream.clone(), - request: RelayProjectConfigsRequest { - public_keys, - extra: request.extra.clone(), - }, - } - }) - .collect() - } - - /// Build a merged response from collected configs in relay format. - /// - /// Uses headers from the highest priority cell (same cell used for global config). - /// Filters out hop-by-hop headers and Content-Length (which is recalculated for the new body). - fn build_merged_response( - &self, - merged_configs: HashMap, - merged_pending: Vec, - merged_extra: HashMap, - mut upstream_headers: HeaderMap, - ) -> Result>, IngestRouterError> { - // Wrap in relay response format - let response = RelayProjectConfigsResponse { - configs: merged_configs, - pending: if merged_pending.is_empty() { - None - } else { - Some(merged_pending) - }, - extra: merged_extra, - }; - - let merged_json = serde_json::to_vec(&response) - .map_err(|e| IngestRouterError::ResponseSerializationError(e.to_string()))?; - - // Filter hop-by-hop headers from upstream (assumes HTTP/1.1) - // These headers are connection-specific and shouldn't be forwarded - shared::http::filter_hop_by_hop(&mut upstream_headers, hyper::Version::HTTP_11); - - // Remove Content-Length since we're creating a new body with different length - upstream_headers.remove(CONTENT_LENGTH); - - // Build response with filtered headers from highest priority cell - let mut builder = Response::builder().status(StatusCode::OK); - - // Copy filtered headers from upstream - for (name, value) in upstream_headers.iter() { - builder = builder.header(name, value); - } - - // Always set/override Content-Type to ensure it's correct - builder = builder.header(CONTENT_TYPE, "application/json"); - - builder - .body( - Full::new(Bytes::from(merged_json)) - .map_err(|e| match e {}) - .boxed(), - ) - .map_err(|e| IngestRouterError::ResponseBuildError(e.to_string())) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use url::Url; - - #[tokio::test] - async fn test_response_building_and_v3_protocol() { - let handler = RelayProjectConfigsHandler::new(HashMap::new(), RelayTimeouts::default()); - - // Test: Empty response - let response = handler - .build_merged_response(HashMap::new(), Vec::new(), HashMap::new(), HeaderMap::new()) - .unwrap(); - assert_eq!(response.status(), StatusCode::OK); - let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); - let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); - assert_eq!(parsed, serde_json::json!({"configs": {}})); - - // Test: Multiple configs with field preservation - let mut configs = HashMap::new(); - configs.insert( - "project1".to_string(), - serde_json::json!({ - "disabled": false, - "slug": "test-project", - "organizationId": 42, - "projectId": 100, - "customField": "customValue" - }), - ); - configs.insert( - "project2".to_string(), - serde_json::json!({"config": "value2"}), - ); - - let response = handler - .build_merged_response(configs, Vec::new(), HashMap::new(), HeaderMap::new()) - .unwrap(); - let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); - let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); - assert!(parsed["configs"].get("project1").is_some()); - assert!(parsed["configs"].get("project2").is_some()); - assert_eq!(parsed["configs"]["project1"]["disabled"], false); - assert_eq!(parsed["configs"]["project1"]["slug"], "test-project"); - assert_eq!(parsed["configs"]["project1"]["customField"], "customValue"); - - // Test: V3 protocol - Pending array with values - let pending = vec!["key1".to_string(), "key2".to_string(), "key3".to_string()]; - let response = handler - .build_merged_response(HashMap::new(), pending, HashMap::new(), HeaderMap::new()) - .unwrap(); - let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); - let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); - assert_eq!(parsed["pending"].as_array().unwrap().len(), 3); - assert_eq!(parsed["pending"][0], "key1"); - - // Test: V3 protocol - Empty pending omitted - let response = handler - .build_merged_response(HashMap::new(), Vec::new(), HashMap::new(), HeaderMap::new()) - .unwrap(); - let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); - let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); - assert!(parsed.get("pending").is_none()); - - // Test: V3 protocol - Global config and extra fields - let mut extra = HashMap::new(); - extra.insert( - "global".to_string(), - serde_json::json!({"measurements": {"maxCustomMeasurements": 10}}), - ); - extra.insert("global_status".to_string(), serde_json::json!("ready")); - extra.insert( - "futureFeature".to_string(), - serde_json::json!({"enabled": true}), - ); - - let response = handler - .build_merged_response(HashMap::new(), Vec::new(), extra, HeaderMap::new()) - .unwrap(); - let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); - let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); - assert_eq!( - parsed["global"]["measurements"]["maxCustomMeasurements"], - 10 - ); - assert_eq!(parsed["global_status"], "ready"); - assert_eq!(parsed["futureFeature"]["enabled"], true); - } - - #[tokio::test] - async fn test_all_upstreams_fail_returns_error() { - // Set up handler with invalid upstreams - let mut locales = HashMap::new(); - locales.insert( - "us".to_string(), - vec![CellConfig { - name: "us-cell-1".to_string(), - relay_url: Url::parse("http://invalid-relay.example.com:8080").unwrap(), - sentry_url: Url::parse("http://invalid-sentry.example.com:8080").unwrap(), - }], - ); - - let handler = RelayProjectConfigsHandler::new(locales, RelayTimeouts::default()); - - // Create a request with public keys - let request_body = serde_json::json!({ - "publicKeys": ["test-key-1", "test-key-2"] - }); - let body_bytes = Bytes::from(serde_json::to_vec(&request_body).unwrap()); - - // Create empty cells to simulate all upstreams failing - let empty_cells = Cells { - cell_list: Vec::new(), - cell_to_upstreams: HashMap::new(), - }; - - // Create a base request - let request = Request::builder().method("POST").uri("/").body(()).unwrap(); - - // This should return an error since no upstreams will succeed (empty targets) - let result = handler - .handle_with_targets(&empty_cells, request, body_bytes) - .await; - - assert!(result.is_err()); - match result { - Err(IngestRouterError::ServiceUnavailable(msg)) => { - assert_eq!(msg, "All upstream cells are unavailable"); - // Verify the error maps to 503 status code - let error = IngestRouterError::ServiceUnavailable(msg); - assert_eq!(error.status_code(), hyper::StatusCode::SERVICE_UNAVAILABLE); - } - _ => panic!("Expected ServiceUnavailable error"), - } - } - - #[tokio::test] - async fn test_v3_upstream_failure_adds_keys_to_pending() { - // Set up handler with invalid upstream that will fail to connect - let mut locales = HashMap::new(); - locales.insert( - "us".to_string(), - vec![CellConfig { - name: "us-cell-1".to_string(), - relay_url: Url::parse("http://localhost:1").unwrap(), // Invalid port - sentry_url: Url::parse("http://localhost:1").unwrap(), // Will fail to connect - }], - ); - - let handler = RelayProjectConfigsHandler::new(locales.clone(), RelayTimeouts::default()); - - // Create a request with public keys - let request_body = serde_json::json!({ - "publicKeys": ["test-key-1", "test-key-2", "test-key-3"] - }); - let body_bytes = Bytes::from(serde_json::to_vec(&request_body).unwrap()); - - // Get cells - let locales = Locales::new(locales); - let cells = locales.get_cells("us").unwrap(); - - // Create a base request - let request = Request::builder().method("POST").uri("/").body(()).unwrap(); - - // When upstream fails, all its keys should be added to pending - let result = handler - .handle_with_targets(cells, request, body_bytes) - .await; - - // Should succeed (v3 protocol - returning pending is valid) - assert!(result.is_ok()); - let response = result.unwrap(); - assert_eq!(response.status(), StatusCode::OK); - - // Parse response - let body = response.into_body(); - let body_bytes = body.collect().await.unwrap().to_bytes(); - let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); - - // Verify configs is empty (upstream failed) - assert!(parsed["configs"].as_object().unwrap().is_empty()); - - // Verify all keys are in pending - assert!(parsed.get("pending").is_some()); - let pending = parsed["pending"].as_array().unwrap(); - assert_eq!(pending.len(), 3); - assert!(pending.contains(&serde_json::json!("test-key-1"))); - assert!(pending.contains(&serde_json::json!("test-key-2"))); - assert!(pending.contains(&serde_json::json!("test-key-3"))); - } - - #[test] - fn test_extra_fields_passthrough() { - // Test 1: Split behavior - all upstreams get the same extra fields (including global) - let mut locales = HashMap::new(); - locales.insert( - "us".to_string(), - vec![ - CellConfig { - name: "us-cell-1".to_string(), - relay_url: Url::parse("http://us1-relay:8080").unwrap(), - sentry_url: Url::parse("http://us1-sentry:8080").unwrap(), - }, - CellConfig { - name: "us-cell-2".to_string(), - relay_url: Url::parse("http://us2-relay:8080").unwrap(), - sentry_url: Url::parse("http://us2-sentry:8080").unwrap(), - }, - ], - ); - - let handler = RelayProjectConfigsHandler::new(locales.clone(), RelayTimeouts::default()); - let mut extra = HashMap::new(); - extra.insert("global".to_string(), serde_json::json!(true)); - let request = RelayProjectConfigsRequest { - public_keys: vec!["key1".to_string(), "key2".to_string()], - extra, - }; - - let locales_obj = Locales::new(locales); - let cells = locales_obj.get_cells("us").unwrap(); - let splits = handler.split_keys_by_upstream(&request, cells); - - assert_eq!(splits.len(), 2); - // All upstreams get the same extra fields (including global: true) - assert_eq!( - splits[0].request.extra.get("global"), - Some(&serde_json::json!(true)) - ); - assert_eq!( - splits[1].request.extra.get("global"), - Some(&serde_json::json!(true)) - ); - - // Test 2: Serialization - extra fields are included - let request = RelayProjectConfigsRequest { - public_keys: vec!["key1".to_string()], - extra: HashMap::new(), - }; - - let json = serde_json::to_string(&request).unwrap(); - let parsed: JsonValue = serde_json::from_str(&json).unwrap(); - assert!(parsed.get("global").is_none()); - assert!(parsed.get("publicKeys").is_some()); - } - - #[tokio::test] - async fn test_headers_from_highest_priority_cell() { - use hyper::header::{CACHE_CONTROL, HeaderValue}; - - let mut locales = HashMap::new(); - locales.insert( - "test".to_string(), - vec![CellConfig { - name: "test-cell".to_string(), - relay_url: Url::parse("http://localhost:8090").unwrap(), - sentry_url: Url::parse("http://localhost:8080").unwrap(), - }], - ); - - let handler = RelayProjectConfigsHandler::new(locales, RelayTimeouts::default()); - - // Create headers from upstream (simulating what we'd get from highest priority cell) - let mut upstream_headers = HeaderMap::new(); - upstream_headers.insert(CACHE_CONTROL, HeaderValue::from_static("max-age=300")); - upstream_headers.insert( - "X-Sentry-Rate-Limit-Remaining", - HeaderValue::from_static("100"), - ); - - // Add a hop-by-hop header that should be filtered - upstream_headers.insert( - hyper::header::CONNECTION, - HeaderValue::from_static("keep-alive"), - ); - - let response = handler - .build_merged_response(HashMap::new(), Vec::new(), HashMap::new(), upstream_headers) - .unwrap(); - - // Verify headers are copied - assert_eq!( - response.headers().get(CACHE_CONTROL), - Some(&HeaderValue::from_static("max-age=300")) - ); - assert_eq!( - response.headers().get("X-Sentry-Rate-Limit-Remaining"), - Some(&HeaderValue::from_static("100")) - ); - - // Verify hop-by-hop headers are filtered out - assert!(response.headers().get(hyper::header::CONNECTION).is_none()); - - // Verify Content-Type is always set - assert_eq!( - response.headers().get(CONTENT_TYPE), - Some(&HeaderValue::from_static("application/json")) - ); - } -} diff --git a/ingest-router/src/relay_project_config_handler/merger.rs b/ingest-router/src/relay_project_config_handler/merger.rs new file mode 100644 index 0000000..0550685 --- /dev/null +++ b/ingest-router/src/relay_project_config_handler/merger.rs @@ -0,0 +1,249 @@ +//! Response merging logic for combining upstream responses. +//! +//! This module handles merging responses from multiple upstream Sentry instances +//! following the v3 protocol merge strategy: +//! - Configs: HashMap merge (all configs from all upstreams) +//! - Pending: Array concatenation (includes failed keys) +//! - Extra fields: Priority-based selection (highest priority cell wins) + +use crate::errors::IngestRouterError; +use http_body_util::{BodyExt, Full, combinators::BoxBody}; +use hyper::body::Bytes; +use hyper::header::{CONTENT_LENGTH, CONTENT_TYPE, HeaderMap}; +use hyper::{Response, StatusCode}; +use serde_json::Value as JsonValue; +use std::collections::HashMap; + +use super::protocol::ProjectConfigsResponse; + +/// Merged results from all upstream tasks. +pub struct MergedResults { + /// All configs merged from successful upstreams. + pub project_configs: HashMap, + /// All pending keys (from failed upstreams or upstream pending arrays). + pub pending_keys: Vec, + /// Extra fields (global config, status, etc.). + pub extra_fields: HashMap, + /// Headers from the highest priority upstream + pub http_headers: HeaderMap, +} + +impl MergedResults { + /// Creates a new empty MergedResults. + pub fn new() -> Self { + Self { + project_configs: HashMap::new(), + pending_keys: Vec::new(), + extra_fields: HashMap::new(), + http_headers: HeaderMap::new(), + } + } + + /// Merges configs from a successful upstream response. + pub fn merge_project_configs(&mut self, configs: HashMap) { + self.project_configs.extend(configs); + } + + /// Adds keys to the pending array (for failed upstreams or retry). + pub fn add_pending_keys(&mut self, keys: Vec) { + self.pending_keys.extend(keys); + } + + /// Checks if results are empty (no configs, no pending, had keys to request). + pub fn is_empty(&self, had_keys_to_request: bool) -> bool { + self.project_configs.is_empty() && self.pending_keys.is_empty() && had_keys_to_request + } + + /// Builds an HTTP response from the merged results. + /// + /// Uses headers from the highest priority cell (same cell used for global config). + /// Filters out hop-by-hop headers and Content-Length (which is recalculated for the new body). + pub fn into_response( + mut self, + ) -> Result>, IngestRouterError> { + let response = ProjectConfigsResponse { + project_configs: self.project_configs, + pending_keys: if self.pending_keys.is_empty() { + None + } else { + Some(self.pending_keys) + }, + extra_fields: self.extra_fields, + }; + + let merged_json = serde_json::to_vec(&response) + .map_err(|e| IngestRouterError::ResponseSerializationError(e.to_string()))?; + + // Filter hop-by-hop headers (connection-specific, not forwarded) + shared::http::filter_hop_by_hop(&mut self.http_headers, hyper::Version::HTTP_11); + + // Remove Content-Length since body size changed after merging + self.http_headers.remove(CONTENT_LENGTH); + + let mut builder = Response::builder().status(StatusCode::OK); + + for (name, value) in self.http_headers.iter() { + builder = builder.header(name, value); + } + + builder = builder.header(CONTENT_TYPE, "application/json"); + + builder + .body( + Full::new(Bytes::from(merged_json)) + .map_err(|e| match e {}) + .boxed(), + ) + .map_err(|e| IngestRouterError::ResponseBuildError(e.to_string())) + } +} + +impl Default for MergedResults { + fn default() -> Self { + Self::new() + } +} + +#[cfg(test)] +mod tests { + use super::*; + use http_body_util::BodyExt; + use hyper::header::{CACHE_CONTROL, HeaderValue}; + + #[tokio::test] + async fn test_empty_response() { + let results = MergedResults::new(); + let response = results.into_response().unwrap(); + + assert_eq!(response.status(), StatusCode::OK); + let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); + let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); + assert_eq!(parsed, serde_json::json!({"configs": {}})); + } + + #[tokio::test] + async fn test_configs_merge() { + let mut results = MergedResults::new(); + + let mut configs1 = HashMap::new(); + configs1.insert( + "key1".to_string(), + serde_json::json!({"disabled": false, "slug": "project1"}), + ); + + let mut configs2 = HashMap::new(); + configs2.insert( + "key2".to_string(), + serde_json::json!({"disabled": false, "slug": "project2"}), + ); + + results.merge_project_configs(configs1); + results.merge_project_configs(configs2); + + let response = results.into_response().unwrap(); + let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); + let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); + + assert_eq!(parsed["configs"].as_object().unwrap().len(), 2); + assert!(parsed["configs"].get("key1").is_some()); + assert!(parsed["configs"].get("key2").is_some()); + } + + #[tokio::test] + async fn test_pending_handling() { + let mut results = MergedResults::new(); + + results.add_pending_keys(vec!["key1".to_string(), "key2".to_string()]); + results.add_pending_keys(vec!["key3".to_string()]); + + let response = results.into_response().unwrap(); + let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); + let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); + + assert_eq!(parsed["pending"].as_array().unwrap().len(), 3); + } + + #[tokio::test] + async fn test_empty_pending_omitted() { + let results = MergedResults::new(); + let response = results.into_response().unwrap(); + let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); + let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); + + assert!(parsed.get("pending").is_none()); + } + + #[tokio::test] + async fn test_headers_forwarding() { + let mut results = MergedResults::new(); + + // Add upstream headers + results + .http_headers + .insert(CACHE_CONTROL, HeaderValue::from_static("max-age=300")); + results.http_headers.insert( + "X-Sentry-Rate-Limit-Remaining", + HeaderValue::from_static("100"), + ); + // Add hop-by-hop header that should be filtered + results.http_headers.insert( + hyper::header::CONNECTION, + HeaderValue::from_static("keep-alive"), + ); + + let response = results.into_response().unwrap(); + + // Verify headers are copied + assert_eq!( + response.headers().get(CACHE_CONTROL), + Some(&HeaderValue::from_static("max-age=300")) + ); + assert_eq!( + response.headers().get("X-Sentry-Rate-Limit-Remaining"), + Some(&HeaderValue::from_static("100")) + ); + + // Verify hop-by-hop headers are filtered out + assert!(response.headers().get(hyper::header::CONNECTION).is_none()); + + // Verify Content-Type is always set + assert_eq!( + response.headers().get(CONTENT_TYPE), + Some(&HeaderValue::from_static("application/json")) + ); + } + + #[tokio::test] + async fn test_extra_fields() { + let mut results = MergedResults::new(); + + results.extra_fields.insert( + "global".to_string(), + serde_json::json!({"measurements": {"maxCustomMeasurements": 10}}), + ); + results + .extra_fields + .insert("global_status".to_string(), serde_json::json!("ready")); + + let response = results.into_response().unwrap(); + let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); + let parsed: JsonValue = serde_json::from_slice(&body_bytes).unwrap(); + + assert_eq!( + parsed["global"]["measurements"]["maxCustomMeasurements"], + 10 + ); + assert_eq!(parsed["global_status"], "ready"); + } + + #[test] + fn test_is_empty() { + let results = MergedResults::new(); + + // Empty with keys requested = empty + assert!(results.is_empty(true)); + + // Empty without keys requested = not empty (valid state) + assert!(!results.is_empty(false)); + } +} diff --git a/ingest-router/src/relay_project_config_handler/mod.rs b/ingest-router/src/relay_project_config_handler/mod.rs new file mode 100644 index 0000000..1a7bb1c --- /dev/null +++ b/ingest-router/src/relay_project_config_handler/mod.rs @@ -0,0 +1,489 @@ +//! Relay Project Configuration Handler +//! +//! This module implements the split-merge strategy for handling Sentry's +//! `/api/0/relays/projectconfigs/` endpoint across multiple upstream Sentry instances +//! in a multi-cell architecture. +//! +//! # Protocol Overview +//! +//! The Relay Project Configs endpoint (version 3) is used by Relay instances to fetch +//! project configurations needed to process events. This implementation acts as an +//! aggregation layer that: +//! 1. Splits requests across multiple upstream Sentry cells based on project ownership +//! 2. Fans out requests in parallel to multiple upstreams +//! 3. Merges responses back into a single v3-compliant response +//! 4. Passes through all config data unchanged +//! +//! ## Endpoint Details +//! The endpoint implementation is at +//! +//! - **Path**: `/api/0/relays/projectconfigs/` +//! - **Method**: `POST` +//! - **Protocol Version**: 3 +//! - **Authentication**: RelayAuthentication (X-Sentry-Relay-Id, X-Sentry-Relay-Signature) +//! +//! # Request Format (Version 3) +//! +//! ```json +//! { +//! "publicKeys": ["key1", "key2", "key3"], +//! "noCache": false, +//! "global": true +//! } +//! ``` +//! +//! ## Request Fields +//! +//! - **`publicKeys`** (required): Array of project public keys (DSN keys) to fetch configs for +//! - **`noCache`** (optional): If `true`, bypass caching and compute fresh configs (downgrades to v2 behavior) +//! - **`global`** (optional): If `true`, include global relay configuration in response +//! +//! # Response Format (Version 3) +//! +//! ```json +//! { +//! "configs": { +//! "key1": { +//! "disabled": false, +//! "slug": "project-slug", +//! "publicKeys": [...], +//! "config": {...}, +//! "organizationId": 1, +//! "projectId": 42 +//! } +//! }, +//! "pending": ["key2", "key3"], +//! "global": { +//! "measurements": {...}, +//! "options": {...} +//! }, +//! "global_status": "ready" +//! } +//! ``` +//! +//! ## Response Fields +//! +//! - **`configs`**: Map of public keys to their project configurations +//! - Configs are passed through unchanged from upstream Sentry instances +//! - They will add the relevant processing relay URL in the response +//! - **`pending`**: Array of public keys for which configs are being computed asynchronously +//! - Relay should retry the request later to fetch these configs +//! - Also used when upstreams fail/timeout (graceful degradation) +//! - **`global`**: Global relay configuration (only if `global: true` in request) +//! - **`global_status`**: Status of global config (always "ready" when present) +//! +//! # Implementation Details +//! +//! ## Request Splitting Strategy +//! +//! When a request arrives with multiple public keys: +//! +//! 1. **Route each key to its owning cell** +//! - Currently: Round-robin distribution (TODO: replace with control plane lookup) +//! - TODO: Query locator service for each public key to get cell name +//! +//! 2. **Group keys by target upstream** +//! - Keys routed to the same cell are batched into one request +//! +//! 3. **Handle global config flag** +//! - All upstreams receive the same `global` flag value as the original request +//! - Global config is selected from the highest priority cell that returns it +//! - Priority is determined by cell order in configuration (first = highest priority) +//! - Enables failover: if highest priority cell fails, next cell's global config is used +//! +//! ## Response Merging Strategy +//! +//! Responses from multiple upstreams are merged as follows: +//! +//! ### Configs (HashMap merge) +//! - Merge all `configs` HashMaps from all upstreams +//! - Configs are passed through unchanged (no modifications) +//! - relay_url is expected to be added in the upstream response +//! +//! ### Pending (Array concatenation) +//! - Concatenate all `pending` arrays from all upstream responses +//! - Include keys from failed/timed-out upstreams +//! - Relay will retry these keys in a subsequent request +//! +//! ### Extra fields (Priority-based selection) +//! - Select `extra` fields from highest priority cell that responds successfully +//! - Priority determined by cell order in configuration (first = highest) +//! - Forward compatibility: new fields are automatically preserved +//! +//! ## Error Handling +//! +//! ### Partial Failures (Graceful Degradation) +//! - If some upstreams succeed: Return successful configs + pending for failed keys +//! - Failed keys are added to `pending` array (v3 protocol) +//! - Logged but does not block response +//! +//! ### Total Failure +//! - If all upstreams fail: Check if any keys were added to pending +//! - If pending is not empty: Return 200 OK with pending array (relay will retry) +//! - If pending is empty: Return 503 error (no recoverable state) +//! +//! ### Upstream Failure Scenarios +//! - **Timeout**: All keys from that upstream → pending +//! - **Connection error**: All keys from that upstream → pending +//! - **Parse error**: All keys from that upstream → pending +//! - **Task panic**: Logged error (extreme edge case, keys lost) +//! +//! ## Request Flow +//! +//! ### Success Scenario +//! +//! ```text +//! ┌─────────────┐ +//! │ Relay │ +//! └──────┬──────┘ +//! │ +//! │ POST /api/0/relays/projectconfigs/ +//! │ {publicKeys: [A,B,C,D,E,F]} +//! │ +//! ▼ +//! ┌──────────────────────────────────────┐ +//! │ Ingest Router (this handler) │ +//! │ │ +//! │ 1. Parse request │ +//! │ 2. Split keys by cell: │ +//! │ • US1 → [A,C,E] │ +//! │ • US2 → [B,D,F] │ +//! └───┬──────────────────────────┬───────┘ +//! │ │ +//! │ {publicKeys: [A,C,E], │ {publicKeys: [B,D,F], +//! │ global: true} │ global: true} +//! │ │ +//! ▼ ▼ +//! ┌──────────┐ ┌──────────┐ +//! │Cell US1 │ │Cell US2 │ +//! │(Sentry) │ │(Sentry) │ +//! └────┬─────┘ └─────┬────┘ +//! │ │ +//! │ {configs: {A,C,E}} │ {configs: {B,D,F}} +//! │ │ +//! └──────────┬───────────────┘ +//! ▼ +//! ┌──────────────────────────────────────┐ +//! │ Ingest Router (this handler) │ +//! │ │ +//! │ 3. Merge responses: │ +//! │ • Combine all configs │ +//! │ • Merge pending arrays │ +//! │ • Select others from priority │ +//! └──────────────┬───────────────────────┘ +//! │ +//! │ {configs: {A,B,C,D,E,F}, +//! │ global: {...}} +//! │ +//! ▼ +//! ┌─────────────┐ +//! │ Relay │ +//! └─────────────┘ +//! ``` +//! +//! ### Failure Scenario (Graceful Degradation) +//! +//! When an upstream fails or times out, keys are added to the `pending` array: +//! +//! ```text +//! ┌─────────────┐ +//! │ Relay │ +//! └──────┬──────┘ +//! │ +//! │ POST {publicKeys: [A,B,C,D,E,F]} +//! │ +//! ▼ +//! ┌──────────────────────────────────────┐ +//! │ Ingest Router (this handler) │ +//! │ Split: US1→[A,C,E], US2→[B,D,F] │ +//! └───┬──────────────────────────┬───────┘ +//! │ │ +//! ▼ ▼ +//! ┌──────────┐ ┌──────────┐ +//! │Cell US1 │ │Cell US2 │ +//! └────┬─────┘ └─────┬────┘ +//! │ │ +//! │ ✓ Success │ ✗ Timeout/Error +//! │ {configs: {A,C,E}} │ +//! │ │ +//! └──────────┬───────────────┘ +//! ▼ +//! ┌──────────────────────────────────────┐ +//! │ Ingest Router (this handler) │ +//! │ │ +//! │ • Collect successful: {A,C,E} │ +//! │ • Add failed to pending: [B,D,F] │ +//! └──────────────┬───────────────────────┘ +//! │ +//! │ {configs: {A,C,E}, +//! │ pending: [B,D,F]} +//! │ +//! ▼ +//! ┌─────────────┐ +//! │ Relay │ (will retry pending) +//! └─────────────┘ +//! ``` +//! +//! # Examples +//! +//! ## Example 1: All upstreams succeed +//! +//! **Request**: +//! ```json +//! {"publicKeys": ["key1", "key2"]} +//! ``` +//! +//! **Response**: +//! ```json +//! { +//! "configs": { +//! "key1": {"disabled": false, "slug": "project-us1", ...}, +//! "key2": {"disabled": false, "slug": "project-us2", ...} +//! } +//! } +//! ``` +//! +//! ## Example 2: One upstream fails +//! +//! **Request**: +//! ```json +//! {"publicKeys": ["key1", "key2", "key3"]} +//! ``` +//! +//! **Response** (if upstream with key2,key3 times out): +//! ```json +//! { +//! "configs": { +//! "key1": {"disabled": false, "slug": "project-us1", ...} +//! }, +//! "pending": ["key2", "key3"] +//! } +//! ``` +//! +//! ## Example 3: Request with global config +//! +//! **Request**: +//! ```json +//! {"publicKeys": ["key1", "key2"], "global": true} +//! ``` +//! +//! **Splitting**: +//! - Request to US1: `{"publicKeys": ["key1"], "global": true}` +//! - Request to US2: `{"publicKeys": ["key2"], "global": true}` +//! +//! (US1 has higher priority, so its global config will be used in the final response) +//! +//! **Response**: +//! ```json +//! { +//! "configs": {...}, +//! "global": {"measurements": {...}}, +//! "global_status": "ready" +//! } +//! ``` +//! +//! # Timeout Configuration +//! +//! The handler uses a two-layer timeout strategy: +//! +//! ## HTTP Timeout (per-request) +//! - Applied to individual HTTP requests to upstream Sentry instances +//! - Fixed timeout for each request +//! - Default: 15 seconds +//! - Configured via `relay_timeouts.http_timeout_secs` +//! +//! ## Task Timeout (adaptive) +//! - **Initial timeout**: Wait for first upstream to respond +//! - Default: 20 seconds +//! - Must be >= HTTP timeout to allow at least one request to complete +//! - Configured via `relay_timeouts.task_initial_timeout_secs` +//! +//! - **Subsequent timeout**: Global deadline after first success +//! - Once first upstream succeeds, ALL remaining tasks have this much time total +//! - Prevents slow/down cells from blocking progress +//! - Default: 5 seconds +//! - Configured via `relay_timeouts.task_subsequent_timeout_secs` +//! +//! ### Rationale +//! +//! The adaptive timeout strategy ensures: +//! 1. We wait long enough for at least one cell to respond +//! 2. Fast cells aren't blocked by slow cells +//! 3. We don't wait indefinitely for stragglers once we have good data +//! 4. Failed/timed-out keys are added to `pending` for retry +//! +//! # See Also +//! +//! - [`RelayProjectConfigsHandler`] - Main handler struct for processing requests + +mod merger; +mod protocol; +mod splitter; +mod task_executor; + +use crate::config::{CellConfig, RelayTimeouts}; +use crate::errors::IngestRouterError; +use crate::locale::Locales; +use http_body_util::combinators::BoxBody; +use hyper::body::{Body, Bytes}; +use hyper::{Request, Response}; +use hyper_util::client::legacy::Client; +use hyper_util::client::legacy::connect::HttpConnector; +use hyper_util::rt::TokioExecutor; +use std::collections::HashMap; + +use protocol::ProjectConfigsRequest; +use splitter::PublicKeyRouter; +use task_executor::UpstreamTaskExecutor; + +/// Handler for Relay Project Configs endpoint. +pub struct RelayProjectConfigsHandler { + /// Executes parallel upstream requests with adaptive timeout strategy + executor: UpstreamTaskExecutor, + + /// Maps locale names to their configured upstream Sentry cells + locales: Locales, + + /// Routes public keys to their owning cells + router: PublicKeyRouter, +} + +impl RelayProjectConfigsHandler { + pub fn new(locales_config: HashMap>, timeouts: RelayTimeouts) -> Self { + let connector = HttpConnector::new(); + let client = Client::builder(TokioExecutor::new()).build(connector); + + Self { + executor: UpstreamTaskExecutor::new(client, timeouts), + locales: Locales::new(locales_config), + router: PublicKeyRouter::new(), + } + } + + pub async fn handle( + &self, + locale: &str, + request: Request, + ) -> Result>, IngestRouterError> + where + B: Body + Send + 'static, + B::Data: Send, + B::Error: std::error::Error + Send + Sync + 'static, + { + let cells = self.locales.get_cells(locale).ok_or_else(|| { + IngestRouterError::InternalError(format!("No cells configured for locale: {}", locale)) + })?; + + let (parts, body) = request.into_parts(); + let body_bytes = http_body_util::BodyExt::collect(body) + .await + .map(|collected| collected.to_bytes()) + .map_err(|e| IngestRouterError::RequestBodyError(e.to_string()))?; + + let base_request = Request::from_parts(parts, ()); + + let request_data = ProjectConfigsRequest::from_bytes(&body_bytes).map_err(|e| { + IngestRouterError::RequestBodyError(format!("Failed to parse request: {e}")) + })?; + + let split_requests = self.router.split(&request_data, cells); + + let results = self + .executor + .execute(split_requests, &base_request, cells) + .await?; + + if results.is_empty(!request_data.public_keys.is_empty()) { + return Err(IngestRouterError::ServiceUnavailable( + "All upstream cells are unavailable".to_string(), + )); + } + + results.into_response() + } +} + +#[cfg(test)] +mod tests { + use super::*; + use http_body_util::BodyExt; + use url::Url; + + #[tokio::test] + async fn test_no_cells_returns_error() { + let mut locales = HashMap::new(); + locales.insert("us".to_string(), vec![]); + + let handler = RelayProjectConfigsHandler::new(locales, RelayTimeouts::default()); + + let request_body = serde_json::json!({ + "publicKeys": ["test-key-1", "test-key-2"] + }); + let body_bytes = Bytes::from(serde_json::to_vec(&request_body).unwrap()); + + let request = Request::builder() + .method("POST") + .uri("/") + .body( + http_body_util::Full::new(body_bytes) + .map_err(|e| match e {}) + .boxed(), + ) + .unwrap(); + + let result = handler.handle("us", request).await; + + // With no cells, we get no split requests, no results, and should error + assert!(result.is_err()); + match result { + Err(IngestRouterError::ServiceUnavailable(msg)) => { + assert_eq!(msg, "All upstream cells are unavailable"); + } + _ => panic!("Expected ServiceUnavailable error"), + } + } + + #[tokio::test] + async fn test_v3_upstream_failure_adds_keys_to_pending() { + let mut locales = HashMap::new(); + locales.insert( + "us".to_string(), + vec![CellConfig { + name: "us-cell-1".to_string(), + relay_url: Url::parse("http://localhost:1").unwrap(), + sentry_url: Url::parse("http://localhost:1").unwrap(), + }], + ); + + let handler = RelayProjectConfigsHandler::new(locales, RelayTimeouts::default()); + + let request_body = serde_json::json!({ + "publicKeys": ["test-key-1", "test-key-2", "test-key-3"] + }); + let body_bytes = Bytes::from(serde_json::to_vec(&request_body).unwrap()); + + let request = Request::builder() + .method("POST") + .uri("/") + .body( + http_body_util::Full::new(body_bytes) + .map_err(|e| match e {}) + .boxed(), + ) + .unwrap(); + + let result = handler.handle("us", request).await; + + assert!(result.is_ok()); + let response = result.unwrap(); + assert_eq!(response.status(), hyper::StatusCode::OK); + + let body_bytes = response.into_body().collect().await.unwrap().to_bytes(); + let parsed: serde_json::Value = serde_json::from_slice(&body_bytes).unwrap(); + + assert!(parsed["configs"].as_object().unwrap().is_empty()); + assert!(parsed.get("pending").is_some()); + let pending = parsed["pending"].as_array().unwrap(); + assert_eq!(pending.len(), 3); + } +} diff --git a/ingest-router/src/relay_project_config_handler/protocol.rs b/ingest-router/src/relay_project_config_handler/protocol.rs new file mode 100644 index 0000000..9f7b553 --- /dev/null +++ b/ingest-router/src/relay_project_config_handler/protocol.rs @@ -0,0 +1,136 @@ +//! Protocol types for the Relay Project Configs endpoint (v3). +//! +//! This module defines the request and response structures for Sentry's +//! `/api/0/relays/projectconfigs/` endpoint. +//! +//! See the module-level documentation in `mod.rs` for complete protocol details. + +use hyper::body::Bytes; +use serde::{Deserialize, Serialize}; +use serde_json::Value as JsonValue; +use std::collections::HashMap; + +/// Request format for the relay project configs endpoint. +/// +/// # Example +/// ```json +/// { +/// "publicKeys": ["key1", "key2", "key3"], +/// "noCache": false, +/// "global": true +/// } +/// ``` +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct ProjectConfigsRequest { + /// DSN public keys to fetch configs for. + #[serde(rename = "publicKeys")] + pub public_keys: Vec, + + /// Other fields (`global`, `noCache`, future fields) for forward compatibility. + /// All fields are passed through as-is to upstreams. + #[serde(flatten)] + pub extra_fields: HashMap, +} + +impl ProjectConfigsRequest { + pub fn from_bytes(bytes: &Bytes) -> Result { + serde_json::from_slice(bytes) + } + + pub fn to_bytes(&self) -> Result { + let json = serde_json::to_vec(self)?; + Ok(Bytes::from(json)) + } +} + +/// Response format for the relay project configs endpoint. +/// +/// # Example +/// ```json +/// { +/// "configs": { +/// "key1": { +/// "disabled": false, +/// "slug": "project-slug", +/// "publicKeys": [...], +/// "config": {...}, +/// "organizationId": 1, +/// "projectId": 42 +/// } +/// }, +/// "pending": ["key2", "key3"], +/// "global": {...}, +/// "global_status": "ready" +/// } +/// ``` +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct ProjectConfigsResponse { + /// Project configs (HashMap merged from all upstreams). + #[serde(rename = "configs")] + pub project_configs: HashMap, + + /// Keys being computed async or from failed upstreams (concatenated). + #[serde(rename = "pending", skip_serializing_if = "Option::is_none")] + pub pending_keys: Option>, + + /// Other fields (`global`, `global_status`, future fields). + #[serde(flatten)] + pub extra_fields: HashMap, +} + +impl ProjectConfigsResponse { + pub fn from_bytes(bytes: &Bytes) -> Result { + serde_json::from_slice(bytes) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_request_serialization() { + let mut extra = HashMap::new(); + extra.insert("global".to_string(), serde_json::json!(true)); + extra.insert("noCache".to_string(), serde_json::json!(false)); + + let request = ProjectConfigsRequest { + public_keys: vec!["key1".to_string(), "key2".to_string()], + extra_fields: extra, + }; + + let bytes = request.to_bytes().unwrap(); + let parsed = ProjectConfigsRequest::from_bytes(&bytes).unwrap(); + + assert_eq!(parsed.public_keys.len(), 2); + assert_eq!( + parsed.extra_fields.get("global"), + Some(&serde_json::json!(true)) + ); + } + + #[test] + fn test_response_serialization() { + let mut configs = HashMap::new(); + configs.insert( + "key1".to_string(), + serde_json::json!({ + "disabled": false, + "slug": "test-project" + }), + ); + + let response = ProjectConfigsResponse { + project_configs: configs, + pending_keys: Some(vec!["key2".to_string()]), + extra_fields: HashMap::new(), + }; + + let bytes = Bytes::from(serde_json::to_vec(&response).unwrap()); + let parsed = ProjectConfigsResponse::from_bytes(&bytes).unwrap(); + + assert_eq!(parsed.project_configs.len(), 1); + assert!(parsed.pending_keys.is_some()); + assert_eq!(parsed.pending_keys.unwrap().len(), 1); + } +} diff --git a/ingest-router/src/relay_project_config_handler/splitter.rs b/ingest-router/src/relay_project_config_handler/splitter.rs new file mode 100644 index 0000000..d4c28b3 --- /dev/null +++ b/ingest-router/src/relay_project_config_handler/splitter.rs @@ -0,0 +1,111 @@ +//! Key routing logic for distributing requests across upstream cells. +//! +//! Current: Round-robin distribution (placeholder) +//! TODO: Replace with control plane locator service lookup + +use crate::locale::{Cells, Upstream}; +use std::collections::HashMap; + +use super::protocol::ProjectConfigsRequest; + +/// A request split for a specific upstream cell. +#[derive(Clone)] +pub struct SplitRequest { + pub cell_name: String, + pub upstream: Upstream, + pub request: ProjectConfigsRequest, +} + +/// Routes public keys to their owning cells. +pub struct PublicKeyRouter { + // TODO: Add locator service in future PR +} + +impl PublicKeyRouter { + pub fn new() -> Self { + Self {} + } + + /// Splits request across cells using round-robin distribution. + /// + /// TODO: Replace with locator service lookup per key. + pub fn split(&self, request: &ProjectConfigsRequest, cells: &Cells) -> Vec { + if cells.cell_list.is_empty() { + return Vec::new(); + } + + let mut split: HashMap> = HashMap::new(); + + // TODO: Replace with locator service lookup + for (index, public_key) in request.public_keys.iter().enumerate() { + let cell_name = &cells.cell_list[index % cells.cell_list.len()]; + split + .entry(cell_name.clone()) + .or_default() + .push(public_key.clone()); + } + + split + .into_iter() + .map(|(cell_name, public_keys)| { + let upstream = cells + .cell_to_upstreams + .get(&cell_name) + .expect("Cell name in list must exist in HashMap"); + + SplitRequest { + cell_name, + upstream: upstream.clone(), + request: ProjectConfigsRequest { + public_keys, + extra_fields: request.extra_fields.clone(), + }, + } + }) + .collect() + } +} + +impl Default for PublicKeyRouter { + fn default() -> Self { + Self::new() + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::config::CellConfig; + use crate::locale::Locales; + use std::collections::HashMap; + use url::Url; + + #[test] + fn test_extra_fields_passthrough() { + let splitter = PublicKeyRouter::new(); + + let locales = HashMap::from([( + "us".to_string(), + vec![CellConfig { + name: "us1".to_string(), + sentry_url: Url::parse("http://us1:8080").unwrap(), + relay_url: Url::parse("http://us1:8090").unwrap(), + }], + )]); + + let locales_obj = Locales::new(locales); + let cells = locales_obj.get_cells("us").unwrap(); + + let mut extra = HashMap::new(); + extra.insert("global".to_string(), serde_json::json!(true)); + + let request = ProjectConfigsRequest { + public_keys: vec!["key1".to_string()], + extra_fields: extra.clone(), + }; + + let splits = splitter.split(&request, cells); + + assert_eq!(splits[0].request.extra_fields, extra); + } +} diff --git a/ingest-router/src/relay_project_config_handler/task_executor.rs b/ingest-router/src/relay_project_config_handler/task_executor.rs new file mode 100644 index 0000000..6624f61 --- /dev/null +++ b/ingest-router/src/relay_project_config_handler/task_executor.rs @@ -0,0 +1,606 @@ +//! Task execution for parallel upstream requests. + +use crate::config::RelayTimeouts; +use crate::errors::IngestRouterError; +use crate::http::send_to_upstream; +use crate::locale::Cells; +use http_body_util::Full; +use hyper::body::Bytes; +use hyper::{Request, Response}; +use hyper_util::client::legacy::Client; +use hyper_util::client::legacy::connect::HttpConnector; +use std::collections::HashMap; +use std::time::Duration; +use tokio::task::JoinSet; +use tokio::time::Instant; + +use super::merger::MergedResults; +use super::protocol::{ProjectConfigsRequest, ProjectConfigsResponse}; +use super::splitter::SplitRequest; + +/// Result from a single upstream task execution. +struct UpstreamTaskResult { + /// The cell name this result came from + cell_name: String, + + /// The public keys that were requested from this upstream. + /// If the request fails, these keys will be added to the pending array + public_keys: Vec, + + /// The HTTP response from the upstream + result: Result, IngestRouterError>, +} + +/// Collection of spawned upstream tasks with tracking metadata. +/// +/// Used to track which public keys were sent to which tasks, so that +/// if a task fails or is aborted, we can add its keys to the pending array. +struct SpawnedTasks { + /// The set of spawned async tasks making requests to upstreams + join_set: JoinSet, + + /// Maps task IDs to their public keys for failure handling + task_keys: HashMap>, +} + +/// State machine for adaptive timeout collection strategy. +/// +/// The collection process has two phases: +/// 1. **WaitingForFirst**: Initial phase waiting for first successful response +/// 2. **CollectingRemaining**: After first success, collecting remaining responses with shorter deadline +enum CollectionState { + /// Initial phase: waiting for first successful response with long timeout. + WaitingForFirst, + + /// Subsequent phase: first success received, collecting remaining with shorter timeout. + /// Tracks when the first success occurred to calculate the subsequent deadline. + CollectingRemaining { first_success_at: Instant }, +} + +impl CollectionState { + /// Calculates the current deadline based on state and timeout configuration. + fn current_deadline(&self, timeouts: &RelayTimeouts) -> Instant { + match self { + CollectionState::WaitingForFirst => { + Instant::now() + Duration::from_secs(timeouts.task_initial_timeout_secs as u64) + } + CollectionState::CollectingRemaining { first_success_at } => { + *first_success_at + + Duration::from_secs(timeouts.task_subsequent_timeout_secs as u64) + } + } + } + + /// Transitions to the subsequent collection phase after first success. + /// Only transitions if currently in WaitingForFirst state. + fn transition_to_subsequent(&mut self) { + if matches!(self, CollectionState::WaitingForFirst) { + *self = CollectionState::CollectingRemaining { + first_success_at: Instant::now(), + }; + } + } +} + +/// Orchestrates parallel execution of upstream requests with adaptive timeout strategy. +/// +/// This executor fans out requests to multiple upstream Sentry instances simultaneously, +/// collects their responses, and merges them. It implements a two-phase adaptive timeout +/// strategy to balance responsiveness with resilience: +/// +/// 1. **Initial phase**: Wait up to `task_initial_timeout_secs` for the first upstream to respond +/// 2. **Subsequent phase**: Once first success occurs, give all remaining tasks `task_subsequent_timeout_secs` total to complete +/// +/// This ensures fast cells aren't blocked by slow/failing cells, while still allowing +/// sufficient time for healthy upstreams to respond. +pub struct UpstreamTaskExecutor { + /// HTTP client for making requests to upstream Sentry instances + client: Client>, + + /// Timeout configuration for HTTP requests and task-level deadlines + timeouts: RelayTimeouts, +} + +impl UpstreamTaskExecutor { + pub fn new(client: Client>, timeouts: RelayTimeouts) -> Self { + Self { client, timeouts } + } + + /// Spawns and collects results from all upstream tasks. + pub async fn execute( + &self, + split_requests: Vec, + base_request: &Request<()>, + cells: &Cells, + ) -> Result { + let spawned_tasks = self.spawn_upstream_tasks(split_requests, base_request)?; + let merged = self.collect_results(spawned_tasks, cells).await; + Ok(merged) + } + + /// Spawns parallel tasks for all upstream requests. + /// + /// Returns spawned tasks with tracking metadata for failure handling. + fn spawn_upstream_tasks( + &self, + split_requests: Vec, + base_request: &Request<()>, + ) -> Result { + let mut join_set = JoinSet::new(); + let mut task_keys = HashMap::new(); + + for split in split_requests { + let request = self.build_upstream_request(&split.request, base_request)?; + let public_keys_for_tracking = split.request.public_keys.clone(); + + let client = self.client.clone(); + let sentry_url = split.upstream.sentry_url; + let http_timeout = self.timeouts.http_timeout_secs as u64; + let cell_name = split.cell_name; + let public_keys = split.request.public_keys; + + let abort_handle = join_set.spawn(async move { + let result = send_to_upstream(&client, &sentry_url, request, http_timeout).await; + + UpstreamTaskResult { + cell_name, + public_keys, + result, + } + }); + + task_keys.insert(abort_handle.id(), public_keys_for_tracking); + } + + Ok(SpawnedTasks { + join_set, + task_keys, + }) + } + + /// Builds an HTTP request to send to an upstream Sentry instance. + /// + /// Copies method, URI, version, and headers from the original request, + /// but replaces the body with the split request data. + fn build_upstream_request( + &self, + split_request: &ProjectConfigsRequest, + base_request: &Request<()>, + ) -> Result>, IngestRouterError> { + let request_body = split_request.to_bytes().map_err(|e| { + IngestRouterError::InternalError(format!("Failed to serialize request: {e}")) + })?; + + let mut req_builder = Request::builder() + .method(base_request.method()) + .uri(base_request.uri()) + .version(base_request.version()); + + for (name, value) in base_request.headers() { + req_builder = req_builder.header(name, value); + } + + req_builder.body(Full::new(request_body)).map_err(|e| { + IngestRouterError::InternalError(format!("Failed to build HTTP request: {e}")) + }) + } + + /// Collects results with adaptive timeout strategy using state machine. + async fn collect_results(&self, spawned_tasks: SpawnedTasks, cells: &Cells) -> MergedResults { + let SpawnedTasks { + mut join_set, + mut task_keys, + } = spawned_tasks; + let mut results = MergedResults::new(); + let mut extra_by_cell = HashMap::new(); + let mut headers_by_cell = HashMap::new(); + + let mut state = CollectionState::WaitingForFirst; + let deadline = tokio::time::sleep_until(state.current_deadline(&self.timeouts)); + tokio::pin!(deadline); + + loop { + tokio::select! { + Some(join_result) = join_set.join_next_with_id() => { + if let Some((cell_name, extra, headers, request_succeeded)) = + self.handle_task_completion(join_result, &mut task_keys, &mut results) + { + extra_by_cell.insert(cell_name.clone(), extra); + headers_by_cell.insert(cell_name, headers); + + if request_succeeded && matches!(state, CollectionState::WaitingForFirst) { + state.transition_to_subsequent(); + deadline.as_mut().reset(state.current_deadline(&self.timeouts)); + } + } + } + _ = &mut deadline => { + let remaining = join_set.len(); + match state { + CollectionState::WaitingForFirst => { + tracing::error!( + "Initial timeout reached with no successful responses, aborting {} tasks", + remaining + ); + } + CollectionState::CollectingRemaining { .. } => { + tracing::debug!( + "Subsequent deadline reached after first success, aborting {} remaining tasks", + remaining + ); + } + } + join_set.abort_all(); + break; + } + else => { + tracing::debug!("All tasks completed"); + break; + } + } + } + + while let Some(result) = join_set.join_next_with_id().await { + let task_id = match result { + Ok((id, _)) => id, + Err(e) => e.id(), + }; + if let Some(keys) = task_keys.remove(&task_id) { + tracing::debug!("Adding {} keys from aborted task to pending", keys.len()); + results.add_pending_keys(keys); + } + } + + if let Some((extra, headers)) = cells.cell_list.iter().find_map(|name| { + extra_by_cell + .remove(name) + .map(|e| (e, headers_by_cell.remove(name).unwrap_or_default())) + }) { + results.extra_fields = extra; + results.http_headers = headers; + } + + results + } + + /// Handles the completion of a single upstream task. + /// + /// Processes task join results, handles failures, and extracts response metadata. + /// Returns cell name, extra fields, headers, and success status for successful requests. + /// Failed tasks have their keys added to pending. + fn handle_task_completion( + &self, + join_result: Result<(tokio::task::Id, UpstreamTaskResult), tokio::task::JoinError>, + task_keys: &mut HashMap>, + results: &mut MergedResults, + ) -> Option<( + String, + HashMap, + hyper::header::HeaderMap, + bool, + )> { + let (task_id, upstream_result) = match join_result { + Ok((id, result)) => (id, result), + Err(e) => { + tracing::error!("Task failed: {e}"); + if let Some(keys) = task_keys.remove(&e.id()) { + results.add_pending_keys(keys); + } + return None; + } + }; + + task_keys.remove(&task_id); + + let request_succeeded = upstream_result.result.is_ok(); + let (cell_name, extra, headers) = self.process_result(upstream_result, results)?; + Some((cell_name, extra, headers, request_succeeded)) + } + + fn process_result( + &self, + upstream_result: UpstreamTaskResult, + results: &mut MergedResults, + ) -> Option<( + String, + HashMap, + hyper::header::HeaderMap, + )> { + let cell_name = upstream_result.cell_name; + + let Ok(response) = upstream_result.result else { + results.add_pending_keys(upstream_result.public_keys); + return None; + }; + + let (parts, body) = response.into_parts(); + + match ProjectConfigsResponse::from_bytes(&body) { + Ok(data) => { + results.merge_project_configs(data.project_configs); + if let Some(pending) = data.pending_keys { + results.add_pending_keys(pending); + } + Some((cell_name, data.extra_fields, parts.headers)) + } + Err(e) => { + tracing::error!(error = %e, "Failed to parse response"); + results.add_pending_keys(upstream_result.public_keys); + None + } + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::config::CellConfig; + use crate::locale::Locales; + use hyper::Method; + use hyper::service::service_fn; + use hyper_util::rt::TokioExecutor; + use std::convert::Infallible; + use std::sync::Arc; + use tokio::net::TcpListener; + use tokio::sync::Mutex; + use url::Url; + + /// Start a mock HTTP server that responds with custom data + async fn start_mock_server(response_fn: F) -> u16 + where + F: Fn() -> ProjectConfigsResponse + Send + Sync + 'static, + { + let listener = TcpListener::bind("127.0.0.1:0").await.unwrap(); + let port = listener.local_addr().unwrap().port(); + let response_fn = Arc::new(Mutex::new(response_fn)); + + tokio::spawn(async move { + loop { + let (stream, _) = listener.accept().await.unwrap(); + let io = hyper_util::rt::TokioIo::new(stream); + let response_fn = response_fn.clone(); + + tokio::spawn(async move { + let service = service_fn(move |_req: Request| { + let response_fn = response_fn.clone(); + async move { + let response = (response_fn.lock().await)(); + let json = serde_json::to_vec(&response).unwrap(); + Ok::<_, Infallible>(Response::new(Full::new(Bytes::from(json)))) + } + }); + + let _ = hyper_util::server::conn::auto::Builder::new(TokioExecutor::new()) + .serve_connection(io, service) + .await; + }); + } + }); + + tokio::time::sleep(Duration::from_millis(50)).await; + port + } + + fn test_cells(ports: Vec<(&str, u16)>) -> Cells { + let cell_configs: Vec = ports + .into_iter() + .map(|(name, port)| CellConfig { + name: name.to_string(), + sentry_url: Url::parse(&format!("http://127.0.0.1:{}", port)).unwrap(), + relay_url: Url::parse(&format!("http://127.0.0.1:{}", port)).unwrap(), + }) + .collect(); + + let mut locales_map = HashMap::new(); + locales_map.insert("test".to_string(), cell_configs); + Locales::new(locales_map).get_cells("test").unwrap().clone() + } + + fn test_executor() -> UpstreamTaskExecutor { + let connector = HttpConnector::new(); + let client = Client::builder(TokioExecutor::new()).build(connector); + let timeouts = RelayTimeouts { + http_timeout_secs: 5, + task_initial_timeout_secs: 10, + task_subsequent_timeout_secs: 2, + }; + UpstreamTaskExecutor::new(client, timeouts) + } + + fn test_split_request(cells: &Cells, cell_name: &str, keys: Vec) -> SplitRequest { + let upstream = cells.cell_to_upstreams.get(cell_name).unwrap().clone(); + SplitRequest { + cell_name: cell_name.to_string(), + upstream, + request: ProjectConfigsRequest { + public_keys: keys, + extra_fields: HashMap::new(), + }, + } + } + + #[tokio::test] + async fn test_multiple_upstreams_all_succeed() { + // Mock server 1 returns config for key1 + let port1 = start_mock_server(|| { + let mut resp = ProjectConfigsResponse { + project_configs: HashMap::new(), + pending_keys: None, + extra_fields: HashMap::new(), + }; + resp.project_configs.insert( + "key1".to_string(), + serde_json::json!({"disabled": false, "slug": "project1"}), + ); + resp.extra_fields + .insert("global".to_string(), serde_json::json!({"from": "cell1"})); + resp + }) + .await; + + // Mock server 2 returns config for key2 + let port2 = start_mock_server(|| { + let mut resp = ProjectConfigsResponse { + project_configs: HashMap::new(), + pending_keys: None, + extra_fields: HashMap::new(), + }; + resp.project_configs.insert( + "key2".to_string(), + serde_json::json!({"disabled": false, "slug": "project2"}), + ); + resp.extra_fields + .insert("global".to_string(), serde_json::json!({"from": "cell2"})); + resp + }) + .await; + + let cells = test_cells(vec![("cell1", port1), ("cell2", port2)]); + let executor = test_executor(); + + let split_requests = vec![ + test_split_request(&cells, "cell1", vec!["key1".to_string()]), + test_split_request(&cells, "cell2", vec!["key2".to_string()]), + ]; + + let base_request = Request::builder() + .method(Method::POST) + .uri("/test") + .body(()) + .unwrap(); + + let results = executor + .execute(split_requests, &base_request, &cells) + .await + .unwrap(); + + // Both configs should be merged + assert_eq!(results.project_configs.len(), 2); + assert!(results.project_configs.contains_key("key1")); + assert!(results.project_configs.contains_key("key2")); + + // Should select extra fields from cell1 (highest priority) + assert_eq!( + results.extra_fields.get("global"), + Some(&serde_json::json!({"from": "cell1"})) + ); + + assert!(results.pending_keys.is_empty()); + } + + #[tokio::test] + async fn test_partial_failure() { + // Mock server 1 succeeds + let port1 = start_mock_server(|| { + let mut resp = ProjectConfigsResponse { + project_configs: HashMap::new(), + pending_keys: None, + extra_fields: HashMap::new(), + }; + resp.project_configs + .insert("key1".to_string(), serde_json::json!({"disabled": false})); + resp + }) + .await; + + // cell2 will fail (invalid port) + let cells = test_cells(vec![("cell1", port1), ("cell2", 1)]); + let executor = test_executor(); + + let split_requests = vec![ + test_split_request(&cells, "cell1", vec!["key1".to_string()]), + test_split_request( + &cells, + "cell2", + vec!["key2".to_string(), "key3".to_string()], + ), + ]; + + let base_request = Request::builder() + .method(Method::POST) + .uri("/test") + .body(()) + .unwrap(); + + let results = executor + .execute(split_requests, &base_request, &cells) + .await + .unwrap(); + + // Successful config from cell1 + assert_eq!(results.project_configs.len(), 1); + assert!(results.project_configs.contains_key("key1")); + + // Failed keys from cell2 should be in pending + assert_eq!(results.pending_keys.len(), 2); + assert!(results.pending_keys.contains(&"key2".to_string())); + assert!(results.pending_keys.contains(&"key3".to_string())); + } + + #[tokio::test] + async fn test_upstream_returns_pending_keys() { + let port = start_mock_server(|| { + let mut resp = ProjectConfigsResponse { + project_configs: HashMap::new(), + pending_keys: None, + extra_fields: HashMap::new(), + }; + resp.project_configs + .insert("key1".to_string(), serde_json::json!({"disabled": false})); + // Upstream says key2 is pending + resp.pending_keys = Some(vec!["key2".to_string()]); + resp + }) + .await; + + let cells = test_cells(vec![("cell1", port)]); + let executor = test_executor(); + + let split_requests = vec![test_split_request( + &cells, + "cell1", + vec!["key1".to_string(), "key2".to_string()], + )]; + + let base_request = Request::builder() + .method(Method::POST) + .uri("/test") + .body(()) + .unwrap(); + + let results = executor + .execute(split_requests, &base_request, &cells) + .await + .unwrap(); + + // key1 in configs + assert!(results.project_configs.contains_key("key1")); + + // key2 in pending (from upstream) + assert_eq!(results.pending_keys.len(), 1); + assert!(results.pending_keys.contains(&"key2".to_string())); + } + + #[tokio::test] + async fn test_empty_split_requests() { + let cells = test_cells(vec![("cell1", 8080)]); + let executor = test_executor(); + + let split_requests = vec![]; + let base_request = Request::builder() + .method(Method::POST) + .uri("/test") + .body(()) + .unwrap(); + + let results = executor + .execute(split_requests, &base_request, &cells) + .await + .unwrap(); + + assert!(results.project_configs.is_empty()); + assert!(results.pending_keys.is_empty()); + assert!(results.extra_fields.is_empty()); + } +}