From eeb4be5ae096c0697f35806a1af6cd1738cd2e2e Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Mon, 12 Feb 2024 12:58:39 +0000 Subject: [PATCH 01/60] Some functionality for assembling batches This version isn't working, it hangs during batch assembly. If I take the `receiver.await` out, then batch assembly does proceed, so something is stopping the execution from proceeding further up the pipeline is my guess. --- apollo-router/src/services/router/service.rs | 167 ++++++++++++++++-- apollo-router/src/services/subgraph.rs | 1 + .../src/services/subgraph_service.rs | 74 +++++++- .../src/services/supergraph/service.rs | 14 +- 4 files changed, 232 insertions(+), 24 deletions(-) diff --git a/apollo-router/src/services/router/service.rs b/apollo-router/src/services/router/service.rs index e3671a31e1..894feca300 100644 --- a/apollo-router/src/services/router/service.rs +++ b/apollo-router/src/services/router/service.rs @@ -1,5 +1,7 @@ //! Implements the router phase of the request lifecycle. +use std::collections::HashMap; +use std::fmt; use std::sync::Arc; use std::task::Poll; @@ -26,7 +28,9 @@ use http_body::Body as _; use hyper::Body; use mime::APPLICATION_JSON; use multimap::MultiMap; +use parking_lot::Mutex; use router_bridge::planner::Planner; +use tokio::sync::oneshot; use tower::BoxError; use tower::Layer; use tower::ServiceBuilder; @@ -62,6 +66,8 @@ use crate::services::HasPlugins; use crate::services::HasSchema; use crate::services::RouterRequest; use crate::services::RouterResponse; +use crate::services::SubgraphRequest; +use crate::services::SubgraphResponse; use crate::services::SupergraphCreator; use crate::services::SupergraphRequest; use crate::services::SupergraphResponse; @@ -609,13 +615,17 @@ impl RouterService { let ok_results = graphql_requests?; let mut results = Vec::with_capacity(ok_results.len()); + let batch_size = ok_results.len(); - if ok_results.len() > 1 { + let shared_batch_details: Option>> = if ok_results.len() > 1 { context .extensions() .lock() .insert(self.experimental_batching.clone()); - } + Some(Arc::new(Mutex::new(SharedBatchDetails::new(batch_size)))) + } else { + None + }; let mut ok_results_it = ok_results.into_iter(); let first = ok_results_it @@ -628,16 +638,16 @@ impl RouterService { // through the pipeline. This is because there is simply no way to clone http // extensions. // - // Secondly, we can't clone private_entries, but we need to propagate at least + // Secondly, we can't clone extensions, but we need to propagate at least // ClientRequestAccepts to ensure correct processing of the response. We do that manually, - // but the concern is that there may be other private_entries that wish to propagate into + // but the concern is that there may be other extensions that wish to propagate into // each request or we may add them in future and not know about it here... // - // (Technically we could clone private entries, since it is held under an `Arc`, but that - // would mean all the requests in a batch shared the same set of private entries and review + // (Technically we could clone extensions, since it is held under an `Arc`, but that + // would mean all the requests in a batch shared the same set of extensions and review // comments expressed the sentiment that this may be a bad thing...) // - for graphql_request in ok_results_it { + for (index, graphql_request) in ok_results_it.enumerate() { // XXX Lose http extensions, is that ok? let mut new = http_ext::clone_http_request(&sg); *new.body_mut() = graphql_request; @@ -649,22 +659,34 @@ impl RouterService { .lock() .get::() .cloned(); - if let Some(client_request_accepts) = client_request_accepts_opt { - new_context - .extensions() - .lock() - .insert(client_request_accepts); + { + // Sub-scope so that new_context_guard is dropped before pushing into the new + // SupergraphRequest + let mut new_context_guard = new_context.extensions().lock(); + if let Some(client_request_accepts) = client_request_accepts_opt { + new_context_guard.insert(client_request_accepts); + } + new_context_guard.insert(self.experimental_batching.clone()); + if let Some(shared_batch_details) = &shared_batch_details { + new_context_guard + .insert(BatchDetails::new(index + 1, shared_batch_details.clone())); + } } - new_context - .extensions() - .lock() - .insert(self.experimental_batching.clone()); + results.push(SupergraphRequest { supergraph_request: new, // Build a new context. Cloning would cause issues. context: new_context, }); } + + if let Some(shared_batch_details) = shared_batch_details { + context + .extensions() + .lock() + .insert(BatchDetails::new(0, shared_batch_details)); + } + results.insert( 0, SupergraphRequest { @@ -676,6 +698,119 @@ impl RouterService { } } +#[derive(Clone, Debug, Default)] +pub(crate) struct BatchDetails { + pub(crate) index: usize, + // Request Details + pub(crate) request: Option, + pub(crate) body: Option, + pub(crate) context: Option, + pub(crate) service_name: Option, + // Shared Request Details + shared: Arc>, +} + +impl fmt::Display for BatchDetails { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "index: {}", self.index)?; + // Use try_lock. If the shared details are locked, we won't display them. + // TODO: Maybe improve to handle the error...? + let guard = self.shared.try_lock().ok_or(fmt::Error)?; + write!(f, "size: {}", guard.size)?; + write!(f, "expected: {:?}", guard.expected)?; + write!(f, "seen: {:?}", guard.seen)?; + write!(f, "waiters: {:?}", guard.waiters) + } +} + +impl BatchDetails { + fn new(index: usize, shared: Arc>) -> Self { + Self { + index, + shared, + ..Default::default() + } + } + + pub(crate) fn ready(&self) -> bool { + self.shared.lock().ready() + } + + pub(crate) fn get_waiter( + &mut self, + request: SubgraphRequest, + body: graphql::Request, + context: Context, + service_name: &str, + ) -> oneshot::Receiver> { + tracing::info!("getting a waiter for {}", self.index); + self.request = Some(request); + self.body = Some(body); + self.context = Some(context); + self.service_name = Some(service_name.to_string()); + self.shared.lock().get_waiter(self.index) + } + + pub(crate) fn increment_subgraph_seen(&self) { + let mut shared_guard = self.shared.lock(); + let value = shared_guard.seen.entry(self.index).or_default(); + *value += 1; + } + + pub(crate) fn set_subgraph_fetches(&self, fetches: usize) { + let mut shared_guard = self.shared.lock(); + let value = shared_guard.expected.entry(self.index).or_default(); + *value = fetches; + } +} + +#[derive(Debug, Default)] +pub(crate) struct SharedBatchDetails { + pub(crate) size: usize, + pub(crate) expected: HashMap, + pub(crate) seen: HashMap, + pub(crate) waiters: HashMap>>>, +} + +impl SharedBatchDetails { + fn new(size: usize) -> Self { + Self { + size, + expected: HashMap::new(), + seen: HashMap::new(), + waiters: HashMap::new(), + } + } + + pub(crate) fn ready(&self) -> bool { + self.expected.len() == self.size && self.expected == self.seen + } + + pub(crate) fn get_waiter( + &mut self, + index: usize, + ) -> oneshot::Receiver> { + let (tx, rx) = oneshot::channel(); + let value = self.waiters.entry(index).or_default(); + value.push(tx); + rx + } +} + +/* + let (tx, mut rx) = mpsc::channel::(10); + let (b_tx, _b_rx) = broadcast::channel::<(usize, String)>(10); + + // Build up our batch co-ordinator for later use in subgraph processing + tokio::task::spawn(async move { + while let Some(body) = rx.recv().await { + todo!(); + } + // At this point we have our accumulated stuff, we need to figure out a way to send + // this to clients. + todo!() + }); +*/ struct TranslateError<'a> { status: StatusCode, error: &'a str, diff --git a/apollo-router/src/services/subgraph.rs b/apollo-router/src/services/subgraph.rs index 661b2b0d86..2f822ce213 100644 --- a/apollo-router/src/services/subgraph.rs +++ b/apollo-router/src/services/subgraph.rs @@ -37,6 +37,7 @@ pub(crate) type BoxGqlStream = Pin + Se assert_impl_all!(Request: Send); #[non_exhaustive] +#[derive(Debug)] pub struct Request { /// Original request to the Router. pub supergraph_request: Arc>, diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index 527c201ec7..6f8948cf66 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -57,6 +57,7 @@ use crate::protocols::websocket::convert_websocket_stream; use crate::protocols::websocket::GraphqlWebSocket; use crate::query_planner::OperationKind; use crate::services::layers::apq; +use crate::services::router::service::BatchDetails; use crate::services::SubgraphRequest; use crate::services::SubgraphResponse; use crate::Configuration; @@ -194,6 +195,7 @@ impl tower::Service for SubgraphService { } fn call(&mut self, mut request: SubgraphRequest) -> Self::Future { + tracing::info!("SUBGRAPH SERVICE CALLED"); let subscription_config = (request.operation_kind == OperationKind::Subscription) .then(|| self.subscription_config.clone()) .flatten(); @@ -234,6 +236,7 @@ impl tower::Service for SubgraphService { let arc_apq_enabled = self.apq.clone(); let mut notify = self.notify.clone(); + let make_calls = async move { // Subscription handling if request.operation_kind == OperationKind::Subscription @@ -364,7 +367,7 @@ impl tower::Service for SubgraphService { // with the same request body. let apq_enabled = arc_apq_enabled.as_ref(); if !apq_enabled.load(Relaxed) { - return call_http(request, body, context, client, &service_name).await; + return call_batched_http(request, body, context, client, &service_name).await; } // Else, if APQ is enabled, @@ -394,7 +397,7 @@ impl tower::Service for SubgraphService { extensions: extensions_with_apq, }; - let response = call_http( + let response = call_batched_http( request.clone(), apq_body.clone(), context.clone(), @@ -411,11 +414,11 @@ impl tower::Service for SubgraphService { match get_apq_error(gql_response) { APQError::PersistedQueryNotSupported => { apq_enabled.store(false, Relaxed); - call_http(request, body, context, client, &service_name).await + call_batched_http(request, body, context, client, &service_name).await } APQError::PersistedQueryNotFound => { apq_body.query = query; - call_http(request, apq_body, context, client, &service_name).await + call_batched_http(request, apq_body, context, client, &service_name).await } _ => Ok(response), } @@ -614,8 +617,68 @@ async fn call_websocket( )) } +/* +use tokio::sync::mpsc; +use tokio::sync::broadcast; +use tokio::sync::oneshot; + let (tx, mut rx) = mpsc::channel::(10); + let (b_tx, _b_rx) = broadcast::channel::<(usize, String)>(10); + + // Build up our batch co-ordinator for later use in subgraph processing + tokio::task::spawn(async move { + while let Some(body) = rx.recv().await { + todo!(); + } + // At this point we have our accumulated stuff, we need to figure out a way to send + // this to clients. + todo!() + }); +*/ + +async fn call_batched_http( + request: SubgraphRequest, + body: graphql::Request, + context: Context, + client: crate::services::http::BoxService, + service_name: &str, +) -> Result { + // We'd like to park a task here, but we can't park it whilst we have the context extensions + // lock held. That would be very bad... + // So, we set an optional listener and wait to hear back from the batch processor + let mut batch_responder: Option< + tokio::sync::oneshot::Receiver>, + > = None; + if let Some(batching) = context.extensions().lock().get_mut::() { + tracing::info!("in subgraph we have batching: {batching:?}, service: {service_name}"); + { + batching.increment_subgraph_seen(); + tracing::info!("ready to process batch?: {}", batching.ready()); + } + if batching.ready() { + //TODO: This is where we start processing our accumulated batch data + tracing::info!("Batch data: {batching:?}"); + todo!() + } else { + batch_responder = Some(batching.get_waiter( + request.clone(), + body.clone(), + context.clone(), + service_name, + )); + } + } + if let Some(receiver) = batch_responder { + println!("WE HAVE A WAITER"); + receiver.await; + todo!() + } else { + println!("WE CALLED HTTP"); + call_http(request, body, context, client, service_name).await + } +} + /// call_http makes http calls with modified graphql::Request (body) -async fn call_http( +pub(crate) async fn call_http( request: SubgraphRequest, body: graphql::Request, context: Context, @@ -634,6 +697,7 @@ async fn call_http( let (parts, _) = subgraph_request.into_parts(); let body = serde_json::to_string(&body).expect("JSON serialization should not fail"); + tracing::info!("our JSON body: {body:?}"); let mut request = http::Request::from_parts(parts, Body::from(body)); request diff --git a/apollo-router/src/services/supergraph/service.rs b/apollo-router/src/services/supergraph/service.rs index dff0594e40..0f0413f9e7 100644 --- a/apollo-router/src/services/supergraph/service.rs +++ b/apollo-router/src/services/supergraph/service.rs @@ -54,6 +54,7 @@ use crate::services::layers::query_analysis::ParsedDocument; use crate::services::layers::query_analysis::QueryAnalysisLayer; use crate::services::new_service::ServiceFactory; use crate::services::query_planner; +use crate::services::router::service::BatchDetails; use crate::services::router::ClientRequestAccepts; use crate::services::subgraph::BoxGqlStream; use crate::services::subgraph_service::MakeSubgraphService; @@ -606,19 +607,26 @@ async fn plan_query( context.extensions().lock().insert::(doc); } - planning + let qpr = planning .call( query_planner::CachingRequest::builder() .query(query_str) .and_operation_name(operation_name) - .context(context) + .context(context.clone()) .build(), ) .instrument(tracing::info_span!( QUERY_PLANNING_SPAN_NAME, "otel.kind" = "INTERNAL" )) - .await + .await?; + + if let Some(batching) = context.extensions().lock().get::() { + if let Some(QueryPlannerContent::Plan { plan, .. }) = &qpr.content { + batching.set_subgraph_fetches(plan.root.subgraph_fetches()); + } + } + Ok(qpr) } fn clone_supergraph_request( From bc131b03f37a881385ab0051925e16a1607f8c52 Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Wed, 14 Feb 2024 11:37:13 +0000 Subject: [PATCH 02/60] milestone reached: round trip, simple batches, works. As long as no elements of your query have fetches than "require" results, then your batch query will work. I still have more work to do to complete the "requires" components also: - There's a bunch of really horrible byte soup that needs to be cleaned up along with an absolute ton of re-factoring and functional decomposition. --- apollo-router/src/json_ext.rs | 9 + apollo-router/src/query_planner/plan.rs | 54 +++ apollo-router/src/request.rs | 4 +- apollo-router/src/response.rs | 38 +- apollo-router/src/services/router/service.rs | 60 +++- .../src/services/subgraph_service.rs | 335 +++++++++++++++--- .../src/services/supergraph/service.rs | 7 +- 7 files changed, 444 insertions(+), 63 deletions(-) diff --git a/apollo-router/src/json_ext.rs b/apollo-router/src/json_ext.rs index 3c7d7bd07f..2aaa92d5dd 100644 --- a/apollo-router/src/json_ext.rs +++ b/apollo-router/src/json_ext.rs @@ -37,6 +37,15 @@ macro_rules! extract_key_value_from_object { }}; } +macro_rules! ensure_array { + ($value:expr) => {{ + match $value { + crate::json_ext::Value::Array(a) => Ok(a), + _ => Err("invalid type, expected an array"), + } + }}; +} + macro_rules! ensure_object { ($value:expr) => {{ match $value { diff --git a/apollo-router/src/query_planner/plan.rs b/apollo-router/src/query_planner/plan.rs index e4c331738d..f4a229bb9c 100644 --- a/apollo-router/src/query_planner/plan.rs +++ b/apollo-router/src/query_planner/plan.rs @@ -223,6 +223,60 @@ impl PlanNode { } } + pub(crate) fn subgraph_fetches_no_requires(&self) -> usize { + match self { + PlanNode::Sequence { nodes } => { + nodes.iter().map(|n| n.subgraph_fetches_no_requires()).sum() + } + PlanNode::Parallel { nodes } => { + nodes.iter().map(|n| n.subgraph_fetches_no_requires()).sum() + } + PlanNode::Fetch(node) => { + if node.requires.is_empty() { + 1 + } else { + 0 + } + } + PlanNode::Flatten(node) => node.node.subgraph_fetches_no_requires(), + PlanNode::Defer { primary, deferred } => { + primary + .node + .as_ref() + .map_or(0, |n| n.subgraph_fetches_no_requires()) + + deferred + .iter() + .map(|n| { + n.node + .as_ref() + .map_or(0, |n| n.subgraph_fetches_no_requires()) + }) + .sum::() + } + // A `SubscriptionNode` makes a request to a subgraph, so counting it as 1 + PlanNode::Subscription { rest, .. } => { + rest.as_ref() + .map_or(0, |n| n.subgraph_fetches_no_requires()) + + 1 + } + // Compute the highest possible value for condition nodes + PlanNode::Condition { + if_clause, + else_clause, + .. + } => std::cmp::max( + if_clause + .as_ref() + .map(|n| n.subgraph_fetches_no_requires()) + .unwrap_or(0), + else_clause + .as_ref() + .map(|n| n.subgraph_fetches_no_requires()) + .unwrap_or(0), + ), + } + } + pub(crate) fn hash_subqueries(&mut self, schema: &apollo_compiler::Schema) { match self { PlanNode::Fetch(fetch_node) => { diff --git a/apollo-router/src/request.rs b/apollo-router/src/request.rs index 4988f7bed2..d895ea4955 100644 --- a/apollo-router/src/request.rs +++ b/apollo-router/src/request.rs @@ -177,8 +177,8 @@ impl Request { /// Convert Bytes into a GraphQL [`Request`]. /// - /// An error will be produced in the event that the query string parameters - /// cannot be turned into a valid GraphQL `Request`. + /// An error will be produced in the event that the bytes array cannot be + /// turned into a valid GraphQL `Request`. pub(crate) fn batch_from_bytes(bytes: &[u8]) -> Result, serde_json::Error> { let value: serde_json::Value = serde_json::from_slice(bytes).map_err(serde_json::Error::custom)?; diff --git a/apollo-router/src/response.rs b/apollo-router/src/response.rs index ad0da7f268..d590f49545 100644 --- a/apollo-router/src/response.rs +++ b/apollo-router/src/response.rs @@ -93,6 +93,36 @@ impl Response { self.errors.append(errors) } + /// Create a Vec of [`Response`] from the supplied [`Bytes`]. + /// + /// This will return a Vec of response/errors (identifying the faulty service) if the input is invalid. + pub(crate) fn array_from_bytes( + service_name: &str, + b: Bytes, + len: usize, + ) -> Result, FetchError> { + let mut result = Vec::with_capacity(len); + let value = + Value::from_bytes(b).map_err(|error| FetchError::SubrequestMalformedResponse { + service: service_name.to_string(), + reason: error.to_string(), + })?; + let array = + ensure_array!(value).map_err(|error| FetchError::SubrequestMalformedResponse { + service: service_name.to_string(), + reason: error.to_string(), + })?; + for value in array { + let object = + ensure_object!(value).map_err(|error| FetchError::SubrequestMalformedResponse { + service: service_name.to_string(), + reason: error.to_string(), + })?; + result.push(Response::from_object(service_name, object)?); + } + Ok(result) + } + /// Create a [`Response`] from the supplied [`Bytes`]. /// /// This will return an error (identifying the faulty service) if the input is invalid. @@ -102,12 +132,18 @@ impl Response { service: service_name.to_string(), reason: error.to_string(), })?; - let mut object = + let object = ensure_object!(value).map_err(|error| FetchError::SubrequestMalformedResponse { service: service_name.to_string(), reason: error.to_string(), })?; + Response::from_object(service_name, object) + } + pub(crate) fn from_object( + service_name: &str, + mut object: Object, + ) -> Result { let data = object.remove("data"); let errors = extract_key_value_from_object!(object, "errors", Value::Array(v) => v) .map_err(|err| FetchError::SubrequestMalformedResponse { diff --git a/apollo-router/src/services/router/service.rs b/apollo-router/src/services/router/service.rs index 894feca300..2d381fa41d 100644 --- a/apollo-router/src/services/router/service.rs +++ b/apollo-router/src/services/router/service.rs @@ -701,25 +701,23 @@ impl RouterService { #[derive(Clone, Debug, Default)] pub(crate) struct BatchDetails { pub(crate) index: usize, - // Request Details - pub(crate) request: Option, - pub(crate) body: Option, - pub(crate) context: Option, - pub(crate) service_name: Option, // Shared Request Details shared: Arc>, } impl fmt::Display for BatchDetails { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "index: {}", self.index)?; + write!(f, "index: {}, ", self.index)?; // Use try_lock. If the shared details are locked, we won't display them. // TODO: Maybe improve to handle the error...? let guard = self.shared.try_lock().ok_or(fmt::Error)?; - write!(f, "size: {}", guard.size)?; - write!(f, "expected: {:?}", guard.expected)?; + write!(f, "size: {}, ", guard.size)?; + write!(f, "expected: {:?}, ", guard.expected)?; write!(f, "seen: {:?}", guard.seen)?; - write!(f, "waiters: {:?}", guard.waiters) + for (service, details) in guard.waiters.iter() { + write!(f, ", service: {}, waiters: {}", service, details.len())?; + } + Ok(()) } } @@ -737,18 +735,31 @@ impl BatchDetails { } pub(crate) fn get_waiter( - &mut self, + &self, request: SubgraphRequest, body: graphql::Request, context: Context, service_name: &str, ) -> oneshot::Receiver> { tracing::info!("getting a waiter for {}", self.index); - self.request = Some(request); - self.body = Some(body); - self.context = Some(context); - self.service_name = Some(service_name.to_string()); - self.shared.lock().get_waiter(self.index) + self.shared + .lock() + .get_waiter(request, body, context, service_name.to_string()) + } + + pub(crate) fn get_waiters( + &self, + ) -> HashMap< + String, + Vec<( + SubgraphRequest, + graphql::Request, + Context, + oneshot::Sender>, + )>, + > { + let mut guard = self.shared.lock(); + std::mem::take(&mut guard.waiters) } pub(crate) fn increment_subgraph_seen(&self) { @@ -769,7 +780,15 @@ pub(crate) struct SharedBatchDetails { pub(crate) size: usize, pub(crate) expected: HashMap, pub(crate) seen: HashMap, - pub(crate) waiters: HashMap>>>, + pub(crate) waiters: HashMap< + String, + Vec<( + SubgraphRequest, + graphql::Request, + Context, + oneshot::Sender>, + )>, + >, } impl SharedBatchDetails { @@ -788,11 +807,14 @@ impl SharedBatchDetails { pub(crate) fn get_waiter( &mut self, - index: usize, + request: SubgraphRequest, + body: graphql::Request, + context: Context, + service: String, ) -> oneshot::Receiver> { let (tx, rx) = oneshot::channel(); - let value = self.waiters.entry(index).or_default(); - value.push(tx); + let value = self.waiters.entry(service).or_default(); + value.push((request, body, context, tx)); rx } } diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index 6f8948cf66..fe10c24076 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -6,7 +6,9 @@ use std::sync::atomic::Ordering::Relaxed; use std::sync::Arc; use std::task::Poll; +use bytes::BufMut; use bytes::Bytes; +use bytes::BytesMut; use futures::future::BoxFuture; use futures::SinkExt; use futures::StreamExt; @@ -17,6 +19,7 @@ use http::header::{self}; use http::response::Parts; use http::HeaderValue; use http::Request; +use http::StatusCode; use hyper::Body; use hyper_rustls::ConfigBuilderExt; use mediatype::names::APPLICATION; @@ -24,6 +27,7 @@ use mediatype::names::JSON; use mediatype::MediaType; use mime::APPLICATION_JSON; use rustls::RootCertStore; +use serde::de::Error; use serde::Serialize; use tokio_tungstenite::connect_async; use tokio_tungstenite::connect_async_tls_with_config; @@ -195,7 +199,6 @@ impl tower::Service for SubgraphService { } fn call(&mut self, mut request: SubgraphRequest) -> Self::Future { - tracing::info!("SUBGRAPH SERVICE CALLED"); let subscription_config = (request.operation_kind == OperationKind::Subscription) .then(|| self.subscription_config.clone()) .flatten(); @@ -367,7 +370,15 @@ impl tower::Service for SubgraphService { // with the same request body. let apq_enabled = arc_apq_enabled.as_ref(); if !apq_enabled.load(Relaxed) { - return call_batched_http(request, body, context, client, &service_name).await; + return call_batched_http( + request, + body, + context, + client, + client_factory.clone(), + &service_name, + ) + .await; } // Else, if APQ is enabled, @@ -402,6 +413,7 @@ impl tower::Service for SubgraphService { apq_body.clone(), context.clone(), client_factory.create(&service_name), + client_factory.clone(), &service_name, ) .await?; @@ -414,11 +426,27 @@ impl tower::Service for SubgraphService { match get_apq_error(gql_response) { APQError::PersistedQueryNotSupported => { apq_enabled.store(false, Relaxed); - call_batched_http(request, body, context, client, &service_name).await + call_batched_http( + request, + body, + context, + client, + client_factory.clone(), + &service_name, + ) + .await } APQError::PersistedQueryNotFound => { apq_body.query = query; - call_batched_http(request, apq_body, context, client, &service_name).await + call_batched_http( + request, + apq_body, + context, + client, + client_factory.clone(), + &service_name, + ) + .await } _ => Ok(response), } @@ -617,62 +645,288 @@ async fn call_websocket( )) } -/* -use tokio::sync::mpsc; -use tokio::sync::broadcast; -use tokio::sync::oneshot; - let (tx, mut rx) = mpsc::channel::(10); - let (b_tx, _b_rx) = broadcast::channel::<(usize, String)>(10); - - // Build up our batch co-ordinator for later use in subgraph processing - tokio::task::spawn(async move { - while let Some(body) = rx.recv().await { - todo!(); - } - // At this point we have our accumulated stuff, we need to figure out a way to send - // this to clients. - todo!() - }); -*/ - async fn call_batched_http( request: SubgraphRequest, body: graphql::Request, context: Context, client: crate::services::http::BoxService, + client_factory: crate::services::http::HttpClientServiceFactory, service_name: &str, ) -> Result { + let mut do_fetch_count = 0; // We'd like to park a task here, but we can't park it whilst we have the context extensions // lock held. That would be very bad... // So, we set an optional listener and wait to hear back from the batch processor let mut batch_responder: Option< tokio::sync::oneshot::Receiver>, > = None; + let mut waiters_opt = None; if let Some(batching) = context.extensions().lock().get_mut::() { - tracing::info!("in subgraph we have batching: {batching:?}, service: {service_name}"); - { - batching.increment_subgraph_seen(); - tracing::info!("ready to process batch?: {}", batching.ready()); - } + tracing::info!("in subgraph we have batching: {batching}, service: {service_name}"); + batching.increment_subgraph_seen(); + tracing::info!("ready to process batch?: {}", batching.ready()); + batch_responder = + Some(batching.get_waiter(request.clone(), body.clone(), context.clone(), service_name)); if batching.ready() { //TODO: This is where we start processing our accumulated batch data - tracing::info!("Batch data: {batching:?}"); - todo!() - } else { - batch_responder = Some(batching.get_waiter( - request.clone(), - body.clone(), - context.clone(), - service_name, - )); + // Now we need to "batch up" our data and send it to our subgraphs + // We need our own batch aware version of call_http which only makes one call to each + // subgraph, but is able to decode the responses. I'll probably need to break call_http + // down into sub-functions. + tracing::info!("Batch data: {batching}"); + waiters_opt = Some(batching.get_waiters()); } } if let Some(receiver) = batch_responder { - println!("WE HAVE A WAITER"); - receiver.await; - todo!() + tracing::info!("WE HAVE A WAITER"); + match waiters_opt { + Some(waiters) => { + let mut total_count_to_send = waiters.values().fold(0, |acc, v| acc + v.len()); + for (service, requests) in waiters.into_iter() { + let mut txs = Vec::with_capacity(requests.len()); + let mut requests_it = requests.into_iter(); + let first = requests_it + .next() + .expect("we should have at least one request"); + let context = first.2; // XXX SHADOWING + txs.push(first.3); + let SubgraphRequest { + subgraph_request, .. + } = first.0; + let operation_name = subgraph_request + .body() + .operation_name + .clone() + .unwrap_or_default(); + + let (parts, _) = subgraph_request.into_parts(); + let body = serde_json::to_string(&first.1) + .expect("JSON serialization should not fail"); + let mut bytes = BytesMut::new(); + bytes.put_u8(b'['); + bytes.extend_from_slice(&hyper::body::to_bytes(body).await?); + for request in requests_it { + txs.push(request.3); + bytes.put(&b", "[..]); + let body = serde_json::to_string(&request.1) + .expect("JSON serialization should not fail"); + bytes.extend_from_slice(&hyper::body::to_bytes(body).await?); + } + bytes.put_u8(b']'); + let body_bytes = bytes.freeze(); + tracing::info!("ABOUT TO SUBMIT BATCH: {:?}", body_bytes); + let mut request = http::Request::from_parts(parts, Body::from(body_bytes)); + + request + .headers_mut() + .insert(CONTENT_TYPE, APPLICATION_JSON_HEADER_VALUE.clone()); + request + .headers_mut() + .append(ACCEPT, ACCEPT_GRAPHQL_JSON.clone()); + + let schema_uri = request.uri(); + let host = schema_uri.host().unwrap_or_default(); + let port = schema_uri.port_u16().unwrap_or_else(|| { + let scheme = schema_uri.scheme_str(); + if scheme == Some("https") { + 443 + } else if scheme == Some("http") { + 80 + } else { + 0 + } + }); + + let path = schema_uri.path(); + + let subgraph_req_span = tracing::info_span!("subgraph_request", + "otel.kind" = "CLIENT", + "net.peer.name" = %host, + "net.peer.port" = %port, + "http.route" = %path, + "http.url" = %schema_uri, + "net.transport" = "ip_tcp", + "apollo.subgraph.name" = %service_name, + "graphql.operation.name" = %operation_name, + ); + + // The graphql spec is lax about what strategy to use for processing responses: https://github.com/graphql/graphql-over-http/blob/main/spec/GraphQLOverHTTP.md#processing-the-response + // + // "If the response uses a non-200 status code and the media type of the response payload is application/json + // then the client MUST NOT rely on the body to be a well-formed GraphQL response since the source of the response + // may not be the server but instead some intermediary such as API gateways, proxies, firewalls, etc." + // + // The TLDR of this is that it's really asking us to do the best we can with whatever information we have with some modifications depending on content type. + // Our goal is to give the user the most relevant information possible in the response errors + // + // Rules: + // 1. If the content type of the response is not `application/json` or `application/graphql-response+json` then we won't try to parse. + // 2. If an HTTP status is not 2xx it will always be attached as a graphql error. + // 3. If the response type is `application/json` and status is not 2xx and the body the entire body will be output if the response is not valid graphql. + + let display_body = context.contains_key(LOGGING_DISPLAY_BODY); + + let client = client_factory.create(&service); + // Perform the actual fetch. If this fails then we didn't manage to make the call at all, so we can't do anything with it. + let (parts, content_type, body) = + do_fetch(client, &context, service_name, request, display_body) + .instrument(subgraph_req_span) + .await?; + do_fetch_count += 1; + + if display_body { + if let Some(Ok(b)) = &body { + tracing::info!( + response.body = %String::from_utf8_lossy(b), apollo.subgraph.name = %service_name, "Raw response body from subgraph {service_name:?} received" + ); + } + } + + tracing::info!( + "parts: {parts:?}, content_type: {content_type:?}, body: {body:?}" + ); + + let value = serde_json::from_slice(&body.unwrap().unwrap()) + .map_err(serde_json::Error::custom)?; + + tracing::info!("JSON VALUE FROM BODY IS: {value:?}"); + + let array = ensure_array!(value).map_err(|error| { + FetchError::SubrequestMalformedResponse { + service: service_name.to_string(), + reason: error.to_string(), + } + })?; + let mut graphql_responses = Vec::with_capacity(array.len()); + for value in array { + let object = ensure_object!(value).map_err(|error| { + FetchError::SubrequestMalformedResponse { + service: service_name.to_string(), + reason: error.to_string(), + } + })?; + let body = Some(serde_json::to_vec(&object)); + + let mut graphql_response = + match (content_type.clone(), body, parts.status.is_success()) { + ( + Ok(ContentType::ApplicationGraphqlResponseJson), + Some(Ok(body)), + _, + ) + | (Ok(ContentType::ApplicationJson), Some(Ok(body)), true) => { + // Application graphql json expects valid graphql response + // Application json expects valid graphql response if 2xx + tracing::debug_span!("parse_subgraph_response").in_scope(|| { + // Application graphql json expects valid graphql response + graphql::Response::from_bytes(service_name, body.into()) + .unwrap_or_else(|error| { + graphql::Response::builder() + .error(error.to_graphql_error(None)) + .build() + }) + }) + } + (Ok(ContentType::ApplicationJson), Some(Ok(body)), false) => { + // Application json does not expect a valid graphql response if not 2xx. + // If parse fails then attach the entire payload as an error + tracing::debug_span!("parse_subgraph_response").in_scope(|| { + // Application graphql json expects valid graphql response + let mut original_response = + String::from_utf8_lossy(&body).to_string(); + if original_response.is_empty() { + original_response = "".into() + } + graphql::Response::from_bytes(service_name, body.into()) + .unwrap_or_else(|_error| { + graphql::Response::builder() + .error( + FetchError::SubrequestMalformedResponse { + service: service_name.to_string(), + reason: original_response, + } + .to_graphql_error(None), + ) + .build() + }) + }) + } + (content_type, body, _) => { + // Something went wrong, compose a response with errors if they are present + let mut graphql_response = graphql::Response::builder().build(); + if let Err(err) = content_type { + graphql_response.errors.push(err.to_graphql_error(None)); + } + /* TODO: XXX NEED TO UNDERSTAND + if let Some(Err(err)) = body { + graphql_response.errors.push(err.to_graphql_error(None)); + } + */ + graphql_response + } + }; + + // Add an error for response codes that are not 2xx + if !parts.status.is_success() { + let status = parts.status; + graphql_response.errors.insert( + 0, + FetchError::SubrequestHttpError { + service: service_name.to_string(), + status_code: Some(status.as_u16()), + reason: format!( + "{}: {}", + status.as_str(), + status.canonical_reason().unwrap_or("Unknown") + ), + } + .to_graphql_error(None), + ) + } + graphql_responses.push(graphql_response); + } + + tracing::info!("we have a vec of graphql_responses: {graphql_responses:?}"); + // Reverse txs to get things back in the right order + txs.reverse(); + for graphql_response in graphql_responses { + total_count_to_send -= 1; + // Build an http Response + let mut resp = http::Response::builder() + .status(StatusCode::OK) + .body(graphql_response) + .expect("Response is serializable; qed"); + + // *response.headers_mut() = headers.unwrap_or_default(); + // let resp = http::Response::from_parts(parts, graphql_response); + tracing::info!("we have a resp: {resp:?}"); + let subgraph_response = + SubgraphResponse::new_from_response(resp, context.clone()); + if total_count_to_send > 0 { + let tx = txs + .pop() + .expect("should have the same number of txs as responses"); + tx.send(Ok(subgraph_response)); + } else { + tracing::info!( + "TO PROCESS THIS BATCH WE ISSUE: {do_fetch_count} fetches" + ); + return Ok(subgraph_response); + } + } + // Ok(SubgraphResponse::new_from_response(resp, context)) + } + // Instead of todo return a fake error for now + return Err(Box::new(FetchError::SubrequestMalformedResponse { + service: service_name.to_string(), + reason: "fake_error".to_string(), + })); + // todo!() + } + None => receiver.await?, + } } else { - println!("WE CALLED HTTP"); + tracing::info!("WE CALLED HTTP"); call_http(request, body, context, client, service_name).await } } @@ -834,6 +1088,7 @@ pub(crate) async fn call_http( Ok(SubgraphResponse::new_from_response(resp, context)) } +#[derive(Clone, Debug)] enum ContentType { ApplicationJson, ApplicationGraphqlResponseJson, diff --git a/apollo-router/src/services/supergraph/service.rs b/apollo-router/src/services/supergraph/service.rs index 0f0413f9e7..cde9ae18f8 100644 --- a/apollo-router/src/services/supergraph/service.rs +++ b/apollo-router/src/services/supergraph/service.rs @@ -623,7 +623,12 @@ async fn plan_query( if let Some(batching) = context.extensions().lock().get::() { if let Some(QueryPlannerContent::Plan { plan, .. }) = &qpr.content { - batching.set_subgraph_fetches(plan.root.subgraph_fetches()); + batching.set_subgraph_fetches(plan.root.subgraph_fetches_no_requires()); + tracing::info!("subgraph fetches: {}", plan.root.subgraph_fetches()); + tracing::info!( + "subgraph fetches (no requires): {}", + plan.root.subgraph_fetches_no_requires() + ); } } Ok(qpr) From 72a39cee77828f7551c713200bd59ec0f506322e Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Wed, 14 Feb 2024 14:03:54 +0000 Subject: [PATCH 03/60] second milestone: complex batches are now working with caveats Once we've processed up all the "no requires" fetches, we can no longer create batches from remaining fetches. I get the same hanging waiter problem that I encountered earlier in prototyping. For now, I've implemented a solution which creates one set of batches and then forces remaining queries to execute independently. Hopefully that will be enough to satisfy the requirements of the feature. --- apollo-router/src/services/router/service.rs | 29 +++--- .../src/services/subgraph_service.rs | 89 ++++++++++++------- .../src/services/supergraph/service.rs | 8 +- 3 files changed, 75 insertions(+), 51 deletions(-) diff --git a/apollo-router/src/services/router/service.rs b/apollo-router/src/services/router/service.rs index 2d381fa41d..f6e215b50e 100644 --- a/apollo-router/src/services/router/service.rs +++ b/apollo-router/src/services/router/service.rs @@ -734,6 +734,10 @@ impl BatchDetails { self.shared.lock().ready() } + pub(crate) fn finished(&self) -> bool { + self.shared.lock().finished() + } + pub(crate) fn get_waiter( &self, request: SubgraphRequest, @@ -759,6 +763,7 @@ impl BatchDetails { )>, > { let mut guard = self.shared.lock(); + guard.finished = true; std::mem::take(&mut guard.waiters) } @@ -789,6 +794,7 @@ pub(crate) struct SharedBatchDetails { oneshot::Sender>, )>, >, + finished: bool, } impl SharedBatchDetails { @@ -798,14 +804,19 @@ impl SharedBatchDetails { expected: HashMap::new(), seen: HashMap::new(), waiters: HashMap::new(), + finished: false, } } - pub(crate) fn ready(&self) -> bool { + fn ready(&self) -> bool { self.expected.len() == self.size && self.expected == self.seen } - pub(crate) fn get_waiter( + fn finished(&self) -> bool { + self.finished + } + + fn get_waiter( &mut self, request: SubgraphRequest, body: graphql::Request, @@ -819,20 +830,6 @@ impl SharedBatchDetails { } } -/* - let (tx, mut rx) = mpsc::channel::(10); - let (b_tx, _b_rx) = broadcast::channel::<(usize, String)>(10); - - // Build up our batch co-ordinator for later use in subgraph processing - tokio::task::spawn(async move { - while let Some(body) = rx.recv().await { - todo!(); - } - // At this point we have our accumulated stuff, we need to figure out a way to send - // this to clients. - todo!() - }); -*/ struct TranslateError<'a> { status: StatusCode, error: &'a str, diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index fe10c24076..87228fa7f7 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -662,26 +662,30 @@ async fn call_batched_http( > = None; let mut waiters_opt = None; if let Some(batching) = context.extensions().lock().get_mut::() { - tracing::info!("in subgraph we have batching: {batching}, service: {service_name}"); - batching.increment_subgraph_seen(); - tracing::info!("ready to process batch?: {}", batching.ready()); - batch_responder = - Some(batching.get_waiter(request.clone(), body.clone(), context.clone(), service_name)); - if batching.ready() { - //TODO: This is where we start processing our accumulated batch data - // Now we need to "batch up" our data and send it to our subgraphs - // We need our own batch aware version of call_http which only makes one call to each - // subgraph, but is able to decode the responses. I'll probably need to break call_http - // down into sub-functions. - tracing::info!("Batch data: {batching}"); - waiters_opt = Some(batching.get_waiters()); + if !batching.finished() { + tracing::info!("in subgraph we have batching: {batching}, service: {service_name}"); + batching.increment_subgraph_seen(); + tracing::info!("ready to process batch?: {}", batching.ready()); + batch_responder = Some(batching.get_waiter( + request.clone(), + body.clone(), + context.clone(), + service_name, + )); + if batching.ready() { + //TODO: This is where we start processing our accumulated batch data + // Now we need to "batch up" our data and send it to our subgraphs + // We need our own batch aware version of call_http which only makes one call to each + // subgraph, but is able to decode the responses. I'll probably need to break call_http + // down into sub-functions. + tracing::info!("Batch data: {batching}"); + waiters_opt = Some(batching.get_waiters()); + } } } if let Some(receiver) = batch_responder { - tracing::info!("WE HAVE A WAITER"); match waiters_opt { Some(waiters) => { - let mut total_count_to_send = waiters.values().fold(0, |acc, v| acc + v.len()); for (service, requests) in waiters.into_iter() { let mut txs = Vec::with_capacity(requests.len()); let mut requests_it = requests.into_iter(); @@ -714,7 +718,7 @@ async fn call_batched_http( } bytes.put_u8(b']'); let body_bytes = bytes.freeze(); - tracing::info!("ABOUT TO SUBMIT BATCH: {:?}", body_bytes); + tracing::info!("ABOUT TO CREATE BATCH: {:?}", body_bytes); let mut request = http::Request::from_parts(parts, Body::from(body_bytes)); request @@ -890,9 +894,8 @@ async fn call_batched_http( // Reverse txs to get things back in the right order txs.reverse(); for graphql_response in graphql_responses { - total_count_to_send -= 1; // Build an http Response - let mut resp = http::Response::builder() + let resp = http::Response::builder() .status(StatusCode::OK) .body(graphql_response) .expect("Response is serializable; qed"); @@ -902,28 +905,54 @@ async fn call_batched_http( tracing::info!("we have a resp: {resp:?}"); let subgraph_response = SubgraphResponse::new_from_response(resp, context.clone()); - if total_count_to_send > 0 { - let tx = txs - .pop() - .expect("should have the same number of txs as responses"); - tx.send(Ok(subgraph_response)); - } else { - tracing::info!( - "TO PROCESS THIS BATCH WE ISSUE: {do_fetch_count} fetches" - ); - return Ok(subgraph_response); + match txs.pop() { + Some(tx) => { + tx.send(Ok(subgraph_response)).map_err(|_error| { + FetchError::SubrequestMalformedResponse { + service: service_name.to_string(), + reason: "tx send failed".to_string(), + } + })?; + } + None => { + tracing::info!( + "TO PROCESS THIS BATCH WE ISSUE: {do_fetch_count} fetches" + ); + return Ok(subgraph_response); + } } } - // Ok(SubgraphResponse::new_from_response(resp, context)) } + /* // Instead of todo return a fake error for now return Err(Box::new(FetchError::SubrequestMalformedResponse { service: service_name.to_string(), reason: "fake_error".to_string(), })); + */ // todo!() } - None => receiver.await?, + None => { + tracing::info!("WE HAVE A NONE WAITER"); + /* + tracing::info!("WE HAVE A WAITER"); + // receiver.await? + match receiver.await { + Ok(v) => v, + Err(err) => { + panic!("A RECEIVER FAILED: {err}"); + } + } + */ + } + } + tracing::info!("WE HAVE A WAITER"); + // receiver.await? + match receiver.await { + Ok(v) => v, + Err(err) => { + panic!("A RECEIVER FAILED: {err}"); + } } } else { tracing::info!("WE CALLED HTTP"); diff --git a/apollo-router/src/services/supergraph/service.rs b/apollo-router/src/services/supergraph/service.rs index cde9ae18f8..f47e57c62c 100644 --- a/apollo-router/src/services/supergraph/service.rs +++ b/apollo-router/src/services/supergraph/service.rs @@ -623,12 +623,10 @@ async fn plan_query( if let Some(batching) = context.extensions().lock().get::() { if let Some(QueryPlannerContent::Plan { plan, .. }) = &qpr.content { - batching.set_subgraph_fetches(plan.root.subgraph_fetches_no_requires()); + let no_requires_fetches = plan.root.subgraph_fetches_no_requires(); + batching.set_subgraph_fetches(no_requires_fetches); tracing::info!("subgraph fetches: {}", plan.root.subgraph_fetches()); - tracing::info!( - "subgraph fetches (no requires): {}", - plan.root.subgraph_fetches_no_requires() - ); + tracing::info!("subgraph fetches (no requires): {}", no_requires_fetches); } } Ok(qpr) From 49d2bfae594bcb1f157798d10df086e01d8e11cd Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Wed, 14 Feb 2024 16:14:39 +0000 Subject: [PATCH 04/60] Remove some of the code duplication I don't need two separate functions for counting fetches. Just provide a parameter. --- apollo-router/src/query_planner/execution.rs | 2 +- apollo-router/src/query_planner/plan.rs | 69 ++++--------------- .../src/services/supergraph/service.rs | 3 +- 3 files changed, 17 insertions(+), 57 deletions(-) diff --git a/apollo-router/src/query_planner/execution.rs b/apollo-router/src/query_planner/execution.rs index d1793dafd7..539eb2c808 100644 --- a/apollo-router/src/query_planner/execution.rs +++ b/apollo-router/src/query_planner/execution.rs @@ -88,7 +88,7 @@ impl QueryPlan { } pub fn subgraph_fetches(&self) -> usize { - self.root.subgraph_fetches() + self.root.subgraph_fetches(true) } } diff --git a/apollo-router/src/query_planner/plan.rs b/apollo-router/src/query_planner/plan.rs index f4a229bb9c..04d3f380d0 100644 --- a/apollo-router/src/query_planner/plan.rs +++ b/apollo-router/src/query_planner/plan.rs @@ -188,75 +188,36 @@ impl PlanNode { } } - pub(crate) fn subgraph_fetches(&self) -> usize { + pub(crate) fn subgraph_fetches(&self, include_requires: bool) -> usize { match self { - PlanNode::Sequence { nodes } => nodes.iter().map(|n| n.subgraph_fetches()).sum(), - PlanNode::Parallel { nodes } => nodes.iter().map(|n| n.subgraph_fetches()).sum(), + PlanNode::Sequence { nodes } => nodes + .iter() + .map(|n| n.subgraph_fetches(include_requires)) + .sum(), + PlanNode::Parallel { nodes } => nodes + .iter() + .map(|n| n.subgraph_fetches(include_requires)) + .sum(), PlanNode::Fetch(_) => 1, - PlanNode::Flatten(node) => node.node.subgraph_fetches(), - PlanNode::Defer { primary, deferred } => { - primary.node.as_ref().map_or(0, |n| n.subgraph_fetches()) - + deferred - .iter() - .map(|n| n.node.as_ref().map_or(0, |n| n.subgraph_fetches())) - .sum::() - } - // A `SubscriptionNode` makes a request to a subgraph, so counting it as 1 - PlanNode::Subscription { rest, .. } => { - rest.as_ref().map_or(0, |n| n.subgraph_fetches()) + 1 - } - // Compute the highest possible value for condition nodes - PlanNode::Condition { - if_clause, - else_clause, - .. - } => std::cmp::max( - if_clause - .as_ref() - .map(|n| n.subgraph_fetches()) - .unwrap_or(0), - else_clause - .as_ref() - .map(|n| n.subgraph_fetches()) - .unwrap_or(0), - ), - } - } - - pub(crate) fn subgraph_fetches_no_requires(&self) -> usize { - match self { - PlanNode::Sequence { nodes } => { - nodes.iter().map(|n| n.subgraph_fetches_no_requires()).sum() - } - PlanNode::Parallel { nodes } => { - nodes.iter().map(|n| n.subgraph_fetches_no_requires()).sum() - } - PlanNode::Fetch(node) => { - if node.requires.is_empty() { - 1 - } else { - 0 - } - } - PlanNode::Flatten(node) => node.node.subgraph_fetches_no_requires(), + PlanNode::Flatten(node) => node.node.subgraph_fetches(include_requires), PlanNode::Defer { primary, deferred } => { primary .node .as_ref() - .map_or(0, |n| n.subgraph_fetches_no_requires()) + .map_or(0, |n| n.subgraph_fetches(include_requires)) + deferred .iter() .map(|n| { n.node .as_ref() - .map_or(0, |n| n.subgraph_fetches_no_requires()) + .map_or(0, |n| n.subgraph_fetches(include_requires)) }) .sum::() } // A `SubscriptionNode` makes a request to a subgraph, so counting it as 1 PlanNode::Subscription { rest, .. } => { rest.as_ref() - .map_or(0, |n| n.subgraph_fetches_no_requires()) + .map_or(0, |n| n.subgraph_fetches(include_requires)) + 1 } // Compute the highest possible value for condition nodes @@ -267,11 +228,11 @@ impl PlanNode { } => std::cmp::max( if_clause .as_ref() - .map(|n| n.subgraph_fetches_no_requires()) + .map(|n| n.subgraph_fetches(include_requires)) .unwrap_or(0), else_clause .as_ref() - .map(|n| n.subgraph_fetches_no_requires()) + .map(|n| n.subgraph_fetches(include_requires)) .unwrap_or(0), ), } diff --git a/apollo-router/src/services/supergraph/service.rs b/apollo-router/src/services/supergraph/service.rs index f47e57c62c..89a731a333 100644 --- a/apollo-router/src/services/supergraph/service.rs +++ b/apollo-router/src/services/supergraph/service.rs @@ -623,9 +623,8 @@ async fn plan_query( if let Some(batching) = context.extensions().lock().get::() { if let Some(QueryPlannerContent::Plan { plan, .. }) = &qpr.content { - let no_requires_fetches = plan.root.subgraph_fetches_no_requires(); + let no_requires_fetches = plan.root.subgraph_fetches(false); batching.set_subgraph_fetches(no_requires_fetches); - tracing::info!("subgraph fetches: {}", plan.root.subgraph_fetches()); tracing::info!("subgraph fetches (no requires): {}", no_requires_fetches); } } From e76c0df2c1d14e124d7751d239b1b104affe45c5 Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Wed, 14 Feb 2024 16:25:22 +0000 Subject: [PATCH 05/60] fix my code reduction I forgot that I had to fixup the subgraph_fetches function with my check to `include_requires`. --- apollo-router/src/query_planner/plan.rs | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/apollo-router/src/query_planner/plan.rs b/apollo-router/src/query_planner/plan.rs index 04d3f380d0..e6350345b6 100644 --- a/apollo-router/src/query_planner/plan.rs +++ b/apollo-router/src/query_planner/plan.rs @@ -188,6 +188,8 @@ impl PlanNode { } } + /// Count the number of fetches + /// pub(crate) fn subgraph_fetches(&self, include_requires: bool) -> usize { match self { PlanNode::Sequence { nodes } => nodes @@ -198,7 +200,13 @@ impl PlanNode { .iter() .map(|n| n.subgraph_fetches(include_requires)) .sum(), - PlanNode::Fetch(_) => 1, + PlanNode::Fetch(node) => { + if include_requires || node.requires.is_empty() { + 1 + } else { + 0 + } + } PlanNode::Flatten(node) => node.node.subgraph_fetches(include_requires), PlanNode::Defer { primary, deferred } => { primary From d5e92d3a989634901f5f2568026a5c67ad3273b6 Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Thu, 15 Feb 2024 11:49:47 +0000 Subject: [PATCH 06/60] Add some draft documentation for early evaluation by docs Add some content to the existing pages for early review by docs. --- .../executing-operations/query-batching.mdx | 116 +++++++++++++++++- 1 file changed, 113 insertions(+), 3 deletions(-) diff --git a/docs/source/executing-operations/query-batching.mdx b/docs/source/executing-operations/query-batching.mdx index 2fdebf80a5..e4c59bf129 100644 --- a/docs/source/executing-operations/query-batching.mdx +++ b/docs/source/executing-operations/query-batching.mdx @@ -11,18 +11,22 @@ Learn about query batching and how to configure the Apollo Router to receive que Modern applications often require several requests to render a single page. This is usually the result of a component-based architecture where individual micro-frontends (MFE) make requests separately to fetch data relevant to them. Not only does this cause a performance overhead—different components may be requesting the same data—it can also cause a consistency issue. To combat this, MFE-based UIs batch multiple client operations, issued close together, into a single HTTP request. This is supported in Apollo Client and Apollo Server. -The Apollo Router supports client query batching. If you’re using Apollo Client, you can leverage the built-in support for batching to reduce the number of individual operations sent to the router. +The Apollo Router supports client and subgraph query batching. + +If you’re using Apollo Client, you can leverage the built-in support for batching to reduce the number of individual operations sent to the router. Once configured, Apollo Client automatically combines multiple operations into a single HTTP request. The number of operations within a batch is client-configurable, including the maximum number in a batch and the maximum duration to wait for operations to accumulate before sending the batch. The Apollo Router must be configured to receive query batches, otherwise it rejects them. When processing a batch, the router deserializes and processes each operation of a batch independently, and it responds to the client only after all operations of the batch have been completed. Each operation executes concurrently with respect to other operations in the batch. -## Configure query batching +## Configure Client query batching Both the Apollo Router and client need to be configured to support query batching. ### Configure router +#### Client Query Batching + By default, receiving client query batches is _not_ enabled in the Apollo Router. To enable query batching, set the following fields in your `router.yaml` configuration file: @@ -38,6 +42,109 @@ experimental_batching: | `enabled` | Flag to enable reception of client query batches | boolean | `false` | | `mode` | Supported client batching mode | `batch_http_link`: the client uses Apollo Link and its [`BatchHttpLink`](/react/api/link/apollo-link-batch-http) link. | No Default | +#### Subgraph Query Batching + +If Client query batching is enabled, and the router subgraphs [support query batching](https://www.apollographql.com/docs/apollo-server/api/apollo-server#allowbatchedhttprequests), then subgraph query batching may be enabled by setting the following fields in your `router.yaml` configuration file: + +```yaml title="router.yaml" +experimental_batching: + enabled: true + mode: batch_http_link + subgraph: + all: + enabled: true + subgraphs: + subgraph_1: + enabled: true + subgraph_2: + enabled: true +``` + + +There are limitations on the ability of the router to preserve batches from the client request into the subgraph requests. In particular, certain forms of queries will require data to be present before they are processed. Bearing this constraint in mind, the router will only be able to generate batches from queries which are processed which don't contain such constraints. + +##### Example 1 Federated Subgraph Batching + +```json title="federated-batch.json" +[ + {"query":"query MeQuery1 {\n me {\n id\n }\n}"}, + {"query":"query MeQuery2 {\n me {\n reviews {\n body\n }\n }\n}"}, + {"query":"query MeQuery3 {\n topProducts {\n upc\n reviews {\n author {\n name\n }\n }\n }\n me {\n name\n }\n}"}, + {"query":"query MeQuery4 {\n me {\n name\n }\n}"}, + {"query":"query MeQuery5 {\n me {\n id\n }\n}"} +] +``` + +For that input and with the knowledge that our federated graph has 3 subgraphs, accounts, products and reviews, the router will generate a set of queries which look like this: +``` +"query MeQuery1__accounts__0{me{id}}", +"query MeQuery2__accounts__0{me{__typename id}}", +"query MeQuery3__products__0{topProducts{__typename upc}}", +"query MeQuery3__accounts__3{me{name}}", +"query MeQuery4__accounts__0{me{name}}", +"query MeQuery5__accounts__0{me{id}}", +"query MeQuery2__reviews__1($representations:[_Any!]!){_entities(representations:$representations){...on User{reviews{body}}}}", +"query MeQuery3__reviews__1($representations:[_Any!]!){_entities(representations:$representations){...on Product{reviews{author{__typename id}}}}}", +"query MeQuery3__accounts__2($representations:[_Any!]!){_entities(representations:$representations){...on User{name}}}", +``` +The first 6 queries may be batched and must be fetched before the final 3 queries can be executed. So, in this case, without subgraph batching you would make 9 fetches in total across the 3 subgraphs. With batching, that would reduce to 5 fetches. + +| Subgraph | Fetch Count (without)| Fetch Count (with) | +|----------|----------------------|--------------------| +| accounts | 6 | 2 | +| products | 1 | 1 | +| reviews | 2 | 2 | + +##### Example 2 Simple Subgraph Batching + +For queries which don't have required fetch constraints, subgraph batching is extremely efficient: + +```json title="simple-batch.json" +[ + {"query":"query MeQuery1 {\n me {\n id\n }\n}"} + {"query":"query MeQuery2 {\n me {\n name\n }\n}"}, + {"query":"query MeQuery3 {\n me {\n id\n }\n}"} + {"query":"query MeQuery4 {\n me {\n name\n }\n}"}, + {"query":"query MeQuery5 {\n me {\n id\n }\n}"} + {"query":"query MeQuery6 {\n me {\n name\n }\n}"}, + {"query":"query MeQuery7 {\n me {\n id\n }\n}"} + {"query":"query MeQuery8 {\n me {\n name\n }\n}"}, + {"query":"query MeQuery9 {\n me {\n id\n }\n}"} + {"query":"query MeQuery10 {\n me {\n name\n }\n}"}, + {"query":"query MeQuery11 {\n me {\n id\n }\n}"} + {"query":"query MeQuery12 {\n me {\n name\n }\n}"}, + {"query":"query MeQuery13 {\n me {\n id\n }\n}"} + {"query":"query MeQuery14 {\n me {\n name\n }\n}"}, + {"query":"query MeQuery15 {\n me {\n id\n }\n}"} +] +``` + +For that input and with the knowledge that our federated graph has 3 subgraphs, accounts, products and reviews, the router will generate a set of queries which look like this: +``` +"query MeQuery1__accounts__0{me{id}}", +"query MeQuery2__accounts__0{me{name}}", +"query MeQuery3__accounts__0{me{id}}", +"query MeQuery4__accounts__0{me{name}}", +"query MeQuery5__accounts__0{me{id}}", +"query MeQuery6__accounts__0{me{name}}", +"query MeQuery7__accounts__0{me{id}}", +"query MeQuery8__accounts__0{me{name}}", +"query MeQuery9__accounts__0{me{id}}", +"query MeQuery10__accounts__0{me{name}}", +"query MeQuery11__accounts__0{me{id}}", +"query MeQuery12__accounts__0{me{name}}", +"query MeQuery13__accounts__0{me{id}}", +"query MeQuery14__accounts__0{me{name}}", +"query MeQuery15__accounts__0{me{id}}", +``` +All of the queries may be batched. So, in this case, 15 fetches would be reduced to 1. + +| Subgraph | Fetch Count (without)| Fetch Count (with) | +|----------|----------------------|--------------------| +| accounts | 15 | 1 | + + + ### Configure client To enable batching in an Apollo client, configure `BatchHttpLink`. For details on implementing `BatchHttpLink`, see [batching operations](/react/api/link/apollo-link-batch-http/). @@ -69,11 +176,12 @@ Metrics in the Apollo Router for query batching: mode +[subgraph] -Counter for the number of received batches. +Counter for the number of received (from client) or dispatched (to subgraph) batches. @@ -87,6 +195,7 @@ Counter for the number of received batches. mode +[subgraph] @@ -98,6 +207,7 @@ Histogram for the size of received batches. +The `subgraph` attribute is optional and if not present indicates that the metric is identifying batches received from clients. If a `subgraph` attribute is present, then the metric will be identifying batches sent to a particular subgraph. ## Query batch formats From 553cb4203c5fb0f4fc25553428230335da90b5cc Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Thu, 15 Feb 2024 12:52:40 +0000 Subject: [PATCH 07/60] Draft changeset for the feature Outlines new capabilities and adds illustrative configuration. --- .../feat_garypen_2002_subgraph_batching.md | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) create mode 100644 .changesets/feat_garypen_2002_subgraph_batching.md diff --git a/.changesets/feat_garypen_2002_subgraph_batching.md b/.changesets/feat_garypen_2002_subgraph_batching.md new file mode 100644 index 0000000000..015a3c84e8 --- /dev/null +++ b/.changesets/feat_garypen_2002_subgraph_batching.md @@ -0,0 +1,27 @@ +### Subgraph support for query batching ([Issue #2002](https://github.com/apollographql/router/issues/2002)) + +This project is an extension of the existing work to support [client side batching in the router](https://github.com/apollographql/router/issues/126). The current implementation is experimental and is publicly [documented](https://www.apollographql.com/docs/router/executing-operations/query-batching/). + +Currently the concept of a batch is preserved until the end of the `RouterRequest` processing. At this point, we convert each batch request item into a separate `SupergraphRequest`. These are then planned and executed concurrently within the router and re-assembled into a batch when they complete. It's important to note that, with this implementation, the concept of a batch, from the perspective of an executing router, now disappears and each request is planned and executed separately. + +This extension will modify the router so that the concept of a batch is preserved, at least outwardly, so that multiple subgraph requests are "batched" (in exactly the same format as a client batch request) for onward transmission to subgraphs. The goal of this work is to provide an optimisation by reducing the number of round-trips to a subgraph from the router. + +Illustrative configuration. + +```yaml +batching: + enabled: true + mode: batch_http_link + subgraph: + all: + enabled: true + subgraphs: + subgraph_1: + enabled: true + subgraph_2: + enabled:true +```` + +As with other router subgraph configuration options, `all` and `subgraphs` are mutually exclusive. + +By [@garypen](https://github.com/garypen) in https://github.com/apollographql/router/pull/4661 \ No newline at end of file From d99648e4b58c37f6490331d7d3d92dcf0f799f8a Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Fri, 16 Feb 2024 10:46:08 +0000 Subject: [PATCH 08/60] create batching.rs and start to organise code more professionally Isolate the batching support into its own module and start to reduce code duplication. --- apollo-router/src/lib.rs | 1 + apollo-router/src/services/router/service.rs | 139 +----- .../src/services/subgraph_service.rs | 453 ++++++++---------- .../src/services/supergraph/service.rs | 2 +- 4 files changed, 215 insertions(+), 380 deletions(-) diff --git a/apollo-router/src/lib.rs b/apollo-router/src/lib.rs index cef6ac055f..7ab014b2e8 100644 --- a/apollo-router/src/lib.rs +++ b/apollo-router/src/lib.rs @@ -51,6 +51,7 @@ pub mod plugin; pub(crate) mod metrics; pub(crate) mod axum_factory; +mod batching; mod cache; mod configuration; mod context; diff --git a/apollo-router/src/services/router/service.rs b/apollo-router/src/services/router/service.rs index f6e215b50e..f7873344fc 100644 --- a/apollo-router/src/services/router/service.rs +++ b/apollo-router/src/services/router/service.rs @@ -1,7 +1,5 @@ //! Implements the router phase of the request lifecycle. -use std::collections::HashMap; -use std::fmt; use std::sync::Arc; use std::task::Poll; @@ -30,7 +28,6 @@ use mime::APPLICATION_JSON; use multimap::MultiMap; use parking_lot::Mutex; use router_bridge::planner::Planner; -use tokio::sync::oneshot; use tower::BoxError; use tower::Layer; use tower::ServiceBuilder; @@ -39,6 +36,8 @@ use tower_service::Service; use tracing::Instrument; use super::ClientRequestAccepts; +use crate::batching::BatchDetails; +use crate::batching::SharedBatchDetails; use crate::cache::DeduplicatingCache; use crate::configuration::Batching; use crate::configuration::BatchingMode; @@ -66,8 +65,6 @@ use crate::services::HasPlugins; use crate::services::HasSchema; use crate::services::RouterRequest; use crate::services::RouterResponse; -use crate::services::SubgraphRequest; -use crate::services::SubgraphResponse; use crate::services::SupergraphCreator; use crate::services::SupergraphRequest; use crate::services::SupergraphResponse; @@ -698,138 +695,6 @@ impl RouterService { } } -#[derive(Clone, Debug, Default)] -pub(crate) struct BatchDetails { - pub(crate) index: usize, - // Shared Request Details - shared: Arc>, -} - -impl fmt::Display for BatchDetails { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "index: {}, ", self.index)?; - // Use try_lock. If the shared details are locked, we won't display them. - // TODO: Maybe improve to handle the error...? - let guard = self.shared.try_lock().ok_or(fmt::Error)?; - write!(f, "size: {}, ", guard.size)?; - write!(f, "expected: {:?}, ", guard.expected)?; - write!(f, "seen: {:?}", guard.seen)?; - for (service, details) in guard.waiters.iter() { - write!(f, ", service: {}, waiters: {}", service, details.len())?; - } - Ok(()) - } -} - -impl BatchDetails { - fn new(index: usize, shared: Arc>) -> Self { - Self { - index, - shared, - ..Default::default() - } - } - - pub(crate) fn ready(&self) -> bool { - self.shared.lock().ready() - } - - pub(crate) fn finished(&self) -> bool { - self.shared.lock().finished() - } - - pub(crate) fn get_waiter( - &self, - request: SubgraphRequest, - body: graphql::Request, - context: Context, - service_name: &str, - ) -> oneshot::Receiver> { - tracing::info!("getting a waiter for {}", self.index); - self.shared - .lock() - .get_waiter(request, body, context, service_name.to_string()) - } - - pub(crate) fn get_waiters( - &self, - ) -> HashMap< - String, - Vec<( - SubgraphRequest, - graphql::Request, - Context, - oneshot::Sender>, - )>, - > { - let mut guard = self.shared.lock(); - guard.finished = true; - std::mem::take(&mut guard.waiters) - } - - pub(crate) fn increment_subgraph_seen(&self) { - let mut shared_guard = self.shared.lock(); - let value = shared_guard.seen.entry(self.index).or_default(); - *value += 1; - } - - pub(crate) fn set_subgraph_fetches(&self, fetches: usize) { - let mut shared_guard = self.shared.lock(); - let value = shared_guard.expected.entry(self.index).or_default(); - *value = fetches; - } -} - -#[derive(Debug, Default)] -pub(crate) struct SharedBatchDetails { - pub(crate) size: usize, - pub(crate) expected: HashMap, - pub(crate) seen: HashMap, - pub(crate) waiters: HashMap< - String, - Vec<( - SubgraphRequest, - graphql::Request, - Context, - oneshot::Sender>, - )>, - >, - finished: bool, -} - -impl SharedBatchDetails { - fn new(size: usize) -> Self { - Self { - size, - expected: HashMap::new(), - seen: HashMap::new(), - waiters: HashMap::new(), - finished: false, - } - } - - fn ready(&self) -> bool { - self.expected.len() == self.size && self.expected == self.seen - } - - fn finished(&self) -> bool { - self.finished - } - - fn get_waiter( - &mut self, - request: SubgraphRequest, - body: graphql::Request, - context: Context, - service: String, - ) -> oneshot::Receiver> { - let (tx, rx) = oneshot::channel(); - let value = self.waiters.entry(service).or_default(); - value.push((request, body, context, tx)); - rx - } -} - struct TranslateError<'a> { status: StatusCode, error: &'a str, diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index 87228fa7f7..f1d3f05785 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -43,6 +43,7 @@ use super::http::HttpClientServiceFactory; use super::http::HttpRequest; use super::layers::content_negotiation::GRAPHQL_JSON_RESPONSE_HEADER_VALUE; use super::Plugins; +use crate::batching::BatchDetails; use crate::configuration::TlsClientAuth; use crate::error::FetchError; use crate::graphql; @@ -61,7 +62,6 @@ use crate::protocols::websocket::convert_websocket_stream; use crate::protocols::websocket::GraphqlWebSocket; use crate::query_planner::OperationKind; use crate::services::layers::apq; -use crate::services::router::service::BatchDetails; use crate::services::SubgraphRequest; use crate::services::SubgraphResponse; use crate::Configuration; @@ -684,276 +684,245 @@ async fn call_batched_http( } } if let Some(receiver) = batch_responder { - match waiters_opt { - Some(waiters) => { - for (service, requests) in waiters.into_iter() { - let mut txs = Vec::with_capacity(requests.len()); - let mut requests_it = requests.into_iter(); - let first = requests_it - .next() - .expect("we should have at least one request"); - let context = first.2; // XXX SHADOWING - txs.push(first.3); - let SubgraphRequest { - subgraph_request, .. - } = first.0; - let operation_name = subgraph_request - .body() - .operation_name - .clone() - .unwrap_or_default(); - - let (parts, _) = subgraph_request.into_parts(); - let body = serde_json::to_string(&first.1) + if let Some(waiters) = waiters_opt { + for (service, requests) in waiters.into_iter() { + let mut txs = Vec::with_capacity(requests.len()); + let mut requests_it = requests.into_iter(); + let first = requests_it + .next() + .expect("we should have at least one request"); + let context = first.2; // XXX SHADOWING + txs.push(first.3); + let SubgraphRequest { + subgraph_request, .. + } = first.0; + let operation_name = subgraph_request + .body() + .operation_name + .clone() + .unwrap_or_default(); + + let (parts, _) = subgraph_request.into_parts(); + let body = + serde_json::to_string(&first.1).expect("JSON serialization should not fail"); + let mut bytes = BytesMut::new(); + bytes.put_u8(b'['); + bytes.extend_from_slice(&hyper::body::to_bytes(body).await?); + for request in requests_it { + txs.push(request.3); + bytes.put(&b", "[..]); + let body = serde_json::to_string(&request.1) .expect("JSON serialization should not fail"); - let mut bytes = BytesMut::new(); - bytes.put_u8(b'['); bytes.extend_from_slice(&hyper::body::to_bytes(body).await?); - for request in requests_it { - txs.push(request.3); - bytes.put(&b", "[..]); - let body = serde_json::to_string(&request.1) - .expect("JSON serialization should not fail"); - bytes.extend_from_slice(&hyper::body::to_bytes(body).await?); + } + bytes.put_u8(b']'); + let body_bytes = bytes.freeze(); + tracing::info!("ABOUT TO CREATE BATCH: {:?}", body_bytes); + let mut request = http::Request::from_parts(parts, Body::from(body_bytes)); + + request + .headers_mut() + .insert(CONTENT_TYPE, APPLICATION_JSON_HEADER_VALUE.clone()); + request + .headers_mut() + .append(ACCEPT, ACCEPT_GRAPHQL_JSON.clone()); + + let schema_uri = request.uri(); + let host = schema_uri.host().unwrap_or_default(); + let port = schema_uri.port_u16().unwrap_or_else(|| { + let scheme = schema_uri.scheme_str(); + if scheme == Some("https") { + 443 + } else if scheme == Some("http") { + 80 + } else { + 0 } - bytes.put_u8(b']'); - let body_bytes = bytes.freeze(); - tracing::info!("ABOUT TO CREATE BATCH: {:?}", body_bytes); - let mut request = http::Request::from_parts(parts, Body::from(body_bytes)); - - request - .headers_mut() - .insert(CONTENT_TYPE, APPLICATION_JSON_HEADER_VALUE.clone()); - request - .headers_mut() - .append(ACCEPT, ACCEPT_GRAPHQL_JSON.clone()); - - let schema_uri = request.uri(); - let host = schema_uri.host().unwrap_or_default(); - let port = schema_uri.port_u16().unwrap_or_else(|| { - let scheme = schema_uri.scheme_str(); - if scheme == Some("https") { - 443 - } else if scheme == Some("http") { - 80 - } else { - 0 - } - }); - - let path = schema_uri.path(); - - let subgraph_req_span = tracing::info_span!("subgraph_request", - "otel.kind" = "CLIENT", - "net.peer.name" = %host, - "net.peer.port" = %port, - "http.route" = %path, - "http.url" = %schema_uri, - "net.transport" = "ip_tcp", - "apollo.subgraph.name" = %service_name, - "graphql.operation.name" = %operation_name, - ); + }); - // The graphql spec is lax about what strategy to use for processing responses: https://github.com/graphql/graphql-over-http/blob/main/spec/GraphQLOverHTTP.md#processing-the-response - // - // "If the response uses a non-200 status code and the media type of the response payload is application/json - // then the client MUST NOT rely on the body to be a well-formed GraphQL response since the source of the response - // may not be the server but instead some intermediary such as API gateways, proxies, firewalls, etc." - // - // The TLDR of this is that it's really asking us to do the best we can with whatever information we have with some modifications depending on content type. - // Our goal is to give the user the most relevant information possible in the response errors - // - // Rules: - // 1. If the content type of the response is not `application/json` or `application/graphql-response+json` then we won't try to parse. - // 2. If an HTTP status is not 2xx it will always be attached as a graphql error. - // 3. If the response type is `application/json` and status is not 2xx and the body the entire body will be output if the response is not valid graphql. - - let display_body = context.contains_key(LOGGING_DISPLAY_BODY); - - let client = client_factory.create(&service); - // Perform the actual fetch. If this fails then we didn't manage to make the call at all, so we can't do anything with it. - let (parts, content_type, body) = - do_fetch(client, &context, service_name, request, display_body) - .instrument(subgraph_req_span) - .await?; - do_fetch_count += 1; + let path = schema_uri.path(); + + let subgraph_req_span = tracing::info_span!("subgraph_request", + "otel.kind" = "CLIENT", + "net.peer.name" = %host, + "net.peer.port" = %port, + "http.route" = %path, + "http.url" = %schema_uri, + "net.transport" = "ip_tcp", + "apollo.subgraph.name" = %service_name, + "graphql.operation.name" = %operation_name, + ); - if display_body { - if let Some(Ok(b)) = &body { - tracing::info!( - response.body = %String::from_utf8_lossy(b), apollo.subgraph.name = %service_name, "Raw response body from subgraph {service_name:?} received" - ); - } + // The graphql spec is lax about what strategy to use for processing responses: https://github.com/graphql/graphql-over-http/blob/main/spec/GraphQLOverHTTP.md#processing-the-response + // + // "If the response uses a non-200 status code and the media type of the response payload is application/json + // then the client MUST NOT rely on the body to be a well-formed GraphQL response since the source of the response + // may not be the server but instead some intermediary such as API gateways, proxies, firewalls, etc." + // + // The TLDR of this is that it's really asking us to do the best we can with whatever information we have with some modifications depending on content type. + // Our goal is to give the user the most relevant information possible in the response errors + // + // Rules: + // 1. If the content type of the response is not `application/json` or `application/graphql-response+json` then we won't try to parse. + // 2. If an HTTP status is not 2xx it will always be attached as a graphql error. + // 3. If the response type is `application/json` and status is not 2xx and the body the entire body will be output if the response is not valid graphql. + + let display_body = context.contains_key(LOGGING_DISPLAY_BODY); + + let client = client_factory.create(&service); + // Perform the actual fetch. If this fails then we didn't manage to make the call at all, so we can't do anything with it. + let (parts, content_type, body) = + do_fetch(client, &context, service_name, request, display_body) + .instrument(subgraph_req_span) + .await?; + do_fetch_count += 1; + + if display_body { + if let Some(Ok(b)) = &body { + tracing::info!( + response.body = %String::from_utf8_lossy(b), apollo.subgraph.name = %service_name, "Raw response body from subgraph {service_name:?} received" + ); } + } - tracing::info!( - "parts: {parts:?}, content_type: {content_type:?}, body: {body:?}" - ); + tracing::info!("parts: {parts:?}, content_type: {content_type:?}, body: {body:?}"); - let value = serde_json::from_slice(&body.unwrap().unwrap()) - .map_err(serde_json::Error::custom)?; + let value = serde_json::from_slice(&body.unwrap().unwrap()) + .map_err(serde_json::Error::custom)?; - tracing::info!("JSON VALUE FROM BODY IS: {value:?}"); + tracing::info!("JSON VALUE FROM BODY IS: {value:?}"); - let array = ensure_array!(value).map_err(|error| { + let array = ensure_array!(value).map_err(|error| { + FetchError::SubrequestMalformedResponse { + service: service_name.to_string(), + reason: error.to_string(), + } + })?; + let mut graphql_responses = Vec::with_capacity(array.len()); + for value in array { + let object = ensure_object!(value).map_err(|error| { FetchError::SubrequestMalformedResponse { service: service_name.to_string(), reason: error.to_string(), } })?; - let mut graphql_responses = Vec::with_capacity(array.len()); - for value in array { - let object = ensure_object!(value).map_err(|error| { - FetchError::SubrequestMalformedResponse { - service: service_name.to_string(), - reason: error.to_string(), + let body = Some(serde_json::to_vec(&object)); + + let mut graphql_response = + match (content_type.clone(), body, parts.status.is_success()) { + ( + Ok(ContentType::ApplicationGraphqlResponseJson), + Some(Ok(body)), + _, + ) + | (Ok(ContentType::ApplicationJson), Some(Ok(body)), true) => { + // Application graphql json expects valid graphql response + // Application json expects valid graphql response if 2xx + tracing::debug_span!("parse_subgraph_response").in_scope(|| { + // Application graphql json expects valid graphql response + graphql::Response::from_bytes(service_name, body.into()) + .unwrap_or_else(|error| { + graphql::Response::builder() + .error(error.to_graphql_error(None)) + .build() + }) + }) } - })?; - let body = Some(serde_json::to_vec(&object)); - - let mut graphql_response = - match (content_type.clone(), body, parts.status.is_success()) { - ( - Ok(ContentType::ApplicationGraphqlResponseJson), - Some(Ok(body)), - _, - ) - | (Ok(ContentType::ApplicationJson), Some(Ok(body)), true) => { + (Ok(ContentType::ApplicationJson), Some(Ok(body)), false) => { + // Application json does not expect a valid graphql response if not 2xx. + // If parse fails then attach the entire payload as an error + tracing::debug_span!("parse_subgraph_response").in_scope(|| { // Application graphql json expects valid graphql response - // Application json expects valid graphql response if 2xx - tracing::debug_span!("parse_subgraph_response").in_scope(|| { - // Application graphql json expects valid graphql response - graphql::Response::from_bytes(service_name, body.into()) - .unwrap_or_else(|error| { - graphql::Response::builder() - .error(error.to_graphql_error(None)) - .build() - }) - }) - } - (Ok(ContentType::ApplicationJson), Some(Ok(body)), false) => { - // Application json does not expect a valid graphql response if not 2xx. - // If parse fails then attach the entire payload as an error - tracing::debug_span!("parse_subgraph_response").in_scope(|| { - // Application graphql json expects valid graphql response - let mut original_response = - String::from_utf8_lossy(&body).to_string(); - if original_response.is_empty() { - original_response = "".into() - } - graphql::Response::from_bytes(service_name, body.into()) - .unwrap_or_else(|_error| { - graphql::Response::builder() - .error( - FetchError::SubrequestMalformedResponse { - service: service_name.to_string(), - reason: original_response, - } - .to_graphql_error(None), - ) - .build() - }) - }) - } - (content_type, body, _) => { - // Something went wrong, compose a response with errors if they are present - let mut graphql_response = graphql::Response::builder().build(); - if let Err(err) = content_type { - graphql_response.errors.push(err.to_graphql_error(None)); + let mut original_response = + String::from_utf8_lossy(&body).to_string(); + if original_response.is_empty() { + original_response = "".into() } - /* TODO: XXX NEED TO UNDERSTAND - if let Some(Err(err)) = body { - graphql_response.errors.push(err.to_graphql_error(None)); - } - */ - graphql_response + graphql::Response::from_bytes(service_name, body.into()) + .unwrap_or_else(|_error| { + graphql::Response::builder() + .error( + FetchError::SubrequestMalformedResponse { + service: service_name.to_string(), + reason: original_response, + } + .to_graphql_error(None), + ) + .build() + }) + }) + } + (content_type, body, _) => { + // Something went wrong, compose a response with errors if they are present + let mut graphql_response = graphql::Response::builder().build(); + if let Err(err) = content_type { + graphql_response.errors.push(err.to_graphql_error(None)); } - }; - - // Add an error for response codes that are not 2xx - if !parts.status.is_success() { - let status = parts.status; - graphql_response.errors.insert( - 0, - FetchError::SubrequestHttpError { - service: service_name.to_string(), - status_code: Some(status.as_u16()), - reason: format!( - "{}: {}", - status.as_str(), - status.canonical_reason().unwrap_or("Unknown") - ), + /* TODO: XXX NEED TO UNDERSTAND + if let Some(Err(err)) = body { + graphql_response.errors.push(err.to_graphql_error(None)); } - .to_graphql_error(None), - ) - } - graphql_responses.push(graphql_response); - } - - tracing::info!("we have a vec of graphql_responses: {graphql_responses:?}"); - // Reverse txs to get things back in the right order - txs.reverse(); - for graphql_response in graphql_responses { - // Build an http Response - let resp = http::Response::builder() - .status(StatusCode::OK) - .body(graphql_response) - .expect("Response is serializable; qed"); - - // *response.headers_mut() = headers.unwrap_or_default(); - // let resp = http::Response::from_parts(parts, graphql_response); - tracing::info!("we have a resp: {resp:?}"); - let subgraph_response = - SubgraphResponse::new_from_response(resp, context.clone()); - match txs.pop() { - Some(tx) => { - tx.send(Ok(subgraph_response)).map_err(|_error| { - FetchError::SubrequestMalformedResponse { - service: service_name.to_string(), - reason: "tx send failed".to_string(), - } - })?; + */ + graphql_response } - None => { - tracing::info!( - "TO PROCESS THIS BATCH WE ISSUE: {do_fetch_count} fetches" - ); - return Ok(subgraph_response); + }; + + // Add an error for response codes that are not 2xx + if !parts.status.is_success() { + let status = parts.status; + graphql_response.errors.insert( + 0, + FetchError::SubrequestHttpError { + service: service_name.to_string(), + status_code: Some(status.as_u16()), + reason: format!( + "{}: {}", + status.as_str(), + status.canonical_reason().unwrap_or("Unknown") + ), } - } + .to_graphql_error(None), + ) } + graphql_responses.push(graphql_response); } - /* - // Instead of todo return a fake error for now - return Err(Box::new(FetchError::SubrequestMalformedResponse { - service: service_name.to_string(), - reason: "fake_error".to_string(), - })); - */ - // todo!() - } - None => { - tracing::info!("WE HAVE A NONE WAITER"); - /* - tracing::info!("WE HAVE A WAITER"); - // receiver.await? - match receiver.await { - Ok(v) => v, - Err(err) => { - panic!("A RECEIVER FAILED: {err}"); + + tracing::info!("we have a vec of graphql_responses: {graphql_responses:?}"); + // Reverse txs to get things back in the right order + txs.reverse(); + for graphql_response in graphql_responses { + // Build an http Response + let resp = http::Response::builder() + .status(StatusCode::OK) + .body(graphql_response) + .expect("Response is serializable; qed"); + + // *response.headers_mut() = headers.unwrap_or_default(); + // let resp = http::Response::from_parts(parts, graphql_response); + tracing::info!("we have a resp: {resp:?}"); + let subgraph_response = + SubgraphResponse::new_from_response(resp, context.clone()); + match txs.pop() { + Some(tx) => { + tx.send(Ok(subgraph_response)).map_err(|_error| { + FetchError::SubrequestMalformedResponse { + service: service_name.to_string(), + reason: "tx send failed".to_string(), + } + })?; + } + None => { + tracing::info!( + "TO PROCESS THIS BATCH WE ISSUE: {do_fetch_count} fetches" + ); + return Ok(subgraph_response); + } } } - */ } } tracing::info!("WE HAVE A WAITER"); - // receiver.await? - match receiver.await { - Ok(v) => v, - Err(err) => { - panic!("A RECEIVER FAILED: {err}"); - } - } + receiver.await? } else { tracing::info!("WE CALLED HTTP"); call_http(request, body, context, client, service_name).await diff --git a/apollo-router/src/services/supergraph/service.rs b/apollo-router/src/services/supergraph/service.rs index 89a731a333..c50f2e77b3 100644 --- a/apollo-router/src/services/supergraph/service.rs +++ b/apollo-router/src/services/supergraph/service.rs @@ -24,6 +24,7 @@ use tracing::field; use tracing::Span; use tracing_futures::Instrument; +use crate::batching::BatchDetails; use crate::configuration::Batching; use crate::context::OPERATION_NAME; use crate::error::CacheResolverError; @@ -54,7 +55,6 @@ use crate::services::layers::query_analysis::ParsedDocument; use crate::services::layers::query_analysis::QueryAnalysisLayer; use crate::services::new_service::ServiceFactory; use crate::services::query_planner; -use crate::services::router::service::BatchDetails; use crate::services::router::ClientRequestAccepts; use crate::services::subgraph::BoxGqlStream; use crate::services::subgraph_service::MakeSubgraphService; From ac9a0f8edbf2261f14053b6ec61e94355987a3cf Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Fri, 16 Feb 2024 11:07:38 +0000 Subject: [PATCH 09/60] remember to include the new source file in commits sigh... --- apollo-router/src/batching.rs | 151 ++++++++++++++++++ .../src/services/subgraph_service.rs | 12 +- 2 files changed, 158 insertions(+), 5 deletions(-) create mode 100644 apollo-router/src/batching.rs diff --git a/apollo-router/src/batching.rs b/apollo-router/src/batching.rs new file mode 100644 index 0000000000..daf01921e1 --- /dev/null +++ b/apollo-router/src/batching.rs @@ -0,0 +1,151 @@ +//! An assembly of utility functions and core structures used to implement batching support within +//! the router. +//! +//! Apart from the core batching functionality, as expressed in `BatchDetails` and +//! `SharedBatchDetails`, there are a series of utility functions for efficiently converting +//! graphql Requests to/from batch representation in a variety of formats: JSON, bytes + +use std::collections::HashMap; +use std::fmt; +use std::sync::Arc; + +use parking_lot::Mutex; +use tokio::sync::oneshot; +use tower::BoxError; + +use crate::graphql; +use crate::services::SubgraphRequest; +use crate::services::SubgraphResponse; +use crate::Context; + +#[derive(Clone, Debug, Default)] +pub(crate) struct BatchDetails { + index: usize, + // Shared Request Details + shared: Arc>, +} + +impl fmt::Display for BatchDetails { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "index: {}, ", self.index)?; + // Use try_lock. If the shared details are locked, we won't display them. + // TODO: Maybe improve to handle the error...? + let guard = self.shared.try_lock().ok_or(fmt::Error)?; + write!(f, "size: {}, ", guard.size)?; + write!(f, "expected: {:?}, ", guard.expected)?; + write!(f, "seen: {:?}", guard.seen)?; + for (service, details) in guard.waiters.iter() { + write!(f, ", service: {}, waiters: {}", service, details.len())?; + } + Ok(()) + } +} + +impl BatchDetails { + pub(crate) fn new(index: usize, shared: Arc>) -> Self { + Self { + index, + shared, + ..Default::default() + } + } + + pub(crate) fn ready(&self) -> bool { + self.shared.lock().ready() + } + + pub(crate) fn finished(&self) -> bool { + self.shared.lock().finished() + } + + pub(crate) fn get_waiter( + &self, + request: SubgraphRequest, + body: graphql::Request, + context: Context, + service_name: &str, + ) -> oneshot::Receiver> { + tracing::info!("getting a waiter for {}", self.index); + self.shared + .lock() + .get_waiter(request, body, context, service_name.to_string()) + } + + pub(crate) fn get_waiters( + &self, + ) -> HashMap< + String, + Vec<( + SubgraphRequest, + graphql::Request, + Context, + oneshot::Sender>, + )>, + > { + let mut guard = self.shared.lock(); + guard.finished = true; + std::mem::take(&mut guard.waiters) + } + + pub(crate) fn increment_subgraph_seen(&self) { + let mut shared_guard = self.shared.lock(); + let value = shared_guard.seen.entry(self.index).or_default(); + *value += 1; + } + + pub(crate) fn set_subgraph_fetches(&self, fetches: usize) { + let mut shared_guard = self.shared.lock(); + let value = shared_guard.expected.entry(self.index).or_default(); + *value = fetches; + } +} + +#[derive(Debug, Default)] +pub(crate) struct SharedBatchDetails { + size: usize, + expected: HashMap, + seen: HashMap, + waiters: HashMap< + String, + Vec<( + SubgraphRequest, + graphql::Request, + Context, + oneshot::Sender>, + )>, + >, + finished: bool, +} + +impl SharedBatchDetails { + pub(crate) fn new(size: usize) -> Self { + Self { + size, + expected: HashMap::new(), + seen: HashMap::new(), + waiters: HashMap::new(), + finished: false, + } + } + + fn ready(&self) -> bool { + self.expected.len() == self.size && self.expected == self.seen + } + + fn finished(&self) -> bool { + self.finished + } + + fn get_waiter( + &mut self, + request: SubgraphRequest, + body: graphql::Request, + context: Context, + service: String, + ) -> oneshot::Receiver> { + let (tx, rx) = oneshot::channel(); + let value = self.waiters.entry(service).or_default(); + value.push((request, body, context, tx)); + rx + } +} diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index f1d3f05785..8eee8d05f5 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -673,11 +673,9 @@ async fn call_batched_http( service_name, )); if batching.ready() { - //TODO: This is where we start processing our accumulated batch data - // Now we need to "batch up" our data and send it to our subgraphs - // We need our own batch aware version of call_http which only makes one call to each - // subgraph, but is able to decode the responses. I'll probably need to break call_http - // down into sub-functions. + // This is where we start processing our accumulated batch data. + // We can't do it whilst holding the context extensions lock, so signal we are + // ready to proceed by updating waiters_opt. tracing::info!("Batch data: {batching}"); waiters_opt = Some(batching.get_waiters()); } @@ -686,6 +684,10 @@ async fn call_batched_http( if let Some(receiver) = batch_responder { if let Some(waiters) = waiters_opt { for (service, requests) in waiters.into_iter() { + // Now we need to "batch up" our data and send it to our subgraphs + // We need our own batch aware version of call_http which only makes one call to each + // subgraph, but is able to decode the responses. I'll probably need to break call_http + // down into sub-functions, that's a TODO for now. let mut txs = Vec::with_capacity(requests.len()); let mut requests_it = requests.into_iter(); let first = requests_it From 6e0b5802dc85c24e3a9801ab1ab1a26cb1e19290 Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Tue, 20 Feb 2024 14:02:18 +0000 Subject: [PATCH 10/60] Rename core batching structs BatchDetails -> BatchQuery SharedBatchDetails -> Batch which better reflects their nature. --- apollo-router/src/batching.rs | 26 +++++++++---------- apollo-router/src/services/router/service.rs | 12 ++++----- .../src/services/subgraph_service.rs | 4 +-- .../src/services/supergraph/service.rs | 4 +-- 4 files changed, 23 insertions(+), 23 deletions(-) diff --git a/apollo-router/src/batching.rs b/apollo-router/src/batching.rs index daf01921e1..4b792a7804 100644 --- a/apollo-router/src/batching.rs +++ b/apollo-router/src/batching.rs @@ -1,8 +1,8 @@ //! An assembly of utility functions and core structures used to implement batching support within //! the router. //! -//! Apart from the core batching functionality, as expressed in `BatchDetails` and -//! `SharedBatchDetails`, there are a series of utility functions for efficiently converting +//! Apart from the core batching functionality, as expressed in `BatchQuery` and +//! `Batch`, there are a series of utility functions for efficiently converting //! graphql Requests to/from batch representation in a variety of formats: JSON, bytes use std::collections::HashMap; @@ -19,30 +19,30 @@ use crate::services::SubgraphResponse; use crate::Context; #[derive(Clone, Debug, Default)] -pub(crate) struct BatchDetails { +pub(crate) struct BatchQuery { index: usize, - // Shared Request Details - shared: Arc>, + // Shared Batch + shared: Arc>, } -impl fmt::Display for BatchDetails { +impl fmt::Display for BatchQuery { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { write!(f, "index: {}, ", self.index)?; - // Use try_lock. If the shared details are locked, we won't display them. + // Use try_lock. If the shared batch is locked, we won't display it. // TODO: Maybe improve to handle the error...? let guard = self.shared.try_lock().ok_or(fmt::Error)?; write!(f, "size: {}, ", guard.size)?; write!(f, "expected: {:?}, ", guard.expected)?; write!(f, "seen: {:?}", guard.seen)?; - for (service, details) in guard.waiters.iter() { - write!(f, ", service: {}, waiters: {}", service, details.len())?; + for (service, waiters) in guard.waiters.iter() { + write!(f, ", service: {}, waiters: {}", service, waiters.len())?; } Ok(()) } } -impl BatchDetails { - pub(crate) fn new(index: usize, shared: Arc>) -> Self { +impl BatchQuery { + pub(crate) fn new(index: usize, shared: Arc>) -> Self { Self { index, shared, @@ -101,7 +101,7 @@ impl BatchDetails { } #[derive(Debug, Default)] -pub(crate) struct SharedBatchDetails { +pub(crate) struct Batch { size: usize, expected: HashMap, seen: HashMap, @@ -117,7 +117,7 @@ pub(crate) struct SharedBatchDetails { finished: bool, } -impl SharedBatchDetails { +impl Batch { pub(crate) fn new(size: usize) -> Self { Self { size, diff --git a/apollo-router/src/services/router/service.rs b/apollo-router/src/services/router/service.rs index f7873344fc..1430bef899 100644 --- a/apollo-router/src/services/router/service.rs +++ b/apollo-router/src/services/router/service.rs @@ -36,8 +36,8 @@ use tower_service::Service; use tracing::Instrument; use super::ClientRequestAccepts; -use crate::batching::BatchDetails; -use crate::batching::SharedBatchDetails; +use crate::batching::Batch; +use crate::batching::BatchQuery; use crate::cache::DeduplicatingCache; use crate::configuration::Batching; use crate::configuration::BatchingMode; @@ -614,12 +614,12 @@ impl RouterService { let mut results = Vec::with_capacity(ok_results.len()); let batch_size = ok_results.len(); - let shared_batch_details: Option>> = if ok_results.len() > 1 { + let shared_batch_details: Option>> = if ok_results.len() > 1 { context .extensions() .lock() .insert(self.experimental_batching.clone()); - Some(Arc::new(Mutex::new(SharedBatchDetails::new(batch_size)))) + Some(Arc::new(Mutex::new(Batch::new(batch_size)))) } else { None }; @@ -666,7 +666,7 @@ impl RouterService { new_context_guard.insert(self.experimental_batching.clone()); if let Some(shared_batch_details) = &shared_batch_details { new_context_guard - .insert(BatchDetails::new(index + 1, shared_batch_details.clone())); + .insert(BatchQuery::new(index + 1, shared_batch_details.clone())); } } @@ -681,7 +681,7 @@ impl RouterService { context .extensions() .lock() - .insert(BatchDetails::new(0, shared_batch_details)); + .insert(BatchQuery::new(0, shared_batch_details)); } results.insert( diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index 8eee8d05f5..41c77cd354 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -43,7 +43,7 @@ use super::http::HttpClientServiceFactory; use super::http::HttpRequest; use super::layers::content_negotiation::GRAPHQL_JSON_RESPONSE_HEADER_VALUE; use super::Plugins; -use crate::batching::BatchDetails; +use crate::batching::BatchQuery; use crate::configuration::TlsClientAuth; use crate::error::FetchError; use crate::graphql; @@ -661,7 +661,7 @@ async fn call_batched_http( tokio::sync::oneshot::Receiver>, > = None; let mut waiters_opt = None; - if let Some(batching) = context.extensions().lock().get_mut::() { + if let Some(batching) = context.extensions().lock().get_mut::() { if !batching.finished() { tracing::info!("in subgraph we have batching: {batching}, service: {service_name}"); batching.increment_subgraph_seen(); diff --git a/apollo-router/src/services/supergraph/service.rs b/apollo-router/src/services/supergraph/service.rs index c50f2e77b3..d1b1a52124 100644 --- a/apollo-router/src/services/supergraph/service.rs +++ b/apollo-router/src/services/supergraph/service.rs @@ -24,7 +24,7 @@ use tracing::field; use tracing::Span; use tracing_futures::Instrument; -use crate::batching::BatchDetails; +use crate::batching::BatchQuery; use crate::configuration::Batching; use crate::context::OPERATION_NAME; use crate::error::CacheResolverError; @@ -621,7 +621,7 @@ async fn plan_query( )) .await?; - if let Some(batching) = context.extensions().lock().get::() { + if let Some(batching) = context.extensions().lock().get::() { if let Some(QueryPlannerContent::Plan { plan, .. }) = &qpr.content { let no_requires_fetches = plan.root.subgraph_fetches(false); batching.set_subgraph_fetches(no_requires_fetches); From 46d9bec20826080beb7301ec182a0cca30ace5cb Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Tue, 20 Feb 2024 15:25:03 +0000 Subject: [PATCH 11/60] Introduce Waiter struct and make code xtask lint clean Replace the 4 element tuple with a struct which captures all the relevant data for waiting for a request. At some point the impl for a Waiter will have functions that cover formatting and transforming from different representations, but not yet... --- apollo-router/src/batching.rs | 54 ++++++++++--------- apollo-router/src/response.rs | 1 + .../src/services/subgraph_service.rs | 16 +++--- 3 files changed, 37 insertions(+), 34 deletions(-) diff --git a/apollo-router/src/batching.rs b/apollo-router/src/batching.rs index 4b792a7804..246be3edbb 100644 --- a/apollo-router/src/batching.rs +++ b/apollo-router/src/batching.rs @@ -43,11 +43,7 @@ impl fmt::Display for BatchQuery { impl BatchQuery { pub(crate) fn new(index: usize, shared: Arc>) -> Self { - Self { - index, - shared, - ..Default::default() - } + Self { index, shared } } pub(crate) fn ready(&self) -> bool { @@ -71,17 +67,7 @@ impl BatchQuery { .get_waiter(request, body, context, service_name.to_string()) } - pub(crate) fn get_waiters( - &self, - ) -> HashMap< - String, - Vec<( - SubgraphRequest, - graphql::Request, - Context, - oneshot::Sender>, - )>, - > { + pub(crate) fn get_waiters(&self) -> HashMap> { let mut guard = self.shared.lock(); guard.finished = true; std::mem::take(&mut guard.waiters) @@ -105,15 +91,7 @@ pub(crate) struct Batch { size: usize, expected: HashMap, seen: HashMap, - waiters: HashMap< - String, - Vec<( - SubgraphRequest, - graphql::Request, - Context, - oneshot::Sender>, - )>, - >, + waiters: HashMap>, finished: bool, } @@ -145,7 +123,31 @@ impl Batch { ) -> oneshot::Receiver> { let (tx, rx) = oneshot::channel(); let value = self.waiters.entry(service).or_default(); - value.push((request, body, context, tx)); + value.push(Waiter::new(request, body, context, tx)); rx } } + +#[derive(Debug)] +pub(crate) struct Waiter { + pub(crate) sg_request: SubgraphRequest, + pub(crate) gql_request: graphql::Request, + pub(crate) context: Context, + pub(crate) sender: oneshot::Sender>, +} + +impl Waiter { + fn new( + sg_request: SubgraphRequest, + gql_request: graphql::Request, + context: Context, + sender: oneshot::Sender>, + ) -> Self { + Self { + sg_request, + gql_request, + context, + sender, + } + } +} diff --git a/apollo-router/src/response.rs b/apollo-router/src/response.rs index d590f49545..a3a3f3c5e5 100644 --- a/apollo-router/src/response.rs +++ b/apollo-router/src/response.rs @@ -96,6 +96,7 @@ impl Response { /// Create a Vec of [`Response`] from the supplied [`Bytes`]. /// /// This will return a Vec of response/errors (identifying the faulty service) if the input is invalid. + #[allow(dead_code)] pub(crate) fn array_from_bytes( service_name: &str, b: Bytes, diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index 41c77cd354..fed4ebcba6 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -693,11 +693,11 @@ async fn call_batched_http( let first = requests_it .next() .expect("we should have at least one request"); - let context = first.2; // XXX SHADOWING - txs.push(first.3); + let context = first.context; // XXX SHADOWING + txs.push(first.sender); let SubgraphRequest { subgraph_request, .. - } = first.0; + } = first.sg_request; let operation_name = subgraph_request .body() .operation_name @@ -705,15 +705,15 @@ async fn call_batched_http( .unwrap_or_default(); let (parts, _) = subgraph_request.into_parts(); - let body = - serde_json::to_string(&first.1).expect("JSON serialization should not fail"); + let body = serde_json::to_string(&first.gql_request) + .expect("JSON serialization should not fail"); let mut bytes = BytesMut::new(); bytes.put_u8(b'['); bytes.extend_from_slice(&hyper::body::to_bytes(body).await?); for request in requests_it { - txs.push(request.3); + txs.push(request.sender); bytes.put(&b", "[..]); - let body = serde_json::to_string(&request.1) + let body = serde_json::to_string(&request.gql_request) .expect("JSON serialization should not fail"); bytes.extend_from_slice(&hyper::body::to_bytes(body).await?); } @@ -854,7 +854,7 @@ async fn call_batched_http( }) }) } - (content_type, body, _) => { + (content_type, _body, _) => { // Something went wrong, compose a response with errors if they are present let mut graphql_response = graphql::Response::builder().build(); if let Err(err) = content_type { From 110c9ba8ff2ee7139d2a8d6676437e8c185bdb99 Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Wed, 21 Feb 2024 11:43:37 +0000 Subject: [PATCH 12/60] cleanup some of the code and add some comments about future work Rename some things to make it more obvious what's going on. Get ready to add some utility functions to batching.rs --- .../src/services/subgraph_service.rs | 36 ++++++++++--------- 1 file changed, 19 insertions(+), 17 deletions(-) diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index fed4ebcba6..4acbd2521c 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -683,14 +683,14 @@ async fn call_batched_http( } if let Some(receiver) = batch_responder { if let Some(waiters) = waiters_opt { - for (service, requests) in waiters.into_iter() { + for (service, service_waiters) in waiters.into_iter() { // Now we need to "batch up" our data and send it to our subgraphs // We need our own batch aware version of call_http which only makes one call to each // subgraph, but is able to decode the responses. I'll probably need to break call_http // down into sub-functions, that's a TODO for now. - let mut txs = Vec::with_capacity(requests.len()); - let mut requests_it = requests.into_iter(); - let first = requests_it + let mut txs = Vec::with_capacity(service_waiters.len()); + let mut service_waiters_it = service_waiters.into_iter(); + let first = service_waiters_it .next() .expect("we should have at least one request"); let context = first.context; // XXX SHADOWING @@ -710,10 +710,10 @@ async fn call_batched_http( let mut bytes = BytesMut::new(); bytes.put_u8(b'['); bytes.extend_from_slice(&hyper::body::to_bytes(body).await?); - for request in requests_it { - txs.push(request.sender); + for waiter in service_waiters_it { + txs.push(waiter.sender); bytes.put(&b", "[..]); - let body = serde_json::to_string(&request.gql_request) + let body = serde_json::to_string(&waiter.gql_request) .expect("JSON serialization should not fail"); bytes.extend_from_slice(&hyper::body::to_bytes(body).await?); } @@ -744,6 +744,8 @@ async fn call_batched_http( let path = schema_uri.path(); + // TODO: We have multiple operation names but we are just using the first operation + // name in the span. Should we report all operation names? let subgraph_req_span = tracing::info_span!("subgraph_request", "otel.kind" = "CLIENT", "net.peer.name" = %host, @@ -751,7 +753,7 @@ async fn call_batched_http( "http.route" = %path, "http.url" = %schema_uri, "net.transport" = "ip_tcp", - "apollo.subgraph.name" = %service_name, + "apollo.subgraph.name" = %&service, "graphql.operation.name" = %operation_name, ); @@ -774,7 +776,7 @@ async fn call_batched_http( let client = client_factory.create(&service); // Perform the actual fetch. If this fails then we didn't manage to make the call at all, so we can't do anything with it. let (parts, content_type, body) = - do_fetch(client, &context, service_name, request, display_body) + do_fetch(client, &context, &service, request, display_body) .instrument(subgraph_req_span) .await?; do_fetch_count += 1; @@ -782,7 +784,7 @@ async fn call_batched_http( if display_body { if let Some(Ok(b)) = &body { tracing::info!( - response.body = %String::from_utf8_lossy(b), apollo.subgraph.name = %service_name, "Raw response body from subgraph {service_name:?} received" + response.body = %String::from_utf8_lossy(b), apollo.subgraph.name = %&service, "Raw response body from subgraph {service:?} received" ); } } @@ -796,7 +798,7 @@ async fn call_batched_http( let array = ensure_array!(value).map_err(|error| { FetchError::SubrequestMalformedResponse { - service: service_name.to_string(), + service: service.to_string(), reason: error.to_string(), } })?; @@ -804,7 +806,7 @@ async fn call_batched_http( for value in array { let object = ensure_object!(value).map_err(|error| { FetchError::SubrequestMalformedResponse { - service: service_name.to_string(), + service: service.to_string(), reason: error.to_string(), } })?; @@ -822,7 +824,7 @@ async fn call_batched_http( // Application json expects valid graphql response if 2xx tracing::debug_span!("parse_subgraph_response").in_scope(|| { // Application graphql json expects valid graphql response - graphql::Response::from_bytes(service_name, body.into()) + graphql::Response::from_bytes(&service, body.into()) .unwrap_or_else(|error| { graphql::Response::builder() .error(error.to_graphql_error(None)) @@ -840,12 +842,12 @@ async fn call_batched_http( if original_response.is_empty() { original_response = "".into() } - graphql::Response::from_bytes(service_name, body.into()) + graphql::Response::from_bytes(&service, body.into()) .unwrap_or_else(|_error| { graphql::Response::builder() .error( FetchError::SubrequestMalformedResponse { - service: service_name.to_string(), + service: service.to_string(), reason: original_response, } .to_graphql_error(None), @@ -875,7 +877,7 @@ async fn call_batched_http( graphql_response.errors.insert( 0, FetchError::SubrequestHttpError { - service: service_name.to_string(), + service: service.to_string(), status_code: Some(status.as_u16()), reason: format!( "{}: {}", @@ -908,7 +910,7 @@ async fn call_batched_http( Some(tx) => { tx.send(Ok(subgraph_response)).map_err(|_error| { FetchError::SubrequestMalformedResponse { - service: service_name.to_string(), + service: service.to_string(), reason: "tx send failed".to_string(), } })?; From df6e4bf81f0ff93bba2bb01f9c61fde8124734b3 Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Wed, 21 Feb 2024 12:39:32 +0000 Subject: [PATCH 13/60] Implement Waiter::assemble_batch To move some of the code into a function. --- apollo-router/src/batching.rs | 58 +++++++++++++++++++ .../src/services/subgraph_service.rs | 40 +------------ 2 files changed, 61 insertions(+), 37 deletions(-) diff --git a/apollo-router/src/batching.rs b/apollo-router/src/batching.rs index 246be3edbb..c5634cc37b 100644 --- a/apollo-router/src/batching.rs +++ b/apollo-router/src/batching.rs @@ -9,6 +9,9 @@ use std::collections::HashMap; use std::fmt; use std::sync::Arc; +use bytes::BufMut; +use bytes::BytesMut; +use hyper::Body; use parking_lot::Mutex; use tokio::sync::oneshot; use tower::BoxError; @@ -150,4 +153,59 @@ impl Waiter { sender, } } + + // Form a batch from a collection of waiting requests. The first operation provides: + // - operation name + // - context + // - parts + // This is ok, because when the batch was created, the parts and context were primarily created + // by extracting and duplicating information from the single batch request. Maybe we should use + // a different operation name, maybe chain them all together? TODO: Decide operation name + pub(crate) async fn assemble_batch( + service_waiters: Vec, + ) -> Result< + ( + String, + Context, + http::Request, + Vec>>, + ), + BoxError, + > { + let mut txs = Vec::with_capacity(service_waiters.len()); + let mut service_waiters_it = service_waiters.into_iter(); + let first = service_waiters_it + .next() + .expect("we should have at least one request"); + let context = first.context; + txs.push(first.sender); + let SubgraphRequest { + subgraph_request, .. + } = first.sg_request; + let operation_name = subgraph_request + .body() + .operation_name + .clone() + .unwrap_or_default(); + + let (parts, _) = subgraph_request.into_parts(); + let body = + serde_json::to_string(&first.gql_request).expect("JSON serialization should not fail"); + let mut bytes = BytesMut::new(); + bytes.put_u8(b'['); + bytes.extend_from_slice(&hyper::body::to_bytes(body).await?); + for waiter in service_waiters_it { + txs.push(waiter.sender); + bytes.put(&b", "[..]); + let body = serde_json::to_string(&waiter.gql_request) + .expect("JSON serialization should not fail"); + bytes.extend_from_slice(&hyper::body::to_bytes(body).await?); + } + bytes.put_u8(b']'); + let body_bytes = bytes.freeze(); + // Reverse txs to get them in the right order + txs.reverse(); + let request = http::Request::from_parts(parts, Body::from(body_bytes)); + Ok((operation_name, context, request, txs)) + } } diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index 4acbd2521c..93acafe1e5 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -6,9 +6,7 @@ use std::sync::atomic::Ordering::Relaxed; use std::sync::Arc; use std::task::Poll; -use bytes::BufMut; use bytes::Bytes; -use bytes::BytesMut; use futures::future::BoxFuture; use futures::SinkExt; use futures::StreamExt; @@ -44,6 +42,7 @@ use super::http::HttpRequest; use super::layers::content_negotiation::GRAPHQL_JSON_RESPONSE_HEADER_VALUE; use super::Plugins; use crate::batching::BatchQuery; +use crate::batching::Waiter; use crate::configuration::TlsClientAuth; use crate::error::FetchError; use crate::graphql; @@ -688,39 +687,8 @@ async fn call_batched_http( // We need our own batch aware version of call_http which only makes one call to each // subgraph, but is able to decode the responses. I'll probably need to break call_http // down into sub-functions, that's a TODO for now. - let mut txs = Vec::with_capacity(service_waiters.len()); - let mut service_waiters_it = service_waiters.into_iter(); - let first = service_waiters_it - .next() - .expect("we should have at least one request"); - let context = first.context; // XXX SHADOWING - txs.push(first.sender); - let SubgraphRequest { - subgraph_request, .. - } = first.sg_request; - let operation_name = subgraph_request - .body() - .operation_name - .clone() - .unwrap_or_default(); - - let (parts, _) = subgraph_request.into_parts(); - let body = serde_json::to_string(&first.gql_request) - .expect("JSON serialization should not fail"); - let mut bytes = BytesMut::new(); - bytes.put_u8(b'['); - bytes.extend_from_slice(&hyper::body::to_bytes(body).await?); - for waiter in service_waiters_it { - txs.push(waiter.sender); - bytes.put(&b", "[..]); - let body = serde_json::to_string(&waiter.gql_request) - .expect("JSON serialization should not fail"); - bytes.extend_from_slice(&hyper::body::to_bytes(body).await?); - } - bytes.put_u8(b']'); - let body_bytes = bytes.freeze(); - tracing::info!("ABOUT TO CREATE BATCH: {:?}", body_bytes); - let mut request = http::Request::from_parts(parts, Body::from(body_bytes)); + let (operation_name, context, mut request, mut txs) = + Waiter::assemble_batch(service_waiters).await?; request .headers_mut() @@ -892,8 +860,6 @@ async fn call_batched_http( } tracing::info!("we have a vec of graphql_responses: {graphql_responses:?}"); - // Reverse txs to get things back in the right order - txs.reverse(); for graphql_response in graphql_responses { // Build an http Response let resp = http::Response::builder() From 015c860d9768f175f7bace622829a4d0c1e76d16 Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Wed, 21 Feb 2024 15:28:13 +0000 Subject: [PATCH 14/60] extract get_uri_details() fn Start trying to break down the megafunction that is call_http(). --- .../src/services/subgraph_service.rs | 43 ++++++++----------- 1 file changed, 17 insertions(+), 26 deletions(-) diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index 93acafe1e5..ecf0f58352 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -644,6 +644,21 @@ async fn call_websocket( )) } +fn get_uri_details(uri: &hyper::Uri) -> (&str, u16, &str) { + let port = uri.port_u16().unwrap_or_else(|| { + let scheme = uri.scheme_str(); + if scheme == Some("https") { + 443 + } else if scheme == Some("http") { + 80 + } else { + 0 + } + }); + + (uri.host().unwrap_or_default(), port, uri.path()) +} + async fn call_batched_http( request: SubgraphRequest, body: graphql::Request, @@ -698,19 +713,7 @@ async fn call_batched_http( .append(ACCEPT, ACCEPT_GRAPHQL_JSON.clone()); let schema_uri = request.uri(); - let host = schema_uri.host().unwrap_or_default(); - let port = schema_uri.port_u16().unwrap_or_else(|| { - let scheme = schema_uri.scheme_str(); - if scheme == Some("https") { - 443 - } else if scheme == Some("http") { - 80 - } else { - 0 - } - }); - - let path = schema_uri.path(); + let (host, port, path) = get_uri_details(schema_uri); // TODO: We have multiple operation names but we are just using the first operation // name in the span. Should we report all operation names? @@ -930,19 +933,7 @@ pub(crate) async fn call_http( .append(ACCEPT, ACCEPT_GRAPHQL_JSON.clone()); let schema_uri = request.uri(); - let host = schema_uri.host().unwrap_or_default(); - let port = schema_uri.port_u16().unwrap_or_else(|| { - let scheme = schema_uri.scheme_str(); - if scheme == Some("https") { - 443 - } else if scheme == Some("http") { - 80 - } else { - 0 - } - }); - - let path = schema_uri.path(); + let (host, port, path) = get_uri_details(schema_uri); let subgraph_req_span = tracing::info_span!("subgraph_request", "otel.kind" = "CLIENT", From c9bbdf5c090637a2a5445ea9e0a84536f47ba92f Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Thu, 22 Feb 2024 14:27:04 +0000 Subject: [PATCH 15/60] impl Drop for Batch until I figure out implications We can't leave waiters in a Batch, since that would mean a request had been "lost" with all the negative implications for that client. So, until I figure out the details, impl Drop for Batch and panic if self.waiters is not empty. --- apollo-router/src/batching.rs | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/apollo-router/src/batching.rs b/apollo-router/src/batching.rs index c5634cc37b..0b7e0755b9 100644 --- a/apollo-router/src/batching.rs +++ b/apollo-router/src/batching.rs @@ -209,3 +209,15 @@ impl Waiter { Ok((operation_name, context, request, txs)) } } + +// If a Batch is dropped and it still contains waiters, it's important to notify those waiters that +// their calls have failed. +// +// TODO: Figure out the implications, but panic for now if waiters is not empty +impl Drop for Batch { + fn drop(&mut self) { + if !self.waiters.is_empty() { + panic!("TODO: waiters must be empty when a Batch is dropped"); + } + } +} From 76e3aeace6f7bb21cd71c3eebb936cec843f1ee5 Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Thu, 22 Feb 2024 15:50:40 +0000 Subject: [PATCH 16/60] Introduce SubrequestBatchingError and clean up code a little We need a new error to represent things that can go wrong when we are manipulating batches. This new type of FetchError does the job for us. I've also cleaned up some of the interactions with other functions, and tracing. --- apollo-router/src/error.rs | 10 ++++ .../src/services/subgraph_service.rs | 51 ++++++++++--------- 2 files changed, 37 insertions(+), 24 deletions(-) diff --git a/apollo-router/src/error.rs b/apollo-router/src/error.rs index 6ffba0fdc3..94472b40ea 100644 --- a/apollo-router/src/error.rs +++ b/apollo-router/src/error.rs @@ -98,6 +98,15 @@ pub(crate) enum FetchError { /// could not find path: {reason} ExecutionPathNotFound { reason: String }, + + /// Batching error for '{service}': {reason} + SubrequestBatchingError { + /// The service for which batch processing failed. + service: String, + + /// The reason batch processing failed. + reason: String, + }, } impl FetchError { @@ -171,6 +180,7 @@ impl ErrorExtension for FetchError { FetchError::ExecutionPathNotFound { .. } => "EXECUTION_PATH_NOT_FOUND", FetchError::MalformedRequest { .. } => "MALFORMED_REQUEST", FetchError::MalformedResponse { .. } => "MALFORMED_RESPONSE", + FetchError::SubrequestBatchingError { .. } => "SUBREQUEST_BATCHING_ERROR", } .to_string() } diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index bf62c708af..be96ceac67 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -25,7 +25,6 @@ use mediatype::names::JSON; use mediatype::MediaType; use mime::APPLICATION_JSON; use rustls::RootCertStore; -use serde::de::Error; use serde::Serialize; use tokio_tungstenite::connect_async; use tokio_tungstenite::connect_async_tls_with_config; @@ -364,8 +363,6 @@ impl tower::Service for SubgraphService { } } - let client = client_factory.create(&service_name); - // If APQ is not enabled, simply make the graphql call // with the same request body. let apq_enabled = arc_apq_enabled.as_ref(); @@ -374,7 +371,6 @@ impl tower::Service for SubgraphService { request, body, context, - client, client_factory.clone(), &service_name, ) @@ -412,7 +408,6 @@ impl tower::Service for SubgraphService { request.clone(), apq_body.clone(), context.clone(), - client_factory.create(&service_name), client_factory.clone(), &service_name, ) @@ -430,7 +425,6 @@ impl tower::Service for SubgraphService { request, body, context, - client, client_factory.clone(), &service_name, ) @@ -442,7 +436,6 @@ impl tower::Service for SubgraphService { request, apq_body, context, - client, client_factory.clone(), &service_name, ) @@ -664,7 +657,6 @@ async fn call_batched_http( request: SubgraphRequest, body: graphql::Request, context: Context, - client: crate::services::http::BoxService, client_factory: crate::services::http::HttpClientServiceFactory, service_name: &str, ) -> Result { @@ -678,9 +670,9 @@ async fn call_batched_http( let mut waiters_opt = None; if let Some(batching) = context.extensions().lock().get_mut::() { if !batching.finished() { - tracing::info!("in subgraph we have batching: {batching}, service: {service_name}"); + tracing::debug!("in subgraph we have batching: {batching}, service: {service_name}"); batching.increment_subgraph_seen(); - tracing::info!("ready to process batch?: {}", batching.ready()); + tracing::debug!("ready to process batch?: {}", batching.ready()); batch_responder = Some(batching.get_waiter( request.clone(), body.clone(), @@ -691,7 +683,7 @@ async fn call_batched_http( // This is where we start processing our accumulated batch data. // We can't do it whilst holding the context extensions lock, so signal we are // ready to proceed by updating waiters_opt. - tracing::info!("Batch data: {batching}"); + tracing::debug!("Batch data: {batching}"); waiters_opt = Some(batching.get_waiters()); } } @@ -761,12 +753,16 @@ async fn call_batched_http( } } - tracing::info!("parts: {parts:?}, content_type: {content_type:?}, body: {body:?}"); + tracing::debug!("parts: {parts:?}, content_type: {content_type:?}, body: {body:?}"); - let value = serde_json::from_slice(&body.unwrap().unwrap()) - .map_err(serde_json::Error::custom)?; + let value = serde_json::from_slice(&body.unwrap().unwrap()).map_err(|error| { + FetchError::SubrequestMalformedResponse { + service: service.to_string(), + reason: error.to_string(), + } + })?; - tracing::info!("JSON VALUE FROM BODY IS: {value:?}"); + tracing::debug!("JSON VALUE FROM BODY IS: {value:?}"); let array = ensure_array!(value).map_err(|error| { FetchError::SubrequestMalformedResponse { @@ -863,7 +859,7 @@ async fn call_batched_http( graphql_responses.push(graphql_response); } - tracing::info!("we have a vec of graphql_responses: {graphql_responses:?}"); + tracing::debug!("we have a vec of graphql_responses: {graphql_responses:?}"); for graphql_response in graphql_responses { // Build an http Response let resp = http::Response::builder() @@ -873,20 +869,20 @@ async fn call_batched_http( // *response.headers_mut() = headers.unwrap_or_default(); // let resp = http::Response::from_parts(parts, graphql_response); - tracing::info!("we have a resp: {resp:?}"); + tracing::debug!("we have a resp: {resp:?}"); let subgraph_response = SubgraphResponse::new_from_response(resp, context.clone()); match txs.pop() { Some(tx) => { - tx.send(Ok(subgraph_response)).map_err(|_error| { - FetchError::SubrequestMalformedResponse { + tx.send(Ok(subgraph_response)).map_err(|error| { + FetchError::SubrequestBatchingError { service: service.to_string(), - reason: "tx send failed".to_string(), + reason: format!("tx send failed: {error:?}"), } })?; } None => { - tracing::info!( + tracing::debug!( "TO PROCESS THIS BATCH WE ISSUE: {do_fetch_count} fetches" ); return Ok(subgraph_response); @@ -895,10 +891,17 @@ async fn call_batched_http( } } } - tracing::info!("WE HAVE A WAITER"); - receiver.await? + tracing::debug!("WE HAVE A WAITER"); + // If we get an error whilst waiting, then something bad has happened + receiver + .await + .map_err(|err| FetchError::SubrequestBatchingError { + service: service_name.to_string(), + reason: format!("tx receive failed: {err}"), + })? } else { - tracing::info!("WE CALLED HTTP"); + tracing::debug!("WE CALLED HTTP"); + let client = client_factory.create(service_name); call_http(request, body, context, client, service_name).await } } From cf254e6e52d50da75b316f816e5cd0492dff430e Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Tue, 5 Mar 2024 13:35:22 +0000 Subject: [PATCH 17/60] Clean up the batching functionality a bit more Extract out the http_response_into_graphql_response() function and improve the building of arrays of batch responses. Also: - add checks/invariants for array length comparison - indicate when code is unreachable - add some comments --- apollo-router/src/batching.rs | 2 +- .../src/services/subgraph_service.rs | 291 ++++++++---------- 2 files changed, 131 insertions(+), 162 deletions(-) diff --git a/apollo-router/src/batching.rs b/apollo-router/src/batching.rs index 0b7e0755b9..f4a6cfeb76 100644 --- a/apollo-router/src/batching.rs +++ b/apollo-router/src/batching.rs @@ -1,7 +1,7 @@ //! An assembly of utility functions and core structures used to implement batching support within //! the router. //! -//! Apart from the core batching functionality, as expressed in `BatchQuery` and +//! In addition to the core batching functionality, as expressed in `BatchQuery` and //! `Batch`, there are a series of utility functions for efficiently converting //! graphql Requests to/from batch representation in a variety of formats: JSON, bytes diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index be96ceac67..2f81791283 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -17,7 +17,6 @@ use http::header::{self}; use http::response::Parts; use http::HeaderValue; use http::Request; -use http::StatusCode; use hyper::Body; use hyper_rustls::ConfigBuilderExt; use mediatype::names::APPLICATION; @@ -638,6 +637,7 @@ async fn call_websocket( )) } +// Utility function to extract uri details. fn get_uri_details(uri: &hyper::Uri) -> (&str, u16, &str) { let port = uri.port_u16().unwrap_or_else(|| { let scheme = uri.scheme_str(); @@ -653,6 +653,82 @@ fn get_uri_details(uri: &hyper::Uri) -> (&str, u16, &str) { (uri.host().unwrap_or_default(), port, uri.path()) } +// Utility function to create a graphql response from HTTP response components +fn http_response_to_graphql_response( + service_name: &str, + content_type: Result, + body: Option>, + parts: &Parts, +) -> graphql::Response { + let mut graphql_response = match (content_type, body, parts.status.is_success()) { + (Ok(ContentType::ApplicationGraphqlResponseJson), Some(Ok(body)), _) + | (Ok(ContentType::ApplicationJson), Some(Ok(body)), true) => { + // Application graphql json expects valid graphql response + // Application json expects valid graphql response if 2xx + tracing::debug_span!("parse_subgraph_response").in_scope(|| { + // Application graphql json expects valid graphql response + graphql::Response::from_bytes(service_name, body).unwrap_or_else(|error| { + graphql::Response::builder() + .error(error.to_graphql_error(None)) + .build() + }) + }) + } + (Ok(ContentType::ApplicationJson), Some(Ok(body)), false) => { + // Application json does not expect a valid graphql response if not 2xx. + // If parse fails then attach the entire payload as an error + tracing::debug_span!("parse_subgraph_response").in_scope(|| { + // Application graphql json expects valid graphql response + let mut original_response = String::from_utf8_lossy(&body).to_string(); + if original_response.is_empty() { + original_response = "".into() + } + graphql::Response::from_bytes(service_name, body).unwrap_or_else(|_error| { + graphql::Response::builder() + .error( + FetchError::SubrequestMalformedResponse { + service: service_name.to_string(), + reason: original_response, + } + .to_graphql_error(None), + ) + .build() + }) + }) + } + (content_type, body, _) => { + // Something went wrong, compose a response with errors if they are present + let mut graphql_response = graphql::Response::builder().build(); + if let Err(err) = content_type { + graphql_response.errors.push(err.to_graphql_error(None)); + } + if let Some(Err(err)) = body { + graphql_response.errors.push(err.to_graphql_error(None)); + } + graphql_response + } + }; + + // Add an error for response codes that are not 2xx + if !parts.status.is_success() { + let status = parts.status; + graphql_response.errors.insert( + 0, + FetchError::SubrequestHttpError { + service: service_name.to_string(), + status_code: Some(status.as_u16()), + reason: format!( + "{}: {}", + status.as_str(), + status.canonical_reason().unwrap_or("Unknown") + ), + } + .to_graphql_error(None), + ) + } + graphql_response +} + async fn call_batched_http( request: SubgraphRequest, body: graphql::Request, @@ -660,7 +736,6 @@ async fn call_batched_http( client_factory: crate::services::http::HttpClientServiceFactory, service_name: &str, ) -> Result { - let mut do_fetch_count = 0; // We'd like to park a task here, but we can't park it whilst we have the context extensions // lock held. That would be very bad... // So, we set an optional listener and wait to hear back from the batch processor @@ -688,13 +763,17 @@ async fn call_batched_http( } } } + // We've dropped the extensions lock, check to see if we have batches to process or just a + // normal http call. + // TODO: Think about the impact on the router if a batch is never finished/ready. Can that happen? if let Some(receiver) = batch_responder { + // If waiters_opt is Some, then our batch is full and it's time to process it. if let Some(waiters) = waiters_opt { for (service, service_waiters) in waiters.into_iter() { // Now we need to "batch up" our data and send it to our subgraphs // We need our own batch aware version of call_http which only makes one call to each // subgraph, but is able to decode the responses. I'll probably need to break call_http - // down into sub-functions, that's a TODO for now. + // down into sub-functions, and I've started this, but it's not finished. let (operation_name, context, mut request, mut txs) = Waiter::assemble_batch(service_waiters).await?; @@ -743,7 +822,6 @@ async fn call_batched_http( do_fetch(client, &context, &service, request, display_body) .instrument(subgraph_req_span) .await?; - do_fetch_count += 1; if display_body { if let Some(Ok(b)) = &body { @@ -762,7 +840,7 @@ async fn call_batched_http( } })?; - tracing::debug!("JSON VALUE FROM BODY IS: {value:?}"); + tracing::debug!("json value from body is: {value:?}"); let array = ensure_array!(value).map_err(|error| { FetchError::SubrequestMalformedResponse { @@ -778,100 +856,52 @@ async fn call_batched_http( reason: error.to_string(), } })?; - let body = Some(serde_json::to_vec(&object)); - - let mut graphql_response = - match (content_type.clone(), body, parts.status.is_success()) { - ( - Ok(ContentType::ApplicationGraphqlResponseJson), - Some(Ok(body)), - _, - ) - | (Ok(ContentType::ApplicationJson), Some(Ok(body)), true) => { - // Application graphql json expects valid graphql response - // Application json expects valid graphql response if 2xx - tracing::debug_span!("parse_subgraph_response").in_scope(|| { - // Application graphql json expects valid graphql response - graphql::Response::from_bytes(&service, body.into()) - .unwrap_or_else(|error| { - graphql::Response::builder() - .error(error.to_graphql_error(None)) - .build() - }) - }) - } - (Ok(ContentType::ApplicationJson), Some(Ok(body)), false) => { - // Application json does not expect a valid graphql response if not 2xx. - // If parse fails then attach the entire payload as an error - tracing::debug_span!("parse_subgraph_response").in_scope(|| { - // Application graphql json expects valid graphql response - let mut original_response = - String::from_utf8_lossy(&body).to_string(); - if original_response.is_empty() { - original_response = "".into() - } - graphql::Response::from_bytes(&service, body.into()) - .unwrap_or_else(|_error| { - graphql::Response::builder() - .error( - FetchError::SubrequestMalformedResponse { - service: service.to_string(), - reason: original_response, - } - .to_graphql_error(None), - ) - .build() - }) - }) - } - (content_type, _body, _) => { - // Something went wrong, compose a response with errors if they are present - let mut graphql_response = graphql::Response::builder().build(); - if let Err(err) = content_type { - graphql_response.errors.push(err.to_graphql_error(None)); - } - /* TODO: XXX NEED TO UNDERSTAND - if let Some(Err(err)) = body { - graphql_response.errors.push(err.to_graphql_error(None)); - } - */ - graphql_response - } - }; + // Map our Vec into Bytes + // Map our serde conversion error to a FetchError + let body = Some(serde_json::to_vec(&object).map(|v| v.into()).map_err( + |error| FetchError::SubrequestMalformedResponse { + service: service.to_string(), + reason: error.to_string(), + }, + )); - // Add an error for response codes that are not 2xx - if !parts.status.is_success() { - let status = parts.status; - graphql_response.errors.insert( - 0, - FetchError::SubrequestHttpError { - service: service.to_string(), - status_code: Some(status.as_u16()), - reason: format!( - "{}: {}", - status.as_str(), - status.canonical_reason().unwrap_or("Unknown") - ), - } - .to_graphql_error(None), - ) - } + let graphql_response = http_response_to_graphql_response( + service_name, + content_type.clone(), + body, + &parts, + ); graphql_responses.push(graphql_response); } tracing::debug!("we have a vec of graphql_responses: {graphql_responses:?}"); + // Before we process our graphql responses, ensure that we have a tx for each + // response + if txs.len() != graphql_responses.len() { + return Err(Box::new(FetchError::SubrequestBatchingError { + service: service.to_string(), + reason: format!( + "number of txs ({}) is not equal to number of graphql responses ({})", + txs.len(), + graphql_responses.len() + ), + })); + } for graphql_response in graphql_responses { // Build an http Response - let resp = http::Response::builder() - .status(StatusCode::OK) - .body(graphql_response) - .expect("Response is serializable; qed"); - - // *response.headers_mut() = headers.unwrap_or_default(); - // let resp = http::Response::from_parts(parts, graphql_response); + let mut resp = http::Response::builder() + .status(parts.status) + .version(parts.version) + .body(graphql_response)?; + *resp.headers_mut() = parts.headers.clone(); tracing::debug!("we have a resp: {resp:?}"); let subgraph_response = SubgraphResponse::new_from_response(resp, context.clone()); + // We have checked before we started looping that we had a tx for every + // graphql_response, so None should be unreachable. + // Use the popped tx to send a graphql_response message to each waiter. + // We must have a tx for each response, so pop a tx and send the individual + // response to the waiter match txs.pop() { Some(tx) => { tx.send(Ok(subgraph_response)).map_err(|error| { @@ -882,16 +912,19 @@ async fn call_batched_http( })?; } None => { - tracing::debug!( - "TO PROCESS THIS BATCH WE ISSUE: {do_fetch_count} fetches" - ); - return Ok(subgraph_response); + // This can't happen since we checked that our iterator lengths + // are equal, but the compiler wants something. + unreachable!(); } } } + // Confirm that txs is now empty (i.e.: all waiters are notified) + if !txs.is_empty() { + panic!("we have remaining txs, this is a bug and should not occur"); + } } } - tracing::debug!("WE HAVE A WAITER"); + tracing::debug!("we have a waiter"); // If we get an error whilst waiting, then something bad has happened receiver .await @@ -900,7 +933,7 @@ async fn call_batched_http( reason: format!("tx receive failed: {err}"), })? } else { - tracing::debug!("WE CALLED HTTP"); + tracing::debug!("we called http"); let client = client_factory.create(service_name); call_http(request, body, context, client, service_name).await } @@ -983,72 +1016,8 @@ pub(crate) async fn call_http( } } - let mut graphql_response = match (content_type, body, parts.status.is_success()) { - (Ok(ContentType::ApplicationGraphqlResponseJson), Some(Ok(body)), _) - | (Ok(ContentType::ApplicationJson), Some(Ok(body)), true) => { - // Application graphql json expects valid graphql response - // Application json expects valid graphql response if 2xx - tracing::debug_span!("parse_subgraph_response").in_scope(|| { - // Application graphql json expects valid graphql response - graphql::Response::from_bytes(service_name, body).unwrap_or_else(|error| { - graphql::Response::builder() - .error(error.to_graphql_error(None)) - .build() - }) - }) - } - (Ok(ContentType::ApplicationJson), Some(Ok(body)), false) => { - // Application json does not expect a valid graphql response if not 2xx. - // If parse fails then attach the entire payload as an error - tracing::debug_span!("parse_subgraph_response").in_scope(|| { - // Application graphql json expects valid graphql response - let mut original_response = String::from_utf8_lossy(&body).to_string(); - if original_response.is_empty() { - original_response = "".into() - } - graphql::Response::from_bytes(service_name, body).unwrap_or_else(|_error| { - graphql::Response::builder() - .error( - FetchError::SubrequestMalformedResponse { - service: service_name.to_string(), - reason: original_response, - } - .to_graphql_error(None), - ) - .build() - }) - }) - } - (content_type, body, _) => { - // Something went wrong, compose a response with errors if they are present - let mut graphql_response = graphql::Response::builder().build(); - if let Err(err) = content_type { - graphql_response.errors.push(err.to_graphql_error(None)); - } - if let Some(Err(err)) = body { - graphql_response.errors.push(err.to_graphql_error(None)); - } - graphql_response - } - }; - - // Add an error for response codes that are not 2xx - if !parts.status.is_success() { - let status = parts.status; - graphql_response.errors.insert( - 0, - FetchError::SubrequestHttpError { - service: service_name.to_string(), - status_code: Some(status.as_u16()), - reason: format!( - "{}: {}", - status.as_str(), - status.canonical_reason().unwrap_or("Unknown") - ), - } - .to_graphql_error(None), - ) - } + let graphql_response = + http_response_to_graphql_response(service_name, content_type, body, &parts); let resp = http::Response::from_parts(parts, graphql_response); Ok(SubgraphResponse::new_from_response(resp, context)) From b19a267ceee88065af034f390a5d64d0766c1d0e Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Thu, 7 Mar 2024 09:12:47 +0000 Subject: [PATCH 18/60] add subgraph support to batching metrics Add the optional "subgraph" attribute to capture batching metrics. --- apollo-router/src/services/subgraph_service.rs | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index 2f81791283..ee50796e1a 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -41,6 +41,7 @@ use super::layers::content_negotiation::GRAPHQL_JSON_RESPONSE_HEADER_VALUE; use super::Plugins; use crate::batching::BatchQuery; use crate::batching::Waiter; +use crate::configuration::BatchingMode; use crate::configuration::TlsClientAuth; use crate::error::FetchError; use crate::graphql; @@ -817,6 +818,20 @@ async fn call_batched_http( let display_body = context.contains_key(LOGGING_DISPLAY_BODY); let client = client_factory.create(&service); + + // Update our batching metrics (just before we fetch) + tracing::info!( + histogram.apollo.router.operations.batching.size = txs.len() as f64, + mode = %BatchingMode::BatchHttpLink, // Only supported mode right now + subgraph = &service + ); + + tracing::info!( + monotonic_counter.apollo.router.operations.batching = 1u64, + mode = %BatchingMode::BatchHttpLink, // Only supported mode right now + subgraph = &service + ); + // Perform the actual fetch. If this fails then we didn't manage to make the call at all, so we can't do anything with it. let (parts, content_type, body) = do_fetch(client, &context, &service, request, display_body) From 5665508057a75c424277d03c50a0d4de5c9f8542 Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Thu, 7 Mar 2024 15:19:16 +0000 Subject: [PATCH 19/60] Add a comment about the configuration work enabling subgraph filtering We need to add support for subgraph filtering to the subgraph service, but we can't do that until the configuration work is completed. Add a comment so we don't forget about it. --- apollo-router/src/services/subgraph_service.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index ee50796e1a..c5db4bd5a2 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -740,6 +740,10 @@ async fn call_batched_http( // We'd like to park a task here, but we can't park it whilst we have the context extensions // lock held. That would be very bad... // So, we set an optional listener and wait to hear back from the batch processor + // TODO: When we have the configuration work in place, we'll need to use the configuration + // settings here to determine if batching is even possible for a service. i.e.: If subgraphs + // all is set or (if not) if a specific subgraph name (synonym for service_name here) is set. + // Make sure to pick this up as part of the configuration work. let mut batch_responder: Option< tokio::sync::oneshot::Receiver>, > = None; From a6d3be5c0f32f68c7ee36e2c519bb789d8cb7e85 Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Thu, 7 Mar 2024 15:27:54 +0000 Subject: [PATCH 20/60] Set operation name to "batch" As per decision made with Product. We may change this later, but it's a reasonable decision for now. --- apollo-router/src/batching.rs | 1 + apollo-router/src/services/subgraph_service.rs | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/apollo-router/src/batching.rs b/apollo-router/src/batching.rs index f4a6cfeb76..6ec30ca57c 100644 --- a/apollo-router/src/batching.rs +++ b/apollo-router/src/batching.rs @@ -161,6 +161,7 @@ impl Waiter { // This is ok, because when the batch was created, the parts and context were primarily created // by extracting and duplicating information from the single batch request. Maybe we should use // a different operation name, maybe chain them all together? TODO: Decide operation name + // DECISION: For now we will ignore the operation name which is extracted here. pub(crate) async fn assemble_batch( service_waiters: Vec, ) -> Result< diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index c5db4bd5a2..55b4ca982a 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -779,7 +779,7 @@ async fn call_batched_http( // We need our own batch aware version of call_http which only makes one call to each // subgraph, but is able to decode the responses. I'll probably need to break call_http // down into sub-functions, and I've started this, but it's not finished. - let (operation_name, context, mut request, mut txs) = + let (_operation_name, context, mut request, mut txs) = Waiter::assemble_batch(service_waiters).await?; request @@ -794,6 +794,7 @@ async fn call_batched_http( // TODO: We have multiple operation names but we are just using the first operation // name in the span. Should we report all operation names? + // CURRENT DECISION: hard code to "batch" let subgraph_req_span = tracing::info_span!("subgraph_request", "otel.kind" = "CLIENT", "net.peer.name" = %host, @@ -802,7 +803,7 @@ async fn call_batched_http( "http.url" = %schema_uri, "net.transport" = "ip_tcp", "apollo.subgraph.name" = %&service, - "graphql.operation.name" = %operation_name, + "graphql.operation.name" = "batch" ); // The graphql spec is lax about what strategy to use for processing responses: https://github.com/graphql/graphql-over-http/blob/main/spec/GraphQLOverHTTP.md#processing-the-response From afadbdd75bbddde997713f815ae2d0d0209aec20 Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Thu, 7 Mar 2024 16:01:28 +0000 Subject: [PATCH 21/60] make experimental_batching a commercial feature Remember that we will be renaming: experimental_batching -> batching so we'll need to rename this here as well once that configuration change lands. --- apollo-router/src/uplink/license_enforcement.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/apollo-router/src/uplink/license_enforcement.rs b/apollo-router/src/uplink/license_enforcement.rs index ab7d90aec9..de96bcf59b 100644 --- a/apollo-router/src/uplink/license_enforcement.rs +++ b/apollo-router/src/uplink/license_enforcement.rs @@ -361,6 +361,10 @@ impl LicenseEnforcementReport { .path("$.preview_file_uploads") .name("File uploads plugin") .build(), + ConfigurationRestriction::builder() + .path("$.experimental_batching") + .name("Batching support") + .build(), ] } From 1fd302097166469c0592075e53f8bf99e5afe17a Mon Sep 17 00:00:00 2001 From: Nicholas Cioli Date: Mon, 11 Mar 2024 05:06:25 -0400 Subject: [PATCH 22/60] Add config for router-subgraph batching (#4777) This commit adds configuration options for later use in the router to subgraph epic. More changes will be necessary to actually use these new configuration changes. --- **Checklist** Complete the checklist (and note appropriate exceptions) before the PR is marked ready-for-review. - [ ] Changes are compatible[^1] - [ ] Documentation[^2] completed - [ ] Performance impact assessed and acceptable - Tests added and passing[^3] - [ ] Unit Tests - [ ] Integration Tests - [ ] Manual Tests **Exceptions** *Note any exceptions here* **Notes** [^1]: It may be appropriate to bring upcoming changes to the attention of other (impacted) groups. Please endeavour to do this before seeking PR approval. The mechanism for doing this will vary considerably, so use your judgement as to how and when to do this. [^2]: Configuration is an important part of many changes. Where applicable please try to document configuration examples. [^3]: Tick whichever testing boxes are applicable. If you are adding Manual Tests, please document the manual testing (extensively) in the Exceptions. --- apollo-router/feature_discussions.json | 5 +- apollo-router/src/configuration/metrics.rs | 2 +- apollo-router/src/configuration/mod.rs | 39 ++++-- ...nfiguration__tests__schema_generation.snap | 118 +++++++++++++----- .../testdata/metrics/batching.router.yaml | 2 +- apollo-router/src/services/router/service.rs | 42 +++---- apollo-router/src/services/router/tests.rs | 8 +- .../src/uplink/license_enforcement.rs | 2 +- .../fixtures/apollo_reports_batch.router.yaml | 4 +- ...ecycle_tests__cli_config_experimental.snap | 2 - docs/source/configuration/traffic-shaping.mdx | 6 +- .../executing-operations/query-batching.mdx | 28 +++-- 12 files changed, 170 insertions(+), 88 deletions(-) diff --git a/apollo-router/feature_discussions.json b/apollo-router/feature_discussions.json index 59f5a84608..446162650a 100644 --- a/apollo-router/feature_discussions.json +++ b/apollo-router/feature_discussions.json @@ -2,10 +2,9 @@ "experimental": { "experimental_retry": "https://github.com/apollographql/router/discussions/2241", "experimental_response_trace_id": "https://github.com/apollographql/router/discussions/2147", - "experimental_when_header": "https://github.com/apollographql/router/discussions/1961", - "experimental_batching": "https://github.com/apollographql/router/discussions/3840" + "experimental_when_header": "https://github.com/apollographql/router/discussions/1961" }, "preview": { "preview_entity_cache": "https://github.com/apollographql/router/discussions/4592" } -} \ No newline at end of file +} diff --git a/apollo-router/src/configuration/metrics.rs b/apollo-router/src/configuration/metrics.rs index 8d42c9b1ef..59f72112a4 100644 --- a/apollo-router/src/configuration/metrics.rs +++ b/apollo-router/src/configuration/metrics.rs @@ -325,7 +325,7 @@ impl InstrumentData { populate_config_instrument!( apollo.router.config.batching, - "$.experimental_batching[?(@.enabled == true)]", + "$.batching[?(@.enabled == true)]", opt.mode, "$.mode" ); diff --git a/apollo-router/src/configuration/mod.rs b/apollo-router/src/configuration/mod.rs index 536b529b13..6a55090af8 100644 --- a/apollo-router/src/configuration/mod.rs +++ b/apollo-router/src/configuration/mod.rs @@ -11,6 +11,7 @@ mod tests; mod upgrade; mod yaml; +use std::collections::HashMap; use std::fmt; use std::io; use std::io::BufReader; @@ -190,7 +191,7 @@ pub struct Configuration { /// Batching configuration. #[serde(default)] - pub(crate) experimental_batching: Batching, + pub(crate) batching: Batching, } impl PartialEq for Configuration { @@ -255,7 +256,7 @@ impl<'de> serde::Deserialize<'de> for Configuration { limits: Limits, experimental_chaos: Chaos, experimental_graphql_validation_mode: GraphQLValidationMode, - experimental_batching: Batching, + batching: Batching, } let ad_hoc: AdHocConfiguration = serde::Deserialize::deserialize(deserializer)?; @@ -274,7 +275,7 @@ impl<'de> serde::Deserialize<'de> for Configuration { .chaos(ad_hoc.experimental_chaos) .uplink(ad_hoc.uplink) .graphql_validation_mode(ad_hoc.experimental_graphql_validation_mode) - .experimental_batching(ad_hoc.experimental_batching) + .batching(ad_hoc.batching) .build() .map_err(|e| serde::de::Error::custom(e.to_string())) } @@ -312,7 +313,7 @@ impl Configuration { uplink: Option, graphql_validation_mode: Option, experimental_api_schema_generation_mode: Option, - experimental_batching: Option, + batching: Option, ) -> Result { #[cfg(not(test))] let notify_queue_cap = match apollo_plugins.get(APOLLO_SUBSCRIPTION_PLUGIN_NAME) { @@ -348,7 +349,7 @@ impl Configuration { }, tls: tls.unwrap_or_default(), uplink, - experimental_batching: experimental_batching.unwrap_or_default(), + batching: batching.unwrap_or_default(), #[cfg(test)] notify: notify.unwrap_or_default(), #[cfg(not(test))] @@ -387,7 +388,7 @@ impl Configuration { chaos: Option, uplink: Option, graphql_validation_mode: Option, - experimental_batching: Option, + batching: Option, experimental_api_schema_generation_mode: Option, ) -> Result { let configuration = Self { @@ -413,7 +414,7 @@ impl Configuration { apq: apq.unwrap_or_default(), persisted_queries: persisted_query.unwrap_or_default(), uplink, - experimental_batching: experimental_batching.unwrap_or_default(), + batching: batching.unwrap_or_default(), }; configuration.validate() @@ -1391,4 +1392,28 @@ pub(crate) struct Batching { /// Batching mode pub(crate) mode: BatchingMode, + + /// Subgraph options for batching + /// + /// Note: Batching from the router to subgraphs can either be enabled for + /// all subgraphs or configured per subgraph, but not both. + pub(crate) subgraph: Option, +} + +/// Batching configuration for subgraphs +#[derive(Debug, Clone, Deserialize, Serialize, JsonSchema)] +#[serde(deny_unknown_fields, rename_all = "lowercase")] +pub(crate) enum SubgraphBatchingConfig { + /// Batching options for all known subgraphs + All(CommonBatchingConfig), + + /// Per-subgraph batching options + Subgraphs(HashMap), +} + +/// Common options for configuring subgraph batching +#[derive(Debug, Clone, Default, Deserialize, Serialize, JsonSchema)] +pub(crate) struct CommonBatchingConfig { + /// Whether this batching config should be enabled + enabled: bool, } diff --git a/apollo-router/src/configuration/snapshots/apollo_router__configuration__tests__schema_generation.snap b/apollo-router/src/configuration/snapshots/apollo_router__configuration__tests__schema_generation.snap index 46e3461afb..1e8c339bee 100644 --- a/apollo-router/src/configuration/snapshots/apollo_router__configuration__tests__schema_generation.snap +++ b/apollo-router/src/configuration/snapshots/apollo_router__configuration__tests__schema_generation.snap @@ -661,6 +661,93 @@ expression: "&schema" } } }, + "batching": { + "description": "Batching configuration.", + "default": { + "enabled": false, + "mode": "batch_http_link", + "subgraph": null + }, + "type": "object", + "required": [ + "mode" + ], + "properties": { + "enabled": { + "description": "Activates Batching (disabled by default)", + "default": false, + "type": "boolean" + }, + "mode": { + "description": "Batching mode", + "oneOf": [ + { + "description": "batch_http_link", + "type": "string", + "enum": [ + "batch_http_link" + ] + } + ] + }, + "subgraph": { + "description": "Subgraph options for batching\n\nNote: Batching from the router to subgraphs can either be enabled for all subgraphs or configured per subgraph, but not both.", + "oneOf": [ + { + "description": "Batching options for all known subgraphs", + "type": "object", + "required": [ + "all" + ], + "properties": { + "all": { + "description": "Common options for configuring subgraph batching", + "type": "object", + "required": [ + "enabled" + ], + "properties": { + "enabled": { + "description": "Whether this batching config should be enabled", + "type": "boolean" + } + } + } + }, + "additionalProperties": false + }, + { + "description": "Per-subgraph batching options", + "type": "object", + "required": [ + "subgraphs" + ], + "properties": { + "subgraphs": { + "type": "object", + "additionalProperties": { + "description": "Common options for configuring subgraph batching", + "type": "object", + "required": [ + "enabled" + ], + "properties": { + "enabled": { + "description": "Whether this batching config should be enabled", + "type": "boolean" + } + } + } + } + }, + "additionalProperties": false + } + ], + "nullable": true + } + }, + "additionalProperties": false + }, "coprocessor": { "description": "Configures the externalization plugin", "type": "object", @@ -1267,37 +1354,6 @@ expression: "&schema" } ] }, - "experimental_batching": { - "description": "Batching configuration.", - "default": { - "enabled": false, - "mode": "batch_http_link" - }, - "type": "object", - "required": [ - "mode" - ], - "properties": { - "enabled": { - "description": "Activates Batching (disabled by default)", - "default": false, - "type": "boolean" - }, - "mode": { - "description": "Batching mode", - "oneOf": [ - { - "description": "batch_http_link", - "type": "string", - "enum": [ - "batch_http_link" - ] - } - ] - } - }, - "additionalProperties": false - }, "experimental_chaos": { "description": "Configuration for chaos testing, trying to reproduce bugs that require uncommon conditions. You probably don’t want this in production!", "default": { diff --git a/apollo-router/src/configuration/testdata/metrics/batching.router.yaml b/apollo-router/src/configuration/testdata/metrics/batching.router.yaml index c177d3f45e..169f3824a9 100644 --- a/apollo-router/src/configuration/testdata/metrics/batching.router.yaml +++ b/apollo-router/src/configuration/testdata/metrics/batching.router.yaml @@ -1,3 +1,3 @@ -experimental_batching: +batching: enabled: true mode: batch_http_link diff --git a/apollo-router/src/services/router/service.rs b/apollo-router/src/services/router/service.rs index b7758a4eb6..2d7ddaac4b 100644 --- a/apollo-router/src/services/router/service.rs +++ b/apollo-router/src/services/router/service.rs @@ -92,7 +92,7 @@ pub(crate) struct RouterService { persisted_query_layer: Arc, query_analysis_layer: QueryAnalysisLayer, http_max_request_bytes: usize, - experimental_batching: Batching, + batching: Batching, } impl RouterService { @@ -102,7 +102,7 @@ impl RouterService { persisted_query_layer: Arc, query_analysis_layer: QueryAnalysisLayer, http_max_request_bytes: usize, - experimental_batching: Batching, + batching: Batching, ) -> Self { RouterService { supergraph_creator, @@ -110,7 +110,7 @@ impl RouterService { persisted_query_layer, query_analysis_layer, http_max_request_bytes, - experimental_batching, + batching, } } } @@ -444,8 +444,8 @@ impl RouterService { Err(err) => { // It may be a batch of requests, so try that (if config allows) before // erroring out - if self.experimental_batching.enabled - && matches!(self.experimental_batching.mode, BatchingMode::BatchHttpLink) + if self.batching.enabled + && matches!(self.batching.mode, BatchingMode::BatchHttpLink) { result = graphql::Request::batch_from_urlencoded_query(q.to_string()) .map_err(|e| TranslateError { @@ -457,9 +457,9 @@ impl RouterService { ), })?; } else if !q.is_empty() && q.as_bytes()[0] == b'[' { - let extension_details = if self.experimental_batching.enabled - && !matches!(self.experimental_batching.mode, BatchingMode::BatchHttpLink) { - format!("batching not supported for mode `{}`", self.experimental_batching.mode) + let extension_details = if self.batching.enabled + && !matches!(self.batching.mode, BatchingMode::BatchHttpLink) { + format!("batching not supported for mode `{}`", self.batching.mode) } else { "batching not enabled".to_string() }; @@ -503,8 +503,8 @@ impl RouterService { result.push(request); } Err(err) => { - if self.experimental_batching.enabled - && matches!(self.experimental_batching.mode, BatchingMode::BatchHttpLink) + if self.batching.enabled + && matches!(self.batching.mode, BatchingMode::BatchHttpLink) { result = graphql::Request::batch_from_bytes(bytes).map_err(|e| TranslateError { @@ -516,13 +516,10 @@ impl RouterService { ), })?; } else if !bytes.is_empty() && bytes[0] == b'[' { - let extension_details = if self.experimental_batching.enabled - && !matches!(self.experimental_batching.mode, BatchingMode::BatchHttpLink) + let extension_details = if self.batching.enabled + && !matches!(self.batching.mode, BatchingMode::BatchHttpLink) { - format!( - "batching not supported for mode `{}`", - self.experimental_batching.mode - ) + format!("batching not supported for mode `{}`", self.batching.mode) } else { "batching not enabled".to_string() }; @@ -613,10 +610,7 @@ impl RouterService { let batch_size = ok_results.len(); let shared_batch_details: Option>> = if ok_results.len() > 1 { - context - .extensions() - .lock() - .insert(self.experimental_batching.clone()); + context.extensions().lock().insert(self.batching.clone()); Some(Arc::new(Mutex::new(Batch::new(batch_size)))) } else { None @@ -661,7 +655,7 @@ impl RouterService { if let Some(client_request_accepts) = client_request_accepts_opt { new_context_guard.insert(client_request_accepts); } - new_context_guard.insert(self.experimental_batching.clone()); + new_context_guard.insert(self.batching.clone()); if let Some(shared_batch_details) = &shared_batch_details { new_context_guard .insert(BatchQuery::new(index + 1, shared_batch_details.clone())); @@ -717,7 +711,7 @@ pub(crate) struct RouterCreator { pub(crate) persisted_query_layer: Arc, query_analysis_layer: QueryAnalysisLayer, http_max_request_bytes: usize, - experimental_batching: Batching, + batching: Batching, } impl ServiceFactory for RouterCreator { @@ -768,7 +762,7 @@ impl RouterCreator { query_analysis_layer, http_max_request_bytes: configuration.limits.http_max_request_bytes, persisted_query_layer, - experimental_batching: configuration.experimental_batching.clone(), + batching: configuration.batching.clone(), }) } @@ -786,7 +780,7 @@ impl RouterCreator { self.persisted_query_layer.clone(), self.query_analysis_layer.clone(), self.http_max_request_bytes, - self.experimental_batching.clone(), + self.batching.clone(), )); ServiceBuilder::new() diff --git a/apollo-router/src/services/router/tests.rs b/apollo-router/src/services/router/tests.rs index 884b5d0c3a..3e58e50b44 100644 --- a/apollo-router/src/services/router/tests.rs +++ b/apollo-router/src/services/router/tests.rs @@ -305,7 +305,7 @@ async fn it_processes_a_valid_query_batch() { hyper::Body::from(result) }); let config = serde_json::json!({ - "experimental_batching": { + "batching": { "enabled": true, "mode" : "batch_http_link" } @@ -394,7 +394,7 @@ async fn it_will_not_process_a_poorly_formatted_query_batch() { hyper::Body::from(result) }); let config = serde_json::json!({ - "experimental_batching": { + "batching": { "enabled": true, "mode" : "batch_http_link" } @@ -448,7 +448,7 @@ async fn it_will_process_a_non_batched_defered_query() { hyper::Body::from(bytes) }); let config = serde_json::json!({ - "experimental_batching": { + "batching": { "enabled": true, "mode" : "batch_http_link" } @@ -508,7 +508,7 @@ async fn it_will_not_process_a_batched_deferred_query() { hyper::Body::from(result) }); let config = serde_json::json!({ - "experimental_batching": { + "batching": { "enabled": true, "mode" : "batch_http_link" } diff --git a/apollo-router/src/uplink/license_enforcement.rs b/apollo-router/src/uplink/license_enforcement.rs index de96bcf59b..1f150b208d 100644 --- a/apollo-router/src/uplink/license_enforcement.rs +++ b/apollo-router/src/uplink/license_enforcement.rs @@ -362,7 +362,7 @@ impl LicenseEnforcementReport { .name("File uploads plugin") .build(), ConfigurationRestriction::builder() - .path("$.experimental_batching") + .path("$.batching") .name("Batching support") .build(), ] diff --git a/apollo-router/tests/fixtures/apollo_reports_batch.router.yaml b/apollo-router/tests/fixtures/apollo_reports_batch.router.yaml index fdbb1ed4dd..238b8a00dd 100644 --- a/apollo-router/tests/fixtures/apollo_reports_batch.router.yaml +++ b/apollo-router/tests/fixtures/apollo_reports_batch.router.yaml @@ -1,4 +1,4 @@ -experimental_batching: +batching: enabled: true mode: batch_http_link rhai: @@ -28,5 +28,3 @@ telemetry: send_variable_values: only: - "sendValue" - - diff --git a/apollo-router/tests/snapshots/lifecycle_tests__cli_config_experimental.snap b/apollo-router/tests/snapshots/lifecycle_tests__cli_config_experimental.snap index 882345bcb4..e5da3688af 100644 --- a/apollo-router/tests/snapshots/lifecycle_tests__cli_config_experimental.snap +++ b/apollo-router/tests/snapshots/lifecycle_tests__cli_config_experimental.snap @@ -9,8 +9,6 @@ stderr: stdout: List of all experimental configurations with related GitHub discussions: - - experimental_batching: https://github.com/apollographql/router/discussions/3840 - experimental_response_trace_id: https://github.com/apollographql/router/discussions/2147 - experimental_retry: https://github.com/apollographql/router/discussions/2241 - experimental_when_header: https://github.com/apollographql/router/discussions/1961 - diff --git a/docs/source/configuration/traffic-shaping.mdx b/docs/source/configuration/traffic-shaping.mdx index f227a73f3c..5f2a7c8d98 100644 --- a/docs/source/configuration/traffic-shaping.mdx +++ b/docs/source/configuration/traffic-shaping.mdx @@ -64,7 +64,7 @@ You can change the default timeout for client requests to the router like so: ```yaml title="router.yaml" traffic_shaping: - router: + router: timeout: 50s # If client requests to the router take more than 50 seconds, cancel the request (30 seconds by default) ``` @@ -72,7 +72,7 @@ You can change the default timeout for all requests between the router and subgr ```yaml title="router.yaml" traffic_shaping: - all: + all: timeout: 50s # If subgraph requests take more than 50 seconds, cancel the request (30 seconds by default) ``` @@ -91,7 +91,7 @@ Compression is automatically supported on the client side, depending on the `Acc The Apollo Router has _experimental_ support for receiving client query batches: ```yaml title="router.yaml" -experimental_batching: +batching: enabled: true mode: batch_http_link ``` diff --git a/docs/source/executing-operations/query-batching.mdx b/docs/source/executing-operations/query-batching.mdx index e4c59bf129..d040905b47 100644 --- a/docs/source/executing-operations/query-batching.mdx +++ b/docs/source/executing-operations/query-batching.mdx @@ -1,6 +1,6 @@ --- title: Query batching -description: Receive query batches with the Apollo Router +description: Receive query batches with the Apollo Router --- @@ -15,7 +15,7 @@ The Apollo Router supports client and subgraph query batching. If you’re using Apollo Client, you can leverage the built-in support for batching to reduce the number of individual operations sent to the router. -Once configured, Apollo Client automatically combines multiple operations into a single HTTP request. The number of operations within a batch is client-configurable, including the maximum number in a batch and the maximum duration to wait for operations to accumulate before sending the batch. +Once configured, Apollo Client automatically combines multiple operations into a single HTTP request. The number of operations within a batch is client-configurable, including the maximum number in a batch and the maximum duration to wait for operations to accumulate before sending the batch. The Apollo Router must be configured to receive query batches, otherwise it rejects them. When processing a batch, the router deserializes and processes each operation of a batch independently, and it responds to the client only after all operations of the batch have been completed. Each operation executes concurrently with respect to other operations in the batch. @@ -27,12 +27,12 @@ Both the Apollo Router and client need to be configured to support query batchin #### Client Query Batching -By default, receiving client query batches is _not_ enabled in the Apollo Router. +By default, receiving client query batches is _not_ enabled in the Apollo Router. To enable query batching, set the following fields in your `router.yaml` configuration file: ```yaml title="router.yaml" -experimental_batching: +batching: enabled: true mode: batch_http_link ``` @@ -46,13 +46,22 @@ experimental_batching: If Client query batching is enabled, and the router subgraphs [support query batching](https://www.apollographql.com/docs/apollo-server/api/apollo-server#allowbatchedhttprequests), then subgraph query batching may be enabled by setting the following fields in your `router.yaml` configuration file: -```yaml title="router.yaml" -experimental_batching: +```yaml title="router.all_enabled.yaml" +batching: enabled: true mode: batch_http_link subgraph: + # Enable batching on all subgraphs all: enabled: true +``` + +```yaml title="router.yaml" +batching: + enabled: true + mode: batch_http_link + subgraph: + # Configure batching support per subgraph subgraphs: subgraph_1: enabled: true @@ -60,6 +69,9 @@ experimental_batching: enabled: true ``` + +The router can be configured to either support batching for all subgraphs or configured per each subgraph, but not both. + There are limitations on the ability of the router to preserve batches from the client request into the subgraph requests. In particular, certain forms of queries will require data to be present before they are processed. Bearing this constraint in mind, the router will only be able to generate batches from queries which are processed which don't contain such constraints. @@ -276,7 +288,7 @@ As a result, the router returns an invalid batch error: ### Individual query error -If a single query in a batch cannot be processed, this results in an individual error. +If a single query in a batch cannot be processed, this results in an individual error. For example, the query `MyFirstQuery` is accessing a field that doesn't exist, while the rest of the batch query is valid. @@ -313,7 +325,7 @@ As a result, an error is returned for the individual invalid query and the other ## Known limitations ### Unsupported query modes - + When batching is enabled, any batch operation that results in a stream of responses is unsupported, including: - [`@defer`](/graphos/operations/defer/) - [subscriptions](/graphos/operations/subscriptions/) From 9743e8ebd4dc76c3ea6e5529933d71e002d77999 Mon Sep 17 00:00:00 2001 From: Gary Pennington Date: Tue, 12 Mar 2024 08:19:55 +0000 Subject: [PATCH 23/60] [Batching] Build upon the recent configuration changes to enable subgraph filtering (#4780) Now that we have Batching subgraph configuration we can use it to control filtering by batching enabled subgraphs. also: - Add a migration from `experimental_batching` -> `batching` - Add subgraph filtering utility function to batching configuration - Add some subgraph filtering unit tests - merge dev Note: @nicholascioli I merged in `dev`, so for reviewing purposes, just look at [this commit](https://github.com/apollographql/router/commit/984d2c60d67c2fec9635ea5d50f3ddb335307e13). --- .changesets/feat_geal_entity_cache_span.md | 5 - .changesets/feat_geal_jwks_headers.md | 5 + .changesets/feat_geal_jwt_source.md | 5 + .changesets/feat_geal_redis_fail_open.md | 7 - .changesets/feat_glasser_pq_gzip.md | 5 - .changesets/fix_bnjjj_fix_4699.md | 15 - .changesets/fix_bryn_datadog_trace_id.md | 17 + .changesets/fix_geal_non_blocking_stdout.md | 9 - ...fix_geal_subgraph_streaming_compression.md | 5 - .circleci/config.yml | 67 ++-- .gitleaks.toml | 1 + CHANGELOG.md | 69 ++++ Cargo.lock | 17 +- apollo-router-benchmarks/Cargo.toml | 2 +- apollo-router-scaffold/Cargo.toml | 2 +- .../templates/base/Cargo.toml | 2 +- .../templates/base/xtask/Cargo.toml | 2 +- apollo-router/Cargo.toml | 3 +- .../axum_factory/axum_http_server_factory.rs | 47 ++- apollo-router/src/axum_factory/utils.rs | 88 ----- .../migrations/0023-batching.yaml | 5 + apollo-router/src/configuration/mod.rs | 17 +- ...nfiguration__tests__schema_generation.snap | 76 ++++ ...grade_old_configuration@batching.yaml.snap | 8 + .../testdata/migrations/batching.yaml | 3 + apollo-router/src/configuration/tests.rs | 74 ++++ apollo-router/src/executable.rs | 29 -- .../src/plugins/authentication/jwks.rs | 20 +- .../src/plugins/authentication/mod.rs | 231 +++++++---- .../src/plugins/authentication/tests.rs | 269 ++++++++++++- .../src/plugins/telemetry/fmt_layer.rs | 8 +- .../src/plugins/telemetry/formatters/json.rs | 118 ++++++ apollo-router/src/services/http/service.rs | 7 + .../services/layers/content_negotiation.rs | 11 + apollo-router/src/services/router/service.rs | 26 +- .../src/services/subgraph_service.rs | 61 +-- apollo-router/tests/common.rs | 150 ++++++-- .../file_upload/multiple_subgraph.graphql | 122 ------ ...single_subgraph.graphql => schema.graphql} | 77 ++-- .../tests/integration/file_upload.rs | 364 +++++++++++++----- apollo-router/tests/lifecycle_tests.rs | 2 +- apollo-router/tests/telemetry/metrics.rs | 5 +- .../tracing/docker-compose.datadog.yml | 2 +- dockerfiles/tracing/docker-compose.jaeger.yml | 2 +- dockerfiles/tracing/docker-compose.zipkin.yml | 2 +- docs/shared/redis-tls.mdx | 16 + docs/source/configuration/authn-jwt.mdx | 42 +- docs/source/configuration/authorization.mdx | 40 +- .../configuration/distributed-caching.mdx | 6 +- docs/source/configuration/overview.mdx | 6 + .../telemetry/exporters/metrics/overview.mdx | 2 +- .../telemetry/exporters/tracing/datadog.mdx | 16 + docs/source/customizations/native.mdx | 18 - helm/chart/router/Chart.yaml | 4 +- helm/chart/router/README.md | 6 +- licenses.html | 118 +----- scripts/install.sh | 2 +- 57 files changed, 1574 insertions(+), 764 deletions(-) delete mode 100644 .changesets/feat_geal_entity_cache_span.md create mode 100644 .changesets/feat_geal_jwks_headers.md create mode 100644 .changesets/feat_geal_jwt_source.md delete mode 100644 .changesets/feat_geal_redis_fail_open.md delete mode 100644 .changesets/feat_glasser_pq_gzip.md delete mode 100644 .changesets/fix_bnjjj_fix_4699.md create mode 100644 .changesets/fix_bryn_datadog_trace_id.md delete mode 100644 .changesets/fix_geal_non_blocking_stdout.md delete mode 100644 .changesets/fix_geal_subgraph_streaming_compression.md create mode 100644 apollo-router/src/configuration/migrations/0023-batching.yaml create mode 100644 apollo-router/src/configuration/snapshots/apollo_router__configuration__tests__upgrade_old_configuration@batching.yaml.snap create mode 100644 apollo-router/src/configuration/testdata/migrations/batching.yaml delete mode 100644 apollo-router/tests/fixtures/file_upload/multiple_subgraph.graphql rename apollo-router/tests/fixtures/file_upload/{single_subgraph.graphql => schema.graphql} (50%) create mode 100644 docs/shared/redis-tls.mdx diff --git a/.changesets/feat_geal_entity_cache_span.md b/.changesets/feat_geal_entity_cache_span.md deleted file mode 100644 index 124035cbc2..0000000000 --- a/.changesets/feat_geal_entity_cache_span.md +++ /dev/null @@ -1,5 +0,0 @@ -### Entity cache: Add spans around redis interaction ([PR #4667](https://github.com/apollographql/router/pull/4667)) - -This adds the `cache_lookup` and `cache_store` spans to show the entity cache's Redis calls in traces. This also changes the behavior slightly so that storing in Redis does not stop the execution of the rest of the query - -By [@Geal](https://github.com/Geal) in https://github.com/apollographql/router/pull/4667 \ No newline at end of file diff --git a/.changesets/feat_geal_jwks_headers.md b/.changesets/feat_geal_jwks_headers.md new file mode 100644 index 0000000000..f0557d3749 --- /dev/null +++ b/.changesets/feat_geal_jwks_headers.md @@ -0,0 +1,5 @@ +### Add headers to the JWKS download request ([Issue #4651](https://github.com/apollographql/router/issues/4651)) + +This adds the ability to set static headers on HTTP requests to download a JWKS from an identity provider + +By [@Geal](https://github.com/Geal) in https://github.com/apollographql/router/pull/4688 \ No newline at end of file diff --git a/.changesets/feat_geal_jwt_source.md b/.changesets/feat_geal_jwt_source.md new file mode 100644 index 0000000000..96f509a215 --- /dev/null +++ b/.changesets/feat_geal_jwt_source.md @@ -0,0 +1,5 @@ +### Support loading JWT from other sources ([PR #4711](https://github.com/apollographql/router/pull/4711)) + +The token can be stored in different headers, but it can also be carried by a cookie. This adds cookies as an alternative source of tokens, and allows multiple alternative sources + +By [@Geal](https://github.com/Geal) in https://github.com/apollographql/router/pull/4711 \ No newline at end of file diff --git a/.changesets/feat_geal_redis_fail_open.md b/.changesets/feat_geal_redis_fail_open.md deleted file mode 100644 index 175b68b4e4..0000000000 --- a/.changesets/feat_geal_redis_fail_open.md +++ /dev/null @@ -1,7 +0,0 @@ -### Redis: add a fail open option ([Issue #4334](https://github.com/apollographql/router/issues/4334)) - -This option configures the Router's behavior in case it cannot connect to Redis: -- by default, it will still start, so requests will still be handled in a degraded state -- when active, that option will prevent the router from starting if it cannot connect - -By [@Geal](https://github.com/Geal) in https://github.com/apollographql/router/pull/4534 \ No newline at end of file diff --git a/.changesets/feat_glasser_pq_gzip.md b/.changesets/feat_glasser_pq_gzip.md deleted file mode 100644 index 2265ab4d3c..0000000000 --- a/.changesets/feat_glasser_pq_gzip.md +++ /dev/null @@ -1,5 +0,0 @@ -### Use gzip compression when downloading Persisted Query manifests - -Router will now request gzip compression when downloading Persisted Query manifests for improved network efficiency. - -By [@glasser](https://github.com/glasser) in https://github.com/apollographql/router/pull/4622 \ No newline at end of file diff --git a/.changesets/fix_bnjjj_fix_4699.md b/.changesets/fix_bnjjj_fix_4699.md deleted file mode 100644 index a54c60b158..0000000000 --- a/.changesets/fix_bnjjj_fix_4699.md +++ /dev/null @@ -1,15 +0,0 @@ -### Default header correctly set in `experimental_response_trace_id` when enabled ([Issue #4699](https://github.com/apollographql/router/issues/4699)) - -When configuring the `experimental_response_trace_id` without an explicit header it now correctly takes the default one `apollo-trace-id`. - -Example of configuration: - -```yaml -telemetry: - exporters: - tracing: - experimental_response_trace_id: - enabled: true -``` - -By [@bnjjj](https://github.com/bnjjj) in https://github.com/apollographql/router/pull/4702 \ No newline at end of file diff --git a/.changesets/fix_bryn_datadog_trace_id.md b/.changesets/fix_bryn_datadog_trace_id.md new file mode 100644 index 0000000000..e26cdb3b02 --- /dev/null +++ b/.changesets/fix_bryn_datadog_trace_id.md @@ -0,0 +1,17 @@ +### Attach `dd.trace_id` to JSON formatted log messages ([PR #4764](https://github.com/apollographql/router/pull/4764)) + +To enable correlation between DataDog tracing and logs, `dd.trace_id` must appear as a span attribute on the root of each JSON formatted log message. +Once you configure the `dd.trace_id` attribute in router.yaml, it will automatically be extracted from the root span and attached to the logs: + +```yaml title="router.yaml" +telemetry: + instrumentation: + spans: + mode: spec_compliant + router: + attributes: + dd.trace_id: true +``` + + +By [@BrynCooke](https://github.com/BrynCooke) in https://github.com/apollographql/router/pull/4764 diff --git a/.changesets/fix_geal_non_blocking_stdout.md b/.changesets/fix_geal_non_blocking_stdout.md deleted file mode 100644 index d3249c208b..0000000000 --- a/.changesets/fix_geal_non_blocking_stdout.md +++ /dev/null @@ -1,9 +0,0 @@ -### use a non blocking stdout and stderr ([Issue #4612](https://github.com/apollographql/router/issues/4612)) - -If the router's output was piped into another process, and that process did not consume that output, it could entirely lock up the router. New connections were accepted, but requests never got an answer. -This is due to Rust protecting stdout and stderr access by a lock, to prevent multiple threads from interleaving their writes. When the process receiving the output from the router does not consume, then the logger's writes to the stream start to block, which means the current thread is blocked while holding the lock. And then any other thread that might want to log something will end up blocked too, waiting for that lock to be released. - -This is fixed by marking stdout and stderr as non blocking, which means that logs will be dropped silently when the buffer is full. This has another side effect that should be pointed out: -**if we write to stdout or sdtderr directly without handling errors (example: using `println!` or `eprintln!`) while the output is not consumed, then the router will panic. While that may look concerning, we consider that panicking, which will immediately reject the in flight requests and may trigger a restart of the router, is a better outcome than the router amking requests hang indefinitely.** - -By [@Geal](https://github.com/Geal) in https://github.com/apollographql/router/pull/4625 \ No newline at end of file diff --git a/.changesets/fix_geal_subgraph_streaming_compression.md b/.changesets/fix_geal_subgraph_streaming_compression.md deleted file mode 100644 index a1814fb673..0000000000 --- a/.changesets/fix_geal_subgraph_streaming_compression.md +++ /dev/null @@ -1,5 +0,0 @@ -### Implement streaming compression for subgraph requests ([Issue #4648](https://github.com/apollographql/router/issues/4648)) - -This fixe subgraph HTTP requests to compress the body in streaming instead of loading it entirely in the compression engine before sending everything at once. This reuses the compression layer that the router uses to compress client responses. - -By [@Geal](https://github.com/Geal) in https://github.com/apollographql/router/pull/4672 \ No newline at end of file diff --git a/.circleci/config.yml b/.circleci/config.yml index ed770fe96c..89daf9d9f0 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -42,13 +42,20 @@ executors: environment: CARGO_BUILD_JOBS: 8 RUST_TEST_THREADS: 8 - macos_build: &macos_build_executor + arm_macos_build: &arm_macos_build_executor macos: # See https://circleci.com/docs/xcode-policy along with the support matrix # at https://circleci.com/docs/using-macos#supported-xcode-versions. # We use the major.minor notation to bring in compatible patches. xcode: 14.2 resource_class: macos.m1.medium.gen1 + intel_macos_build: &intel_macos_build_executor + macos: + # See https://circleci.com/docs/xcode-policy along with the support matrix + # at https://circleci.com/docs/using-macos#supported-xcode-versions. + # We use the major.minor notation to bring in compatible patches. + xcode: 14.2 + resource_class: macos.x86.medium.gen2 macos_test: &macos_test_executor macos: # See https://circleci.com/docs/xcode-policy along with the support matrix @@ -159,7 +166,7 @@ commands: - when: condition: or: - - equal: [ *macos_build_executor, << parameters.platform >> ] + - equal: [ *arm_macos_build_executor, << parameters.platform >> ] - equal: [ *macos_test_executor, << parameters.platform >> ] steps: - run: @@ -170,7 +177,20 @@ commands: - run: name: Write arch command: | - echo 'osx' >> ~/.arch + echo 'osx-aarch64' >> ~/.arch + - when: + condition: + equal: [ *intel_macos_build_executor, << parameters.platform >> ] + steps: + - run: + name: Make link to md5 + command: | + mkdir -p ~/.local/aliases + ln -s /sbin/md5 ~/.local/aliases/md5sum + - run: + name: Write arch + command: | + echo 'osx-x86' >> ~/.arch - when: condition: or: @@ -243,7 +263,8 @@ commands: - when: condition: or: - - equal: [ *macos_build_executor, << parameters.platform >> ] + - equal: [ *intel_macos_build_executor, << parameters.platform >> ] + - equal: [ *arm_macos_build_executor, << parameters.platform >> ] - equal: [ *macos_test_executor, << parameters.platform >> ] steps: - run: @@ -284,7 +305,8 @@ commands: - when: condition: or: - - equal: [ *macos_build_executor, << parameters.platform >> ] + - equal: [ *intel_macos_build_executor, << parameters.platform >> ] + - equal: [ *arm_macos_build_executor, << parameters.platform >> ] - equal: [ *macos_test_executor, << parameters.platform >> ] steps: - run: @@ -333,15 +355,6 @@ commands: name: Special case for Windows because of ssh-agent command: | printf "[net]\ngit-fetch-with-cli = true" >> ~/.cargo/Cargo.toml - - when: - condition: - or: - - equal: [ *macos_build_executor, << parameters.platform >> ] - steps: - - run: - name: Special case for OSX x86_64 builds - command: | - rustup target add x86_64-apple-darwin install_extra_tools: steps: @@ -593,7 +606,10 @@ jobs: platform: << parameters.platform >> - when: condition: - equal: [ *macos_build_executor, << parameters.platform >> ] + or: + - equal: [ *intel_macos_build_executor, << parameters.platform >> ] + - equal: [ *arm_macos_build_executor, << parameters.platform >> ] + steps: - when: condition: @@ -602,28 +618,13 @@ jobs: - run: cargo xtask release prepare nightly - run: command: > - cargo xtask dist --target aarch64-apple-darwin - - run: - command: > - cargo xtask dist --target x86_64-apple-darwin + cargo xtask dist - run: command: > mkdir -p artifacts - run: command: > cargo xtask package - --target aarch64-apple-darwin - --apple-team-id ${APPLE_TEAM_ID} - --apple-username ${APPLE_USERNAME} - --cert-bundle-base64 ${MACOS_CERT_BUNDLE_BASE64} - --cert-bundle-password ${MACOS_CERT_BUNDLE_PASSWORD} - --keychain-password ${MACOS_KEYCHAIN_PASSWORD} - --notarization-password ${MACOS_NOTARIZATION_PASSWORD} - --output artifacts/ - - run: - command: > - cargo xtask package - --target x86_64-apple-darwin --apple-team-id ${APPLE_TEAM_ID} --apple-username ${APPLE_USERNAME} --cert-bundle-base64 ${MACOS_CERT_BUNDLE_BASE64} @@ -956,7 +957,7 @@ workflows: matrix: parameters: platform: - [ macos_build, windows_build, amd_linux_build, arm_linux_build ] + [ intel_macos_build, arm_macos_build, windows_build, amd_linux_build, arm_linux_build ] - secops/wiz-docker: context: - platform-docker-ro @@ -1053,7 +1054,7 @@ workflows: matrix: parameters: platform: - [ macos_build, windows_build, amd_linux_build, arm_linux_build ] + [ intel_macos_build, arm_macos_build, windows_build, amd_linux_build, arm_linux_build ] filters: branches: ignore: /.*/ diff --git a/.gitleaks.toml b/.gitleaks.toml index fe9e4e5395..7551d923a4 100644 --- a/.gitleaks.toml +++ b/.gitleaks.toml @@ -69,6 +69,7 @@ paths = [ '''^apollo-router\/src\/.+\/testdata\/.+''', + '''^apollo-router/src/plugins/authentication/tests.rs$''' ] [[ rules ]] diff --git a/CHANGELOG.md b/CHANGELOG.md index c6525abc40..8e630f2797 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -4,6 +4,75 @@ All notable changes to Router will be documented in this file. This project adheres to [Semantic Versioning v2.0.0](https://semver.org/spec/v2.0.0.html). + +# [1.41.1] - 2024-03-08 + +> [!NOTE] +> +> v1.41.1 replaces a failed publish of v1.41.0. The version number had to be moved from v1.41.0 to v1.41.1, but the release is otherwise the same. Apologies for the confusion! + +## 🚀 Features + +### Entity caching: Add tracing spans around Redis interactions ([PR #4667](https://github.com/apollographql/router/pull/4667)) + +This adds `cache_lookup` and `cache_store` spans to traces which show Redis calls related to our recently announced [entity caching](https://www.apollographql.com/docs/router/configuration/entity-caching/) feature. This also changes the behavior slightly so that storing in Redis does not stop the execution of the rest of the query. + +By [@Geal](https://github.com/Geal) in https://github.com/apollographql/router/pull/4667 + +### Use Gzip compression when downloading Persisted Query manifests ([PR #4622](https://github.com/apollographql/router/pull/4622)) + +Router will now request Gzip compression when downloading Persisted Query manifests for improved network efficiency. + +By [@glasser](https://github.com/glasser) in https://github.com/apollographql/router/pull/4622 + +### Redis: add a fail open option ([Issue #4334](https://github.com/apollographql/router/issues/4334)) + +This option configures the Router's behavior in case it cannot connect to Redis: +- By default, the router will start and all requests will be handled in a degraded state. +- Alternatively, this option can be configured to prevent the router from starting if it can't connect to Redis. + +By [@Geal](https://github.com/Geal) in https://github.com/apollographql/router/pull/4534 + +## 🐛 Fixes + +### Default header now correctly set when `experimental_response_trace_id` is enabled ([Issue #4699](https://github.com/apollographql/router/issues/4699)) + +When configuring the `experimental_response_trace_id` without an explicit header it now correctly takes the default one `apollo-trace-id`. + +Example of configuration: + +```yaml +telemetry: + exporters: + tracing: + experimental_response_trace_id: + enabled: true +``` + +By [@bnjjj](https://github.com/bnjjj) in https://github.com/apollographql/router/pull/4702 + +# [1.41.0] - 2024-03-08 + +> [!NOTE] +> +> The release of v1.41.0 failed unexpectedly late in the deployment process due to a preventable publishing failure. The release has been yanked from Crates.io on account of not being published successfully across all deployment targets. This release is fully replaced by v1.41.1. Apologies for the confusion! + +# [1.40.2] - 2024-03-06 + +## 🔒 Security + +### Apply `limits.http_max_request_bytes` on streaming request body decompression ([PR #4759](https://github.com/apollographql/router/pull/4759)) + +This release fixes a Denial-of-Service (DoS) type vulnerability which exists in affected versions of the Router according to our [published security advistory](https://github.com/apollographql/router/security/advisories/GHSA-cgqf-3cq5-wvcj). The fix changes the evaluation of the `limits.http_max_request_bytes` configuration to take place on a stream of bytes, allowing it to be applied to compressed HTTP payloads, prior to decompression. Previously, the limit was only being applied after the entirety of the compressed payload was decompressed, which could result in significant memory consumption which exceeded configured expectations while compressed payloads were expanded. + +## 🐛 Fixes + +### Re-activate the macOS Intel builder ([PR #4723](https://github.com/apollographql/router/pull/4723)) + +We have re-activated macOS Intel (x86) builds in CircleCI, despite their upcoming deprecation, while we take a different approach to solving this and maintaining Intel support for the time-being. This became necessary since cross-compiling the router from ARM to x86 resulted in issues with V8 snapshots and runtime issues on the macOS Intel binaries produced by those Apple Silicon build machines. + +By [@Geal](https://github.com/Geal) in https://github.com/apollographql/router/pull/4723 + # [1.40.1] - 2024-02-16 ## 🐛 Fixes diff --git a/Cargo.lock b/Cargo.lock index 3f67f176af..6124a387ff 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -251,7 +251,7 @@ dependencies = [ [[package]] name = "apollo-router" -version = "1.40.1" +version = "1.41.1" dependencies = [ "access-json", "anyhow", @@ -278,6 +278,7 @@ dependencies = [ "clap", "console", "console-subscriber", + "cookie", "dashmap", "derivative", "derive_more", @@ -408,7 +409,7 @@ dependencies = [ [[package]] name = "apollo-router-benchmarks" -version = "1.40.1" +version = "1.41.1" dependencies = [ "apollo-parser", "apollo-router", @@ -424,7 +425,7 @@ dependencies = [ [[package]] name = "apollo-router-scaffold" -version = "1.40.1" +version = "1.41.1" dependencies = [ "anyhow", "cargo-scaffold", @@ -1668,6 +1669,16 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6245d59a3e82a7fc217c5828a6692dbc6dfb63a0c8c90495621f7b9d79704a0e" +[[package]] +name = "cookie" +version = "0.18.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3cd91cf61412820176e137621345ee43b3f4423e589e7ae4e50d601d93e35ef8" +dependencies = [ + "time", + "version_check", +] + [[package]] name = "cookie-factory" version = "0.3.2" diff --git a/apollo-router-benchmarks/Cargo.toml b/apollo-router-benchmarks/Cargo.toml index 3fcfbd8839..b3c43ad056 100644 --- a/apollo-router-benchmarks/Cargo.toml +++ b/apollo-router-benchmarks/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "apollo-router-benchmarks" -version = "1.40.1" +version = "1.41.1" authors = ["Apollo Graph, Inc. "] edition = "2021" license = "Elastic-2.0" diff --git a/apollo-router-scaffold/Cargo.toml b/apollo-router-scaffold/Cargo.toml index fe61cfa946..2537f93a36 100644 --- a/apollo-router-scaffold/Cargo.toml +++ b/apollo-router-scaffold/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "apollo-router-scaffold" -version = "1.40.1" +version = "1.41.1" authors = ["Apollo Graph, Inc. "] edition = "2021" license = "Elastic-2.0" diff --git a/apollo-router-scaffold/templates/base/Cargo.toml b/apollo-router-scaffold/templates/base/Cargo.toml index fe1f09a95f..ff95741747 100644 --- a/apollo-router-scaffold/templates/base/Cargo.toml +++ b/apollo-router-scaffold/templates/base/Cargo.toml @@ -22,7 +22,7 @@ apollo-router = { path ="{{integration_test}}apollo-router" } apollo-router = { git="https://github.com/apollographql/router.git", branch="{{branch}}" } {{else}} # Note if you update these dependencies then also update xtask/Cargo.toml -apollo-router = "1.40.1" +apollo-router = "1.41.1" {{/if}} {{/if}} async-trait = "0.1.52" diff --git a/apollo-router-scaffold/templates/base/xtask/Cargo.toml b/apollo-router-scaffold/templates/base/xtask/Cargo.toml index 9d7834e090..19dc26d984 100644 --- a/apollo-router-scaffold/templates/base/xtask/Cargo.toml +++ b/apollo-router-scaffold/templates/base/xtask/Cargo.toml @@ -13,7 +13,7 @@ apollo-router-scaffold = { path ="{{integration_test}}apollo-router-scaffold" } {{#if branch}} apollo-router-scaffold = { git="https://github.com/apollographql/router.git", branch="{{branch}}" } {{else}} -apollo-router-scaffold = { git = "https://github.com/apollographql/router.git", tag = "v1.40.1" } +apollo-router-scaffold = { git = "https://github.com/apollographql/router.git", tag = "v1.41.1" } {{/if}} {{/if}} anyhow = "1.0.58" diff --git a/apollo-router/Cargo.toml b/apollo-router/Cargo.toml index 42a5c06f54..31fbec63c3 100644 --- a/apollo-router/Cargo.toml +++ b/apollo-router/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "apollo-router" -version = "1.40.1" +version = "1.41.1" authors = ["Apollo Graph, Inc. "] repository = "https://github.com/apollographql/router/" documentation = "https://docs.rs/apollo-router" @@ -86,6 +86,7 @@ clap = { version = "4.5.1", default-features = false, features = [ "help", ] } console-subscriber = { version = "0.2.0", optional = true } +cookie = { version = "0.18.0", default-features = false } ci_info = { version = "0.14.14", features = ["serde-1"] } dashmap = { version = "5.5.3", features = ["serde"] } derivative = "2.2.0" diff --git a/apollo-router/src/axum_factory/axum_http_server_factory.rs b/apollo-router/src/axum_factory/axum_http_server_factory.rs index d0632fe963..c59842076e 100644 --- a/apollo-router/src/axum_factory/axum_http_server_factory.rs +++ b/apollo-router/src/axum_factory/axum_http_server_factory.rs @@ -6,6 +6,7 @@ use std::sync::atomic::Ordering; use std::sync::Arc; use std::time::Instant; +use axum::error_handling::HandleErrorLayer; use axum::extract::Extension; use axum::extract::State; use axum::http::StatusCode; @@ -32,14 +33,15 @@ use tokio::sync::mpsc; use tokio_rustls::TlsAcceptor; use tower::service_fn; use tower::BoxError; +use tower::ServiceBuilder; use tower::ServiceExt; +use tower_http::decompression::DecompressionBody; use tower_http::trace::TraceLayer; use super::listeners::ensure_endpoints_consistency; use super::listeners::ensure_listenaddrs_consistency; use super::listeners::extra_endpoints; use super::listeners::ListenersAndRouters; -use super::utils::decompress_request_body; use super::utils::PropagatingMakeSpan; use super::ListenAddrAndRouter; use super::ENDPOINT_CALLBACK; @@ -57,6 +59,7 @@ use crate::plugins::traffic_shaping::RateLimited; use crate::router::ApolloRouterError; use crate::router_factory::Endpoint; use crate::router_factory::RouterFactory; +use crate::services::http::service::BodyStream; use crate::services::router; use crate::uplink::license_enforcement::LicenseState; use crate::uplink::license_enforcement::APOLLO_ROUTER_LICENSE_EXPIRED; @@ -173,11 +176,9 @@ where tracing::trace!(?health, request = ?req.router_request, "health check"); async move { Ok(router::Response { - response: http::Response::builder() - .status(status_code) - .body::( - serde_json::to_vec(&health).map_err(BoxError::from)?.into(), - )?, + response: http::Response::builder().status(status_code).body::( + serde_json::to_vec(&health).map_err(BoxError::from)?.into(), + )?, context: req.context, }) } @@ -422,6 +423,10 @@ pub(crate) fn span_mode(configuration: &Configuration) -> SpanMode { .unwrap_or_default() } +async fn decompression_error(_error: BoxError) -> axum::response::Response { + (StatusCode::BAD_REQUEST, "cannot decompress request body").into_response() +} + fn main_endpoint( service_factory: RF, configuration: &Configuration, @@ -436,8 +441,16 @@ where })?; let span_mode = span_mode(configuration); + let decompression = ServiceBuilder::new() + .layer(HandleErrorLayer::<_, ()>::new(decompression_error)) + .layer( + tower_http::decompression::RequestDecompressionLayer::new() + .br(true) + .gzip(true) + .deflate(true), + ); let mut main_route = main_router::(configuration) - .layer(middleware::from_fn(decompress_request_body)) + .layer(decompression) .layer(middleware::from_fn_with_state( (license, Instant::now(), Arc::new(AtomicU64::new(0))), license_handler, @@ -530,19 +543,21 @@ async fn license_handler( } } -pub(super) fn main_router(configuration: &Configuration) -> axum::Router +pub(super) fn main_router( + configuration: &Configuration, +) -> axum::Router<(), DecompressionBody> where RF: RouterFactory, { let mut router = Router::new().route( &configuration.supergraph.sanitized_path(), get({ - move |Extension(service): Extension, request: Request| { + move |Extension(service): Extension, request: Request>| { handle_graphql(service.create().boxed(), request) } }) .post({ - move |Extension(service): Extension, request: Request| { + move |Extension(service): Extension, request: Request>| { handle_graphql(service.create().boxed(), request) } }), @@ -552,12 +567,14 @@ where router = router.route( "/", get({ - move |Extension(service): Extension, request: Request| { + move |Extension(service): Extension, + request: Request>| { handle_graphql(service.create().boxed(), request) } }) .post({ - move |Extension(service): Extension, request: Request| { + move |Extension(service): Extension, + request: Request>| { handle_graphql(service.create().boxed(), request) } }), @@ -569,10 +586,14 @@ where async fn handle_graphql( service: router::BoxService, - http_request: Request, + http_request: Request>, ) -> impl IntoResponse { let _guard = SessionCountGuard::start(); + let (parts, body) = http_request.into_parts(); + + let http_request = http::Request::from_parts(parts, Body::wrap_stream(BodyStream::new(body))); + let request: router::Request = http_request.into(); let context = request.context.clone(); let accept_encoding = request diff --git a/apollo-router/src/axum_factory/utils.rs b/apollo-router/src/axum_factory/utils.rs index 96c42bb6ce..d1e4f51a51 100644 --- a/apollo-router/src/axum_factory/utils.rs +++ b/apollo-router/src/axum_factory/utils.rs @@ -2,19 +2,8 @@ use std::net::SocketAddr; -use async_compression::tokio::write::BrotliDecoder; -use async_compression::tokio::write::GzipDecoder; -use async_compression::tokio::write::ZlibDecoder; -use axum::http::StatusCode; -use axum::middleware::Next; -use axum::response::*; -use futures::prelude::*; -use http::header::CONTENT_ENCODING; -use http::Request; -use hyper::Body; use opentelemetry::global; use opentelemetry::trace::TraceContextExt; -use tokio::io::AsyncWriteExt; use tower_http::trace::MakeSpan; use tower_service::Service; use tracing::Span; @@ -26,83 +15,6 @@ use crate::uplink::license_enforcement::LICENSE_EXPIRED_SHORT_MESSAGE; pub(crate) const REQUEST_SPAN_NAME: &str = "request"; -pub(super) async fn decompress_request_body( - req: Request, - next: Next, -) -> Result { - let (parts, body) = req.into_parts(); - let content_encoding = parts.headers.get(&CONTENT_ENCODING); - macro_rules! decode_body { - ($decoder: ident, $error_message: expr) => {{ - let body_bytes = hyper::body::to_bytes(body) - .map_err(|err| { - ( - StatusCode::BAD_REQUEST, - format!("cannot read request body: {err}"), - ) - .into_response() - }) - .await?; - let mut decoder = $decoder::new(Vec::new()); - decoder.write_all(&body_bytes).await.map_err(|err| { - ( - StatusCode::BAD_REQUEST, - format!("{}: {err}", $error_message), - ) - .into_response() - })?; - decoder.shutdown().await.map_err(|err| { - ( - StatusCode::BAD_REQUEST, - format!("{}: {err}", $error_message), - ) - .into_response() - })?; - - Ok(next - .run(Request::from_parts(parts, Body::from(decoder.into_inner()))) - .await) - }}; - } - - match content_encoding { - Some(content_encoding) => match content_encoding.to_str() { - Ok(content_encoding_str) => match content_encoding_str { - "br" => decode_body!(BrotliDecoder, "cannot decompress (brotli) request body"), - "gzip" => decode_body!(GzipDecoder, "cannot decompress (gzip) request body"), - "deflate" => decode_body!(ZlibDecoder, "cannot decompress (deflate) request body"), - "identity" => Ok(next.run(Request::from_parts(parts, body)).await), - unknown => { - let message = format!("unknown content-encoding header value {unknown:?}"); - tracing::error!(message); - u64_counter!( - "apollo_router_http_requests_total", - "Total number of HTTP requests made.", - 1, - status = StatusCode::BAD_REQUEST.as_u16() as i64, - error = message.clone() - ); - - Err((StatusCode::BAD_REQUEST, message).into_response()) - } - }, - - Err(err) => { - let message = format!("cannot read content-encoding header: {err}"); - u64_counter!( - "apollo_router_http_requests_total", - "Total number of HTTP requests made.", - 1, - status = 400, - error = message.clone() - ); - Err((StatusCode::BAD_REQUEST, message).into_response()) - } - }, - None => Ok(next.run(Request::from_parts(parts, body)).await), - } -} - #[derive(Clone, Default)] pub(crate) struct PropagatingMakeSpan { pub(crate) license: LicenseState, diff --git a/apollo-router/src/configuration/migrations/0023-batching.yaml b/apollo-router/src/configuration/migrations/0023-batching.yaml new file mode 100644 index 0000000000..7457467524 --- /dev/null +++ b/apollo-router/src/configuration/migrations/0023-batching.yaml @@ -0,0 +1,5 @@ +description: Batching is no longer experimental +actions: + - type: move + from: experimental_batching + to: batching diff --git a/apollo-router/src/configuration/mod.rs b/apollo-router/src/configuration/mod.rs index 6a55090af8..f353791fe5 100644 --- a/apollo-router/src/configuration/mod.rs +++ b/apollo-router/src/configuration/mod.rs @@ -1415,5 +1415,20 @@ pub(crate) enum SubgraphBatchingConfig { #[derive(Debug, Clone, Default, Deserialize, Serialize, JsonSchema)] pub(crate) struct CommonBatchingConfig { /// Whether this batching config should be enabled - enabled: bool, + pub(crate) enabled: bool, +} + +impl Batching { + // Check if we should enable batching for a particular subgraph (service_name) + pub(crate) fn batch_include(&self, service_name: &str) -> bool { + match &self.subgraph { + Some(subgraph_batching_config) => match subgraph_batching_config { + SubgraphBatchingConfig::All(all_config) => all_config.enabled, + SubgraphBatchingConfig::Subgraphs(subgraphs) => subgraphs + .iter() + .any(|(k, v)| k == service_name && v.enabled), + }, + None => false, + } + } } diff --git a/apollo-router/src/configuration/snapshots/apollo_router__configuration__tests__schema_generation.snap b/apollo-router/src/configuration/snapshots/apollo_router__configuration__tests__schema_generation.snap index 1e8c339bee..9d21ab1ca5 100644 --- a/apollo-router/src/configuration/snapshots/apollo_router__configuration__tests__schema_generation.snap +++ b/apollo-router/src/configuration/snapshots/apollo_router__configuration__tests__schema_generation.snap @@ -264,6 +264,29 @@ expression: "&schema" }, "nullable": true }, + "headers": { + "description": "List of headers to add to the JWKS request", + "type": "array", + "items": { + "description": "Insert a header", + "type": "object", + "required": [ + "name", + "value" + ], + "properties": { + "name": { + "description": "The name of the header", + "type": "string" + }, + "value": { + "description": "The value for the header", + "type": "string" + } + }, + "additionalProperties": false + } + }, "issuer": { "description": "Expected issuer for tokens verified by that JWKS", "type": "string", @@ -284,6 +307,59 @@ expression: "&schema" }, "additionalProperties": false } + }, + "sources": { + "description": "Alternative sources to extract the JWT", + "type": "array", + "items": { + "oneOf": [ + { + "type": "object", + "required": [ + "type" + ], + "properties": { + "name": { + "description": "HTTP header expected to contain JWT", + "default": "authorization", + "type": "string" + }, + "type": { + "type": "string", + "enum": [ + "header" + ] + }, + "value_prefix": { + "description": "Header value prefix", + "default": "Bearer", + "type": "string" + } + }, + "additionalProperties": false + }, + { + "type": "object", + "required": [ + "name", + "type" + ], + "properties": { + "name": { + "description": "Name of the cookie containing the JWT", + "type": "string" + }, + "type": { + "type": "string", + "enum": [ + "cookie" + ] + } + }, + "additionalProperties": false + } + ] + } } }, "additionalProperties": false diff --git a/apollo-router/src/configuration/snapshots/apollo_router__configuration__tests__upgrade_old_configuration@batching.yaml.snap b/apollo-router/src/configuration/snapshots/apollo_router__configuration__tests__upgrade_old_configuration@batching.yaml.snap new file mode 100644 index 0000000000..daec7b3f14 --- /dev/null +++ b/apollo-router/src/configuration/snapshots/apollo_router__configuration__tests__upgrade_old_configuration@batching.yaml.snap @@ -0,0 +1,8 @@ +--- +source: apollo-router/src/configuration/tests.rs +expression: new_config +--- +--- +batching: + enabled: true + mode: batch_http_link diff --git a/apollo-router/src/configuration/testdata/migrations/batching.yaml b/apollo-router/src/configuration/testdata/migrations/batching.yaml new file mode 100644 index 0000000000..c177d3f45e --- /dev/null +++ b/apollo-router/src/configuration/testdata/migrations/batching.yaml @@ -0,0 +1,3 @@ +experimental_batching: + enabled: true + mode: batch_http_link diff --git a/apollo-router/src/configuration/tests.rs b/apollo-router/src/configuration/tests.rs index 4b15fe66a5..87ccf9ba3d 100644 --- a/apollo-router/src/configuration/tests.rs +++ b/apollo-router/src/configuration/tests.rs @@ -968,6 +968,80 @@ fn it_adds_slash_to_custom_health_check_path_if_missing() { assert_eq!(&conf.health_check.path, "/healthz"); } +#[test] +fn it_processes_batching_subgraph_all_enabled_correctly() { + let json_config = json!({ + "enabled": true, + "mode": "batch_http_link", + "subgraph": { + "all": { + "enabled": true + } + } + }); + + let config: Batching = serde_json::from_value(json_config).unwrap(); + + assert!(config.batch_include("anything")); +} + +#[test] +fn it_processes_batching_subgraph_all_disabled_correctly() { + let json_config = json!({ + "enabled": true, + "mode": "batch_http_link", + "subgraph": { + "all": { + "enabled": false + } + } + }); + + let config: Batching = serde_json::from_value(json_config).unwrap(); + + assert!(!config.batch_include("anything")); +} + +#[test] +fn it_processes_batching_subgraph_accounts_enabled_correctly() { + let json_config = json!({ + "enabled": true, + "mode": "batch_http_link", + "subgraph": { + "subgraphs": { + "accounts": { + "enabled": true + } + } + } + }); + + let config: Batching = serde_json::from_value(json_config).unwrap(); + + assert!(!config.batch_include("anything")); + assert!(config.batch_include("accounts")); +} + +#[test] +fn it_processes_batching_subgraph_accounts_disabled_correctly() { + let json_config = json!({ + "enabled": true, + "mode": "batch_http_link", + "subgraph": { + "subgraphs": { + "accounts": { + "enabled": false + } + } + } + }); + + let config: Batching = serde_json::from_value(json_config).unwrap(); + + assert!(!config.batch_include("anything")); + assert!(!config.batch_include("accounts")); +} + fn has_field_level_serde_defaults(lines: &[&str], line_number: usize) -> bool { let serde_field_default = Regex::new( r#"^\s*#[\s\n]*\[serde\s*\((.*,)?\s*default\s*=\s*"[a-zA-Z0-9_:]+"\s*(,.*)?\)\s*\]\s*$"#, diff --git a/apollo-router/src/executable.rs b/apollo-router/src/executable.rs index 1be97f6840..0d5675d92e 100644 --- a/apollo-router/src/executable.rs +++ b/apollo-router/src/executable.rs @@ -437,15 +437,6 @@ impl Executable { return Ok(()); } - // mark stdout and stderr as non blocking. If they are blocking and piped - // to a program that does not consume them, the router starts hanging on - // all requests: https://github.com/apollographql/router/issues/4612 - #[cfg(not(target_os = "windows"))] - { - let _ = set_blocking(libc::STDOUT_FILENO, false); - let _ = set_blocking(libc::STDERR_FILENO, false); - } - copy_args_to_env(); let apollo_telemetry_initialized = if graph_os() { @@ -753,26 +744,6 @@ fn copy_args_to_env() { }); } -#[cfg(not(target_os = "windows"))] -fn set_blocking(fd: std::os::fd::RawFd, blocking: bool) -> std::io::Result<()> { - let flags = unsafe { libc::fcntl(fd, libc::F_GETFL, 0) }; - if flags < 0 { - return Err(std::io::Error::last_os_error()); - } - - let flags = if blocking { - flags & !libc::O_NONBLOCK - } else { - flags | libc::O_NONBLOCK - }; - let res = unsafe { libc::fcntl(fd, libc::F_SETFL, flags) }; - if res != 0 { - return Err(std::io::Error::last_os_error()); - } - - Ok(()) -} - #[cfg(test)] mod tests { use crate::executable::add_log_filter; diff --git a/apollo-router/src/plugins/authentication/jwks.rs b/apollo-router/src/plugins/authentication/jwks.rs index 6c2d6c7d02..b517b10577 100644 --- a/apollo-router/src/plugins/authentication/jwks.rs +++ b/apollo-router/src/plugins/authentication/jwks.rs @@ -24,6 +24,7 @@ use tower::BoxError; use tracing_futures::Instrument; use url::Url; +use super::Header; use super::CLIENT; use super::DEFAULT_AUTHENTICATION_NETWORK_TIMEOUT; @@ -40,6 +41,7 @@ pub(super) struct JwksConfig { pub(super) issuer: Option, pub(super) algorithms: Option>, pub(super) poll_interval: Duration, + pub(super) headers: Vec
, } #[derive(Clone)] @@ -56,9 +58,9 @@ impl JwksManager { let downloads = list .iter() .cloned() - .map(|JwksConfig { url, .. }| { + .map(|JwksConfig { url, headers, .. }| { let span = tracing::info_span!("fetch jwks", url = %url); - get_jwks(url.clone()) + get_jwks(url.clone(), headers.clone()) .map(|opt_jwks| opt_jwks.map(|jwks| (url, jwks))) .instrument(span) }) @@ -110,7 +112,7 @@ async fn poll( repeat((config, jwks_map)).then(|(config, jwks_map)| async move { tokio::time::sleep(config.poll_interval).await; - if let Some(jwks) = get_jwks(config.url.clone()).await { + if let Some(jwks) = get_jwks(config.url.clone(), config.headers.clone()).await { if let Ok(mut map) = jwks_map.write() { map.insert(config.url, jwks); } @@ -140,7 +142,7 @@ async fn poll( // This function is expected to return an Optional value, but we'd like to let // users know the various failure conditions. Hence the various clumsy map_err() // scattered through the processing. -pub(super) async fn get_jwks(url: Url) -> Option { +pub(super) async fn get_jwks(url: Url, headers: Vec
) -> Option { let data = if url.scheme() == "file" { let path = url .to_file_path() @@ -166,10 +168,16 @@ pub(super) async fn get_jwks(url: Url) -> Option { .ok()? .clone(); - my_client + let mut builder = my_client .get(url) .header(ACCEPT, APPLICATION_JSON.essence_str()) - .header(CONTENT_TYPE, APPLICATION_JSON.essence_str()) + .header(CONTENT_TYPE, APPLICATION_JSON.essence_str()); + + for header in headers.into_iter() { + builder = builder.header(header.name, header.value); + } + + builder .timeout(DEFAULT_AUTHENTICATION_NETWORK_TIMEOUT) .send() .await diff --git a/apollo-router/src/plugins/authentication/mod.rs b/apollo-router/src/plugins/authentication/mod.rs index 03693a1106..29be795960 100644 --- a/apollo-router/src/plugins/authentication/mod.rs +++ b/apollo-router/src/plugins/authentication/mod.rs @@ -8,6 +8,9 @@ use std::time::SystemTime; use std::time::UNIX_EPOCH; use displaydoc::Display; +use http::HeaderMap; +use http::HeaderName; +use http::HeaderValue; use http::StatusCode; use jsonwebtoken::decode; use jsonwebtoken::decode_header; @@ -39,6 +42,8 @@ use self::subgraph::SigningParamsConfig; use self::subgraph::SubgraphAuth; use crate::graphql; use crate::layers::ServiceBuilderExt; +use crate::plugin::serde::deserialize_header_name; +use crate::plugin::serde::deserialize_header_value; use crate::plugin::Plugin; use crate::plugin::PluginInit; use crate::plugins::authentication::jwks::JwkSetInfo; @@ -128,6 +133,10 @@ struct JWTConf { /// Header value prefix #[serde(default = "default_header_value_prefix")] header_value_prefix: String, + + /// Alternative sources to extract the JWT + #[serde(default)] + sources: Vec, } #[derive(Clone, Debug, Deserialize, JsonSchema)] @@ -148,7 +157,43 @@ struct JwksConf { #[schemars(with = "Option>", default)] #[serde(default)] algorithms: Option>, + /// List of headers to add to the JWKS request + #[serde(default)] + headers: Vec
, +} + +#[derive(Clone, Debug, JsonSchema, Deserialize)] +#[serde(rename_all = "snake_case", deny_unknown_fields)] +/// Insert a header +struct Header { + /// The name of the header + #[schemars(with = "String")] + #[serde(deserialize_with = "deserialize_header_name")] + name: HeaderName, + + /// The value for the header + #[schemars(with = "String")] + #[serde(deserialize_with = "deserialize_header_value")] + value: HeaderValue, } + +#[derive(Clone, Debug, Deserialize, JsonSchema)] +#[serde(deny_unknown_fields, rename_all = "lowercase", tag = "type")] +enum Source { + Header { + /// HTTP header expected to contain JWT + #[serde(default = "default_header_name")] + name: String, + /// Header value prefix + #[serde(default = "default_header_value_prefix")] + value_prefix: String, + }, + Cookie { + /// Name of the cookie containing the JWT + name: String, + }, +} + /// Authentication #[derive(Clone, Debug, Default, Deserialize, JsonSchema)] #[serde(deny_unknown_fields)] @@ -385,7 +430,7 @@ impl Plugin for AuthenticationPlugin { None }; - let router = if let Some(router_conf) = init.config.router { + let router = if let Some(mut router_conf) = init.config.router { if router_conf .jwt .header_value_prefix @@ -395,6 +440,23 @@ impl Plugin for AuthenticationPlugin { { return Err(Error::BadHeaderValuePrefix.into()); } + + for source in &router_conf.jwt.sources { + if let Source::Header { value_prefix, .. } = source { + if value_prefix.as_bytes().iter().any(u8::is_ascii_whitespace) { + return Err(Error::BadHeaderValuePrefix.into()); + } + } + } + + router_conf.jwt.sources.insert( + 0, + Source::Header { + name: router_conf.jwt.header_name.clone(), + value_prefix: router_conf.jwt.header_value_prefix.clone(), + }, + ); + let mut list = vec![]; for jwks_conf in &router_conf.jwt.jwks { let url: Url = Url::from_str(jwks_conf.url.as_str())?; @@ -406,6 +468,7 @@ impl Plugin for AuthenticationPlugin { .as_ref() .map(|algs| algs.iter().cloned().collect()), poll_interval: jwks_conf.poll_interval, + headers: jwks_conf.headers.clone(), }); } @@ -506,73 +569,23 @@ fn authenticate( ControlFlow::Break(response) } - // The http_request is stored in a `Router::Request` context. - // We are going to check the headers for the presence of the configured header - let jwt_value_result = match request.router_request.headers().get(&config.header_name) { - Some(value) => value.to_str(), - None => { - return ControlFlow::Continue(request); - } - }; - - // If we find the header, but can't convert it to a string, let the client know - let jwt_value_untrimmed = match jwt_value_result { - Ok(value) => value, - Err(_not_a_string_error) => { - return failure_message( - request.context, - AuthenticationError::CannotConvertToString, - StatusCode::BAD_REQUEST, - ); + let mut jwt = None; + for source in &config.sources { + match extract_jwt(source, request.router_request.headers()) { + None => continue, + Some(Err(error)) => { + return failure_message(request.context, error, StatusCode::BAD_REQUEST) + } + Some(Ok(extracted_jwt)) => { + jwt = Some(extracted_jwt); + break; + } } - }; - - // Let's trim out leading and trailing whitespace to be accommodating - let jwt_value = jwt_value_untrimmed.trim(); - - // Make sure the format of our message matches our expectations - // Technically, the spec is case sensitive, but let's accept - // case variations - // - let prefix_len = config.header_value_prefix.len(); - if jwt_value.len() < prefix_len - || !&jwt_value[..prefix_len].eq_ignore_ascii_case(&config.header_value_prefix) - { - return failure_message( - request.context, - AuthenticationError::InvalidPrefix(jwt_value_untrimmed, &config.header_value_prefix), - StatusCode::BAD_REQUEST, - ); } - // If there's no header prefix, we need to avoid splitting the header - let jwt = if config.header_value_prefix.is_empty() { - // check for whitespace- we've already trimmed, so this means the request has a prefix that shouldn't exist - if jwt_value.contains(' ') { - return failure_message( - request.context, - AuthenticationError::InvalidPrefix( - jwt_value_untrimmed, - &config.header_value_prefix, - ), - StatusCode::BAD_REQUEST, - ); - } - // we can simply assign the jwt to the jwt_value; we'll validate down below - jwt_value - } else { - // Otherwise, we need to split our string in (at most 2) sections. - let jwt_parts: Vec<&str> = jwt_value.splitn(2, ' ').collect(); - if jwt_parts.len() != 2 { - return failure_message( - request.context, - AuthenticationError::MissingJWT(jwt_value), - StatusCode::BAD_REQUEST, - ); - } - - // We have our jwt - jwt_parts[1] + let jwt = match jwt { + Some(jwt) => jwt, + None => return ControlFlow::Continue(request), }; // Try to create a valid header to work with @@ -661,6 +674,94 @@ fn authenticate( } } +fn extract_jwt<'a, 'b: 'a>( + source: &'a Source, + headers: &'b HeaderMap, +) -> Option>> { + match source { + Source::Header { name, value_prefix } => { + // The http_request is stored in a `Router::Request` context. + // We are going to check the headers for the presence of the configured header + let jwt_value_result = match headers.get(name) { + Some(value) => value.to_str(), + None => return None, + }; + + // If we find the header, but can't convert it to a string, let the client know + let jwt_value_untrimmed = match jwt_value_result { + Ok(value) => value, + Err(_not_a_string_error) => { + return Some(Err(AuthenticationError::CannotConvertToString)); + } + }; + + // Let's trim out leading and trailing whitespace to be accommodating + let jwt_value = jwt_value_untrimmed.trim(); + + // Make sure the format of our message matches our expectations + // Technically, the spec is case sensitive, but let's accept + // case variations + // + let prefix_len = value_prefix.len(); + if jwt_value.len() < prefix_len + || !&jwt_value[..prefix_len].eq_ignore_ascii_case(value_prefix) + { + return Some(Err(AuthenticationError::InvalidPrefix( + jwt_value_untrimmed, + value_prefix, + ))); + } + // If there's no header prefix, we need to avoid splitting the header + let jwt = if value_prefix.is_empty() { + // check for whitespace- we've already trimmed, so this means the request has a prefix that shouldn't exist + if jwt_value.contains(' ') { + return Some(Err(AuthenticationError::InvalidPrefix( + jwt_value_untrimmed, + value_prefix, + ))); + } + + // we can simply assign the jwt to the jwt_value; we'll validate down below + jwt_value + } else { + // Otherwise, we need to split our string in (at most 2) sections. + let jwt_parts: Vec<&str> = jwt_value.splitn(2, ' ').collect(); + if jwt_parts.len() != 2 { + return Some(Err(AuthenticationError::MissingJWT(jwt_value))); + } + + // We have our jwt + jwt_parts[1] + }; + Some(Ok(jwt)) + } + Source::Cookie { name } => { + for header in headers.get_all("cookie") { + let value = match header.to_str() { + Ok(value) => value, + Err(_not_a_string_error) => { + return Some(Err(AuthenticationError::CannotConvertToString)); + } + }; + for cookie in cookie::Cookie::split_parse(value) { + match cookie { + Err(_) => continue, + Ok(cookie) => { + if cookie.name() == name { + if let Some(value) = cookie.value_raw() { + return Some(Ok(value)); + } + } + } + } + } + } + + None + } + } +} + fn decode_jwt( jwt: &str, keys: Vec<(Option, Jwk)>, diff --git a/apollo-router/src/plugins/authentication/tests.rs b/apollo-router/src/plugins/authentication/tests.rs index af3cd14590..8382ab5206 100644 --- a/apollo-router/src/plugins/authentication/tests.rs +++ b/apollo-router/src/plugins/authentication/tests.rs @@ -1,9 +1,18 @@ use std::collections::HashMap; use std::collections::HashSet; +use std::io; use std::path::Path; +use std::sync::atomic::AtomicBool; +use std::sync::atomic::Ordering; +use std::sync::Arc; use base64::prelude::BASE64_URL_SAFE_NO_PAD; use base64::Engine as _; +use http::header::CONTENT_TYPE; +use hyper::server::conn::AddrIncoming; +use hyper::service::make_service_fn; +use hyper::service::service_fn; +use hyper::Server; use insta::assert_yaml_snapshot; use jsonwebtoken::encode; use jsonwebtoken::get_current_timestamp; @@ -12,6 +21,7 @@ use jsonwebtoken::jwk::EllipticCurveKeyParameters; use jsonwebtoken::jwk::EllipticCurveKeyType; use jsonwebtoken::jwk::JwkSet; use jsonwebtoken::EncodingKey; +use mime::APPLICATION_JSON; use p256::ecdsa::SigningKey; use p256::pkcs8::EncodePrivateKey; use rand_core::OsRng; @@ -19,6 +29,7 @@ use serde::Serialize; use serde_json::Value; use tracing::subscriber; +use super::Header; use super::*; use crate::assert_snapshot_subscriber; use crate::plugin::test; @@ -592,6 +603,188 @@ async fn it_panics_when_auth_prefix_has_correct_format_but_contains_trailing_whi let _test_harness = build_a_test_harness(None, Some("SOMETHING ".to_string()), false).await; } +#[tokio::test] +async fn it_extracts_the_token_from_cookies() { + let mut mock_service = test::MockSupergraphService::new(); + mock_service.expect_clone().return_once(move || { + println!("cloned to supergraph mock"); + let mut mock_service = test::MockSupergraphService::new(); + mock_service + .expect_call() + .once() + .returning(move |req: supergraph::Request| { + Ok(supergraph::Response::fake_builder() + .data("response created within the mock") + .context(req.context) + .build() + .unwrap()) + }); + mock_service + }); + let jwks_url = create_an_url("jwks.json"); + + let config = serde_json::json!({ + "authentication": { + "router": { + "jwt" : { + "jwks": [ + { + "url": &jwks_url + } + ], + "sources": [ + { + "type": "cookie", + "name": "authz" + } + ], + } + } + }, + "rhai": { + "scripts":"tests/fixtures", + "main":"require_authentication.rhai" + } + }); + let test_harness = crate::TestHarness::builder() + .configuration_json(config) + .unwrap() + .supergraph_hook(move |_| mock_service.clone().boxed()) + .build_router() + .await + .unwrap(); + + let token = "eyJ0eXAiOiJKV1QiLCJhbGciOiJIUzI1NiIsImtpZCI6ImtleTEifQ.eyJleHAiOjEwMDAwMDAwMDAwLCJhbm90aGVyIGNsYWltIjoidGhpcyBpcyBhbm90aGVyIGNsYWltIn0.4GrmfxuUST96cs0YUC0DfLAG218m7vn8fO_ENfXnu5A"; + + // Let's create a request with our operation name + let request_with_appropriate_name = supergraph::Request::canned_builder() + .operation_name("me".to_string()) + .header( + http::header::COOKIE, + format!("a= b; c = d HttpOnly; authz = {token}; e = f"), + ) + .build() + .unwrap(); + + // ...And call our service stack with it + let mut service_response = test_harness + .oneshot(request_with_appropriate_name.try_into().unwrap()) + .await + .unwrap(); + let response: graphql::Response = serde_json::from_slice( + service_response + .next_response() + .await + .unwrap() + .unwrap() + .to_vec() + .as_slice(), + ) + .unwrap(); + + assert_eq!(response.errors, vec![]); + + assert_eq!(StatusCode::OK, service_response.response.status()); + + let expected_mock_response_data = "response created within the mock"; + // with the expected message + assert_eq!(expected_mock_response_data, response.data.as_ref().unwrap()); +} + +#[tokio::test] +async fn it_supports_multiple_sources() { + let mut mock_service = test::MockSupergraphService::new(); + mock_service.expect_clone().return_once(move || { + println!("cloned to supergraph mock"); + let mut mock_service = test::MockSupergraphService::new(); + mock_service + .expect_call() + .once() + .returning(move |req: supergraph::Request| { + Ok(supergraph::Response::fake_builder() + .data("response created within the mock") + .context(req.context) + .build() + .unwrap()) + }); + mock_service + }); + let jwks_url = create_an_url("jwks.json"); + + let config = serde_json::json!({ + "authentication": { + "router": { + "jwt" : { + "jwks": [ + { + "url": &jwks_url + } + ], + "sources": [ + { + "type": "cookie", + "name": "authz" + }, + { + "type": "header", + "name": "authz1" + }, + { + "type": "header", + "name": "authz2", + "value_prefix": "bear" + } + ], + } + } + }, + "rhai": { + "scripts":"tests/fixtures", + "main":"require_authentication.rhai" + } + }); + let test_harness = crate::TestHarness::builder() + .configuration_json(config) + .unwrap() + .supergraph_hook(move |_| mock_service.clone().boxed()) + .build_router() + .await + .unwrap(); + + let token = "eyJ0eXAiOiJKV1QiLCJhbGciOiJIUzI1NiIsImtpZCI6ImtleTEifQ.eyJleHAiOjEwMDAwMDAwMDAwLCJhbm90aGVyIGNsYWltIjoidGhpcyBpcyBhbm90aGVyIGNsYWltIn0.4GrmfxuUST96cs0YUC0DfLAG218m7vn8fO_ENfXnu5A"; + + // Let's create a request with our operation name + let request_with_appropriate_name = supergraph::Request::canned_builder() + .operation_name("me".to_string()) + .header("Authz2", format!("Bear {token}")) + .build() + .unwrap(); + + // ...And call our service stack with it + let mut service_response = test_harness + .oneshot(request_with_appropriate_name.try_into().unwrap()) + .await + .unwrap(); + let response: graphql::Response = serde_json::from_slice( + service_response + .next_response() + .await + .unwrap() + .unwrap() + .to_vec() + .as_slice(), + ) + .unwrap(); + + assert_eq!(response.errors, vec![]); + + assert_eq!(StatusCode::OK, service_response.response.status()); + + let expected_mock_response_data = "response created within the mock"; + // with the expected message + assert_eq!(expected_mock_response_data, response.data.as_ref().unwrap()); +} + async fn build_jwks_search_components() -> JwksManager { let mut sets = vec![]; let mut urls = vec![]; @@ -607,6 +800,7 @@ async fn build_jwks_search_components() -> JwksManager { issuer: None, algorithms: None, poll_interval: Duration::from_secs(60), + headers: Vec::new(), }); } @@ -717,6 +911,7 @@ fn make_manager(jwk: &Jwk, issuer: Option) -> JwksManager { issuer, algorithms: None, poll_interval: Duration::from_secs(60), + headers: Vec::new(), }]; let map = HashMap::from([(url, jwks); 1]); @@ -767,7 +962,12 @@ async fn issuer_check() { .build() .unwrap(); - match authenticate(&JWTConf::default(), &manager, request.try_into().unwrap()) { + let mut config = JWTConf::default(); + config.sources.push(Source::Header { + name: super::default_header_name(), + value_prefix: super::default_header_value_prefix(), + }); + match authenticate(&config, &manager, request.try_into().unwrap()) { ControlFlow::Break(res) => { panic!("unexpected response: {res:?}"); } @@ -800,7 +1000,7 @@ async fn issuer_check() { .build() .unwrap(); - match authenticate(&JWTConf::default(), &manager, request.try_into().unwrap()) { + match authenticate(&config, &manager, request.try_into().unwrap()) { ControlFlow::Break(res) => { let response: graphql::Response = serde_json::from_slice( &hyper::body::to_bytes(res.response.into_body()) @@ -840,7 +1040,7 @@ async fn issuer_check() { .build() .unwrap(); - match authenticate(&JWTConf::default(), &manager, request.try_into().unwrap()) { + match authenticate(&config, &manager, request.try_into().unwrap()) { ControlFlow::Break(res) => { let response: graphql::Response = serde_json::from_slice( &hyper::body::to_bytes(res.response.into_body()) @@ -875,7 +1075,7 @@ async fn issuer_check() { .build() .unwrap(); - match authenticate(&JWTConf::default(), &manager, request.try_into().unwrap()) { + match authenticate(&config, &manager, request.try_into().unwrap()) { ControlFlow::Break(res) => { let response: graphql::Response = serde_json::from_slice( &hyper::body::to_bytes(res.response.into_body()) @@ -914,6 +1114,7 @@ async fn it_rejects_key_with_restricted_algorithm() { issuer: None, algorithms: Some(HashSet::from([Algorithm::RS256])), poll_interval: Duration::from_secs(60), + headers: Vec::new(), }); } @@ -945,6 +1146,7 @@ async fn it_rejects_and_accepts_keys_with_restricted_algorithms_and_unknown_jwks issuer: None, algorithms: Some(HashSet::from([Algorithm::RS256])), poll_interval: Duration::from_secs(60), + headers: Vec::new(), }); } @@ -983,6 +1185,7 @@ async fn it_accepts_key_without_use_or_keyops() { issuer: None, algorithms: None, poll_interval: Duration::from_secs(60), + headers: Vec::new(), }); } @@ -1013,6 +1216,7 @@ async fn it_accepts_elliptic_curve_key_without_alg() { issuer: None, algorithms: None, poll_interval: Duration::from_secs(60), + headers: Vec::new(), }); } @@ -1043,6 +1247,7 @@ async fn it_accepts_rsa_key_without_alg() { issuer: None, algorithms: None, poll_interval: Duration::from_secs(60), + headers: Vec::new(), }); } @@ -1064,3 +1269,59 @@ fn test_parse_failure_logs() { assert_yaml_snapshot!(jwks); }); } + +#[tokio::test(flavor = "multi_thread")] +async fn jwks_send_headers() { + let listener = tokio::net::TcpListener::bind("127.0.0.1:0").await.unwrap(); + let socket_addr = listener.local_addr().unwrap(); + + let got_header = Arc::new(AtomicBool::new(false)); + let gh = got_header.clone(); + let service = make_service_fn(move |_| { + let gh = gh.clone(); + async move { + //let gh1 = gh.clone(); + Ok::<_, io::Error>(service_fn(move |req| { + println!("got re: {:?}", req.headers()); + let gh: Arc = gh.clone(); + async move { + if req + .headers() + .get("jwks-authz") + .and_then(|v| v.to_str().ok()) + == Some("user1") + { + gh.store(true, Ordering::Release); + } + Ok::<_, io::Error>( + http::Response::builder() + .header(CONTENT_TYPE, APPLICATION_JSON.essence_str()) + .status(StatusCode::OK) + .version(http::Version::HTTP_11) + .body::(include_str!("testdata/jwks.json").into()) + .unwrap(), + ) + } + })) + } + }); + let server = Server::builder(AddrIncoming::from_listener(listener).unwrap()).serve(service); + tokio::task::spawn(server); + + let url = Url::parse(&format!("http://{socket_addr}/")).unwrap(); + + let _jwks_manager = JwksManager::new(vec![JwksConfig { + url, + issuer: None, + algorithms: Some(HashSet::from([Algorithm::RS256])), + poll_interval: Duration::from_secs(60), + headers: vec![Header { + name: HeaderName::from_static("jwks-authz"), + value: HeaderValue::from_static("user1"), + }], + }]) + .await + .unwrap(); + + assert!(got_header.load(Ordering::Acquire)); +} diff --git a/apollo-router/src/plugins/telemetry/fmt_layer.rs b/apollo-router/src/plugins/telemetry/fmt_layer.rs index f9ac06ebd1..ff28719fdb 100644 --- a/apollo-router/src/plugins/telemetry/fmt_layer.rs +++ b/apollo-router/src/plugins/telemetry/fmt_layer.rs @@ -2,7 +2,6 @@ use std::cell::RefCell; use std::collections::HashMap; use std::collections::HashSet; use std::io::IsTerminal; -use std::io::Write; use std::marker::PhantomData; use opentelemetry::Key; @@ -183,12 +182,7 @@ where if self.fmt_event.format_event(&ctx, &mut buf, event).is_ok() { let mut writer = self.make_writer.make_writer(); if let Err(err) = std::io::Write::write_all(&mut writer, buf.as_bytes()) { - if err.kind() != std::io::ErrorKind::WouldBlock { - let _ = std::io::stderr().write_all( - format!("cannot flush the logging buffer, this is a bug: {err:?}") - .as_bytes(), - ); - } + eprintln!("cannot flush the logging buffer, this is a bug: {err:?}"); } } buf.clear(); diff --git a/apollo-router/src/plugins/telemetry/formatters/json.rs b/apollo-router/src/plugins/telemetry/formatters/json.rs index f44ed0b1b6..aa0b6b9e1b 100644 --- a/apollo-router/src/plugins/telemetry/formatters/json.rs +++ b/apollo-router/src/plugins/telemetry/formatters/json.rs @@ -250,6 +250,14 @@ where } } + // dd.trace_id is special. It must appear as a root attribute on log lines, so we need to extract it from the root span. + // We're just going to assume if it's there then we should output it, as the user will have to have configured it to be there. + if let Some(span) = ¤t_span { + if let Some(dd_trace_id) = extract_dd_trace_id(span) { + serializer.serialize_entry("dd.trace_id", &dd_trace_id)?; + } + } + if self.config.display_span_list && current_span.is_some() { serializer.serialize_entry( "spans", @@ -269,6 +277,38 @@ where } } +fn extract_dd_trace_id<'a, 'b, T: LookupSpan<'a>>(span: &SpanRef<'a, T>) -> Option { + let mut dd_trace_id = None; + let mut root = span.scope().from_root(); + if let Some(root_span) = root.next() { + let ext = root_span.extensions(); + // Extract dd_trace_id, this could be in otel data or log attributes + if let Some(otel_data) = root_span.extensions().get::() { + if let Some(attributes) = otel_data.builder.attributes.as_ref() { + if let Some((_k, v)) = attributes + .iter() + .find(|(k, _v)| k.as_str() == "dd.trace_id") + { + dd_trace_id = Some(v.to_string()); + } + } + }; + + if dd_trace_id.is_none() { + if let Some(log_attr) = ext.get::() { + if let Some(kv) = log_attr + .attributes() + .iter() + .find(|kv| kv.key.as_str() == "dd.trace_id") + { + dd_trace_id = Some(kv.value.to_string()); + } + } + } + } + dd_trace_id +} + struct WriteAdaptor<'a> { fmt_write: &'a mut dyn fmt::Write, } @@ -301,3 +341,81 @@ impl<'a> fmt::Debug for WriteAdaptor<'a> { f.pad("WriteAdaptor { .. }") } } + +#[cfg(test)] +mod test { + use tracing::subscriber; + use tracing_core::Event; + use tracing_core::Subscriber; + use tracing_subscriber::layer::Context; + use tracing_subscriber::layer::SubscriberExt; + use tracing_subscriber::registry::LookupSpan; + use tracing_subscriber::Layer; + use tracing_subscriber::Registry; + + use crate::plugins::telemetry::dynamic_attribute::DynAttribute; + use crate::plugins::telemetry::dynamic_attribute::DynAttributeLayer; + use crate::plugins::telemetry::formatters::json::extract_dd_trace_id; + + struct RequiresDatadogLayer; + impl Layer for RequiresDatadogLayer + where + S: Subscriber + for<'lookup> LookupSpan<'lookup>, + { + fn on_event(&self, event: &Event<'_>, ctx: Context<'_, S>) { + let current_span = event + .parent() + .and_then(|id| ctx.span(id)) + .or_else(|| ctx.lookup_current()) + .expect("current span expected"); + let extracted = extract_dd_trace_id(¤t_span); + assert_eq!(extracted, Some("1234".to_string())); + } + } + + #[test] + fn test_extract_dd_trace_id_span_attribute() { + subscriber::with_default( + Registry::default() + .with(RequiresDatadogLayer) + .with(tracing_opentelemetry::layer()), + || { + let root_span = tracing::info_span!("root", dd.trace_id = "1234"); + let _root_span = root_span.enter(); + tracing::info!("test"); + }, + ); + } + + #[test] + fn test_extract_dd_trace_id_dyn_attribute() { + subscriber::with_default( + Registry::default() + .with(RequiresDatadogLayer) + .with(DynAttributeLayer) + .with(tracing_opentelemetry::layer()), + || { + let root_span = tracing::info_span!("root"); + root_span.set_dyn_attribute("dd.trace_id".into(), "1234".into()); + let _root_span = root_span.enter(); + tracing::info!("test"); + }, + ); + } + + #[test] + #[should_panic] + fn test_missing_dd_attribute() { + subscriber::with_default( + Registry::default() + .with(RequiresDatadogLayer) + .with(DynAttributeLayer) + .with(tracing_opentelemetry::layer()), + || { + let root_span = tracing::info_span!("root"); + let _root_span = root_span.enter(); + tracing::info!("test"); + }, + ); + } +} diff --git a/apollo-router/src/services/http/service.rs b/apollo-router/src/services/http/service.rs index f1d7029b53..d087f2517d 100644 --- a/apollo-router/src/services/http/service.rs +++ b/apollo-router/src/services/http/service.rs @@ -342,6 +342,13 @@ pin_project! { } } +impl BodyStream { + /// Create a new `BodyStream`. + pub(crate) fn new(body: DecompressionBody) -> Self { + Self { inner: body } + } +} + impl Stream for BodyStream where B: hyper::body::HttpBody, diff --git a/apollo-router/src/services/layers/content_negotiation.rs b/apollo-router/src/services/layers/content_negotiation.rs index fe33d3adb8..59ccc5498e 100644 --- a/apollo-router/src/services/layers/content_negotiation.rs +++ b/apollo-router/src/services/layers/content_negotiation.rs @@ -71,6 +71,17 @@ where .to_string(), )) .expect("cannot fail"); + u64_counter!( + "apollo_router_http_requests_total", + "Total number of HTTP requests made.", + 1, + status = StatusCode::UNSUPPORTED_MEDIA_TYPE.as_u16() as i64, + error = format!( + r#"'content-type' header must be one of: {:?} or {:?}"#, + APPLICATION_JSON.essence_str(), + GRAPHQL_JSON_RESPONSE_HEADER_VALUE, + ) + ); return Ok(ControlFlow::Break(response.into())); } diff --git a/apollo-router/src/services/router/service.rs b/apollo-router/src/services/router/service.rs index 2d7ddaac4b..ac0d7153e8 100644 --- a/apollo-router/src/services/router/service.rs +++ b/apollo-router/src/services/router/service.rs @@ -40,6 +40,7 @@ use crate::batching::BatchQuery; use crate::cache::DeduplicatingCache; use crate::configuration::Batching; use crate::configuration::BatchingMode; +use crate::configuration::SubgraphBatchingConfig; use crate::graphql; use crate::http_ext; #[cfg(test)] @@ -609,9 +610,27 @@ impl RouterService { let mut results = Vec::with_capacity(ok_results.len()); let batch_size = ok_results.len(); + // Insert our batch configuration into Context extensions. + // If the results len > 1, we always insert our batching configuration + // If subgraph batching configuration exists and is enabled for any of our subgraphs, we create our shared batch details let shared_batch_details: Option>> = if ok_results.len() > 1 { context.extensions().lock().insert(self.batching.clone()); - Some(Arc::new(Mutex::new(Batch::new(batch_size)))) + match &self.batching.subgraph { + Some(subgraph_batching_config) => { + let enabled = match subgraph_batching_config { + SubgraphBatchingConfig::All(all_config) => all_config.enabled, + SubgraphBatchingConfig::Subgraphs(subgraphs) => { + subgraphs.values().any(|v| v.enabled) + } + }; + if enabled { + Some(Arc::new(Mutex::new(Batch::new(batch_size)))) + } else { + None + } + } + None => None, + } } else { None }; @@ -648,14 +667,15 @@ impl RouterService { .lock() .get::() .cloned(); + // Sub-scope so that new_context_guard is dropped before pushing into the new + // SupergraphRequest { - // Sub-scope so that new_context_guard is dropped before pushing into the new - // SupergraphRequest let mut new_context_guard = new_context.extensions().lock(); if let Some(client_request_accepts) = client_request_accepts_opt { new_context_guard.insert(client_request_accepts); } new_context_guard.insert(self.batching.clone()); + // We are only going to insert a BatchQuery if Subgraph processing is enabled if let Some(shared_batch_details) = &shared_batch_details { new_context_guard .insert(BatchQuery::new(index + 1, shared_batch_details.clone())); diff --git a/apollo-router/src/services/subgraph_service.rs b/apollo-router/src/services/subgraph_service.rs index 55b4ca982a..61e2744e72 100644 --- a/apollo-router/src/services/subgraph_service.rs +++ b/apollo-router/src/services/subgraph_service.rs @@ -41,6 +41,7 @@ use super::layers::content_negotiation::GRAPHQL_JSON_RESPONSE_HEADER_VALUE; use super::Plugins; use crate::batching::BatchQuery; use crate::batching::Waiter; +use crate::configuration::Batching; use crate::configuration::BatchingMode; use crate::configuration::TlsClientAuth; use crate::error::FetchError; @@ -737,40 +738,52 @@ async fn call_batched_http( client_factory: crate::services::http::HttpClientServiceFactory, service_name: &str, ) -> Result { - // We'd like to park a task here, but we can't park it whilst we have the context extensions - // lock held. That would be very bad... - // So, we set an optional listener and wait to hear back from the batch processor - // TODO: When we have the configuration work in place, we'll need to use the configuration - // settings here to determine if batching is even possible for a service. i.e.: If subgraphs - // all is set or (if not) if a specific subgraph name (synonym for service_name here) is set. - // Make sure to pick this up as part of the configuration work. + // We use configuration to determine if calls may be batched. If we have Batching + // configuration, then we check (batch_include()) if the current subgraph has batching enabled + // in configuration. If it does, we then start to process a potential batch. + // + // If we are processing a batch, then we'd like to park tasks here, but we can't park them whilst + // we have the context extensions lock held. That would be very bad... + // We set optional batch_responder and waiters_opt to control waiting behaviour without + // holding the extensions lock. let mut batch_responder: Option< tokio::sync::oneshot::Receiver>, > = None; let mut waiters_opt = None; - if let Some(batching) = context.extensions().lock().get_mut::() { - if !batching.finished() { - tracing::debug!("in subgraph we have batching: {batching}, service: {service_name}"); - batching.increment_subgraph_seen(); - tracing::debug!("ready to process batch?: {}", batching.ready()); - batch_responder = Some(batching.get_waiter( - request.clone(), - body.clone(), - context.clone(), - service_name, - )); - if batching.ready() { - // This is where we start processing our accumulated batch data. - // We can't do it whilst holding the context extensions lock, so signal we are - // ready to proceed by updating waiters_opt. - tracing::debug!("Batch data: {batching}"); - waiters_opt = Some(batching.get_waiters()); + { + let mut extensions_guard = context.extensions().lock(); + let batching_opt = extensions_guard.get::(); + if let Some(batching) = batching_opt { + if batching.batch_include(service_name) { + if let Some(batch_query) = extensions_guard.get_mut::() { + if !batch_query.finished() { + tracing::debug!("in subgraph we have batch_query: {batch_query}, service: {service_name}"); + batch_query.increment_subgraph_seen(); + tracing::debug!("ready to process batch?: {}", batch_query.ready()); + batch_responder = Some(batch_query.get_waiter( + request.clone(), + body.clone(), + context.clone(), + service_name, + )); + if batch_query.ready() { + // This is where we start processing our accumulated batch data. + // We can't do it whilst holding the context extensions lock, so signal we are + // ready to proceed by updating waiters_opt. + tracing::debug!("Batch data: {batch_query}"); + waiters_opt = Some(batch_query.get_waiters()); + } + } + } } } } // We've dropped the extensions lock, check to see if we have batches to process or just a // normal http call. // TODO: Think about the impact on the router if a batch is never finished/ready. Can that happen? + // I think this is a limitation of this prototype. For instance, a query may be rejected for + // various reasons during execution or query planning (e.g.: authz), so we need to figure out + // how to handle that situation. if let Some(receiver) = batch_responder { // If waiters_opt is Some, then our batch is full and it's time to process it. if let Some(waiters) = waiters_opt { diff --git a/apollo-router/tests/common.rs b/apollo-router/tests/common.rs index c94e6c2da5..32d1ad4a1e 100644 --- a/apollo-router/tests/common.rs +++ b/apollo-router/tests/common.rs @@ -10,7 +10,6 @@ use std::time::SystemTime; use buildstructor::buildstructor; use http::header::ACCEPT; -use http::header::CONTENT_ENCODING; use http::header::CONTENT_TYPE; use http::HeaderValue; use jsonpath_lib::Selector; @@ -67,6 +66,9 @@ pub struct IntegrationTest { supergraph: PathBuf, _subgraphs: wiremock::MockServer, subscriber: Option, + + _subgraph_overrides: HashMap, + pub bind_addr: SocketAddr, } struct TracedResponder(pub(crate) ResponseTemplate); @@ -126,27 +128,18 @@ impl IntegrationTest { // Add a default override for products, if not specified subgraph_overrides.entry("products".into()).or_insert(url); - // Insert the overrides into the config - let overrides = subgraph_overrides - .into_iter() - .map(|(name, url)| (name, serde_json::Value::String(url))); - let mut config: Value = serde_yaml::from_str(config).unwrap(); - match config - .as_object_mut() - .and_then(|o| o.get_mut("override_subgraph_url")) - .and_then(|o| o.as_object_mut()) - { - None => { - if let Some(o) = config.as_object_mut() { - o.insert("override_subgraph_url".to_string(), overrides.collect()); - } - } - Some(override_url) => { - override_url.extend(overrides); - } - } + // Bind to a random port + // Note: This might still fail if a different process binds to the port found here + // before the router is started. + // Note: We need the nested scope so that the listener gets dropped once its address + // is resolved. + let addr = { + let bound = TcpListener::bind(SocketAddr::from(([127, 0, 0, 1], 0))).unwrap(); + bound.local_addr().unwrap() + }; - let config_str = serde_yaml::to_string(&config).unwrap(); + // Insert the overrides into the config + let config_str = merge_overrides(config, &subgraph_overrides, &addr); let supergraph = supergraph.unwrap_or(PathBuf::from_iter([ "..", @@ -186,6 +179,8 @@ impl IntegrationTest { supergraph, _subgraphs: subgraphs, subscriber, + _subgraph_overrides: subgraph_overrides, + bind_addr: addr, } } @@ -357,9 +352,12 @@ impl IntegrationTest { #[allow(dead_code)] pub async fn update_config(&self, yaml: &str) { - tokio::fs::write(&self.test_config_location, yaml) - .await - .expect("must be able to write config"); + tokio::fs::write( + &self.test_config_location, + &merge_overrides(yaml, &self._subgraph_overrides, &self.bind_addr), + ) + .await + .expect("must be able to write config"); } #[allow(dead_code)] @@ -395,7 +393,7 @@ impl IntegrationTest { } #[allow(dead_code)] - pub fn execute_bad_content_encoding( + pub fn execute_bad_content_type( &self, ) -> impl std::future::Future { self.execute_query_internal(&json!({"garbage":{}}), Some("garbage")) @@ -404,7 +402,7 @@ impl IntegrationTest { fn execute_query_internal( &self, query: &Value, - content_encoding: Option<&'static str>, + content_type: Option<&'static str>, ) -> impl std::future::Future { assert!( self.router.is_some(), @@ -412,6 +410,8 @@ impl IntegrationTest { ); let dispatch = self.subscriber.clone(); let query = query.clone(); + let url = format!("http://{}", self.bind_addr); + async move { let span = info_span!("client_request"); let span_id = span.context().span().span_context().trace_id().to_string(); @@ -420,9 +420,11 @@ impl IntegrationTest { let client = reqwest::Client::new(); let mut request = client - .post("http://localhost:4000") - .header(CONTENT_TYPE, APPLICATION_JSON.essence_str()) - .header(CONTENT_ENCODING, content_encoding.unwrap_or("identity")) + .post(url) + .header( + CONTENT_TYPE, + content_type.unwrap_or(APPLICATION_JSON.essence_str()), + ) .header("apollographql-client-name", "custom_name") .header("apollographql-client-version", "1.0") .header("x-my-header", "test") @@ -460,12 +462,13 @@ impl IntegrationTest { ); let query = query.clone(); let dispatch = self.subscriber.clone(); + let url = format!("http://{}", self.bind_addr); async move { let client = reqwest::Client::new(); let mut request = client - .post("http://localhost:4000") + .post(url) .header(CONTENT_TYPE, APPLICATION_JSON.essence_str()) .header("apollographql-client-name", "custom_name") .header("apollographql-client-version", "1.0") @@ -507,6 +510,7 @@ impl IntegrationTest { ); let dispatch = self.subscriber.clone(); + let url = format!("http://{}", self.bind_addr); async move { let span = info_span!("client_raw_request"); let span_id = span.context().span().span_context().trace_id().to_string(); @@ -521,7 +525,7 @@ impl IntegrationTest { }; let mut request = client - .post("http://localhost:4000") + .post(url) .header(CONTENT_TYPE, mime.to_string()) .header("apollographql-client-name", "custom_name") .header("apollographql-client-version", "1.0") @@ -565,7 +569,7 @@ impl IntegrationTest { let _span_guard = span.enter(); let mut request = client - .post("http://localhost:4000") + .post(format!("http://{}", self.bind_addr)) .header(CONTENT_TYPE, APPLICATION_JSON.essence_str()) .header(ACCEPT, "multipart/mixed;subscriptionSpec=1.0") .header("apollographql-client-name", "custom_name") @@ -594,7 +598,7 @@ impl IntegrationTest { let client = reqwest::Client::new(); let request = client - .get("http://localhost:4000/metrics") + .get(format!("http://{}/metrics", self.bind_addr)) .header("apollographql-client-name", "custom_name") .header("apollographql-client-version", "1.0") .build() @@ -658,7 +662,7 @@ impl IntegrationTest { #[allow(dead_code)] pub async fn assert_log_contains(&mut self, msg: &str) { let now = Instant::now(); - while now.elapsed() < Duration::from_secs(5) { + while now.elapsed() < Duration::from_secs(10) { if let Ok(line) = self.stdio_rx.try_recv() { if line.contains(msg) { return; @@ -799,3 +803,81 @@ impl ValueExt for Value { self.as_str().map(|s| s.to_string()) } } + +/// Merge in overrides to a yaml config. +/// +/// The test harness needs some options to be present for it to work, so this +/// function allows patching any config to include the needed values. +fn merge_overrides( + yaml: &str, + subgraph_overrides: &HashMap, + bind_addr: &SocketAddr, +) -> String { + // Parse the config as yaml + let mut config: Value = serde_yaml::from_str(yaml).unwrap(); + + // Insert subgraph overrides, making sure to keep other overrides if present + let overrides = subgraph_overrides + .iter() + .map(|(name, url)| (name.clone(), serde_json::Value::String(url.clone()))); + match config + .as_object_mut() + .and_then(|o| o.get_mut("override_subgraph_url")) + .and_then(|o| o.as_object_mut()) + { + None => { + if let Some(o) = config.as_object_mut() { + o.insert("override_subgraph_url".to_string(), overrides.collect()); + } + } + Some(override_url) => { + override_url.extend(overrides); + } + } + + // Override the listening address always since we spawn the router on a + // random port. + match config + .as_object_mut() + .and_then(|o| o.get_mut("supergraph")) + .and_then(|o| o.as_object_mut()) + { + None => { + if let Some(o) = config.as_object_mut() { + o.insert( + "supergraph".to_string(), + serde_json::json!({ + "listen": bind_addr.to_string(), + }), + ); + } + } + Some(supergraph_conf) => { + supergraph_conf.insert( + "listen".to_string(), + serde_json::Value::String(bind_addr.to_string()), + ); + } + } + + // Override the metrics listening address always since we spawn the router on a + // random port. + if let Some(prom_config) = config + .as_object_mut() + .and_then(|o| o.get_mut("telemetry")) + .and_then(|o| o.as_object_mut()) + .and_then(|o| o.get_mut("exporters")) + .and_then(|o| o.as_object_mut()) + .and_then(|o| o.get_mut("metrics")) + .and_then(|o| o.as_object_mut()) + .and_then(|o| o.get_mut("prometheus")) + .and_then(|o| o.as_object_mut()) + { + prom_config.insert( + "listen".to_string(), + serde_json::Value::String(bind_addr.to_string()), + ); + } + + serde_yaml::to_string(&config).unwrap() +} diff --git a/apollo-router/tests/fixtures/file_upload/multiple_subgraph.graphql b/apollo-router/tests/fixtures/file_upload/multiple_subgraph.graphql deleted file mode 100644 index 369b305118..0000000000 --- a/apollo-router/tests/fixtures/file_upload/multiple_subgraph.graphql +++ /dev/null @@ -1,122 +0,0 @@ -schema - @link(url: "https://specs.apollo.dev/link/v1.0") - @link(url: "https://specs.apollo.dev/join/v0.3", for: EXECUTION) -{ - query: Query - mutation: Mutation -} - -directive @join__enumValue(graph: join__Graph!) repeatable on ENUM_VALUE - -directive @join__field(graph: join__Graph, requires: join__FieldSet, provides: join__FieldSet, type: String, external: Boolean, override: String, usedOverridden: Boolean) repeatable on FIELD_DEFINITION | INPUT_FIELD_DEFINITION - -directive @join__graph(name: String!, url: String!) on ENUM_VALUE - -directive @join__implements(graph: join__Graph!, interface: String!) repeatable on OBJECT | INTERFACE - -directive @join__type(graph: join__Graph!, key: join__FieldSet, extension: Boolean! = false, resolvable: Boolean! = true, isInterfaceObject: Boolean! = false) repeatable on OBJECT | INTERFACE | UNION | ENUM | INPUT_OBJECT | SCALAR - -directive @join__unionMember(graph: join__Graph!, member: String!) repeatable on UNION - -directive @link(url: String, as: String, for: link__Purpose, import: [link__Import]) repeatable on SCHEMA - -scalar join__FieldSet - -enum join__Graph { - ACCOUNTS @join__graph(name: "accounts", url: "https://accounts.demo.starstuff.dev/") - INVENTORY @join__graph(name: "inventory", url: "https://inventory.demo.starstuff.dev/") - PRODUCTS @join__graph(name: "products", url: "https://products.demo.starstuff.dev/") - REVIEWS @join__graph(name: "reviews", url: "https://reviews.demo.starstuff.dev/") - UPLOADS1 @join__graph(name: "uploads1", url: "http://127.0.0.1:4005/s1") - UPLOADS2 @join__graph(name: "uploads2", url: "http://127.0.0.1:4005/s2") -} - -scalar link__Import - -enum link__Purpose { - """ - `SECURITY` features provide metadata necessary to securely resolve fields. - """ - SECURITY - - """ - `EXECUTION` features provide metadata necessary for operation execution. - """ - EXECUTION -} - -type Product - @join__type(graph: INVENTORY, key: "upc") - @join__type(graph: PRODUCTS, key: "upc") - @join__type(graph: REVIEWS, key: "upc") -{ - upc: String! - weight: Int @join__field(graph: INVENTORY, external: true) @join__field(graph: PRODUCTS) - price: Int @join__field(graph: INVENTORY, external: true) @join__field(graph: PRODUCTS) - inStock: Boolean @join__field(graph: INVENTORY) - shippingEstimate: Int @join__field(graph: INVENTORY, requires: "price weight") - name: String @join__field(graph: PRODUCTS) - reviews: [Review] @join__field(graph: REVIEWS) - reviewsForAuthor(authorID: ID!): [Review] @join__field(graph: REVIEWS) -} - -type Query - @join__type(graph: ACCOUNTS) - @join__type(graph: INVENTORY) - @join__type(graph: PRODUCTS) - @join__type(graph: REVIEWS) -{ - me: User @join__field(graph: ACCOUNTS) - topProducts(first: Int = 5): [Product] @join__field(graph: PRODUCTS) -} - -type Review - @join__type(graph: REVIEWS, key: "id") -{ - id: ID! - body: String - author: User @join__field(graph: REVIEWS, provides: "username") - product: Product -} - -type User - @join__type(graph: ACCOUNTS, key: "id") - @join__type(graph: REVIEWS, key: "id") -{ - id: ID! - name: String @join__field(graph: ACCOUNTS) - username: String @join__field(graph: ACCOUNTS) @join__field(graph: REVIEWS, external: true) - reviews: [Review] @join__field(graph: REVIEWS) -} - - -scalar Upload1 - @join__type(graph: UPLOADS1) - -scalar Upload2 - @join__type(graph: UPLOADS2) - -type File1 - @join__type(graph: UPLOADS1) -{ - filename: String! - mimetype: String! - encoding: String! - body: String! -} -type File2 - @join__type(graph: UPLOADS2) -{ - filename: String! - mimetype: String! - encoding: String! - body: String! -} - -type Mutation - @join__type(graph: UPLOADS1) - @join__type(graph: UPLOADS2) -{ - singleUpload1(file: Upload1): File1 @join__field(graph: UPLOADS1) - singleUpload2(file: Upload2): File2 @join__field(graph: UPLOADS2) -} diff --git a/apollo-router/tests/fixtures/file_upload/single_subgraph.graphql b/apollo-router/tests/fixtures/file_upload/schema.graphql similarity index 50% rename from apollo-router/tests/fixtures/file_upload/single_subgraph.graphql rename to apollo-router/tests/fixtures/file_upload/schema.graphql index 23d2733cb4..4816f46fa8 100644 --- a/apollo-router/tests/fixtures/file_upload/single_subgraph.graphql +++ b/apollo-router/tests/fixtures/file_upload/schema.graphql @@ -23,11 +23,8 @@ directive @link(url: String, as: String, for: link__Purpose, import: [link__Impo scalar join__FieldSet enum join__Graph { - ACCOUNTS @join__graph(name: "accounts", url: "https://accounts.demo.starstuff.dev/") - INVENTORY @join__graph(name: "inventory", url: "https://inventory.demo.starstuff.dev/") - PRODUCTS @join__graph(name: "products", url: "https://products.demo.starstuff.dev/") - REVIEWS @join__graph(name: "reviews", url: "https://reviews.demo.starstuff.dev/") UPLOADS @join__graph(name: "uploads", url: "http://127.0.0.1:4005/") + UPLOADS_CLONE @join__graph(name: "uploads_clone", url: "http://127.0.0.1:4006/") } scalar link__Import @@ -44,53 +41,11 @@ enum link__Purpose { EXECUTION } -type Product - @join__type(graph: INVENTORY, key: "upc") - @join__type(graph: PRODUCTS, key: "upc") - @join__type(graph: REVIEWS, key: "upc") -{ - upc: String! - weight: Int @join__field(graph: INVENTORY, external: true) @join__field(graph: PRODUCTS) - price: Int @join__field(graph: INVENTORY, external: true) @join__field(graph: PRODUCTS) - inStock: Boolean @join__field(graph: INVENTORY) - shippingEstimate: Int @join__field(graph: INVENTORY, requires: "price weight") - name: String @join__field(graph: PRODUCTS) - reviews: [Review] @join__field(graph: REVIEWS) - reviewsForAuthor(authorID: ID!): [Review] @join__field(graph: REVIEWS) -} - -type Query - @join__type(graph: ACCOUNTS) - @join__type(graph: INVENTORY) - @join__type(graph: PRODUCTS) - @join__type(graph: REVIEWS) -{ - me: User @join__field(graph: ACCOUNTS) - topProducts(first: Int = 5): [Product] @join__field(graph: PRODUCTS) -} - -type Review - @join__type(graph: REVIEWS, key: "id") -{ - id: ID! - body: String - author: User @join__field(graph: REVIEWS, provides: "username") - product: Product -} - -type User - @join__type(graph: ACCOUNTS, key: "id") - @join__type(graph: REVIEWS, key: "id") -{ - id: ID! - name: String @join__field(graph: ACCOUNTS) - username: String @join__field(graph: ACCOUNTS) @join__field(graph: REVIEWS, external: true) - reviews: [Review] @join__field(graph: REVIEWS) -} - scalar Upload @join__type(graph: UPLOADS) +scalar UploadClone + @join__type(graph: UPLOADS_CLONE) type File @join__type(graph: UPLOADS) @@ -100,9 +55,35 @@ type File encoding: String! body: String! } +type FileClone + @join__type(graph: UPLOADS_CLONE) +{ + filename: String! + mimetype: String! + encoding: String! + body: String! +} + +input NestedUpload + @join__type(graph: UPLOADS) +{ + file: Upload! +} + +type Query + @join__type(graph: UPLOADS) + @join__type(graph: UPLOADS_CLONE) +{ + empty: String +} type Mutation @join__type(graph: UPLOADS) + @join__type(graph: UPLOADS_CLONE) { singleUpload(file: Upload): File @join__field(graph: UPLOADS) + singleUploadNonNull(file: Upload!): File! @join__field(graph: UPLOADS) + singleUploadClone(file: UploadClone): FileClone @join__field(graph: UPLOADS_CLONE) + multiUpload(files: [Upload!]!): [File!]! @join__field(graph: UPLOADS) + nestedUpload(nested: NestedUpload): File @join__field(graph: UPLOADS) } diff --git a/apollo-router/tests/integration/file_upload.rs b/apollo-router/tests/integration/file_upload.rs index cef8bd1d81..5081509027 100644 --- a/apollo-router/tests/integration/file_upload.rs +++ b/apollo-router/tests/integration/file_upload.rs @@ -1,5 +1,4 @@ use std::collections::BTreeMap; -use std::path::PathBuf; use bytes::Bytes; use http::header::CONTENT_ENCODING; @@ -40,12 +39,6 @@ async fn it_uploads_a_single_file() -> Result<(), BoxError> { .handler(make_handler!(helper::echo_single_file)) .request(request) .subgraph_mapping("uploads", "/") - .supergraph(PathBuf::from_iter([ - "tests", - "fixtures", - "file_upload", - "single_subgraph.graphql", - ])) .build() .run_test(|response| { insta::assert_json_snapshot!(response, @r###" @@ -100,12 +93,6 @@ async fn it_uploads_multiple_files() -> Result<(), BoxError> { .handler(make_handler!(helper::echo_files)) .request(request) .subgraph_mapping("uploads", "/") - .supergraph(PathBuf::from_iter([ - "tests", - "fixtures", - "file_upload", - "single_subgraph.graphql", - ])) .build() .run_test(move |response| { insta::assert_json_snapshot!(response, @r###" @@ -160,12 +147,6 @@ async fn it_uploads_a_massive_file() -> Result<(), BoxError> { .handler(make_handler!(helper::verify_stream).with_state((TEN_GB, 0xAA))) .request(request) .subgraph_mapping("uploads", "/") - .supergraph(PathBuf::from_iter([ - "tests", - "fixtures", - "file_upload", - "single_subgraph.graphql", - ])) .build() .run_test(|response| { insta::assert_json_snapshot!(response, @r###" @@ -193,9 +174,9 @@ async fn it_uploads_to_multiple_subgraphs() -> Result<(), BoxError> { "operations", Part::text( serde_json::json!({ - "query": "mutation SomeMutation($file0: Upload1, $file1: Upload2) { - file0: singleUpload1(file: $file0) { filename body } - file1: singleUpload2(file: $file1) { filename body } + "query": "mutation SomeMutation($file0: Upload, $file1: UploadClone) { + file0: singleUpload(file: $file0) { filename body } + file1: singleUploadClone(file: $file1) { filename body } }", "variables": { "file0": null, @@ -226,14 +207,8 @@ async fn it_uploads_to_multiple_subgraphs() -> Result<(), BoxError> { "/s2" => helper::echo_single_file )) .request(request) - .subgraph_mapping("uploads1", "/s1") - .subgraph_mapping("uploads2", "/s2") - .supergraph(PathBuf::from_iter([ - "tests", - "fixtures", - "file_upload", - "multiple_subgraph.graphql", - ])) + .subgraph_mapping("uploads", "/s1") + .subgraph_mapping("uploads_clone", "/s2") .build() .run_test(|response| { insta::assert_json_snapshot!(response, @r###" @@ -381,12 +356,6 @@ async fn it_supports_compression() -> Result<(), BoxError> { .handler(make_handler!(helper::echo_single_file)) .request(Form::new()) // Gets overwritten by the `compress` transformer .subgraph_mapping("uploads", "/") - .supergraph(PathBuf::from_iter([ - "tests", - "fixtures", - "file_upload", - "single_subgraph.graphql", - ])) .transformer(compress) .build() .run_test(|request| { @@ -404,6 +373,185 @@ async fn it_supports_compression() -> Result<(), BoxError> { .await } +#[tokio::test(flavor = "multi_thread")] +async fn it_supports_non_nullable_file() -> Result<(), BoxError> { + use reqwest::multipart::Form; + use reqwest::multipart::Part; + + // Construct a manual request for non nullable checks + let request = Form::new() + .part( + "operations", + Part::text( + serde_json::json!({ + "query": "mutation SomeMutation($file0: Upload!) { + file0: singleUploadNonNull(file: $file0) { filename body } + }", + "variables": { + "file0": null, + }, + }) + .to_string(), + ), + ) + .part( + "map", + Part::text( + serde_json::json!({ + "0": ["variables.file0"], + }) + .to_string(), + ), + ) + .part("0", Part::text("file0 contents").file_name("file0")); + + helper::FileUploadTestServer::builder() + .config(FILE_CONFIG) + .handler(make_handler!(helper::echo_single_file)) + .request(request) + .subgraph_mapping("uploads", "/") + .build() + .run_test(|request| { + insta::assert_json_snapshot!(request, @r###" + { + "data": { + "file0": { + "filename": "file0", + "body": "file0 contents" + } + } + } + "###); + }) + .await +} + +#[tokio::test(flavor = "multi_thread")] +async fn it_supports_nested_file() -> Result<(), BoxError> { + use reqwest::multipart::Form; + use reqwest::multipart::Part; + + // Construct a manual request that sets up a nested structure containing a file to upload + let request = Form::new() + .part( + "operations", + Part::text( + serde_json::json!({ + "query": "mutation SomeMutation($file0: NestedUpload) { + file0: nestedUpload(nested: $file0) { filename body } + }", + "variables": { + "file0": { + "file": null, + }, + }, + }) + .to_string(), + ), + ) + .part( + "map", + Part::text( + serde_json::json!({ + "0": ["variables.file0.file"], + }) + .to_string(), + ), + ) + .part("0", Part::text("file0 contents").file_name("file0")); + + helper::FileUploadTestServer::builder() + .config(FILE_CONFIG) + .handler(make_handler!(helper::echo_single_file)) + .request(request) + .subgraph_mapping("uploads", "/") + .build() + .run_test(|request| { + insta::assert_json_snapshot!(request, @r###" + { + "data": { + "file0": { + "filename": "file0", + "body": "file0 contents" + } + } + } + "###); + }) + .await +} + +#[tokio::test(flavor = "multi_thread")] +async fn it_supports_nested_file_list() -> Result<(), BoxError> { + use reqwest::multipart::Form; + use reqwest::multipart::Part; + + // Construct a manual request that sets up a nested structure containing a file to upload + let request = Form::new() + .part( + "operations", + Part::text( + serde_json::json!({ + "query": "mutation SomeMutation($files: [Upload!]!) { + files: multiUpload(files: $files) { filename body } + }", + "variables": { + "files": { + "0": null, + "1": null, + "2": null, + }, + }, + }) + .to_string(), + ), + ) + .part( + "map", + Part::text( + serde_json::json!({ + "0": ["variables.files.0"], + "1": ["variables.files.1"], + "2": ["variables.files.2"], + }) + .to_string(), + ), + ) + .part("0", Part::text("file0 contents").file_name("file0")) + .part("1", Part::text("file1 contents").file_name("file1")) + .part("2", Part::text("file2 contents").file_name("file2")); + + helper::FileUploadTestServer::builder() + .config(FILE_CONFIG) + .handler(make_handler!(helper::echo_file_list)) + .request(request) + .subgraph_mapping("uploads", "/") + .build() + .run_test(|request| { + insta::assert_json_snapshot!(request, @r###" + { + "data": { + "files": [ + { + "filename": "file0", + "body": "file0 contents" + }, + { + "filename": "file1", + "body": "file1 contents" + }, + { + "filename": "file2", + "body": "file2 contents" + } + ] + } + } + "###); + }) + .await +} + #[tokio::test(flavor = "multi_thread")] async fn it_fails_upload_without_file() -> Result<(), BoxError> { // Construct a request with no attached files @@ -415,12 +563,6 @@ async fn it_fails_upload_without_file() -> Result<(), BoxError> { .handler(make_handler!(helper::always_fail)) .request(request) .subgraph_mapping("uploads", "/") - .supergraph(PathBuf::from_iter([ - "tests", - "fixtures", - "file_upload", - "single_subgraph.graphql", - ])) .build() .run_test(|response| { insta::assert_json_snapshot!(response, @r###" @@ -461,12 +603,6 @@ async fn it_fails_with_file_count_limits() -> Result<(), BoxError> { .handler(make_handler!(helper::always_fail)) .request(request) .subgraph_mapping("uploads", "/") - .supergraph(PathBuf::from_iter([ - "tests", - "fixtures", - "file_upload", - "single_subgraph.graphql", - ])) .build() .run_test(|response| { insta::assert_json_snapshot!(response, @r###" @@ -505,12 +641,6 @@ async fn it_fails_with_file_size_limit() -> Result<(), BoxError> { .handler(make_handler!(helper::always_fail)) .request(request) .subgraph_mapping("uploads", "/") - .supergraph(PathBuf::from_iter([ - "tests", - "fixtures", - "file_upload", - "single_subgraph.graphql", - ])) .build() .run_test(|response| { insta::assert_json_snapshot!(response, @r###" @@ -561,12 +691,6 @@ async fn it_fails_invalid_multipart_order() -> Result<(), BoxError> { .handler(make_handler!(helper::always_fail)) .request(request) .subgraph_mapping("uploads", "/") - .supergraph(PathBuf::from_iter([ - "tests", - "fixtures", - "file_upload", - "single_subgraph.graphql", - ])) .build() .run_test(|response| { insta::assert_json_snapshot!(response, @r###" @@ -596,9 +720,9 @@ async fn it_fails_invalid_file_order() -> Result<(), BoxError> { "operations", Part::text( serde_json::json!({ - "query": "mutation ($file0: Upload1, $file1: Upload2) { - file0: singleUpload1(file: $file0) { filename body } - file1: singleUpload2(file: $file1) { filename body } + "query": "mutation ($file0: Upload, $file1: UploadClone) { + file0: singleUpload(file: $file0) { filename body } + file1: singleUploadClone(file: $file1) { filename body } }", "variables": { "file0": null, @@ -629,14 +753,8 @@ async fn it_fails_invalid_file_order() -> Result<(), BoxError> { "/s2" => helper::always_fail )) .request(request) - .subgraph_mapping("uploads1", "/s1") - .subgraph_mapping("uploads2", "/s2") - .supergraph(PathBuf::from_iter([ - "tests", - "fixtures", - "file_upload", - "multiple_subgraph.graphql", - ])) + .subgraph_mapping("uploads", "/s1") + .subgraph_mapping("uploads_clone", "/s2") .build() .run_test(|response| { insta::assert_json_snapshot!(response, @r###" @@ -650,12 +768,12 @@ async fn it_fails_invalid_file_order() -> Result<(), BoxError> { }, "errors": [ { - "message": "HTTP fetch failed from 'uploads2': HTTP fetch failed from 'uploads2': error from user's HttpBody stream: error reading a body from connection: Missing files in the request: '1'.", + "message": "HTTP fetch failed from 'uploads_clone': HTTP fetch failed from 'uploads_clone': error from user's HttpBody stream: error reading a body from connection: Missing files in the request: '1'.", "path": [], "extensions": { "code": "SUBREQUEST_HTTP_ERROR", - "service": "uploads2", - "reason": "HTTP fetch failed from 'uploads2': error from user's HttpBody stream: error reading a body from connection: Missing files in the request: '1'." + "service": "uploads_clone", + "reason": "HTTP fetch failed from 'uploads_clone': error from user's HttpBody stream: error reading a body from connection: Missing files in the request: '1'." } } ] @@ -689,12 +807,6 @@ async fn it_fails_with_no_boundary_in_multipart() -> Result<(), BoxError> { .handler(make_handler!(helper::always_fail)) .request(request) .subgraph_mapping("uploads", "/") - .supergraph(PathBuf::from_iter([ - "tests", - "fixtures", - "file_upload", - "single_subgraph.graphql", - ])) .transformer(strip_boundary) .build() .run_test(|response| { @@ -727,9 +839,9 @@ async fn it_fails_incompatible_query_order() -> Result<(), BoxError> { "operations", Part::text( serde_json::json!({ - "query": "mutation SomeMutation($file0: Upload2, $file1: Upload1) { - file1: singleUpload1(file: $file1) { filename } - file0: singleUpload2(file: $file0) { filename } + "query": "mutation SomeMutation($file0: UploadClone, $file1: Upload) { + file1: singleUpload(file: $file1) { filename } + file0: singleUploadClone(file: $file0) { filename } }", "variables": { "file0": null, @@ -760,14 +872,8 @@ async fn it_fails_incompatible_query_order() -> Result<(), BoxError> { "/s2" => helper::always_fail )) .request(request) - .subgraph_mapping("uploads1", "/s1") - .subgraph_mapping("uploads2", "/s2") - .supergraph(PathBuf::from_iter([ - "tests", - "fixtures", - "file_upload", - "multiple_subgraph.graphql", - ])) + .subgraph_mapping("uploads", "/s1") + .subgraph_mapping("uploads_clone", "/s2") .build() .run_test(|response| { insta::assert_json_snapshot!(response, @r###" @@ -834,7 +940,6 @@ mod helper { handler: Router, request: Form, subgraph_mappings: HashMap, - supergraph: PathBuf, transformer: Option reqwest::Request>, } @@ -851,7 +956,6 @@ mod helper { handler: Router, subgraph_mappings: HashMap, request: Form, - supergraph: PathBuf, transformer: Option reqwest::Request>, ) -> Self { Self { @@ -859,7 +963,6 @@ mod helper { handler, request, subgraph_mappings, - supergraph, transformer, } } @@ -894,7 +997,12 @@ mod helper { .map(|(name, path)| (name, format!("{bound_url}{path}"))) .collect(), ) - .supergraph(self.supergraph) + .supergraph(PathBuf::from_iter([ + "tests", + "fixtures", + "file_upload", + "schema.graphql", + ])) .build() .await; @@ -959,6 +1067,9 @@ mod helper { #[error("expected a file with name '{0}' but found nothing")] MissingFile(String), + #[error("expected a list of files but found nothing")] + MissingList, + #[error("expected a set of mappings but found nothing")] MissingMapping, @@ -1141,6 +1252,69 @@ mod helper { }))) } + /// Handler that echos back the contents of the list of files that it receives + pub async fn echo_file_list( + mut request: Request, + ) -> Result, FileUploadError> { + let (operation, map, mut multipart) = decode_request(&mut request).await?; + + // Make sure that we have some mappings + if map.is_empty() { + return Err(FileUploadError::MissingMapping); + } + + // Make sure that we have one list input + let file_list = { + let Some((_, list)) = operation.variables.first_key_value() else { + return Err(FileUploadError::MissingList); + }; + + let Some(list) = list.as_object() else { + return Err(FileUploadError::MissingList); + }; + + list + }; + + // Make sure that the list has the correct amount of slots for the files + if file_list.len() != map.len() { + return Err(FileUploadError::VariableMismatch( + map.len(), + file_list.len(), + )); + } + + // Extract all of the files + let mut files = Vec::new(); + for file_mapping in map.into_keys() { + let f = multipart + .next_field() + .await? + .ok_or(FileUploadError::MissingFile(file_mapping.clone()))?; + + let field_name = f + .name() + .and_then(|name| (name == file_mapping).then_some(name)) + .ok_or(FileUploadError::UnexpectedField( + file_mapping, + f.name().map(String::from), + ))?; + let file_name = f.file_name().unwrap_or(field_name).to_string(); + let body = f.bytes().await?; + + files.push(Upload { + filename: Some(file_name), + body: Some(String::from_utf8_lossy(&body).to_string()), + }); + } + + Ok(Json(json!({ + "data": { + "files": files, + }, + }))) + } + /// A handler that always fails. Useful for tests that should not reach the subgraph at all. pub async fn always_fail(mut request: Request) -> Result, FileUploadError> { // Consume the stream diff --git a/apollo-router/tests/lifecycle_tests.rs b/apollo-router/tests/lifecycle_tests.rs index cd05f58ad7..4a1c9459f0 100644 --- a/apollo-router/tests/lifecycle_tests.rs +++ b/apollo-router/tests/lifecycle_tests.rs @@ -179,7 +179,7 @@ async fn test_shutdown_with_idle_connection() -> Result<(), BoxError> { .await; router.start().await; router.assert_started().await; - let _conn = std::net::TcpStream::connect("127.0.0.1:4000").unwrap(); + let _conn = std::net::TcpStream::connect(router.bind_addr).unwrap(); router.execute_default_query().await; tokio::time::timeout(Duration::from_secs(1), router.graceful_shutdown()) .await diff --git a/apollo-router/tests/telemetry/metrics.rs b/apollo-router/tests/telemetry/metrics.rs index 5d859a1e40..a195566b17 100644 --- a/apollo-router/tests/telemetry/metrics.rs +++ b/apollo-router/tests/telemetry/metrics.rs @@ -135,10 +135,11 @@ async fn test_bad_queries() { None, ) .await; - router.execute_bad_content_encoding().await; + router.execute_bad_content_type().await; + router .assert_metrics_contains( - r#"apollo_router_http_requests_total{error="unknown content-encoding header value \"garbage\"",status="400",otel_scope_name="apollo/router"}"#, + r#"apollo_router_http_requests_total{error="'content-type' header must be one of: \"application/json\" or \"application/graphql-response+json\"",status="415",otel_scope_name="apollo/router"}"#, None, ) .await; diff --git a/dockerfiles/tracing/docker-compose.datadog.yml b/dockerfiles/tracing/docker-compose.datadog.yml index 7914c7c671..8a06049b32 100644 --- a/dockerfiles/tracing/docker-compose.datadog.yml +++ b/dockerfiles/tracing/docker-compose.datadog.yml @@ -3,7 +3,7 @@ services: apollo-router: container_name: apollo-router - image: ghcr.io/apollographql/router:v1.40.1 + image: ghcr.io/apollographql/router:v1.41.1 volumes: - ./supergraph.graphql:/etc/config/supergraph.graphql - ./router/datadog.router.yaml:/etc/config/configuration.yaml diff --git a/dockerfiles/tracing/docker-compose.jaeger.yml b/dockerfiles/tracing/docker-compose.jaeger.yml index 84d4effa5e..4dd012e7ff 100644 --- a/dockerfiles/tracing/docker-compose.jaeger.yml +++ b/dockerfiles/tracing/docker-compose.jaeger.yml @@ -4,7 +4,7 @@ services: apollo-router: container_name: apollo-router #build: ./router - image: ghcr.io/apollographql/router:v1.40.1 + image: ghcr.io/apollographql/router:v1.41.1 volumes: - ./supergraph.graphql:/etc/config/supergraph.graphql - ./router/jaeger.router.yaml:/etc/config/configuration.yaml diff --git a/dockerfiles/tracing/docker-compose.zipkin.yml b/dockerfiles/tracing/docker-compose.zipkin.yml index 4fefad7d24..4b5a59d619 100644 --- a/dockerfiles/tracing/docker-compose.zipkin.yml +++ b/dockerfiles/tracing/docker-compose.zipkin.yml @@ -4,7 +4,7 @@ services: apollo-router: container_name: apollo-router build: ./router - image: ghcr.io/apollographql/router:v1.40.1 + image: ghcr.io/apollographql/router:v1.41.1 volumes: - ./supergraph.graphql:/etc/config/supergraph.graphql - ./router/zipkin.router.yaml:/etc/config/configuration.yaml diff --git a/docs/shared/redis-tls.mdx b/docs/shared/redis-tls.mdx new file mode 100644 index 0000000000..9e9e9a2e8f --- /dev/null +++ b/docs/shared/redis-tls.mdx @@ -0,0 +1,16 @@ +For Redis TLS connections, you can set up a client certificate or override the root certificate authority by configuring `tls` in your router's [YAML config file](https://www.apollographql.com/docs/router/overview/#yaml-config-file). For example: + +```yaml +apq: + router: + cache: + redis: + urls: [ "rediss://redis.example.com:6379" ] + #highlight-start + tls: + certificate_authorities: ${file./path/to/ca.crt} + client_authentication: + certificate_chain: ${file./path/to/certificate_chain.pem} + key: ${file./path/to/key.pem} + #highlight-end +``` diff --git a/docs/source/configuration/authn-jwt.mdx b/docs/source/configuration/authn-jwt.mdx index 626d2be233..a2d43144d2 100644 --- a/docs/source/configuration/authn-jwt.mdx +++ b/docs/source/configuration/authn-jwt.mdx @@ -47,9 +47,19 @@ Otherwise, if you issue JWTs via a popular third-party IdP (Auth0, Okta, PingOne - url: https://dev-zzp5enui.us.auth0.com/.well-known/jwks.json issuer: poll_interval: + headers: # optional list of static headers added to the HTTP request to the JWKS URL + - name: User-Agent + value: router # These keys are optional. Default values are shown. header_name: Authorization header_value_prefix: Bearer + # array of alternative token sources + sources: + - type: header + name: X-Authorization + value_prefix: Bearer + - type: cookie + name: authz ``` These options are documented [below](#configuration-options). @@ -99,6 +109,7 @@ The following configuration options are supported: - `issuer`: **optional** name of the issuer, that will be compared to the `iss` claim in the JWT if present. If it does not match, the request will be rejected. - `algorithms`: **optional** list of accepted algorithms. Possible values are `HS256`, `HS384`, `HS512`, `ES256`, `ES384`, `RS256`, `RS384`, `RS512`, `PS256`, `PS384`, `PS512`, `EdDSA` - `poll_interval`: **optional** interval in human-readable format (e.g. `60s` or `1hour 30s`) at which the JWKS will be polled for changes. If not specified, the JWKS endpoint will be polled every 60 seconds. +- `headers`: **optional** a list of headers sent when downloading from the JWKS URL @@ -133,6 +144,33 @@ The default value is `Bearer`. + + + +##### `sources` + + + + +This is an array of possible token sources, as it could be provided in different headers depending on the client, or it could be stored in a cookie. If the default token source defined by the above `header_name` and `header_value_prefix` does not find the token, then each of the alternative sources is tried until one matches. + +```yaml title="router.yaml" + authentication: + router: + jwt: + jwks: + - url: https://dev-zzp5enui.us.auth0.com/.well-known/jwks.json + sources: + - type: header + name: X-Authorization + value_prefix: Bearer + - type: cookie + name: authz +``` + + + + @@ -629,7 +667,7 @@ The y-coordinate of the elliptic curve point for this public key, as the base64- // highlight-start // Symmetric-algorithm-specific property - "k": "c2VjcmV0Cg==" // ⚠️ This is a base64-encoded shared secret! ⚠️ + "k": "c2VjcmV0Cg" // ⚠️ This is a base64-encoded shared secret! ⚠️ // highlight-end } ``` @@ -651,7 +689,7 @@ The y-coordinate of the elliptic curve point for this public key, as the base64- -The value of the shared symmetric key, as the base64-encoded value of the key's octet sequence representation. +The value of the shared symmetric key, as the [(URL safe, without padding) base64-encoded value](https://datatracker.ietf.org/doc/html/rfc7515#section-2) of the key's octet sequence representation. **⚠️ If your JWK uses a symmetric signature algorithm, always [provide your JWKS to the router](#jwks) via a `file://` URL!** Shared keys should never be made available over the network. diff --git a/docs/source/configuration/authorization.mdx b/docs/source/configuration/authorization.mdx index 190d2aa8d2..938fb306fa 100644 --- a/docs/source/configuration/authorization.mdx +++ b/docs/source/configuration/authorization.mdx @@ -289,6 +289,40 @@ query { The response would include an error at the `/users/@/email` path since that field requires the `read:emails` scope. The router can execute the entire query successfully if the request includes the `read:others read:emails` scope set. +The router returns `null` for unauthorized fields and applies the [standard GraphQL null propagation rules](https://www.apollographql.com/blog/graphql/basics/using-nullability-in-graphql/#what-happens-if-you-try-to-return-null-for-a-non-null-field). + +```json title="Unauthorized request response" +{ + "data": { + "me": null, + "post": { + "title": "Securing supergraphs", + } + }, + "errors": [ + { + "message": "Unauthorized field or type", + "path": [ + "me" + ], + "extensions": { + "code": "UNAUTHORIZED_FIELD_OR_TYPE" + } + }, + { + "message": "Unauthorized field or type", + "path": [ + "post", + "views" + ], + "extensions": { + "code": "UNAUTHORIZED_FIELD_OR_TYPE" + } + } + ] +} +``` + ### `@authenticated` @@ -499,7 +533,7 @@ You can nest arrays and elements as needed to achieve your desired logic. For th #### Example `@policy` use case -#### Usage with a coprocessor +##### Usage with a coprocessor Diving even deeper into the [social media example](#example-requiresscopes-use-case): suppose you want only a user to have access to their own profile and credit card information. Of the available authorization directives, you use `@policy` instead of `@requiresScopes` because the validation logic relies on more than the scopes of an access token. @@ -635,6 +669,10 @@ fn supergraph_service(service) { } ``` +#### Special case for subscriptions + +When using subscriptions along with `@policy` authorization, subscription events restart from the execution service, which means that if the authorization status of the subscription session changed, then it cannot go through query planning again, and the session should be closed. To that end, the policies should be evaluated again at the execution service level, and if they changed, an error should be returned to stop the subscription. + ## Composition and federation GraphOS's composition strategy for authorization directives is intentionally accumulative. When you define authorization directives on fields and types in subgraphs, GraphOS composes them into the supergraph schema. In other words, if subgraph fields or types include `@requiresScopes`, `@authenticated`, or `@policy` directives, they are set on the supergraph too. diff --git a/docs/source/configuration/distributed-caching.mdx b/docs/source/configuration/distributed-caching.mdx index c3a836a18d..147fddff1b 100644 --- a/docs/source/configuration/distributed-caching.mdx +++ b/docs/source/configuration/distributed-caching.mdx @@ -4,6 +4,8 @@ subtitle: Redis-backed caching for query plans and APQ description: Distributed caching for Apollo Router with GraphOS Enterprise. Configure a Redis-backed cache for query plans and automatic persisted queries (APQ). --- +import RedisTLS from '../../shared/redis-tls.mdx' + If you have multiple Apollo Router instances, those instances can share a Redis-backed cache for their query plans and automatic persisted queries (APQ). This means that if _any_ of your router instances caches a particular value, _all_ of your instances can look up that value to significantly improve responsiveness. For more details on query plans and APQ, see the article on [in-memory caching](./in-memory-caching). @@ -151,8 +153,8 @@ When using the same Redis instance for multiple puposes, the `namespace` option ### TLS -TLS for Redis connections can be configured in the same way as subgraphs, to [override the list of certificate authorities](./overview/#overriding-certificate-authorities-for-subgraphs) or [set up client authentication](./overview/#tls-client-authentication-for-subgraph-requests). + ### Required to start -When active, the `required_to_start` option will prevent the Router from starting if it cannot connect to Redis. By default, the Router will still start without a connection to Redis, which would result in only using the in-memory cache for APQ and query planning, and entity caching sending the requestsz to subgraphs undisturbed. \ No newline at end of file +When active, the `required_to_start` option will prevent the Router from starting if it cannot connect to Redis. By default, the Router will still start without a connection to Redis, which would result in only using the in-memory cache for APQ and query planning, and entity caching sending the requestsz to subgraphs undisturbed. diff --git a/docs/source/configuration/overview.mdx b/docs/source/configuration/overview.mdx index 68a80f1425..aac9c1cae3 100644 --- a/docs/source/configuration/overview.mdx +++ b/docs/source/configuration/overview.mdx @@ -3,6 +3,8 @@ title: Configuring the Apollo Router description: Command arguments and YAML config --- +import RedisTLS from '../../shared/redis-tls.mdx' + Learn how to customize the behavior of your Apollo Router with environment variables, command-line commands and options, and YAML file configuration. ## Environment variables @@ -712,6 +714,10 @@ tls: key: ${file./path/to/key.pem} ``` +#### Redis TLS configuration + + + ### Request limits The Apollo Router supports enforcing three types of request limits for enhanced security: diff --git a/docs/source/configuration/telemetry/exporters/metrics/overview.mdx b/docs/source/configuration/telemetry/exporters/metrics/overview.mdx index b9bd7e006c..45e67483c6 100644 --- a/docs/source/configuration/telemetry/exporters/metrics/overview.mdx +++ b/docs/source/configuration/telemetry/exporters/metrics/overview.mdx @@ -118,7 +118,7 @@ telemetry: You can add custom attributes (OpenTelemetry) and labels (Prometheus) to the `apollo_router_http_requests` metric. Attributes can be: -* static values (preferably using a [resource](#resource-attribute)) +* static values (preferably using a [resource](#resource)) * headers from the request or response * a value from a context * a value from the request or response body ([JSON path](https://goessner.net/articles/JsonPath/)) diff --git a/docs/source/configuration/telemetry/exporters/tracing/datadog.mdx b/docs/source/configuration/telemetry/exporters/tracing/datadog.mdx index fe56622028..b889e42e3c 100644 --- a/docs/source/configuration/telemetry/exporters/tracing/datadog.mdx +++ b/docs/source/configuration/telemetry/exporters/tracing/datadog.mdx @@ -41,6 +41,22 @@ telemetry: For more details about Datadog configuration, see [Datadog Agent configuration](https://docs.datadoghq.com/opentelemetry/otlp_ingest_in_the_agent/?tab=host). +### Enabling log correlation + +To enable Datadog log correlation, you must configure `dd.trace_id` to appear on the `router` span: + +```yaml title="router.yaml" +telemetry: + instrumentation: + spans: + mode: spec_compliant + router: + attributes: + dd.trace_id: true #highlight-line +``` + +Your JSON formatted log messages will automatically output `dd.trace_id` on each log message if `dd.trace_id` was detected on the `router` span. + ## Datadog native configuration diff --git a/docs/source/customizations/native.mdx b/docs/source/customizations/native.mdx index a588870d1c..ab18e3bcd9 100644 --- a/docs/source/customizations/native.mdx +++ b/docs/source/customizations/native.mdx @@ -281,24 +281,6 @@ tracing::info!( ); ``` -## Logging - -The Router has a logging infrastructure based on the [tracing macros](https://docs.rs/tracing/latest/tracing/index.html#macros), like metrics, and those should be the preferred way to write logs, as they will be configurable through the Router's telemetry infrastructure. - -Example usage: - -```rust -tracing::info!(service_name = name, "received request"); -``` - -If writing to stdout or stderr directly is still needed, additional steps should be taken to prevent errors: due to the Router using non blocking IO for stdout and stderr, `println!` and `eprintln!` cannot be used, as they would panic if those streams were blocking. -As an example, writing to stdout would be done like this: - -```rust -use std::io::Write; -let _ = writeln!(std::io::stdout(), "value={}", 1); -``` - ## Plugin Lifecycle Like individual requests, plugins follow their own strict lifecycle that helps provide structure to the Apollo Router's execution. diff --git a/helm/chart/router/Chart.yaml b/helm/chart/router/Chart.yaml index 18abc116e9..9c2aace167 100644 --- a/helm/chart/router/Chart.yaml +++ b/helm/chart/router/Chart.yaml @@ -20,10 +20,10 @@ type: application # so it matches the shape of our release process and release automation. # By proxy of that decision, this version uses SemVer 2.0.0, though the prefix # of "v" is not included. -version: 1.40.1 +version: 1.41.1 # This is the version number of the application being deployed. This version number should be # incremented each time you make changes to the application. Versions are not expected to # follow Semantic Versioning. They should reflect the version the application is using. # It is recommended to use it with quotes. -appVersion: "v1.40.1" +appVersion: "v1.41.1" diff --git a/helm/chart/router/README.md b/helm/chart/router/README.md index ca7386b099..c9d4fe3d73 100644 --- a/helm/chart/router/README.md +++ b/helm/chart/router/README.md @@ -2,7 +2,7 @@ [router](https://github.com/apollographql/router) Rust Graph Routing runtime for Apollo Federation -![Version: 1.40.1](https://img.shields.io/badge/Version-1.40.1-informational?style=flat-square) ![Type: application](https://img.shields.io/badge/Type-application-informational?style=flat-square) ![AppVersion: v1.40.1](https://img.shields.io/badge/AppVersion-v1.40.1-informational?style=flat-square) +![Version: 1.41.1](https://img.shields.io/badge/Version-1.41.1-informational?style=flat-square) ![Type: application](https://img.shields.io/badge/Type-application-informational?style=flat-square) ![AppVersion: v1.41.1](https://img.shields.io/badge/AppVersion-v1.41.1-informational?style=flat-square) ## Prerequisites @@ -11,7 +11,7 @@ ## Get Repo Info ```console -helm pull oci://ghcr.io/apollographql/helm-charts/router --version 1.40.1 +helm pull oci://ghcr.io/apollographql/helm-charts/router --version 1.41.1 ``` ## Install Chart @@ -19,7 +19,7 @@ helm pull oci://ghcr.io/apollographql/helm-charts/router --version 1.40.1 **Important:** only helm3 is supported ```console -helm upgrade --install [RELEASE_NAME] oci://ghcr.io/apollographql/helm-charts/router --version 1.40.1 --values my-values.yaml +helm upgrade --install [RELEASE_NAME] oci://ghcr.io/apollographql/helm-charts/router --version 1.41.1 --values my-values.yaml ``` _See [configuration](#configuration) below._ diff --git a/licenses.html b/licenses.html index adf542cc66..ee72e6e6da 100644 --- a/licenses.html +++ b/licenses.html @@ -3834,6 +3834,7 @@

Used by:

  • anstyle
  • anstyle-query
  • anstyle-wincon
  • +
  • bytesize
  • clap
  • colorchoice
  • concolor
  • @@ -11339,15 +11340,6 @@

    Used by:

    additional terms or conditions. -
  • -

    Apache License 2.0

    -

    Used by:

    - -
    ../../LICENSE-APACHE
    -
  • Apache License 2.0

    Used by:

    @@ -11574,7 +11566,6 @@

    Used by:

    Apache License 2.0

    Used by:

    @@ -11995,8 +11986,11 @@

    Used by:

    Apache License 2.0

    Used by: