From 7b33bdd703aff60cc8e7a35dca2297c7cdaee12b Mon Sep 17 00:00:00 2001 From: Adrien Guillo Date: Thu, 7 May 2026 14:39:04 -0400 Subject: [PATCH] Add distributed tracing to gRPC stack --- quickwit/Cargo.lock | 5 + quickwit/quickwit-codegen/example/Cargo.toml | 2 + .../example/src/codegen/hello.rs | 107 +- quickwit/quickwit-codegen/src/codegen.rs | 178 ++- quickwit/quickwit-common/Cargo.toml | 2 + quickwit/quickwit-common/src/tower/mod.rs | 1 + .../src/tower/one_task_per_call_layer.rs | 8 +- quickwit/quickwit-common/src/tower/otel.rs | 135 ++ quickwit/quickwit-ingest/Cargo.toml | 1 + .../src/codegen/ingest_service.rs | 99 +- .../src/ingest_v2/broadcast/capacity_score.rs | 49 +- .../src/ingest_v2/broadcast/local_shards.rs | 62 +- .../src/ingest_v2/doc_mapper.rs | 118 +- .../quickwit-ingest/src/ingest_v2/idle.rs | 81 +- .../quickwit-ingest/src/ingest_v2/ingester.rs | 4 +- .../src/ingest_v2/mrecordlog_utils.rs | 11 + .../src/ingest_v2/replication.rs | 10 +- .../quickwit-ingest/src/ingest_v2/state.rs | 10 +- .../quickwit-ingest/src/mrecordlog_async.rs | 100 +- .../src/metastore/control_plane_metastore.rs | 44 + .../src/metastore/file_backed/mod.rs | 44 + .../src/metastore/postgres/metastore.rs | 15 + quickwit/quickwit-proto/build.rs | 7 + .../src/codegen/quickwit/quickwit.cluster.rs | 33 +- .../quickwit/quickwit.control_plane.rs | 330 +++- .../codegen/quickwit/quickwit.developer.rs | 33 +- .../src/codegen/quickwit/quickwit.indexing.rs | 33 +- .../quickwit/quickwit.ingest.ingester.rs | 320 +++- .../quickwit/quickwit.ingest.router.rs | 39 +- .../codegen/quickwit/quickwit.metastore.rs | 1423 +++++++++++++---- quickwit/quickwit-proto/src/lib.rs | 88 - quickwit/quickwit-search/src/client.rs | 3 +- .../src/search_api/grpc_adapter.rs | 25 +- .../src/local_file_storage.rs | 6 + .../src/object_storage/azure_blob_storage.rs | 5 + .../object_storage/s3_compatible_storage.rs | 5 + .../src/opendal_storage/base.rs | 9 + 37 files changed, 2560 insertions(+), 885 deletions(-) create mode 100644 quickwit/quickwit-common/src/tower/otel.rs diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index df5058a551c..a3785627af6 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -8248,6 +8248,8 @@ dependencies = [ "tonic 0.14.5", "tonic-prost", "tower 0.5.3", + "tracing", + "tracing-opentelemetry", "utoipa", ] @@ -8272,6 +8274,7 @@ dependencies = [ "hyper 1.9.0", "hyper-util", "itertools 0.14.0", + "opentelemetry", "pin-project", "pnet", "prometheus", @@ -8293,6 +8296,7 @@ dependencies = [ "tonic 0.14.5", "tower 0.5.3", "tracing", + "tracing-opentelemetry", ] [[package]] @@ -8608,6 +8612,7 @@ dependencies = [ "tonic-prost", "tower 0.5.3", "tracing", + "tracing-opentelemetry", "ulid", "utoipa", ] diff --git a/quickwit/quickwit-codegen/example/Cargo.toml b/quickwit/quickwit-codegen/example/Cargo.toml index 0d21ff6bf9e..b50900438c7 100644 --- a/quickwit/quickwit-codegen/example/Cargo.toml +++ b/quickwit/quickwit-codegen/example/Cargo.toml @@ -25,6 +25,8 @@ tokio-stream = { workspace = true } tonic = { workspace = true } tonic-prost = { workspace = true } tower = { workspace = true } +tracing = { workspace = true } +tracing-opentelemetry = { workspace = true } utoipa = { workspace = true } quickwit-actors = { workspace = true } diff --git a/quickwit/quickwit-codegen/example/src/codegen/hello.rs b/quickwit/quickwit-codegen/example/src/codegen/hello.rs index bacf8b3edac..af0657a0042 100644 --- a/quickwit/quickwit-codegen/example/src/codegen/hello.rs +++ b/quickwit/quickwit-codegen/example/src/codegen/hello.rs @@ -168,15 +168,18 @@ impl HelloClient { } #[async_trait::async_trait] impl Hello for HelloClient { + #[tracing::instrument(skip_all, name = "hello.hello")] async fn hello(&self, request: HelloRequest) -> crate::HelloResult { self.inner.0.hello(request).await } + #[tracing::instrument(skip_all, name = "hello.goodbye")] async fn goodbye( &self, request: GoodbyeRequest, ) -> crate::HelloResult { self.inner.0.goodbye(request).await } + #[tracing::instrument(skip_all, name = "hello.ping")] async fn ping( &self, request: quickwit_common::ServiceStream, @@ -742,9 +745,13 @@ where T::Future: Send, { async fn hello(&self, request: HelloRequest) -> crate::HelloResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .hello(request) + .hello(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -756,9 +763,13 @@ where &self, request: GoodbyeRequest, ) -> crate::HelloResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .goodbye(request) + .goodbye(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -770,9 +781,13 @@ where &self, request: quickwit_common::ServiceStream, ) -> crate::HelloResult> { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .ping(request) + .ping(tonic_request) .await .map(|response| { let streaming: tonic::Streaming<_> = response.into_inner(); @@ -822,42 +837,76 @@ impl HelloGrpcServerAdapter { impl hello_grpc_server::HelloGrpc for HelloGrpcServerAdapter { async fn hello( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .hello(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("hello.hello"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .hello(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn goodbye( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .goodbye(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("hello.goodbye"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .goodbye(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } type PingStream = quickwit_common::ServiceStream>; async fn ping( &self, - request: tonic::Request>, + tonic_request: tonic::Request>, ) -> Result, tonic::Status> { - self.inner - .0 - .ping({ - let streaming: tonic::Streaming<_> = request.into_inner(); - quickwit_common::ServiceStream::from(streaming) - }) - .await - .map(|stream| tonic::Response::new( - stream.map_err(crate::error::grpc_error_to_grpc_status), - )) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let streaming: tonic::Streaming<_> = tonic_request.into_inner(); + let request = quickwit_common::ServiceStream::from(streaming); + let span = tracing::info_span!("hello.ping"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .ping(request) + .await + .map(|stream| tonic::Response::new( + stream.map_err(crate::error::grpc_error_to_grpc_status), + )) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } } /// Generated client implementations. diff --git a/quickwit/quickwit-codegen/src/codegen.rs b/quickwit/quickwit-codegen/src/codegen.rs index a7d6d311be0..d8ef9df782c 100644 --- a/quickwit/quickwit-codegen/src/codegen.rs +++ b/quickwit/quickwit-codegen/src/codegen.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; + use anyhow::ensure; use heck::{ToSnakeCase, ToUpperCamelCase}; use proc_macro2::TokenStream; @@ -30,6 +32,7 @@ impl Codegen { args.error_type_path, args.generate_extra_service_methods, args.generate_prom_labels_for_requests, + args.traced_request_fields, )); args.prost_config .protoc_arg("--experimental_allow_proto3_optional") @@ -67,6 +70,12 @@ pub struct CodegenBuilder { error_type_path: String, generate_extra_service_methods: bool, generate_prom_labels_for_requests: bool, + /// Maps a request message type (short name, e.g. `IngestRequestV2`) to the + /// list of top-level fields that should be recorded on the generated + /// tracing span. Each entry pairs a field name with `use_debug_format`: + /// `false` emits `%request.` (Display), `true` emits `?request.` + /// (Debug). Default is empty: spans carry only the method name. + traced_request_fields: HashMap>, } impl CodegenBuilder { @@ -111,6 +120,38 @@ impl CodegenBuilder { self } + /// Records the value of a top-level scalar field of the given request + /// message on the generated tracing span using the field's `Display` + /// representation. Use [`Self::with_traced_request_field_debug`] for + /// types that only implement `Debug` (e.g. prost-generated enums). + /// + /// `message` is the short, unqualified Rust type name of the request + /// (e.g. `"IngestRequestV2"`). `field` is a top-level field name (no + /// nested paths). Repeat the call to record multiple fields. + pub fn with_traced_request_field(self, message: &str, field: &str) -> Self { + self.with_traced_request_field_inner(message, field, false) + } + + /// Like [`Self::with_traced_request_field`] but records the field via + /// its `Debug` representation. Use this for proto enums and other types + /// that don't implement `Display`. + pub fn with_traced_request_field_debug(self, message: &str, field: &str) -> Self { + self.with_traced_request_field_inner(message, field, true) + } + + fn with_traced_request_field_inner( + mut self, + message: &str, + field: &str, + use_debug_format: bool, + ) -> Self { + self.traced_request_fields + .entry(message.to_string()) + .or_default() + .push((field.to_string(), use_debug_format)); + self + } + pub fn run(self) -> anyhow::Result<()> { ensure!(!self.protos.is_empty(), "proto file list is empty"); ensure!(!self.output_dir.is_empty(), "output directory is undefined"); @@ -126,6 +167,7 @@ struct QuickwitServiceGenerator { error_type_path: String, generate_extra_service_methods: bool, generate_prom_labels_for_requests: bool, + traced_request_fields: HashMap>, inner: Box, } @@ -135,6 +177,7 @@ impl QuickwitServiceGenerator { error_type_path: String, generate_extra_service_methods: bool, generate_prom_labels_for_requests: bool, + traced_request_fields: HashMap>, ) -> Self { let inner = Box::new(WithSuffixServiceGenerator::new( "Grpc", @@ -145,6 +188,7 @@ impl QuickwitServiceGenerator { error_type_path, generate_extra_service_methods, generate_prom_labels_for_requests, + traced_request_fields, inner, } } @@ -158,6 +202,7 @@ impl ServiceGenerator for QuickwitServiceGenerator { &self.error_type_path, self.generate_extra_service_methods, self.generate_prom_labels_for_requests, + &self.traced_request_fields, ); let ast: syn::File = syn::parse2(tokens).expect("Tokenstream should be a valid Syn AST."); let pretty_code = prettyplease::unparse(&ast); @@ -194,6 +239,7 @@ struct CodegenContext { grpc_server_package_name: Ident, grpc_service_name: Ident, generate_extra_service_methods: bool, + traced_request_fields: HashMap>, } impl CodegenContext { @@ -202,6 +248,7 @@ impl CodegenContext { result_type_path: &str, error_type_path: &str, generate_extra_service_methods: bool, + traced_request_fields: HashMap>, ) -> Self { let service_name = quote::format_ident!("{}", service.name); let mock_mod_name = quote::format_ident!("mock_{}", service.name.to_snake_case()); @@ -264,6 +311,7 @@ impl CodegenContext { grpc_server_package_name, grpc_service_name, generate_extra_service_methods, + traced_request_fields, } } } @@ -274,12 +322,14 @@ fn generate_all( error_type_path: &str, generate_extra_service_methods: bool, generate_prom_labels_for_requests: bool, + traced_request_fields: &HashMap>, ) -> TokenStream { let context = CodegenContext::from_service( service, result_type_path, error_type_path, generate_extra_service_methods, + traced_request_fields.clone(), ); let stream_type_alias = &context.stream_type_alias; let service_trait = generate_service_trait(&context); @@ -399,6 +449,59 @@ impl SynMethod { } syn_methods } + + /// Short, unqualified Rust type name of the request message + /// (e.g. `IngestRequestV2`). Used as the lookup key for opt-in span + /// fields configured via `with_traced_request_field`. + fn request_type_short_name(&self) -> String { + self.request_type + .segments + .last() + .map(|segment| segment.ident.to_string()) + .unwrap_or_default() + } +} + +/// Builds the span name for a method: `"{package}.{method}"`, +/// where `{package}` is the proto package with any leading `quickwit.` +/// prefix stripped (e.g. `quickwit.ingest.ingester` becomes +/// `ingest.ingester`). The service name is omitted to keep span names +/// short and readable. +fn span_name(context: &CodegenContext, method: &SynMethod) -> String { + let package = context + .package_name + .strip_prefix("quickwit.") + .unwrap_or(&context.package_name); + format!("{}.{}", package, method.name) +} + +/// Builds the contents of `fields(...)` for a method's tracing span, +/// recording opt-in scalar fields from the request using their `Debug` +/// representation. Returns an empty `TokenStream` when the method has no +/// configured fields or has a streaming request (where the wrapping +/// `quickwit_common::ServiceStream` has no per-call value). +fn traced_request_fields_tokens(context: &CodegenContext, method: &SynMethod) -> TokenStream { + if method.client_streaming { + return TokenStream::new(); + } + let Some(fields) = context + .traced_request_fields + .get(&method.request_type_short_name()) + else { + return TokenStream::new(); + }; + let pieces: Vec = fields + .iter() + .map(|(field, use_debug_format)| { + let ident = quote::format_ident!("{field}"); + if *use_debug_format { + quote! { #ident = ?request.#ident } + } else { + quote! { #ident = %request.#ident } + } + }) + .collect(); + quote! { #(#pieces),* } } fn generate_prom_labels_impl_for_requests(context: &CodegenContext) -> TokenStream { @@ -676,7 +779,27 @@ fn generate_client_methods(context: &CodegenContext, mock: bool) -> TokenStream self.inner.lock().await.#method_name(request).await } }; + + // Spans go on the user-facing client only — not the mock wrapper, to + // keep test output free of incidental spans. + let instrument_attr = if !mock { + let span_name = span_name(context, syn_method); + let traced_fields = traced_request_fields_tokens(context, syn_method); + if traced_fields.is_empty() { + quote! { + #[tracing::instrument(skip_all, name = #span_name)] + } + } else { + quote! { + #[tracing::instrument(skip_all, name = #span_name, fields(#traced_fields))] + } + } + } else { + TokenStream::new() + }; + let method = quote! { + #instrument_attr async fn #method_name(&self, request: #request_type) -> #result_type<#response_type> { #body } @@ -1197,11 +1320,16 @@ fn generate_grpc_client_adapter_methods(context: &CodegenContext) -> TokenStream } else { quote! { |response| response.into_inner() } }; + // Wrap the request and inject the active span's W3C trace context into + // the gRPC metadata so the receiving server can stitch its span into + // the same trace. No-op when no propagator is installed (tests). let method = quote! { async fn #method_name(&self, request: #request_type) -> #result_type<#response_type> { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context(tonic_request.metadata_mut()); self.inner .clone() - .#method_name(request) + .#method_name(tonic_request) .await .map(#into_response_type) .map_err(|status| crate::error::grpc_status_to_service_error(status, #rpc_name)) @@ -1253,15 +1381,18 @@ fn generate_grpc_server_adapter_methods(context: &CodegenContext) -> TokenStream } else { syn_method.request_type.to_token_stream() }; - let method_arg = if syn_method.client_streaming { + // Bind the inner request *outside* the instrumented future so opt-in + // span fields can reference `request.` synchronously when the + // span is constructed. + let inner_binding = if syn_method.client_streaming { quote! { - { - let streaming: tonic::Streaming<_> = request.into_inner(); - quickwit_common::ServiceStream::from(streaming) - } + let streaming: tonic::Streaming<_> = tonic_request.into_inner(); + let request = quickwit_common::ServiceStream::from(streaming); } } else { - quote! { request.into_inner() } + quote! { + let request = tonic_request.into_inner(); + } }; let response_type = if syn_method.server_streaming { let associated_type_name = quote::format_ident!("{}Stream", syn_method.proto_name); @@ -1283,16 +1414,35 @@ fn generate_grpc_server_adapter_methods(context: &CodegenContext) -> TokenStream } else { quote! { tonic::Response::new } }; + let span_name = span_name(context, syn_method); + let traced_fields = traced_request_fields_tokens(context, syn_method); + let span_macro = if traced_fields.is_empty() { + quote! { tracing::info_span!(#span_name) } + } else { + quote! { tracing::info_span!(#span_name, #traced_fields) } + }; + // Extract the W3C trace context from incoming metadata and link the + // server-side span to the caller's trace via `set_parent`. Then run + // the inner handler inside that span. When no propagator is + // installed, the parent context is empty and the span becomes a + // local root. let method = quote! { #associated_type - async fn #method_name(&self, request: tonic::Request<#request_type>) -> Result, tonic::Status> { - self.inner - .0 - .#method_name(#method_arg) - .await - .map(#into_response_type) - .map_err(crate::error::grpc_error_to_grpc_status) + async fn #method_name(&self, tonic_request: tonic::Request<#request_type>) -> Result, tonic::Status> { + let parent_context = quickwit_common::tower::otel::extract_context(tonic_request.metadata()); + #inner_binding + let span = #span_macro; + let _ = ::set_parent(&span, parent_context); + let fut = async move { + self.inner + .0 + .#method_name(request) + .await + .map(#into_response_type) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } }; stream.extend(method); diff --git a/quickwit/quickwit-common/Cargo.toml b/quickwit/quickwit-common/Cargo.toml index 14c05e19c5e..761d1078371 100644 --- a/quickwit/quickwit-common/Cargo.toml +++ b/quickwit/quickwit-common/Cargo.toml @@ -28,6 +28,7 @@ http = { workspace = true } hyper = { workspace = true } hyper-util = { workspace = true, optional = true } itertools = { workspace = true } +opentelemetry = { workspace = true } pin-project = { workspace = true } pnet = { workspace = true } prometheus = { workspace = true } @@ -50,6 +51,7 @@ tonic = { workspace = true, features = [ ] } tower = { workspace = true } tracing = { workspace = true } +tracing-opentelemetry = { workspace = true } [features] testsuite = ["hyper-util"] diff --git a/quickwit/quickwit-common/src/tower/mod.rs b/quickwit/quickwit-common/src/tower/mod.rs index 1d62433e520..bb04a9711a3 100644 --- a/quickwit/quickwit-common/src/tower/mod.rs +++ b/quickwit/quickwit-common/src/tower/mod.rs @@ -23,6 +23,7 @@ mod event_listener; mod load_shed; mod metrics; mod one_task_per_call_layer; +pub mod otel; mod pool; mod rate; mod rate_estimator; diff --git a/quickwit/quickwit-common/src/tower/one_task_per_call_layer.rs b/quickwit/quickwit-common/src/tower/one_task_per_call_layer.rs index 0a9a6225c08..5ef1ff07463 100644 --- a/quickwit/quickwit-common/src/tower/one_task_per_call_layer.rs +++ b/quickwit/quickwit-common/src/tower/one_task_per_call_layer.rs @@ -20,7 +20,7 @@ use std::task::{Context, Poll}; use pin_project::pin_project; use tokio::task::{JoinError, JoinHandle}; use tower::{Layer, Service}; -use tracing::error; +use tracing::{Instrument, error}; use crate::tower::RpcName; @@ -72,7 +72,11 @@ where fn call(&mut self, request: Request) -> Self::Future { let request_name: &'static str = Request::rpc_name(); let future = self.service.call(request); - let join_handle = tokio::spawn(future); + // `tokio::spawn` does not propagate the caller's tracing span — task + // locals are reset on a fresh task. `in_current_span` re-enters the + // caller's span inside the spawned future so emitted events and + // child spans stay attached to the same trace. + let join_handle = tokio::spawn(future.in_current_span()); UnwrapOrElseFuture { request_name, join_handle, diff --git a/quickwit/quickwit-common/src/tower/otel.rs b/quickwit/quickwit-common/src/tower/otel.rs new file mode 100644 index 00000000000..40698d23346 --- /dev/null +++ b/quickwit/quickwit-common/src/tower/otel.rs @@ -0,0 +1,135 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! W3C trace context propagation over gRPC metadata. +//! +//! Used by the codegen-emitted gRPC client adapters (to inject) and server +//! adapters (to extract). Becomes a no-op when no global text-map propagator +//! is installed — desirable for tests. + +use std::str::FromStr; + +use opentelemetry::propagation::{Extractor, Injector}; +use opentelemetry::{Context, global}; +use tonic::Status; +use tonic::metadata::{KeyAndValueRef, MetadataKey, MetadataMap, MetadataValue}; +use tonic::service::Interceptor; +use tracing_opentelemetry::OpenTelemetrySpanExt; + +struct MetadataInjector<'a>(&'a mut MetadataMap); + +impl Injector for MetadataInjector<'_> { + fn set(&mut self, key: &str, value: String) { + let Ok(metadata_key) = MetadataKey::from_str(key) else { + return; + }; + let Ok(metadata_value) = MetadataValue::try_from(value) else { + return; + }; + self.0.insert(metadata_key, metadata_value); + } +} + +struct MetadataExtractor<'a>(&'a MetadataMap); + +impl Extractor for MetadataExtractor<'_> { + fn get(&self, key: &str) -> Option<&str> { + self.0.get(key).and_then(|value| value.to_str().ok()) + } + + fn keys(&self) -> Vec<&str> { + self.0 + .iter() + .filter_map(|key_and_value| match key_and_value { + KeyAndValueRef::Ascii(key, _) => Some(key.as_str()), + KeyAndValueRef::Binary(_, _) => None, + }) + .collect() + } +} + +/// Injects the OpenTelemetry context of the currently-active tracing span +/// into the gRPC request metadata as W3C `traceparent` / `tracestate`. +/// +/// No-op when no global text-map propagator is installed. +pub fn inject_current_context(metadata: &mut MetadataMap) { + let context = tracing::Span::current().context(); + let mut injector = MetadataInjector(metadata); + global::get_text_map_propagator(|propagator| { + propagator.inject_context(&context, &mut injector); + }); +} + +/// Extracts an OpenTelemetry context from incoming gRPC request metadata. +/// Returns the empty context when no propagator is installed or no headers +/// are present. +pub fn extract_context(metadata: &MetadataMap) -> Context { + let extractor = MetadataExtractor(metadata); + global::get_text_map_propagator(|propagator| propagator.extract(&extractor)) +} + +/// Extracts a W3C trace context from incoming gRPC request metadata and +/// installs it as the parent of the currently-active tracing span. Use this +/// at the entry of a gRPC handler that is itself wrapped in a +/// `#[tracing::instrument]` so the handler's span is stitched into the +/// caller's trace. +pub fn set_current_span_parent_from_metadata(metadata: &MetadataMap) { + let parent_context = extract_context(metadata); + let _ = tracing::Span::current().set_parent(parent_context); +} + +/// Tonic interceptor that injects the active span's W3C trace context into +/// the outgoing gRPC metadata. Drop-in replacement for the legacy +/// `quickwit_proto::SpanContextInterceptor`. +#[derive(Clone, Debug)] +pub struct SpanContextInterceptor; + +impl Interceptor for SpanContextInterceptor { + fn call(&mut self, mut request: tonic::Request<()>) -> Result, Status> { + inject_current_context(request.metadata_mut()); + Ok(request) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn inject_does_not_panic() { + let mut metadata = MetadataMap::new(); + // Without an active span this is essentially a no-op — what matters + // is that injection of the (possibly empty) current context does not + // panic. + inject_current_context(&mut metadata); + } + + #[test] + fn extract_without_propagator_is_noop() { + let metadata = MetadataMap::new(); + // Should not panic; returns the empty context. + let _ = extract_context(&metadata); + } + + #[test] + fn extract_skips_binary_metadata_keys() { + let mut metadata = MetadataMap::new(); + metadata.insert_bin( + "x-bin-bin", + tonic::metadata::MetadataValue::from_bytes(b"\x01\x02"), + ); + // Should not panic even though the key isn't ASCII. + let _ = extract_context(&metadata); + } +} diff --git a/quickwit/quickwit-ingest/Cargo.toml b/quickwit/quickwit-ingest/Cargo.toml index 1bf15d76fd4..2a18eff5738 100644 --- a/quickwit/quickwit-ingest/Cargo.toml +++ b/quickwit/quickwit-ingest/Cargo.toml @@ -32,6 +32,7 @@ tonic = { workspace = true } tonic-prost = { workspace = true } tower = { workspace = true } tracing = { workspace = true } +tracing-opentelemetry = { workspace = true } ulid = { workspace = true } utoipa = { workspace = true } diff --git a/quickwit/quickwit-ingest/src/codegen/ingest_service.rs b/quickwit/quickwit-ingest/src/codegen/ingest_service.rs index 435c9b6d2f9..51297864085 100644 --- a/quickwit/quickwit-ingest/src/codegen/ingest_service.rs +++ b/quickwit/quickwit-ingest/src/codegen/ingest_service.rs @@ -304,12 +304,15 @@ impl IngestServiceClient { } #[async_trait::async_trait] impl IngestService for IngestServiceClient { + #[tracing::instrument(skip_all, name = "ingest_service.ingest")] async fn ingest(&self, request: IngestRequest) -> crate::Result { self.inner.0.ingest(request).await } + #[tracing::instrument(skip_all, name = "ingest_service.fetch")] async fn fetch(&self, request: FetchRequest) -> crate::Result { self.inner.0.fetch(request).await } + #[tracing::instrument(skip_all, name = "ingest_service.tail")] async fn tail(&self, request: TailRequest) -> crate::Result { self.inner.0.tail(request).await } @@ -833,9 +836,13 @@ where T::Future: Send, { async fn ingest(&self, request: IngestRequest) -> crate::Result { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .ingest(request) + .ingest(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -844,9 +851,13 @@ where )) } async fn fetch(&self, request: FetchRequest) -> crate::Result { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .fetch(request) + .fetch(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -855,9 +866,13 @@ where )) } async fn tail(&self, request: TailRequest) -> crate::Result { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .tail(request) + .tail(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -884,36 +899,72 @@ impl IngestServiceGrpcServerAdapter { impl ingest_service_grpc_server::IngestServiceGrpc for IngestServiceGrpcServerAdapter { async fn ingest( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .ingest(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("ingest_service.ingest"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .ingest(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn fetch( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .fetch(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("ingest_service.fetch"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .fetch(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn tail( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .tail(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("ingest_service.tail"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .tail(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } } /// Generated client implementations. diff --git a/quickwit/quickwit-ingest/src/ingest_v2/broadcast/capacity_score.rs b/quickwit/quickwit-ingest/src/ingest_v2/broadcast/capacity_score.rs index 8b7ee37a3ca..90cdff3bed5 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/broadcast/capacity_score.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/broadcast/capacity_score.rs @@ -23,7 +23,7 @@ use quickwit_proto::ingest::ingester::IngesterStatus; use quickwit_proto::types::{NodeId, SourceUid}; use serde::{Deserialize, Serialize}; use tokio::task::JoinHandle; -use tracing::{info, warn}; +use tracing::{info, instrument, warn}; use super::{BROADCAST_INTERVAL_PERIOD, make_key, parse_key}; use crate::OpenShardCounts; @@ -40,15 +40,19 @@ pub struct IngesterCapacityScore { pub struct BroadcastIngesterCapacityScoreTask { cluster: Cluster, weak_state: WeakIngesterState, + /// Sources broadcast on the previous tick. Carried across iterations so we + /// can detect sources that disappeared and clear their Chitchat keys. + previous_sources: BTreeSet, } impl BroadcastIngesterCapacityScoreTask { pub fn spawn(cluster: Cluster, weak_state: WeakIngesterState) -> JoinHandle<()> { - let mut broadcaster = Self { + let broadcaster = Self { cluster, weak_state, + previous_sources: BTreeSet::new(), }; - tokio::spawn(async move { broadcaster.run().await }) + tokio::spawn(broadcaster.run()) } async fn snapshot(&self) -> Result> { @@ -79,28 +83,36 @@ impl BroadcastIngesterCapacityScoreTask { Ok(Some((capacity_score, open_shard_counts))) } - async fn run(&mut self) { + async fn run(mut self) { let mut interval = tokio::time::interval(BROADCAST_INTERVAL_PERIOD); - let mut previous_sources: BTreeSet = BTreeSet::new(); loop { interval.tick().await; - let (capacity_score, open_shard_counts) = match self.snapshot().await { - Ok(Some(snapshot)) => snapshot, - Ok(None) => continue, - Err(error) => { - info!("stopping ingester capacity broadcast: {error}"); - return; - } - }; - - previous_sources = self - .broadcast_capacity(capacity_score, &open_shard_counts, &previous_sources) - .await; + if !self.run_once().await { + return; + } } } + /// Single iteration of the capacity-score broadcast loop. Returns `false` + /// when the task should stop. + #[instrument(name = "broadcast_capacity_score.tick", skip_all)] + async fn run_once(&mut self) -> bool { + let (capacity_score, open_shard_counts) = match self.snapshot().await { + Ok(Some(snapshot)) => snapshot, + Ok(None) => return true, + Err(error) => { + info!("stopping ingester capacity broadcast: {error}"); + return false; + } + }; + self.previous_sources = self + .broadcast_capacity(capacity_score, &open_shard_counts, &self.previous_sources) + .await; + true + } + async fn broadcast_capacity( &self, capacity_score: usize, @@ -206,6 +218,7 @@ mod tests { let task = BroadcastIngesterCapacityScoreTask { cluster, weak_state, + previous_sources: BTreeSet::new(), }; assert!(task.snapshot().await.is_err()); } @@ -242,6 +255,7 @@ mod tests { let task = BroadcastIngesterCapacityScoreTask { cluster: cluster.clone(), weak_state: state.weak(), + previous_sources: BTreeSet::new(), }; let update_counter = Arc::new(AtomicUsize::new(0)); @@ -287,6 +301,7 @@ mod tests { let task = BroadcastIngesterCapacityScoreTask { cluster: cluster.clone(), weak_state: state.weak(), + previous_sources: BTreeSet::new(), }; let index_uid = IndexUid::for_test("test-index", 0); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/broadcast/local_shards.rs b/quickwit/quickwit-ingest/src/ingest_v2/broadcast/local_shards.rs index 86f59f6101e..9a2a6f3f95d 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/broadcast/local_shards.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/broadcast/local_shards.rs @@ -26,7 +26,7 @@ use quickwit_proto::ingest::ShardState; use quickwit_proto::types::{NodeId, ShardId, SourceUid}; use serde::{Deserialize, Serialize, Serializer}; use tokio::task::JoinHandle; -use tracing::{debug, warn}; +use tracing::{debug, instrument, warn}; use super::{BROADCAST_INTERVAL_PERIOD, make_key, parse_key}; use crate::RateMibPerSec; @@ -152,6 +152,9 @@ pub struct BroadcastLocalShardsTask { cluster: Cluster, weak_state: WeakIngesterState, shard_throughput_time_series_map: ShardThroughputTimeSeriesMap, + /// Snapshot broadcast on the previous tick. Carried across iterations so + /// we can diff against the new snapshot and only broadcast changes. + previous_snapshot: LocalShardsSnapshot, } const SHARD_THROUGHPUT_LONG_TERM_WINDOW_LEN: usize = 12; @@ -248,12 +251,13 @@ impl ShardThroughputTimeSeries { impl BroadcastLocalShardsTask { pub fn spawn(cluster: Cluster, weak_state: WeakIngesterState) -> JoinHandle<()> { - let mut broadcaster = Self { + let broadcaster = Self { cluster, weak_state, shard_throughput_time_series_map: Default::default(), + previous_snapshot: LocalShardsSnapshot::default(), }; - tokio::spawn(async move { broadcaster.run().await }) + tokio::spawn(broadcaster.run()) } async fn snapshot_local_shards(&mut self) -> Option { @@ -335,24 +339,32 @@ impl BroadcastLocalShardsTask { } } - async fn run(&mut self) { + async fn run(mut self) { let mut interval = tokio::time::interval(BROADCAST_INTERVAL_PERIOD); - let mut previous_snapshot = LocalShardsSnapshot::default(); loop { interval.tick().await; - let Some(new_snapshot) = self.snapshot_local_shards().await else { + if !self.run_once().await { // The state has been dropped, we can stop the task. debug!("stopping local shards broadcast task"); return; - }; - self.broadcast_local_shards(&previous_snapshot, &new_snapshot) - .await; - - previous_snapshot = new_snapshot; + } } } + + /// Single iteration of the broadcast loop. Returns `false` when the task + /// should stop. + #[instrument(name = "broadcast_local_shards.tick", skip_all)] + async fn run_once(&mut self) -> bool { + let Some(new_snapshot) = self.snapshot_local_shards().await else { + return false; + }; + self.broadcast_local_shards(&self.previous_snapshot, &new_snapshot) + .await; + self.previous_snapshot = new_snapshot; + true + } } #[derive(Debug, Clone)] @@ -533,14 +545,12 @@ mod tests { .await .unwrap(); let (_temp_dir, state) = IngesterState::for_test(cluster.clone()).await; - let weak_state = state.weak(); let mut task = BroadcastLocalShardsTask { cluster, - weak_state, + weak_state: state.weak(), shard_throughput_time_series_map: Default::default(), + previous_snapshot: LocalShardsSnapshot::default(), }; - let previous_snapshot = task.snapshot_local_shards().await.unwrap(); - assert!(previous_snapshot.per_source_shard_infos.is_empty()); let mut state_guard = state.lock_partially("test").await.unwrap(); @@ -560,7 +570,8 @@ mod tests { ) .advertisable() .build(); - state_guard.shards.insert(shard_01.queue_id(), shard_01); + let queue_id_01 = shard_01.queue_id(); + state_guard.shards.insert(queue_id_01.clone(), shard_01); let shard_02 = IngesterShard::new_replica( index_uid.clone(), @@ -573,11 +584,10 @@ mod tests { state_guard.shards.insert(shard_02.queue_id(), shard_02); drop(state_guard); - let new_snapshot = task.snapshot_local_shards().await.unwrap(); - assert_eq!(new_snapshot.per_source_shard_infos.len(), 1); - - task.broadcast_local_shards(&previous_snapshot, &new_snapshot) - .await; + // First tick: shard_01 (advertisable, non-replica) is the only one + // contributing to the snapshot — broadcast publishes it. + assert!(task.run_once().await); + assert_eq!(task.previous_snapshot.per_source_shard_infos.len(), 1); tokio::time::sleep(Duration::from_millis(100)).await; @@ -587,8 +597,14 @@ mod tests { ); task.cluster.get_self_key_value(&key).await.unwrap(); - task.broadcast_local_shards(&new_snapshot, &previous_snapshot) - .await; + // Remove the only advertisable shard, run again: snapshot empty, + // broadcast clears the chitchat key. + let mut state_guard = state.lock_partially("test").await.unwrap(); + state_guard.shards.remove(&queue_id_01); + drop(state_guard); + + assert!(task.run_once().await); + assert!(task.previous_snapshot.per_source_shard_infos.is_empty()); tokio::time::sleep(Duration::from_millis(100)).await; diff --git a/quickwit/quickwit-ingest/src/ingest_v2/doc_mapper.rs b/quickwit/quickwit-ingest/src/ingest_v2/doc_mapper.rs index e3ebfc8bfa7..e221602bce6 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/doc_mapper.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/doc_mapper.rs @@ -25,7 +25,7 @@ use quickwit_proto::ingest::{ }; use quickwit_proto::types::{DocMappingUid, DocUid}; use serde_json_borrow::Value as JsonValue; -use tracing::info; +use tracing::{info, instrument}; use crate::DocBatchV2Builder; @@ -69,31 +69,44 @@ pub(super) fn try_build_doc_mapper(doc_mapping_json: &str) -> IngestV2Result Result<(), (ParseFailureReason, String)> { - let Ok(json_doc) = serde_json::from_slice::(doc_bytes) else { - return Err(( - ParseFailureReason::InvalidJson, - "failed to parse JSON document".to_string(), - )); - }; - let JsonValue::Object(json_obj) = json_doc else { - return Err(( - ParseFailureReason::InvalidJson, - "JSON document is not an object".to_string(), - )); - }; - if let Err(error) = doc_mapper.validate_json_obj(&json_obj) { - rate_limited_error!( - limit_per_min = 6, - "failed to validate JSON document: {}", - error - ); - return Err((ParseFailureReason::InvalidSchema, error.to_string())); +/// Parses the JSON documents contained in the batch and applies the doc mapper. Returns the +/// original batch and a list of parse failures. +/// +/// Note: Validation is skipped for Arrow IPC format batches since they have their own schema +/// and are processed by specialized pipelines (e.g., ParquetDocProcessor). +pub(super) async fn validate_doc_batch( + doc_batch: DocBatchV2, + doc_mapper: Arc, +) -> IngestV2Result<(DocBatchV2, Vec)> { + // Skip validation for Arrow IPC format - it has its own schema and is processed + // by specialized pipelines (e.g., metrics pipeline) + if doc_batch.doc_format() == DocFormat::ArrowIpc { + return Ok((doc_batch, Vec::new())); } - Ok(()) + if is_document_validation_enabled() { + return validate_doc_batch_cpu_intensive(doc_batch, doc_mapper).await; + } + Ok((doc_batch, Vec::new())) +} + +fn is_document_validation_enabled() -> bool { + static IS_DOCUMENT_VALIDATION_ENABLED: LazyLock = LazyLock::new(|| { + !quickwit_common::get_bool_from_env("QW_DISABLE_DOCUMENT_VALIDATION", false) + }); + *IS_DOCUMENT_VALIDATION_ENABLED +} + +#[instrument(name = "ingester.validate_doc_batch", skip_all, fields(num_docs = doc_batch.num_docs(), num_bytes = doc_batch.num_bytes()))] +async fn validate_doc_batch_cpu_intensive( + doc_batch: DocBatchV2, + doc_mapper: Arc, +) -> IngestV2Result<(DocBatchV2, Vec)> { + run_cpu_intensive(move || validate_doc_batch_impl(doc_batch, &doc_mapper)) + .await + .map_err(|error| { + let message = format!("failed to validate documents: {error}"); + IngestV2Error::Internal(message) + }) } /// Validates a batch of docs. @@ -134,38 +147,31 @@ fn validate_doc_batch_impl( (valid_doc_batch, parse_failures) } -fn is_document_validation_enabled() -> bool { - static IS_DOCUMENT_VALIDATION_ENABLED: LazyLock = LazyLock::new(|| { - !quickwit_common::get_bool_from_env("QW_DISABLE_DOCUMENT_VALIDATION", false) - }); - *IS_DOCUMENT_VALIDATION_ENABLED -} - -/// Parses the JSON documents contained in the batch and applies the doc mapper. Returns the -/// original batch and a list of parse failures. -/// -/// Note: Validation is skipped for Arrow IPC format batches since they have their own schema -/// and are processed by specialized pipelines (e.g., ParquetDocProcessor). -pub(super) async fn validate_doc_batch( - doc_batch: DocBatchV2, - doc_mapper: Arc, -) -> IngestV2Result<(DocBatchV2, Vec)> { - // Skip validation for Arrow IPC format - it has its own schema and is processed - // by specialized pipelines (e.g., metrics pipeline) - if doc_batch.doc_format() == DocFormat::ArrowIpc { - return Ok((doc_batch, Vec::new())); - } - - if is_document_validation_enabled() { - run_cpu_intensive(move || validate_doc_batch_impl(doc_batch, &doc_mapper)) - .await - .map_err(|error| { - let message = format!("failed to validate documents: {error}"); - IngestV2Error::Internal(message) - }) - } else { - Ok((doc_batch, Vec::new())) +fn validate_document( + doc_mapper: &DocMapper, + doc_bytes: &[u8], +) -> Result<(), (ParseFailureReason, String)> { + let Ok(json_doc) = serde_json::from_slice::(doc_bytes) else { + return Err(( + ParseFailureReason::InvalidJson, + "failed to parse JSON document".to_string(), + )); + }; + let JsonValue::Object(json_obj) = json_doc else { + return Err(( + ParseFailureReason::InvalidJson, + "JSON document is not an object".to_string(), + )); + }; + if let Err(error) = doc_mapper.validate_json_obj(&json_obj) { + rate_limited_error!( + limit_per_min = 6, + "failed to validate JSON document: {}", + error + ); + return Err((ParseFailureReason::InvalidSchema, error.to_string())); } + Ok(()) } #[cfg(test)] diff --git a/quickwit/quickwit-ingest/src/ingest_v2/idle.rs b/quickwit/quickwit-ingest/src/ingest_v2/idle.rs index 76de2caccd2..57a529c25b9 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/idle.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/idle.rs @@ -15,7 +15,7 @@ use std::time::{Duration, Instant}; use tokio::task::JoinHandle; -use tracing::info; +use tracing::{info, instrument}; use super::state::WeakIngesterState; @@ -37,39 +37,47 @@ impl CloseIdleShardsTask { weak_state, idle_shard_timeout, }; - tokio::spawn(async move { - let Some(mut state) = task.weak_state.upgrade() else { - return; - }; - state.wait_for_ready().await; - drop(state); - - task.run().await - }) + tokio::spawn(task.run()) } - async fn run(&self) { + async fn run(mut self) { + let Some(mut state) = self.weak_state.upgrade() else { + return; + }; + state.wait_for_ready().await; + drop(state); + let mut interval = tokio::time::interval(RUN_INTERVAL_PERIOD); loop { interval.tick().await; - let Some(state) = self.weak_state.upgrade() else { - return; - }; - let Ok(mut state_guard) = state.lock_partially("close_idle_shards").await else { + if !self.run_once().await { return; - }; + } + } + } + + /// Single iteration of the close-idle-shards loop. Returns `false` when the + /// task should stop (state dropped or ingester shutting down). + #[instrument(name = "close_idle_shards.tick", skip_all)] + async fn run_once(&mut self) -> bool { + let Some(state) = self.weak_state.upgrade() else { + return false; + }; + let Ok(mut state_guard) = state.lock_partially("close_idle_shards").await else { + return false; + }; - let now = Instant::now(); + let now = Instant::now(); - for (queue_id, shard) in &mut state_guard.shards { - if shard.is_open() && shard.is_idle(now, self.idle_shard_timeout) { - shard.close(); - info!("closed idle shard `{queue_id}`"); - } + for (queue_id, shard) in &mut state_guard.shards { + if shard.is_open() && shard.is_idle(now, self.idle_shard_timeout) { + shard.close(); + info!("closed idle shard `{queue_id}`"); } } + true } } @@ -94,9 +102,11 @@ mod tests { .await .unwrap(); let (_temp_dir, state) = IngesterState::for_test(cluster).await; - let weak_state = state.weak(); let idle_shard_timeout = RUN_INTERVAL_PERIOD * 4; - let join_handle = CloseIdleShardsTask::spawn(weak_state, idle_shard_timeout); + let mut task = CloseIdleShardsTask { + weak_state: state.weak(), + idle_shard_timeout, + }; let mut state_guard = state.lock_partially("test").await.unwrap(); let now = Instant::now(); @@ -122,8 +132,8 @@ mod tests { state_guard.shards.insert(queue_id_02.clone(), shard_02); drop(state_guard); - tokio::time::sleep(RUN_INTERVAL_PERIOD * 2).await; - + // First iteration: only shard_01 (with stale last-write) is idle. + assert!(task.run_once().await); let state_guard = state.lock_partially("test").await.unwrap(); state_guard .shards @@ -137,8 +147,17 @@ mod tests { .assert_is_open(); drop(state_guard); - tokio::time::sleep(idle_shard_timeout).await; + // Age shard_02 directly so the next iteration considers it idle — + // no need to sleep for the timeout to pass. + let mut state_guard = state.lock_partially("test").await.unwrap(); + state_guard + .shards + .get_mut(&queue_id_02) + .unwrap() + .last_write_instant = now - idle_shard_timeout; + drop(state_guard); + assert!(task.run_once().await); let state_guard = state.lock_partially("test").await.unwrap(); state_guard .shards @@ -146,11 +165,9 @@ mod tests { .unwrap() .assert_is_closed(); drop(state_guard); - drop(state); - tokio::time::timeout(Duration::from_secs(1), join_handle) - .await - .unwrap() - .unwrap(); + // Once the strong reference is dropped, the task signals stop. + drop(state); + assert!(!task.run_once().await); } } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs index 4680472cdce..cdc64c0cc14 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs @@ -44,7 +44,7 @@ use quickwit_proto::types::{ use serde_json::{Value as JsonValue, json}; use tokio::sync::Semaphore; use tokio::time::{sleep, timeout}; -use tracing::{debug, error, info, warn}; +use tracing::{debug, error, info, instrument, warn}; use super::IngesterPool; use super::broadcast::{BroadcastIngesterCapacityScoreTask, BroadcastLocalShardsTask}; @@ -271,6 +271,7 @@ impl Ingester { /// /// This operation should be triggered very rarely when the ingester has not been able to delete /// or truncate its shards by other means (RPCs from indexers, gossip, etc.). + #[instrument(name = "ingester.reset_shards", skip_all)] async fn reset_shards(&mut self) { let Ok(_permit) = self.reset_shards_permits.try_acquire() else { return; @@ -1248,6 +1249,7 @@ impl IngesterService for Ingester { #[async_trait] impl EventSubscriber for WeakIngesterState { + #[instrument(name = "ingester.shard_positions_gossip", skip_all)] async fn handle_event(&mut self, shard_positions_update: ShardPositionsUpdate) { let Some(state) = self.upgrade() else { warn!("ingester state update failed"); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/mrecordlog_utils.rs b/quickwit/quickwit-ingest/src/ingest_v2/mrecordlog_utils.rs index 02e838dcdad..8ce2dac0312 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/mrecordlog_utils.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/mrecordlog_utils.rs @@ -22,6 +22,7 @@ use fail::fail_point; use mrecordlog::error::{AppendError, DeleteQueueError}; use quickwit_proto::ingest::DocBatchV2; use quickwit_proto::types::{Position, QueueId}; +use tracing::instrument; use crate::MRecord; use crate::mrecordlog_async::MultiRecordLogAsync; @@ -39,6 +40,16 @@ pub(super) enum AppendDocBatchError { /// # Panics /// /// Panics if `doc_batch` is empty. +#[instrument( + name = "ingester.append_doc_batch", + skip_all, + fields( + queue_id, + num_docs = doc_batch.num_docs(), + num_bytes = doc_batch.num_bytes(), + force_commit, + ) +)] pub(super) async fn append_non_empty_doc_batch( mrecordlog: &mut MultiRecordLogAsync, queue_id: &QueueId, diff --git a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs index 62d20fc8567..88d54a5348e 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs @@ -31,7 +31,7 @@ use quickwit_proto::types::{NodeId, QueueId}; use tokio::sync::mpsc::error::TryRecvError; use tokio::sync::{mpsc, oneshot}; use tokio::task::JoinHandle; -use tracing::{error, warn}; +use tracing::{error, instrument, warn}; use super::metrics::report_wal_usage; use super::models::IngesterShard; @@ -413,7 +413,7 @@ impl ReplicationTask { disk_capacity: ByteSize, memory_capacity: ByteSize, ) -> ReplicationTaskHandle { - let mut replication_task = Self { + let replication_task = Self { leader_id, follower_id, state, @@ -423,10 +423,11 @@ impl ReplicationTask { disk_capacity, memory_capacity, }; - let join_handle = tokio::spawn(async move { replication_task.run().await }); + let join_handle = tokio::spawn(replication_task.run()); ReplicationTaskHandle { join_handle } } + #[instrument(name = "replication.init_replica", skip_all)] async fn init_replica( &mut self, init_replica_request: InitReplicaRequest, @@ -479,6 +480,7 @@ impl ReplicationTask { Ok(init_replica_response) } + #[instrument(name = "replication.replicate", skip_all)] async fn replicate( &mut self, replicate_request: ReplicateRequest, @@ -715,7 +717,7 @@ impl ReplicationTask { Ok(replicate_response) } - async fn run(&mut self) -> IngestV2Result<()> { + async fn run(mut self) -> IngestV2Result<()> { while let Some(syn_replication_message) = self.syn_replication_stream.next().await { let ack_replication_message = match syn_replication_message.message { Some(syn_replication_message::Message::OpenRequest(_)) => { diff --git a/quickwit/quickwit-ingest/src/ingest_v2/state.rs b/quickwit/quickwit-ingest/src/ingest_v2/state.rs index 69bcfae1619..5af08783bb2 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/state.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/state.rs @@ -32,7 +32,7 @@ use quickwit_proto::ingest::ingester::IngesterStatus; use quickwit_proto::ingest::{IngestV2Error, IngestV2Result, ShardIds, ShardState}; use quickwit_proto::types::{DocMappingUid, IndexUid, Position, QueueId, SourceId, split_queue_id}; use tokio::sync::{Mutex, MutexGuard, RwLock, RwLockMappedWriteGuard, RwLockWriteGuard, watch}; -use tracing::{error, info}; +use tracing::{error, info, instrument}; use super::models::IngesterShard; use super::rate_meter::RateMeter; @@ -310,6 +310,7 @@ impl IngesterState { .expect("channel should be open"); } + #[instrument(name = "ingester.lock_partially", skip_all, fields(operation))] pub async fn lock_partially( &self, operation: &'static str, @@ -335,6 +336,7 @@ impl IngesterState { Ok(partially_locked_state) } + #[instrument(name = "ingester.lock_fully", skip_all, fields(operation))] pub async fn lock_fully( &self, operation: &'static str, @@ -525,6 +527,7 @@ where impl FullyLockedIngesterState<'_> { /// Deletes the shard identified by `queue_id` from the ingester state. It removes the /// mrecordlog queue first and then removes the associated in-memory shard and rate trackers. + #[instrument(name = "ingester.delete_shard", skip_all, fields(queue_id, initiator))] pub async fn delete_shard(&mut self, queue_id: &QueueId, initiator: &'static str) { match self.mrecordlog.delete_queue(queue_id).await { Ok(_) | Err(DeleteQueueError::MissingQueue(_)) => { @@ -554,6 +557,11 @@ impl FullyLockedIngesterState<'_> { /// Truncates the shard identified by `queue_id` up to `truncate_up_to_position_inclusive` only /// if the current truncation position of the shard is smaller. + #[instrument( + name = "ingester.truncate_shard", + skip_all, + fields(queue_id, truncate_up_to_position_inclusive, initiator) + )] pub async fn truncate_shard( &mut self, queue_id: &QueueId, diff --git a/quickwit/quickwit-ingest/src/mrecordlog_async.rs b/quickwit/quickwit-ingest/src/mrecordlog_async.rs index d4ff6f8321e..b458f959f70 100644 --- a/quickwit/quickwit-ingest/src/mrecordlog_async.rs +++ b/quickwit/quickwit-ingest/src/mrecordlog_async.rs @@ -20,7 +20,7 @@ use bytes::Buf; use mrecordlog::error::*; use mrecordlog::{MultiRecordLog, PersistAction, PersistPolicy, Record, ResourceUsage}; use tokio::task::JoinError; -use tracing::error; +use tracing::{Span, error, info_span, instrument}; /// A light wrapper to allow async operation in mrecordlog. pub struct MultiRecordLogAsync { @@ -48,6 +48,7 @@ impl MultiRecordLogAsync { Self::open_with_prefs(directory_path, PersistPolicy::Always(PersistAction::Flush)).await } + #[instrument(name = "mrecordlog.open_async", skip_all, fields(directory_path = %directory_path.display(), ?persist_policy))] pub async fn open_with_prefs( directory_path: &Path, persist_policy: PersistPolicy, @@ -66,7 +67,11 @@ impl MultiRecordLogAsync { }) } - async fn run_operation(&mut self, operation: F) -> T + /// Runs `operation` on a blocking task, instrumenting it with `inner_span` + /// so the span's duration covers only the synchronous mrecordlog work. + /// The caller's outer `_async` span continues to measure end-to-end time; + /// the difference is the `spawn_blocking` scheduling overhead. + async fn run_operation(&mut self, inner_span: Span, operation: F) -> T where F: FnOnce(&mut MultiRecordLog) -> T + Send + 'static, T: Send + 'static, @@ -74,6 +79,7 @@ impl MultiRecordLogAsync { let mut mrecordlog = self.take(); let join_res: Result<(T, MultiRecordLog), JoinError> = tokio::task::spawn_blocking(move || { + let _entered = inner_span.entered(); let res = operation(&mut mrecordlog); (res, mrecordlog) }) @@ -91,31 +97,78 @@ impl MultiRecordLogAsync { } } + #[instrument(name = "mrecordlog.create_queue_async", skip_all, fields(queue))] pub async fn create_queue(&mut self, queue: &str) -> Result<(), CreateQueueError> { + let span = info_span!("mrecordlog.create_queue", queue); let queue = queue.to_string(); - self.run_operation(move |mrecordlog| mrecordlog.create_queue(&queue)) + self.run_operation(span, move |mrecordlog| mrecordlog.create_queue(&queue)) .await } + #[instrument(name = "mrecordlog.delete_queue_async", skip_all, fields(queue))] pub async fn delete_queue(&mut self, queue: &str) -> Result<(), DeleteQueueError> { + let span = info_span!("mrecordlog.delete_queue", queue); let queue = queue.to_string(); - self.run_operation(move |mrecordlog| mrecordlog.delete_queue(&queue)) + self.run_operation(span, move |mrecordlog| mrecordlog.delete_queue(&queue)) .await } + #[instrument(name = "mrecordlog.append_records_async", skip_all, fields(queue))] pub async fn append_records + Send + 'static>( &mut self, queue: &str, position_opt: Option, payloads: T, ) -> Result, AppendError> { + let span = info_span!("mrecordlog.append_records", queue); let queue = queue.to_string(); - self.run_operation(move |mrecordlog| { + self.run_operation(span, move |mrecordlog| { mrecordlog.append_records(&queue, position_opt, payloads) }) .await } + #[instrument(name = "mrecordlog.truncate_async", skip_all, fields(queue, position))] + pub async fn truncate(&mut self, queue: &str, position: u64) -> Result { + let span = info_span!("mrecordlog.truncate", queue, position); + let queue = queue.to_string(); + self.run_operation(span, move |mrecordlog| { + mrecordlog.truncate(&queue, position) + }) + .await + } + + pub fn range( + &self, + queue: &str, + range: R, + ) -> Result> + '_, MissingQueue> + where + R: RangeBounds + 'static, + { + self.mrecordlog_ref().range(queue, range) + } + + pub fn queue_exists(&self, queue: &str) -> bool { + self.mrecordlog_ref().queue_exists(queue) + } + + pub fn list_queues(&self) -> impl Iterator { + self.mrecordlog_ref().list_queues() + } + + pub fn last_record(&self, queue: &str) -> Result>, MissingQueue> { + self.mrecordlog_ref().last_record(queue) + } + + pub fn resource_usage(&self) -> ResourceUsage { + self.mrecordlog_ref().resource_usage() + } + + pub fn summary(&self) -> mrecordlog::QueuesSummary { + self.mrecordlog_ref().summary() + } + #[track_caller] #[cfg(test)] pub fn assert_records_eq( @@ -159,41 +212,4 @@ impl MultiRecordLogAsync { ); } } - - pub async fn truncate(&mut self, queue: &str, position: u64) -> Result { - let queue = queue.to_string(); - self.run_operation(move |mrecordlog| mrecordlog.truncate(&queue, position)) - .await - } - - pub fn range( - &self, - queue: &str, - range: R, - ) -> Result> + '_, MissingQueue> - where - R: RangeBounds + 'static, - { - self.mrecordlog_ref().range(queue, range) - } - - pub fn queue_exists(&self, queue: &str) -> bool { - self.mrecordlog_ref().queue_exists(queue) - } - - pub fn list_queues(&self) -> impl Iterator { - self.mrecordlog_ref().list_queues() - } - - pub fn last_record(&self, queue: &str) -> Result>, MissingQueue> { - self.mrecordlog_ref().last_record(queue) - } - - pub fn resource_usage(&self) -> ResourceUsage { - self.mrecordlog_ref().resource_usage() - } - - pub fn summary(&self) -> mrecordlog::QueuesSummary { - self.mrecordlog_ref().summary() - } } diff --git a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs index 68e43f286f5..e6ff4e83bb4 100644 --- a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs @@ -39,6 +39,7 @@ use quickwit_proto::metastore::{ StageSplitsRequest, ToggleSourceRequest, UpdateIndexRequest, UpdateSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; +use tracing::instrument; /// A [`MetastoreService`] implementation that proxies some requests to the control plane so it can /// track the state of the metastore accurately and react to events in real-time. @@ -79,6 +80,7 @@ impl MetastoreService for ControlPlaneMetastore { // Proxied metastore API calls. + #[instrument(skip_all)] async fn create_index( &self, request: CreateIndexRequest, @@ -87,6 +89,7 @@ impl MetastoreService for ControlPlaneMetastore { Ok(response) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn update_index( &self, request: UpdateIndexRequest, @@ -95,32 +98,38 @@ impl MetastoreService for ControlPlaneMetastore { Ok(response) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn delete_index(&self, request: DeleteIndexRequest) -> MetastoreResult { let response = self.control_plane.delete_index(request).await?; Ok(response) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn add_source(&self, request: AddSourceRequest) -> MetastoreResult { let response = self.control_plane.add_source(request).await?; Ok(response) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn update_source(&self, request: UpdateSourceRequest) -> MetastoreResult { let response = self.control_plane.update_source(request).await?; Ok(response) } + #[instrument(skip_all, fields(index_uid = %request.index_uid(), source_id = %request.source_id))] async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { let response = self.control_plane.toggle_source(request).await?; Ok(response) } + #[instrument(skip_all, fields(index_uid = %request.index_uid(), source_id = %request.source_id))] async fn delete_source(&self, request: DeleteSourceRequest) -> MetastoreResult { let response = self.control_plane.delete_source(request).await?; Ok(response) } // Proxy through the control plane to debounce queries + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn prune_shards(&self, request: PruneShardsRequest) -> MetastoreResult { self.control_plane.prune_shards(request).await?; Ok(EmptyResponse {}) @@ -128,6 +137,7 @@ impl MetastoreService for ControlPlaneMetastore { // Other metastore API calls. + #[instrument(skip(self))] async fn index_metadata( &self, request: IndexMetadataRequest, @@ -135,6 +145,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.index_metadata(request).await } + #[instrument(skip_all, fields(num_subrequests = request.subrequests.len()))] async fn indexes_metadata( &self, request: IndexesMetadataRequest, @@ -142,6 +153,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.indexes_metadata(request).await } + #[instrument(skip_all, fields(index_id_patterns = ?request.index_id_patterns))] async fn list_indexes_metadata( &self, request: ListIndexesMetadataRequest, @@ -149,10 +161,12 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.list_indexes_metadata(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult { self.metastore.stage_splits(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn publish_splits( &self, request: PublishSplitsRequest, @@ -160,6 +174,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.publish_splits(request).await } + #[instrument(skip_all)] async fn list_splits( &self, request: ListSplitsRequest, @@ -167,6 +182,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.list_splits(request).await } + #[instrument(skip_all, fields(index_id_patterns = ?request.index_id_patterns))] async fn list_index_stats( &self, request: ListIndexStatsRequest, @@ -174,6 +190,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.list_index_stats(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn list_stale_splits( &self, request: ListStaleSplitsRequest, @@ -181,6 +198,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.list_stale_splits(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn mark_splits_for_deletion( &self, request: MarkSplitsForDeletionRequest, @@ -188,10 +206,12 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.mark_splits_for_deletion(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult { self.metastore.delete_splits(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid(), source_id = %request.source_id))] async fn reset_source_checkpoint( &self, request: ResetSourceCheckpointRequest, @@ -201,10 +221,12 @@ impl MetastoreService for ControlPlaneMetastore { // Delete tasks API + #[instrument(skip_all, fields(index_uid = %delete_query.index_uid()))] async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { self.metastore.create_delete_task(delete_query).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn last_delete_opstamp( &self, request: LastDeleteOpstampRequest, @@ -212,6 +234,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.last_delete_opstamp(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn update_splits_delete_opstamp( &self, request: UpdateSplitsDeleteOpstampRequest, @@ -219,6 +242,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.update_splits_delete_opstamp(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn list_delete_tasks( &self, request: ListDeleteTasksRequest, @@ -228,10 +252,12 @@ impl MetastoreService for ControlPlaneMetastore { // Shard API + #[instrument(skip_all, fields(num_subrequests = request.subrequests.len()))] async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult { self.metastore.open_shards(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn acquire_shards( &self, request: AcquireShardsRequest, @@ -239,10 +265,12 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.acquire_shards(request).await } + #[instrument(skip_all, fields(num_subrequests = request.subrequests.len()))] async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { self.metastore.list_shards(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn delete_shards( &self, request: DeleteShardsRequest, @@ -252,6 +280,7 @@ impl MetastoreService for ControlPlaneMetastore { // Index Template API + #[instrument(skip(self))] async fn create_index_template( &self, request: CreateIndexTemplateRequest, @@ -259,6 +288,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.create_index_template(request).await } + #[instrument(skip(self))] async fn get_index_template( &self, request: GetIndexTemplateRequest, @@ -266,6 +296,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.get_index_template(request).await } + #[instrument(skip(self))] async fn find_index_template_matches( &self, request: FindIndexTemplateMatchesRequest, @@ -273,6 +304,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.find_index_template_matches(request).await } + #[instrument(skip_all)] async fn list_index_templates( &self, request: ListIndexTemplatesRequest, @@ -280,6 +312,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.list_index_templates(request).await } + #[instrument(skip(self))] async fn delete_index_templates( &self, request: DeleteIndexTemplatesRequest, @@ -287,6 +320,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.delete_index_templates(request).await } + #[instrument(skip_all)] async fn get_cluster_identity( &self, request: GetClusterIdentityRequest, @@ -296,6 +330,7 @@ impl MetastoreService for ControlPlaneMetastore { // Metrics Splits API - Proxy to underlying metastore + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn stage_metrics_splits( &self, request: StageMetricsSplitsRequest, @@ -303,6 +338,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.stage_metrics_splits(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn publish_metrics_splits( &self, request: PublishMetricsSplitsRequest, @@ -310,6 +346,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.publish_metrics_splits(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn list_metrics_splits( &self, request: ListMetricsSplitsRequest, @@ -317,6 +354,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.list_metrics_splits(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn mark_metrics_splits_for_deletion( &self, request: MarkMetricsSplitsForDeletionRequest, @@ -326,6 +364,7 @@ impl MetastoreService for ControlPlaneMetastore { .await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn delete_metrics_splits( &self, request: DeleteMetricsSplitsRequest, @@ -333,6 +372,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.delete_metrics_splits(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn stage_sketch_splits( &self, request: StageSketchSplitsRequest, @@ -340,6 +380,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.stage_sketch_splits(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn publish_sketch_splits( &self, request: PublishSketchSplitsRequest, @@ -347,6 +388,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.publish_sketch_splits(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn list_sketch_splits( &self, request: ListSketchSplitsRequest, @@ -354,6 +396,7 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.list_sketch_splits(request).await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn mark_sketch_splits_for_deletion( &self, request: MarkSketchSplitsForDeletionRequest, @@ -363,6 +406,7 @@ impl MetastoreService for ControlPlaneMetastore { .await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn delete_sketch_splits( &self, request: DeleteSketchSplitsRequest, diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs index 1e2e122d7e4..d26bb095d17 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs @@ -67,6 +67,7 @@ use quickwit_proto::types::{IndexId, IndexUid}; use quickwit_storage::Storage; use time::OffsetDateTime; use tokio::sync::{Mutex, OwnedMutexGuard, RwLock}; +use tracing::instrument; use ulid::Ulid; use uuid::Uuid; @@ -483,6 +484,7 @@ impl MetastoreService for FileBackedMetastore { // ------------------------------------------------------------------------------- // Mutations over the high-level index. + #[instrument(skip_all)] async fn create_index( &self, request: CreateIndexRequest, @@ -563,6 +565,7 @@ impl MetastoreService for FileBackedMetastore { Ok(response) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn update_index( &self, request: UpdateIndexRequest, @@ -595,6 +598,7 @@ impl MetastoreService for FileBackedMetastore { IndexMetadataResponse::try_from_index_metadata(&index_metadata) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn delete_index(&self, request: DeleteIndexRequest) -> MetastoreResult { // We pick the outer lock here, so that we enter a critical section. let mut state_wlock_guard = self.state.write().await; @@ -649,6 +653,7 @@ impl MetastoreService for FileBackedMetastore { // ------------------------------------------------------------------------------- // Mutations over a single index + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult { let index_uid = request.index_uid().clone(); let splits_metadata = request.deserialize_splits_metadata()?; @@ -682,6 +687,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn publish_splits( &self, request: PublishSplitsRequest, @@ -702,6 +708,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn mark_splits_for_deletion( &self, request: MarkSplitsForDeletionRequest, @@ -725,6 +732,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult { let index_uid = request.index_uid().clone(); @@ -736,6 +744,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn add_source(&self, request: AddSourceRequest) -> MetastoreResult { let source_config = request.deserialize_source_config()?; let index_uid = request.index_uid(); @@ -748,6 +757,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn update_source(&self, request: UpdateSourceRequest) -> MetastoreResult { let source_config = request.deserialize_source_config()?; let index_uid = request.index_uid(); @@ -760,6 +770,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid(), source_id = %request.source_id))] async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { let index_uid = request.index_uid(); @@ -772,6 +783,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid(), source_id = %request.source_id))] async fn delete_source(&self, request: DeleteSourceRequest) -> MetastoreResult { let index_uid = request.index_uid(); @@ -783,6 +795,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid(), source_id = %request.source_id))] async fn reset_source_checkpoint( &self, request: ResetSourceCheckpointRequest, @@ -803,6 +816,7 @@ impl MetastoreService for FileBackedMetastore { /// Streams of splits for the given request. /// No error is returned if any of the requested `index_uid` does not exist. + #[instrument(skip_all)] async fn list_splits( &self, request: ListSplitsRequest, @@ -816,6 +830,7 @@ impl MetastoreService for FileBackedMetastore { Ok(ServiceStream::new(splits_responses_stream)) } + #[instrument(skip_all, fields(index_id_patterns = ?request.index_id_patterns))] async fn list_index_stats( &self, request: ListIndexStatsRequest, @@ -861,6 +876,7 @@ impl MetastoreService for FileBackedMetastore { Ok(ListIndexStatsResponse { index_stats }) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn list_stale_splits( &self, request: ListStaleSplitsRequest, @@ -877,6 +893,7 @@ impl MetastoreService for FileBackedMetastore { ListSplitsResponse::try_from_splits(splits) } + #[instrument(skip(self))] async fn index_metadata( &self, request: IndexMetadataRequest, @@ -889,6 +906,7 @@ impl MetastoreService for FileBackedMetastore { Ok(response) } + #[instrument(skip_all, fields(num_subrequests = request.subrequests.len()))] async fn indexes_metadata( &self, request: IndexesMetadataRequest, @@ -935,6 +953,7 @@ impl MetastoreService for FileBackedMetastore { Ok(response) } + #[instrument(skip_all, fields(index_id_patterns = ?request.index_id_patterns))] async fn list_indexes_metadata( &self, request: ListIndexesMetadataRequest, @@ -975,6 +994,7 @@ impl MetastoreService for FileBackedMetastore { // Shard API + #[instrument(skip_all, fields(num_subrequests = request.subrequests.len()))] async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult { let mut response = OpenShardsResponse { subresponses: Vec::with_capacity(request.subrequests.len()), @@ -995,6 +1015,7 @@ impl MetastoreService for FileBackedMetastore { Ok(response) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn acquire_shards( &self, request: AcquireShardsRequest, @@ -1006,6 +1027,7 @@ impl MetastoreService for FileBackedMetastore { Ok(response) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn delete_shards( &self, request: DeleteShardsRequest, @@ -1017,6 +1039,7 @@ impl MetastoreService for FileBackedMetastore { Ok(response) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn prune_shards(&self, request: PruneShardsRequest) -> MetastoreResult { let index_uid = request.index_uid().clone(); self.mutate(&index_uid, |index| index.prune_shards(request)) @@ -1024,6 +1047,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(num_subrequests = request.subrequests.len()))] async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { let mut subresponses = Vec::with_capacity(request.subrequests.len()); @@ -1041,6 +1065,7 @@ impl MetastoreService for FileBackedMetastore { // ------------------------------------------------------------------------------- // Delete tasks + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn last_delete_opstamp( &self, request: LastDeleteOpstampRequest, @@ -1051,6 +1076,7 @@ impl MetastoreService for FileBackedMetastore { Ok(LastDeleteOpstampResponse::new(last_delete_opstamp)) } + #[instrument(skip_all, fields(index_uid = %delete_query.index_uid()))] async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { let index_uid = delete_query.index_uid().clone(); let delete_task = self @@ -1063,6 +1089,7 @@ impl MetastoreService for FileBackedMetastore { Ok(delete_task) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn update_splits_delete_opstamp( &self, request: UpdateSplitsDeleteOpstampRequest, @@ -1083,6 +1110,7 @@ impl MetastoreService for FileBackedMetastore { Ok(UpdateSplitsDeleteOpstampResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn list_delete_tasks( &self, request: ListDeleteTasksRequest, @@ -1100,6 +1128,7 @@ impl MetastoreService for FileBackedMetastore { // Index Template API + #[instrument(skip(self))] async fn create_index_template( &self, request: CreateIndexTemplateRequest, @@ -1157,6 +1186,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip(self))] async fn get_index_template( &self, request: GetIndexTemplateRequest, @@ -1177,6 +1207,7 @@ impl MetastoreService for FileBackedMetastore { Ok(response) } + #[instrument(skip(self))] async fn find_index_template_matches( &self, request: FindIndexTemplateMatchesRequest, @@ -1208,6 +1239,7 @@ impl MetastoreService for FileBackedMetastore { Ok(response) } + #[instrument(skip_all)] async fn list_index_templates( &self, _request: ListIndexTemplatesRequest, @@ -1225,6 +1257,7 @@ impl MetastoreService for FileBackedMetastore { Ok(response) } + #[instrument(skip(self))] async fn delete_index_templates( &self, request: DeleteIndexTemplatesRequest, @@ -1261,6 +1294,7 @@ impl MetastoreService for FileBackedMetastore { // this returns a constant uuid. on first call, it generate said uuid if it doesn't already // exists + #[instrument(skip_all)] async fn get_cluster_identity( &self, _: GetClusterIdentityRequest, @@ -1285,6 +1319,7 @@ impl MetastoreService for FileBackedMetastore { // Metrics Splits API + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn stage_metrics_splits( &self, request: StageMetricsSplitsRequest, @@ -1309,6 +1344,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn publish_metrics_splits( &self, request: PublishMetricsSplitsRequest, @@ -1338,6 +1374,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn list_metrics_splits( &self, request: ListMetricsSplitsRequest, @@ -1363,6 +1400,7 @@ impl MetastoreService for FileBackedMetastore { ListMetricsSplitsResponse::try_from_splits(&split_records) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn mark_metrics_splits_for_deletion( &self, request: MarkMetricsSplitsForDeletionRequest, @@ -1387,6 +1425,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn delete_metrics_splits( &self, request: DeleteMetricsSplitsRequest, @@ -1411,6 +1450,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn stage_sketch_splits( &self, request: StageSketchSplitsRequest, @@ -1435,6 +1475,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn publish_sketch_splits( &self, request: PublishSketchSplitsRequest, @@ -1464,6 +1505,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn list_sketch_splits( &self, request: ListSketchSplitsRequest, @@ -1489,6 +1531,7 @@ impl MetastoreService for FileBackedMetastore { ListSketchSplitsResponse::try_from_splits(&split_records) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn mark_sketch_splits_for_deletion( &self, request: MarkSketchSplitsForDeletionRequest, @@ -1513,6 +1556,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn delete_sketch_splits( &self, request: DeleteSketchSplitsRequest, diff --git a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs index 4f637de4109..baedaa07928 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs @@ -422,6 +422,7 @@ impl MetastoreService for PostgresqlMetastore { Ok(response) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn update_index( &self, request: UpdateIndexRequest, @@ -914,6 +915,7 @@ impl MetastoreService for PostgresqlMetastore { Ok(service_stream) } + #[instrument(skip_all, fields(index_id_patterns = ?request.index_id_patterns))] async fn list_index_stats( &self, request: ListIndexStatsRequest, @@ -1402,6 +1404,7 @@ impl MetastoreService for PostgresqlMetastore { } // TODO: Issue a single SQL query. + #[instrument(skip_all, fields(num_subrequests = request.subrequests.len()))] async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult { let mut subresponses = Vec::with_capacity(request.subrequests.len()); @@ -1416,6 +1419,7 @@ impl MetastoreService for PostgresqlMetastore { Ok(OpenShardsResponse { subresponses }) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn acquire_shards( &self, request: AcquireShardsRequest, @@ -1440,6 +1444,7 @@ impl MetastoreService for PostgresqlMetastore { Ok(response) } + #[instrument(skip_all, fields(num_subrequests = request.subrequests.len()))] async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { if request.subrequests.is_empty() { return Ok(Default::default()); @@ -1512,6 +1517,7 @@ impl MetastoreService for PostgresqlMetastore { Ok(response) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn delete_shards( &self, request: DeleteShardsRequest, @@ -1584,6 +1590,7 @@ impl MetastoreService for PostgresqlMetastore { Ok(response) } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn prune_shards(&self, request: PruneShardsRequest) -> MetastoreResult { const PRUNE_AGE_SHARDS_QUERY: &str = include_str!("queries/shards/prune_age.sql"); const PRUNE_COUNT_SHARDS_QUERY: &str = include_str!("queries/shards/prune_count.sql"); @@ -1612,6 +1619,7 @@ impl MetastoreService for PostgresqlMetastore { // Index Template API + #[instrument(skip(self))] async fn create_index_template( &self, request: CreateIndexTemplateRequest, @@ -1672,6 +1680,7 @@ impl MetastoreService for PostgresqlMetastore { Ok(EmptyResponse {}) } + #[instrument(skip(self))] async fn get_index_template( &self, request: GetIndexTemplateRequest, @@ -1692,6 +1701,7 @@ impl MetastoreService for PostgresqlMetastore { Ok(response) } + #[instrument(skip(self))] async fn find_index_template_matches( &self, request: FindIndexTemplateMatchesRequest, @@ -1722,6 +1732,7 @@ impl MetastoreService for PostgresqlMetastore { Ok(response) } + #[instrument(skip_all)] async fn list_index_templates( &self, _request: ListIndexTemplatesRequest, @@ -1741,6 +1752,7 @@ impl MetastoreService for PostgresqlMetastore { Ok(response) } + #[instrument(skip(self))] async fn delete_index_templates( &self, request: DeleteIndexTemplatesRequest, @@ -1752,6 +1764,7 @@ impl MetastoreService for PostgresqlMetastore { Ok(EmptyResponse {}) } + #[instrument(skip_all)] async fn get_cluster_identity( &self, _: GetClusterIdentityRequest, @@ -1774,6 +1787,7 @@ impl MetastoreService for PostgresqlMetastore { // Metrics Splits API + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn stage_metrics_splits( &self, request: StageMetricsSplitsRequest, @@ -1840,6 +1854,7 @@ impl MetastoreService for PostgresqlMetastore { .await } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] async fn stage_sketch_splits( &self, request: StageSketchSplitsRequest, diff --git a/quickwit/quickwit-proto/build.rs b/quickwit/quickwit-proto/build.rs index a3fdef08f90..ac1a657a87d 100644 --- a/quickwit/quickwit-proto/build.rs +++ b/quickwit/quickwit-proto/build.rs @@ -180,6 +180,13 @@ fn main() -> Result<(), Box> { .with_result_type_path("crate::ingest::IngestV2Result") .with_error_type_path("crate::ingest::IngestV2Error") .generate_rpc_name_impls() + // Surface a couple of top-level scalar fields on the generated tracing + // spans so traces are immediately filterable by leader / commit type. + // `leader_id` is `String` (Display ok); `commit_type` is the prost + // enum `CommitTypeV2` which only derives `Debug`. + .with_traced_request_field_debug("IngestRequestV2", "commit_type") + .with_traced_request_field("PersistRequest", "leader_id") + .with_traced_request_field_debug("PersistRequest", "commit_type") .run() .unwrap(); diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.cluster.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.cluster.rs index f326b8e7c29..7e9afed2c6e 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.cluster.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.cluster.rs @@ -203,6 +203,7 @@ impl ClusterServiceClient { } #[async_trait::async_trait] impl ClusterService for ClusterServiceClient { + #[tracing::instrument(skip_all, name = "cluster.fetch_cluster_state")] async fn fetch_cluster_state( &self, request: FetchClusterStateRequest, @@ -532,9 +533,13 @@ where &self, request: FetchClusterStateRequest, ) -> crate::cluster::ClusterResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .fetch_cluster_state(request) + .fetch_cluster_state(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -562,14 +567,26 @@ impl cluster_service_grpc_server::ClusterServiceGrpc for ClusterServiceGrpcServerAdapter { async fn fetch_cluster_state( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .fetch_cluster_state(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("cluster.fetch_cluster_state"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .fetch_cluster_state(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } } /// Generated client implementations. diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs index 09cfbdebf58..243d63d037e 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs @@ -288,6 +288,7 @@ impl ControlPlaneServiceClient { } #[async_trait::async_trait] impl ControlPlaneService for ControlPlaneServiceClient { + #[tracing::instrument(skip_all, name = "control_plane.create_index")] async fn create_index( &self, request: super::metastore::CreateIndexRequest, @@ -296,6 +297,7 @@ impl ControlPlaneService for ControlPlaneServiceClient { > { self.inner.0.create_index(request).await } + #[tracing::instrument(skip_all, name = "control_plane.update_index")] async fn update_index( &self, request: super::metastore::UpdateIndexRequest, @@ -304,48 +306,56 @@ impl ControlPlaneService for ControlPlaneServiceClient { > { self.inner.0.update_index(request).await } + #[tracing::instrument(skip_all, name = "control_plane.delete_index")] async fn delete_index( &self, request: super::metastore::DeleteIndexRequest, ) -> crate::control_plane::ControlPlaneResult { self.inner.0.delete_index(request).await } + #[tracing::instrument(skip_all, name = "control_plane.add_source")] async fn add_source( &self, request: super::metastore::AddSourceRequest, ) -> crate::control_plane::ControlPlaneResult { self.inner.0.add_source(request).await } + #[tracing::instrument(skip_all, name = "control_plane.update_source")] async fn update_source( &self, request: super::metastore::UpdateSourceRequest, ) -> crate::control_plane::ControlPlaneResult { self.inner.0.update_source(request).await } + #[tracing::instrument(skip_all, name = "control_plane.toggle_source")] async fn toggle_source( &self, request: super::metastore::ToggleSourceRequest, ) -> crate::control_plane::ControlPlaneResult { self.inner.0.toggle_source(request).await } + #[tracing::instrument(skip_all, name = "control_plane.delete_source")] async fn delete_source( &self, request: super::metastore::DeleteSourceRequest, ) -> crate::control_plane::ControlPlaneResult { self.inner.0.delete_source(request).await } + #[tracing::instrument(skip_all, name = "control_plane.get_or_create_open_shards")] async fn get_or_create_open_shards( &self, request: GetOrCreateOpenShardsRequest, ) -> crate::control_plane::ControlPlaneResult { self.inner.0.get_or_create_open_shards(request).await } + #[tracing::instrument(skip_all, name = "control_plane.advise_reset_shards")] async fn advise_reset_shards( &self, request: AdviseResetShardsRequest, ) -> crate::control_plane::ControlPlaneResult { self.inner.0.advise_reset_shards(request).await } + #[tracing::instrument(skip_all, name = "control_plane.prune_shards")] async fn prune_shards( &self, request: super::metastore::PruneShardsRequest, @@ -1780,9 +1790,13 @@ where ) -> crate::control_plane::ControlPlaneResult< super::metastore::CreateIndexResponse, > { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .create_index(request) + .create_index(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -1796,9 +1810,13 @@ where ) -> crate::control_plane::ControlPlaneResult< super::metastore::IndexMetadataResponse, > { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .update_index(request) + .update_index(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -1810,9 +1828,13 @@ where &self, request: super::metastore::DeleteIndexRequest, ) -> crate::control_plane::ControlPlaneResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .delete_index(request) + .delete_index(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -1824,9 +1846,13 @@ where &self, request: super::metastore::AddSourceRequest, ) -> crate::control_plane::ControlPlaneResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .add_source(request) + .add_source(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -1838,9 +1864,13 @@ where &self, request: super::metastore::UpdateSourceRequest, ) -> crate::control_plane::ControlPlaneResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .update_source(request) + .update_source(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -1852,9 +1882,13 @@ where &self, request: super::metastore::ToggleSourceRequest, ) -> crate::control_plane::ControlPlaneResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .toggle_source(request) + .toggle_source(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -1866,9 +1900,13 @@ where &self, request: super::metastore::DeleteSourceRequest, ) -> crate::control_plane::ControlPlaneResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .delete_source(request) + .delete_source(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -1880,9 +1918,13 @@ where &self, request: GetOrCreateOpenShardsRequest, ) -> crate::control_plane::ControlPlaneResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .get_or_create_open_shards(request) + .get_or_create_open_shards(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -1894,9 +1936,13 @@ where &self, request: AdviseResetShardsRequest, ) -> crate::control_plane::ControlPlaneResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .advise_reset_shards(request) + .advise_reset_shards(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -1908,9 +1954,13 @@ where &self, request: super::metastore::PruneShardsRequest, ) -> crate::control_plane::ControlPlaneResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .prune_shards(request) + .prune_shards(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -1938,116 +1988,236 @@ impl control_plane_service_grpc_server::ControlPlaneServiceGrpc for ControlPlaneServiceGrpcServerAdapter { async fn create_index( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .create_index(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("control_plane.create_index"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .create_index(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn update_index( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result< tonic::Response, tonic::Status, > { - self.inner - .0 - .update_index(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("control_plane.update_index"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .update_index(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn delete_index( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .delete_index(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("control_plane.delete_index"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .delete_index(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn add_source( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .add_source(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("control_plane.add_source"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .add_source(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn update_source( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .update_source(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("control_plane.update_source"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .update_source(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn toggle_source( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .toggle_source(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("control_plane.toggle_source"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .toggle_source(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn delete_source( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .delete_source(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("control_plane.delete_source"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .delete_source(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn get_or_create_open_shards( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .get_or_create_open_shards(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("control_plane.get_or_create_open_shards"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .get_or_create_open_shards(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn advise_reset_shards( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .advise_reset_shards(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("control_plane.advise_reset_shards"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .advise_reset_shards(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn prune_shards( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .prune_shards(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("control_plane.prune_shards"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .prune_shards(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } } /// Generated client implementations. diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.developer.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.developer.rs index 38520a259ec..0b4565666b8 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.developer.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.developer.rs @@ -137,6 +137,7 @@ impl DeveloperServiceClient { } #[async_trait::async_trait] impl DeveloperService for DeveloperServiceClient { + #[tracing::instrument(skip_all, name = "developer.get_debug_info")] async fn get_debug_info( &self, request: GetDebugInfoRequest, @@ -465,9 +466,13 @@ where &self, request: GetDebugInfoRequest, ) -> crate::developer::DeveloperResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .get_debug_info(request) + .get_debug_info(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -495,14 +500,26 @@ impl developer_service_grpc_server::DeveloperServiceGrpc for DeveloperServiceGrpcServerAdapter { async fn get_debug_info( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .get_debug_info(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("developer.get_debug_info"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .get_debug_info(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } } /// Generated client implementations. diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs index c22235f36ec..e52cb3de55d 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs @@ -148,6 +148,7 @@ impl IndexingServiceClient { } #[async_trait::async_trait] impl IndexingService for IndexingServiceClient { + #[tracing::instrument(skip_all, name = "indexing.apply_indexing_plan")] async fn apply_indexing_plan( &self, request: ApplyIndexingPlanRequest, @@ -477,9 +478,13 @@ where &self, request: ApplyIndexingPlanRequest, ) -> crate::indexing::IndexingResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .apply_indexing_plan(request) + .apply_indexing_plan(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -507,14 +512,26 @@ impl indexing_service_grpc_server::IndexingServiceGrpc for IndexingServiceGrpcServerAdapter { async fn apply_indexing_plan( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .apply_indexing_plan(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("indexing.apply_indexing_plan"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .apply_indexing_plan(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } } /// Generated client implementations. diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs index c472e2ea3ec..6c96134b0b3 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -743,54 +743,67 @@ impl IngesterServiceClient { } #[async_trait::async_trait] impl IngesterService for IngesterServiceClient { + #[tracing::instrument( + skip_all, + name = "ingest.ingester.persist", + fields(leader_id = %request.leader_id, commit_type = ?request.commit_type) + )] async fn persist( &self, request: PersistRequest, ) -> crate::ingest::IngestV2Result { self.inner.0.persist(request).await } + #[tracing::instrument(skip_all, name = "ingest.ingester.open_replication_stream")] async fn open_replication_stream( &self, request: quickwit_common::ServiceStream, ) -> crate::ingest::IngestV2Result> { self.inner.0.open_replication_stream(request).await } + #[tracing::instrument(skip_all, name = "ingest.ingester.open_fetch_stream")] async fn open_fetch_stream( &self, request: OpenFetchStreamRequest, ) -> crate::ingest::IngestV2Result> { self.inner.0.open_fetch_stream(request).await } + #[tracing::instrument(skip_all, name = "ingest.ingester.open_observation_stream")] async fn open_observation_stream( &self, request: OpenObservationStreamRequest, ) -> crate::ingest::IngestV2Result> { self.inner.0.open_observation_stream(request).await } + #[tracing::instrument(skip_all, name = "ingest.ingester.init_shards")] async fn init_shards( &self, request: InitShardsRequest, ) -> crate::ingest::IngestV2Result { self.inner.0.init_shards(request).await } + #[tracing::instrument(skip_all, name = "ingest.ingester.retain_shards")] async fn retain_shards( &self, request: RetainShardsRequest, ) -> crate::ingest::IngestV2Result { self.inner.0.retain_shards(request).await } + #[tracing::instrument(skip_all, name = "ingest.ingester.truncate_shards")] async fn truncate_shards( &self, request: TruncateShardsRequest, ) -> crate::ingest::IngestV2Result { self.inner.0.truncate_shards(request).await } + #[tracing::instrument(skip_all, name = "ingest.ingester.close_shards")] async fn close_shards( &self, request: CloseShardsRequest, ) -> crate::ingest::IngestV2Result { self.inner.0.close_shards(request).await } + #[tracing::instrument(skip_all, name = "ingest.ingester.decommission")] async fn decommission( &self, request: DecommissionRequest, @@ -2033,9 +2046,13 @@ where &self, request: PersistRequest, ) -> crate::ingest::IngestV2Result { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .persist(request) + .persist(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -2047,9 +2064,13 @@ where &self, request: quickwit_common::ServiceStream, ) -> crate::ingest::IngestV2Result> { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .open_replication_stream(request) + .open_replication_stream(tonic_request) .await .map(|response| { let streaming: tonic::Streaming<_> = response.into_inner(); @@ -2069,9 +2090,13 @@ where &self, request: OpenFetchStreamRequest, ) -> crate::ingest::IngestV2Result> { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .open_fetch_stream(request) + .open_fetch_stream(tonic_request) .await .map(|response| { let streaming: tonic::Streaming<_> = response.into_inner(); @@ -2091,9 +2116,13 @@ where &self, request: OpenObservationStreamRequest, ) -> crate::ingest::IngestV2Result> { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .open_observation_stream(request) + .open_observation_stream(tonic_request) .await .map(|response| { let streaming: tonic::Streaming<_> = response.into_inner(); @@ -2113,9 +2142,13 @@ where &self, request: InitShardsRequest, ) -> crate::ingest::IngestV2Result { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .init_shards(request) + .init_shards(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -2127,9 +2160,13 @@ where &self, request: RetainShardsRequest, ) -> crate::ingest::IngestV2Result { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .retain_shards(request) + .retain_shards(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -2141,9 +2178,13 @@ where &self, request: TruncateShardsRequest, ) -> crate::ingest::IngestV2Result { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .truncate_shards(request) + .truncate_shards(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -2155,9 +2196,13 @@ where &self, request: CloseShardsRequest, ) -> crate::ingest::IngestV2Result { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .close_shards(request) + .close_shards(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -2169,9 +2214,13 @@ where &self, request: DecommissionRequest, ) -> crate::ingest::IngestV2Result { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .decommission(request) + .decommission(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -2199,120 +2248,229 @@ impl ingester_service_grpc_server::IngesterServiceGrpc for IngesterServiceGrpcServerAdapter { async fn persist( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .persist(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!( + "ingest.ingester.persist", leader_id = % request.leader_id, commit_type = ? + request.commit_type + ); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .persist(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } type OpenReplicationStreamStream = quickwit_common::ServiceStream< tonic::Result, >; async fn open_replication_stream( &self, - request: tonic::Request>, + tonic_request: tonic::Request>, ) -> Result, tonic::Status> { - self.inner - .0 - .open_replication_stream({ - let streaming: tonic::Streaming<_> = request.into_inner(); - quickwit_common::ServiceStream::from(streaming) - }) - .await - .map(|stream| tonic::Response::new( - stream.map_err(crate::error::grpc_error_to_grpc_status), - )) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let streaming: tonic::Streaming<_> = tonic_request.into_inner(); + let request = quickwit_common::ServiceStream::from(streaming); + let span = tracing::info_span!("ingest.ingester.open_replication_stream"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .open_replication_stream(request) + .await + .map(|stream| tonic::Response::new( + stream.map_err(crate::error::grpc_error_to_grpc_status), + )) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } type OpenFetchStreamStream = quickwit_common::ServiceStream< tonic::Result, >; async fn open_fetch_stream( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .open_fetch_stream(request.into_inner()) - .await - .map(|stream| tonic::Response::new( - stream.map_err(crate::error::grpc_error_to_grpc_status), - )) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("ingest.ingester.open_fetch_stream"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .open_fetch_stream(request) + .await + .map(|stream| tonic::Response::new( + stream.map_err(crate::error::grpc_error_to_grpc_status), + )) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } type OpenObservationStreamStream = quickwit_common::ServiceStream< tonic::Result, >; async fn open_observation_stream( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .open_observation_stream(request.into_inner()) - .await - .map(|stream| tonic::Response::new( - stream.map_err(crate::error::grpc_error_to_grpc_status), - )) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("ingest.ingester.open_observation_stream"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .open_observation_stream(request) + .await + .map(|stream| tonic::Response::new( + stream.map_err(crate::error::grpc_error_to_grpc_status), + )) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn init_shards( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .init_shards(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("ingest.ingester.init_shards"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .init_shards(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn retain_shards( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .retain_shards(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("ingest.ingester.retain_shards"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .retain_shards(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn truncate_shards( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .truncate_shards(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("ingest.ingester.truncate_shards"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .truncate_shards(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn close_shards( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .close_shards(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("ingest.ingester.close_shards"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .close_shards(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn decommission( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .decommission(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("ingest.ingester.decommission"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .decommission(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } } /// Generated client implementations. diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs index f502783e038..5367b5f5c73 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs @@ -246,6 +246,11 @@ impl IngestRouterServiceClient { } #[async_trait::async_trait] impl IngestRouterService for IngestRouterServiceClient { + #[tracing::instrument( + skip_all, + name = "ingest.router.ingest", + fields(commit_type = ?request.commit_type) + )] async fn ingest( &self, request: IngestRequestV2, @@ -576,9 +581,13 @@ where &self, request: IngestRequestV2, ) -> crate::ingest::IngestV2Result { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .ingest(request) + .ingest(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -606,14 +615,28 @@ impl ingest_router_service_grpc_server::IngestRouterServiceGrpc for IngestRouterServiceGrpcServerAdapter { async fn ingest( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .ingest(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!( + "ingest.router.ingest", commit_type = ? request.commit_type + ); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .ingest(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } } /// Generated client implementations. diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index e05001fa7e9..8c3b9e99027 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -1312,258 +1312,301 @@ impl MetastoreServiceClient { } #[async_trait::async_trait] impl MetastoreService for MetastoreServiceClient { + #[tracing::instrument(skip_all, name = "metastore.create_index")] async fn create_index( &self, request: CreateIndexRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.create_index(request).await } + #[tracing::instrument(skip_all, name = "metastore.update_index")] async fn update_index( &self, request: UpdateIndexRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.update_index(request).await } + #[tracing::instrument(skip_all, name = "metastore.index_metadata")] async fn index_metadata( &self, request: IndexMetadataRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.index_metadata(request).await } + #[tracing::instrument(skip_all, name = "metastore.indexes_metadata")] async fn indexes_metadata( &self, request: IndexesMetadataRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.indexes_metadata(request).await } + #[tracing::instrument(skip_all, name = "metastore.list_indexes_metadata")] async fn list_indexes_metadata( &self, request: ListIndexesMetadataRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.list_indexes_metadata(request).await } + #[tracing::instrument(skip_all, name = "metastore.delete_index")] async fn delete_index( &self, request: DeleteIndexRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.delete_index(request).await } + #[tracing::instrument(skip_all, name = "metastore.list_index_stats")] async fn list_index_stats( &self, request: ListIndexStatsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.list_index_stats(request).await } + #[tracing::instrument(skip_all, name = "metastore.list_splits")] async fn list_splits( &self, request: ListSplitsRequest, ) -> crate::metastore::MetastoreResult> { self.inner.0.list_splits(request).await } + #[tracing::instrument(skip_all, name = "metastore.stage_splits")] async fn stage_splits( &self, request: StageSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.stage_splits(request).await } + #[tracing::instrument(skip_all, name = "metastore.publish_splits")] async fn publish_splits( &self, request: PublishSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.publish_splits(request).await } + #[tracing::instrument(skip_all, name = "metastore.mark_splits_for_deletion")] async fn mark_splits_for_deletion( &self, request: MarkSplitsForDeletionRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.mark_splits_for_deletion(request).await } + #[tracing::instrument(skip_all, name = "metastore.delete_splits")] async fn delete_splits( &self, request: DeleteSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.delete_splits(request).await } + #[tracing::instrument(skip_all, name = "metastore.add_source")] async fn add_source( &self, request: AddSourceRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.add_source(request).await } + #[tracing::instrument(skip_all, name = "metastore.update_source")] async fn update_source( &self, request: UpdateSourceRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.update_source(request).await } + #[tracing::instrument(skip_all, name = "metastore.toggle_source")] async fn toggle_source( &self, request: ToggleSourceRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.toggle_source(request).await } + #[tracing::instrument(skip_all, name = "metastore.delete_source")] async fn delete_source( &self, request: DeleteSourceRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.delete_source(request).await } + #[tracing::instrument(skip_all, name = "metastore.reset_source_checkpoint")] async fn reset_source_checkpoint( &self, request: ResetSourceCheckpointRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.reset_source_checkpoint(request).await } + #[tracing::instrument(skip_all, name = "metastore.last_delete_opstamp")] async fn last_delete_opstamp( &self, request: LastDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.last_delete_opstamp(request).await } + #[tracing::instrument(skip_all, name = "metastore.create_delete_task")] async fn create_delete_task( &self, request: DeleteQuery, ) -> crate::metastore::MetastoreResult { self.inner.0.create_delete_task(request).await } + #[tracing::instrument(skip_all, name = "metastore.update_splits_delete_opstamp")] async fn update_splits_delete_opstamp( &self, request: UpdateSplitsDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.update_splits_delete_opstamp(request).await } + #[tracing::instrument(skip_all, name = "metastore.list_delete_tasks")] async fn list_delete_tasks( &self, request: ListDeleteTasksRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.list_delete_tasks(request).await } + #[tracing::instrument(skip_all, name = "metastore.list_stale_splits")] async fn list_stale_splits( &self, request: ListStaleSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.list_stale_splits(request).await } + #[tracing::instrument(skip_all, name = "metastore.open_shards")] async fn open_shards( &self, request: OpenShardsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.open_shards(request).await } + #[tracing::instrument(skip_all, name = "metastore.acquire_shards")] async fn acquire_shards( &self, request: AcquireShardsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.acquire_shards(request).await } + #[tracing::instrument(skip_all, name = "metastore.delete_shards")] async fn delete_shards( &self, request: DeleteShardsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.delete_shards(request).await } + #[tracing::instrument(skip_all, name = "metastore.prune_shards")] async fn prune_shards( &self, request: PruneShardsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.prune_shards(request).await } + #[tracing::instrument(skip_all, name = "metastore.list_shards")] async fn list_shards( &self, request: ListShardsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.list_shards(request).await } + #[tracing::instrument(skip_all, name = "metastore.create_index_template")] async fn create_index_template( &self, request: CreateIndexTemplateRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.create_index_template(request).await } + #[tracing::instrument(skip_all, name = "metastore.get_index_template")] async fn get_index_template( &self, request: GetIndexTemplateRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.get_index_template(request).await } + #[tracing::instrument(skip_all, name = "metastore.find_index_template_matches")] async fn find_index_template_matches( &self, request: FindIndexTemplateMatchesRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.find_index_template_matches(request).await } + #[tracing::instrument(skip_all, name = "metastore.list_index_templates")] async fn list_index_templates( &self, request: ListIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.list_index_templates(request).await } + #[tracing::instrument(skip_all, name = "metastore.delete_index_templates")] async fn delete_index_templates( &self, request: DeleteIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.delete_index_templates(request).await } + #[tracing::instrument(skip_all, name = "metastore.get_cluster_identity")] async fn get_cluster_identity( &self, request: GetClusterIdentityRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.get_cluster_identity(request).await } + #[tracing::instrument(skip_all, name = "metastore.stage_metrics_splits")] async fn stage_metrics_splits( &self, request: StageMetricsSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.stage_metrics_splits(request).await } + #[tracing::instrument(skip_all, name = "metastore.publish_metrics_splits")] async fn publish_metrics_splits( &self, request: PublishMetricsSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.publish_metrics_splits(request).await } + #[tracing::instrument(skip_all, name = "metastore.list_metrics_splits")] async fn list_metrics_splits( &self, request: ListMetricsSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.list_metrics_splits(request).await } + #[tracing::instrument(skip_all, name = "metastore.mark_metrics_splits_for_deletion")] async fn mark_metrics_splits_for_deletion( &self, request: MarkMetricsSplitsForDeletionRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.mark_metrics_splits_for_deletion(request).await } + #[tracing::instrument(skip_all, name = "metastore.delete_metrics_splits")] async fn delete_metrics_splits( &self, request: DeleteMetricsSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.delete_metrics_splits(request).await } + #[tracing::instrument(skip_all, name = "metastore.stage_sketch_splits")] async fn stage_sketch_splits( &self, request: StageSketchSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.stage_sketch_splits(request).await } + #[tracing::instrument(skip_all, name = "metastore.publish_sketch_splits")] async fn publish_sketch_splits( &self, request: PublishSketchSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.publish_sketch_splits(request).await } + #[tracing::instrument(skip_all, name = "metastore.list_sketch_splits")] async fn list_sketch_splits( &self, request: ListSketchSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.list_sketch_splits(request).await } + #[tracing::instrument(skip_all, name = "metastore.mark_sketch_splits_for_deletion")] async fn mark_sketch_splits_for_deletion( &self, request: MarkSketchSplitsForDeletionRequest, ) -> crate::metastore::MetastoreResult { self.inner.0.mark_sketch_splits_for_deletion(request).await } + #[tracing::instrument(skip_all, name = "metastore.delete_sketch_splits")] async fn delete_sketch_splits( &self, request: DeleteSketchSplitsRequest, @@ -6715,9 +6758,13 @@ where &self, request: CreateIndexRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .create_index(request) + .create_index(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6729,9 +6776,13 @@ where &self, request: UpdateIndexRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .update_index(request) + .update_index(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6743,9 +6794,13 @@ where &self, request: IndexMetadataRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .index_metadata(request) + .index_metadata(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6757,9 +6812,13 @@ where &self, request: IndexesMetadataRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .indexes_metadata(request) + .indexes_metadata(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6771,9 +6830,13 @@ where &self, request: ListIndexesMetadataRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .list_indexes_metadata(request) + .list_indexes_metadata(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6785,9 +6848,13 @@ where &self, request: DeleteIndexRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .delete_index(request) + .delete_index(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6799,9 +6866,13 @@ where &self, request: ListIndexStatsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .list_index_stats(request) + .list_index_stats(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6813,9 +6884,13 @@ where &self, request: ListSplitsRequest, ) -> crate::metastore::MetastoreResult> { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .list_splits(request) + .list_splits(tonic_request) .await .map(|response| { let streaming: tonic::Streaming<_> = response.into_inner(); @@ -6835,9 +6910,13 @@ where &self, request: StageSplitsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .stage_splits(request) + .stage_splits(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6849,9 +6928,13 @@ where &self, request: PublishSplitsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .publish_splits(request) + .publish_splits(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6863,9 +6946,13 @@ where &self, request: MarkSplitsForDeletionRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .mark_splits_for_deletion(request) + .mark_splits_for_deletion(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6877,9 +6964,13 @@ where &self, request: DeleteSplitsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .delete_splits(request) + .delete_splits(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6891,9 +6982,13 @@ where &self, request: AddSourceRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .add_source(request) + .add_source(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6905,9 +7000,13 @@ where &self, request: UpdateSourceRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .update_source(request) + .update_source(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6919,9 +7018,13 @@ where &self, request: ToggleSourceRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .toggle_source(request) + .toggle_source(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6933,9 +7036,13 @@ where &self, request: DeleteSourceRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .delete_source(request) + .delete_source(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6947,9 +7054,13 @@ where &self, request: ResetSourceCheckpointRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .reset_source_checkpoint(request) + .reset_source_checkpoint(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6961,9 +7072,13 @@ where &self, request: LastDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .last_delete_opstamp(request) + .last_delete_opstamp(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6975,9 +7090,13 @@ where &self, request: DeleteQuery, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .create_delete_task(request) + .create_delete_task(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -6989,9 +7108,13 @@ where &self, request: UpdateSplitsDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .update_splits_delete_opstamp(request) + .update_splits_delete_opstamp(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7003,9 +7126,13 @@ where &self, request: ListDeleteTasksRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .list_delete_tasks(request) + .list_delete_tasks(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7017,9 +7144,13 @@ where &self, request: ListStaleSplitsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .list_stale_splits(request) + .list_stale_splits(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7031,9 +7162,13 @@ where &self, request: OpenShardsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .open_shards(request) + .open_shards(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7045,9 +7180,13 @@ where &self, request: AcquireShardsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .acquire_shards(request) + .acquire_shards(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7059,9 +7198,13 @@ where &self, request: DeleteShardsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .delete_shards(request) + .delete_shards(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7073,9 +7216,13 @@ where &self, request: PruneShardsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .prune_shards(request) + .prune_shards(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7087,9 +7234,13 @@ where &self, request: ListShardsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .list_shards(request) + .list_shards(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7101,9 +7252,13 @@ where &self, request: CreateIndexTemplateRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .create_index_template(request) + .create_index_template(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7115,9 +7270,13 @@ where &self, request: GetIndexTemplateRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .get_index_template(request) + .get_index_template(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7129,9 +7288,13 @@ where &self, request: FindIndexTemplateMatchesRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .find_index_template_matches(request) + .find_index_template_matches(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7143,9 +7306,13 @@ where &self, request: ListIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .list_index_templates(request) + .list_index_templates(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7157,9 +7324,13 @@ where &self, request: DeleteIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .delete_index_templates(request) + .delete_index_templates(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7171,9 +7342,13 @@ where &self, request: GetClusterIdentityRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .get_cluster_identity(request) + .get_cluster_identity(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7185,9 +7360,13 @@ where &self, request: StageMetricsSplitsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .stage_metrics_splits(request) + .stage_metrics_splits(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7199,9 +7378,13 @@ where &self, request: PublishMetricsSplitsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .publish_metrics_splits(request) + .publish_metrics_splits(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7213,9 +7396,13 @@ where &self, request: ListMetricsSplitsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .list_metrics_splits(request) + .list_metrics_splits(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7227,9 +7414,13 @@ where &self, request: MarkMetricsSplitsForDeletionRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .mark_metrics_splits_for_deletion(request) + .mark_metrics_splits_for_deletion(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7241,9 +7432,13 @@ where &self, request: DeleteMetricsSplitsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .delete_metrics_splits(request) + .delete_metrics_splits(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7255,9 +7450,13 @@ where &self, request: StageSketchSplitsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .stage_sketch_splits(request) + .stage_sketch_splits(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7269,9 +7468,13 @@ where &self, request: PublishSketchSplitsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .publish_sketch_splits(request) + .publish_sketch_splits(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7283,9 +7486,13 @@ where &self, request: ListSketchSplitsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .list_sketch_splits(request) + .list_sketch_splits(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7297,9 +7504,13 @@ where &self, request: MarkSketchSplitsForDeletionRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .mark_sketch_splits_for_deletion(request) + .mark_sketch_splits_for_deletion(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7311,9 +7522,13 @@ where &self, request: DeleteSketchSplitsRequest, ) -> crate::metastore::MetastoreResult { + let mut tonic_request = tonic::Request::new(request); + quickwit_common::tower::otel::inject_current_context( + tonic_request.metadata_mut(), + ); self.inner .clone() - .delete_sketch_splits(request) + .delete_sketch_splits(tonic_request) .await .map(|response| response.into_inner()) .map_err(|status| crate::error::grpc_status_to_service_error( @@ -7356,481 +7571,997 @@ impl metastore_service_grpc_server::MetastoreServiceGrpc for MetastoreServiceGrpcServerAdapter { async fn create_index( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .create_index(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.create_index"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .create_index(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn update_index( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .update_index(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.update_index"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .update_index(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn index_metadata( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .index_metadata(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.index_metadata"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .index_metadata(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn indexes_metadata( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .indexes_metadata(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.indexes_metadata"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .indexes_metadata(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn list_indexes_metadata( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .list_indexes_metadata(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.list_indexes_metadata"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .list_indexes_metadata(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn delete_index( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .delete_index(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.delete_index"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .delete_index(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn list_index_stats( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .list_index_stats(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.list_index_stats"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .list_index_stats(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } type ListSplitsStream = quickwit_common::ServiceStream< tonic::Result, >; async fn list_splits( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .list_splits(request.into_inner()) - .await - .map(|stream| tonic::Response::new( - stream.map_err(crate::error::grpc_error_to_grpc_status), - )) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.list_splits"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .list_splits(request) + .await + .map(|stream| tonic::Response::new( + stream.map_err(crate::error::grpc_error_to_grpc_status), + )) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn stage_splits( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .stage_splits(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.stage_splits"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .stage_splits(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn publish_splits( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .publish_splits(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.publish_splits"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .publish_splits(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn mark_splits_for_deletion( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .mark_splits_for_deletion(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.mark_splits_for_deletion"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .mark_splits_for_deletion(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn delete_splits( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .delete_splits(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.delete_splits"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .delete_splits(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn add_source( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .add_source(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.add_source"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .add_source(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn update_source( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .update_source(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.update_source"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .update_source(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn toggle_source( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .toggle_source(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.toggle_source"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .toggle_source(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn delete_source( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .delete_source(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.delete_source"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .delete_source(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn reset_source_checkpoint( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .reset_source_checkpoint(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.reset_source_checkpoint"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .reset_source_checkpoint(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn last_delete_opstamp( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .last_delete_opstamp(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.last_delete_opstamp"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .last_delete_opstamp(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn create_delete_task( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .create_delete_task(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.create_delete_task"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .create_delete_task(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn update_splits_delete_opstamp( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .update_splits_delete_opstamp(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.update_splits_delete_opstamp"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .update_splits_delete_opstamp(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn list_delete_tasks( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .list_delete_tasks(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.list_delete_tasks"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .list_delete_tasks(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn list_stale_splits( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .list_stale_splits(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.list_stale_splits"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .list_stale_splits(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn open_shards( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .open_shards(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.open_shards"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .open_shards(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn acquire_shards( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .acquire_shards(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.acquire_shards"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .acquire_shards(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn delete_shards( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .delete_shards(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.delete_shards"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .delete_shards(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn prune_shards( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .prune_shards(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.prune_shards"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .prune_shards(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn list_shards( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .list_shards(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.list_shards"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .list_shards(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn create_index_template( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .create_index_template(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.create_index_template"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .create_index_template(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn get_index_template( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .get_index_template(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.get_index_template"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .get_index_template(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn find_index_template_matches( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .find_index_template_matches(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.find_index_template_matches"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .find_index_template_matches(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn list_index_templates( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .list_index_templates(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.list_index_templates"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .list_index_templates(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn delete_index_templates( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .delete_index_templates(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.delete_index_templates"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .delete_index_templates(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn get_cluster_identity( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .get_cluster_identity(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.get_cluster_identity"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .get_cluster_identity(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn stage_metrics_splits( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .stage_metrics_splits(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.stage_metrics_splits"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .stage_metrics_splits(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn publish_metrics_splits( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .publish_metrics_splits(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.publish_metrics_splits"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .publish_metrics_splits(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn list_metrics_splits( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .list_metrics_splits(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.list_metrics_splits"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .list_metrics_splits(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn mark_metrics_splits_for_deletion( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .mark_metrics_splits_for_deletion(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.mark_metrics_splits_for_deletion"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .mark_metrics_splits_for_deletion(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn delete_metrics_splits( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .delete_metrics_splits(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.delete_metrics_splits"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .delete_metrics_splits(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn stage_sketch_splits( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .stage_sketch_splits(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.stage_sketch_splits"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .stage_sketch_splits(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn publish_sketch_splits( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .publish_sketch_splits(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.publish_sketch_splits"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .publish_sketch_splits(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn list_sketch_splits( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .list_sketch_splits(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.list_sketch_splits"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .list_sketch_splits(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn mark_sketch_splits_for_deletion( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .mark_sketch_splits_for_deletion(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.mark_sketch_splits_for_deletion"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .mark_sketch_splits_for_deletion(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } async fn delete_sketch_splits( &self, - request: tonic::Request, + tonic_request: tonic::Request, ) -> Result, tonic::Status> { - self.inner - .0 - .delete_sketch_splits(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) + let parent_context = quickwit_common::tower::otel::extract_context( + tonic_request.metadata(), + ); + let request = tonic_request.into_inner(); + let span = tracing::info_span!("metastore.delete_sketch_splits"); + let _ = ::set_parent( + &span, + parent_context, + ); + let fut = async move { + self.inner + .0 + .delete_sketch_splits(request) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + }; + <_ as tracing::Instrument>::instrument(fut, span).await } } /// Generated client implementations. diff --git a/quickwit/quickwit-proto/src/lib.rs b/quickwit/quickwit-proto/src/lib.rs index 3a7f7ea8992..428387c6353 100644 --- a/quickwit/quickwit-proto/src/lib.rs +++ b/quickwit/quickwit-proto/src/lib.rs @@ -20,13 +20,6 @@ use std::cmp::Ordering; -use ::opentelemetry::global; -use ::opentelemetry::propagation::{Extractor, Injector}; -use tonic::Status; -use tonic::service::Interceptor; -use tracing::Span; -use tracing_opentelemetry::OpenTelemetrySpanExt; - pub mod cluster; pub mod control_plane; pub use bytes; @@ -126,87 +119,6 @@ impl TryFrom for search::SearchRequest { } } -/// `MutMetadataMap` used to extract [`tonic::metadata::MetadataMap`] from a request. -pub struct MutMetadataMap<'a>(&'a mut tonic::metadata::MetadataMap); - -impl Injector for MutMetadataMap<'_> { - /// Sets a key-value pair in the [`tonic::metadata::MetadataMap`]. No-op if the key or value - /// is invalid. - fn set(&mut self, key: &str, value: String) { - if let Ok(metadata_key) = tonic::metadata::MetadataKey::from_bytes(key.as_bytes()) - && let Ok(metadata_value) = tonic::metadata::MetadataValue::try_from(&value) - { - self.0.insert(metadata_key, metadata_value); - } - } -} - -impl Extractor for MutMetadataMap<'_> { - /// Gets a value for a key from the `MetadataMap`. If the value can't be converted to &str, - /// returns None. - fn get(&self, key: &str) -> Option<&str> { - self.0.get(key).and_then(|metadata| metadata.to_str().ok()) - } - - /// Collect all the keys from the `MetadataMap`. - fn keys(&self) -> Vec<&str> { - self.0 - .keys() - .map(|key| match key { - tonic::metadata::KeyRef::Ascii(v) => v.as_str(), - tonic::metadata::KeyRef::Binary(v) => v.as_str(), - }) - .collect::>() - } -} - -/// [`tonic::service::interceptor::Interceptor`] which injects the span context into -/// [`tonic::metadata::MetadataMap`]. -#[derive(Clone, Debug)] -pub struct SpanContextInterceptor; - -impl Interceptor for SpanContextInterceptor { - fn call(&mut self, mut request: tonic::Request<()>) -> Result, Status> { - global::get_text_map_propagator(|propagator| { - propagator.inject_context( - &tracing::Span::current().context(), - &mut MutMetadataMap(request.metadata_mut()), - ) - }); - Ok(request) - } -} - -/// `MetadataMap` extracts OpenTelemetry -/// tracing keys from request's headers. -struct MetadataMap<'a>(&'a tonic::metadata::MetadataMap); - -impl Extractor for MetadataMap<'_> { - /// Gets a value for a key from the `MetadataMap`. If the value can't be converted to &str, - /// returns None. - fn get(&self, key: &str) -> Option<&str> { - self.0.get(key).and_then(|metadata| metadata.to_str().ok()) - } - - /// Collect all the keys from the `MetadataMap`. - fn keys(&self) -> Vec<&str> { - self.0 - .keys() - .map(|key| match key { - tonic::metadata::KeyRef::Ascii(v) => v.as_str(), - tonic::metadata::KeyRef::Binary(v) => v.as_str(), - }) - .collect::>() - } -} - -/// Sets parent span context derived from [`tonic::metadata::MetadataMap`]. -pub fn set_parent_span_from_request_metadata(request_metadata: &tonic::metadata::MetadataMap) { - let parent_cx = - global::get_text_map_propagator(|prop| prop.extract(&MetadataMap(request_metadata))); - let _ = Span::current().set_parent(parent_cx); -} - impl search::SortOrder { #[inline(always)] pub fn compare_opt(&self, this: &Option, other: &Option) -> Ordering { diff --git a/quickwit/quickwit-search/src/client.rs b/quickwit/quickwit-search/src/client.rs index 194bf0b2bd0..eb75d2b5782 100644 --- a/quickwit/quickwit-search/src/client.rs +++ b/quickwit/quickwit-search/src/client.rs @@ -19,11 +19,12 @@ use std::time::Duration; use bytesize::ByteSize; use http::Uri; +use quickwit_common::tower::otel::SpanContextInterceptor; use quickwit_proto::search::{GetKvRequest, PutKvRequest, ReportSplitsRequest}; +use quickwit_proto::tonic; use quickwit_proto::tonic::Request; use quickwit_proto::tonic::codegen::InterceptedService; use quickwit_proto::tonic::transport::{Channel, Endpoint}; -use quickwit_proto::{SpanContextInterceptor, tonic}; use tower::timeout::Timeout; use tracing::warn; diff --git a/quickwit/quickwit-serve/src/search_api/grpc_adapter.rs b/quickwit/quickwit-serve/src/search_api/grpc_adapter.rs index c5250ee2465..43866227397 100644 --- a/quickwit/quickwit-serve/src/search_api/grpc_adapter.rs +++ b/quickwit/quickwit-serve/src/search_api/grpc_adapter.rs @@ -15,12 +15,13 @@ use std::sync::Arc; use async_trait::async_trait; +use quickwit_common::tower::otel::set_current_span_parent_from_metadata; use quickwit_proto::error::convert_to_grpc_result; use quickwit_proto::search::{ GetKvRequest, GetKvResponse, LeafListFieldsRequest, ListFieldsRequest, ListFieldsResponse, ReportSplitsRequest, ReportSplitsResponse, search_service_server as grpc, }; -use quickwit_proto::{set_parent_span_from_request_metadata, tonic}; +use quickwit_proto::tonic; use quickwit_search::SearchService; use tracing::instrument; @@ -40,7 +41,7 @@ impl grpc::SearchService for GrpcSearchAdapter { &self, request: tonic::Request, ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); + set_current_span_parent_from_metadata(request.metadata()); let search_request = request.into_inner(); let search_result = self.0.root_search(search_request).await; convert_to_grpc_result(search_result) @@ -51,7 +52,7 @@ impl grpc::SearchService for GrpcSearchAdapter { &self, request: tonic::Request, ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); + set_current_span_parent_from_metadata(request.metadata()); let leaf_search_request = request.into_inner(); let leaf_search_result = self.0.leaf_search(leaf_search_request).await; convert_to_grpc_result(leaf_search_result) @@ -62,7 +63,7 @@ impl grpc::SearchService for GrpcSearchAdapter { &self, request: tonic::Request, ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); + set_current_span_parent_from_metadata(request.metadata()); let fetch_docs_request = request.into_inner(); let fetch_docs_result = self.0.fetch_docs(fetch_docs_request).await; convert_to_grpc_result(fetch_docs_result) @@ -73,7 +74,7 @@ impl grpc::SearchService for GrpcSearchAdapter { &self, request: tonic::Request, ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); + set_current_span_parent_from_metadata(request.metadata()); let search_request = request.into_inner(); let search_result = self.0.root_list_terms(search_request).await; convert_to_grpc_result(search_result) @@ -84,7 +85,7 @@ impl grpc::SearchService for GrpcSearchAdapter { &self, request: tonic::Request, ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); + set_current_span_parent_from_metadata(request.metadata()); let leaf_search_request = request.into_inner(); let leaf_search_result = self.0.leaf_list_terms(leaf_search_request).await; convert_to_grpc_result(leaf_search_result) @@ -104,7 +105,7 @@ impl grpc::SearchService for GrpcSearchAdapter { &self, request: tonic::Request, ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); + set_current_span_parent_from_metadata(request.metadata()); let put_request = request.into_inner(); self.0.put_kv(put_request).await; Ok(tonic::Response::new( @@ -117,7 +118,7 @@ impl grpc::SearchService for GrpcSearchAdapter { &self, request: tonic::Request, ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); + set_current_span_parent_from_metadata(request.metadata()); let get_search_after_context_request = request.into_inner(); let payload = self.0.get_kv(get_search_after_context_request).await; let get_response = GetKvResponse { payload }; @@ -129,7 +130,7 @@ impl grpc::SearchService for GrpcSearchAdapter { &self, request: tonic::Request, ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); + set_current_span_parent_from_metadata(request.metadata()); let get_search_after_context_request = request.into_inner(); self.0.report_splits(get_search_after_context_request).await; Ok(tonic::Response::new(ReportSplitsResponse {})) @@ -140,7 +141,7 @@ impl grpc::SearchService for GrpcSearchAdapter { &self, request: tonic::Request, ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); + set_current_span_parent_from_metadata(request.metadata()); let resp = self.0.root_list_fields(request.into_inner()).await; convert_to_grpc_result(resp) } @@ -149,7 +150,7 @@ impl grpc::SearchService for GrpcSearchAdapter { &self, request: tonic::Request, ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); + set_current_span_parent_from_metadata(request.metadata()); let resp = self.0.leaf_list_fields(request.into_inner()).await; convert_to_grpc_result(resp) } @@ -159,7 +160,7 @@ impl grpc::SearchService for GrpcSearchAdapter { &self, request: tonic::Request, ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); + set_current_span_parent_from_metadata(request.metadata()); let search_request = request.into_inner(); let search_result = self.0.search_plan(search_request).await; convert_to_grpc_result(search_result) diff --git a/quickwit/quickwit-storage/src/local_file_storage.rs b/quickwit/quickwit-storage/src/local_file_storage.rs index ec5fdbbd633..ad4c068472c 100644 --- a/quickwit/quickwit-storage/src/local_file_storage.rs +++ b/quickwit/quickwit-storage/src/local_file_storage.rs @@ -171,6 +171,7 @@ impl Storage for LocalFileStorage { Ok(()) } + #[tracing::instrument(level = "debug", skip(self, payload), fields(payload_len = payload.len()))] async fn put( &self, path: &Path, @@ -199,6 +200,7 @@ impl Storage for LocalFileStorage { Ok(()) } + #[tracing::instrument(level = "debug", skip(self, output))] async fn copy_to(&self, path: &Path, output: &mut dyn SendableAsync) -> StorageResult<()> { let full_path = self.full_path(path)?; let mut file = tokio::fs::File::open(&full_path).await?; @@ -242,6 +244,7 @@ impl Storage for LocalFileStorage { Ok(Box::new(file.take(range.len() as u64))) } + #[tracing::instrument(level = "debug", skip(self))] async fn delete(&self, path: &Path) -> StorageResult<()> { self.delete_single_file(path).await?; if let Some(parent) = path.parent() @@ -255,6 +258,7 @@ impl Storage for LocalFileStorage { /// Deletes the files identified by `paths` concurrently, with a maximum of `10` syscalls at a /// time. Additionally, deletes the parent directories of `paths` if they are empty after the /// first round of deletions. + #[tracing::instrument(level = "debug", skip(self, paths), fields(num_paths = paths.len()))] async fn bulk_delete<'a>(&self, paths: &[&'a Path]) -> Result<(), BulkDeleteError> { let mut successes = Vec::with_capacity(paths.len()); let mut failures = HashMap::new(); @@ -306,6 +310,7 @@ impl Storage for LocalFileStorage { }) } + #[tracing::instrument(level = "debug", skip(self))] async fn get_all(&self, path: &Path) -> StorageResult { let full_path = self.full_path(path)?; let content_bytes = tokio::fs::read(full_path).await.map_err(|err| { @@ -322,6 +327,7 @@ impl Storage for LocalFileStorage { &self.uri } + #[tracing::instrument(level = "debug", skip(self))] async fn file_num_bytes(&self, path: &Path) -> StorageResult { let full_path = self.full_path(path)?; match tokio::fs::metadata(full_path).await { diff --git a/quickwit/quickwit-storage/src/object_storage/azure_blob_storage.rs b/quickwit/quickwit-storage/src/object_storage/azure_blob_storage.rs index e10ec20b7c6..4ac552fc0a4 100644 --- a/quickwit/quickwit-storage/src/object_storage/azure_blob_storage.rs +++ b/quickwit/quickwit-storage/src/object_storage/azure_blob_storage.rs @@ -344,6 +344,7 @@ impl Storage for AzureBlobStorage { Ok(()) } + #[instrument(level = "debug", skip(self, payload), fields(payload_len = payload.len()))] async fn put( &self, path: &Path, @@ -363,6 +364,7 @@ impl Storage for AzureBlobStorage { Ok(()) } + #[instrument(level = "debug", skip(self, output))] async fn copy_to(&self, path: &Path, output: &mut dyn SendableAsync) -> StorageResult<()> { let name = self.blob_name(path); let mut output_stream = self.container_client.blob_client(name).get().into_stream(); @@ -384,6 +386,7 @@ impl Storage for AzureBlobStorage { Ok(()) } + #[instrument(level = "debug", skip(self))] async fn delete(&self, path: &Path) -> StorageResult<()> { let blob_name = self.blob_name(path); let delete_res: Result<_, StorageError> = self @@ -397,6 +400,7 @@ impl Storage for AzureBlobStorage { Ok(()) } + #[instrument(level = "debug", skip(self, paths), fields(num_paths = paths.len()))] async fn bulk_delete<'a>(&self, paths: &[&'a Path]) -> Result<(), BulkDeleteError> { // See https://github.com/Azure/azure-sdk-for-rust/issues/1068 warn!( @@ -506,6 +510,7 @@ impl Storage for AzureBlobStorage { Ok(data) } + #[instrument(level = "debug", skip(self))] async fn file_num_bytes(&self, path: &Path) -> StorageResult { let name = self.blob_name(path); let properties_result = self diff --git a/quickwit/quickwit-storage/src/object_storage/s3_compatible_storage.rs b/quickwit/quickwit-storage/src/object_storage/s3_compatible_storage.rs index aede14b7e21..c7c124ffda4 100644 --- a/quickwit/quickwit-storage/src/object_storage/s3_compatible_storage.rs +++ b/quickwit/quickwit-storage/src/object_storage/s3_compatible_storage.rs @@ -769,6 +769,7 @@ impl Storage for S3CompatibleObjectStorage { Ok(()) } + #[instrument(level = "debug", skip(self, payload), fields(payload_len = payload.len()))] async fn put( &self, path: &Path, @@ -788,6 +789,7 @@ impl Storage for S3CompatibleObjectStorage { Ok(()) } + #[instrument(level = "debug", skip(self, output))] async fn copy_to(&self, path: &Path, output: &mut dyn SendableAsync) -> StorageResult<()> { let _permit = REQUEST_SEMAPHORE.acquire().await; let get_object_output = @@ -801,6 +803,7 @@ impl Storage for S3CompatibleObjectStorage { Ok(()) } + #[instrument(level = "debug", skip(self))] async fn delete(&self, path: &Path) -> StorageResult<()> { let _permit = REQUEST_SEMAPHORE.acquire().await; let bucket = self.bucket.clone(); @@ -828,6 +831,7 @@ impl Storage for S3CompatibleObjectStorage { } } + #[instrument(level = "debug", skip(self, paths), fields(num_paths = paths.len()))] async fn bulk_delete<'a>(&self, paths: &[&'a Path]) -> Result<(), BulkDeleteError> { if self.disable_multi_object_delete { self.bulk_delete_single(paths).await @@ -887,6 +891,7 @@ impl Storage for S3CompatibleObjectStorage { Ok(bytes) } + #[instrument(level = "debug", skip(self))] async fn file_num_bytes(&self, path: &Path) -> StorageResult { let _permit = REQUEST_SEMAPHORE.acquire().await; let bucket = self.bucket.clone(); diff --git a/quickwit/quickwit-storage/src/opendal_storage/base.rs b/quickwit/quickwit-storage/src/opendal_storage/base.rs index 9bdb21378c9..b517c7f339a 100644 --- a/quickwit/quickwit-storage/src/opendal_storage/base.rs +++ b/quickwit/quickwit-storage/src/opendal_storage/base.rs @@ -22,6 +22,7 @@ use opendal::{DeleteInput, IntoDeleteInput, Operator}; use quickwit_common::uri::Uri; use tokio::io::{AsyncRead, AsyncWriteExt as TokioAsyncWriteExt}; use tokio_util::compat::{FuturesAsyncReadCompatExt, FuturesAsyncWriteCompatExt}; +use tracing::instrument; use crate::metrics::object_storage_get_slice_in_flight_guards; use crate::stable_deref_bytes::into_owned_bytes; @@ -79,6 +80,7 @@ impl Storage for OpendalStorage { Ok(()) } + #[instrument(level = "debug", skip(self, payload), fields(payload_len = payload.len()))] async fn put(&self, path: &Path, payload: Box) -> StorageResult<()> { crate::STORAGE_METRICS.object_storage_put_total.inc(); let path = path.as_os_str().to_string_lossy(); @@ -99,6 +101,7 @@ impl Storage for OpendalStorage { Ok(()) } + #[instrument(level = "debug", skip(self, output))] async fn copy_to(&self, path: &Path, output: &mut dyn SendableAsync) -> StorageResult<()> { let path = path.as_os_str().to_string_lossy(); let mut storage_reader = self @@ -116,6 +119,7 @@ impl Storage for OpendalStorage { Ok(()) } + #[instrument(level = "debug", skip(self, range), fields(range.start = range.start, range.end = range.end))] async fn get_slice(&self, path: &Path, range: Range) -> StorageResult { let path = path.as_os_str().to_string_lossy(); let size = range.len(); @@ -131,6 +135,7 @@ impl Storage for OpendalStorage { Ok(into_owned_bytes(storage_content)) } + #[instrument(level = "debug", skip(self, range), fields(range.start = range.start, range.end = range.end))] async fn get_slice_stream( &self, path: &Path, @@ -148,12 +153,14 @@ impl Storage for OpendalStorage { Ok(Box::new(storage_reader)) } + #[instrument(level = "debug", skip(self))] async fn get_all(&self, path: &Path) -> StorageResult { let path = path.as_os_str().to_string_lossy(); let storage_content = self.op.read(&path).await?.to_bytes(); Ok(into_owned_bytes(storage_content)) } + #[instrument(level = "debug", skip(self))] async fn delete(&self, path: &Path) -> StorageResult<()> { let path = path.as_os_str().to_string_lossy(); crate::STORAGE_METRICS @@ -166,6 +173,7 @@ impl Storage for OpendalStorage { Ok(()) } + #[instrument(level = "debug", skip(self, paths), fields(num_paths = paths.len()))] async fn bulk_delete<'a>(&self, paths: &[&'a Path]) -> Result<(), BulkDeleteError> { // The mock service we used in integration testsuite doesn't support bulk delete. // Let's fallback to delete one by one in this case. @@ -225,6 +233,7 @@ impl Storage for OpendalStorage { Ok(()) } + #[instrument(level = "debug", skip(self))] async fn file_num_bytes(&self, path: &Path) -> StorageResult { let path = path.as_os_str().to_string_lossy(); let meta = self.op.stat(&path).await?;