diff --git a/rust/otap-dataflow/.gitignore b/rust/otap-dataflow/.gitignore index 3b65317179..360cd5149a 100644 --- a/rust/otap-dataflow/.gitignore +++ b/rust/otap-dataflow/.gitignore @@ -40,4 +40,9 @@ output.json resolved-schema.yaml # Ignore the temporary repo dir created by `cargo xtask history` command -history-temp-repo/ \ No newline at end of file +history-temp-repo/ + +*.patch +perf.data +perf.data.* +*.strace diff --git a/rust/otap-dataflow/Cargo.toml b/rust/otap-dataflow/Cargo.toml index 3924a1b525..ad2490255c 100644 --- a/rust/otap-dataflow/Cargo.toml +++ b/rust/otap-dataflow/Cargo.toml @@ -37,6 +37,8 @@ thiserror.workspace = true serde_json.workspace = true clap.workspace = true mimalloc-rust.workspace = true +env_logger.workspace = true +jemallocator = { workspace = true, optional = true } [workspace.dependencies] otap-df-pdata-otlp-macros = { path = "./crates/pdata/src/otlp/macros"} @@ -64,6 +66,7 @@ criterion = "0.7.0" data-encoding = "2.9.0" fluke-hpack = "0.3.1" flume = { version = "0.11.1", default-features = false, features = ["async"] } +flate2 = "1.1.5" futures = "0.3.31" futures-channel = "0.3" futures-timer = "3.0" @@ -76,7 +79,9 @@ local-sync = "0.1.1" log = "0.4" miette = { version="7.6.0", features = ["fancy"] } mimalloc-rust = "0.2.1" -nix = { version = "0.30.0", features = ["process", "signal"] } +jemalloc-ctl = "0.5.4" +jemallocator = "0.5.4" +nix = { version = "0.30.0", features = ["process", "signal", "resource", "feature"] } num_enum = "0.7" object_store = "0.12.3" once_cell = "1.20.2" @@ -105,9 +110,9 @@ socket2 = { version = "0.6.0", features = ["all"] } syn = { version = "2.0", features = ["full", "extra-traits"] } tempfile = "3" thiserror = "2.0.12" -tokio = { version = "1.46.1", features = ["rt", "time", "net", "io-util", "sync", "macros", "rt-multi-thread", "fs", "io-std", "process"] } +tokio = { version = "1.48.0", features = ["rt", "time", "net", "io-util", "sync", "macros", "rt-multi-thread", "fs", "io-std", "process"] } tokio-stream = "0.1.17" -tokio-util = { version = "0.7.16" } +tokio-util = { version = "0.7.17" } tonic = { version = "0.14", default-features = false, features = [ "channel", "codegen", @@ -133,15 +138,22 @@ weaver_resolved_schema = { git = "https://github.com/open-telemetry/weaver.git", weaver_resolver = { git = "https://github.com/open-telemetry/weaver.git", tag = "v0.17.0"} weaver_semconv = { git = "https://github.com/open-telemetry/weaver.git", tag = "v0.17.0"} zip = "=4.2.0" +byte-unit = "5.2.0" +bytes = "1.10.1" +env_logger = "0.11" +h2 = "0.4.7" +zstd = "0.13" [features] default = [] -unsafe-optimizations = ["unchecked-index", "unchecked-arithmetic"] +unsafe-optimizations = ["unchecked-index", "unchecked-arithmetic", "otap-df-otap/unsafe-optimizations"] unchecked-index = [] unchecked-arithmetic = [] # Experimental exporters (opt-in) experimental-exporters = ["otap-df-otap/experimental-exporters"] geneva-exporter = ["otap-df-otap/geneva-exporter"] +jemalloc-metrics = ["otap-df-otap/jemalloc-metrics"] +jemalloc-allocator = ["dep:jemallocator"] [workspace.lints.rust] # General compatibility lints @@ -201,6 +213,12 @@ missing_crate_level_docs = "deny" [profile.release] debug = "line-tables-only" # minimum required for profiling +#opt-level = 3 +#lto = "thin" +#codegen-units = 1 +#panic = "abort" +#incremental = false +#strip = "debuginfo" # A more in-depth analysis is necessary to determine the optimal parameters for the release profile. #[profile.release] diff --git a/rust/otap-dataflow/benchmarks/benches/exporter/main.rs b/rust/otap-dataflow/benchmarks/benches/exporter/main.rs index 2216b21eee..76bf915be1 100644 --- a/rust/otap-dataflow/benchmarks/benches/exporter/main.rs +++ b/rust/otap-dataflow/benchmarks/benches/exporter/main.rs @@ -413,7 +413,7 @@ fn bench_exporter(c: &mut Criterion) { |b, otap_signals| { b.to_async(&rt).iter(|| async { // start perf exporter - let config = Config::new(1000, 0.3, true, true, true, true, true); + let config = Config::new(1000, 0.3, true, true, true, true, true, false); let exporter_config = ExporterConfig::new("perf_exporter"); let node_config = Arc::new(NodeUserConfig::new_exporter_config(OTAP_PERF_EXPORTER_URN)); @@ -473,7 +473,7 @@ fn bench_exporter(c: &mut Criterion) { |b, otap_signals| { b.to_async(&rt).iter(|| async { // start perf exporter - let config = Config::new(1000, 0.3, false, false, false, false, false); + let config = Config::new(1000, 0.3, false, false, false, false, false, false); let exporter_config = ExporterConfig::new("perf_exporter"); let node_config = Arc::new(NodeUserConfig::new_exporter_config(OTAP_PERF_EXPORTER_URN)); diff --git a/rust/otap-dataflow/configs/fake-otlp.yaml b/rust/otap-dataflow/configs/fake-otlp.yaml index 7c358fcc55..4511750be1 100644 --- a/rust/otap-dataflow/configs/fake-otlp.yaml +++ b/rust/otap-dataflow/configs/fake-otlp.yaml @@ -14,14 +14,13 @@ nodes: dispatch_strategy: round_robin config: traffic_config: - signals_per_second: 100000 - max_signal_count: null - metric_weight: 0 - trace_weight: 0 - log_weight: 30 + max_batch_size: 1000 + signals_per_second: 75000 + log_weight: 100 registry_path: https://github.com/open-telemetry/semantic-conventions.git[model] exporter: kind: exporter plugin_urn: "urn:otel:otlp:exporter" config: - grpc_endpoint: "http://127.0.0.1:4317" + grpc_endpoint: "http://127.0.0.1:4327" + #compression_method: zstd diff --git a/rust/otap-dataflow/configs/otlp-otlp.yaml b/rust/otap-dataflow/configs/otlp-otlp.yaml index ff974256d0..3c1c5b19b5 100644 --- a/rust/otap-dataflow/configs/otlp-otlp.yaml +++ b/rust/otap-dataflow/configs/otlp-otlp.yaml @@ -6,18 +6,20 @@ settings: nodes: receiver: kind: receiver - plugin_urn: "urn:otel:otlp:receiver" + plugin_urn: "urn:otel:otel:receiver" out_ports: out_port: destinations: - exporter dispatch_strategy: round_robin config: - listening_addr: "127.0.0.1:4317" + listening_addr: "127.0.0.1:4327" + request_compression: none # timeout: "30s" # Optional: timeout for RPC requests exporter: kind: exporter plugin_urn: "urn:otel:otlp:exporter" config: - grpc_endpoint: "http://127.0.0.1:4318" + grpc_endpoint: "http://127.0.0.1:4328" + #compression_method: zstd # timeout: "15s" # Optional: timeout for RPC requests diff --git a/rust/otap-dataflow/configs/otlp-perf.yaml b/rust/otap-dataflow/configs/otlp-perf.yaml index f63e63f947..9906815608 100644 --- a/rust/otap-dataflow/configs/otlp-perf.yaml +++ b/rust/otap-dataflow/configs/otlp-perf.yaml @@ -6,20 +6,23 @@ settings: nodes: receiver: kind: receiver - plugin_urn: "urn:otel:otlp:receiver" + plugin_urn: "urn:otel:otel:receiver" out_ports: out_port: destinations: - perf dispatch_strategy: round_robin config: - listening_addr: "127.0.0.1:4318" + listening_addr: "127.0.0.1:4328" + request_compression: none + timeout: "10s" perf: kind: exporter plugin_urn: "urn:otel:otap:perf:exporter" config: frequency: 1000 - cpu_usage: false - mem_usage: false + display_throughput: true + cpu_usage: true + mem_usage: true disk_usage: false io_usage: false diff --git a/rust/otap-dataflow/crates/config/Cargo.toml b/rust/otap-dataflow/crates/config/Cargo.toml index 37b993445b..02782fb980 100644 --- a/rust/otap-dataflow/crates/config/Cargo.toml +++ b/rust/otap-dataflow/crates/config/Cargo.toml @@ -20,3 +20,4 @@ serde_yaml = { workspace = true } miette = { workspace = true } urn = { workspace = true } schemars = { workspace = true } +byte-unit = { workspace = true } \ No newline at end of file diff --git a/rust/otap-dataflow/crates/config/src/byte_units.rs b/rust/otap-dataflow/crates/config/src/byte_units.rs new file mode 100644 index 0000000000..c4bed89c7f --- /dev/null +++ b/rust/otap-dataflow/crates/config/src/byte_units.rs @@ -0,0 +1,127 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Support for byte units like "KB / KiB", "MB / MiB", "GB / GiB" in configuration files. + +use byte_unit::Byte; +use serde::de::Error as DeError; +use serde::{Deserialize, Deserializer}; + +#[derive(Deserialize)] +#[serde(untagged)] +enum Value { + Number(u64), + String(String), +} + +/// Deserialize an optional byte size that can be specified either as a number (in bytes) +/// or as a string with units (e.g. "1 KB", "2 MiB"). +pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> +where + D: Deserializer<'de>, +{ + let value = Option::::deserialize(deserializer)?; + let Some(value) = value else { + return Ok(None); + }; + + let (bytes, repr) = match value { + Value::Number(value) => (value as u128, value.to_string()), + Value::String(text) => { + let parsed: Byte = text.parse().map_err(DeError::custom)?; + (parsed.as_u64() as u128, text) + } + }; + + if bytes > u32::MAX as u128 { + return Err(DeError::custom(format!( + "byte size '{}' ({} bytes) exceeds u32::MAX ({} bytes)", + repr, + bytes, + u32::MAX + ))); + } + + Ok(Some(bytes as u32)) +} + +#[cfg(test)] +mod tests { + use super::deserialize; + use serde::Deserialize; + + #[derive(Debug, Deserialize)] + struct Holder { + #[serde(default, deserialize_with = "deserialize")] + value: Option, + } + + fn de_yaml(input: &str) -> Result { + serde_yaml::from_str::(input) + } + + #[test] + fn parses_number_as_bytes() { + let cfg = de_yaml("value: 1024").expect("should parse numeric bytes"); + assert_eq!(cfg.value, Some(1024)); + } + + #[test] + fn parses_string_with_iec_units() { + // 1 KiB == 1024 bytes + let cfg = de_yaml("value: 1 KiB").expect("should parse 1 KiB"); + assert_eq!(cfg.value, Some(1024)); + + // 2 MiB == 2 * 1024 * 1024 bytes + let cfg = de_yaml("value: '2 MiB'").expect("should parse 2 MiB"); + assert_eq!(cfg.value, Some(2 * 1024 * 1024)); + } + + #[test] + fn parses_plain_string_number() { + let cfg = de_yaml("value: '2048'").expect("should parse plain numeric string"); + assert_eq!(cfg.value, Some(2048)); + } + + #[test] + fn missing_value_is_none() { + let cfg = de_yaml("{}").expect("should parse with missing field as None"); + assert_eq!(cfg.value, None); + } + + #[test] + fn overflow_is_rejected() { + // 4 GiB == 4 * 1024^3 bytes = 4_294_967_296 > u32::MAX (4_294_967_295) + let err = de_yaml("value: 4 GiB").expect_err("should error for overflow"); + let msg = err.to_string(); + assert!( + msg.contains("exceeds u32::MAX"), + "unexpected error: {}", + msg + ); + } + + #[test] + fn parses_no_space_decimal_units() { + let cfg = de_yaml("value: 1KB").expect("should parse 1KB without space"); + assert_eq!(cfg.value, Some(1000)); + + let cfg = de_yaml("value: 10MB").expect("should parse 10MB without space"); + assert_eq!(cfg.value, Some(10_000_000)); + + // Lowercase 'b' should still be treated as bytes per crate behavior + let cfg = de_yaml("value: 1kb").expect("should parse 1kb as 1000 bits => 125 bytes"); + assert_eq!(cfg.value, Some(125)); + } + + #[test] + fn parses_fractional_values_and_rounding() { + // Decimal unit with fraction + let cfg = de_yaml("value: '1.5 MB'").expect("should parse 1.5 MB"); + assert_eq!(cfg.value, Some(1_500_000)); + + // Binary unit with fraction (exact) + let cfg = de_yaml("value: '0.5 KiB'").expect("should parse 0.5 KiB to 512 bytes"); + assert_eq!(cfg.value, Some(512)); + } +} diff --git a/rust/otap-dataflow/crates/config/src/lib.rs b/rust/otap-dataflow/crates/config/src/lib.rs index d97f890724..58fed90485 100644 --- a/rust/otap-dataflow/crates/config/src/lib.rs +++ b/rust/otap-dataflow/crates/config/src/lib.rs @@ -15,6 +15,7 @@ use serde::{Deserialize, Serialize}; use std::borrow::Cow; +pub mod byte_units; pub mod engine; pub mod error; pub mod health; diff --git a/rust/otap-dataflow/crates/engine/src/admitter.rs b/rust/otap-dataflow/crates/engine/src/admitter.rs new file mode 100644 index 0000000000..7a014cee34 --- /dev/null +++ b/rust/otap-dataflow/crates/engine/src/admitter.rs @@ -0,0 +1,254 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Admission Controller (single-threaded, thread-per-core friendly) +//! +//! ToDo Add a memory usage based admitter that tracks memory usage and rejects new connections/datagrams when memory usage exceeds a threshold. +//! ToDo Add other types of admitters (e.g. rate-limiting, token-bucket, ...) + +use std::cell::Cell; +use std::rc::Rc; + +/// The top-level admission classes the per-cpu controller understands. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum AdmissionClass { + /// A new transport connection (e.g. TCP / HTTP/2). + Connection, + /// A single connectionless datagram of work (e.g. UDP, QUIC, Unix datagram). + Datagram, +} + +/// The decision returned by admission attempts. +/// +/// The type parameter `T` is the guard returned on success (e.g. `ConnectionGuard`, +/// `DatagramGuard`, or a wrapper `Guard` enum for convenience). +#[derive(Debug)] +#[must_use] +pub enum AdmitDecision { + /// The request was admitted. The returned guard must be kept alive for the + /// lifetime of the work. Dropping it releases the reserved slot(s). + Admitted(T), + /// Temporary backpressure (limit reached but system otherwise healthy). The caller + /// may retry, pause accepting, or send a soft error (e.g. gRPC `RESOURCE_EXHAUSTED`). + Busy, + /// Hard rejection (e.g. circuit breaker/policy). Consider dropping the connection or + /// returning a definitive error. + Reject { + /// Stable, human-readable reason (useful for metrics/alerts). + message: &'static str, + }, +} + +/// Internal counters shared by the `Admitter` and its guards. +/// +/// All counters are "single-threaded" and updated via `Cell`. +#[derive(Debug)] +struct Inner { + // Connections + max_conns: u32, + inflight_conns: Cell, + + // Streams-per-connection (enforced inside each connection's state) + max_streams_per_conn: u32, + + // Datagrams + max_datagrams: u32, + inflight_datagrams: Cell, + + // Per-processor circuit breaker + breaker_open: Cell, +} + +/// Admission controller (single-threaded). +/// +/// This controller caps in-flight connections, in-flight streams per connection, and in-flight +/// datagrams. It also exposes a per-processor circuit breaker to hard-reject new admissions +/// immediately under global pressure. +/// +/// Note: This type is `Rc`-based and not thread-safe. It is intended to be used +/// in a thread-per-core model where each core has its own `Admitter` instance. It also means +/// that cloning is cheap and can be done freely to hand out references to guards. +#[derive(Debug, Clone)] +#[must_use] +pub struct Admitter { + inner: Rc, +} + +impl Admitter { + /// Construct a new `Admitter`. + /// + /// - `max_conns`: maximum in-flight connections. + /// - `max_streams_per_conn`: maximum in-flight streams per connection. + /// - `max_datagrams`: maximum in-flight datagrams (connectionless work). + pub fn new(max_conns: u32, max_streams_per_conn: u32, max_datagrams: u32) -> Self { + Self { + inner: Rc::new(Inner { + max_conns, + inflight_conns: Cell::new(0), + max_streams_per_conn, + max_datagrams, + inflight_datagrams: Cell::new(0), + breaker_open: Cell::new(false), + }), + } + } + + /// Open or close the local circuit breaker. + /// + /// When open, all new admissions are hard-rejected (`Reject { message: "circuit_breaker_open" }`). + pub fn set_breaker(&self, open: bool) { + self.inner.breaker_open.set(open); + } + + /// Admit a connection, returning a `ConnectionGuard` on success. + pub fn try_admit_connection(&self) -> AdmitDecision { + if self.inner.breaker_open.get() { + return AdmitDecision::Reject { + message: "circuit_breaker_open", + }; + } + let cur = self.inner.inflight_conns.get(); + if cur >= self.inner.max_conns { + return AdmitDecision::Busy; + } + self.inner.inflight_conns.set(cur + 1); + + let state = Rc::new(ConnState { + streams_inflight: Cell::new(0), + max_streams: self.inner.max_streams_per_conn, + }); + + AdmitDecision::Admitted(ConnectionGuard { + inner: Rc::clone(&self.inner), + state, + }) + } + + /// Admit a datagram (connectionless unit), returning a `DatagramGuard` on success. + pub fn try_admit_datagram(&self) -> AdmitDecision { + if self.inner.breaker_open.get() { + return AdmitDecision::Reject { + message: "circuit_breaker_open", + }; + } + let cur = self.inner.inflight_datagrams.get(); + if cur >= self.inner.max_datagrams { + return AdmitDecision::Busy; + } + self.inner.inflight_datagrams.set(cur + 1); + AdmitDecision::Admitted(DatagramGuard { + inner: Rc::clone(&self.inner), + }) + } + + /// Read-only observability snapshot. + pub fn report(&self) -> Report { + Report { + inflight_conns: self.inner.inflight_conns.get(), + max_conns: self.inner.max_conns, + inflight_datagrams: self.inner.inflight_datagrams.get(), + max_datagrams: self.inner.max_datagrams, + breaker_open: self.inner.breaker_open.get(), + } + } +} + +/// Per-connection state tracking in-flight streams. +#[derive(Debug)] +struct ConnState { + streams_inflight: Cell, + max_streams: u32, +} + +/// Guard representing a live connection. +/// +/// While this guard is alive: +/// - the global in-flight connection count is incremented +/// - you can open up to `max_streams_per_conn` streams via `try_open_stream()`. +#[derive(Debug)] +pub struct ConnectionGuard { + inner: Rc, + state: Rc, +} + +impl ConnectionGuard { + /// Attempt to open a stream on this connection. + /// + /// Enforces the per-connection stream limit. On success, returns a `StreamGuard` + /// that must be held for the stream's lifetime. + pub fn try_open_stream(&self) -> AdmitDecision { + if self.inner.breaker_open.get() { + return AdmitDecision::Reject { + message: "circuit_breaker_open", + }; + } + let cur = self.state.streams_inflight.get(); + if cur >= self.state.max_streams { + return AdmitDecision::Busy; + } + self.state.streams_inflight.set(cur + 1); + AdmitDecision::Admitted(StreamGuard { + state: Rc::clone(&self.state), + }) + } + + /// Current number of in-flight streams on this connection. + #[must_use] + pub fn streams_inflight(&self) -> u32 { + self.state.streams_inflight.get() + } +} + +impl Drop for ConnectionGuard { + fn drop(&mut self) { + // Return one connection slot. + let cur = self.inner.inflight_conns.get(); + self.inner.inflight_conns.set(cur.saturating_sub(1)); + // Any remaining streams are assumed dropped earlier; if not, + // they will decrement their own counters as they drop. + } +} + +/// Guard representing a live stream (logical sub-unit inside a connection). +/// +/// While this guard is alive, the connection's in-flight stream count is incremented. +#[derive(Debug)] +pub struct StreamGuard { + state: Rc, +} + +impl Drop for StreamGuard { + fn drop(&mut self) { + let cur = self.state.streams_inflight.get(); + self.state.streams_inflight.set(cur.saturating_sub(1)); + } +} + +/// Guard representing a live datagram (connectionless unit). +#[derive(Debug)] +pub struct DatagramGuard { + inner: Rc, +} + +impl Drop for DatagramGuard { + fn drop(&mut self) { + let cur = self.inner.inflight_datagrams.get(); + self.inner.inflight_datagrams.set(cur.saturating_sub(1)); + } +} + +/// Lightweight observability snapshot for counters and breaker state. +#[derive(Debug, Clone, Copy)] +#[must_use] +pub struct Report { + /// Current number of in-flight connections. + pub inflight_conns: u32, + /// Configured maximum number of in-flight connections. + pub max_conns: u32, + /// Current number of in-flight datagrams. + pub inflight_datagrams: u32, + /// Configured maximum number of in-flight datagrams. + pub max_datagrams: u32, + /// Whether the circuit breaker is open (true) or closed (false). + pub breaker_open: bool, +} diff --git a/rust/otap-dataflow/crates/engine/src/effect_handler.rs b/rust/otap-dataflow/crates/engine/src/effect_handler.rs index 5e2db73cc6..ed7b60d5cd 100644 --- a/rust/otap-dataflow/crates/engine/src/effect_handler.rs +++ b/rust/otap-dataflow/crates/engine/src/effect_handler.rs @@ -159,7 +159,7 @@ impl EffectHandlerCore { /// Reports the provided metrics to the engine. #[allow(dead_code)] // Will be used in the future. ToDo report metrics from channel and messages. pub(crate) fn report_metrics( - &mut self, + &self, metrics: &mut MetricSet, ) -> Result<(), TelemetryError> { self.metrics_reporter.report(metrics) diff --git a/rust/otap-dataflow/crates/engine/src/lib.rs b/rust/otap-dataflow/crates/engine/src/lib.rs index 11fd54e3c3..89f54a34a5 100644 --- a/rust/otap-dataflow/crates/engine/src/lib.rs +++ b/rust/otap-dataflow/crates/engine/src/lib.rs @@ -35,6 +35,7 @@ pub mod message; pub mod processor; pub mod receiver; +pub mod admitter; mod attributes; pub mod config; pub mod context; diff --git a/rust/otap-dataflow/crates/engine/src/local/receiver.rs b/rust/otap-dataflow/crates/engine/src/local/receiver.rs index c9167b4e1a..adc9e87e33 100644 --- a/rust/otap-dataflow/crates/engine/src/local/receiver.rs +++ b/rust/otap-dataflow/crates/engine/src/local/receiver.rs @@ -46,6 +46,7 @@ use otap_df_telemetry::metrics::{MetricSet, MetricSetHandler}; use otap_df_telemetry::reporter::MetricsReporter; use std::collections::HashMap; use std::net::SocketAddr; +use std::rc::Rc; use std::time::Duration; use tokio::net::{TcpListener, UdpSocket}; @@ -121,10 +122,14 @@ impl ControlChannel { } /// A `!Send` implementation of the EffectHandler. +/// Note: This handler MUST be cloneable with low overhead. #[derive(Clone)] pub struct EffectHandler { - core: EffectHandlerCore, + inner: Rc>, +} +struct EffectHandlerInner { + core: EffectHandlerCore, /// A sender used to forward messages from the receiver. /// Supports multiple named output ports. msg_senders: HashMap>, @@ -155,23 +160,24 @@ impl EffectHandler { None }; - EffectHandler { + let inner = Rc::new(EffectHandlerInner { core, msg_senders, default_sender, - } + }); + EffectHandler { inner } } /// Returns the id of the receiver associated with this handler. #[must_use] pub fn receiver_id(&self) -> NodeId { - self.core.node_id() + self.inner.core.node_id() } /// Returns the list of connected out ports for this receiver. #[must_use] pub fn connected_ports(&self) -> Vec { - self.msg_senders.keys().cloned().collect() + self.inner.msg_senders.keys().cloned().collect() } /// Sends a message to the next node(s) in the pipeline using the default port. @@ -185,7 +191,7 @@ impl EffectHandler { /// [`TypedError::Error::ReceiverError`] if the default port is not configured. #[inline] pub async fn send_message(&self, data: PData) -> Result<(), TypedError> { - match &self.default_sender { + match &self.inner.default_sender { Some(sender) => sender .send(data) .await @@ -213,7 +219,7 @@ impl EffectHandler { P: Into, { let port_name: PortName = port.into(); - match self.msg_senders.get(&port_name) { + match self.inner.msg_senders.get(&port_name) { Some(sender) => sender .send(data) .await @@ -238,7 +244,7 @@ impl EffectHandler { /// /// Returns an [`Error::IoError`] if any step in the process fails. pub fn tcp_listener(&self, addr: SocketAddr) -> Result { - self.core.tcp_listener(addr, self.receiver_id()) + self.inner.core.tcp_listener(addr, self.receiver_id()) } /// Creates a non-blocking UDP socket on the given address with socket options defined by the @@ -249,7 +255,7 @@ impl EffectHandler { /// /// Returns an [`Error::IoError`] if any step in the process fails. pub fn udp_socket(&self, addr: SocketAddr) -> Result { - self.core.udp_socket(addr, self.receiver_id()) + self.inner.core.udp_socket(addr, self.receiver_id()) } /// Print an info message to stdout. @@ -257,7 +263,7 @@ impl EffectHandler { /// This method provides a standardized way for receivers to output /// informational messages without blocking the async runtime. pub async fn info(&self, message: &str) { - self.core.info(message).await; + self.inner.core.info(message).await; } /// Starts a cancellable periodic timer that emits TimerTick on the control channel. @@ -268,7 +274,7 @@ impl EffectHandler { &self, duration: Duration, ) -> Result, Error> { - self.core.start_periodic_timer(duration).await + self.inner.core.start_periodic_timer(duration).await } /// Starts a cancellable periodic telemetry timer that emits CollectTelemetry. @@ -276,16 +282,16 @@ impl EffectHandler { &self, duration: Duration, ) -> Result, Error> { - self.core.start_periodic_telemetry(duration).await + self.inner.core.start_periodic_telemetry(duration).await } /// Reports metrics collected by the receiver. #[allow(dead_code)] // Will be used in the future. ToDo report metrics from channel and messages. pub(crate) fn report_metrics( - &mut self, + &self, metrics: &mut MetricSet, ) -> Result<(), TelemetryError> { - self.core.report_metrics(metrics) + self.inner.core.report_metrics(metrics) } // More methods will be added in the future as needed. diff --git a/rust/otap-dataflow/crates/engine/src/shared/receiver.rs b/rust/otap-dataflow/crates/engine/src/shared/receiver.rs index e2e0bd4d1d..92e421e29b 100644 --- a/rust/otap-dataflow/crates/engine/src/shared/receiver.rs +++ b/rust/otap-dataflow/crates/engine/src/shared/receiver.rs @@ -46,6 +46,7 @@ use otap_df_telemetry::metrics::{MetricSet, MetricSetHandler}; use otap_df_telemetry::reporter::MetricsReporter; use std::collections::HashMap; use std::net::SocketAddr; +use std::sync::Arc; use std::time::Duration; use tokio::net::TcpListener; @@ -89,14 +90,18 @@ impl ControlChannel { } /// A `Send` implementation of the EffectHandlerTrait. +/// Note: This handler MUST be cloneable with low overhead. #[derive(Clone)] pub struct EffectHandler { - core: EffectHandlerCore, + inner: Arc>, +} +struct EffectHandlerInner { + core: EffectHandlerCore, /// A sender used to forward messages from the receiver. /// Supports multiple named output ports. msg_senders: HashMap>, - /// Cached default sender for fast access in the hot path + /// Cached default sender for fast access in the hot path. default_sender: Option>, } @@ -126,23 +131,25 @@ impl EffectHandler { None }; - EffectHandler { + let inner = Arc::new(EffectHandlerInner { core, msg_senders, default_sender, - } + }); + + EffectHandler { inner } } /// Returns the name of the receiver associated with this handler. #[must_use] pub fn receiver_id(&self) -> NodeId { - self.core.node_id() + self.inner.core.node_id() } /// Returns the list of connected out ports for this receiver. #[must_use] pub fn connected_ports(&self) -> Vec { - self.msg_senders.keys().cloned().collect() + self.inner.msg_senders.keys().cloned().collect() } /// Sends a message to the next node(s) in the pipeline. @@ -152,7 +159,7 @@ impl EffectHandler { /// Returns an [`Error::ReceiverError`] if the message could not be routed to a port. #[inline] pub async fn send_message(&self, data: PData) -> Result<(), TypedError> { - match &self.default_sender { + match &self.inner.default_sender { Some(sender) => sender .send(data) .await @@ -175,7 +182,7 @@ impl EffectHandler { P: Into, { let port_name: PortName = port.into(); - match self.msg_senders.get(&port_name) { + match self.inner.msg_senders.get(&port_name) { Some(sender) => sender .send(data) .await @@ -200,7 +207,7 @@ impl EffectHandler { /// /// Returns an [`Error::IoError`] if any step in the process fails. pub fn tcp_listener(&self, addr: SocketAddr) -> Result { - self.core.tcp_listener(addr, self.receiver_id()) + self.inner.core.tcp_listener(addr, self.receiver_id()) } /// Print an info message to stdout. @@ -208,7 +215,7 @@ impl EffectHandler { /// This method provides a standardized way for receivers to output /// informational messages without blocking the async runtime. pub async fn info(&self, message: &str) { - self.core.info(message).await; + self.inner.core.info(message).await; } /// Starts a cancellable periodic timer that emits TimerTick on the control channel. @@ -219,7 +226,7 @@ impl EffectHandler { &self, duration: Duration, ) -> Result, Error> { - self.core.start_periodic_timer(duration).await + self.inner.core.start_periodic_timer(duration).await } /// Starts a cancellable periodic telemetry timer that emits CollectTelemetry. @@ -227,7 +234,7 @@ impl EffectHandler { &self, duration: Duration, ) -> Result, Error> { - self.core.start_periodic_telemetry(duration).await + self.inner.core.start_periodic_telemetry(duration).await } /// Reports metrics collected by the receiver. @@ -236,7 +243,7 @@ impl EffectHandler { &mut self, metrics: &mut MetricSet, ) -> Result<(), TelemetryError> { - self.core.report_metrics(metrics) + self.inner.core.report_metrics(metrics) } // More methods will be added in the future as needed. diff --git a/rust/otap-dataflow/crates/otap/Cargo.toml b/rust/otap-dataflow/crates/otap/Cargo.toml index d5ac0784da..c19f5d91a2 100644 --- a/rust/otap-dataflow/crates/otap/Cargo.toml +++ b/rust/otap-dataflow/crates/otap/Cargo.toml @@ -29,16 +29,24 @@ serde_with = { workspace = true } serde_json = { workspace = true } slotmap = { workspace = true } tokio = { workspace = true } +tokio-util = { workspace = true } linkme = { workspace = true } uuid.workspace = true weaver_forge.workspace = true weaver_common.workspace = true tonic = { workspace = true } tonic-middleware = { workspace = true } +h2 = { workspace = true } +zstd = { workspace = true } tonic-prost = { workspace = true } prost = { workspace = true } smallvec = { workspace = true } bitflags = { workspace = true } +bytes = { workspace = true } +parking_lot = { workspace = true } +flate2 = { workspace = true } +nix = { workspace = true } +jemalloc-ctl = { workspace = true, optional = true } otap-df-engine = { path = "../engine" } otap-df-engine-macros = { path = "../engine-macros" } @@ -60,6 +68,7 @@ weaver_resolved_schema.workspace = true weaver_resolver.workspace = true rand.workspace = true zip.workspace = true +pin-project-lite = "0.2.16" # Geneva exporter dependencies geneva-uploader = { version = "0.3.0", optional = true } @@ -70,6 +79,9 @@ opentelemetry-proto = { workspace = true, optional = true } experimental-exporters = [] # Experimental exporters geneva-exporter = ["experimental-exporters", "dep:geneva-uploader", "dep:opentelemetry-proto"] +# Opt-in unsafe micro-optimizations used in the OTEL receiver hot path. +unsafe-optimizations = [] +jemalloc-metrics = ["dep:jemalloc-ctl"] [dev-dependencies] flume = { workspace = true } diff --git a/rust/otap-dataflow/crates/otap/README.md b/rust/otap-dataflow/crates/otap/README.md index dd92ba2afd..7fe1a9ae80 100644 --- a/rust/otap-dataflow/crates/otap/README.md +++ b/rust/otap-dataflow/crates/otap/README.md @@ -1,7 +1,7 @@ # OTAP Pipeline The OTAP (OpenTelemetry Arrow Protocol) crate contains receivers, processors, -and exporters supporting natively the OTAP Pdata. +and exporters natively supporting OTAP Pdata. ## Receivers @@ -42,3 +42,72 @@ In the root of the repository, run: ```bash cargo xtask compile-proto ``` + +## OTel Receiver (experimental) + +The experimental OTEL receiver implements both the OTAP Arrow streaming +endpoints and the OTLP unary `Export` services directly on top of `h2`, +optimized for the thread-per-core runtime. A single listener now accepts +standard OTLP gRPC clients alongside the Arrow-based OTAP streams, sharing the +same concurrency/ACK handling. + +![OTel Receiver Design](assets/otel_receiver_design.png) + +### High-level flow + +1. **Local receiver** - The `OtelReceiver` implements the `local::Receiver` + trait so it fits into the thread-per-core design of the OTAP pipeline + engine. +2. **TCP accept + admission control** - The receiver listens on the configured + address using the framework's `EffectHandler`. Every accepted socket passes + through an `Admitter`, which enforces global connection limits before we + spawn a local task dedicated to that connection. +3. **HTTP/2 handshake and keepalive** - `handle_tcp_conn` performs the h2 + handshake, wires up optional keepalive pings, and then repeatedly accepts + inbound gRPC streams. A per-connection `ConnectionGuard` provides + `try_open_stream` so we can apply stream-level backpressure. +4. **Request routing** - Each gRPC stream is handled by + `GrpcRequestRouter::handle_request`, which: + - Parses compression headers and negotiates the response encoding. + - Wraps the raw `RecvStream` in a `GrpcStreamingBody`, decoding the 5-byte + gRPC frame headers and decompressing payloads on demand (Zstd/Gzip/Deflate). + - Dispatches to `stream_batch_statuses`, providing the signal-specific + `AckRegistry` when `wait_for_result` is enabled. +5. **Pipeline dispatch + ACK/NACK** - `stream_batch_statuses` converts each + `BatchArrowRecords` into `OtapPdata`, pushes it through the local + `EffectHandler::send_message` (bounded by `max_concurrent_requests`), and + optionally registers an ACK wait slot. ACK/NACK control messages from the + pipeline re-enter the receiver via the control channel and are routed back to + the `AckRegistry`, completing futures inside `stream_batch_statuses`. The + resulting `BatchStatus` entries stream back to the client. + +### Configurable parameters + +- **Network / TCP** - `listening_addr`, `tcp_nodelay`, optional HTTP/2 + keepalive interval/timeout. (Future work: TLS acceptor, keepalive retries, ...) +- **HTTP/2 / gRPC** - `initial_stream_window_size`, + `initial_connection_window_size`, `max_frame_size`, request/response + compression lists, `max_concurrent_requests`, per-connection + `max_in_flight_per_connection`, optional `timeout` enforced via + `RequestTimeout`. +- **Flow control** - The global `Admitter` (connection/stream limits) and the + per-request ACK registry both bind in-flight work. When `wait_for_result` is + disabled, batches still flow through the effect handler but ACK registries are + skipped. + +### Key implementation notes + +- The entire stack runs `!Send`: each connection, stream, and ACK future stays on + the per-core runtime. This avoids cross-thread synchronization and matches the + "thread-per-core" approach of the overall engine. +- Compression handling lives in `grpc.rs` (`GrpcStreamingBody` and + `GrpcResponseFrameEncoder`). Zstd allocates long-lived encoder/decoder instances per + stream, while gzip/deflate instantiate per-frame decoders because they are + lightweight. +- `stream.rs` contains the core state machine (`StatusStreamState`) that keeps a + bounded set of in-flight batches, feeds the pipeline, and yields + `BatchStatus` results as soon as ACK/NACK completions arrive. +- Control-channel integration happens at the top of `start`: ACK/NACK messages + from downstream nodes are routed via `route_local_ack_response` / + `route_local_nack_response`, completing the registries highlighted in the + design diagram. diff --git a/rust/otap-dataflow/crates/otap/assets/otel_receiver_design.png b/rust/otap-dataflow/crates/otap/assets/otel_receiver_design.png new file mode 100644 index 0000000000..9c25b497e6 Binary files /dev/null and b/rust/otap-dataflow/crates/otap/assets/otel_receiver_design.png differ diff --git a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs index 0f8c19cb64..dc9f407e3d 100644 --- a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs @@ -284,9 +284,7 @@ impl local::Processor for AttributesProcessor { ) -> Result<(), EngineError> { match msg { Message::Control(control_msg) => match control_msg { - otap_df_engine::control::NodeControlMsg::CollectTelemetry { - mut metrics_reporter, - } => { + otap_df_engine::control::NodeControlMsg::CollectTelemetry { metrics_reporter } => { if let Some(metrics) = self.metrics.as_mut() { let _ = metrics_reporter.report(metrics); } @@ -629,7 +627,7 @@ mod tests { let mut bytes = Vec::new(); input.encode(&mut bytes).expect("encode"); let pdata_in = - OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes.into()).into()); ctx.process(Message::PData(pdata_in)) .await .expect("process"); @@ -644,7 +642,7 @@ mod tests { OtlpProtoBytes::ExportLogsRequest(b) => b, _ => panic!("unexpected otlp variant"), }; - let decoded = ExportLogsServiceRequest::decode(bytes.as_slice()).expect("decode"); + let decoded = ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode"); // Resource should still have key "a" let res_attrs = &decoded.resource_logs[0] @@ -709,7 +707,7 @@ mod tests { .run_test(|mut ctx| async move { let mut bytes = Vec::new(); input.encode(&mut bytes).expect("encode"); - let pdata_in = OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + let pdata_in = OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes.into()).into()); ctx.process(Message::PData(pdata_in)) .await .expect("process"); @@ -722,7 +720,7 @@ mod tests { OtlpProtoBytes::ExportLogsRequest(b) => b, _ => panic!("unexpected otlp variant"), }; - let decoded = ExportLogsServiceRequest::decode(bytes.as_slice()).expect("decode"); + let decoded = ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode"); // Resource should still have key "a" let res_attrs = &decoded.resource_logs[0] @@ -793,7 +791,7 @@ mod tests { let mut bytes = Vec::new(); input.encode(&mut bytes).expect("encode"); let pdata_in = - OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes.into()).into()); ctx.process(Message::PData(pdata_in)) .await .expect("process"); @@ -805,7 +803,7 @@ mod tests { OtlpProtoBytes::ExportLogsRequest(b) => b, _ => panic!("unexpected otlp variant"), }; - let decoded = ExportLogsServiceRequest::decode(bytes.as_slice()).expect("decode"); + let decoded = ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode"); // Resource 'a' should be deleted let res_attrs = &decoded.resource_logs[0] @@ -868,7 +866,7 @@ mod tests { let mut bytes = Vec::new(); input.encode(&mut bytes).expect("encode"); let pdata_in = - OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes.into()).into()); ctx.process(Message::PData(pdata_in)) .await .expect("process"); @@ -880,7 +878,7 @@ mod tests { OtlpProtoBytes::ExportLogsRequest(b) => b, _ => panic!("unexpected otlp variant"), }; - let decoded = ExportLogsServiceRequest::decode(bytes.as_slice()).expect("decode"); + let decoded = ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode"); // Resource 'a' should remain let res_attrs = &decoded.resource_logs[0] @@ -947,7 +945,7 @@ mod tests { let mut bytes = Vec::new(); input.encode(&mut bytes).expect("encode"); let pdata_in = - OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes.into()).into()); ctx.process(Message::PData(pdata_in)) .await .expect("process"); @@ -959,7 +957,7 @@ mod tests { OtlpProtoBytes::ExportLogsRequest(b) => b, _ => panic!("unexpected otlp variant"), }; - let decoded = ExportLogsServiceRequest::decode(bytes.as_slice()).expect("decode"); + let decoded = ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode"); // Resource 'a' should be deleted; 'r' should remain let res_attrs = &decoded.resource_logs[0] @@ -1083,8 +1081,9 @@ mod telemetry_tests { phase .run_test(|mut ctx| async move { // Process one message - let pdata = - OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(input_bytes).into()); + let pdata = OtapPdata::new_default( + OtlpProtoBytes::ExportLogsRequest(input_bytes.into()).into(), + ); ctx.process(Message::PData(pdata)).await.expect("pdata"); // Trigger telemetry snapshot diff --git a/rust/otap-dataflow/crates/otap/src/batch_processor.rs b/rust/otap-dataflow/crates/otap/src/batch_processor.rs index 568579a49d..ae1fb73d82 100644 --- a/rust/otap-dataflow/crates/otap/src/batch_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/batch_processor.rs @@ -692,13 +692,11 @@ impl local::Processor for OtapBatchProcessor { effect.info(LOG_MSG_SHUTTING_DOWN).await; Ok(()) } - NodeControlMsg::CollectTelemetry { - mut metrics_reporter, - } => metrics_reporter.report(&mut self.metrics).map_err(|e| { - EngineError::InternalError { + NodeControlMsg::CollectTelemetry { metrics_reporter } => metrics_reporter + .report(&mut self.metrics) + .map_err(|e| EngineError::InternalError { message: e.to_string(), - } - }), + }), NodeControlMsg::DelayedData { .. } => { unreachable!("unused"); } @@ -1409,8 +1407,9 @@ mod tests { let validation = phase.run_test(|mut ctx| async move { // Metrics OTLP bytes are not yet supported for conversion -> should be dropped - let pdata = - OtapPdata::new_default(OtlpProtoBytes::ExportMetricsRequest(vec![1, 2, 3]).into()); + let pdata = OtapPdata::new_default( + OtlpProtoBytes::ExportMetricsRequest(bytes::Bytes::from(vec![1, 2, 3])).into(), + ); ctx.process(Message::PData(pdata)).await.expect("process 1"); let emitted = ctx.drain_pdata().await; assert_eq!( @@ -1437,8 +1436,9 @@ mod tests { let phase = test_rt.set_processor(proc); let validation = phase.run_test(|mut ctx| async move { - let pdata = - OtapPdata::new_default(OtlpProtoBytes::ExportMetricsRequest(vec![9, 9, 9]).into()); + let pdata = OtapPdata::new_default( + OtlpProtoBytes::ExportMetricsRequest(bytes::Bytes::from(vec![9, 9, 9])).into(), + ); ctx.process(Message::PData(pdata)).await.expect("process"); let emitted = ctx.drain_pdata().await; assert_eq!(emitted.len(), 0, "no flush before shutdown"); diff --git a/rust/otap-dataflow/crates/otap/src/compression.rs b/rust/otap-dataflow/crates/otap/src/compression.rs index 72d2df7d6d..0b60be2bcb 100644 --- a/rust/otap-dataflow/crates/otap/src/compression.rs +++ b/rust/otap-dataflow/crates/otap/src/compression.rs @@ -1,15 +1,15 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 -//! -//! Defines a compression enum to abstract from tonic and allows the exporter and receiver to get the respective tonic equivalent -//! +//! Defines a compression enum to abstract from tonic and allows the exporter and receiver to get +//! the respective tonic equivalent. -use serde::{Deserialize, Serialize}; +use crate::otel_receiver::grpc::GrpcEncoding; +use serde::{Deserialize, Deserializer, Serialize}; use tonic::codec::CompressionEncoding; /// Enum to represent various compression methods -#[derive(Debug, Clone, Serialize, Deserialize)] +#[derive(Debug, Clone, Copy, Serialize, Deserialize, PartialEq, Eq)] #[serde(rename_all = "snake_case")] pub enum CompressionMethod { /// Fastest compression @@ -26,16 +26,84 @@ impl CompressionMethod { #[must_use] pub const fn map_to_compression_encoding(&self) -> CompressionEncoding { match *self { - CompressionMethod::Gzip => CompressionEncoding::Gzip, CompressionMethod::Zstd => CompressionEncoding::Zstd, + CompressionMethod::Gzip => CompressionEncoding::Gzip, CompressionMethod::Deflate => CompressionEncoding::Deflate, } } + + /// Maps a negotiated GrpcEncoding back to a CompressionMethod, if known. + #[must_use] + pub fn from_grpc_encoding(enc: GrpcEncoding) -> Option { + match enc { + GrpcEncoding::Zstd => Some(CompressionMethod::Zstd), + GrpcEncoding::Gzip => Some(CompressionMethod::Gzip), + GrpcEncoding::Deflate => Some(CompressionMethod::Deflate), + GrpcEncoding::Identity => None, + } + } +} + +/// Default set of compression methods that are accepted when no configuration is provided. +pub const DEFAULT_COMPRESSION_METHODS: [CompressionMethod; 3] = [ + CompressionMethod::Zstd, + CompressionMethod::Gzip, + CompressionMethod::Deflate, +]; + +#[derive(Deserialize)] +#[serde(untagged)] +enum CompressionConfigValue { + Single(CompressionMethod), + List(Vec), + NoneKeyword(CompressionNone), +} + +#[derive(Deserialize)] +#[serde(rename_all = "snake_case")] +enum CompressionNone { + None, +} + +/// Deserializer that accepts either a single compression method, a list, or the string `"none"`. +/// Absence of the field keeps the default behaviour (all methods). +pub fn deserialize_compression_methods<'de, D>( + deserializer: D, +) -> Result>, D::Error> +where + D: Deserializer<'de>, +{ + let value = Option::::deserialize(deserializer)?; + let Some(value) = value else { + return Ok(None); + }; + + let methods = match value { + CompressionConfigValue::Single(method) => vec![method], + CompressionConfigValue::List(methods) => methods, + CompressionConfigValue::NoneKeyword(CompressionNone::None) => Vec::new(), + }; + + let mut deduped = Vec::with_capacity(methods.len()); + for method in methods { + if !deduped.contains(&method) { + deduped.push(method); + } + } + + Ok(Some(deduped)) } #[cfg(test)] mod tests { use super::*; + use serde::Deserialize; + + #[derive(Debug, Deserialize)] + struct ConfWithCompression { + #[serde(default, deserialize_with = "deserialize_compression_methods")] + methods: Option>, + } #[test] fn compression_method_accepts_snake_case_only() { @@ -52,4 +120,37 @@ mod tests { assert!(serde_json::from_str::("\"Zstd\"").is_err()); assert!(serde_json::from_str::("\"Deflate\"").is_err()); } + + #[test] + fn deserialize_supports_single_value() { + let conf: ConfWithCompression = serde_json::from_str(r#"{ "methods": "gzip" }"#).unwrap(); + assert_eq!(conf.methods, Some(vec![CompressionMethod::Gzip])); + } + + #[test] + fn deserialize_supports_list() { + let conf: ConfWithCompression = + serde_json::from_str(r#"{ "methods": ["gzip", "zstd", "gzip"] }"#).unwrap(); + assert_eq!( + conf.methods, + Some(vec![CompressionMethod::Gzip, CompressionMethod::Zstd]) + ); + } + + #[test] + fn deserialize_supports_none_keyword() { + let conf: ConfWithCompression = serde_json::from_str(r#"{ "methods": "none" }"#).unwrap(); + assert_eq!(conf.methods, Some(vec![])); + } + + #[test] + fn deserialize_supports_absence() { + #[derive(Debug, Deserialize)] + struct Conf { + #[serde(default, deserialize_with = "deserialize_compression_methods")] + methods: Option>, + } + let conf: Conf = serde_json::from_str("{}").unwrap(); + assert_eq!(conf.methods, None); + } } diff --git a/rust/otap-dataflow/crates/otap/src/debug_processor.rs b/rust/otap-dataflow/crates/otap/src/debug_processor.rs index c8efbb9023..cc8216783b 100644 --- a/rust/otap-dataflow/crates/otap/src/debug_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/debug_processor.rs @@ -256,9 +256,7 @@ impl local::Processor for DebugProcessor { .output_message(&format!("NACK received after {:?}\n", dd.elapsed())) .await?; } - NodeControlMsg::CollectTelemetry { - mut metrics_reporter, - } => { + NodeControlMsg::CollectTelemetry { metrics_reporter } => { _ = metrics_reporter.report(&mut self.metrics); } _ => {} @@ -289,7 +287,7 @@ impl local::Processor for DebugProcessor { match otlp_bytes { OtlpProtoBytes::ExportLogsRequest(bytes) => { if active_signals.contains(&SignalActive::Logs) { - let req = LogsData::decode(bytes.as_slice()).map_err(|e| { + let req = LogsData::decode(bytes.as_ref()).map_err(|e| { Error::PdataConversionError { error: format!("error decoding proto bytes: {e}"), } @@ -300,7 +298,7 @@ impl local::Processor for DebugProcessor { } OtlpProtoBytes::ExportMetricsRequest(bytes) => { if active_signals.contains(&SignalActive::Metrics) { - let req = MetricsData::decode(bytes.as_slice()).map_err(|e| { + let req = MetricsData::decode(bytes.as_ref()).map_err(|e| { Error::PdataConversionError { error: format!("error decoding proto bytes: {e}"), } @@ -311,7 +309,7 @@ impl local::Processor for DebugProcessor { } OtlpProtoBytes::ExportTracesRequest(bytes) => { if active_signals.contains(&SignalActive::Spans) { - let req = TracesData::decode(bytes.as_slice()).map_err(|e| { + let req = TracesData::decode(bytes.as_ref()).map_err(|e| { Error::PdataConversionError { error: format!("error decoding proto bytes: {e}"), } @@ -617,7 +615,7 @@ mod tests { .encode(&mut bytes) .expect("failed to encode log data into bytes"); let otlp_logs_bytes = - OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes.into()).into()); ctx.process(Message::PData(otlp_logs_bytes)) .await .expect("failed to process"); @@ -671,8 +669,9 @@ mod tests { metrics_data .encode(&mut bytes) .expect("failed to encode log data into bytes"); - let otlp_metrics_bytes = - OtapPdata::new_default(OtlpProtoBytes::ExportMetricsRequest(bytes).into()); + let otlp_metrics_bytes = OtapPdata::new_default( + OtlpProtoBytes::ExportMetricsRequest(bytes.into()).into(), + ); ctx.process(Message::PData(otlp_metrics_bytes)) .await .expect("failed to process"); @@ -735,8 +734,9 @@ mod tests { traces_data .encode(&mut bytes) .expect("failed to encode log data into bytes"); - let otlp_traces_bytes = - OtapPdata::new_default(OtlpProtoBytes::ExportTracesRequest(bytes).into()); + let otlp_traces_bytes = OtapPdata::new_default( + OtlpProtoBytes::ExportTracesRequest(bytes.into()).into(), + ); ctx.process(Message::PData(otlp_traces_bytes)) .await .expect("failed to process"); diff --git a/rust/otap-dataflow/crates/otap/src/fake_data_generator.rs b/rust/otap-dataflow/crates/otap/src/fake_data_generator.rs index 07a2ad0a81..5e62a2a85e 100644 --- a/rust/otap-dataflow/crates/otap/src/fake_data_generator.rs +++ b/rust/otap-dataflow/crates/otap/src/fake_data_generator.rs @@ -137,7 +137,7 @@ impl local::Receiver for FakeGeneratorReceiver { ctrl_msg = ctrl_msg_recv.recv() => { match ctrl_msg { Ok(NodeControlMsg::CollectTelemetry { - mut metrics_reporter, + metrics_reporter, }) => { _ = metrics_reporter.report(&mut self.metrics); } @@ -410,15 +410,19 @@ impl TryFrom for OtapPdata { Ok(match value { OtlpProtoMessage::Logs(logs_data) => { logs_data.encode(&mut bytes)?; - OtapPdata::new_todo_context(OtlpProtoBytes::ExportLogsRequest(bytes).into()) + OtapPdata::new_todo_context(OtlpProtoBytes::ExportLogsRequest(bytes.into()).into()) } OtlpProtoMessage::Metrics(metrics_data) => { metrics_data.encode(&mut bytes)?; - OtapPdata::new_todo_context(OtlpProtoBytes::ExportMetricsRequest(bytes).into()) + OtapPdata::new_todo_context( + OtlpProtoBytes::ExportMetricsRequest(bytes.into()).into(), + ) } OtlpProtoMessage::Traces(trace_data) => { trace_data.encode(&mut bytes)?; - OtapPdata::new_todo_context(OtlpProtoBytes::ExportTracesRequest(bytes).into()) + OtapPdata::new_todo_context( + OtlpProtoBytes::ExportTracesRequest(bytes.into()).into(), + ) } }) } diff --git a/rust/otap-dataflow/crates/otap/src/fake_data_generator/fake_signal.rs b/rust/otap-dataflow/crates/otap/src/fake_data_generator/fake_signal.rs index 1af45bab45..921a73bf81 100644 --- a/rust/otap-dataflow/crates/otap/src/fake_data_generator/fake_signal.rs +++ b/rust/otap-dataflow/crates/otap/src/fake_data_generator/fake_signal.rs @@ -10,6 +10,7 @@ use crate::fake_data_generator::attributes::get_attribute_name_value; use crate::fake_data_generator::fake_data::{ current_time, delay, gen_span_id, gen_trace_id, get_scope_name, get_scope_version, }; +use otap_df_pdata::proto::opentelemetry::trace::v1::status::StatusCode; use otap_df_pdata::proto::opentelemetry::{ common::v1::{AnyValue, InstrumentationScope, KeyValue}, logs::v1::{LogRecord, LogsData, ResourceLogs, ScopeLogs, SeverityNumber}, @@ -136,6 +137,11 @@ fn spans(signal_count: usize, registry: &ResolvedRegistry) -> Vec { ) .kind(otel_span_kind(group.span_kind.as_ref())) .end_time_unix_nano(end_time) + // Note: without status set, the OTAP encoder fails at runtime + .status(otap_df_pdata::proto::opentelemetry::trace::v1::Status::new( + StatusCode::Ok, + "ok", + )) .finish(), ); } diff --git a/rust/otap-dataflow/crates/otap/src/filter_processor.rs b/rust/otap-dataflow/crates/otap/src/filter_processor.rs index 0ea34d9852..7ef48f00dd 100644 --- a/rust/otap-dataflow/crates/otap/src/filter_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/filter_processor.rs @@ -398,7 +398,7 @@ mod tests { .encode(&mut bytes) .expect("failed to encode log data into bytes"); let otlp_logs_bytes = - OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes.into()).into()); ctx.process(Message::PData(otlp_logs_bytes)) .await .expect("failed to process"); @@ -410,7 +410,7 @@ mod tests { .try_into() .expect("failed to convert to OtlpProtoBytes"); let received_logs_data = match otlp_bytes { - OtlpProtoBytes::ExportLogsRequest(bytes) => LogsData::decode(bytes.as_slice()) + OtlpProtoBytes::ExportLogsRequest(bytes) => LogsData::decode(bytes.as_ref()) .expect("failed to decode logs into logsdata"), _ => panic!("expected logs type"), }; @@ -567,7 +567,7 @@ mod tests { .encode(&mut bytes) .expect("failed to encode log data into bytes"); let otlp_logs_bytes = - OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes.into()).into()); ctx.process(Message::PData(otlp_logs_bytes)) .await .expect("failed to process"); @@ -579,7 +579,7 @@ mod tests { .try_into() .expect("failed to convert to OtlpProtoBytes"); let received_logs_data = match otlp_bytes { - OtlpProtoBytes::ExportLogsRequest(bytes) => LogsData::decode(bytes.as_slice()) + OtlpProtoBytes::ExportLogsRequest(bytes) => LogsData::decode(bytes.as_ref()) .expect("failed to decode logs into logsdata"), _ => panic!("expected logs type"), }; @@ -723,7 +723,7 @@ mod tests { .encode(&mut bytes) .expect("failed to encode log data into bytes"); let otlp_logs_bytes = - OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes.into()).into()); ctx.process(Message::PData(otlp_logs_bytes)) .await .expect("failed to process"); @@ -735,7 +735,7 @@ mod tests { .try_into() .expect("failed to convert to OtlpProtoBytes"); let received_logs_data = match otlp_bytes { - OtlpProtoBytes::ExportLogsRequest(bytes) => LogsData::decode(bytes.as_slice()) + OtlpProtoBytes::ExportLogsRequest(bytes) => LogsData::decode(bytes.as_ref()) .expect("failed to decode logs into logsdata"), _ => panic!("expected logs type"), }; @@ -883,7 +883,7 @@ mod tests { .encode(&mut bytes) .expect("failed to encode log data into bytes"); let otlp_logs_bytes = - OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes.into()).into()); ctx.process(Message::PData(otlp_logs_bytes)) .await .expect("failed to process"); @@ -895,7 +895,7 @@ mod tests { .try_into() .expect("failed to convert to OtlpProtoBytes"); let received_logs_data = match otlp_bytes { - OtlpProtoBytes::ExportLogsRequest(bytes) => LogsData::decode(bytes.as_slice()) + OtlpProtoBytes::ExportLogsRequest(bytes) => LogsData::decode(bytes.as_ref()) .expect("failed to decode logs into logsdata"), _ => panic!("expected logs type"), }; diff --git a/rust/otap-dataflow/crates/otap/src/fixtures.rs b/rust/otap-dataflow/crates/otap/src/fixtures.rs index 74176b2832..a21ea14204 100644 --- a/rust/otap-dataflow/crates/otap/src/fixtures.rs +++ b/rust/otap-dataflow/crates/otap/src/fixtures.rs @@ -126,7 +126,7 @@ pub fn create_single_logs_pdata_with_attrs(attributes: Vec) -> OtapPda }; let mut bytes = vec![]; log_req_1.encode(&mut bytes).unwrap(); - OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()) + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes.into()).into()) } pub fn create_simple_logs_arrow_record_batches(options: SimpleDataGenOptions) -> BatchArrowRecords { diff --git a/rust/otap-dataflow/crates/otap/src/lib.rs b/rust/otap-dataflow/crates/otap/src/lib.rs index 7645f66c46..43464c7da3 100644 --- a/rust/otap-dataflow/crates/otap/src/lib.rs +++ b/rust/otap-dataflow/crates/otap/src/lib.rs @@ -13,6 +13,8 @@ pub mod otap_exporter; pub mod otap_grpc; /// Implementation of OTAP Receiver that implements the receiver trait pub mod otap_receiver; +/// Implementation of OTEL Receiver (OTLP+OTAP gRPC receiver) that implements the receiver trait +pub mod otel_receiver; /// This receiver receives OTLP bytes from the grpc service request and /// produce for the pipeline OTAP PData diff --git a/rust/otap-dataflow/crates/otap/src/metrics.rs b/rust/otap-dataflow/crates/otap/src/metrics.rs index 08a1603d71..6459757fd9 100644 --- a/rust/otap-dataflow/crates/otap/src/metrics.rs +++ b/rust/otap-dataflow/crates/otap/src/metrics.rs @@ -55,26 +55,47 @@ pub struct ExporterPDataMetrics { impl ExporterPDataMetrics { pub fn inc_consumed(&mut self, st: SignalType) { + self.add_consumed(st, 1); + } + + pub fn inc_exported(&mut self, st: SignalType) { + self.add_exported(st, 1); + } + + pub fn inc_failed(&mut self, st: SignalType) { + self.add_failed(st, 1); + } + + pub fn add_consumed(&mut self, st: SignalType, count: u64) { + if count == 0 { + return; + } match st { - SignalType::Metrics => self.metrics_consumed.inc(), - SignalType::Logs => self.logs_consumed.inc(), - SignalType::Traces => self.traces_consumed.inc(), + SignalType::Metrics => self.metrics_consumed.add(count), + SignalType::Logs => self.logs_consumed.add(count), + SignalType::Traces => self.traces_consumed.add(count), } } - pub fn inc_exported(&mut self, st: SignalType) { + pub fn add_exported(&mut self, st: SignalType, count: u64) { + if count == 0 { + return; + } match st { - SignalType::Metrics => self.metrics_exported.inc(), - SignalType::Logs => self.logs_exported.inc(), - SignalType::Traces => self.traces_exported.inc(), + SignalType::Metrics => self.metrics_exported.add(count), + SignalType::Logs => self.logs_exported.add(count), + SignalType::Traces => self.traces_exported.add(count), } } - pub fn inc_failed(&mut self, st: SignalType) { + pub fn add_failed(&mut self, st: SignalType, count: u64) { + if count == 0 { + return; + } match st { - SignalType::Metrics => self.metrics_failed.inc(), - SignalType::Logs => self.logs_failed.inc(), - SignalType::Traces => self.traces_failed.inc(), + SignalType::Metrics => self.metrics_failed.add(count), + SignalType::Logs => self.logs_failed.add(count), + SignalType::Traces => self.traces_failed.add(count), } } } diff --git a/rust/otap-dataflow/crates/otap/src/otap_exporter.rs b/rust/otap-dataflow/crates/otap/src/otap_exporter.rs index 9e16093b1c..21c5fc3bb3 100644 --- a/rust/otap-dataflow/crates/otap/src/otap_exporter.rs +++ b/rust/otap-dataflow/crates/otap/src/otap_exporter.rs @@ -110,7 +110,7 @@ impl local::Exporter for OTAPExporter { ) -> Result { effect_handler .info(&format!( - "Exporting OTLP traffic to endpoint: {}", + "Exporting OTAP traffic to endpoint: {}", self.config.grpc_endpoint )) .await; @@ -204,7 +204,7 @@ impl local::Exporter for OTAPExporter { Message::Control(NodeControlMsg::TimerTick { .. }) | Message::Control(NodeControlMsg::Config { .. }) => {} Message::Control(NodeControlMsg::CollectTelemetry { - mut metrics_reporter, + metrics_reporter, }) => { _ = metrics_reporter.report(&mut self.pdata_metrics); } @@ -357,10 +357,13 @@ async fn stream_arrow_batches( shutdown_rx.clone() ).await; } - Err(_e) => { + Err(err) => { // there was an error initiating the streaming request _ = pdata_metrics_tx.send(PDataMetricsUpdate::IncFailed(signal_type)).await; - log::error!("failed request, waiting {failed_request_backoff:?}"); + log::error!( + "failed request to {:?}: {err}; waiting {failed_request_backoff:?}", + signal_type + ); tokio::time::sleep(failed_request_backoff).await; failed_request_backoff = std::cmp::min(failed_request_backoff * BACKOFF_MULTIPLIER, MAX_BACKOFF); } diff --git a/rust/otap-dataflow/crates/otap/src/otap_grpc.rs b/rust/otap-dataflow/crates/otap/src/otap_grpc.rs index a81d207a36..b64a98368f 100644 --- a/rust/otap-dataflow/crates/otap/src/otap_grpc.rs +++ b/rust/otap-dataflow/crates/otap/src/otap_grpc.rs @@ -1,17 +1,23 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 +//! Provides a set of structs and enums that interact with the gRPC Server with BiDirectional +//! streaming. //! -//! Provides a set of structs and enums that interact with the gRPC Server with BiDirectional streaming -//! -//! Implements the necessary service traits for OTLP data +//! Implements the necessary service traits for OTLP data. //! //! ToDo: Modify OTAPData -> Optimize message transport //! ToDo: Handle Ack and Nack, return proper batch status //! ToDo: Change how channel sizes are handled? Currently defined when creating otap_receiver -> passing channel size to the ServiceImpl -//! -use otap_df_engine::{Interests, ProducerEffectHandlerExtension, shared::receiver as shared}; +use async_trait::async_trait; +use futures::{ + StreamExt, + stream::{self, FuturesUnordered}, +}; +use otap_df_engine::{ + Interests, ProducerEffectHandlerExtension, control::NackMsg, shared::receiver as shared, +}; use otap_df_pdata::{ Consumer, otap::{Logs, Metrics, OtapArrowRecords, OtapBatchStore, Traces, from_record_messages}, @@ -21,36 +27,64 @@ use otap_df_pdata::{ arrow_traces_service_server::ArrowTracesService, }, }; +use std::future::Future; +use std::marker::PhantomData; use std::pin::Pin; +use std::task::{Context as TaskContext, Poll}; use tokio::sync::oneshot; use tokio_stream::Stream; -use tokio_stream::wrappers::ReceiverStream; use tonic::{Request, Response, Status}; use crate::{ - otap_grpc::otlp::server::{SharedState, SlotGuard}, + otap_grpc::otlp::server::{AckSubscriptionState, SlotGuard}, pdata::{Context, OtapPdata}, }; +pub mod client_settings; +pub mod common; pub mod middleware; pub mod otlp; +pub mod server_settings; + +pub use client_settings::GrpcClientSettings; +pub use server_settings::GrpcServerSettings; /// Common settings for OTLP receivers. #[derive(Clone, Debug)] pub struct Settings { - /// Size of the channel used to buffer outgoing responses to the client. - pub response_stream_channel_size: usize, /// Maximum concurrent requests per receiver instance (per core). pub max_concurrent_requests: usize, /// Whether the receiver should wait. pub wait_for_result: bool, } +/// Abstraction over inbound OTAP Arrow request streams. +#[async_trait] +pub trait ArrowRequestStream: Send + 'static { + /// Returns the next OTAP Arrow batch in the stream. + async fn next_message(&mut self) -> Result, Status>; +} + +#[async_trait] +impl ArrowRequestStream for tonic::Streaming { + async fn next_message(&mut self) -> Result, Status> { + self.message().await + } +} + +pub(crate) fn per_connection_limit(settings: &Settings) -> usize { + if settings.wait_for_result { + settings.max_concurrent_requests.max(1) + } else { + 1 + } +} + /// struct that implements the ArrowLogsService trait pub struct ArrowLogsServiceImpl { effect_handler: shared::EffectHandler, - state: Option, - settings: Settings, + state: Option, + max_in_flight_per_connection: usize, } impl ArrowLogsServiceImpl { @@ -61,22 +95,22 @@ impl ArrowLogsServiceImpl { effect_handler, state: settings .wait_for_result - .then(|| SharedState::new(settings.max_concurrent_requests)), - settings: settings.clone(), + .then(|| AckSubscriptionState::new(settings.max_concurrent_requests)), + max_in_flight_per_connection: per_connection_limit(settings), } } /// Get this server's shared state for Ack/Nack routing #[must_use] - pub fn state(&self) -> Option { + pub fn state(&self) -> Option { self.state.clone() } } /// struct that implements the ArrowMetricsService trait pub struct ArrowMetricsServiceImpl { effect_handler: shared::EffectHandler, - state: Option, - settings: Settings, + state: Option, + max_in_flight_per_connection: usize, } impl ArrowMetricsServiceImpl { @@ -87,14 +121,14 @@ impl ArrowMetricsServiceImpl { effect_handler, state: settings .wait_for_result - .then(|| SharedState::new(settings.max_concurrent_requests)), - settings: settings.clone(), + .then(|| AckSubscriptionState::new(settings.max_concurrent_requests)), + max_in_flight_per_connection: per_connection_limit(settings), } } /// Get this server's shared state for Ack/Nack routing #[must_use] - pub fn state(&self) -> Option { + pub fn state(&self) -> Option { self.state.clone() } } @@ -102,8 +136,8 @@ impl ArrowMetricsServiceImpl { /// struct that implements the ArrowTracesService trait pub struct ArrowTracesServiceImpl { effect_handler: shared::EffectHandler, - state: Option, - settings: Settings, + state: Option, + max_in_flight_per_connection: usize, } impl ArrowTracesServiceImpl { @@ -114,14 +148,14 @@ impl ArrowTracesServiceImpl { effect_handler, state: settings .wait_for_result - .then(|| SharedState::new(settings.max_concurrent_requests)), - settings: settings.clone(), + .then(|| AckSubscriptionState::new(settings.max_concurrent_requests)), + max_in_flight_per_connection: per_connection_limit(settings), } } /// Get this server's shared state for Ack/Nack routing #[must_use] - pub fn state(&self) -> Option { + pub fn state(&self) -> Option { self.state.clone() } } @@ -134,41 +168,15 @@ impl ArrowLogsService for ArrowLogsServiceImpl { &self, request: Request>, ) -> Result, Status> { - let mut input_stream = request.into_inner(); - // ToDo [LQ] How can we abstract this to avoid any dependency on Tokio inside receiver implementations. - let (tx, rx) = tokio::sync::mpsc::channel(self.settings.response_stream_channel_size); - let effect_handler_clone = self.effect_handler.clone(); - let state_clone = self.state.clone(); - - // Provide client a stream to listen to - let output = ReceiverStream::new(rx); - - // write to the channel - // ToDo [LQ] How can we abstract this to avoid any dependency on Tokio inside receiver implementations. - _ = tokio::spawn(async move { - let mut consumer = Consumer::default(); - - // Process messages until stream ends or error occurs - while let Ok(Some(batch)) = input_stream.message().await { - // accept the batch data and handle output response - if accept_data::( - OtapArrowRecords::Logs, - &mut consumer, - batch, - &effect_handler_clone, - state_clone.clone(), - &tx, - ) - .await - .is_err() - { - // end loop if error occurs - break; - } - } - }); - - Ok(Response::new(Box::pin(output) as Self::ArrowLogsStream)) + let input_stream = request.into_inner(); + let output = stream_arrow_batch_statuses::<_, Logs, _>( + input_stream, + self.effect_handler.clone(), + self.state.clone(), + OtapArrowRecords::Logs, + self.max_in_flight_per_connection, + ); + Ok(Response::new(output)) } } @@ -180,39 +188,15 @@ impl ArrowMetricsService for ArrowMetricsServiceImpl { &self, request: Request>, ) -> Result, Status> { - let mut input_stream = request.into_inner(); - let (tx, rx) = tokio::sync::mpsc::channel(self.settings.response_stream_channel_size); - let effect_handler_clone = self.effect_handler.clone(); - let state_clone = self.state.clone(); - - // Provide client a stream to listen to - let output = ReceiverStream::new(rx); - - // write to the channel - _ = tokio::spawn(async move { - let mut consumer = Consumer::default(); - - // Process messages until stream ends or error occurs - while let Ok(Some(batch)) = input_stream.message().await { - // accept the batch data and handle output response - if accept_data::( - OtapArrowRecords::Metrics, - &mut consumer, - batch, - &effect_handler_clone, - state_clone.clone(), - &tx, - ) - .await - .is_err() - { - // end loop if error occurs - break; - } - } - }); - - Ok(Response::new(Box::pin(output) as Self::ArrowMetricsStream)) + let input_stream = request.into_inner(); + let output = stream_arrow_batch_statuses::<_, Metrics, _>( + input_stream, + self.effect_handler.clone(), + self.state.clone(), + OtapArrowRecords::Metrics, + self.max_in_flight_per_connection, + ); + Ok(Response::new(output)) } } @@ -224,162 +208,282 @@ impl ArrowTracesService for ArrowTracesServiceImpl { &self, request: Request>, ) -> Result, Status> { - let mut input_stream = request.into_inner(); - let (tx, rx) = tokio::sync::mpsc::channel(self.settings.response_stream_channel_size); - let effect_handler_clone = self.effect_handler.clone(); - let state_clone = self.state.clone(); - - // create a stream to output result to - let output = ReceiverStream::new(rx); - - // write to the channel - _ = tokio::spawn(async move { - let mut consumer = Consumer::default(); - - // Process messages until stream ends or error occurs - while let Ok(Some(batch)) = input_stream.message().await { - // accept the batch data and handle output response - if accept_data::( - OtapArrowRecords::Traces, - &mut consumer, - batch, - &effect_handler_clone, - state_clone.clone(), - &tx, - ) - .await - .is_err() - { - // end loop if error occurs - break; - } - } - }); - - Ok(Response::new(Box::pin(output) as Self::ArrowTracesStream)) + let input_stream = request.into_inner(); + let output = stream_arrow_batch_statuses::<_, Traces, _>( + input_stream, + self.effect_handler.clone(), + self.state.clone(), + OtapArrowRecords::Traces, + self.max_in_flight_per_connection, + ); + Ok(Response::new(output)) } } -/// handles sending the data down the pipeline via effect_handler and generating the appropriate response -async fn accept_data( +/// Streams `BatchStatus` updates for the Arrow gRPC services. +/// +/// `ArrowLogsServiceImpl::arrow_logs`, `ArrowMetricsServiceImpl::arrow_metrics`, and +/// `ArrowTracesServiceImpl::arrow_traces` all delegate to this helper. Each service passes its +/// inbound `Streaming` plus a converter that turns a decoded batch into the +/// signal-specific variant of `OtapArrowRecords`. The returned stream forwards every received Arrow +/// batch to the pipeline and yields the corresponding `BatchStatus` updates the OTLP Arrow clients +/// expect to read. +/// +/// Internally an `ArrowBatchStreamState` pulls the next `BatchArrowRecords` from the tonic stream, +/// decodes it into `OtapPdata`, and optionally registers an `AckSubscriptionState` slot when +/// `wait_for_result` is enabled. Once the pipeline acknowledges (or rejects) the batch, the stream +/// emits a success or error status before continuing with the next request. To avoid per-connection +/// serialization, the state now keeps up to `max_in_flight_per_connection` batches in flight: it +/// eagerly reads, decodes, and dispatches new Arrow batches while prior ones wait for ACK/NACK +/// responses, only falling back to serialized processing once the limit is reached. +/// +/// This design replaces the previous channel-plus-background-task approach. Expressing the control +/// flow as a single `Stream` keeps backpressure aligned with gRPC demand, removes the bookkeeping +/// around extra channels/tasks, and makes it easier to follow how every request progresses through +/// decoding, dispatch, acknowledgement, and now limited parallelism. +pub(crate) fn stream_arrow_batch_statuses( + input_stream: S, + effect_handler: shared::EffectHandler, + state: Option, + otap_batch: F, + max_in_flight_per_connection: usize, +) -> Pin> + Send + 'static>> +where + S: ArrowRequestStream + Send, + T: OtapBatchStore + 'static, + F: Fn(T) -> OtapArrowRecords + Send + Copy + 'static, +{ + let state = ArrowBatchStreamState::new( + input_stream, + effect_handler, + state, + otap_batch, + max_in_flight_per_connection, + ); + Box::pin(build_status_stream(state).boxed()) +} + +fn build_status_stream( + state: ArrowBatchStreamState, +) -> impl Stream> +where + S: ArrowRequestStream + Send, + T: OtapBatchStore + 'static, + F: Fn(T) -> OtapArrowRecords + Send + Copy + 'static, +{ + stream::unfold(state, |mut state| async move { + match state.next_item().await { + StreamStep::Yield(item) => Some((item, state)), + StreamStep::Done => None, + } + }) +} + +pub(crate) struct ArrowBatchStreamState +where + S: ArrowRequestStream + Send, + T: OtapBatchStore + 'static, + F: Fn(T) -> OtapArrowRecords + Send + Copy + 'static, +{ + input_stream: S, + consumer: Consumer, + effect_handler: shared::EffectHandler, + state: Option, otap_batch: F, - consumer: &mut Consumer, - mut batch: BatchArrowRecords, - effect_handler: &shared::EffectHandler, - state: Option, - tx: &tokio::sync::mpsc::Sender>, -) -> Result<(), ()> + in_flight: FuturesUnordered, + max_in_flight: usize, + finished: bool, + _marker: PhantomData T>, +} + +enum StreamStep { + Yield(Result), + Done, +} + +enum PreparedBatch { + Enqueued, + Immediate(StreamStep), +} + +impl ArrowBatchStreamState where - F: Fn(T) -> OtapArrowRecords, + S: ArrowRequestStream + Send, + T: OtapBatchStore + 'static, + F: Fn(T) -> OtapArrowRecords + Send + Copy + 'static, { - let batch_id = batch.batch_id; - let batch = consumer.consume_bar(&mut batch).map_err(|e| { - log::error!("Error decoding OTAP Batch: {e:?}. Closing stream"); - })?; - - let batch = from_record_messages::(batch); - let otap_batch_as_otap_arrow_records = otap_batch(batch); - let mut otap_pdata = - OtapPdata::new(Context::default(), otap_batch_as_otap_arrow_records.into()); - - let cancel_rx = if let Some(state) = state { - // Try to allocate a slot (under the mutex) for calldata. - let allocation_result = { - let guard_result = state.0.lock(); - match guard_result { - Ok(mut guard) => guard.allocate(|| oneshot::channel()), - Err(_) => { - log::error!("Mutex poisoned"); - return Err(()); + fn new( + input_stream: S, + effect_handler: shared::EffectHandler, + state: Option, + otap_batch: F, + max_in_flight_per_connection: usize, + ) -> Self { + Self { + input_stream, + consumer: Consumer::default(), + effect_handler, + state, + otap_batch, + in_flight: FuturesUnordered::new(), + max_in_flight: max_in_flight_per_connection.max(1), + finished: false, + _marker: PhantomData, + } + } + + async fn next_item(&mut self) -> StreamStep { + if let Some(step) = self.fill_inflight().await { + return step; + } + + match self.in_flight.next().await { + Some(step) => { + if matches!(step, StreamStep::Done) { + self.finished = true; } + step } - }; // MutexGuard is dropped here - - let (key, rx) = match allocation_result { - None => { - log::error!("Too many concurrent requests"); - - // Send backpressure response - tx.send(Ok(BatchStatus { - batch_id, - status_code: StatusCode::Unavailable as i32, - status_message: format!( - "Pipeline processing failed: {}", - "Too many concurrent requests" - ), - })) - .await - .map_err(|e| { - log::error!("Error sending BatchStatus response: {e:?}"); - })?; - - return Ok(()); + None => StreamStep::Done, + } + } + + async fn fill_inflight(&mut self) -> Option { + while !self.finished && self.in_flight.len() < self.max_in_flight { + match self.input_stream.next_message().await { + Ok(Some(batch)) => match self.enqueue_batch(batch).await { + PreparedBatch::Enqueued => continue, + PreparedBatch::Immediate(step) => return Some(step), + }, + Ok(None) => { + self.finished = true; + break; + } + Err(status) => { + self.finished = true; + return Some(StreamStep::Yield(Err(status))); + } + } + } + None + } + + async fn enqueue_batch(&mut self, mut batch: BatchArrowRecords) -> PreparedBatch { + let batch_id = batch.batch_id; + + let batch = match self.consumer.consume_bar(&mut batch) { + Ok(batch) => batch, + Err(e) => { + log::error!("Error decoding OTAP Batch: {e:?}. Closing stream"); + self.finished = true; + return PreparedBatch::Immediate(StreamStep::Done); } - Some(pair) => pair, }; - // Enter the subscription. Slot key becomes calldata. - effect_handler.subscribe_to( - Interests::ACKS | Interests::NACKS, - key.into(), - &mut otap_pdata, - ); - Some((SlotGuard { key, state }, rx)) - } else { - None - }; + let batch = from_record_messages::(batch); + let otap_batch_as_otap_arrow_records = (self.otap_batch)(batch); + let mut otap_pdata = + OtapPdata::new(Context::default(), otap_batch_as_otap_arrow_records.into()); + + let cancel_rx = if let Some(state) = self.state.clone() { + let allocation_result = state.0.lock().allocate(|| oneshot::channel()); + let (key, rx) = match allocation_result { + None => { + log::error!("Too many concurrent requests"); + return PreparedBatch::Immediate(StreamStep::Yield(Ok(BatchStatus { + batch_id, + status_code: StatusCode::Unavailable as i32, + status_message: "Pipeline processing failed: Too many concurrent requests" + .to_string(), + }))); + } + Some(pair) => pair, + }; + + self.effect_handler.subscribe_to( + Interests::ACKS | Interests::NACKS, + key.into(), + &mut otap_pdata, + ); + Some((SlotGuard { key, state }, rx)) + } else { + None + }; - // Send and wait for Ack/Nack - match effect_handler.send_message(otap_pdata).await { - Ok(_) => {} - Err(e) => { + if let Err(e) = self.effect_handler.send_message(otap_pdata).await { log::error!("Failed to send to pipeline: {e}"); - return Err(()); + self.finished = true; + return PreparedBatch::Immediate(StreamStep::Done); + }; + + if let Some((cancel_guard, rx)) = cancel_rx { + self.in_flight + .push(AckWaitFuture::new(batch_id, cancel_guard, rx)); + PreparedBatch::Enqueued + } else { + PreparedBatch::Immediate(StreamStep::Yield(Ok(success_status(batch_id)))) } - }; - - // If backpressure, await a response. The guard will cancel and return the - // slot if Tonic times-out this task. - if let Some((_cancel_guard, rx)) = cancel_rx { - match rx.await { - Ok(Ok(())) => { - // Received Ack - // Behavior is similar to `wait_for_result` set to `false` case - // No need to send a response here since success response is sent - // before returning from the function anyway - } - Ok(Err(nack)) => { - // Received Nack - // TODO: Use more specific status codes based on nack reason/type - // when more detailed error information is available from the pipeline - tx.send(Ok(BatchStatus { - batch_id, - status_code: StatusCode::Unavailable as i32, - status_message: format!("Pipeline processing failed: {}", nack.reason), - })) - .await - .map_err(|e| { - log::error!("Error sending BatchStatus response: {e:?}"); - })?; - - return Ok(()); - } - Err(_) => { - log::error!("Response channel closed unexpectedly"); - return Err(()); + } +} + +struct AckWaitFuture { + batch_id: i64, + cancel_guard: Option, + rx: oneshot::Receiver>>, +} + +impl AckWaitFuture { + fn new( + batch_id: i64, + cancel_guard: SlotGuard, + rx: oneshot::Receiver>>, + ) -> Self { + Self { + batch_id, + cancel_guard: Some(cancel_guard), + rx, + } + } +} + +impl Future for AckWaitFuture { + type Output = StreamStep; + + fn poll(self: Pin<&mut Self>, cx: &mut TaskContext<'_>) -> Poll { + let this = self.get_mut(); + let rx = Pin::new(&mut this.rx); + match rx.poll(cx) { + Poll::Pending => Poll::Pending, + Poll::Ready(result) => { + let _ = this.cancel_guard.take(); + let step = match result { + Ok(Ok(())) => StreamStep::Yield(Ok(success_status(this.batch_id))), + Ok(Err(nack)) => StreamStep::Yield(Ok(nack_status(this.batch_id, nack.reason))), + Err(_) => { + log::error!("Response channel closed unexpectedly"); + StreamStep::Done + } + }; + Poll::Ready(step) } } } +} - tx.send(Ok(BatchStatus { +fn success_status(batch_id: i64) -> BatchStatus { + BatchStatus { batch_id, status_code: StatusCode::Ok as i32, status_message: "Successfully received".to_string(), - })) - .await - .map_err(|e| { - log::error!("Error sending BatchStatus response: {e:?}"); - }) + } +} + +fn nack_status(batch_id: i64, reason: String) -> BatchStatus { + BatchStatus { + batch_id, + status_code: StatusCode::Unavailable as i32, + status_message: format!("Pipeline processing failed: {reason}"), + } } /// Enum to describe the Arrow data. diff --git a/rust/otap-dataflow/crates/otap/src/otap_grpc/client_settings.rs b/rust/otap-dataflow/crates/otap/src/otap_grpc/client_settings.rs new file mode 100644 index 0000000000..636503ca15 --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otap_grpc/client_settings.rs @@ -0,0 +1,246 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Shared configuration for gRPC-based clients. + +use crate::compression::CompressionMethod; +use otap_df_config::byte_units; +use serde::Deserialize; +use std::time::Duration; +use tonic::codec::CompressionEncoding; +use tonic::transport::Endpoint; + +/// Common configuration shared across gRPC clients. +#[derive(Debug, Deserialize, Clone)] +#[serde(deny_unknown_fields)] +pub struct GrpcClientSettings { + /// The gRPC endpoint to connect to. + pub grpc_endpoint: String, + + /// Compression method to use for outbound requests. Defaults to no compression. + #[serde(default)] + pub compression: Option, + + /// Maximum number of concurrent in-flight requests allowed by the transport stack. + #[serde(default = "default_concurrency_limit")] + pub concurrency_limit: usize, + + /// Timeout for establishing TCP connections. + #[serde(default = "default_connect_timeout", with = "humantime_serde")] + pub connect_timeout: Duration, + + /// Whether to enable `TCP_NODELAY`. + #[serde(default = "default_tcp_nodelay")] + pub tcp_nodelay: bool, + + /// TCP keepalive timeout for outbound connections. + #[serde(default = "default_tcp_keepalive", with = "humantime_serde")] + pub tcp_keepalive: Option, + + /// Interval between TCP keepalive probes once keepalive is active. + #[serde(default, with = "humantime_serde")] + pub tcp_keepalive_interval: Option, + + /// Number of TCP keepalive probes sent before a connection is declared dead. + #[serde(default)] + pub tcp_keepalive_retries: Option, + + /// Initial HTTP/2 stream window size, in bytes. + #[serde( + default = "default_initial_stream_window_size", + deserialize_with = "byte_units::deserialize" + )] + pub initial_stream_window_size: Option, + + /// Initial HTTP/2 connection window size, in bytes. + #[serde( + default = "default_initial_connection_window_size", + deserialize_with = "byte_units::deserialize" + )] + pub initial_connection_window_size: Option, + + /// Whether to rely on HTTP/2 adaptive window sizing instead of the manual values above. + #[serde(default = "default_http2_adaptive_window")] + pub http2_adaptive_window: bool, + + /// Interval between HTTP/2 keepalive pings. + #[serde(default = "default_http2_keepalive_interval", with = "humantime_serde")] + pub http2_keepalive_interval: Option, + + /// Timeout waiting for an HTTP/2 keepalive acknowledgement. + #[serde(default = "default_http2_keepalive_timeout", with = "humantime_serde")] + pub http2_keepalive_timeout: Option, + + /// Whether to send HTTP/2 keepalives while idle. + #[serde(default = "default_keep_alive_while_idle")] + pub keep_alive_while_idle: bool, + + /// Timeout for RPC requests. If not specified, no timeout is applied. + #[serde(default, with = "humantime_serde")] + pub timeout: Option, + + /// Internal Tower buffer size for the gRPC client. + #[serde(default)] + pub buffer_size: Option, +} + +impl GrpcClientSettings { + /// Returns the compression encoding to apply to requests, if any. + #[must_use] + pub fn compression_encoding(&self) -> Option { + self.compression + .map(|method| method.map_to_compression_encoding()) + } + + /// Returns a non-zero concurrency limit. + #[must_use] + pub fn effective_concurrency_limit(&self) -> usize { + self.concurrency_limit.max(1) + } + + /// Builds the configured [`Endpoint`]. + pub fn build_endpoint(&self) -> Result { + let mut endpoint = Endpoint::from_shared(self.grpc_endpoint.clone())? + .concurrency_limit(self.effective_concurrency_limit()) + .connect_timeout(self.connect_timeout) + .tcp_nodelay(self.tcp_nodelay) + .tcp_keepalive(self.tcp_keepalive) + .initial_stream_window_size(self.initial_stream_window_size) + .initial_connection_window_size(self.initial_connection_window_size) + .keep_alive_while_idle(self.keep_alive_while_idle); + + if let Some(interval) = self.http2_keepalive_interval { + endpoint = endpoint.http2_keep_alive_interval(interval); + } + if let Some(timeout) = self.http2_keepalive_timeout { + endpoint = endpoint.keep_alive_timeout(timeout); + } + if let Some(interval) = self.tcp_keepalive_interval { + endpoint = endpoint.tcp_keepalive_interval(Some(interval)); + } + if let Some(retries) = self.tcp_keepalive_retries { + endpoint = endpoint.tcp_keepalive_retries(Some(retries)); + } + if self.http2_adaptive_window { + endpoint = endpoint.http2_adaptive_window(true); + } + if let Some(buffer_size) = self.buffer_size { + endpoint = endpoint.buffer_size(buffer_size); + } + if let Some(timeout) = self.timeout { + endpoint = endpoint.timeout(timeout); + } + + Ok(endpoint) + } +} + +impl Default for GrpcClientSettings { + fn default() -> Self { + Self { + grpc_endpoint: String::new(), + compression: None, + concurrency_limit: default_concurrency_limit(), + connect_timeout: default_connect_timeout(), + tcp_nodelay: default_tcp_nodelay(), + tcp_keepalive: default_tcp_keepalive(), + tcp_keepalive_interval: None, + tcp_keepalive_retries: None, + initial_stream_window_size: default_initial_stream_window_size(), + initial_connection_window_size: default_initial_connection_window_size(), + http2_adaptive_window: default_http2_adaptive_window(), + http2_keepalive_interval: default_http2_keepalive_interval(), + http2_keepalive_timeout: default_http2_keepalive_timeout(), + keep_alive_while_idle: default_keep_alive_while_idle(), + timeout: None, + buffer_size: None, + } + } +} + +const fn default_concurrency_limit() -> usize { + 256 +} + +const fn default_connect_timeout() -> Duration { + Duration::from_secs(3) +} + +const fn default_tcp_nodelay() -> bool { + true +} + +const fn default_tcp_keepalive() -> Option { + Some(Duration::from_secs(45)) +} + +const fn default_initial_stream_window_size() -> Option { + Some(8 * 1024 * 1024) +} + +const fn default_initial_connection_window_size() -> Option { + Some(32 * 1024 * 1024) +} + +const fn default_http2_adaptive_window() -> bool { + false +} + +const fn default_http2_keepalive_interval() -> Option { + Some(Duration::from_secs(30)) +} + +const fn default_http2_keepalive_timeout() -> Option { + Some(Duration::from_secs(10)) +} + +const fn default_keep_alive_while_idle() -> bool { + true +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn defaults_match_previous_client_tuning() { + let settings: GrpcClientSettings = + serde_json::from_str(r#"{ "grpc_endpoint": "http://localhost:4317" }"#).unwrap(); + + assert_eq!(settings.concurrency_limit, 256); + assert_eq!(settings.connect_timeout, Duration::from_secs(3)); + assert_eq!(settings.tcp_keepalive, Some(Duration::from_secs(45))); + assert_eq!( + settings.http2_keepalive_interval, + Some(Duration::from_secs(30)) + ); + assert_eq!( + settings.http2_keepalive_timeout, + Some(Duration::from_secs(10)) + ); + assert!(settings.keep_alive_while_idle); + } + + #[test] + fn compression_encoding_is_reported() { + let settings: GrpcClientSettings = serde_json::from_str( + r#"{ "grpc_endpoint": "http://localhost:4317", "compression_method": "gzip" }"#, + ) + .unwrap(); + + assert_eq!( + settings.compression_encoding(), + Some(CompressionEncoding::Gzip) + ); + } + + #[test] + fn effective_concurrency_limit_clamps_to_one() { + let settings: GrpcClientSettings = serde_json::from_str( + r#"{ "grpc_endpoint": "http://localhost:4317", "concurrency_limit": 0 }"#, + ) + .unwrap(); + + assert_eq!(settings.effective_concurrency_limit(), 1); + } +} diff --git a/rust/otap-dataflow/crates/otap/src/otap_grpc/common.rs b/rust/otap-dataflow/crates/otap/src/otap_grpc/common.rs new file mode 100644 index 0000000000..b8f5acfe69 --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otap_grpc/common.rs @@ -0,0 +1,138 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Shared helpers for gRPC receivers. + +use crate::otap_grpc::GrpcServerSettings; +use crate::otap_grpc::otlp::server::{AckSubscriptionState, RouteResponse}; +use crate::pdata::OtapPdata; +use otap_df_config::SignalType; +use otap_df_engine::control::{AckMsg, NackMsg}; +use tonic::transport::Server; + +/// Aggregates the per-signal ACK subscription maps that let us route responses back to callers. +#[derive(Clone, Default)] +pub struct SignalAckRoutingState { + /// Subscription map for log acknowledgements. + pub logs: Option, + /// Subscription map for metric acknowledgements. + pub metrics: Option, + /// Subscription map for trace acknowledgements. + pub traces: Option, +} + +impl SignalAckRoutingState { + /// Creates a new bundle of optional subscription maps. + #[must_use] + pub fn new( + logs: Option, + metrics: Option, + traces: Option, + ) -> Self { + Self { + logs, + metrics, + traces, + } + } +} + +/// Routes an Ack message to the appropriate signal's subscription map. +#[must_use] +pub fn route_ack_response(states: &SignalAckRoutingState, ack: AckMsg) -> RouteResponse { + let calldata = ack.calldata; + let resp = Ok(()); + let state = match ack.accepted.signal_type() { + SignalType::Logs => states.logs.as_ref(), + SignalType::Metrics => states.metrics.as_ref(), + SignalType::Traces => states.traces.as_ref(), + }; + + state + .map(|s| s.route_response(calldata, resp)) + .unwrap_or(RouteResponse::None) +} + +/// Routes a Nack message to the appropriate shared state. +#[must_use] +pub fn route_nack_response( + states: &SignalAckRoutingState, + mut nack: NackMsg, +) -> RouteResponse { + let calldata = std::mem::take(&mut nack.calldata); + let signal_type = nack.refused.signal_type(); + let resp = Err(nack); + let state = match signal_type { + SignalType::Logs => states.logs.as_ref(), + SignalType::Metrics => states.metrics.as_ref(), + SignalType::Traces => states.traces.as_ref(), + }; + + state + .map(|s| s.route_response(calldata, resp)) + .unwrap_or(RouteResponse::None) +} + +/// Handles the outcome from routing an Ack/Nack response. +pub fn handle_route_response( + resp: RouteResponse, + state: &mut T, + mut on_sent: F, + mut on_expired_or_invalid: G, +) where + F: FnMut(&mut T), + G: FnMut(&mut T), +{ + match resp { + RouteResponse::Sent => on_sent(state), + RouteResponse::Expired | RouteResponse::Invalid => on_expired_or_invalid(state), + RouteResponse::None => {} + } +} + +/// Tunes the maximum concurrent requests relative to the downstream capacity (channel connecting +/// the receiver to the rest of the pipeline). +pub fn tune_max_concurrent_requests(config: &mut GrpcServerSettings, downstream_capacity: usize) { + // Fall back to the downstream channel capacity when it is tighter than the user setting. + let safe_capacity = downstream_capacity.max(1); + if config.max_concurrent_requests == 0 || config.max_concurrent_requests > safe_capacity { + config.max_concurrent_requests = safe_capacity; + } +} + +/// Applies the shared server tuning options to a tonic server builder. +pub fn apply_server_tuning(builder: Server, config: &GrpcServerSettings) -> Server { + let transport_limit = config + .transport_concurrency_limit + .and_then(|limit| if limit == 0 { None } else { Some(limit) }) + .unwrap_or(config.max_concurrent_requests) + .max(1); + + let fallback_streams = config.max_concurrent_requests.min(u32::MAX as usize) as u32; + + let mut builder = builder + .concurrency_limit_per_connection(transport_limit) + .load_shed(config.load_shed) + .initial_stream_window_size(config.initial_stream_window_size) + .initial_connection_window_size(config.initial_connection_window_size) + .max_frame_size(config.max_frame_size) + .http2_adaptive_window(Some(config.http2_adaptive_window)) + .http2_keepalive_interval(config.http2_keepalive_interval) + .http2_keepalive_timeout(config.http2_keepalive_timeout); + + let mut max_concurrent_streams = config + .max_concurrent_streams + .map(|value| if value == 0 { fallback_streams } else { value }) + .unwrap_or(fallback_streams); + if max_concurrent_streams == 0 { + max_concurrent_streams = 1; + } + builder = builder.max_concurrent_streams(Some(max_concurrent_streams)); + + // Apply timeout if configured + if let Some(timeout) = config.timeout { + builder = builder.timeout(timeout); + } + + builder +} diff --git a/rust/otap-dataflow/crates/otap/src/otap_grpc/otlp/client.rs b/rust/otap-dataflow/crates/otap/src/otap_grpc/otlp/client.rs index d1742299c1..c2a95cc493 100644 --- a/rust/otap-dataflow/crates/otap/src/otap_grpc/otlp/client.rs +++ b/rust/otap-dataflow/crates/otap/src/otap_grpc/otlp/client.rs @@ -8,6 +8,7 @@ //! able to receive GRPC OTLP requests, and if there's no need to serialize them, we can keep //! the payload serialized as protobuf before then forwarding using these clients. +use bytes::Bytes; use http::uri::PathAndQuery; use otap_df_pdata::proto::opentelemetry::collector::logs::v1::ExportLogsServiceResponse; use otap_df_pdata::proto::opentelemetry::collector::metrics::v1::ExportMetricsServiceResponse; @@ -31,7 +32,7 @@ impl Codec for OtlpRequestCodec where T: Message + Default + Send + 'static, { - type Encode = Vec; + type Encode = Bytes; type Decode = T; type Encoder = OtlpRequestEncoder; @@ -56,7 +57,7 @@ struct OtlpRequestEncoder {} impl Encoder for OtlpRequestEncoder { type Error = Status; - type Item = Vec; + type Item = Bytes; fn encode(&mut self, item: Self::Item, dst: &mut EncodeBuf<'_>) -> Result<(), Self::Error> { dst.put(item.as_ref()); @@ -149,7 +150,7 @@ where /// Send the serialized grpc request pub async fn export( &mut self, - request: impl tonic::IntoRequest>, + request: impl tonic::IntoRequest, ) -> Result, Status> { self.inner .ready() diff --git a/rust/otap-dataflow/crates/otap/src/otap_grpc/otlp/server.rs b/rust/otap-dataflow/crates/otap/src/otap_grpc/otlp/server.rs index 0aead23102..a8b1d91398 100644 --- a/rust/otap-dataflow/crates/otap/src/otap_grpc/otlp/server.rs +++ b/rust/otap-dataflow/crates/otap/src/otap_grpc/otlp/server.rs @@ -8,7 +8,7 @@ //! requires it use std::convert::Infallible; -use std::sync::{Arc, Mutex}; +use std::sync::Arc; use std::task::Poll; use crate::accessory::slots::{Key as SlotKey, State as SlotsState}; @@ -23,6 +23,7 @@ use otap_df_pdata::OtlpProtoBytes; use otap_df_pdata::proto::opentelemetry::collector::logs::v1::ExportLogsServiceResponse; use otap_df_pdata::proto::opentelemetry::collector::metrics::v1::ExportMetricsServiceResponse; use otap_df_pdata::proto::opentelemetry::collector::trace::v1::ExportTraceServiceResponse; +use parking_lot::Mutex; use prost::Message; use prost::bytes::Buf; use tokio::sync::oneshot; @@ -31,15 +32,16 @@ use tonic::body::Body; use tonic::codec::{Codec, DecodeBuf, Decoder, EnabledCompressionEncodings, EncodeBuf, Encoder}; use tonic::server::{Grpc, NamedService, UnaryService}; -/// Shared state for binding requests with responses. This map is -/// generally optional depending on wait_for_result: true, we do not -/// create or use the state when ack/nack is not required. +/// Tracks outstanding request subscriptions for a single signal so ACK/NACK responses can be routed +/// back to the waiting caller. When `wait_for_result` is disabled the receiver skips creating this +/// map entirely. #[derive(Clone)] -pub struct SharedState( +pub struct AckSubscriptionState( + // parking_lot mutex keeps the hot ACK/NACK path lock-free from poisoning. pub(crate) Arc>>>>>, ); -impl SharedState { +impl AckSubscriptionState { pub(crate) fn new(max_size: usize) -> Self { Self(Arc::new(Mutex::new(SlotsState::new(max_size)))) } @@ -57,8 +59,8 @@ pub enum RouteResponse { Invalid, } -impl SharedState { - /// Internal helper to route responses to slots +impl AckSubscriptionState { + /// Routes the final outcome into the registered slot matching the provided `CallData`. #[must_use] pub fn route_response( &self, @@ -72,12 +74,7 @@ impl SharedState { }; // Try to take the channel from the slot under the mutex. - let chan = self - .0 - .lock() - .map(|mut state| state.take(key)) - .ok() - .flatten(); + let chan = self.0.lock().take(key); // Try to send. if chan.and_then(|sender| sender.send(result).ok()).is_some() { @@ -95,10 +92,12 @@ pub struct Settings { pub max_concurrent_requests: usize, /// Whether the receiver should wait. pub wait_for_result: bool, + /// Maximum size for inbound gRPC messages. + pub max_decoding_message_size: Option, /// Request compression allowed - pub accept_compression_encodings: EnabledCompressionEncodings, + pub request_compression_encodings: EnabledCompressionEncodings, /// Response compression used - pub send_compression_encodings: EnabledCompressionEncodings, + pub response_compression_encodings: EnabledCompressionEncodings, } /// Tonic `Codec` implementation that returns the bytes of the serialized message @@ -187,9 +186,9 @@ impl Decoder for OtlpBytesDecoder { fn decode(&mut self, src: &mut DecodeBuf<'_>) -> Result, Self::Error> { let buf = src.chunk(); let result = match self.signal { - SignalType::Logs => OtlpProtoBytes::ExportLogsRequest(buf.to_vec()), - SignalType::Metrics => OtlpProtoBytes::ExportMetricsRequest(buf.to_vec()), - SignalType::Traces => OtlpProtoBytes::ExportTracesRequest(buf.to_vec()), + SignalType::Logs => OtlpProtoBytes::ExportLogsRequest(buf.to_vec().into()), + SignalType::Metrics => OtlpProtoBytes::ExportMetricsRequest(buf.to_vec().into()), + SignalType::Traces => OtlpProtoBytes::ExportTracesRequest(buf.to_vec().into()), }; src.advance(buf.len()); Ok(Some(OtapPdata::new(Context::default(), result.into()))) @@ -202,9 +201,13 @@ impl Decoder for OtlpBytesDecoder { /// would require Arc>. fn new_grpc(signal: SignalType, settings: Settings) -> Grpc { let codec = OtlpBytesCodec::new(signal); - Grpc::new(codec).apply_compression_config( - settings.accept_compression_encodings, - settings.send_compression_encodings, + let mut grpc = Grpc::new(codec); + if let Some(limit) = settings.max_decoding_message_size { + grpc = grpc.max_decoding_message_size(limit); + } + grpc.apply_compression_config( + settings.request_compression_encodings, + settings.response_compression_encodings, ) } @@ -213,11 +216,11 @@ fn new_grpc(signal: SignalType, settings: Settings) -> Grpc { /// not require Arc>. struct OtapBatchService { effect_handler: Option>, - state: Option, + state: Option, } impl OtapBatchService { - fn new(effect_handler: EffectHandler, state: Option) -> Self { + fn new(effect_handler: EffectHandler, state: Option) -> Self { Self { effect_handler: Some(effect_handler), state, @@ -229,14 +232,12 @@ impl OtapBatchService { /// drops the future. pub(crate) struct SlotGuard { pub(crate) key: SlotKey, - pub(crate) state: SharedState, + pub(crate) state: AckSubscriptionState, } impl Drop for SlotGuard { fn drop(&mut self) { - if let Ok(mut state) = self.state.0.lock() { - state.cancel(self.key); - } + self.state.0.lock().cancel(self.key); } } @@ -255,17 +256,14 @@ impl UnaryService for OtapBatchService { Box::pin(async move { let cancel_rx = if let Some(state) = state { // Try to allocate a slot (under the mutex) for calldata. - let (key, rx) = match state - .0 - .lock() - .map(|mut state| state.allocate(|| oneshot::channel())) - { - Err(_) => return Err(Status::internal("Mutex poisoned")), - Ok(None) => { + let mut guard = state.0.lock(); + let (key, rx) = match guard.allocate(|| oneshot::channel()) { + None => { return Err(Status::resource_exhausted("Too many concurrent requests")); } - Ok(Some(pair)) => pair, + Some(pair) => pair, }; + drop(guard); // Enter the subscription. Slot key becomes calldata. effect_handler.subscribe_to( @@ -329,14 +327,14 @@ fn unimplemented_resp() -> Response { #[derive(Clone)] pub struct ServerCommon { effect_handler: EffectHandler, - state: Option, + state: Option, settings: Settings, } impl ServerCommon { /// Get this server's shared state for Ack/Nack routing #[must_use] - pub fn state(&self) -> Option { + pub fn state(&self) -> Option { self.state.clone() } @@ -345,7 +343,7 @@ impl ServerCommon { effect_handler, state: settings .wait_for_result - .then(|| SharedState::new(settings.max_concurrent_requests)), + .then(|| AckSubscriptionState::new(settings.max_concurrent_requests)), settings: settings.clone(), } } diff --git a/rust/otap-dataflow/crates/otap/src/otap_grpc/server_settings.rs b/rust/otap-dataflow/crates/otap/src/otap_grpc/server_settings.rs new file mode 100644 index 0000000000..05f5c30b69 --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otap_grpc/server_settings.rs @@ -0,0 +1,400 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Shared configuration for gRPC-based receivers. + +use crate::compression::{self, CompressionMethod}; +use crate::otap_grpc::otlp::server::Settings; +use otap_df_config::byte_units; +use serde::Deserialize; +use std::net::SocketAddr; +use std::time::Duration; +use tokio::net::TcpListener; +use tonic::codec::EnabledCompressionEncodings; +use tonic::transport::server::TcpIncoming; + +/// Common configuration shared across gRPC receivers. +#[derive(Debug, Deserialize, Clone)] +#[serde(deny_unknown_fields)] +pub struct GrpcServerSettings { + /// The endpoint details: protocol, name, port. + pub listening_addr: SocketAddr, + + /// Compression methods accepted for requests. Omitted field defaults to accepting zstd, gzip, + /// and deflate (in that preference order). + #[serde( + default, + deserialize_with = "compression::deserialize_compression_methods" + )] + pub request_compression: Option>, + + /// Compression methods used for responses. Defaults to no compression, falling back to the + /// request list when explicitly configured via the legacy `compression_method` option. + #[serde( + default, + deserialize_with = "compression::deserialize_compression_methods", + alias = "response_compression_method" + )] + pub response_compression: Option>, + + // --- All the following settings have defaults that should be reasonable for most users --- + // ----------------------------------------------------------------------------------------- + /// Maximum number of concurrent in-flight requests. + /// Defaults to `0`, which means the receiver adopts the downstream pdata channel capacity so + /// backpressure flows upstream automatically. Any non-zero value is still clamped to that + /// capacity at runtime. + #[serde(default = "default_max_concurrent_requests")] + pub max_concurrent_requests: usize, + + /// Whether newly accepted sockets should have `TCP_NODELAY` enabled. + /// Keeping this `true` (the default) avoids Nagle's algorithm and minimizes per-export latency. + /// Disabling it trades slightly higher latency for fewer small TCP packets when workloads + /// involve very bursty, tiny messages. + #[serde(default = "default_tcp_nodelay")] + pub tcp_nodelay: bool, + + /// TCP keepalive timeout for accepted sockets. + /// The 45s default evicts dead clients in under a minute without incurring much background + /// traffic. Raise it to reduce keepalive chatter, or set to `null` to disable kernel keepalives + /// entirely (at the cost of slower leak detection on broken links). + #[serde(default = "default_tcp_keepalive", with = "humantime_serde")] + pub tcp_keepalive: Option, + + /// Interval between TCP keepalive probes once keepalive is active. + /// Defaults to 15s so the kernel confirms progress quickly after the keepalive timeout. Longer + /// intervals reduce packets, shorter intervals detect stalled peers faster. Ignored if + /// `tcp_keepalive` is `null`. + #[serde(default = "default_tcp_keepalive_interval", with = "humantime_serde")] + pub tcp_keepalive_interval: Option, + + /// Number of TCP keepalive probes sent before a connection is declared dead. + /// The default (5) balances resilience to transient loss with timely reclamation of resources. + /// Smaller values clean up faster during outages, larger values favor noisy or lossy networks. + #[serde(default = "default_tcp_keepalive_retries")] + pub tcp_keepalive_retries: Option, + + /// Per-connection concurrency limit enforced by the transport layer. + /// By default it mirrors the effective `max_concurrent_requests`, so transport- and + /// application-level backpressure remain aligned. Lower values gate connection bursts earlier, + /// while higher values only help if you also raise `max_concurrent_requests`. Set to `0` to + /// revert to the derived default. + #[serde(default)] + pub transport_concurrency_limit: Option, + + /// Whether the gRPC server should shed load immediately once concurrency limits are hit. + /// Leaving this `true` (default) results in fast `resource_exhausted` responses and protects + /// the single-threaded runtime from unbounded queues. Turning it off allows requests to queue + /// but increases memory usage and tail latency under sustained overload. + #[serde(default = "default_load_shed")] + pub load_shed: bool, + + /// Initial HTTP/2 stream window size, in bytes. + /// Accepts plain integers or suffixed strings such as `8MiB`. The default 8MiB window reduces + /// flow-control stalls for large OTLP batches; trimming it lowers per-stream memory but may + /// throttle throughput, while increasing it benefits high-bandwidth deployments at the cost of + /// larger buffers. + #[serde( + default = "default_initial_stream_window_size", + deserialize_with = "byte_units::deserialize" + )] + pub initial_stream_window_size: Option, + + /// Initial HTTP/2 connection window size, in bytes. + /// Accepts plain integers or suffixed strings such as `32MiB`. Defaults to 24MiB, giving room + /// for several simultaneous large streams; adjust using the same trade-offs as the stream + /// window but applied per connection. + #[serde( + default = "default_initial_connection_window_size", + deserialize_with = "byte_units::deserialize" + )] + pub initial_connection_window_size: Option, + + /// Whether to rely on HTTP/2 adaptive window sizing instead of the manual values above. + /// Disabled by default so the receiver uses predictable static windows. Enabling this lets tonic + /// adjust flow-control windows dynamically, which can improve throughput on high-bandwidth links + /// but makes memory usage and latency more workload dependent (and largely ignores the window + /// sizes configured above). + #[serde(default = "default_http2_adaptive_window")] + pub http2_adaptive_window: bool, + + /// Maximum HTTP/2 frame size, in bytes. + /// Accepts plain integers or suffixed strings such as `16KiB`. The 16KiB default matches the + /// current tuning: large enough to keep framing overhead low for sizeable batches yet still + /// bounded; larger values further decrease framing costs at the expense of bigger per-frame + /// buffers, while smaller values force additional fragmentation and CPU work on jumbo exports. + #[serde( + default = "default_max_frame_size", + deserialize_with = "byte_units::deserialize" + )] + pub max_frame_size: Option, + + /// Maximum size for inbound gRPC messages, in bytes. + /// Accepts plain integers or suffixed strings such as `4MiB`. Defaults to tonic's 4MiB limit. + #[serde( + default = "default_max_decoding_message_size", + deserialize_with = "byte_units::deserialize" + )] + pub max_decoding_message_size: Option, + + /// Interval between HTTP/2 keepalive pings. + /// The default 30s ping keeps intermediaries aware of idle-but-healthy connections. Shorten it + /// to detect broken links faster, lengthen it to reduce ping traffic, or set to `null` to + /// disable HTTP/2 keepalives. + #[serde(default = "default_http2_keepalive_interval", with = "humantime_serde")] + pub http2_keepalive_interval: Option, + + /// Timeout waiting for an HTTP/2 keepalive acknowledgement. + /// Defaults to 10s, balancing rapid detection of stalled peers with tolerance for transient + /// network jitter. Decrease it for quicker failover or increase it for chatty-but-latent paths. + #[serde(default = "default_http2_keepalive_timeout", with = "humantime_serde")] + pub http2_keepalive_timeout: Option, + + /// Upper bound on concurrently active HTTP/2 streams per connection. + /// By default this tracks the effective `max_concurrent_requests`, keeping logical and transport + /// concurrency aligned. Lower values improve fairness between chatty clients. Higher values + /// matter only if you also raise `max_concurrent_requests`. Set to `0` to inherit the derived + /// default. + #[serde(default)] + pub max_concurrent_streams: Option, + + /// Whether to wait for the result (default: false) + /// + /// When enabled, the receiver will not send a response until the + /// immediate downstream component has acknowledged receipt of the + /// data. This does not guarantee that data has been fully + /// processed or successfully exported to the final destination, + /// since components are able acknowledge early. + /// + /// Note when wait_for_result=false, it is impossible to + /// see a failure, errors are effectively suppressed. + #[serde(default = "default_wait_for_result")] + pub wait_for_result: bool, + + /// Timeout for RPC requests. If not specified, no timeout is applied. + /// Format: humantime format (e.g., "30s", "5m", "1h", "500ms") + #[serde(default, with = "humantime_serde")] + pub timeout: Option, + + /// Timeout waiting for the initial HTTP/2 handshake. This is to prevent slowloris-style + /// attacks from holding connections open indefinitely. + /// Defaults to 5s. + #[serde(default = "default_http2_handshake_timeout", with = "humantime_serde")] + pub http2_handshake_timeout: Duration, +} + +impl GrpcServerSettings { + /// Returns the compression methods accepted for requests. + #[must_use] + pub fn request_compression_methods(&self) -> Vec { + match &self.request_compression { + Some(methods) => methods.clone(), + None => compression::DEFAULT_COMPRESSION_METHODS.to_vec(), + } + } + + /// Returns the compression methods configured for responses. + #[must_use] + pub fn response_compression_methods(&self) -> Vec { + match &self.response_compression { + Some(methods) => methods.clone(), + None => Vec::new(), + } + } + + /// Returns the first configured compression method for responses, if any. + #[must_use] + pub fn preferred_response_compression(&self) -> Option { + self.response_compression + .as_ref() + .and_then(|methods| methods.first().copied()) + } + + /// Builds the Tonic TCP Incoming. + #[must_use] + pub fn build_tcp_incoming(&self, tcp_listener: TcpListener) -> TcpIncoming { + TcpIncoming::from(tcp_listener) + .with_nodelay(Some(self.tcp_nodelay)) + .with_keepalive(self.tcp_keepalive) + .with_keepalive_interval(self.tcp_keepalive_interval) + .with_keepalive_retries(self.tcp_keepalive_retries) + } + + /// Returns the compression encodings to use for both requests and responses. + #[must_use] + pub fn compression_encodings( + &self, + ) -> (EnabledCompressionEncodings, EnabledCompressionEncodings) { + let mut request_compression = EnabledCompressionEncodings::default(); + for method in self.request_compression_methods() { + request_compression.enable(method.map_to_compression_encoding()); + } + + let mut response_compression = EnabledCompressionEncodings::default(); + for method in self.response_compression_methods() { + response_compression.enable(method.map_to_compression_encoding()); + } + (request_compression, response_compression) + } + + /// Builds the gRPC server settings from this configuration. + #[must_use] + pub fn build_settings(&self) -> Settings { + let (request_compression_encodings, response_compression_encodings) = + self.compression_encodings(); + + Settings { + max_concurrent_requests: self.max_concurrent_requests, + wait_for_result: self.wait_for_result, + max_decoding_message_size: self.max_decoding_message_size.map(|value| value as usize), + request_compression_encodings, + response_compression_encodings, + } + } +} + +const fn default_max_concurrent_requests() -> usize { + 0 +} + +const fn default_tcp_nodelay() -> bool { + true +} + +const fn default_tcp_keepalive() -> Option { + Some(Duration::from_secs(45)) +} + +const fn default_tcp_keepalive_interval() -> Option { + Some(Duration::from_secs(15)) +} + +const fn default_tcp_keepalive_retries() -> Option { + Some(5) +} + +const fn default_load_shed() -> bool { + true +} + +const fn default_initial_stream_window_size() -> Option { + Some(8 * 1024 * 1024) +} + +const fn default_initial_connection_window_size() -> Option { + Some(24 * 1024 * 1024) +} + +const fn default_max_frame_size() -> Option { + Some(16 * 1024) +} + +const fn default_max_decoding_message_size() -> Option { + Some(4 * 1024 * 1024) +} + +const fn default_http2_keepalive_interval() -> Option { + Some(Duration::from_secs(30)) +} + +const fn default_http2_keepalive_timeout() -> Option { + Some(Duration::from_secs(10)) +} + +const fn default_http2_adaptive_window() -> bool { + false +} + +const fn default_http2_handshake_timeout() -> Duration { + Duration::from_secs(5) +} + +const fn default_wait_for_result() -> bool { + // See https://github.com/open-telemetry/otel-arrow/issues/1311 + // This matches the OTel Collector default for wait_for_result, presently. + false +} + +impl Default for GrpcServerSettings { + fn default() -> Self { + Self { + listening_addr: ([0, 0, 0, 0], 0).into(), + request_compression: None, + response_compression: None, + max_concurrent_requests: default_max_concurrent_requests(), + tcp_nodelay: default_tcp_nodelay(), + tcp_keepalive: default_tcp_keepalive(), + tcp_keepalive_interval: default_tcp_keepalive_interval(), + tcp_keepalive_retries: default_tcp_keepalive_retries(), + transport_concurrency_limit: None, + load_shed: default_load_shed(), + initial_stream_window_size: default_initial_stream_window_size(), + initial_connection_window_size: default_initial_connection_window_size(), + http2_adaptive_window: default_http2_adaptive_window(), + max_frame_size: default_max_frame_size(), + max_decoding_message_size: default_max_decoding_message_size(), + http2_keepalive_interval: default_http2_keepalive_interval(), + http2_keepalive_timeout: default_http2_keepalive_timeout(), + http2_handshake_timeout: default_http2_handshake_timeout(), + max_concurrent_streams: None, + wait_for_result: default_wait_for_result(), + timeout: None, + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::compression::{CompressionMethod, DEFAULT_COMPRESSION_METHODS}; + use tonic::codec::CompressionEncoding; + + #[test] + fn defaults_match_expected_compression() { + let settings = GrpcServerSettings::default(); + + assert_eq!( + settings.request_compression_methods(), + DEFAULT_COMPRESSION_METHODS.to_vec() + ); + assert!(settings.response_compression_methods().is_empty()); + assert_eq!(settings.preferred_response_compression(), None); + } + + #[test] + fn response_compression_prefers_first_entry() { + let settings = GrpcServerSettings { + response_compression: Some(vec![CompressionMethod::Gzip, CompressionMethod::Zstd]), + ..Default::default() + }; + + assert_eq!( + settings.preferred_response_compression(), + Some(CompressionMethod::Gzip) + ); + assert_eq!( + settings.response_compression_methods(), + vec![CompressionMethod::Gzip, CompressionMethod::Zstd] + ); + } + + #[test] + fn build_settings_carries_core_limits_and_compression() { + let settings = GrpcServerSettings { + max_concurrent_requests: 42, + wait_for_result: true, + max_decoding_message_size: Some(8 * 1024 * 1024), + request_compression: Some(vec![CompressionMethod::Deflate]), + response_compression: Some(vec![CompressionMethod::Deflate]), + ..Default::default() + }; + + let built = settings.build_settings(); + assert_eq!(built.max_concurrent_requests, 42); + assert!(built.wait_for_result); + assert_eq!(built.max_decoding_message_size, Some(8 * 1024 * 1024)); + + let (req, resp) = settings.compression_encodings(); + assert!(req.is_enabled(CompressionEncoding::Deflate)); + assert!(resp.is_enabled(CompressionEncoding::Deflate)); + } +} diff --git a/rust/otap-dataflow/crates/otap/src/otap_receiver.rs b/rust/otap-dataflow/crates/otap/src/otap_receiver.rs index 66b9507532..4a27998219 100644 --- a/rust/otap-dataflow/crates/otap/src/otap_receiver.rs +++ b/rust/otap-dataflow/crates/otap/src/otap_receiver.rs @@ -1,30 +1,47 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 -//! Implementation of the OTAP receiver node +//! Implementation of the OTAP receiver node. //! -//! ToDo: implement Ack and Nack control message, wait for receiver node to receive a Ack control message then the service can send a response back -//! ToDo: implement config control message to handle live changing configuration -//! ToDo: Add HTTP support -//! ToDo: Implement proper deadline function for Shutdown ctrl msg +//! # Architecture +//! The OTAP receiver exposes three OTAP-over-gRPC bidirectional streaming services +//! (logs, metrics, traces). Each export call is fed directly into the pipeline via the +//! shared `EffectHandler`, which forwards telemetry batches downstream and handles +//! Ack/Nack routing. The receiver participates in the engine's control plane so that +//! shutdown, telemetry collection, and flow control signals have a single entry point. +//! The server tuning (`GrpcServerConfig`) is synchronized with the downstream channel +//! capacity so transport backpressure aligns with pipeline capacity. +//! +//! # Key optimizations +//! * Response streaming is driven by an async state machine instead of spawning a task +//! per request. This removes the extra `mpsc` hop and keeps backpressure intact. +//! * Ack/Nack correlation slots are protected by a `parking_lot::Mutex`, providing +//! fast, non-poisoning locking in async contexts where poisoned `std::sync::Mutex` +//! would otherwise stall the Tokio worker. +//! * Compression preferences, concurrency limits, and middleware (such as zstd header +//! handling) are applied once per service build so hot-path processing remains lean. +//! +//! ToDo: implement Ack and Nack control message, wait for receiver node to receive a Ack control message then the service can send a response back. +//! ToDo: implement config control message to handle live changing configuration. +//! ToDo: Add HTTP support. +//! ToDo: Implement proper deadline function for Shutdown ctrl msg. //! use crate::OTAP_RECEIVER_FACTORIES; -use crate::compression::CompressionMethod; +use crate::otap_grpc::common::{self, SignalAckRoutingState}; use crate::otap_grpc::middleware::zstd_header::ZstdRequestHeaderAdapter; -use crate::otap_grpc::otlp::server::{RouteResponse, SharedState}; use crate::otap_grpc::{ - ArrowLogsServiceImpl, ArrowMetricsServiceImpl, ArrowTracesServiceImpl, Settings, + ArrowLogsServiceImpl, ArrowMetricsServiceImpl, ArrowTracesServiceImpl, GrpcServerSettings, + Settings, }; use crate::pdata::OtapPdata; use async_trait::async_trait; use linkme::distributed_slice; -use otap_df_config::SignalType; use otap_df_config::node::NodeUserConfig; use otap_df_engine::ReceiverFactory; use otap_df_engine::config::ReceiverConfig; use otap_df_engine::context::PipelineContext; -use otap_df_engine::control::{AckMsg, NackMsg, NodeControlMsg}; +use otap_df_engine::control::NodeControlMsg; use otap_df_engine::error::{Error, ReceiverErrorKind, format_error_sources}; use otap_df_engine::node::NodeId; use otap_df_engine::receiver::ReceiverWrapper; @@ -40,11 +57,9 @@ use otap_df_telemetry::metrics::MetricSet; use otap_df_telemetry_macros::metric_set; use serde::Deserialize; use serde_json::Value; -use std::net::SocketAddr; use std::ops::Add; use std::sync::Arc; use std::time::{Duration, Instant}; -use tonic::codegen::tokio_stream::wrappers::TcpListenerStream; use tonic::transport::Server; use tonic_middleware::MiddlewareLayer; @@ -54,44 +69,9 @@ const OTAP_RECEIVER_URN: &str = "urn:otel:otap:receiver"; #[derive(Debug, Deserialize)] #[serde(deny_unknown_fields)] pub struct Config { - listening_addr: SocketAddr, - - compression_method: Option, - - /// Size of the channel used to buffer outgoing responses to the client. - response_stream_channel_size: usize, - - /// Maximum number of concurrent (in-flight) requests (default: 1000) - #[serde(default = "default_max_concurrent_requests")] - max_concurrent_requests: usize, - - /// Whether to wait for the result (default: true) - /// - /// When enabled, the receiver will not send a response until the - /// immediate downstream component has acknowledged receipt of the - /// data. This does not guarantee that data has been fully - /// processed or successfully exported to the final destination, - /// since components are able acknowledge early. - /// - /// Note when wait_for_result=false, it is impossible to - /// see a failure, errors are effectively suppressed. - #[serde(default = "default_wait_for_result")] - wait_for_result: bool, - - /// Timeout for RPC requests. If not specified, no timeout is applied. - /// Format: humantime format (e.g., "30s", "5m", "1h", "500ms") - #[serde(default, with = "humantime_serde")] - pub timeout: Option, -} - -const fn default_max_concurrent_requests() -> usize { - 1000 -} - -const fn default_wait_for_result() -> bool { - // See https://github.com/open-telemetry/otel-arrow/issues/1311 - // This matches the OTel Collector default for wait_for_result, presently. - false + /// Shared gRPC server settings reused across receivers. + #[serde(flatten)] + pub grpc: GrpcServerSettings, } /// A Receiver that listens for OTAP messages @@ -112,8 +92,11 @@ pub static OTAP_RECEIVER: ReceiverFactory = ReceiverFactory { node: NodeId, node_config: Arc, receiver_config: &ReceiverConfig| { + let mut receiver = OTAPReceiver::from_config(pipeline, &node_config.config)?; + receiver.tune_max_concurrent_requests(receiver_config.output_pdata_channel.capacity); + Ok(ReceiverWrapper::shared( - OTAPReceiver::from_config(pipeline, &node_config.config)?, + receiver, node, node_config, receiver_config, @@ -139,55 +122,8 @@ impl OTAPReceiver { Ok(OTAPReceiver { config, metrics }) } - fn route_ack_response(&self, states: &SharedStates, ack: AckMsg) -> RouteResponse { - let calldata = ack.calldata; - let resp = Ok(()); - let state = match ack.accepted.signal_type() { - SignalType::Logs => states.logs.as_ref(), - SignalType::Metrics => states.metrics.as_ref(), - SignalType::Traces => states.traces.as_ref(), - }; - - state - .map(|s| s.route_response(calldata, resp)) - .unwrap_or(RouteResponse::None) - } - - fn route_nack_response( - &self, - states: &SharedStates, - mut nack: NackMsg, - ) -> RouteResponse { - let calldata = std::mem::take(&mut nack.calldata); - let signal_type = nack.refused.signal_type(); - let resp = Err(nack); - let state = match signal_type { - SignalType::Logs => states.logs.as_ref(), - SignalType::Metrics => states.metrics.as_ref(), - SignalType::Traces => states.traces.as_ref(), - }; - - state - .map(|s| s.route_response(calldata, resp)) - .unwrap_or(RouteResponse::None) - } - - fn handle_ack_response(&mut self, resp: RouteResponse) { - match resp { - RouteResponse::Sent => self.metrics.acks_sent.inc(), - RouteResponse::Expired => self.metrics.acks_nacks_invalid_or_expired.inc(), - RouteResponse::Invalid => self.metrics.acks_nacks_invalid_or_expired.inc(), - RouteResponse::None => {} - } - } - - fn handle_nack_response(&mut self, resp: RouteResponse) { - match resp { - RouteResponse::Sent => self.metrics.nacks_sent.inc(), - RouteResponse::Expired => self.metrics.acks_nacks_invalid_or_expired.inc(), - RouteResponse::Invalid => self.metrics.acks_nacks_invalid_or_expired.inc(), - RouteResponse::None => {} - } + fn tune_max_concurrent_requests(&mut self, downstream_capacity: usize) { + common::tune_max_concurrent_requests(&mut self.config.grpc, downstream_capacity); } } @@ -208,13 +144,6 @@ pub struct OtapReceiverMetrics { pub acks_nacks_invalid_or_expired: Counter, } -/// State shared between gRPC server task and the effect handler. -struct SharedStates { - logs: Option, - metrics: Option, - traces: Option, -} - // Use the async_trait due to the need for thread safety because of tonic requiring Send and Sync traits // The Shared version of the receiver allows us to implement a Receiver that requires the effect handler to be Send and Sync // @@ -225,14 +154,13 @@ impl shared::Receiver for OTAPReceiver { mut ctrl_msg_recv: shared::ControlChannel, effect_handler: shared::EffectHandler, ) -> Result { - // create listener on addr provided from config - let listener = effect_handler.tcp_listener(self.config.listening_addr)?; - let listener_stream = TcpListenerStream::new(listener); - + let config = &self.config.grpc; + let listener = effect_handler.tcp_listener(config.listening_addr)?; + let incoming = config.build_tcp_incoming(listener); + // ToDo `Settings` could be embedded into the `GrpcServerSettings` to avoid this extra step. let settings = Settings { - response_stream_channel_size: self.config.response_stream_channel_size, - max_concurrent_requests: self.config.max_concurrent_requests, - wait_for_result: self.config.wait_for_result, + max_concurrent_requests: config.max_concurrent_requests, + wait_for_result: config.wait_for_result, }; //create services for the grpc server and clone the effect handler to pass message @@ -240,38 +168,33 @@ impl shared::Receiver for OTAPReceiver { let metrics_service = ArrowMetricsServiceImpl::new(effect_handler.clone(), &settings); let traces_service = ArrowTracesServiceImpl::new(effect_handler.clone(), &settings); - let states = SharedStates { - logs: logs_service.state(), - metrics: metrics_service.state(), - traces: traces_service.state(), - }; + let states = SignalAckRoutingState::new( + logs_service.state(), + metrics_service.state(), + traces_service.state(), + ); let mut logs_server = ArrowLogsServiceServer::new(logs_service); let mut metrics_server = ArrowMetricsServiceServer::new(metrics_service); let mut traces_server = ArrowTracesServiceServer::new(traces_service); - // apply the tonic compression if it is set - if let Some(ref compression) = self.config.compression_method { - let encoding = compression.map_to_compression_encoding(); - - logs_server = logs_server - .send_compressed(encoding) - .accept_compressed(encoding); - metrics_server = metrics_server - .send_compressed(encoding) - .accept_compressed(encoding); - traces_server = traces_server - .send_compressed(encoding) - .accept_compressed(encoding); + // apply the tonic compression settings + let request_compressions = config.request_compression_methods(); + for method in &request_compressions { + let encoding = method.map_to_compression_encoding(); + logs_server = logs_server.accept_compressed(encoding); + metrics_server = metrics_server.accept_compressed(encoding); + traces_server = traces_server.accept_compressed(encoding); } - let mut server_builder = Server::builder(); - - // Apply timeout if configured - if let Some(timeout) = self.config.timeout { - server_builder = server_builder.timeout(timeout); + if let Some(method) = config.preferred_response_compression() { + let encoding = method.map_to_compression_encoding(); + logs_server = logs_server.send_compressed(encoding); + metrics_server = metrics_server.send_compressed(encoding); + traces_server = traces_server.send_compressed(encoding); } + let server_builder = common::apply_server_tuning(Server::builder(), config); let server = server_builder .layer(MiddlewareLayer::new(ZstdRequestHeaderAdapter::default())) .add_service(logs_server) @@ -295,15 +218,27 @@ impl shared::Receiver for OTAPReceiver { _ = telemetry_cancel_handle.cancel().await; return Ok(TerminalState::new(deadline, [snapshot])); }, - Ok(NodeControlMsg::CollectTelemetry { mut metrics_reporter }) => { + Ok(NodeControlMsg::CollectTelemetry { metrics_reporter }) => { // Report current receiver metrics. _ = metrics_reporter.report(&mut self.metrics); }, Ok(NodeControlMsg::Ack(ack)) => { - self.handle_ack_response(self.route_ack_response(&states, ack)); + let resp = common::route_ack_response(&states, ack); + common::handle_route_response( + resp, + &mut self.metrics, + |metrics| metrics.acks_sent.inc(), + |metrics| metrics.acks_nacks_invalid_or_expired.inc(), + ); }, Ok(NodeControlMsg::Nack(nack)) => { - self.handle_nack_response(self.route_nack_response(&states, nack)); + let resp = common::route_nack_response(&states, nack); + common::handle_route_response( + resp, + &mut self.metrics, + |metrics| metrics.nacks_sent.inc(), + |metrics| metrics.acks_nacks_invalid_or_expired.inc(), + ); }, Err(e) => { return Err(Error::ChannelRecvError(e)); @@ -318,7 +253,7 @@ impl shared::Receiver for OTAPReceiver { }, // Run server - result = server.serve_with_incoming(listener_stream) => { + result = server.serve_with_incoming(incoming) => { if let Err(error) = result { // Report receiver error let source_detail = format_error_sources(&error); @@ -794,7 +729,6 @@ mod tests { let grpc_port = portpicker::pick_unused_port().expect("No free ports"); let grpc_endpoint = format!("http://{grpc_addr}:{grpc_port}"); let addr: SocketAddr = format!("{grpc_addr}:{grpc_port}").parse().unwrap(); - let response_stream_channel_size = 100; // create our receiver let node_config = Arc::new(NodeUserConfig::new_receiver_config(OTAP_RECEIVER_URN)); @@ -812,11 +746,12 @@ mod tests { // Create config JSON let config = json!({ "listening_addr": addr.to_string(), - "response_stream_channel_size": response_stream_channel_size }); + let mut receiver = OTAPReceiver::from_config(pipeline_ctx, &config).unwrap(); + receiver.tune_max_concurrent_requests(test_runtime.config().output_pdata_channel.capacity); let receiver = ReceiverWrapper::shared( - OTAPReceiver::from_config(pipeline_ctx, &config).unwrap(), + receiver, test_node(test_runtime.config().name.clone()), node_config, test_runtime.config(), @@ -840,86 +775,153 @@ mod tests { let pipeline_ctx = controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); - // Test with custom max_concurrent_requests, max_concurrent_requests defaults to 1000 + // Test with custom max_concurrent_requests, max_concurrent_requests defaults to 0 let config_with_max_concurrent_requests = json!({ "listening_addr": "127.0.0.1:4317", - "response_stream_channel_size": 100, "max_concurrent_requests": 5000 }); let receiver = OTAPReceiver::from_config(pipeline_ctx.clone(), &config_with_max_concurrent_requests) .unwrap(); - assert_eq!(receiver.config.listening_addr.to_string(), "127.0.0.1:4317"); - assert_eq!(receiver.config.response_stream_channel_size, 100); - assert_eq!(receiver.config.max_concurrent_requests, 5000); - assert!(!receiver.config.wait_for_result); - assert!(receiver.config.compression_method.is_none()); - assert!(receiver.config.timeout.is_none()); - - // Test with minimal required fields, max_concurrent_requests defaults to 1000, wait_for_result defaults to false + assert_eq!( + receiver.config.grpc.listening_addr.to_string(), + "127.0.0.1:4317" + ); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 5000); + assert!(!receiver.config.grpc.wait_for_result); + assert!(receiver.config.grpc.request_compression.is_none()); + assert!(receiver.config.grpc.response_compression.is_none()); + assert!( + receiver + .config + .grpc + .preferred_response_compression() + .is_none() + ); + assert!(receiver.config.grpc.timeout.is_none()); + + // Test with minimal required fields, max_concurrent_requests defaults to 0, wait_for_result defaults to false let config_minimal = json!({ "listening_addr": "127.0.0.1:4318", - "response_stream_channel_size": 200 }); let receiver = OTAPReceiver::from_config(pipeline_ctx.clone(), &config_minimal).unwrap(); - assert_eq!(receiver.config.listening_addr.to_string(), "127.0.0.1:4318"); - assert_eq!(receiver.config.response_stream_channel_size, 200); - assert_eq!(receiver.config.max_concurrent_requests, 1000); - assert!(!receiver.config.wait_for_result); - assert!(receiver.config.compression_method.is_none()); - assert!(receiver.config.timeout.is_none()); + assert_eq!( + receiver.config.grpc.listening_addr.to_string(), + "127.0.0.1:4318" + ); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 0); + assert!(!receiver.config.grpc.wait_for_result); + assert!(receiver.config.grpc.request_compression.is_none()); + assert!(receiver.config.grpc.response_compression.is_none()); + assert!( + receiver + .config + .grpc + .preferred_response_compression() + .is_none() + ); + assert!(receiver.config.grpc.timeout.is_none()); // Test with full configuration including gzip compression let config_full_gzip = json!({ "listening_addr": "127.0.0.1:4319", - "response_stream_channel_size": 150, "compression_method": "gzip", "max_concurrent_requests": 2500, "wait_for_result": true, "timeout": "30s" }); let receiver = OTAPReceiver::from_config(pipeline_ctx.clone(), &config_full_gzip).unwrap(); - assert_eq!(receiver.config.listening_addr.to_string(), "127.0.0.1:4319"); - assert_eq!(receiver.config.response_stream_channel_size, 150); - assert_eq!(receiver.config.max_concurrent_requests, 2500); - assert!(receiver.config.wait_for_result); - assert!(matches!( - receiver.config.compression_method, - Some(CompressionMethod::Gzip) - )); - assert_eq!(receiver.config.timeout, Some(Duration::from_secs(30))); + assert_eq!( + receiver.config.grpc.listening_addr.to_string(), + "127.0.0.1:4319" + ); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 2500); + assert!(receiver.config.grpc.wait_for_result); + assert_eq!( + receiver.config.grpc.request_compression, + Some(vec![CompressionMethod::Gzip]) + ); + assert!(receiver.config.grpc.response_compression.is_none()); + assert!( + receiver + .config + .grpc + .preferred_response_compression() + .is_none() + ); + assert_eq!(receiver.config.grpc.timeout, Some(Duration::from_secs(30))); // Test with zstd compression let config_with_zstd = json!({ "listening_addr": "127.0.0.1:4320", - "response_stream_channel_size": 50, "compression_method": "zstd", "wait_for_result": false }); let receiver = OTAPReceiver::from_config(pipeline_ctx.clone(), &config_with_zstd).unwrap(); - assert_eq!(receiver.config.listening_addr.to_string(), "127.0.0.1:4320"); - assert_eq!(receiver.config.response_stream_channel_size, 50); - assert!(!receiver.config.wait_for_result); - assert!(matches!( - receiver.config.compression_method, - Some(CompressionMethod::Zstd) - )); - assert!(receiver.config.timeout.is_none()); + assert_eq!( + receiver.config.grpc.listening_addr.to_string(), + "127.0.0.1:4320" + ); + assert!(!receiver.config.grpc.wait_for_result); + assert_eq!( + receiver.config.grpc.request_compression, + Some(vec![CompressionMethod::Zstd]) + ); + assert!(receiver.config.grpc.response_compression.is_none()); + assert!( + receiver + .config + .grpc + .preferred_response_compression() + .is_none() + ); + assert!(receiver.config.grpc.timeout.is_none()); // Test with deflate compression let config_with_deflate = json!({ "listening_addr": "127.0.0.1:4321", - "response_stream_channel_size": 75, "compression_method": "deflate" }); - let receiver = OTAPReceiver::from_config(pipeline_ctx, &config_with_deflate).unwrap(); - assert_eq!(receiver.config.listening_addr.to_string(), "127.0.0.1:4321"); - assert_eq!(receiver.config.response_stream_channel_size, 75); - assert!(matches!( - receiver.config.compression_method, - Some(CompressionMethod::Deflate) - )); - assert!(receiver.config.timeout.is_none()); + let receiver = + OTAPReceiver::from_config(pipeline_ctx.clone(), &config_with_deflate).unwrap(); + assert_eq!( + receiver.config.grpc.listening_addr.to_string(), + "127.0.0.1:4321" + ); + assert_eq!( + receiver.config.grpc.request_compression, + Some(vec![CompressionMethod::Deflate]) + ); + assert!(receiver.config.grpc.response_compression.is_none()); + assert!( + receiver + .config + .grpc + .preferred_response_compression() + .is_none() + ); + assert!(receiver.config.grpc.timeout.is_none()); + + // Test with explicit response compression configuration only + let config_with_response_only = json!({ + "listening_addr": "127.0.0.1:4322", + "response_compression_method": "gzip" + }); + let receiver = OTAPReceiver::from_config(pipeline_ctx, &config_with_response_only).unwrap(); + assert_eq!( + receiver.config.grpc.listening_addr.to_string(), + "127.0.0.1:4322" + ); + assert!(receiver.config.grpc.request_compression.is_none()); + assert_eq!( + receiver.config.grpc.response_compression, + Some(vec![CompressionMethod::Gzip]) + ); + assert_eq!( + receiver.config.grpc.preferred_response_compression(), + Some(CompressionMethod::Gzip) + ); + assert!(receiver.config.grpc.timeout.is_none()); } #[test] @@ -944,12 +946,13 @@ mod tests { let config = json!({ "listening_addr": addr.to_string(), - "response_stream_channel_size": 100, "wait_for_result": true // Enable ACK handling }); + let mut receiver = OTAPReceiver::from_config(pipeline_ctx, &config).unwrap(); + receiver.tune_max_concurrent_requests(test_runtime.config().output_pdata_channel.capacity); let receiver = ReceiverWrapper::shared( - OTAPReceiver::from_config(pipeline_ctx, &config).unwrap(), + receiver, test_node(test_runtime.config().name.clone()), node_config, test_runtime.config(), @@ -983,12 +986,13 @@ mod tests { let config = json!({ "listening_addr": addr.to_string(), - "response_stream_channel_size": 100, "wait_for_result": true // Enable NACK handling }); + let mut receiver = OTAPReceiver::from_config(pipeline_ctx, &config).unwrap(); + receiver.tune_max_concurrent_requests(test_runtime.config().output_pdata_channel.capacity); let receiver = ReceiverWrapper::shared( - OTAPReceiver::from_config(pipeline_ctx, &config).unwrap(), + receiver, test_node(test_runtime.config().name.clone()), node_config, test_runtime.config(), diff --git a/rust/otap-dataflow/crates/otap/src/otel_receiver.rs b/rust/otap-dataflow/crates/otap/src/otel_receiver.rs new file mode 100644 index 0000000000..8203ac91fa --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otel_receiver.rs @@ -0,0 +1,861 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Experimental receiver that serves OTLP and OTAP gRPC endpoints directly on top of the `h2` +//! crate. +//! +//! This receiver keeps all request handling on the current thread so it can integrate with a +//! thread-per-core runtime without requiring `Send + Sync` futures. +//! +//! Design goals: +//! - Support OTLP and OTAP Arrow over gRPC with minimal dependencies. +//! - Avoid `Send + Sync` bounds on request handlers so we can integrate with a single threaded executor. +//! - No `Arc` or `Mutex` in the hot path. +//! - Minimize heap allocations on the hot path. +//! +//! ToDo: Add snappy support. +//! ToDo: Improve error handling and metrics: surface clear statuses when the client requests +//! unsupported codecs, log negotiation results, and add counters for negotiated and +//! unsupported compression cases. +//! ToDo: Add support for Unix domain sockets as a transport option. + +mod ack; +mod encoder; +pub(crate) mod grpc; +mod response_templates; +mod router; +mod status; +mod stream; + +#[cfg(test)] +mod test_common; + +#[cfg(test)] +mod otlp_tests; + +#[cfg(test)] +mod otap_tests; + +use crate::OTAP_RECEIVER_FACTORIES; +use crate::otap_grpc::common; +use crate::otap_grpc::{GrpcServerSettings, Settings, per_connection_limit}; +use crate::otap_receiver::OtapReceiverMetrics; +use crate::otel_receiver::router::{GrpcRequestRouter, respond_with_error}; +use crate::pdata::OtapPdata; +use ack::{AckRegistries, AckRegistry, route_ack_response, route_nack_response}; +use async_trait::async_trait; +use bytes::Bytes; +use encoder::ResponseEncoderPool; +use futures::{FutureExt, Stream, StreamExt, stream::FuturesUnordered}; +use grpc::{AcceptedGrpcEncodings, build_accept_encoding_header}; +use h2::server::{self, SendResponse}; +use h2::{Ping, PingPong}; +use http::Request; +use linkme::distributed_slice; +use otap_df_config::node::NodeUserConfig; +use otap_df_engine::ReceiverFactory; +use otap_df_engine::admitter::{AdmitDecision, Admitter, ConnectionGuard}; +use otap_df_engine::config::ReceiverConfig; +use otap_df_engine::context::PipelineContext; +use otap_df_engine::control::NodeControlMsg; +use otap_df_engine::error::{Error, ReceiverErrorKind, format_error_sources}; +use otap_df_engine::local::receiver as local; +use otap_df_engine::node::NodeId; +use otap_df_engine::receiver::ReceiverWrapper; +use otap_df_engine::terminal_state::TerminalState; +use otap_df_telemetry::metrics::MetricSet; +use response_templates::ResponseTemplates; +use serde::Deserialize; +use status::Status; +use std::cell::RefCell; +use std::fmt; +use std::future::Future; +use std::io; +use std::ops::Add; +use std::pin::Pin; +use std::rc::Rc; +use std::sync::Arc; +use std::task::{Context as TaskContext, Poll}; +use std::time::{Duration, Instant}; +use tokio::task::JoinSet; +use tokio::time::{Sleep, sleep}; +use tokio_util::sync::CancellationToken; +use tonic::transport::server::TcpIncoming; + +/// URN used to register this receiver implementation in the engine. +const OTEL_RECEIVER_URN: &str = "urn:otel:otel:receiver"; + +/// Default maximum message size for decoding gRPC messages. +/// Set `GrpcServerSettings::max_decoding_message_size` to override. +const DEFAULT_MAX_DECODING_MESSAGE_SIZE: u32 = 64 * 1024 * 1024; /* 64MB */ + +/// Configuration for the experimental OTEL receiver. +#[derive(Debug, Deserialize)] +#[serde(deny_unknown_fields)] +pub struct Config { + /// Shared gRPC server settings reused across receivers. + #[serde(flatten)] + pub grpc: GrpcServerSettings, +} + +/// Experimental OTEL receiver that speaks OTLP and OTAP Arrow over pure `h2`. +pub struct OtelReceiver { + config: Config, + metrics: MetricSet, +} + +#[allow(unsafe_code)] +#[distributed_slice(OTAP_RECEIVER_FACTORIES)] +/// Registers the receiver in the distributed slice used by the engine. +pub static OTEL_RECEIVER: ReceiverFactory = ReceiverFactory { + name: OTEL_RECEIVER_URN, + create: |pipeline: PipelineContext, + node: NodeId, + node_config: Arc, + receiver_config: &ReceiverConfig| { + let mut receiver = OtelReceiver::from_config(pipeline, &node_config.config)?; + receiver.tune_max_concurrent_requests(receiver_config.output_pdata_channel.capacity); + Ok(ReceiverWrapper::local( + receiver, + node, + node_config, + receiver_config, + )) + }, +}; + +impl OtelReceiver { + /// Builds a receiver instance from the user configuration stored on the node. + pub fn from_config( + pipeline_ctx: PipelineContext, + config: &serde_json::Value, + ) -> Result { + let config: Config = serde_json::from_value(config.clone()).map_err(|e| { + otap_df_config::error::Error::InvalidUserConfig { + error: e.to_string(), + } + })?; + + let metrics = pipeline_ctx.register_metrics::(); + Ok(Self { config, metrics }) + } + + fn tune_max_concurrent_requests(&mut self, downstream_capacity: usize) { + common::tune_max_concurrent_requests(&mut self.config.grpc, downstream_capacity); + } +} + +#[async_trait(?Send)] +impl local::Receiver for OtelReceiver { + async fn start( + mut self: Box, + mut ctrl_msg_recv: local::ControlChannel, + effect_handler: local::EffectHandler, + ) -> Result { + // Admission control and TCP listener. + // ToDo The hardcoded limits here are intentionally conservative and will be made tunable soon in a follow-up PR + let admitter = Admitter::new( + 100000, + self.config.grpc.max_concurrent_streams.unwrap_or(100), + 100000, + ); + + let config = Rc::new(self.config.grpc.clone()); + let listener = effect_handler.tcp_listener(config.listening_addr)?; + let mut incoming = config.build_tcp_incoming(listener); + + let settings = Settings { + max_concurrent_requests: config.max_concurrent_requests, + wait_for_result: config.wait_for_result, + }; + let max_in_flight_per_connection = per_connection_limit(&settings); + + // Per signal Ack registries. + // If `wait_for_result` is disabled we skip creating registries entirely and respond + // immediately after enqueueing work into the pipeline. + let logs_ack_registry = settings + .wait_for_result + .then(|| AckRegistry::new(settings.max_concurrent_requests)); + let metrics_ack_registry = settings + .wait_for_result + .then(|| AckRegistry::new(settings.max_concurrent_requests)); + let traces_ack_registry = settings + .wait_for_result + .then(|| AckRegistry::new(settings.max_concurrent_requests)); + let ack_registries = AckRegistries::new( + logs_ack_registry.clone(), + metrics_ack_registry.clone(), + traces_ack_registry.clone(), + ); + + // Compression configuration and response templates. We precompute: + // - which request encodings are allowed, + // - the `grpc-accept-encoding` header we advertise, and + // - response header templates for each configured response compression method. + let request_encoding_methods = config.request_compression_methods(); + let request_encodings = AcceptedGrpcEncodings::from_methods(&request_encoding_methods); + let request_accept_header = build_accept_encoding_header(&request_encoding_methods); + + let response_methods = config.response_compression_methods(); + let response_templates = response_methods.iter().copied().fold( + ResponseTemplates::new(request_accept_header.clone()), + |acc, method| acc.with_method(method, &request_accept_header), + ); + + // The encoder pool size is tied to the number of concurrent requests. + let encoder_pool_capacity = settings.max_concurrent_requests.max(1); + let response_encoders = ResponseEncoderPool::new(&response_methods, encoder_pool_capacity); + + // Router shared by all h2 streams on this thread. The router holds shared state. + let router = Rc::new(GrpcRequestRouter { + effect_handler: effect_handler.clone(), + logs_ack_registry, + metrics_ack_registry, + traces_ack_registry, + max_in_flight_per_connection, + request_encodings, + request_accept_header: request_accept_header.clone(), + response_methods, + request_timeout: config.timeout, + response_encoders, + response_templates, + // Shared zstd decompressor used by gRPC bodies on the current core. + zstd_decompressor: RefCell::new(None), + max_decoding_message_size: self + .config + .grpc + .max_decoding_message_size + .unwrap_or(DEFAULT_MAX_DECODING_MESSAGE_SIZE), + }); + + // Telemetry and cancellation. + let cancel_token = CancellationToken::new(); + let telemetry_cancel_handle = effect_handler + .start_periodic_telemetry(Duration::from_secs(1)) + .await?; + + // Local enum that lets us treat control and server completion uniformly without relying on + // `select!` macros (so no branch cancellation). + enum DriverEvent { + Control(Result), + Server(Result<(), io::Error>), + } + + // Control plane loop: + // - handles shutdown, + // - exposes metrics snapshots, + // - routes Ack and Nack control messages back into the Ack registries. + let control_loop = Box::pin(async { + loop { + match ctrl_msg_recv.recv().await { + Ok(NodeControlMsg::Shutdown { deadline, .. }) => { + let snapshot = self.metrics.snapshot(); + _ = telemetry_cancel_handle.cancel().await; + return Ok(TerminalState::new(deadline, [snapshot])); + } + Ok(NodeControlMsg::CollectTelemetry { metrics_reporter }) => { + // Best effort metrics push; errors are ignored. + _ = metrics_reporter.report(&mut self.metrics); + } + Ok(NodeControlMsg::Ack(ack)) => { + let result = route_ack_response(&ack_registries, ack); + common::handle_route_response( + result, + &mut self.metrics, + |metrics| metrics.acks_sent.inc(), + |metrics| metrics.acks_nacks_invalid_or_expired.inc(), + ); + } + Ok(NodeControlMsg::Nack(nack)) => { + let result = route_nack_response(&ack_registries, nack); + common::handle_route_response( + result, + &mut self.metrics, + |metrics| metrics.nacks_sent.inc(), + |metrics| metrics.acks_nacks_invalid_or_expired.inc(), + ); + } + Err(e) => return Err(Error::ChannelRecvError(e)), + // Other control messages can be added here when needed. + _ => {} + } + } + }); + + // Data plane loop that accepts TCP connections and drives the h2 server. + let grpc_loop = Box::pin(run_grpc_server( + &mut incoming, + config, + router, + cancel_token.clone(), + admitter.clone(), + )); + + // We manually poll both futures and stop as soon as either finishes. + let mut control_future = control_loop; + let mut server_future = grpc_loop; + + let first_loop_done = futures::future::poll_fn(|cx| { + if let Poll::Ready(res) = control_future.as_mut().poll(cx) { + return Poll::Ready(DriverEvent::Control(res)); + } + if let Poll::Ready(res) = server_future.as_mut().poll(cx) { + return Poll::Ready(DriverEvent::Server(res)); + } + Poll::Pending + }) + .await; + + let server_done = match first_loop_done { + DriverEvent::Control(ctrl_msg_result) => { + cancel_token.cancel(); + return ctrl_msg_result; + } + DriverEvent::Server(server_result) => { + if let Err(error) = server_result { + log::error!("OTEL H2 receiver server loop failed: {error}"); + let source_detail = format_error_sources(&error); + return Err(Error::ReceiverError { + receiver: effect_handler.receiver_id(), + kind: ReceiverErrorKind::Transport, + error: error.to_string(), + source_detail, + }); + } + true + } + }; + + drop(control_future); + drop(server_future); + + if server_done { + return Ok(TerminalState::new( + Instant::now().add(Duration::from_secs(1)), + [self.metrics], + )); + } + + Ok(TerminalState::new( + Instant::now().add(Duration::from_secs(1)), + [self.metrics], + )) + } +} + +/// Build an `h2::server::Builder` from the configured gRPC settings. +/// +/// Only values that are explicitly configured are applied. Everything else uses +/// the default from the `h2` library. +fn build_h2_builder(settings: &GrpcServerSettings) -> server::Builder { + let mut builder = server::Builder::new(); + if let Some(window) = settings.initial_stream_window_size { + let _ = builder.initial_window_size(window); + } + if let Some(window) = settings.initial_connection_window_size { + let _ = builder.initial_connection_window_size(window); + } + if let Some(frame) = settings.max_frame_size { + let _ = builder.max_frame_size(frame); + } + builder +} + +/// Top level h2 server loop. +/// +/// This function: +/// - accepts new TCP connections from `incoming`, +/// - applies admission control for each connection, +/// - spawns a per connection task that drives the h2 state machine, and +/// - listens for cancellation so we can drain gracefully on shutdown. +async fn run_grpc_server( + incoming: &mut TcpIncoming, + grpc_config: Rc, + grpc_router: Rc, + cancel: CancellationToken, + admitter: Admitter, +) -> Result<(), io::Error> { + // Track all per TCP connection tasks. + let mut tcp_conn_tasks: JoinSet<()> = JoinSet::new(); + let mut accepting = true; + let h2_builder = build_h2_builder(&grpc_config); + let mut cancel_wait = Box::pin(cancel.cancelled()); + + loop { + // Drain completed connection tasks without awaiting the whole set. + // This lives outside the select style loop to avoid interfering with accept. + while let Some(res) = tcp_conn_tasks.join_next().now_or_never().flatten() { + if let Err(join_err) = res { + if log::log_enabled!(log::Level::Debug) { + log::debug!("h2 connection task join error: {join_err}"); + } + } + } + + enum ServerEvent { + Cancel, + Accept(Result), + IncomingClosed, + } + + let event = futures::future::poll_fn(|cx| { + if cancel_wait.as_mut().poll(cx).is_ready() { + return Poll::Ready(ServerEvent::Cancel); + } + + // Propagate any join errors from connection tasks as debug logs. + if !tcp_conn_tasks.is_empty() { + if let Poll::Ready(Some(Err(join_err))) = tcp_conn_tasks.poll_join_next(cx) { + if log::log_enabled!(log::Level::Debug) { + log::debug!("h2 connection task join error: {join_err}"); + } + } + } + + if accepting { + match StreamExt::poll_next_unpin(incoming, cx) { + Poll::Ready(Some(res)) => return Poll::Ready(ServerEvent::Accept(res)), + Poll::Ready(None) => return Poll::Ready(ServerEvent::IncomingClosed), + Poll::Pending => {} + } + } + + Poll::Pending + }) + .await; + + match event { + ServerEvent::Cancel => break, + ServerEvent::IncomingClosed => { + // No more connections from the listener. We keep running until all + // existing connection tasks finish. + accepting = false; + } + ServerEvent::Accept(res) => match res { + Ok(tcp_conn) => { + // Admission control runs before we spawn the connection task. + match admitter.try_admit_connection() { + AdmitDecision::Admitted(conn_guard) => { + let h2_builder = h2_builder.clone(); + let router = Rc::clone(&grpc_router); + + // Each connection holds its admission guard until it finishes. + // The AbortHandler from the admitter is currently unused. + let grpc_config = grpc_config.clone(); + _ = tcp_conn_tasks.spawn_local(async move { + if let Err(err) = handle_tcp_conn( + tcp_conn, + h2_builder, + router, + conn_guard, + grpc_config, + ) + .await + { + if log::log_enabled!(log::Level::Debug) { + log::debug!("h2 connection ended with error: {err}"); + } + } + }); + } + AdmitDecision::Busy => { + // Soft backpressure: drop the stream so the kernel backlog can absorb spikes. + if log::log_enabled!(log::Level::Trace) { + log::trace!("Connection admission busy; pausing accepts briefly"); + } + drop(tcp_conn); + // Yield to avoid a tight accept/reject loop. + tokio::task::yield_now().await; + } + AdmitDecision::Reject { message } => { + // Hard policy style rejection (circuit breaker etc). + if log::log_enabled!(log::Level::Warn) { + log::warn!("Connection admission rejected: {message}"); + } + drop(tcp_conn); + } + } + } + Err(err) => return Err(err), + }, + } + + // Once no more accepts will arrive and all tasks are drained we can exit. + if !accepting && tcp_conn_tasks.is_empty() { + break; + } + } + + // Graceful drain after cancellation or listener close. + while let Some(join_res) = tcp_conn_tasks.join_next().await { + if let Err(join_err) = join_res { + if log::log_enabled!(log::Level::Debug) { + log::debug!("h2 connection task join error: {join_err}"); + } + } + } + + Ok(()) +} + +/// Drives a single TCP connection through the h2 server state machine. +/// +/// Responsibilities: +/// - perform the h2 handshake, +/// - accept inbound streams (HTTP/2 requests), +/// - enforce per connection stream admission, and +/// - keep the connection alive via HTTP/2 PING when configured. +async fn handle_tcp_conn( + socket: tokio::net::TcpStream, + builder: server::Builder, + router: Rc, + // Keeps one connection slot while the connection is alive. + tcp_conn_guard: ConnectionGuard, + grpc_config: Rc, +) -> Result<(), h2::Error> { + //let keepalive_interval: Option, + //keepalive_timeout: Option, + + // HTTP/2 handshake. + let mut http2_conn = match tokio::time::timeout( + grpc_config.http2_handshake_timeout, + builder.handshake(socket), + ) + .await + { + Ok(Ok(conn)) => conn, + Ok(Err(err)) => return Err(err), + Err(_) => { + // Handshake took too long, drop the connection slot + log::debug!( + "h2 handshake timed out after {:?}", + grpc_config.http2_handshake_timeout + ); + return Ok(()); + } + }; + if log::log_enabled!(log::Level::Trace) { + log::trace!("h2 handshake established"); + } + + let keepalive = Http2Keepalive::new( + http2_conn.ping_pong(), + grpc_config.http2_keepalive_interval, + grpc_config.http2_keepalive_timeout, + ); + + // Wrap the connection in a pinned future so we can build a stream over `poll_accept`. + let mut http2_conn = Box::pin(http2_conn); + let mut accept_stream = futures::stream::poll_fn(move |cx| http2_conn.as_mut().poll_accept(cx)); + + // Keepalive ticks are driven by a custom stream so we reuse timers. + let mut keepalive_stream = KeepaliveStream::new(keepalive); + let mut in_flight = FuturesUnordered::new(); + let mut accepting = true; + let mut idle_spins: u8 = 0; + + let trace_enabled = log::log_enabled!(log::Level::Trace); + let debug_enabled = log::log_enabled!(log::Level::Debug); + + loop { + // Keepalive is only armed when there are no in flight request tasks. + keepalive_stream.set_idle(in_flight.is_empty()); + + let next_event = futures::future::poll_fn(|cx| { + // 1. Drain completed in flight request tasks. + if let Poll::Ready(Some(_)) = Pin::new(&mut in_flight).poll_next(cx) { + return Poll::Ready(Some(StreamEvent::Task)); + } + + // 2. Drive keepalive ticks if configured. + if keepalive_stream.is_active() { + if let Poll::Ready(ev) = Pin::new(&mut keepalive_stream).poll_next(cx) { + return Poll::Ready(ev); + } + } + + // 3. Accept new streams on this connection. + if accepting { + match Pin::new(&mut accept_stream).poll_next(cx) { + Poll::Ready(Some(res)) => { + return Poll::Ready(Some(StreamEvent::Accept(Box::new(res)))); + } + Poll::Ready(None) => { + return Poll::Ready(Some(StreamEvent::AcceptClosed)); + } + Poll::Pending => {} + } + } + + Poll::Pending + }) + .await; + + match next_event { + Some(StreamEvent::Accept(result)) => { + idle_spins = 0; + match *result { + Ok((request, respond)) => { + // Per stream admission for this connection. + match tcp_conn_guard.try_open_stream() { + AdmitDecision::Admitted(stream_guard) => { + let router = router.clone(); + in_flight.push(async move { + if trace_enabled { + log::trace!("New h2 stream: {}", request.uri().path()); + } + if let Err(status) = + router.route_grpc_request(request, respond).await + { + if debug_enabled { + log::debug!("Request failed: {}", status); + } + } + // Release per stream admission slot. + drop(stream_guard); + }); + } + AdmitDecision::Busy => { + // Per connection stream capacity is full: reply with RESOURCE_EXHAUSTED. + respond_with_error( + respond, + Status::resource_exhausted("stream capacity exhausted"), + &router.request_accept_header, + ); + } + AdmitDecision::Reject { message } => { + // Policy level rejection of this stream. + respond_with_error( + respond, + Status::unavailable(message), + &router.request_accept_header, + ); + } + } + } + Err(err) => return Err(err), + } + } + Some(StreamEvent::AcceptClosed) => { + idle_spins = 0; + accepting = false; + } + Some(StreamEvent::Keepalive(result)) => { + idle_spins = 0; + if let Err(err) = result { + if log::log_enabled!(log::Level::Debug) { + log::debug!("h2 keepalive failed: {err}"); + } + break; + } + } + Some(StreamEvent::Task) => { + idle_spins = 0; + } + None => { + // No work this tick. After a couple of tight spins yield to avoid burning CPU. + idle_spins = idle_spins.saturating_add(1); + if idle_spins >= 2 { + tokio::task::yield_now().await; + idle_spins = 0; + } + } + } + + // Exit once there are no more streams to accept, no pending tasks, and keepalive is idle. + if !accepting && in_flight.is_empty() && !keepalive_stream.is_active() { + break; + } + } + + Ok(()) +} + +/// Tracks whether a connection needs an HTTP/2 PING to keep the client alive. +/// +/// The keepalive logic is intentionally simple: +/// - only runs when the connection is idle, +/// - arms a timer for the configured interval, and +/// - waits for a matching PONG within the configured timeout. +struct Http2Keepalive { + ping_pong: PingPong, + interval: Duration, + timeout: Duration, + sleep: Option>>, +} + +impl Http2Keepalive { + fn new( + ping_pong: Option, + interval: Option, + timeout: Option, + ) -> Option { + let (ping_pong, interval, timeout) = match (ping_pong, interval, timeout) { + (Some(ping_pong), Some(interval), Some(timeout)) if !interval.is_zero() => { + (ping_pong, interval, timeout) + } + _ => return None, + }; + Some(Self { + ping_pong, + interval, + timeout, + sleep: None, + }) + } + + /// Updates the idle state of the connection. + /// + /// When the connection becomes idle we arm a sleep; once the connection becomes active + /// again we drop it so the next idle period starts a fresh timer. + fn update_idle_state(&mut self, idle: bool) { + if idle { + if self.sleep.is_none() { + self.sleep = Some(Box::pin(sleep(self.interval))); + } + } else if self.sleep.is_some() { + self.sleep = None; + } + } + + fn is_armed(&self) -> bool { + self.sleep.is_some() + } + + /// Waits for the next keepalive interval and fires a PING. + /// + /// If the PING does not complete within the configured timeout this returns an error + /// and the connection is closed. + async fn poll_tick(&mut self) -> Result<(), Http2KeepaliveError> { + let Some(mut sleeper) = self.sleep.take() else { + return Err(Http2KeepaliveError::Timeout); + }; + sleeper.as_mut().await; + self.sleep = Some(Box::pin(sleep(self.interval))); + + match tokio::time::timeout(self.timeout, self.ping_pong.ping(Ping::opaque())).await { + Ok(Ok(_)) => Ok(()), + Ok(Err(err)) => Err(Http2KeepaliveError::Ping(err)), + Err(_) => Err(Http2KeepaliveError::Timeout), + } + } +} + +enum Http2KeepaliveError { + Timeout, + Ping(h2::Error), +} + +impl fmt::Display for Http2KeepaliveError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Self::Timeout => write!(f, "keepalive timeout waiting for PONG"), + Self::Ping(err) => write!(f, "keepalive ping failed: {err}"), + } + } +} + +/// Events that can drive progress on a connection: +/// - a newly accepted stream, +/// - the accept side closing, +/// - a keepalive tick firing, or +/// - a previously spawned request task finishing. +enum StreamEvent { + Accept(Box, SendResponse), h2::Error>>), + AcceptClosed, + Keepalive(Result<(), Http2KeepaliveError>), + Task, +} + +/// Stream wrapper over `Http2Keepalive` that plugs into the same poll loop as new streams. +/// +/// The keepalive stream is only active when a configured keepalive exists and the +/// connection is idle. +struct KeepaliveStream { + keepalive: Option, + tick: Option, + idle: bool, + idle_streak: u8, +} + +/// Future alias used by `KeepaliveStream` for a single keepalive cycle. +type KeepaliveTick = + Pin, Http2Keepalive)> + 'static>>; + +impl KeepaliveStream { + fn new(keepalive: Option) -> Self { + Self { + keepalive, + tick: None, + idle: true, + idle_streak: 0, + } + } + + /// Mark the connection as idle or active. + /// + /// When we observe consecutive idle polls the underlying keepalive is armed. + fn set_idle(&mut self, idle: bool) { + self.idle = idle; + if !idle { + // Drop any pending tick so the next idle cycle re arms from scratch. + self.tick = None; + self.idle_streak = 0; + } else { + self.idle_streak = self.idle_streak.saturating_add(1); + } + } + + fn is_active(&self) -> bool { + self.keepalive.is_some() || self.tick.is_some() + } +} + +impl Stream for KeepaliveStream { + type Item = StreamEvent; + + fn poll_next(self: Pin<&mut Self>, cx: &mut TaskContext<'_>) -> Poll> { + let this = self.get_mut(); + + if !this.is_active() { + return Poll::Pending; + } + + loop { + // First drive an already armed keepalive tick to completion. + if let Some(tick) = this.tick.as_mut() { + match tick.as_mut().poll(cx) { + Poll::Ready((res, ka)) => { + this.tick = None; + this.keepalive = Some(ka); + return Poll::Ready(Some(StreamEvent::Keepalive(res))); + } + Poll::Pending => return Poll::Pending, + } + } + + let Some(mut keepalive) = this.keepalive.take() else { + return Poll::Pending; + }; + + // We only arm keepalive after a couple of consecutive idle polls. + if !this.idle || this.idle_streak < 2 { + this.keepalive = Some(keepalive); + return Poll::Pending; + } + + keepalive.update_idle_state(true); + if keepalive.is_armed() { + this.tick = Some(Box::pin(async move { + let res = keepalive.poll_tick().await; + (res, keepalive) + })); + // Immediately poll the newly created tick. + continue; + } else { + this.keepalive = Some(keepalive); + return Poll::Pending; + } + } + } +} diff --git a/rust/otap-dataflow/crates/otap/src/otel_receiver/ack.rs b/rust/otap-dataflow/crates/otap/src/otel_receiver/ack.rs new file mode 100644 index 0000000000..21d947a5ab --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otel_receiver/ack.rs @@ -0,0 +1,429 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Ack/Nack bookkeeping for the experimental OTAP receiver. +//! +//! The receiver runs in the thread-per-core engine where tasks, channels, and +//! effect handlers are NOT `Send`. Instead of locking, we rely on `Rc` + +//! `RefCell` and the fact that each registry is only ever touched from the same +//! single-threaded async runtime. The registry acts like a tiny slab of "wait +//! slots", and we bound the number of slots to participate in backpressure: if +//! no slots remain, new batches get an immediate "Too many concurrent requests" +//! status, preventing runaway resource use. When a request wants an ACK/NACK, +//! it allocates a slot, subscribes, and passes the token downstream. Later, the +//! pipeline produces a `NodeControlMsg::Ack/Nack`, the token is recovered, and +//! the waiting future resolves back into a `BatchStatus`. Keeping this state +//! machine isolated here lets the router focus purely on gRPC/H2 plumbing while +//! this module tracks slot lifetimes, wakers, and status formatting. +//! +//! General design goals: +//! - O(1) allocate/free so the hot path stays predictable. +//! - No locking or atomic work, since nothing ever leaves the local executor. +//! - Strict control over memory/concurrency so we can apply backpressure when +//! the pipeline can't keep up. + +use crate::otap_grpc::otlp::server::RouteResponse; +use crate::pdata::OtapPdata; +use otap_df_config::SignalType; +use otap_df_engine::control::{AckMsg, CallData, Context8u8, NackMsg}; +use otap_df_pdata::proto::opentelemetry::arrow::v1::{BatchStatus, StatusCode as ProtoStatusCode}; +use smallvec::smallvec; +use std::cell::RefCell; +use std::future::Future; +use std::mem; +use std::pin::Pin; +use std::rc::Rc; +use std::task::{Context as TaskContext, Poll, Waker}; + +/// Result returned when polling a registry slot for completion. +/// +/// This is used by both the streaming and unary paths: +/// - `Ack` means the pipeline processed the batch successfully, +/// - `Nack(reason)` means it failed with the provided reason, and +/// - `Cancelled` means the slot was reclaimed without a concrete outcome +/// (for example because the wait future was dropped). +pub(crate) enum AckPollResult { + Ack, + Nack(String), + Cancelled, +} + +/// Fixed size slab of wait slots used to correlate `Ack` and `Nack` control messages +/// with in flight requests. +/// +/// The registry: +/// - lives entirely on the single threaded runtime (`Rc>`), +/// - provides O(1) allocation and free via an intrusive free list, and +/// - participates directly in backpressure, since `allocate` returns `None` +/// when the slab is full. +#[derive(Clone)] +pub(crate) struct AckRegistry { + inner: Rc>, +} + +/// We pre-allocate a boxed slice of slots so indices remain stable and +/// `AckToken` can cheaply refer to one. +/// `head_free` points to the index of the first available slot in the intrusive list. +struct AckRegistryInner { + slots: Box<[AckSlot]>, + head_free: Option, +} + +impl AckRegistry { + pub(crate) fn new(max_size: usize) -> Self { + let mut slots = Vec::with_capacity(max_size); + + // Initialize the intrusive list. + // Each slot points to the next one: 0 -> 1 -> 2 ... + // The last slot points to None. + for i in 0..max_size { + let next_free = if i < max_size - 1 { Some(i + 1) } else { None }; + slots.push(AckSlot { + generation: 0, + state: SlotState::Free { next_free }, + }); + } + + let head_free = if max_size > 0 { Some(0) } else { None }; + + Self { + inner: Rc::new(RefCell::new(AckRegistryInner { + slots: slots.into_boxed_slice(), + head_free, + })), + } + } + + /// Attempts to allocate a free slot, returning its token on success. + /// O(1) operation: pops from the head of the intrusive linked list. + pub(crate) fn allocate(&self) -> Option { + let mut inner = self.inner.borrow_mut(); + + // Check if we have any free slots available + let slot_index = inner.head_free?; + + // 1. Extract the next pointer from the current free slot. + // We peek at the slot state. We CANNOT hold 'slot' mutable ref here while writing to head_free later. + let next_free = match &inner.slots[slot_index].state { + SlotState::Free { next_free } => *next_free, + _ => unreachable!("Corrupted AckRegistry: head_free pointed to a non-free slot"), + }; + + // 2. Update the head pointer. + inner.head_free = next_free; + + // 3. Initialize the slot for use. + // Now we can borrow 'slots' mutably again. + let slot = &mut inner.slots[slot_index]; + slot.generation = slot.generation.wrapping_add(1); + slot.state = SlotState::Waiting(WaitingSlot::new()); + + Some(AckToken { + slot_index, + generation: slot.generation, + }) + } + + /// Marks the slot as completed with the provided outcome, waking any waiter. + pub(crate) fn complete(&self, token: AckToken, result: Result<(), String>) -> RouteResponse { + let waker_opt = { + let mut inner = self.inner.borrow_mut(); + let Some(slot) = inner.slots.get_mut(token.slot_index) else { + return RouteResponse::Invalid; + }; + + if slot.generation != token.generation { + return RouteResponse::Expired; + } + + match &mut slot.state { + SlotState::Waiting(waiting) => { + waiting.outcome = match result { + Ok(()) => AckOutcome::Ack, + Err(reason) => AckOutcome::Nack(reason), + }; + waiting.waker.take() + } + SlotState::Free { .. } => return RouteResponse::Expired, + } + }; + + if let Some(waker) = waker_opt { + waker.wake(); + } + + RouteResponse::Sent + } + + /// Polls the slot, registering the waker if it is still pending. + /// If the slot is finished (Ack/Nack), it returns Ready and immediately frees the slot + /// back to the intrusive list. + pub(crate) fn poll_slot( + &self, + token: AckToken, + cx: &mut TaskContext<'_>, + ) -> Poll { + let mut inner = self.inner.borrow_mut(); + + // 1. Check the state of the slot. + // We scope this block so the mutable borrow of `inner.slots` ends before we call `free_slot_inner`. + let result_to_process = { + let slot = match inner.slots.get_mut(token.slot_index) { + Some(s) => s, + None => return Poll::Ready(AckPollResult::Cancelled), + }; + + if slot.generation != token.generation { + return Poll::Ready(AckPollResult::Cancelled); + } + + match &mut slot.state { + SlotState::Free { .. } => return Poll::Ready(AckPollResult::Cancelled), + SlotState::Waiting(waiting) => match &mut waiting.outcome { + AckOutcome::Pending => { + // Still pending: update waker and return. + let replace = match &waiting.waker { + Some(existing) => !existing.will_wake(cx.waker()), + None => true, + }; + if replace { + waiting.waker = Some(cx.waker().clone()); + } + return Poll::Pending; + } + // Completed: Return the result so we can free outside this block. + AckOutcome::Ack => Ok(()), + AckOutcome::Nack(reason) => Err(mem::take(reason)), + }, + } + }; + + // 2. If we are here, the slot is done (Ack or Nack). We must free it. + // The previous borrow of `inner.slots` (via `slot`) is dropped. + Self::free_slot_inner(&mut inner, token.slot_index); + + match result_to_process { + Ok(()) => Poll::Ready(AckPollResult::Ack), + Err(reason) => Poll::Ready(AckPollResult::Nack(reason)), + } + } + + /// Cancels the slot if it is still waiting (e.g. drop without completion). + pub(crate) fn cancel(&self, token: AckToken) { + let mut inner = self.inner.borrow_mut(); + + // 1. Check if we need to free. + // We use a read-only check first to avoid conflicts, though we hold `mut inner` anyway. + // The key is that we don't hold a reference to `slots` when calling `free_slot_inner`. + let should_free = if let Some(slot) = inner.slots.get(token.slot_index) { + if slot.generation != token.generation { + false + } else { + matches!(slot.state, SlotState::Waiting(_)) + } + } else { + false + }; + + if should_free { + Self::free_slot_inner(&mut inner, token.slot_index); + } + } + + /// Helper: transitions a slot at `index` to Free and pushes it onto the head + /// of the free list (LIFO). + fn free_slot_inner(inner: &mut AckRegistryInner, index: usize) { + let old_head = inner.head_free; + inner.slots[index].state = SlotState::Free { + next_free: old_head, + }; + inner.head_free = Some(index); + } +} + +/// Future that resolves once the provided slot receives an ACK/NACK (or is cancelled). +pub(crate) struct AckCompletionFuture { + token: AckToken, + state: AckRegistry, + completed: bool, +} + +impl AckCompletionFuture { + pub(crate) fn new(token: AckToken, state: AckRegistry) -> Self { + Self { + token, + state, + completed: false, + } + } +} + +impl Future for AckCompletionFuture { + type Output = AckPollResult; + + fn poll(self: Pin<&mut Self>, cx: &mut TaskContext<'_>) -> Poll { + let this = self.get_mut(); + match this.state.poll_slot(this.token, cx) { + Poll::Pending => Poll::Pending, + Poll::Ready(result) => { + this.completed = true; + Poll::Ready(result) + } + } + } +} + +impl Drop for AckCompletionFuture { + fn drop(&mut self) { + if !self.completed { + self.state.cancel(self.token); + } + } +} + +/// Individual slot that may be free or waiting for a result. +struct AckSlot { + generation: u32, + state: SlotState, +} + +/// Tracks whether a slot is unused (pointing to next free) or actively waiting. +enum SlotState { + Free { next_free: Option }, + Waiting(WaitingSlot), +} + +/// Carrier for a waiting slot's waker and eventual outcome. +struct WaitingSlot { + waker: Option, + outcome: AckOutcome, +} + +impl WaitingSlot { + fn new() -> Self { + Self { + waker: None, + outcome: AckOutcome::Pending, + } + } +} + +/// Final disposition for a slot once the pipeline responds. +enum AckOutcome { + Pending, + Ack, + Nack(String), +} + +/// Compact handle that identifies a single registry slot. +/// +/// The token is passed downstream as `CallData` and later reconstructed when +/// an `Ack` or `Nack` control message arrives. +#[derive(Clone, Copy)] +pub(crate) struct AckToken { + slot_index: usize, + generation: u32, +} + +impl AckToken { + pub(crate) fn to_calldata(self) -> CallData { + smallvec![ + Context8u8::from(self.slot_index as u64), + Context8u8::from(self.generation as u64) + ] + } + + pub(crate) fn from_calldata(calldata: &CallData) -> Option { + if calldata.len() < 2 { + return None; + } + let slot_index = usize::try_from(u64::from(calldata[0])).ok()?; + let generation = u64::from(calldata[1]) as u32; + Some(Self { + slot_index, + generation, + }) + } +} + +/// Convenience holder for the three per-signal registries. +#[derive(Clone, Default)] +pub(crate) struct AckRegistries { + logs: Option, + metrics: Option, + traces: Option, +} + +impl AckRegistries { + pub(crate) fn new( + logs: Option, + metrics: Option, + traces: Option, + ) -> Self { + Self { + logs, + metrics, + traces, + } + } + + pub(crate) fn ack_registry_for_signal(&self, signal: SignalType) -> Option<&AckRegistry> { + match signal { + SignalType::Logs => self.logs.as_ref(), + SignalType::Metrics => self.metrics.as_ref(), + SignalType::Traces => self.traces.as_ref(), + } + } +} + +/// Routes an Ack control message back into the appropriate registry. +pub(crate) fn route_ack_response(states: &AckRegistries, ack: AckMsg) -> RouteResponse { + let Some(token) = AckToken::from_calldata(&ack.calldata) else { + return RouteResponse::Invalid; + }; + states + .ack_registry_for_signal(ack.accepted.signal_type()) + .map(|state| state.complete(token, Ok(()))) + .unwrap_or(RouteResponse::None) +} + +/// Routes a Nack control message back into the appropriate registry. +pub(crate) fn route_nack_response( + states: &AckRegistries, + nack: NackMsg, +) -> RouteResponse { + let Some(token) = AckToken::from_calldata(&nack.calldata) else { + return RouteResponse::Invalid; + }; + states + .ack_registry_for_signal(nack.refused.signal_type()) + .map(|state| state.complete(token, Err(nack.reason))) + .unwrap_or(RouteResponse::None) +} + +/// Helper to produce the canonical success status used across signals. +pub(crate) fn success_status(batch_id: i64) -> BatchStatus { + BatchStatus { + batch_id, + status_code: ProtoStatusCode::Ok as i32, + status_message: "Successfully received".to_string(), + } +} + +/// Helper to produce a nack status with the provided reason. +pub(crate) fn nack_status(batch_id: i64, reason: String) -> BatchStatus { + BatchStatus { + batch_id, + status_code: ProtoStatusCode::Unavailable as i32, + status_message: format!("Pipeline processing failed: {reason}"), + } +} + +/// Helper to produce the status returned when the registry runs out of slots. +pub(crate) fn overloaded_status(batch_id: i64) -> BatchStatus { + BatchStatus { + batch_id, + status_code: ProtoStatusCode::Unavailable as i32, + status_message: "Pipeline processing failed: Too many concurrent requests".to_string(), + } +} diff --git a/rust/otap-dataflow/crates/otap/src/otel_receiver/architecture.md b/rust/otap-dataflow/crates/otap/src/otel_receiver/architecture.md new file mode 100644 index 0000000000..80dc55492f --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otel_receiver/architecture.md @@ -0,0 +1,176 @@ +# OTel Receiver Architecture + +## Overview and Objectives + +This module implements an experimental OpenTelemetry (OTEL) receiver that speaks: + +- OTAP Arrow over gRPC (streaming logs, metrics, traces) +- OTLP protobuf over gRPC (unary Export endpoints) + +It runs directly on top of `h2` without tonic, is designed to live in a thread-per-core engine, and integrates with the OTAP dataflow engine (pipeline, admission control, telemetry, acks). + +Primary objectives: + +1. Support OTLP and OTAP Arrow gRPC with minimal dependencies and tight control over performance. +2. Fit a single threaded, per core runtime (no `Send + Sync` futures, `Rc` instead of `Arc`). +3. Keep the hot path free of locks and shared atomics, and minimize heap allocations. +4. Provide explicit backpressure and bounded resource usage: + - Connection and stream admission control. + - Bounded in flight requests and ack slots. + - Bounded frame sizes and decompression output. + +## Design Principles + +- **Single threaded by design**: All request handling for a given core stays on that core. +- **Explicit backpressure**: + - `Admitter` gates TCP connections and h2 streams. + - `AckRegistry` is a bounded slab that limits outstanding wait slots. + - `StatusStream` enforces per connection `max_in_flight` limits. + - HTTP/2 flow control credits are returned only for bytes actually consumed. +- **No locks on the hot path**: + - Router, ack registries, encoder pools, decompressor, ... all live in single threaded context. + - Sharing uses `Rc` and `RefCell` only within that context. +- **Minimal allocations and reuse of buffers**: + - `GrpcStreamingBody` reuses a decompression scratch buffer. + - `ResponseEncoderPool` reuses encoders and compression buffers. + - `AckRegistry` preallocates all slots and uses an intrusive free list. +- **Fail fast on protocol or resource violations** + - Strict gRPC header validation. + - Rejection of unsupported compression codes. + - Bounded decoded message size with clear `RESOURCE_EXHAUSTED`. + - Handshake and request timeouts. + - Well defined error paths for admission failures and pipeline errors. + +## Architecture and Main Components + +### Top level receiver (otel_receiver.rs) + +- A **control loop** for control messages (shutdown, metrics collection, Ack/Nack routing). +- A **data plane loop** `run_grpc_server` to handle incoming connections. + +This split isolates cluster control from request serving and ensures clean shutdown behavior. + +### Data plane: `run_grpc_server` and `handle_tcp_conn` + +- `run_grpc_server`: + - Accept new TCP connections. + - One task per TCP connections (`spawn_local`). + - Applies admission via `Admitter::try_admit_connection` + - On cancellation or listener close, drains all connection tasks and exits. + +- `handle_tcp_conn`: + - Performs the HTTP/2 handshake with configured timeout. + - Applies admission via `tcp_conn_guard.try_open_stream`. + - Manages HTTP2 keep-alive. + - Limits number of in-flight streams. + +This is the main HTTP/2 server loop per TCP connection. + +### Request routing: `GrpcRequestRouter` + +- Ack registries per signal (optional, only if `wait_for_result`). +- `max_in_flight_per_connection`: per connection limit of in flight batches that wait for ack. +- Parses and negotiates gRPC encoding. +- Handles compression and timeout. +- Switches on `request.uri().path()`: + - OTAP Arrow streaming services (`ARROW_*`). + - OTLP unary Export services (`OTLP_*`). + - Unknown path: respond with `UNIMPLEMENTED`. +- `serve_otap_stream` (bidirectional OTAP Arrow endpoints). +- `serve_otlp_unary` (OTLP endpoints). + +### Ack tracking: `AckRegistry` and Ack flow + +- Maintains fixed-size slab of `AckSlot`s: + - Each slot has generation counter and state. + - States: + - `Free { next_free }` for intrusive free list. + - `Waiting(WaitingSlot)` with waker and outcome. +- Allocation: + - `allocate` pops from `head_free` list and moves slot into `Waiting`. + - O(1) with no heap allocation. + +### Batch status streaming: `StatusStream` + +Only used for OTAP. + +## Bounded resources and backpressure + +- **Connection and stream counts** through `Admitter`: + - Hard caps on number of TCP connections and per connection h2 streams. + - Rejection or soft drop behavior when the system is overloaded. +- **Ack wait slots**: + - `AckRegistry` has a fixed size slab of slots. + - On exhaustion, new work is rejected with an overloaded status. +- **Per connection in flight batches**: + - `StatusStream` uses `max_in_flight_per_connection` to prevent a single connection from consuming all ack slots. +- **HTTP/2 flow control**: + - `GrpcStreamingBody` tracks unacknowledged bytes and calls `release_capacity` only when bytes are actually consumed. + - Protects both ends from window exhaustion or accidental frame drops. +- **Frame and decompression limits**: + - `validate_frame_len` enforces `max_decoding_message_size`. + - Decompression paths check for output exceeding this limit and fail with `RESOURCE_EXHAUSTED`. + +## Timeout and keepalive guarantees + +- `RequestTimeout`: + - Enforces maximum idle time per request or per stream. + - Resets timer on each successful item or poll. + - Maps timeouts to `DEADLINE_EXCEEDED` status. +- `Http2Keepalive`: + - Sends HTTP/2 PING only when connection is idle and a configured interval has elapsed. + - Uses timeout to fail the connection if PONG is not received. + - Prevents silent half open connections. + +## Error handling + +- Protocol errors (wrong content type, unsupported compression, invalid headers) are surfaced as gRPC errors with log messages. +- Transport errors in the h2 layer result in connection closure and logged errors. +- Pipeline send errors: + - Cancel or complete any outstanding ack tokens. + - Close stream or respond unary error, depending on path. + +## StatusStream and in flight set + +- At most `max_in_flight` concurrent `AckWaitFuture` per stream. +- `fill_inflight` performs at most `max_in_flight` iterations per call, each constant time except for: + - Network reads in `next_message` (underlying cost is dominated by I/O). + - Pipeline send, which may yield. +- `InFlightSet::poll_next` + Delegates to `FuturesUnordered`, which is amortized O(1) per future over the lifetime of the stream. + +## Configurable Parameters + +- Network and HTTP/2: + - `listening_addr` + - `max_concurrent_streams` (per connection) + - `initial_stream_window_size` + - `initial_connection_window_size` + - `max_frame_size` + - `http2_handshake_timeout` + - `http2_keepalive_interval` + - `http2_keepalive_timeout` + +- gRPC behavior: + - `max_concurrent_requests` (used to size encoder pool and ack registries). + - `request_compression_methods()` (server allowed request encodings). + - `response_compression_methods()` (server allowed response encodings). + - `timeout` (per request idle timeout in `RequestTimeout`). + - `max_decoding_message_size` (per frame decoded size limit; default 64 MiB). + +- Admission control: + - `Admitter::new(100000, max_concurrent_streams or 100, 100000)` + Currently hard coded connection and backlog limits, with a note to make them tunable. + +- Ack registries: + - Size per signal is `settings.max_concurrent_requests`. + +- Per connection in flight limit: + - Derived by `per_connection_limit(&settings)` from gRPC settings. + +## Known Limitations and TODOs + +- Better metrics and logging/tracing +- Unix Domain Socket +- Snappy compression (supported by the Go Collector) +- OpenZL exploration diff --git a/rust/otap-dataflow/crates/otap/src/otel_receiver/encoder.rs b/rust/otap-dataflow/crates/otap/src/otel_receiver/encoder.rs new file mode 100644 index 0000000000..53f45d6e38 --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otel_receiver/encoder.rs @@ -0,0 +1,296 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! gRPC response frame encoder with optional compression. + +use crate::compression::CompressionMethod; +use crate::otel_receiver::grpc::{GrpcEncoding, MIN_COMPRESSED_CAPACITY}; +use crate::otel_receiver::status::Status; +use bytes::{BufMut, Bytes, BytesMut}; +use flate2::Compression; +use flate2::write::{GzEncoder, ZlibEncoder}; +use prost::Message; +use std::cell::RefCell; +use std::io::{self, Write}; +use std::mem; +use std::ops::{Deref, DerefMut}; +use zstd::bulk::Compressor as ZstdCompressor; + +#[cfg(feature = "unsafe-optimizations")] +#[inline] +fn set_vec_len(buf: &mut Vec, len: usize) { + // SAFETY: caller guarantees capacity and that bytes are overwritten before read. + #[allow(unsafe_code)] + unsafe { + buf.set_len(len) + } +} + +#[cfg(not(feature = "unsafe-optimizations"))] +#[inline] +fn set_vec_len(buf: &mut Vec, len: usize) { + if buf.len() != len { + buf.resize(len, 0); + } +} + +/// Per encoding pool of reusable gRPC response encoders. +/// +/// Encoders are stored in small vectors keyed by `GrpcEncoding`. This avoids +/// repeatedly allocating compression buffers on the response hot path. +pub(crate) struct ResponseEncoderPool { + inner: RefCell, +} + +/// RAII guard that returns a checked out encoder back to the pool on drop. +pub(crate) struct EncoderGuard<'a> { + encoder: Option, + pool: &'a ResponseEncoderPool, + encoding: GrpcEncoding, +} + +impl ResponseEncoderPool { + pub(crate) fn new(methods: &[CompressionMethod], target_encoders: usize) -> Self { + const MAX_ENCODERS: usize = 1024; + let pool_size = target_encoders.clamp(1, MAX_ENCODERS); + let mut slots = EncoderSlots { + identity: Vec::with_capacity(pool_size), + zstd: Vec::new(), + gzip: Vec::new(), + deflate: Vec::new(), + }; + + // Always seed identity encoder(s) since it is universally supported. + for _ in 0..pool_size { + slots + .identity + .push(GrpcResponseFrameEncoder::new(GrpcEncoding::Identity)); + } + + for method in methods { + let (vec, encoding) = match method { + CompressionMethod::Zstd => (&mut slots.zstd, GrpcEncoding::Zstd), + CompressionMethod::Gzip => (&mut slots.gzip, GrpcEncoding::Gzip), + CompressionMethod::Deflate => (&mut slots.deflate, GrpcEncoding::Deflate), + }; + if vec.is_empty() { + vec.reserve(pool_size); + } + for _ in vec.len()..pool_size { + vec.push(GrpcResponseFrameEncoder::new(encoding)); + } + } + Self { + inner: RefCell::new(slots), + } + } + + pub(crate) fn checkout(&self, encoding: GrpcEncoding) -> EncoderGuard<'_> { + let mut slots = self.inner.borrow_mut(); + let encoder = match encoding { + GrpcEncoding::Identity => slots.identity.pop(), + GrpcEncoding::Zstd => slots.zstd.pop(), + GrpcEncoding::Gzip => slots.gzip.pop(), + GrpcEncoding::Deflate => slots.deflate.pop(), + } + .unwrap_or_else(|| GrpcResponseFrameEncoder::new(encoding)); + + EncoderGuard { + encoder: Some(encoder), + pool: self, + encoding, + } + } +} + +impl<'a> Drop for EncoderGuard<'a> { + fn drop(&mut self) { + if let Some(encoder) = self.encoder.take() { + let mut slots = self.pool.inner.borrow_mut(); + match self.encoding { + GrpcEncoding::Identity => slots.identity.push(encoder), + GrpcEncoding::Zstd => slots.zstd.push(encoder), + GrpcEncoding::Gzip => slots.gzip.push(encoder), + GrpcEncoding::Deflate => slots.deflate.push(encoder), + } + } + } +} + +impl<'a> Deref for EncoderGuard<'a> { + type Target = GrpcResponseFrameEncoder; + + fn deref(&self) -> &Self::Target { + self.encoder.as_ref().expect("encoder should be present") + } +} + +impl<'a> DerefMut for EncoderGuard<'a> { + fn deref_mut(&mut self) -> &mut Self::Target { + self.encoder.as_mut().expect("encoder should be present") + } +} + +pub(crate) struct EncoderSlots { + identity: Vec, + zstd: Vec, + gzip: Vec, + deflate: Vec, +} + +/// Builds length prefixed gRPC response frames with optional compression. +pub(crate) struct GrpcResponseFrameEncoder { + compression: GrpcEncoding, + // Reusable buffer containing the result of the serialization of the response message. + message_buf: BytesMut, + // Reusable buffer for building the gRPC frame header + payload. + frame_buf: BytesMut, + compressed_buf: Vec, + zstd: Option>, +} + +impl GrpcResponseFrameEncoder { + pub(crate) fn new(compression: GrpcEncoding) -> Self { + Self { + compression, + frame_buf: BytesMut::with_capacity(512), + message_buf: BytesMut::with_capacity(512), + compressed_buf: Vec::new(), // By default gRPC responses are uncompressed + zstd: None, + } + } + + /// Serializes a protobuf message and wraps it in a gRPC frame. + pub(crate) fn encode(&mut self, message: &M) -> Result { + // Serialize the message into the reusable buffer. + self.message_buf.clear(); + message + .encode(&mut self.message_buf) + .map_err(|e| Status::internal(format!("failed to encode response: {e}")))?; + let uncompressed = self.message_buf.split().freeze(); + + // Compress & frame according to the negotiated encoding. + match self.compression { + GrpcEncoding::Identity => self.finish_frame(false, uncompressed.as_ref()), + GrpcEncoding::Zstd => { + self.compress_zstd(uncompressed.as_ref())?; + let mut payload = mem::take(&mut self.compressed_buf); + let result = self.finish_frame(true, payload.as_slice()); + payload.clear(); + self.compressed_buf = payload; + result + } + GrpcEncoding::Gzip => { + self.compress_gzip(uncompressed.as_ref())?; + let mut payload = mem::take(&mut self.compressed_buf); + let result = self.finish_frame(true, payload.as_slice()); + payload.clear(); + self.compressed_buf = payload; + result + } + GrpcEncoding::Deflate => { + self.compress_deflate(uncompressed.as_ref())?; + let mut payload = mem::take(&mut self.compressed_buf); + let result = self.finish_frame(true, payload.as_slice()); + payload.clear(); + self.compressed_buf = payload; + result + } + } + } + + /// Builds the 5-byte gRPC frame header plus payload. + fn finish_frame(&mut self, compressed: bool, payload: &[u8]) -> Result { + let needed = 5 + payload.len(); + if self.frame_buf.capacity() < needed { + self.frame_buf.reserve(needed - self.frame_buf.capacity()); + } + self.frame_buf.clear(); + self.frame_buf.put_u8(u8::from(compressed)); + self.frame_buf.put_u32(payload.len() as u32); + self.frame_buf.extend_from_slice(payload); + Ok(self.frame_buf.split().freeze()) + } + + /// Performs zstd compression into `compressed_buf`, growing as needed. + fn compress_zstd(&mut self, payload: &[u8]) -> Result<(), Status> { + self.ensure_zstd_encoder()?; + let mut required_capacity = payload.len().max(MIN_COMPRESSED_CAPACITY); + loop { + // Make sure the scratch buffer is large enough for the next attempt. + if self.compressed_buf.capacity() < required_capacity { + self.compressed_buf + .reserve(required_capacity - self.compressed_buf.capacity()); + } + set_vec_len(&mut self.compressed_buf, required_capacity); + let result = { + // Safe because `ensure_zstd_encoder` guarantees we have an encoder. + let encoder = self.zstd.as_mut().expect("zstd encoder must exist"); + // Compress directly into the reusable scratch buffer to avoid extra allocations. + encoder.compress_to_buffer(payload, self.compressed_buf.as_mut_slice()) + }; + match result { + Ok(written) => { + // Shrink to the actual size once compression finishes successfully. + self.compressed_buf.truncate(written); + return Ok(()); + } + Err(err) + if err.kind() == io::ErrorKind::Other + && err.to_string().contains("Destination buffer is too small") => + { + // Double the capacity and retry when the destination buffer was insufficient. + required_capacity = required_capacity.checked_mul(2).ok_or_else(|| { + Status::internal("zstd compression failed: output too large") + })?; + } + Err(err) => { + // Any other compression failure aborts this response frame. + return Err(Status::internal(format!("zstd compression failed: {err}"))); + } + } + } + } + + /// Compresses with gzip into the scratch buffer. + fn compress_gzip(&mut self, payload: &[u8]) -> Result<(), Status> { + self.compressed_buf.clear(); + { + let mut encoder = GzEncoder::new(&mut self.compressed_buf, Compression::default()); + encoder + .write_all(payload) + .and_then(|_| encoder.try_finish()) + .map_err(|err| Status::internal(format!("gzip compression failed: {err}")))?; + } + Ok(()) + } + + /// Compresses with deflate into the scratch buffer. + fn compress_deflate(&mut self, payload: &[u8]) -> Result<(), Status> { + self.compressed_buf.clear(); + { + let mut encoder = ZlibEncoder::new(&mut self.compressed_buf, Compression::default()); + encoder + .write_all(payload) + .and_then(|_| encoder.try_finish()) + .map_err(|err| Status::internal(format!("deflate compression failed: {err}")))?; + } + Ok(()) + } + + /// Lazily creates the zstd encoder. + fn ensure_zstd_encoder(&mut self) -> Result<(), Status> { + if self.zstd.is_some() { + return Ok(()); + } + match ZstdCompressor::new(0) { + Ok(encoder) => { + self.zstd = Some(encoder); + Ok(()) + } + Err(err) => Err(Status::internal(format!( + "failed to initialize zstd compressor: {err}" + ))), + } + } +} diff --git a/rust/otap-dataflow/crates/otap/src/otel_receiver/grpc.rs b/rust/otap-dataflow/crates/otap/src/otel_receiver/grpc.rs new file mode 100644 index 0000000000..27b1d8acc1 --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otel_receiver/grpc.rs @@ -0,0 +1,1418 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Low-level gRPC/H2 mechanics/helpers for the experimental OTAP receiver. +//! +//! This module takes care of: +//! - negotiating compression, +//! - decoding/encoding length-prefixed frames, +//! - managing request time limits, +//! - abstracting the underlying `RecvStream` so it can be fuzzed in tests. +//! +//! Note: The implementation is heavily inspired by tonic's framing and compression stack, but +//! tailored to the single-threaded OTAP runtime. + +use crate::compression::CompressionMethod; +use crate::otel_receiver::GrpcRequestRouter; +use crate::otel_receiver::status::Status; +use async_trait::async_trait; +use bytes::{Buf, Bytes, BytesMut}; +use flate2::read::{GzDecoder, ZlibDecoder}; +use futures::{Stream, StreamExt}; +use http::{HeaderMap, HeaderValue}; +use otap_df_pdata::proto::opentelemetry::arrow::v1::BatchArrowRecords; +use prost::Message; +use std::collections::VecDeque; +use std::future::Future; +use std::io::{self, Write}; +use std::pin::Pin; +use std::rc::Rc; +use std::task::{Context, Poll}; +use std::time::Duration; +use tokio::time::{Instant as TokioInstant, Sleep, sleep}; +use zstd::bulk::Decompressor; + +/// Floor for compressed buffer allocations to avoid tiny vec growth. +pub(crate) const MIN_COMPRESSED_CAPACITY: usize = 8 * 1024; + +#[cfg(feature = "unsafe-optimizations")] +#[inline] +fn set_bytes_len(buf: &mut BytesMut, len: usize) { + if buf.capacity() < len { + buf.reserve(len - buf.capacity()); + } + // SAFETY: caller ensures bytes are fully overwritten before use. + #[allow(unsafe_code)] + unsafe { + buf.set_len(len) + } +} + +#[cfg(not(feature = "unsafe-optimizations"))] +#[inline] +fn set_bytes_len(buf: &mut BytesMut, len: usize) { + if buf.capacity() < len { + buf.reserve(len - buf.capacity()); + } + buf.resize(len, 0); +} + +/// Parses the client's `grpc-encoding` header and enforces server policy. +/// +/// Note: Non-UTF8 headers, unknown tokens, or disabled algorithms all yield a gRPC `unimplemented` +/// status so clients get immediate, well-scoped errors instead of silently falling back. +pub(crate) fn parse_grpc_encoding( + headers: &HeaderMap, + accepted: &AcceptedGrpcEncodings, +) -> Result { + // The method first validates that the `content-type` header begins with `application/grpc`. + // This keeps HTTP/2 requests that merely mimic the header names from being accepted. + match headers.get(http::header::CONTENT_TYPE) { + Some(value) if value.as_bytes().starts_with(b"application/grpc") => {} + other => { + log::error!("Rejecting stream due to invalid content-type: {other:?}"); + return Err(Status::invalid_argument( + "missing application/grpc content-type", + )); + } + } + + // Only encodings explicitly advertised in `AcceptedGrpcEncodings` are permitted, even though + // the parser understands additional aliases (such as `zstdarrow{n}`), the request is rejected + // unless the server opted in to the corresponding [`CompressionMethod`]. + match headers.get("grpc-encoding") { + None => Ok(GrpcEncoding::Identity), + Some(value) => { + let raw = value.to_str().map_err(|_| { + log::error!("Non-UTF8 grpc-encoding header"); + Status::invalid_argument("invalid grpc-encoding header") + })?; + let trimmed = raw.trim(); + let ascii = trimmed.as_bytes(); + const PREFIX: &[u8] = b"zstdarrow"; + + let encoding = if ascii.is_empty() || eq_ascii_case_insensitive(ascii, b"identity") { + GrpcEncoding::Identity + } else if eq_ascii_case_insensitive(ascii, b"zstd") { + GrpcEncoding::Zstd + } else if eq_ascii_case_insensitive(ascii, b"gzip") { + GrpcEncoding::Gzip + } else if eq_ascii_case_insensitive(ascii, b"deflate") { + GrpcEncoding::Deflate + } else if ascii.len() >= PREFIX.len() + && starts_with_ascii_case_insensitive(ascii, PREFIX) + { + let tail = &ascii[PREFIX.len()..]; + if tail.len() == 1 && tail[0].is_ascii_digit() { + GrpcEncoding::Zstd + } else { + log::error!("Unsupported grpc-encoding {}", trimmed); + return Err(Status::unimplemented("grpc compression not supported")); + } + } else { + log::error!("Unsupported grpc-encoding {}", trimmed); + return Err(Status::unimplemented("grpc compression not supported")); + }; + + if accepted.allows(encoding) { + Ok(encoding) + } else { + log::error!( + "grpc-encoding {} not enabled in server configuration", + trimmed + ); + Err(Status::unimplemented("grpc compression not supported")) + } + } + } +} + +/// Returns true when two ASCII byte slices are equal ignoring case (without allocating or +/// converting to UTF-8). +fn eq_ascii_case_insensitive(value: &[u8], expected: &[u8]) -> bool { + value.len() == expected.len() + && value + .iter() + .zip(expected) + .all(|(lhs, rhs)| ascii_byte_eq_ignore_case(*lhs, *rhs)) +} + +/// Returns true if `value` starts with `prefix`, ignoring ASCII case (without allocating or +/// converting to UTF-8). +fn starts_with_ascii_case_insensitive(value: &[u8], prefix: &[u8]) -> bool { + value.len() >= prefix.len() + && value + .iter() + .zip(prefix) + .all(|(lhs, rhs)| ascii_byte_eq_ignore_case(*lhs, *rhs)) +} + +/// Compares two ASCII bytes without allocating or converting to UTF-8. +fn ascii_byte_eq_ignore_case(lhs: u8, rhs: u8) -> bool { + lhs == rhs || lhs.eq_ignore_ascii_case(&rhs) +} + +/// Parses the client's `grpc-accept-encoding` header into capability flags. +pub(crate) fn parse_grpc_accept_encoding(headers: &HeaderMap) -> ClientAcceptEncodings { + let Some(value) = headers.get("grpc-accept-encoding") else { + return ClientAcceptEncodings::identity_only(); + }; + let raw = match value.to_str() { + Ok(raw) => raw, + Err(_) => return ClientAcceptEncodings::identity_only(), + }; + + let mut encodings = ClientAcceptEncodings { + identity: false, + zstd: false, + gzip: false, + deflate: false, + }; + let mut recognized = false; + + for token in raw.split(',') { + let trimmed = token.trim(); + if trimmed.is_empty() { + continue; + } + let ascii = trimmed.as_bytes(); + if eq_ascii_case_insensitive(ascii, b"identity") { + encodings.identity = true; + recognized = true; + } else if eq_ascii_case_insensitive(ascii, b"zstd") { + encodings.zstd = true; + recognized = true; + } else if eq_ascii_case_insensitive(ascii, b"gzip") { + encodings.gzip = true; + recognized = true; + } else if eq_ascii_case_insensitive(ascii, b"deflate") { + encodings.deflate = true; + recognized = true; + } + } + + if recognized { + encodings + } else { + ClientAcceptEncodings::identity_only() + } +} + +/// Chooses the response encoding based on server preference & client support. +/// +/// The caller provides the ordered list of server-supported compression methods and the client's +/// advertised capabilities. The function walks the server list in order, returning the first method +/// that the client also supports. This gives the server deterministic control over preference +/// ordering (e.g. pick `zstd` when available, otherwise fall back to `gzip`, etc.) while still +/// honoring the client's declared limits. When no overlap exists the function returns +/// `GrpcEncoding::Identity`, signaling that the response must be sent uncompressed. +pub(crate) fn negotiate_response_encoding( + configured: &[CompressionMethod], + client: &ClientAcceptEncodings, +) -> GrpcEncoding { + for method in configured { + if client.supports(*method) { + return match method { + CompressionMethod::Zstd => GrpcEncoding::Zstd, + CompressionMethod::Gzip => GrpcEncoding::Gzip, + CompressionMethod::Deflate => GrpcEncoding::Deflate, + }; + } + } + GrpcEncoding::Identity +} + +#[derive(Clone, Copy, Debug)] +/// Supported compression algorithms for the OTAP receiver responses. +pub enum GrpcEncoding { + /// No compression. + Identity, + /// Zstd compression. + Zstd, + /// Gzip compression. + Gzip, + /// Deflate compression. + Deflate, + // ToDo Add support for Snappy to follow Go implementation + // ToDo Add support for OpenZL in the future +} + +#[derive(Clone, Copy)] +/// Bit-mask indicating which compression methods the server allows. +pub(crate) struct AcceptedGrpcEncodings { + /// Whether server allows zstd for requests. + zstd: bool, + /// Whether server allows gzip for requests. + gzip: bool, + /// Whether server allows deflate for requests. + deflate: bool, +} + +impl AcceptedGrpcEncodings { + pub(crate) fn from_methods(methods: &[CompressionMethod]) -> Self { + let mut encodings = Self { + zstd: false, + gzip: false, + deflate: false, + }; + + for method in methods { + match method { + CompressionMethod::Zstd => encodings.zstd = true, + CompressionMethod::Gzip => encodings.gzip = true, + CompressionMethod::Deflate => encodings.deflate = true, + } + } + + encodings + } + + fn allows(self, encoding: GrpcEncoding) -> bool { + match encoding { + GrpcEncoding::Identity => true, + GrpcEncoding::Zstd => self.zstd, + GrpcEncoding::Gzip => self.gzip, + GrpcEncoding::Deflate => self.deflate, + } + } +} + +#[derive(Clone, Copy)] +/// Parsed view of the client's `grpc-accept-encoding` preference list. +pub(crate) struct ClientAcceptEncodings { + /// Client is willing to accept identity/not-compressed responses. + pub(crate) identity: bool, + /// Client advertised support for zstd responses. + pub(crate) zstd: bool, + /// Client advertised support for gzip responses. + pub(crate) gzip: bool, + /// Client advertised support for deflate responses. + pub(crate) deflate: bool, +} + +impl ClientAcceptEncodings { + fn identity_only() -> Self { + Self { + identity: true, + zstd: false, + gzip: false, + deflate: false, + } + } + + fn supports(self, method: CompressionMethod) -> bool { + match method { + CompressionMethod::Zstd => self.zstd, + CompressionMethod::Gzip => self.gzip, + CompressionMethod::Deflate => self.deflate, + } + } +} + +/// Produces the `grpc-accept-encoding` header to advertise server support. +pub(crate) fn build_accept_encoding_header(methods: &[CompressionMethod]) -> HeaderValue { + let mut tokens = Vec::with_capacity(methods.len() + 1); + for method in methods { + tokens.push(compression_method_token(*method)); + } + // `identity` is always supported but least preferred. + tokens.push("identity"); + let joined = tokens.join(","); + HeaderValue::from_str(&joined).unwrap_or_else(|_| HeaderValue::from_static("identity")) +} + +fn compression_method_token(method: CompressionMethod) -> &'static str { + match method { + CompressionMethod::Zstd => "zstd", + CompressionMethod::Gzip => "gzip", + CompressionMethod::Deflate => "deflate", + } +} + +pub(crate) fn grpc_encoding_token(encoding: GrpcEncoding) -> Option<&'static str> { + match encoding { + GrpcEncoding::Identity => None, + GrpcEncoding::Zstd => Some("zstd"), + GrpcEncoding::Gzip => Some("gzip"), + GrpcEncoding::Deflate => Some("deflate"), + } +} + +pub(crate) type BodyStreamError = String; + +/// Abstraction over the inbound h2 data stream so tests can inject fakes. +#[async_trait(?Send)] +pub(crate) trait BodyStream { + async fn next_chunk(&mut self) -> Option>; + fn release_capacity(&mut self, released: usize) -> Result<(), BodyStreamError>; +} + +pub(crate) struct H2BodyStream { + inner: h2::RecvStream, +} + +impl H2BodyStream { + pub(crate) fn new(inner: h2::RecvStream) -> Self { + Self { inner } + } +} + +#[async_trait(?Send)] +impl BodyStream for H2BodyStream { + /// Pulls the next DATA frame chunk from h2. + async fn next_chunk(&mut self) -> Option> { + self.inner + .data() + .await + .map(|res| res.map_err(|err| err.to_string())) + } + + /// Returns flow-control credits back to the peer. + fn release_capacity(&mut self, released: usize) -> Result<(), BodyStreamError> { + self.inner + .flow_control() + .release_capacity(released) + .map_err(|err| err.to_string()) + } +} + +/// Pull based view over an h2 stream that yields decoded gRPC frames. +/// +/// The body: +/// - reassembles length prefixed frames from arbitrary DATA chunking, +/// - handles optional request compression, and +/// - exposes both decoded Arrow payloads (`next_message`) and raw bytes +/// (`next_message_bytes`) for the OTLP unary path. +/// +/// It is intentionally `!Send` so it can share a zstd decompressor through +/// the router on the local executor. +pub(crate) struct GrpcStreamingBody { + recv: S, + buffer: ChunkBuffer, + current_frame: Option, + finished: bool, + encoding: GrpcEncoding, + /// Shared reference to router so we can reach the pooled decompressor + router: Rc, + decompressed_buf: BytesMut, + /// Maximum decoded message size in bytes (from GrpcServerSettings::max_decoding_message_size) + max_decoding_message_size: usize, + // How many bytes we've read from h2 but not yet released. + unacked_bytes: usize, +} + +#[derive(Clone, Copy)] +/// Metadata describing the next gRPC frame. +struct FrameHeader { + length: usize, + compressed: bool, +} + +/// Simple queue of received bytes backing the gRPC frame decoder. +struct ChunkBuffer { + chunks: VecDeque, + len: usize, +} + +impl ChunkBuffer { + /// Creates an empty buffer that tracks total buffered length. + fn new() -> Self { + Self { + chunks: VecDeque::new(), + len: 0, + } + } + + /// Returns the number of bytes buffered across all chunks. + fn len(&self) -> usize { + self.len + } + + /// Appends a chunk to the tail of the buffer without copying. + fn push(&mut self, chunk: Bytes) { + if chunk.is_empty() { + return; + } + self.len += chunk.len(); + self.chunks.push_back(chunk); + } + + /// Splits off `size` bytes from the front of the buffer. O(size) due to copying references. + fn split_frame(&mut self, size: usize) -> Option { + if size > self.len { + return None; + } + if size == 0 { + return Some(FrameBuf::new(VecDeque::new(), 0)); + } + + let mut needed = size; + let mut parts = VecDeque::new(); + while needed > 0 { + let mut chunk = self.chunks.pop_front()?; + if chunk.len() > needed { + let part = chunk.split_to(needed); + self.len -= needed; + parts.push_back(part); + self.chunks.push_front(chunk); + needed = 0; + } else { + needed -= chunk.len(); + self.len -= chunk.len(); + parts.push_back(chunk); + } + } + Some(FrameBuf::new(parts, size)) + } +} + +/// Thin Buf impl over a small deque of byte chunks. +struct FrameBuf { + chunks: VecDeque, + remaining: usize, +} + +impl FrameBuf { + /// Wraps the supplied deque and remaining length into a `FrameBuf`. + fn new(chunks: VecDeque, remaining: usize) -> Self { + Self { chunks, remaining } + } + + /// Converts the buffered slices into a single `Bytes`, coalescing if needed. + fn into_bytes(mut self) -> Bytes { + match self.chunks.len() { + 0 => Bytes::new(), + 1 => self + .chunks + .pop_front() + .expect("frame buffer length mismatch"), + _ => { + let mut buf = BytesMut::with_capacity(self.remaining); + while let Some(chunk) = self.chunks.pop_front() { + buf.extend_from_slice(&chunk); + } + buf.freeze() + } + } + } +} + +impl Buf for FrameBuf { + fn remaining(&self) -> usize { + self.remaining + } + + fn chunk(&self) -> &[u8] { + self.chunks + .front() + .map(|bytes| bytes.as_ref()) + .unwrap_or(&[]) + } + + fn advance(&mut self, mut cnt: usize) { + assert!(cnt <= self.remaining); + self.remaining -= cnt; + while cnt > 0 { + let Some(front_len) = self.chunks.front().map(|b| b.len()) else { + break; + }; + if cnt < front_len { + if let Some(front) = self.chunks.front_mut() { + front.advance(cnt); + } + break; + } else { + cnt -= front_len; + let _ = self.chunks.pop_front(); + } + } + } +} + +impl GrpcStreamingBody { + pub(crate) fn new( + recv: h2::RecvStream, + encoding: GrpcEncoding, + router: Rc, + max_decoding_message_size: usize, + ) -> Self { + Self::with_stream( + H2BodyStream::new(recv), + encoding, + router, + max_decoding_message_size, + ) + } +} + +impl GrpcStreamingBody +where + S: BodyStream, +{ + fn with_stream( + recv: S, + encoding: GrpcEncoding, + router: Rc, + max_decoding_message_size: usize, + ) -> Self { + Self { + recv, + buffer: ChunkBuffer::new(), + current_frame: None, + finished: false, + encoding, + router, + decompressed_buf: BytesMut::with_capacity(128 * 1024), + max_decoding_message_size, + unacked_bytes: 0, + } + } + + /// Pulls the next chunk from the underlying transport into our buffer. + /// Complexity: O(1) push per chunk. Because we run on the single-threaded + /// runtime, `self.finished` is only toggled here. + async fn fill_buffer(&mut self) -> Result<(), Status> { + if self.finished { + return Ok(()); + } + + match self.recv.next_chunk().await { + Some(Ok(bytes)) => { + let chunk_len = bytes.len(); + + // We DO NOT release capacity here. + // We only track how much we must release later when consumed. + self.unacked_bytes = self.unacked_bytes.saturating_add(chunk_len); + self.buffer.push(bytes); + Ok(()) + } + Some(Err(err)) => Err(Status::internal(format!("stream error: {err}"))), + None => { + self.finished = true; + Ok(()) + } + } + } + + fn validate_frame_len(&self, len: usize) -> Result { + if len > self.max_decoding_message_size { + log::warn!("Rejecting gRPC frame of length {}", len); + Err(Status::resource_exhausted("gRPC frame too large")) + } else { + Ok(len) + } + } + + /// Returns flow-control credits back to the peer for bytes that were + /// actually consumed (header or payload). + /// + /// This enforces proper HTTP/2 backpressure: + /// - We do NOT release capacity when DATA chunks arrive + /// - We release only when those bytes have been removed from `ChunkBuffer` + fn release_consumed_capacity(&mut self, consumed: usize) { + if consumed == 0 || self.unacked_bytes == 0 { + return; + } + + // Only release as many bytes as we have outstanding. + let to_release = consumed.min(self.unacked_bytes); + self.unacked_bytes -= to_release; + + if to_release > 0 { + if let Err(err) = self.recv.release_capacity(to_release) { + log::debug!("release_capacity failed: {err}"); + } + } + } + + /// Reassembles the next gRPC frame payload, including the compression flag bit. + async fn next_payload(&mut self) -> Result, Status> { + loop { + if self.current_frame.is_none() { + if self.buffer.len() < 5 { + if self.finished { + return Ok(None); + } + self.fill_buffer().await?; + continue; + } + + let header = self + .buffer + .split_frame(5) + .expect("buffer len checked above") + .into_bytes(); + // We consumed 5 header bytes, account that in flow control + self.release_consumed_capacity(5); + + let compressed = header[0] == 1; + let len_u32 = u32::from_be_bytes([header[1], header[2], header[3], header[4]]); + let len = self.validate_frame_len(len_u32 as usize)?; + + self.current_frame = Some(FrameHeader { + length: len, + compressed, + }); + } + + if let Some(header) = self.current_frame.take() { + if self.buffer.len() < header.length { + if self.finished { + log::error!("Stream ended before full gRPC frame was received"); + // Drop any remaining buffered bytes from flow control point of view + self.release_consumed_capacity(self.unacked_bytes); + return Err(Status::internal("truncated gRPC frame")); + } + self.fill_buffer().await?; + self.current_frame = Some(header); + continue; + } + + let buf = self + .buffer + .split_frame(header.length) + .expect("buffer len checked above"); + + // Mark the payload as consumed for flow control + self.release_consumed_capacity(header.length); + let payload = buf.into_bytes(); + return Ok(Some((header.compressed, payload))); + } + } + } + + /// Makes sure the scratch buffer is large enough for the decoded payload. + /// Includes heuristics to shrink the buffer if it remains excessively large + /// for small payloads, preventing memory leaks in long-lived connections. + fn reserve_decompressed_capacity(&mut self, payload_len: usize) { + let current_capacity = self.decompressed_buf.capacity(); + let target = payload_len.min(self.max_decoding_message_size); + + // 1. Growth Path (Hot Path) + // If we need more space, grow exponentially to amortize allocation costs. + if current_capacity < target { + let required = target + .saturating_sub(current_capacity) + .max(current_capacity.min(self.max_decoding_message_size)); + self.decompressed_buf.reserve(required); + return; + } + + // 2. Shrink Path (Cold Path - Optimization) + // Only trigger if: + // - The buffer is empty. + // - We are holding a large amount of memory (e.g. > 4MB). + // - The current requirement is tiny compared to capacity (e.g. < 1/8th). + const EXCESSIVE_MEMORY_THRESHOLD: usize = 4 * 1024 * 1024; // 4MB + + if self.decompressed_buf.is_empty() + && current_capacity > EXCESSIVE_MEMORY_THRESHOLD + && payload_len < (current_capacity / 8) + { + // Shrink to a "Safe Baseline" (e.g. 1MB). This prevents thrashing + // if the traffic fluctuates between 10KB and 800KB. + const BASELINE_CAPACITY: usize = 1024 * 1024; // 1MB + + // We allocate a new buffer. The old one is dropped, releasing memory to the OS/Allocator. + // We take the max of payload_len and Baseline to ensure we cover the current packet. + let new_capacity = std::cmp::max(payload_len, BASELINE_CAPACITY); + + // Only proceed if we are actually saving significant memory + if new_capacity < (current_capacity / 2) { + self.decompressed_buf = BytesMut::with_capacity(new_capacity); + } + } + } + + /// Dispatches to the appropriate decompressor for the negotiated encoding. + fn decompress(&mut self, payload: Bytes) -> Result { + match self.encoding { + GrpcEncoding::Identity => { + log::error!("Received compressed frame but grpc-encoding=identity"); + Err(Status::unimplemented("message compression not negotiated")) + } + GrpcEncoding::Zstd => self.decompress_zstd(payload), + GrpcEncoding::Gzip => self.decompress_gzip(payload), + GrpcEncoding::Deflate => self.decompress_deflate(payload), + } + } + + /// Performs a zstd decode, growing the buffer as needed. + /// Complexity: amortized O(n) over the payload size because each retry doubles + /// the buffer. + fn decompress_zstd(&mut self, payload: Bytes) -> Result { + self.reserve_decompressed_capacity(payload.len()); + let mut required_capacity = self + .decompressed_buf + .capacity() + .min(self.max_decoding_message_size); + + loop { + set_bytes_len(&mut self.decompressed_buf, required_capacity); + let buffer = &mut self.decompressed_buf[..]; + + let mut slot = self.router.zstd_decompressor.borrow_mut(); + let mut decompressor = slot.take().unwrap_or_else(|| { + Decompressor::new().expect("failed to initialize zstd decompressor") + }); + drop(slot); + + let result = decompressor.decompress_to_buffer(payload.as_ref(), buffer); + // Return the decompressor to the pool for reuse. + *self.router.zstd_decompressor.borrow_mut() = Some(decompressor); + + match result { + Ok(written) => { + if written > self.max_decoding_message_size { + return Err(Status::resource_exhausted("decompressed message too large")); + } + self.decompressed_buf.truncate(written); + return Ok(self.decompressed_buf.split().freeze()); + } + Err(err) + if err.kind() == io::ErrorKind::Other + && err.to_string().contains("Destination buffer is too small") => + { + let next = required_capacity.checked_mul(2).ok_or_else(|| { + Status::internal("zstd decompression failed: output too large") + })?; + if next > self.max_decoding_message_size { + return Err(Status::resource_exhausted("decompressed message too large")); + } + required_capacity = next; + } + Err(err) => { + return Err(Status::internal(format!( + "zstd decompression failed: {err}" + ))); + } + } + } + } + + /// Performs a gzip inflate into the scratch buffer via streaming decoder. + fn decompress_gzip(&mut self, payload: Bytes) -> Result { + self.reserve_decompressed_capacity(payload.len()); + self.decompressed_buf.clear(); + let mut decoder = GzDecoder::new(payload.as_ref()); + let mut writer = + BytesMutWriter::new(&mut self.decompressed_buf, self.max_decoding_message_size); + _ = io::copy(&mut decoder, &mut writer).map_err(|err| { + log::error!("gzip decompression failed: {err}"); + Status::internal(format!("gzip decompression failed: {err}")) + })?; + Ok(self.decompressed_buf.split().freeze()) + } + + /// Performs a deflate inflate into the scratch buffer via streaming decoder. + fn decompress_deflate(&mut self, payload: Bytes) -> Result { + self.reserve_decompressed_capacity(payload.len()); + self.decompressed_buf.clear(); + let mut decoder = ZlibDecoder::new(payload.as_ref()); + let mut writer = + BytesMutWriter::new(&mut self.decompressed_buf, self.max_decoding_message_size); + _ = io::copy(&mut decoder, &mut writer).map_err(|err| { + log::error!("deflate decompression failed: {err}"); + Status::internal(format!("deflate decompression failed: {err}")) + })?; + Ok(self.decompressed_buf.split().freeze()) + } +} + +struct BytesMutWriter<'a> { + buffer: &'a mut BytesMut, + max: usize, +} + +impl<'a> BytesMutWriter<'a> { + fn new(buffer: &'a mut BytesMut, max: usize) -> Self { + Self { buffer, max } + } +} + +impl Write for BytesMutWriter<'_> { + fn write(&mut self, buf: &[u8]) -> io::Result { + let remaining = self.max.saturating_sub(self.buffer.len()); + if remaining == 0 { + return Err(io::Error::other("decompressed payload exceeds limit")); + } + let to_write = remaining.min(buf.len()); + self.buffer.extend_from_slice(&buf[..to_write]); + if to_write < buf.len() { + return Err(io::Error::other("decompressed payload exceeds limit")); + } + Ok(to_write) + } + + fn flush(&mut self) -> io::Result<()> { + Ok(()) + } +} + +/// Minimal async interface that lets the experimental receiver consume Arrow batches without +/// requiring the underlying stream to be `Send`. It mirrors the subset of `ArrowRequestStream` +/// used by this module so tests can inject local fakes. +#[async_trait(?Send)] +pub(crate) trait RequestStream { + /// Fetches the next `BatchArrowRecords`, or `Ok(None)` when the peer half closes the stream. + /// Implementations translate transport/protobuf failures into gRPC `Status` errors. + async fn next_message(&mut self) -> Result, Status>; +} + +#[async_trait(?Send)] +impl RequestStream for GrpcStreamingBody +where + S: BodyStream + Unpin + 'static, +{ + /// Reassembles length-prefixed gRPC frames, handling optional compression, and yields decoded + /// `BatchArrowRecords` to the caller. + async fn next_message(&mut self) -> Result, Status> { + let Some((compressed, payload)) = self.next_payload().await? else { + return Ok(None); + }; + + let decoded = if compressed { + let bytes = self.decompress(payload)?; + BatchArrowRecords::decode(bytes) + } else { + BatchArrowRecords::decode(payload) + }; + + // Surface decoding failures as gRPC errors so clients know the batch was invalid. + let message = decoded.map_err(|e| { + log::error!("Failed to decode BatchArrowRecords: {e}"); + Status::invalid_argument(format!("failed to decode BatchArrowRecords: {e}")) + })?; + Ok(Some(message)) + } +} + +impl GrpcStreamingBody +where + S: BodyStream + Unpin + 'static, +{ + /// Returns the next raw gRPC frame payload. Compressed messages reuse the decompression + /// scratch buffer and hand back a `Bytes` view without an extra copy. + pub(crate) async fn next_message_bytes(&mut self) -> Result, Status> { + let Some((compressed, payload)) = self.next_payload().await? else { + return Ok(None); + }; + if compressed { + let bytes = self.decompress(payload)?; + Ok(Some(bytes)) + } else { + Ok(Some(payload)) + } + } +} + +/// Utility wrapper that enforces per request idle deadlines. +/// +/// A timeout is only armed when a duration is configured. On each successful +/// poll of the wrapped stream or future the timer is reset. If the timer +/// elapses first, the operation fails with `Err(())` and the caller maps that +/// to `DEADLINE_EXCEEDED`. +pub(crate) struct RequestTimeout { + duration: Option, + sleep: Option>>, +} + +impl RequestTimeout { + pub(crate) fn new(duration: Option) -> Self { + Self { + duration, + sleep: None, + } + } + + pub(crate) async fn next_with(&mut self, stream: &mut S) -> Result, ()> + where + S: Stream + Unpin, + { + futures::future::poll_fn(|cx| self.poll_next_with(cx, stream)).await + } + + /// Awaits the provided future while enforcing the configured timeout. + pub(crate) async fn with_future(&mut self, future: F) -> Result + where + F: Future, + { + futures::pin_mut!(future); + futures::future::poll_fn(|cx| self.poll_with_future(cx, future.as_mut())).await + } + + pub(crate) fn poll_next_with( + &mut self, + cx: &mut Context<'_>, + stream: &mut S, + ) -> Poll, ()>> + where + S: Stream + Unpin, + { + if self.duration.is_none() { + return StreamExt::poll_next_unpin(stream, cx).map(Ok); + } + + self.ensure_sleep(); + if let Some(sleep) = self.sleep.as_mut() { + if sleep.as_mut().poll(cx).is_ready() { + return Poll::Ready(Err(())); + } + } + + match StreamExt::poll_next_unpin(stream, cx) { + Poll::Ready(item) => { + if let (Some(duration), Some(sleep)) = (self.duration, self.sleep.as_mut()) { + sleep.as_mut().reset(TokioInstant::now() + duration); + } + Poll::Ready(Ok(item)) + } + Poll::Pending => Poll::Pending, + } + } + + pub(crate) fn poll_with_future( + &mut self, + cx: &mut Context<'_>, + mut future: Pin<&mut impl Future>, + ) -> Poll> { + if self.duration.is_none() { + return future.as_mut().poll(cx).map(Ok); + } + + self.ensure_sleep(); + if let Some(sleep) = self.sleep.as_mut() { + if sleep.as_mut().poll(cx).is_ready() { + return Poll::Ready(Err(())); + } + } + + match future.as_mut().poll(cx) { + Poll::Ready(out) => { + if let (Some(duration), Some(sleep)) = (self.duration, self.sleep.as_mut()) { + sleep.as_mut().reset(TokioInstant::now() + duration); + } + Poll::Ready(Ok(out)) + } + Poll::Pending => Poll::Pending, + } + } + + fn ensure_sleep(&mut self) { + if let Some(duration) = self.duration { + if self.sleep.is_none() { + self.sleep = Some(Box::pin(sleep(duration))); + } + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::compression::CompressionMethod; + use crate::otel_receiver::DEFAULT_MAX_DECODING_MESSAGE_SIZE; + use crate::otel_receiver::encoder::{GrpcResponseFrameEncoder, ResponseEncoderPool}; + use crate::otel_receiver::response_templates::ResponseTemplates; + use crate::pdata::OtapPdata; + use async_trait::async_trait; + use bytes::{BufMut, Bytes, BytesMut}; + use flate2::Compression; + use flate2::read::{GzDecoder, ZlibDecoder}; + use flate2::write::{GzEncoder, ZlibEncoder}; + use http::{HeaderMap, HeaderValue}; + use otap_df_engine::control::pipeline_ctrl_msg_channel; + use otap_df_engine::local::receiver::EffectHandler as LocalEffectHandler; + use otap_df_engine::node::NodeId; + use otap_df_pdata::proto::opentelemetry::arrow::v1::{BatchArrowRecords, BatchStatus}; + use otap_df_telemetry::reporter::MetricsReporter; + use rand::rngs::StdRng; + use rand::{Rng, SeedableRng}; + use std::cell::RefCell; + use std::collections::{HashMap, VecDeque}; + use std::io::{Read, Write}; + use std::rc::Rc; + use std::sync::{Arc, Mutex}; + use tokio::sync::mpsc; + use tokio::task::yield_now; + use tokio::time::Duration; + use tokio_stream::wrappers::UnboundedReceiverStream; + use zstd::bulk::Compressor as ZstdCompressor; + + fn base_headers() -> HeaderMap { + let mut headers = HeaderMap::new(); + let _ = headers.insert( + http::header::CONTENT_TYPE, + HeaderValue::from_static("application/grpc"), + ); + headers + } + + #[test] + fn test_parse_grpc_encoding_variants() { + let accepted = AcceptedGrpcEncodings::from_methods(&[ + CompressionMethod::Zstd, + CompressionMethod::Gzip, + ]); + let mut headers = base_headers(); + let _ = headers.insert("grpc-encoding", HeaderValue::from_static("zstd")); + assert!(matches!( + parse_grpc_encoding(&headers, &accepted), + Ok(GrpcEncoding::Zstd) + )); + + let _ = headers.insert("grpc-encoding", HeaderValue::from_static("gzip")); + assert!(matches!( + parse_grpc_encoding(&headers, &accepted), + Ok(GrpcEncoding::Gzip) + )); + + let _ = headers.insert("grpc-encoding", HeaderValue::from_static("zstdarrow1")); + assert!(matches!( + parse_grpc_encoding(&headers, &accepted), + Ok(GrpcEncoding::Zstd) + )); + } + + #[test] + fn test_parse_grpc_encoding_respects_config() { + let accepted = AcceptedGrpcEncodings::from_methods(&[CompressionMethod::Deflate]); + let mut headers = base_headers(); + let _ = headers.insert("grpc-encoding", HeaderValue::from_static("gzip")); + assert!(parse_grpc_encoding(&headers, &accepted).is_err()); + } + + #[test] + fn test_parse_grpc_accept_encoding() { + let mut headers = HeaderMap::new(); + let _ = headers.insert( + "grpc-accept-encoding", + HeaderValue::from_static("gzip,zstd, identity "), + ); + let parsed = parse_grpc_accept_encoding(&headers); + assert!(parsed.identity); + assert!(parsed.zstd); + assert!(parsed.gzip); + assert!(!parsed.deflate); + } + + #[test] + fn test_negotiate_response_encoding_prefers_config_order() { + let mut client_headers = HeaderMap::new(); + let _ = client_headers.insert( + "grpc-accept-encoding", + HeaderValue::from_static("zstd,gzip"), + ); + let client = parse_grpc_accept_encoding(&client_headers); + let cfg = vec![CompressionMethod::Gzip, CompressionMethod::Zstd]; + assert!(matches!( + negotiate_response_encoding(&cfg, &client), + GrpcEncoding::Gzip + )); + + let cfg = vec![CompressionMethod::Zstd]; + assert!(matches!( + negotiate_response_encoding(&cfg, &client), + GrpcEncoding::Zstd + )); + } + + #[test] + fn test_build_accept_encoding_header_includes_identity() { + let value = + build_accept_encoding_header(&[CompressionMethod::Zstd, CompressionMethod::Gzip]); + assert_eq!(value.to_str().unwrap(), "zstd,gzip,identity"); + } + + #[tokio::test] + async fn request_timeout_triggers_after_inactivity() { + let mut timeout = RequestTimeout::new(Some(Duration::from_millis(50))); + let (tx, rx) = mpsc::unbounded_channel::>(); + let mut stream = UnboundedReceiverStream::new(rx); + + let _producer = tokio::spawn(async move { + let _ = tx.send(Ok("first")); + sleep(Duration::from_millis(10)).await; + let _ = tx.send(Ok("second")); + sleep(Duration::from_millis(200)).await; + let _ = tx.send(Ok("third")); + }); + + assert!(timeout.next_with(&mut stream).await.unwrap().is_some()); + sleep(Duration::from_millis(15)).await; + assert!(timeout.next_with(&mut stream).await.unwrap().is_some()); + assert!(timeout.next_with(&mut stream).await.is_err()); + } + + #[tokio::test] + async fn request_timeout_disabled_when_unset() { + let mut timeout = RequestTimeout::new(None); + let (tx, rx) = mpsc::unbounded_channel::>(); + let mut stream = UnboundedReceiverStream::new(rx); + + let _producer = tokio::spawn(async move { + sleep(Duration::from_millis(30)).await; + let _ = tx.send(Ok("done")); + }); + + sleep(Duration::from_millis(35)).await; + let next = timeout.next_with(&mut stream).await.unwrap(); + assert!(next.is_some()); + } + + #[test] + fn test_grpc_message_encoder_identity_frame_layout() { + let mut encoder = GrpcResponseFrameEncoder::new(GrpcEncoding::Identity); + let message = BatchStatus { + batch_id: 42, + status_code: 7, + status_message: "ok".to_string(), + }; + let encoded = encoder.encode(&message).expect("identity encode"); + assert_eq!(encoded[0], 0); + let len = u32::from_be_bytes(encoded[1..5].try_into().unwrap()) as usize; + assert_eq!(len, encoded.len() - 5); + assert_eq!( + encoded[5..], + message.encode_to_vec(), + "payload matches prost encoding" + ); + } + + #[test] + fn test_grpc_message_encoder_gzip_round_trip() { + let mut encoder = GrpcResponseFrameEncoder::new(GrpcEncoding::Gzip); + let message = BatchStatus { + batch_id: 99, + status_code: 14, + status_message: "compressed".to_string(), + }; + let encoded = encoder.encode(&message).expect("gzip encode"); + assert_eq!(encoded[0], 1); + let len = u32::from_be_bytes(encoded[1..5].try_into().unwrap()) as usize; + assert_eq!(len, encoded.len() - 5); + let mut decoder = GzDecoder::new(&encoded[5..]); + let mut decompressed = Vec::new(); + let _ = decoder.read_to_end(&mut decompressed).expect("gunzip"); + assert_eq!(decompressed, message.encode_to_vec()); + } + + #[test] + fn test_grpc_message_encoder_deflate_round_trip() { + let mut encoder = GrpcResponseFrameEncoder::new(GrpcEncoding::Deflate); + let message = BatchStatus { + batch_id: 7, + status_code: 3, + status_message: "deflated".to_string(), + }; + let encoded = encoder.encode(&message).expect("deflate encode"); + assert_eq!(encoded[0], 1); + let len = u32::from_be_bytes(encoded[1..5].try_into().unwrap()) as usize; + assert_eq!(len, encoded.len() - 5); + let mut decoder = ZlibDecoder::new(&encoded[5..]); + let mut decompressed = Vec::new(); + let _ = decoder.read_to_end(&mut decompressed).expect("inflate"); + assert_eq!(decompressed, message.encode_to_vec()); + } + + fn test_router() -> Rc { + let (ctrl_tx, _) = pipeline_ctrl_msg_channel::(1); + let (_metrics_rx, metrics_reporter) = MetricsReporter::create_new_and_receiver(1); + let effect_handler = LocalEffectHandler::new( + NodeId { + index: 0, + name: "grpc-test".into(), + }, + HashMap::new(), + None, + ctrl_tx, + metrics_reporter, + ); + + let response_methods = vec![ + CompressionMethod::Zstd, + CompressionMethod::Gzip, + CompressionMethod::Deflate, + ]; + let request_encodings = AcceptedGrpcEncodings::from_methods(&response_methods); + let accept_header = build_accept_encoding_header(&response_methods); + let response_encoders = ResponseEncoderPool::new(&response_methods, 1); + let response_templates = response_methods.iter().copied().fold( + ResponseTemplates::new(accept_header.clone()), + |acc, method| acc.with_method(method, &accept_header), + ); + + Rc::new(GrpcRequestRouter { + effect_handler, + logs_ack_registry: None, + metrics_ack_registry: None, + traces_ack_registry: None, + max_in_flight_per_connection: 1, + request_encodings, + request_accept_header: accept_header, + response_methods, + request_timeout: None, + response_encoders, + response_templates, + zstd_decompressor: RefCell::new(None), + max_decoding_message_size: DEFAULT_MAX_DECODING_MESSAGE_SIZE, + }) + } + + #[tokio::test(flavor = "current_thread")] + async fn grpc_streaming_body_randomized_frames() { + async fn run_case(encoding: GrpcEncoding, seed: u64) { + let router = test_router(); + let mut rng = StdRng::seed_from_u64(seed); + for iteration in 0..32 { + let frame_count = rng.random_range(1..=8); + let mut expected_ids = Vec::with_capacity(frame_count); + let mut chunk_queue: VecDeque> = VecDeque::new(); + let mut expected_release = 0usize; + + for frame_idx in 0..frame_count { + let batch_id = (iteration * 100 + frame_idx) as i64; + expected_ids.push(batch_id); + let batch = BatchArrowRecords { + batch_id, + ..Default::default() + }; + + let frame = build_body_frame(&batch, encoding); + for chunk in split_frame_into_chunks(frame, &mut rng) { + expected_release += chunk.len(); + chunk_queue.push_back(Ok(chunk)); + } + } + + let (stream, state_handle) = MockRecvStream::new(chunk_queue); + let mut body = GrpcStreamingBody::with_stream( + stream, + encoding, + router.clone(), + DEFAULT_MAX_DECODING_MESSAGE_SIZE as usize, + ); + let mut observed_ids = Vec::new(); + while let Some(batch) = body + .next_message() + .await + .expect("fuzzer should decode batches") + { + observed_ids.push(batch.batch_id); + } + drop(body); + + assert_eq!( + observed_ids, expected_ids, + "encoding {:?} iteration {}", + encoding, iteration + ); + let released = state_handle + .lock() + .expect("state lock poisoned") + .released_bytes; + assert_eq!( + released, expected_release, + "flow control release mismatch for {:?}", + encoding + ); + } + } + + run_case(GrpcEncoding::Identity, 0x1111).await; + run_case(GrpcEncoding::Gzip, 0x2222).await; + run_case(GrpcEncoding::Deflate, 0x3333).await; + run_case(GrpcEncoding::Zstd, 0x4444).await; + } + + fn build_body_frame(batch: &BatchArrowRecords, encoding: GrpcEncoding) -> Bytes { + let payload = batch.encode_to_vec(); + let (compressed, encoded_payload) = match encoding { + GrpcEncoding::Identity => (false, payload), + GrpcEncoding::Gzip => (true, compress_payload_gzip(&payload)), + GrpcEncoding::Deflate => (true, compress_payload_deflate(&payload)), + GrpcEncoding::Zstd => (true, compress_payload_zstd(&payload)), + }; + let mut frame = BytesMut::with_capacity(5 + encoded_payload.len()); + frame.put_u8(u8::from(compressed)); + frame.put_u32(encoded_payload.len() as u32); + frame.extend_from_slice(&encoded_payload); + frame.freeze() + } + + fn compress_payload_gzip(payload: &[u8]) -> Vec { + let mut encoder = GzEncoder::new(Vec::new(), Compression::default()); + encoder.write_all(payload).expect("gzip write"); + encoder.finish().expect("gzip finish") + } + + fn compress_payload_deflate(payload: &[u8]) -> Vec { + let mut encoder = ZlibEncoder::new(Vec::new(), Compression::default()); + encoder.write_all(payload).expect("deflate write"); + encoder.finish().expect("deflate finish") + } + + fn compress_payload_zstd(payload: &[u8]) -> Vec { + let mut encoder = ZstdCompressor::new(0).expect("zstd encoder"); + let mut buffer = vec![0u8; payload.len().max(MIN_COMPRESSED_CAPACITY)]; + let written = encoder + .compress_to_buffer(payload, buffer.as_mut_slice()) + .expect("zstd compress"); + buffer.truncate(written); + buffer + } + + fn split_frame_into_chunks(frame: Bytes, rng: &mut StdRng) -> Vec { + let mut offset = 0; + let mut chunks = Vec::new(); + while offset < frame.len() { + let remaining = frame.len() - offset; + let max_chunk = remaining.clamp(1, 64); + let step = rng.random_range(1..=max_chunk); + chunks.push(frame.slice(offset..offset + step)); + offset += step; + } + chunks + } + + struct MockStreamState { + released_bytes: usize, + } + + struct MockRecvStream { + chunks: VecDeque>, + state: Arc>, + } + + impl MockRecvStream { + fn new( + chunks: VecDeque>, + ) -> (Self, Arc>) { + let state = Arc::new(Mutex::new(MockStreamState { released_bytes: 0 })); + ( + Self { + chunks, + state: state.clone(), + }, + state, + ) + } + } + + #[async_trait(?Send)] + impl BodyStream for MockRecvStream { + async fn next_chunk(&mut self) -> Option> { + yield_now().await; + self.chunks + .pop_front() + .map(|res| res.map_err(|err| err.to_string())) + } + + fn release_capacity(&mut self, released: usize) -> Result<(), BodyStreamError> { + if let Ok(mut state) = self.state.lock() { + state.released_bytes += released; + } + Ok(()) + } + } +} diff --git a/rust/otap-dataflow/crates/otap/src/otel_receiver/otap_tests.rs b/rust/otap-dataflow/crates/otap/src/otel_receiver/otap_tests.rs new file mode 100644 index 0000000000..a6dfea0fbb --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otel_receiver/otap_tests.rs @@ -0,0 +1,1128 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +use super::{OTEL_RECEIVER_URN, OtelReceiver}; +use crate::fake_data_generator::fake_signal::{ + fake_otlp_logs, fake_otlp_metrics, fake_otlp_traces, +}; +use crate::otap_mock::create_otap_batch; +use crate::otel_receiver::test_common::{build_test_registry, decode_pdata_to_message}; +use crate::pdata::OtapPdata; +use async_stream::stream; +use bytes::Bytes; +use otap_df_config::node::NodeUserConfig; +use otap_df_engine::context::ControllerContext; +use otap_df_engine::control::{AckMsg, NackMsg, NodeControlMsg}; +use otap_df_engine::receiver::ReceiverWrapper; +use otap_df_engine::testing::{ + receiver::{NotSendValidateContext, TestContext, TestRuntime}, + test_node, +}; +use otap_df_pdata::OtlpProtoBytes; +use otap_df_pdata::Producer; +use otap_df_pdata::otap::OtapArrowRecords; +use otap_df_pdata::proto::OtlpProtoMessage; +use otap_df_pdata::proto::opentelemetry::arrow::v1::{ + ArrowPayloadType, BatchStatus, StatusCode as ProtoStatusCode, + arrow_logs_service_client::ArrowLogsServiceClient, + arrow_metrics_service_client::ArrowMetricsServiceClient, + arrow_traces_service_client::ArrowTracesServiceClient, +}; +use otap_df_pdata::proto::opentelemetry::collector::logs::v1::ExportLogsServiceRequest; +use otap_df_pdata::proto::opentelemetry::collector::metrics::v1::ExportMetricsServiceRequest; +use otap_df_pdata::proto::opentelemetry::collector::trace::v1::ExportTraceServiceRequest; +use otap_df_pdata::testing::equiv::assert_equivalent; +use otap_df_telemetry::registry::MetricsRegistryHandle; +use prost::Message; +use serde_json::json; +use std::collections::HashSet; +use std::future::Future; +use std::net::SocketAddr; +use std::pin::Pin; +use std::sync::Arc; +use std::time::Instant; +use tokio::time::{Duration, timeout}; +use tonic::Status; +use tonic::codec::CompressionEncoding; +use weaver_forge::registry::ResolvedRegistry; + +fn pick_free_port() -> u16 { + portpicker::pick_unused_port().expect("No free ports") +} + +#[derive(Clone, Default)] +struct OtapFakeBatchPlan { + logs: Vec<(OtapArrowRecords, OtlpProtoMessage)>, + metrics: Vec<(OtapArrowRecords, OtlpProtoMessage)>, + traces: Vec<(OtapArrowRecords, OtlpProtoMessage)>, +} + +fn generate_otap_fake_batches(registry: &ResolvedRegistry) -> OtapFakeBatchPlan { + let mut plan = OtapFakeBatchPlan::default(); + for batch_size in 1..=100 { + let logs_data = fake_otlp_logs(batch_size, registry); + let logs_req = ExportLogsServiceRequest { + resource_logs: logs_data.resource_logs.clone(), + }; + let logs_bytes = Bytes::from(logs_req.encode_to_vec()); + let logs_records: OtapArrowRecords = OtlpProtoBytes::ExportLogsRequest(logs_bytes) + .try_into() + .expect("encode logs to arrow"); + plan.logs + .push((logs_records, OtlpProtoMessage::Logs(logs_data))); + + let metrics_data = fake_otlp_metrics(batch_size, registry); + let metrics_req = ExportMetricsServiceRequest { + resource_metrics: metrics_data.resource_metrics.clone(), + }; + let metrics_bytes = Bytes::from(metrics_req.encode_to_vec()); + let metrics_records: OtapArrowRecords = OtlpProtoBytes::ExportMetricsRequest(metrics_bytes) + .try_into() + .expect("encode metrics to arrow"); + plan.metrics + .push((metrics_records, OtlpProtoMessage::Metrics(metrics_data))); + + let traces_data = fake_otlp_traces(batch_size, registry); + let traces_req = ExportTraceServiceRequest { + resource_spans: traces_data.resource_spans.clone(), + }; + let traces_bytes = Bytes::from(traces_req.encode_to_vec()); + let traces_records: OtapArrowRecords = OtlpProtoBytes::ExportTracesRequest(traces_bytes) + .try_into() + .expect("encode traces to arrow"); + plan.traces + .push((traces_records, OtlpProtoMessage::Traces(traces_data))); + } + plan +} + +async fn validate_success_responses( + mut inbound_stream: S, + expected_count: usize, + signal_name: &str, +) where + S: futures::Stream> + Unpin, +{ + use futures::StreamExt; + let mut seen = HashSet::new(); + let mut count = 0usize; + while let Some(result) = inbound_stream.next().await { + let status = result.expect("successful response"); + assert_eq!( + status.status_code, + ProtoStatusCode::Ok as i32, + "Unexpected status code for {} batch {}", + signal_name, + status.batch_id + ); + assert_eq!( + status.status_message, "Successfully received", + "Unexpected status message for {} batch {}", + signal_name, status.batch_id + ); + assert!( + seen.insert(status.batch_id), + "duplicate status for {} batch {}", + signal_name, + status.batch_id + ); + count += 1; + } + assert_eq!( + count, expected_count, + "missing responses for {} (expected {}, got {})", + signal_name, expected_count, count + ); +} + +fn otap_fake_batch_scenario( + grpc_endpoint: String, + plan: OtapFakeBatchPlan, +) -> impl FnOnce(TestContext) -> Pin>> { + move |ctx| { + Box::pin(async move { + let metrics_count = plan.metrics.len(); + let logs_count = plan.logs.len(); + let traces_count = plan.traces.len(); + + let mut metrics_producer = Producer::new(); + let mut metric_bars = Vec::with_capacity(metrics_count); + for (idx, (mut records, _)) in plan.metrics.clone().into_iter().enumerate() { + match metrics_producer.produce_bar(&mut records) { + Ok(bar) => metric_bars.push(bar), + Err(e) => { + let reason = format!("produce metrics bar {} failed: {e}", idx); + _ = ctx.send_shutdown(Instant::now(), &reason).await.ok(); + panic!("{reason}"); + } + } + } + let mut metrics_client = ArrowMetricsServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("metrics client connect"); + #[allow(tail_expr_drop_order)] + let metrics_stream = stream! { + for bar in metric_bars { + yield bar; + } + }; + let metrics_resp = metrics_client + .arrow_metrics(metrics_stream) + .await + .expect("metrics request"); + validate_success_responses(metrics_resp.into_inner(), metrics_count, "metrics").await; + + let mut logs_producer = Producer::new(); + let mut log_bars = Vec::with_capacity(logs_count); + for (idx, (mut records, _)) in plan.logs.clone().into_iter().enumerate() { + match logs_producer.produce_bar(&mut records) { + Ok(bar) => log_bars.push(bar), + Err(e) => { + let reason = format!("produce logs bar {} failed: {e}", idx); + _ = ctx.send_shutdown(Instant::now(), &reason).await.ok(); + panic!("{reason}"); + } + } + } + let mut logs_client = ArrowLogsServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("logs client connect"); + #[allow(tail_expr_drop_order)] + let logs_stream = stream! { + for bar in log_bars { + yield bar; + } + }; + let logs_resp = logs_client + .arrow_logs(logs_stream) + .await + .expect("logs request"); + validate_success_responses(logs_resp.into_inner(), logs_count, "logs").await; + + let mut traces_producer = Producer::new(); + let mut trace_bars = Vec::with_capacity(traces_count); + for (idx, (mut records, _)) in plan.traces.clone().into_iter().enumerate() { + match traces_producer.produce_bar(&mut records) { + Ok(bar) => trace_bars.push(bar), + Err(e) => { + let reason = format!("produce traces bar {} failed: {e}", idx); + _ = ctx.send_shutdown(Instant::now(), &reason).await.ok(); + panic!("{reason}"); + } + } + } + let mut traces_client = ArrowTracesServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("traces client connect"); + #[allow(tail_expr_drop_order)] + let traces_stream = stream! { + for bar in trace_bars { + yield bar; + } + }; + let traces_resp = traces_client + .arrow_traces(traces_stream) + .await + .expect("traces request"); + validate_success_responses(traces_resp.into_inner(), traces_count, "traces").await; + + ctx.send_shutdown(Instant::now(), "OTAP fake batch test") + .await + .expect("shutdown send"); + }) + } +} + +fn otap_fake_batch_validation( + plan: OtapFakeBatchPlan, +) -> impl FnOnce(NotSendValidateContext) -> Pin>> { + move |mut ctx| { + Box::pin(async move { + let mut actual_metrics = Vec::new(); + for _ in 0..plan.metrics.len() { + let pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("metrics timeout") + .expect("missing metrics"); + actual_metrics.push(decode_pdata_to_message(&pdata)); + if let Some((_node_id, ack)) = crate::pdata::Context::next_ack(AckMsg::new(pdata)) { + ctx.send_control_msg(NodeControlMsg::Ack(ack)) + .await + .expect("metrics ack send failed"); + } + } + let expected_metrics: Vec<_> = plan.metrics.into_iter().map(|(_, msg)| msg).collect(); + assert_equivalent(&expected_metrics, &actual_metrics); + + let mut actual_logs = Vec::new(); + for _ in 0..plan.logs.len() { + let pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("logs timeout") + .expect("missing logs"); + actual_logs.push(decode_pdata_to_message(&pdata)); + if let Some((_node_id, ack)) = crate::pdata::Context::next_ack(AckMsg::new(pdata)) { + ctx.send_control_msg(NodeControlMsg::Ack(ack)) + .await + .expect("logs ack send failed"); + } + } + let expected_logs: Vec<_> = plan.logs.into_iter().map(|(_, msg)| msg).collect(); + assert_equivalent(&expected_logs, &actual_logs); + + let mut actual_traces = Vec::new(); + for _ in 0..plan.traces.len() { + let pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("traces timeout") + .expect("missing traces"); + actual_traces.push(decode_pdata_to_message(&pdata)); + if let Some((_node_id, ack)) = crate::pdata::Context::next_ack(AckMsg::new(pdata)) { + ctx.send_control_msg(NodeControlMsg::Ack(ack)) + .await + .expect("traces ack send failed"); + } + } + let expected_traces: Vec<_> = plan.traces.into_iter().map(|(_, msg)| msg).collect(); + assert_equivalent(&expected_traces, &actual_traces); + }) + } +} + +#[test] +#[ignore = "temporarily disabled while investigating produce_bar failure"] +fn test_otap_receiver_round_trip_fake_batches() { + let registry = build_test_registry(); + let plan = generate_otap_fake_batches(®istry); + + let grpc_addr = "127.0.0.1"; + let grpc_port = pick_free_port(); + let grpc_endpoint = format!("http://{grpc_addr}:{grpc_port}"); + let addr: SocketAddr = format!("{grpc_addr}:{grpc_port}").parse().unwrap(); + + let test_runtime = TestRuntime::new(); + let node_config = Arc::new(NodeUserConfig::new_receiver_config(OTEL_RECEIVER_URN)); + + let metrics_registry_handle = MetricsRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let mut receiver = OtelReceiver::from_config( + pipeline_ctx, + &json!({ + "listening_addr": addr.to_string(), + "wait_for_result": true + }), + ) + .unwrap(); + receiver.tune_max_concurrent_requests(test_runtime.config().output_pdata_channel.capacity); + let receiver = ReceiverWrapper::local( + receiver, + test_node(test_runtime.config().name.clone()), + node_config, + test_runtime.config(), + ); + + test_runtime + .set_receiver(receiver) + .run_test(otap_fake_batch_scenario(grpc_endpoint, plan.clone())) + .run_validation_concurrent(otap_fake_batch_validation(plan)); +} + +fn scenario( + grpc_endpoint: String, +) -> impl FnOnce(TestContext) -> Pin>> { + move |ctx| { + Box::pin(async move { + let mut arrow_metrics_client = + ArrowMetricsServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("Failed to connect metrics client"); + #[allow(tail_expr_drop_order)] + let metrics_stream = stream! { + let mut producer = Producer::new(); + for batch_id in 0..3 { + let mut metrics_records = + create_otap_batch(batch_id, ArrowPayloadType::MultivariateMetrics); + let bar = producer.produce_bar(&mut metrics_records).unwrap(); + yield bar + } + }; + let metrics_response = arrow_metrics_client + .arrow_metrics(metrics_stream) + .await + .expect("metrics request failed"); + validate_batch_responses( + metrics_response.into_inner(), + 0, + "Successfully received", + "metrics", + ) + .await; + + let mut arrow_logs_client = ArrowLogsServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("Failed to connect logs client"); + #[allow(tail_expr_drop_order)] + let logs_stream = stream! { + let mut producer = Producer::new(); + for batch_id in 0..3 { + let mut logs_records = create_otap_batch(batch_id, ArrowPayloadType::Logs); + let bar = producer.produce_bar(&mut logs_records).unwrap(); + yield bar; + } + }; + let logs_response = arrow_logs_client + .arrow_logs(logs_stream) + .await + .expect("logs request failed"); + validate_batch_responses( + logs_response.into_inner(), + 0, + "Successfully received", + "logs", + ) + .await; + + let mut arrow_traces_client = ArrowTracesServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("Failed to connect traces client"); + #[allow(tail_expr_drop_order)] + let traces_stream = stream! { + let mut producer = Producer::new(); + for batch_id in 0..3 { + let mut traces_records = + create_otap_batch(batch_id, ArrowPayloadType::Spans); + let bar = producer.produce_bar(&mut traces_records).unwrap(); + yield bar; + } + }; + let traces_response = arrow_traces_client + .arrow_traces(traces_stream) + .await + .expect("traces request failed"); + validate_batch_responses( + traces_response.into_inner(), + 0, + "Successfully received", + "traces", + ) + .await; + + ctx.send_shutdown(Instant::now(), "Test complete") + .await + .expect("shutdown send failed"); + }) + } +} + +fn validation_procedure() +-> impl FnOnce(NotSendValidateContext) -> Pin>> { + |mut ctx| { + Box::pin(async move { + for batch_id in 0..3 { + let metrics_pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("metrics timeout") + .expect("missing metrics"); + let metrics_records: OtapArrowRecords = metrics_pdata + .clone() + .payload() + .try_into() + .expect("metrics conversion"); + let _expected_metrics = + create_otap_batch(batch_id, ArrowPayloadType::MultivariateMetrics); + assert!(matches!(metrics_records, _expected_metrics)); + if let Some((_node_id, ack)) = + crate::pdata::Context::next_ack(AckMsg::new(metrics_pdata)) + { + ctx.send_control_msg(NodeControlMsg::Ack(ack)) + .await + .expect("metrics ack send failed"); + } + } + + for batch_id in 0..3 { + let logs_pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("logs timeout") + .expect("missing logs"); + let logs_records: OtapArrowRecords = logs_pdata + .clone() + .payload() + .try_into() + .expect("logs conversion"); + let _expected_logs = create_otap_batch(batch_id, ArrowPayloadType::Logs); + assert!(matches!(logs_records, _expected_logs)); + if let Some((_node_id, ack)) = + crate::pdata::Context::next_ack(AckMsg::new(logs_pdata)) + { + ctx.send_control_msg(NodeControlMsg::Ack(ack)) + .await + .expect("logs ack send failed"); + } + } + + for batch_id in 0..3 { + let traces_pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("traces timeout") + .expect("missing traces"); + let traces_records: OtapArrowRecords = traces_pdata + .clone() + .payload() + .try_into() + .expect("traces conversion"); + let _expected_traces = create_otap_batch(batch_id, ArrowPayloadType::Spans); + assert!(matches!(traces_records, _expected_traces)); + if let Some((_node_id, ack)) = + crate::pdata::Context::next_ack(AckMsg::new(traces_pdata)) + { + ctx.send_control_msg(NodeControlMsg::Ack(ack)) + .await + .expect("traces ack send failed"); + } + } + }) + } +} + +fn nack_scenario( + grpc_endpoint: String, +) -> impl FnOnce(TestContext) -> Pin>> { + move |ctx| { + Box::pin(async move { + let mut arrow_metrics_client = + ArrowMetricsServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("Failed to connect metrics client"); + #[allow(tail_expr_drop_order)] + let metrics_stream = stream! { + let mut producer = Producer::new(); + for batch_id in 0..3 { + let mut metrics_records = + create_otap_batch(batch_id, ArrowPayloadType::MultivariateMetrics); + let bar = producer.produce_bar(&mut metrics_records).unwrap(); + yield bar + } + }; + let metrics_response = arrow_metrics_client + .arrow_metrics(metrics_stream) + .await + .expect("metrics request failed"); + validate_batch_responses( + metrics_response.into_inner(), + 14, + &format!( + "Pipeline processing failed: {}", + "Test NACK reason for metrics" + ), + "metrics", + ) + .await; + + let mut arrow_logs_client = ArrowLogsServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("Failed to connect logs client"); + #[allow(tail_expr_drop_order)] + let logs_stream = stream! { + let mut producer = Producer::new(); + for batch_id in 0..3 { + let mut logs_records = create_otap_batch(batch_id, ArrowPayloadType::Logs); + let bar = producer.produce_bar(&mut logs_records).unwrap(); + yield bar; + } + }; + let logs_response = arrow_logs_client + .arrow_logs(logs_stream) + .await + .expect("logs request failed"); + validate_batch_responses( + logs_response.into_inner(), + 14, + &format!( + "Pipeline processing failed: {}", + "Test NACK reason for logs" + ), + "logs", + ) + .await; + + let mut arrow_traces_client = ArrowTracesServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("Failed to connect traces client"); + #[allow(tail_expr_drop_order)] + let traces_stream = stream! { + let mut producer = Producer::new(); + for batch_id in 0..3 { + let mut traces_records = + create_otap_batch(batch_id, ArrowPayloadType::Spans); + let bar = producer.produce_bar(&mut traces_records).unwrap(); + yield bar; + } + }; + let traces_response = arrow_traces_client + .arrow_traces(traces_stream) + .await + .expect("traces request failed"); + validate_batch_responses( + traces_response.into_inner(), + 14, + &format!( + "Pipeline processing failed: {}", + "Test NACK reason for traces" + ), + "traces", + ) + .await; + + ctx.send_shutdown(Instant::now(), "Test complete") + .await + .expect("shutdown send failed"); + }) + } +} + +fn nack_validation_procedure() +-> impl FnOnce(NotSendValidateContext) -> Pin>> { + |mut ctx| { + Box::pin(async move { + for _batch_id in 0..3 { + let metrics_pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("metrics timeout") + .expect("missing metrics"); + let nack = NackMsg::new("Test NACK reason for metrics", metrics_pdata); + if let Some((_node_id, nack)) = crate::pdata::Context::next_nack(nack) { + ctx.send_control_msg(NodeControlMsg::Nack(nack)) + .await + .expect("metrics nack send failed"); + } + } + + for _batch_id in 0..3 { + let logs_pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("logs timeout") + .expect("missing logs"); + let nack = NackMsg::new("Test NACK reason for logs", logs_pdata); + if let Some((_node_id, nack)) = crate::pdata::Context::next_nack(nack) { + ctx.send_control_msg(NodeControlMsg::Nack(nack)) + .await + .expect("logs nack send failed"); + } + } + + for _batch_id in 0..3 { + let traces_pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("traces timeout") + .expect("missing traces"); + let nack = NackMsg::new("Test NACK reason for traces", traces_pdata); + if let Some((_node_id, nack)) = crate::pdata::Context::next_nack(nack) { + ctx.send_control_msg(NodeControlMsg::Nack(nack)) + .await + .expect("traces nack send failed"); + } + } + }) + } +} + +type StatusPlan = Vec>; + +/// gRPC client harness for the Zstd log regression test. +/// +/// It streams a predetermined number of OTAP log batches, forces tonic to use request-side +/// Zstd compression, and asserts that the streamed `BatchStatus` items match the provided +/// `status_plan` (ACK vs NACK). Once validation finishes we trigger the runtime shutdown so +/// the paired validator can complete. +fn zstd_logs_scenario( + grpc_endpoint: String, + status_plan: StatusPlan, +) -> impl FnOnce(TestContext) -> Pin>> { + move |ctx| { + Box::pin(async move { + let mut arrow_logs_client = ArrowLogsServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("Failed to connect logs client") + .send_compressed(CompressionEncoding::Zstd); + + let plan_len = status_plan.len(); + #[allow(tail_expr_drop_order)] + let logs_stream = stream! { + let mut producer = Producer::new(); + for batch_id in 0..plan_len { + let mut logs_records = + create_otap_batch(batch_id as i64, ArrowPayloadType::Logs); + let bar = producer.produce_bar(&mut logs_records).unwrap(); + yield bar; + } + }; + + let logs_response = arrow_logs_client + .arrow_logs(logs_stream) + .await + .expect("logs request failed"); + validate_mixed_log_statuses(logs_response.into_inner(), &status_plan, "logs").await; + + ctx.send_shutdown(Instant::now(), "Zstd logs test complete") + .await + .expect("shutdown send failed"); + }) + } +} + +/// Validator companion for `zstd_logs_scenario`. +/// +/// Consumes the decoded batches the receiver pushes into the pipeline channel, verifies each +/// payload matches what the client sent, and sends either an ACK or NACK back into the system +/// to simulate downstream processing outcomes. +fn zstd_logs_validation( + status_plan: StatusPlan, +) -> impl FnOnce(NotSendValidateContext) -> Pin>> { + move |mut ctx| { + Box::pin(async move { + for (batch_id, expected) in status_plan.into_iter().enumerate() { + let pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("logs timeout") + .expect("missing logs"); + let records: OtapArrowRecords = + pdata.clone().payload().try_into().expect("logs conversion"); + let expected_logs = create_otap_batch(batch_id as i64, ArrowPayloadType::Logs); + assert_eq!(records, expected_logs); + match expected { + Ok(()) => { + if let Some((_node_id, ack)) = + crate::pdata::Context::next_ack(AckMsg::new(pdata)) + { + ctx.send_control_msg(NodeControlMsg::Ack(ack)) + .await + .expect("logs ack send failed"); + } + } + Err(reason) => { + let nack_msg = NackMsg::new(reason, pdata); + if let Some((_node_id, nack)) = crate::pdata::Context::next_nack(nack_msg) { + ctx.send_control_msg(NodeControlMsg::Nack(nack)) + .await + .expect("logs nack send failed"); + } + } + } + } + }) + } +} + +/// Confirms that the gRPC response stream mirrors the ACK/NACK plan. +/// +/// ACK responses must arrive as `OK` with the default success message, while NACKs surface as +/// `UNAVAILABLE` plus the provided failure reason. +async fn validate_mixed_log_statuses( + mut inbound_stream: S, + status_plan: &[Result<(), &'static str>], + signal_name: &str, +) where + S: futures::Stream> + Unpin, +{ + use futures::StreamExt; + let mut index = 0; + while let Some(result) = inbound_stream.next().await { + let batch_status = result.expect("Expected successful response"); + let expected = status_plan + .get(index) + .unwrap_or_else(|| panic!("unexpected extra response for {}", signal_name)); + match expected { + Ok(()) => { + assert_eq!( + batch_status.status_code, + ProtoStatusCode::Ok as i32, + "Unexpected success code for {} batch {}", + signal_name, + batch_status.batch_id + ); + assert_eq!( + batch_status.status_message, "Successfully received", + "Unexpected success message for {} batch {}", + signal_name, batch_status.batch_id + ); + } + Err(reason) => { + assert_eq!( + batch_status.status_code, + ProtoStatusCode::Unavailable as i32, + "Unexpected failure code for {} batch {}", + signal_name, + batch_status.batch_id + ); + assert_eq!( + batch_status.status_message, + format!("Pipeline processing failed: {}", reason), + "Unexpected failure message for {} batch {}", + signal_name, + batch_status.batch_id + ); + } + } + index += 1; + } + assert_eq!( + index, + status_plan.len(), + "Missing responses for {} (expected {}, saw {index})", + signal_name, + status_plan.len() + ); +} + +async fn validate_batch_responses( + mut inbound_stream: S, + expected_status_code: i32, + expected_status_message: &str, + signal_name: &str, +) where + S: futures::Stream> + Unpin, +{ + use futures::StreamExt; + let mut received_batch_ids = HashSet::new(); + while let Some(result) = inbound_stream.next().await { + assert!( + result.is_ok(), + "Expected successful response for {}", + signal_name + ); + let batch_status = result.unwrap(); + let batch_id = batch_status.batch_id; + assert!( + received_batch_ids.insert(batch_id), + "Duplicate response for batch {} ({})", + batch_id, + signal_name + ); + assert_eq!( + batch_status.status_code, expected_status_code, + "Unexpected status code for {} batch {}", + signal_name, batch_id + ); + assert_eq!( + batch_status.status_message, expected_status_message, + "Unexpected status message for {} batch {}", + signal_name, batch_id + ); + } + assert_eq!( + received_batch_ids, + (0..3).collect::>(), + "Missing responses for {}", + signal_name + ); +} + +#[test] +fn test_otel_receiver() { + let test_runtime = TestRuntime::new(); + let grpc_addr = "127.0.0.1"; + let grpc_port = pick_free_port(); + let grpc_endpoint = format!("http://{grpc_addr}:{grpc_port}"); + let addr: SocketAddr = format!("{grpc_addr}:{grpc_port}").parse().unwrap(); + + let node_config = Arc::new(NodeUserConfig::new_receiver_config(OTEL_RECEIVER_URN)); + use otap_df_engine::context::ControllerContext; + use otap_df_telemetry::registry::MetricsRegistryHandle; + use serde_json::json; + + let metrics_registry_handle = MetricsRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let config = json!({ "listening_addr": addr.to_string() }); + let mut receiver = OtelReceiver::from_config(pipeline_ctx, &config).unwrap(); + receiver.tune_max_concurrent_requests(test_runtime.config().output_pdata_channel.capacity); + + let receiver = ReceiverWrapper::local( + receiver, + test_node(test_runtime.config().name.clone()), + node_config, + test_runtime.config(), + ); + + test_runtime + .set_receiver(receiver) + .run_test(scenario(grpc_endpoint)) + .run_validation(validation_procedure()); +} + +#[test] +fn test_otel_receiver_ack() { + let test_runtime = TestRuntime::new(); + let grpc_addr = "127.0.0.1"; + let grpc_port = pick_free_port(); + let grpc_endpoint = format!("http://{grpc_addr}:{grpc_port}"); + let addr: SocketAddr = format!("{grpc_addr}:{grpc_port}").parse().unwrap(); + + let node_config = Arc::new(NodeUserConfig::new_receiver_config(OTEL_RECEIVER_URN)); + use otap_df_engine::context::ControllerContext; + use otap_df_telemetry::registry::MetricsRegistryHandle; + use serde_json::json; + + let metrics_registry_handle = MetricsRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let config = json!({ + "listening_addr": addr.to_string(), + "wait_for_result": true + }); + let mut receiver = OtelReceiver::from_config(pipeline_ctx, &config).unwrap(); + receiver.tune_max_concurrent_requests(test_runtime.config().output_pdata_channel.capacity); + let receiver = ReceiverWrapper::local( + receiver, + test_node(test_runtime.config().name.clone()), + node_config, + test_runtime.config(), + ); + + test_runtime + .set_receiver(receiver) + .run_test(scenario(grpc_endpoint)) + .run_validation_concurrent(validation_procedure()); +} + +#[test] +fn test_otel_receiver_nack() { + let test_runtime = TestRuntime::new(); + let grpc_addr = "127.0.0.1"; + let grpc_port = pick_free_port(); + let grpc_endpoint = format!("http://{grpc_addr}:{grpc_port}"); + let addr: SocketAddr = format!("{grpc_addr}:{grpc_port}").parse().unwrap(); + + let node_config = Arc::new(NodeUserConfig::new_receiver_config(OTEL_RECEIVER_URN)); + use otap_df_engine::context::ControllerContext; + use otap_df_telemetry::registry::MetricsRegistryHandle; + use serde_json::json; + + let metrics_registry_handle = MetricsRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let config = json!({ + "listening_addr": addr.to_string(), + "wait_for_result": true + }); + let mut receiver = OtelReceiver::from_config(pipeline_ctx, &config).unwrap(); + receiver.tune_max_concurrent_requests(test_runtime.config().output_pdata_channel.capacity); + let receiver = ReceiverWrapper::local( + receiver, + test_node(test_runtime.config().name.clone()), + node_config, + test_runtime.config(), + ); + + test_runtime + .set_receiver(receiver) + .run_test(nack_scenario(grpc_endpoint)) + .run_validation_concurrent(nack_validation_procedure()); +} + +#[test] +/// End-to-end test for request-side Zstd compression with mixed ACK/NACK outcomes. +/// +/// The receiver is configured with `wait_for_result` and `compression_method = "zstd"` so the +/// gRPC client can compress request frames. The `status_plan` drives alternating ACK/NACK +/// responses from the validation harness, exercising the effect-handler path (decoded batches +/// hit the channel) and the control path (ACK/NACK is reflected back to the client via +/// `BatchStatus`). +fn test_otel_receiver_zstd_logs_ack_nack() { + let test_runtime = TestRuntime::new(); + let grpc_addr = "127.0.0.1"; + let grpc_port = pick_free_port(); + let grpc_endpoint = format!("http://{grpc_addr}:{grpc_port}"); + let addr: SocketAddr = format!("{grpc_addr}:{grpc_port}").parse().unwrap(); + + let node_config = Arc::new(NodeUserConfig::new_receiver_config(OTEL_RECEIVER_URN)); + use otap_df_engine::context::ControllerContext; + use otap_df_telemetry::registry::MetricsRegistryHandle; + use serde_json::json; + + let metrics_registry_handle = MetricsRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let config = json!({ + "listening_addr": addr.to_string(), + "wait_for_result": true, + "compression_method": "zstd" + }); + let mut receiver = OtelReceiver::from_config(pipeline_ctx, &config).unwrap(); + receiver.tune_max_concurrent_requests(test_runtime.config().output_pdata_channel.capacity); + let receiver = ReceiverWrapper::local( + receiver, + test_node(test_runtime.config().name.clone()), + node_config, + test_runtime.config(), + ); + + let status_plan = vec![ + Ok(()), + Err("Test NACK reason for logs 1"), + Ok(()), + Err("Test NACK reason for logs 2"), + ]; + + test_runtime + .set_receiver(receiver) + .run_test(zstd_logs_scenario(grpc_endpoint, status_plan.clone())) + .run_validation_concurrent(zstd_logs_validation(status_plan)); +} + +#[test] +fn test_otel_receiver_config_parsing() { + use crate::compression::CompressionMethod; + use serde_json::json; + + let metrics_registry_handle = MetricsRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let config_with_max_concurrent_requests = json!({ + "listening_addr": "127.0.0.1:4417", + "max_concurrent_requests": 5000 + }); + let receiver = + OtelReceiver::from_config(pipeline_ctx.clone(), &config_with_max_concurrent_requests) + .unwrap(); + assert_eq!( + receiver.config.grpc.listening_addr.to_string(), + "127.0.0.1:4417" + ); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 5000); + assert!(!receiver.config.grpc.wait_for_result); + assert!(receiver.config.grpc.request_compression.is_none()); + assert!(receiver.config.grpc.response_compression.is_none()); + assert!( + receiver + .config + .grpc + .preferred_response_compression() + .is_none() + ); + assert!(receiver.config.grpc.timeout.is_none()); + + let config_minimal = json!({ "listening_addr": "127.0.0.1:4418" }); + let receiver = OtelReceiver::from_config(pipeline_ctx.clone(), &config_minimal).unwrap(); + assert_eq!( + receiver.config.grpc.listening_addr.to_string(), + "127.0.0.1:4418" + ); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 0); + assert!(!receiver.config.grpc.wait_for_result); + assert!(receiver.config.grpc.request_compression.is_none()); + assert!(receiver.config.grpc.response_compression.is_none()); + assert!( + receiver + .config + .grpc + .preferred_response_compression() + .is_none() + ); + assert!(receiver.config.grpc.timeout.is_none()); + + let config_full_gzip = json!({ + "listening_addr": "127.0.0.1:4419", + "compression_method": "gzip", + "max_concurrent_requests": 2500, + "wait_for_result": true, + "timeout": "30s" + }); + let receiver = OtelReceiver::from_config(pipeline_ctx.clone(), &config_full_gzip).unwrap(); + assert_eq!( + receiver.config.grpc.listening_addr.to_string(), + "127.0.0.1:4419" + ); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 2500); + assert!(receiver.config.grpc.wait_for_result); + assert_eq!( + receiver.config.grpc.request_compression, + Some(vec![CompressionMethod::Gzip]) + ); + assert!(receiver.config.grpc.response_compression.is_none()); + assert!( + receiver + .config + .grpc + .preferred_response_compression() + .is_none() + ); + assert_eq!(receiver.config.grpc.timeout, Some(Duration::from_secs(30))); + + let config_with_zstd = json!({ + "listening_addr": "127.0.0.1:4420", + "compression_method": "zstd", + "wait_for_result": false + }); + let receiver = OtelReceiver::from_config(pipeline_ctx.clone(), &config_with_zstd).unwrap(); + assert_eq!( + receiver.config.grpc.listening_addr.to_string(), + "127.0.0.1:4420" + ); + assert!(!receiver.config.grpc.wait_for_result); + assert_eq!( + receiver.config.grpc.request_compression, + Some(vec![CompressionMethod::Zstd]) + ); + assert!(receiver.config.grpc.response_compression.is_none()); + assert!( + receiver + .config + .grpc + .preferred_response_compression() + .is_none() + ); + assert!(receiver.config.grpc.timeout.is_none()); + + let config_with_deflate = json!({ + "listening_addr": "127.0.0.1:4421", + "compression_method": "deflate" + }); + let receiver = OtelReceiver::from_config(pipeline_ctx.clone(), &config_with_deflate).unwrap(); + assert_eq!( + receiver.config.grpc.listening_addr.to_string(), + "127.0.0.1:4421" + ); + assert_eq!( + receiver.config.grpc.request_compression, + Some(vec![CompressionMethod::Deflate]) + ); + assert!(receiver.config.grpc.response_compression.is_none()); + assert!( + receiver + .config + .grpc + .preferred_response_compression() + .is_none() + ); + assert!(receiver.config.grpc.timeout.is_none()); + + let config_with_response_only = json!({ + "listening_addr": "127.0.0.1:4422", + "response_compression_method": "gzip" + }); + let receiver = + OtelReceiver::from_config(pipeline_ctx.clone(), &config_with_response_only).unwrap(); + assert_eq!( + receiver.config.grpc.listening_addr.to_string(), + "127.0.0.1:4422" + ); + assert!(receiver.config.grpc.request_compression.is_none()); + assert_eq!( + receiver.config.grpc.response_compression, + Some(vec![CompressionMethod::Gzip]) + ); + assert_eq!( + receiver.config.grpc.preferred_response_compression(), + Some(CompressionMethod::Gzip) + ); + assert!(receiver.config.grpc.timeout.is_none()); +} diff --git a/rust/otap-dataflow/crates/otap/src/otel_receiver/otlp_tests.rs b/rust/otap-dataflow/crates/otap/src/otel_receiver/otlp_tests.rs new file mode 100644 index 0000000000..b44ab81785 --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otel_receiver/otlp_tests.rs @@ -0,0 +1,744 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +use super::{OTEL_RECEIVER_URN, OtelReceiver}; +use crate::compression::CompressionMethod; +use crate::fake_data_generator::fake_signal::{ + fake_otlp_logs, fake_otlp_metrics, fake_otlp_traces, +}; +use crate::otel_receiver::test_common::{build_test_registry, decode_pdata_to_message}; +use crate::pdata::OtapPdata; +use otap_df_config::node::NodeUserConfig; +use otap_df_engine::context::ControllerContext; +use otap_df_engine::control::{AckMsg, NackMsg, NodeControlMsg}; +use otap_df_engine::receiver::ReceiverWrapper; +use otap_df_engine::testing::{ + receiver::{NotSendValidateContext, TestContext, TestRuntime}, + test_node, +}; +use otap_df_pdata::OtlpProtoBytes; +use otap_df_pdata::proto::OtlpProtoMessage; +use otap_df_pdata::proto::opentelemetry::collector::logs::v1::logs_service_client::LogsServiceClient; +use otap_df_pdata::proto::opentelemetry::collector::logs::v1::{ + ExportLogsServiceRequest, ExportLogsServiceResponse, +}; +use otap_df_pdata::proto::opentelemetry::collector::metrics::v1::metrics_service_client::MetricsServiceClient; +use otap_df_pdata::proto::opentelemetry::collector::metrics::v1::{ + ExportMetricsServiceRequest, ExportMetricsServiceResponse, +}; +use otap_df_pdata::proto::opentelemetry::collector::trace::v1::trace_service_client::TraceServiceClient; +use otap_df_pdata::proto::opentelemetry::collector::trace::v1::{ + ExportTraceServiceRequest, ExportTraceServiceResponse, +}; +use otap_df_pdata::proto::opentelemetry::common::v1::{InstrumentationScope, KeyValue}; +use otap_df_pdata::proto::opentelemetry::logs::v1::{LogRecord, ResourceLogs, ScopeLogs}; +use otap_df_pdata::proto::opentelemetry::metrics::v1::{ResourceMetrics, ScopeMetrics}; +use otap_df_pdata::proto::opentelemetry::resource::v1::Resource; +use otap_df_pdata::proto::opentelemetry::trace::v1::{ResourceSpans, ScopeSpans}; +use otap_df_pdata::testing::equiv::assert_equivalent; +use otap_df_telemetry::registry::MetricsRegistryHandle; +use prost::Message; +use serde_json::json; +use std::future::Future; +use std::net::{SocketAddr, TcpListener}; +use std::pin::Pin; +use std::sync::Arc; +use std::time::{Duration, Instant}; +use tokio::time::timeout; +use tonic::Code; +use weaver_forge::registry::ResolvedRegistry; + +fn create_logs_service_request() -> ExportLogsServiceRequest { + ExportLogsServiceRequest { + resource_logs: vec![ResourceLogs { + resource: Some(Resource { + attributes: vec![KeyValue { + key: "a".to_string(), + ..Default::default() + }], + ..Default::default() + }), + scope_logs: vec![ScopeLogs { + scope: Some(InstrumentationScope { + attributes: vec![KeyValue { + key: "b".to_string(), + ..Default::default() + }], + ..Default::default() + }), + log_records: vec![ + LogRecord { + time_unix_nano: 1, + attributes: vec![KeyValue { + key: "c".to_string(), + ..Default::default() + }], + ..Default::default() + }, + LogRecord { + time_unix_nano: 2, + ..Default::default() + }, + ], + ..Default::default() + }], + ..Default::default() + }], + } +} + +fn create_metrics_service_request() -> ExportMetricsServiceRequest { + ExportMetricsServiceRequest { + resource_metrics: vec![ResourceMetrics { + resource: Some(Resource { + ..Default::default() + }), + scope_metrics: vec![ScopeMetrics { + ..Default::default() + }], + ..Default::default() + }], + } +} + +fn create_traces_service_request() -> ExportTraceServiceRequest { + ExportTraceServiceRequest { + resource_spans: vec![ResourceSpans { + resource: None, + scope_spans: vec![ + ScopeSpans { + ..Default::default() + }, + ScopeSpans { + ..Default::default() + }, + ], + schema_url: "opentelemetry.io/schema/traces".to_string(), + }], + } +} + +#[test] +fn test_otlp_config_parsing() { + let metrics_registry_handle = MetricsRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let config_with_max_concurrent_requests = json!({ + "listening_addr": "127.0.0.1:4317", + "max_concurrent_requests": 5000 + }); + let receiver = + OtelReceiver::from_config(pipeline_ctx.clone(), &config_with_max_concurrent_requests) + .unwrap(); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 5000); + assert!(receiver.config.grpc.request_compression.is_none()); + assert!(receiver.config.grpc.response_compression.is_none()); + assert!(receiver.config.grpc.tcp_nodelay); + assert_eq!( + receiver.config.grpc.tcp_keepalive, + Some(Duration::from_secs(45)) + ); + assert_eq!( + receiver.config.grpc.tcp_keepalive_interval, + Some(Duration::from_secs(15)) + ); + assert_eq!(receiver.config.grpc.tcp_keepalive_retries, Some(5)); + assert_eq!(receiver.config.grpc.transport_concurrency_limit, None); + assert!(receiver.config.grpc.load_shed); + assert_eq!( + receiver.config.grpc.initial_stream_window_size, + Some(8 * 1024 * 1024) + ); + assert_eq!( + receiver.config.grpc.initial_connection_window_size, + Some(24 * 1024 * 1024) + ); + assert_eq!(receiver.config.grpc.max_frame_size, Some(16 * 1024)); + assert_eq!( + receiver.config.grpc.max_decoding_message_size, + Some(4 * 1024 * 1024) + ); + assert_eq!(receiver.config.grpc.max_concurrent_streams, None); + + let config_with_server_overrides = json!({ + "listening_addr": "127.0.0.1:4317", + "max_concurrent_requests": 512, + "tcp_nodelay": false, + "tcp_keepalive": "60s", + "tcp_keepalive_interval": "20s", + "tcp_keepalive_retries": 3, + "transport_concurrency_limit": 256, + "load_shed": false, + "initial_stream_window_size": "8MiB", + "initial_connection_window_size": "32MiB", + "max_frame_size": "16MiB", + "max_decoding_message_size": "6MiB", + "max_concurrent_streams": 1024, + }); + let receiver = + OtelReceiver::from_config(pipeline_ctx.clone(), &config_with_server_overrides).unwrap(); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 512); + assert!(!receiver.config.grpc.tcp_nodelay); + assert_eq!( + receiver.config.grpc.tcp_keepalive, + Some(Duration::from_secs(60)) + ); + assert_eq!( + receiver.config.grpc.tcp_keepalive_interval, + Some(Duration::from_secs(20)) + ); + assert_eq!(receiver.config.grpc.tcp_keepalive_retries, Some(3)); + assert_eq!(receiver.config.grpc.transport_concurrency_limit, Some(256)); + assert!(!receiver.config.grpc.load_shed); + assert_eq!( + receiver.config.grpc.initial_stream_window_size, + Some(8 * 1024 * 1024) + ); + assert_eq!( + receiver.config.grpc.initial_connection_window_size, + Some(32 * 1024 * 1024) + ); + assert_eq!(receiver.config.grpc.max_frame_size, Some(16 * 1024 * 1024)); + assert_eq!( + receiver.config.grpc.max_decoding_message_size, + Some(6 * 1024 * 1024) + ); + assert_eq!(receiver.config.grpc.max_concurrent_streams, Some(1024)); + + let config_with_compression_list = json!({ + "listening_addr": "127.0.0.1:4317", + "compression_method": ["gzip","zstd"] + }); + let receiver = + OtelReceiver::from_config(pipeline_ctx.clone(), &config_with_compression_list).unwrap(); + assert_eq!( + receiver.config.grpc.request_compression, + Some(vec![CompressionMethod::Gzip, CompressionMethod::Zstd]) + ); + + let config_with_compression_none = json!({ + "listening_addr": "127.0.0.1:4317", + "compression_method": "none" + }); + let receiver = + OtelReceiver::from_config(pipeline_ctx.clone(), &config_with_compression_none).unwrap(); + assert_eq!(receiver.config.grpc.request_compression, Some(vec![])); + + let config_with_timeout = json!({ + "listening_addr": "127.0.0.1:4317", + "timeout": "30s" + }); + let receiver = OtelReceiver::from_config(pipeline_ctx.clone(), &config_with_timeout).unwrap(); + assert_eq!(receiver.config.grpc.timeout, Some(Duration::from_secs(30))); + + let config_with_timeout_ms = json!({ + "listening_addr": "127.0.0.1:4317", + "timeout": "500ms" + }); + let receiver = OtelReceiver::from_config(pipeline_ctx, &config_with_timeout_ms).unwrap(); + assert_eq!( + receiver.config.grpc.timeout, + Some(Duration::from_millis(500)) + ); +} + +#[test] +fn test_otlp_tune_max_concurrent_requests() { + let metrics_registry_handle = MetricsRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let config_default = json!({ "listening_addr": "127.0.0.1:4317" }); + let mut receiver = OtelReceiver::from_config(pipeline_ctx.clone(), &config_default).unwrap(); + receiver.tune_max_concurrent_requests(128); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 128); + + let config_small = json!({ + "listening_addr": "127.0.0.1:4317", + "max_concurrent_requests": 32 + }); + let mut receiver = OtelReceiver::from_config(pipeline_ctx.clone(), &config_small).unwrap(); + receiver.tune_max_concurrent_requests(128); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 32); + + let config_zero = json!({ + "listening_addr": "127.0.0.1:4317", + "max_concurrent_requests": 0 + }); + let mut receiver = OtelReceiver::from_config(pipeline_ctx, &config_zero).unwrap(); + receiver.tune_max_concurrent_requests(256); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 256); +} + +#[derive(Clone, Default, Debug)] +struct FakeBatchPlan { + logs_requests: Vec, + metrics_requests: Vec, + traces_requests: Vec, + expected_logs: Vec, + expected_metrics: Vec, + expected_traces: Vec, +} + +fn pick_free_port() -> u16 { + for _ in 0..5 { + if let Ok(listener) = TcpListener::bind("127.0.0.1:0") { + let port = listener.local_addr().expect("local addr").port(); + return port; + } + if let Some(port) = portpicker::pick_unused_port() { + return port; + } + } + panic!("free port"); +} + +fn generate_fake_batches(registry: &ResolvedRegistry) -> FakeBatchPlan { + let mut plan = FakeBatchPlan::default(); + for batch_size in 1..=100 { + let logs = fake_otlp_logs(batch_size, registry); + plan.logs_requests.push(ExportLogsServiceRequest { + resource_logs: logs.resource_logs.clone(), + }); + plan.expected_logs.push(OtlpProtoMessage::Logs(logs)); + + let metrics = fake_otlp_metrics(batch_size, registry); + plan.metrics_requests.push(ExportMetricsServiceRequest { + resource_metrics: metrics.resource_metrics.clone(), + }); + plan.expected_metrics + .push(OtlpProtoMessage::Metrics(metrics)); + + let traces = fake_otlp_traces(batch_size, registry); + plan.traces_requests.push(ExportTraceServiceRequest { + resource_spans: traces.resource_spans.clone(), + }); + plan.expected_traces.push(OtlpProtoMessage::Traces(traces)); + } + plan +} + +fn fake_batch_scenario( + grpc_endpoint: String, + plan: FakeBatchPlan, +) -> impl FnOnce(TestContext) -> Pin>> { + move |ctx| { + Box::pin(async move { + let mut logs_client = LogsServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("logs client connect"); + for request in plan.logs_requests { + let response = logs_client + .export(request) + .await + .expect("logs request succeeds") + .into_inner(); + assert_eq!( + response, + ExportLogsServiceResponse { + partial_success: None + } + ); + } + + let mut metrics_client = MetricsServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("metrics client connect"); + for request in plan.metrics_requests { + let response = metrics_client + .export(request) + .await + .expect("metrics request succeeds") + .into_inner(); + assert_eq!( + response, + ExportMetricsServiceResponse { + partial_success: None + } + ); + } + + let mut traces_client = TraceServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("traces client connect"); + for request in plan.traces_requests { + let response = traces_client + .export(request) + .await + .expect("traces request succeeds") + .into_inner(); + assert_eq!( + response, + ExportTraceServiceResponse { + partial_success: None + } + ); + } + + ctx.send_shutdown(Instant::now(), "OTLP fake batch test") + .await + .expect("shutdown send"); + }) + } +} + +fn fake_batch_validation( + plan: FakeBatchPlan, +) -> impl FnOnce(NotSendValidateContext) -> Pin>> { + move |mut ctx| { + Box::pin(async move { + let mut actual_logs = Vec::new(); + for _ in 0..plan.expected_logs.len() { + let pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("logs timeout") + .expect("missing logs"); + let message = decode_pdata_to_message(&pdata); + assert!( + matches!(message, OtlpProtoMessage::Logs(_)), + "expected logs payload, got {:?}", + message.signal_type() + ); + actual_logs.push(message); + if let Some((_node_id, ack)) = crate::pdata::Context::next_ack(AckMsg::new(pdata)) { + ctx.send_control_msg(NodeControlMsg::Ack(ack)) + .await + .expect("logs ack"); + } + } + + let mut actual_metrics = Vec::new(); + for _ in 0..plan.expected_metrics.len() { + let pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("metrics timeout") + .expect("missing metrics"); + let message = decode_pdata_to_message(&pdata); + assert!( + matches!(message, OtlpProtoMessage::Metrics(_)), + "expected metrics payload, got {:?}", + message.signal_type() + ); + actual_metrics.push(message); + if let Some((_node_id, ack)) = crate::pdata::Context::next_ack(AckMsg::new(pdata)) { + ctx.send_control_msg(NodeControlMsg::Ack(ack)) + .await + .expect("metrics ack"); + } + } + + let mut actual_traces = Vec::new(); + for _ in 0..plan.expected_traces.len() { + let pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("traces timeout") + .expect("missing traces"); + let message = decode_pdata_to_message(&pdata); + assert!( + matches!(message, OtlpProtoMessage::Traces(_)), + "expected traces payload, got {:?}", + message.signal_type() + ); + actual_traces.push(message); + if let Some((_node_id, ack)) = crate::pdata::Context::next_ack(AckMsg::new(pdata)) { + ctx.send_control_msg(NodeControlMsg::Ack(ack)) + .await + .expect("traces ack"); + } + } + + assert_equivalent(&plan.expected_logs, &actual_logs); + assert_equivalent(&plan.expected_metrics, &actual_metrics); + assert_equivalent(&plan.expected_traces, &actual_traces); + }) + } +} + +#[test] +fn test_otlp_receiver_round_trip_fake_batches() { + let registry = build_test_registry(); + let plan = generate_fake_batches(®istry); + let grpc_addr = "127.0.0.1"; + let grpc_port = pick_free_port(); + let grpc_endpoint = format!("http://{grpc_addr}:{grpc_port}"); + let addr: SocketAddr = format!("{grpc_addr}:{grpc_port}").parse().unwrap(); + + let test_runtime = TestRuntime::new(); + let node_config = Arc::new(NodeUserConfig::new_receiver_config(OTEL_RECEIVER_URN)); + + let metrics_registry_handle = MetricsRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let mut receiver = OtelReceiver::from_config( + pipeline_ctx, + &json!({ + "listening_addr": addr.to_string(), + "wait_for_result": true + }), + ) + .unwrap(); + receiver.tune_max_concurrent_requests(test_runtime.config().output_pdata_channel.capacity); + let receiver = ReceiverWrapper::local( + receiver, + test_node(test_runtime.config().name.clone()), + node_config, + test_runtime.config(), + ); + + test_runtime + .set_receiver(receiver) + .run_test(fake_batch_scenario(grpc_endpoint, plan.clone())) + .run_validation_concurrent(fake_batch_validation(plan)); +} + +fn otlp_scenario( + grpc_endpoint: String, +) -> impl FnOnce(TestContext) -> Pin>> { + move |ctx| { + Box::pin(async move { + let mut logs_client = LogsServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("logs client connect"); + let logs_response = logs_client + .export(create_logs_service_request()) + .await + .expect("logs request succeeds") + .into_inner(); + assert_eq!( + logs_response, + ExportLogsServiceResponse { + partial_success: None + } + ); + + let mut metrics_client = MetricsServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("metrics client connect"); + let metrics_response = metrics_client + .export(create_metrics_service_request()) + .await + .expect("metrics request succeeds") + .into_inner(); + assert_eq!( + metrics_response, + ExportMetricsServiceResponse { + partial_success: None + } + ); + + let mut traces_client = TraceServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("traces client connect"); + let traces_response = traces_client + .export(create_traces_service_request()) + .await + .expect("traces request succeeds") + .into_inner(); + assert_eq!( + traces_response, + ExportTraceServiceResponse { + partial_success: None + } + ); + + ctx.send_shutdown(Instant::now(), "OTLP test") + .await + .expect("shutdown send"); + }) + } +} + +fn validation_procedure() +-> impl FnOnce(NotSendValidateContext) -> Pin>> { + |mut ctx| { + Box::pin(async move { + let logs_pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("logs timeout") + .expect("missing logs"); + let logs_proto: OtlpProtoBytes = logs_pdata + .clone() + .payload() + .try_into() + .expect("logs conversion"); + assert!(matches!(logs_proto, OtlpProtoBytes::ExportLogsRequest(_))); + let expected = create_logs_service_request(); + let mut expected_bytes = Vec::new(); + expected.encode(&mut expected_bytes).unwrap(); + assert_eq!(&expected_bytes, logs_proto.as_bytes()); + if let Some((_node_id, ack)) = crate::pdata::Context::next_ack(AckMsg::new(logs_pdata)) + { + ctx.send_control_msg(NodeControlMsg::Ack(ack)) + .await + .expect("logs ack"); + } + + let metrics_pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("metrics timeout") + .expect("missing metrics"); + let metrics_proto: OtlpProtoBytes = metrics_pdata + .clone() + .payload() + .try_into() + .expect("metrics conversion"); + assert!(matches!( + metrics_proto, + OtlpProtoBytes::ExportMetricsRequest(_) + )); + let expected = create_metrics_service_request(); + let mut expected_bytes = Vec::new(); + expected.encode(&mut expected_bytes).unwrap(); + assert_eq!(&expected_bytes, metrics_proto.as_bytes()); + if let Some((_node_id, ack)) = + crate::pdata::Context::next_ack(AckMsg::new(metrics_pdata)) + { + ctx.send_control_msg(NodeControlMsg::Ack(ack)) + .await + .expect("metrics ack"); + } + + let traces_pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("traces timeout") + .expect("missing traces"); + let traces_proto: OtlpProtoBytes = traces_pdata + .clone() + .payload() + .try_into() + .expect("traces conversion"); + assert!(matches!( + traces_proto, + OtlpProtoBytes::ExportTracesRequest(_) + )); + let expected = create_traces_service_request(); + let mut expected_bytes = Vec::new(); + expected.encode(&mut expected_bytes).unwrap(); + assert_eq!(&expected_bytes, traces_proto.as_bytes()); + if let Some((_node_id, ack)) = + crate::pdata::Context::next_ack(AckMsg::new(traces_pdata)) + { + ctx.send_control_msg(NodeControlMsg::Ack(ack)) + .await + .expect("traces ack"); + } + }) + } +} + +#[test] +fn test_otlp_receiver_ack() { + let test_runtime = TestRuntime::new(); + let grpc_addr = "127.0.0.1"; + let grpc_port = portpicker::pick_unused_port().expect("free port"); + let grpc_endpoint = format!("http://{grpc_addr}:{grpc_port}"); + let addr: SocketAddr = format!("{grpc_addr}:{grpc_port}").parse().unwrap(); + + let node_config = Arc::new(NodeUserConfig::new_receiver_config(OTEL_RECEIVER_URN)); + + let metrics_registry_handle = MetricsRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let mut receiver = OtelReceiver::from_config( + pipeline_ctx, + &json!({ + "listening_addr": addr.to_string(), + "wait_for_result": true + }), + ) + .unwrap(); + receiver.tune_max_concurrent_requests(test_runtime.config().output_pdata_channel.capacity); + let receiver = ReceiverWrapper::local( + receiver, + test_node(test_runtime.config().name.clone()), + node_config, + test_runtime.config(), + ); + + test_runtime + .set_receiver(receiver) + .run_test(otlp_scenario(grpc_endpoint)) + .run_validation_concurrent(validation_procedure()); +} + +fn nack_scenario( + grpc_endpoint: String, +) -> impl FnOnce(TestContext) -> Pin>> { + move |ctx| { + Box::pin(async move { + let mut logs_client = LogsServiceClient::connect(grpc_endpoint.clone()) + .await + .expect("logs client connect"); + let result = logs_client.export(create_logs_service_request()).await; + assert!(result.is_err(), "nack should surface error"); + let status = result.unwrap_err(); + assert_eq!(status.code(), Code::Unavailable); + assert!( + status + .message() + .contains("Pipeline processing failed: Test nack reason") + ); + + ctx.send_shutdown(Instant::now(), "OTLP nack test") + .await + .expect("shutdown send"); + }) + } +} + +fn nack_validation() +-> impl FnOnce(NotSendValidateContext) -> Pin>> { + |mut ctx| { + Box::pin(async move { + let logs_pdata = timeout(Duration::from_secs(3), ctx.recv()) + .await + .expect("logs timeout") + .expect("missing logs"); + let nack = NackMsg::new("Test nack reason", logs_pdata); + if let Some((_node_id, nack)) = crate::pdata::Context::next_nack(nack) { + ctx.send_control_msg(NodeControlMsg::Nack(nack)) + .await + .expect("send nack"); + } + }) + } +} + +#[test] +fn test_otlp_receiver_nack() { + let test_runtime = TestRuntime::new(); + let grpc_addr = "127.0.0.1"; + let grpc_port = portpicker::pick_unused_port().expect("free port"); + let grpc_endpoint = format!("http://{grpc_addr}:{grpc_port}"); + let addr: SocketAddr = format!("{grpc_addr}:{grpc_port}").parse().unwrap(); + + let node_config = Arc::new(NodeUserConfig::new_receiver_config(OTEL_RECEIVER_URN)); + + let metrics_registry_handle = MetricsRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let mut receiver = OtelReceiver::from_config( + pipeline_ctx, + &json!({ + "listening_addr": addr.to_string(), + "wait_for_result": true + }), + ) + .unwrap(); + receiver.tune_max_concurrent_requests(test_runtime.config().output_pdata_channel.capacity); + let receiver = ReceiverWrapper::local( + receiver, + test_node(test_runtime.config().name.clone()), + node_config, + test_runtime.config(), + ); + + test_runtime + .set_receiver(receiver) + .run_test(nack_scenario(grpc_endpoint)) + .run_validation_concurrent(nack_validation()); +} diff --git a/rust/otap-dataflow/crates/otap/src/otel_receiver/response_templates.rs b/rust/otap-dataflow/crates/otap/src/otel_receiver/response_templates.rs new file mode 100644 index 0000000000..51fbcfaad2 --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otel_receiver/response_templates.rs @@ -0,0 +1,70 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Prebuilt HTTP response templates for successful unary gRPC responses. +// +//! The router reuses these for OTLP exports so we avoid rebuilding the +//! `content-type`, `grpc-encoding`, and `grpc-accept-encoding` headers +//! for every call. + +use http::{HeaderValue, Response, StatusCode as HttpStatusCode}; + +use crate::compression::CompressionMethod; +use crate::otel_receiver::grpc::{self, grpc_encoding_token}; + +/// Cached HTTP response headers for successful unary responses. +pub struct ResponseTemplates { + /// `ok_plain` is the identity encoding template + pub ok_plain: Response<()>, + /// `ok_encoded` stores one template per configured compression method. + pub ok_encoded: Vec<(CompressionMethod, Response<()>)>, +} + +impl ResponseTemplates { + pub fn new(accept_header: HeaderValue) -> Self { + let ok_plain = build_ok_response(accept_header.clone(), None); + Self { + ok_plain, + ok_encoded: Vec::new(), + } + } + + pub fn get_ok(&self, method: Option) -> Option> { + match method { + None => Some(self.ok_plain.clone()), + Some(method) => self + .ok_encoded + .iter() + .find(|(m, _)| *m == method) + .map(|(_, resp)| resp.clone()), + } + } + + pub fn with_method(mut self, method: CompressionMethod, accept_header: &HeaderValue) -> Self { + let encoding = match method { + CompressionMethod::Zstd => grpc::GrpcEncoding::Zstd, + CompressionMethod::Gzip => grpc::GrpcEncoding::Gzip, + CompressionMethod::Deflate => grpc::GrpcEncoding::Deflate, + }; + if let Some(token) = grpc_encoding_token(encoding) { + let encoded = + build_ok_response(accept_header.clone(), Some(HeaderValue::from_static(token))); + self.ok_encoded.push((method, encoded)); + } + self + } +} + +fn build_ok_response( + accept_header: HeaderValue, + encoding_token: Option, +) -> Response<()> { + let mut builder = Response::builder() + .status(HttpStatusCode::OK) + .header("content-type", "application/grpc") + .header("grpc-accept-encoding", accept_header); + if let Some(token) = encoding_token { + builder = builder.header("grpc-encoding", token); + } + builder.body(()).expect("response build must succeed") +} diff --git a/rust/otap-dataflow/crates/otap/src/otel_receiver/router.rs b/rust/otap-dataflow/crates/otap/src/otel_receiver/router.rs new file mode 100644 index 0000000000..44be5f2541 --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otel_receiver/router.rs @@ -0,0 +1,490 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! gRPC routing for OTAP Arrow and OTLP protobuf endpoints. +//! +//! This module owns: +//! - request header parsing and compression negotiation, +//! - dispatch to OTAP streaming handlers and OTLP unary handlers, +//! - the small per request context that holds encoders and response templates. + +use std::cell::RefCell; +use std::rc::Rc; +use std::time::Duration; + +use super::ack::{AckCompletionFuture, AckPollResult, AckRegistry}; +use super::encoder::{EncoderGuard, GrpcResponseFrameEncoder, ResponseEncoderPool}; +use super::grpc::{ + AcceptedGrpcEncodings, GrpcStreamingBody, RequestTimeout, negotiate_response_encoding, + parse_grpc_accept_encoding, parse_grpc_encoding, +}; +use super::response_templates::ResponseTemplates; +use super::status::Status; +use super::stream::stream_batch_statuses; +use crate::compression::CompressionMethod; +use crate::otel_receiver::grpc; +use crate::pdata::{Context, OtapPdata}; +use bytes::Bytes; +use h2::server::SendResponse; +use http::{HeaderMap, HeaderValue, Request, Response, StatusCode as HttpStatusCode}; +use otap_df_config::SignalType; +use otap_df_engine::local::receiver as local; +use otap_df_engine::{Interests, ProducerEffectHandlerExtension}; +use otap_df_pdata::otap::{Logs, Metrics, OtapBatchStore, Traces}; +use otap_df_pdata::proto::opentelemetry::collector::logs::v1::ExportLogsServiceResponse; +use otap_df_pdata::proto::opentelemetry::collector::metrics::v1::ExportMetricsServiceResponse; +use otap_df_pdata::proto::opentelemetry::collector::trace::v1::ExportTraceServiceResponse; +use otap_df_pdata::{OtapArrowRecords, OtlpProtoBytes}; +use zstd::bulk::Decompressor; + +/// OTAP gRPC service paths. +const ARROW_LOGS_SERVICE: &str = + "/opentelemetry.proto.experimental.arrow.v1.ArrowLogsService/ArrowLogs"; +const ARROW_METRICS_SERVICE: &str = + "/opentelemetry.proto.experimental.arrow.v1.ArrowMetricsService/ArrowMetrics"; +const ARROW_TRACES_SERVICE: &str = + "/opentelemetry.proto.experimental.arrow.v1.ArrowTracesService/ArrowTraces"; + +/// OTLP gRPC service paths. +const OTLP_LOGS_SERVICE: &str = "/opentelemetry.proto.collector.logs.v1.LogsService/Export"; +const OTLP_METRICS_SERVICE: &str = + "/opentelemetry.proto.collector.metrics.v1.MetricsService/Export"; +const OTLP_TRACES_SERVICE: &str = "/opentelemetry.proto.collector.trace.v1.TraceService/Export"; + +/// Routes each inbound gRPC request to the appropriate OTLP or OTAP handler. +/// +/// A single instance lives per core and is shared across all h2 connections. It keeps: +/// - the effect handler into the pipeline, +/// - per signal Ack registries, +/// - request and response compression configuration, +/// - timeouts, encoder pool, and response header templates, and +/// - a shared zstd decompressor used by the streaming body decoder. +pub(crate) struct GrpcRequestRouter { + pub(crate) effect_handler: local::EffectHandler, + pub(crate) logs_ack_registry: Option, + pub(crate) metrics_ack_registry: Option, + pub(crate) traces_ack_registry: Option, + pub(crate) max_in_flight_per_connection: usize, + pub(crate) request_encodings: AcceptedGrpcEncodings, + pub(crate) request_accept_header: HeaderValue, + pub(crate) response_methods: Vec, + pub(crate) request_timeout: Option, + pub(crate) response_encoders: ResponseEncoderPool, + pub(crate) response_templates: ResponseTemplates, + // zstd decompressor shared by every stream on this core. + pub(crate) zstd_decompressor: RefCell>>, + /// Maximum decoded message size in bytes (from GrpcServerSettings::max_decoding_message_size) + pub(crate) max_decoding_message_size: u32, +} + +/// Per request gRPC context used by the router while serving a single RPC. +struct RequestContext<'a> { + request_encoding: grpc::GrpcEncoding, + response: Response<()>, + response_encoder: EncoderGuard<'a>, +} + +impl GrpcRequestRouter { + /// Validates the incoming headers and builds the response template and encoder for this call. + /// + /// This step: + /// - parses the request `grpc-encoding`, + /// - negotiates the response encoding against server preferences and client accept list, and + /// - selects the appropriate prebuilt response header template. + fn prepare_request<'a>(&'a self, headers: &HeaderMap) -> Result, Status> { + let request_encoding = parse_grpc_encoding(headers, &self.request_encodings)?; + let client_accept = parse_grpc_accept_encoding(headers); + let response_encoding = negotiate_response_encoding(&self.response_methods, &client_accept); + let response = self + .response_templates + .get_ok(CompressionMethod::from_grpc_encoding(response_encoding)) + .ok_or_else(|| Status::internal("failed to build response"))?; + let response_encoder = self.response_encoders.checkout(response_encoding); + + Ok(RequestContext { + request_encoding, + response, + response_encoder, + }) + } + + /// Routes a single gRPC request to the correct signal specific handler based on the path. + pub(crate) async fn route_grpc_request( + self: Rc, + request: Request, + respond: SendResponse, + ) -> Result<(), Status> { + let path = request.uri().path(); + match path { + ARROW_LOGS_SERVICE => { + self.serve_otap_stream::( + request, + respond, + OtapArrowRecords::Logs, + self.logs_ack_registry.clone(), + ) + .await + } + ARROW_METRICS_SERVICE => { + self.serve_otap_stream::( + request, + respond, + OtapArrowRecords::Metrics, + self.metrics_ack_registry.clone(), + ) + .await + } + ARROW_TRACES_SERVICE => { + self.serve_otap_stream::( + request, + respond, + OtapArrowRecords::Traces, + self.traces_ack_registry.clone(), + ) + .await + } + OTLP_LOGS_SERVICE => { + self.serve_otlp_unary( + request, + respond, + SignalType::Logs, + self.logs_ack_registry.clone(), + ) + .await + } + OTLP_METRICS_SERVICE => { + self.serve_otlp_unary( + request, + respond, + SignalType::Metrics, + self.metrics_ack_registry.clone(), + ) + .await + } + OTLP_TRACES_SERVICE => { + self.serve_otlp_unary( + request, + respond, + SignalType::Traces, + self.traces_ack_registry.clone(), + ) + .await + } + _ => { + log::warn!("Unknown OTEL gRPC path {}", path); + respond_with_error( + respond, + Status::unimplemented("unknown method"), + &self.request_accept_header, + ); + Ok(()) + } + } + } + + /// Serves an OTAP Arrow bidirectional stream. + /// + /// Batches from the client are decoded and sent into the pipeline. Ack and Nack outcomes + /// are converted back into `BatchStatus` messages on the same stream. + async fn serve_otap_stream( + self: &Rc, + request: Request, + mut respond: SendResponse, + otap_batch: fn(T) -> OtapArrowRecords, + ack_registry: Option, + ) -> Result<(), Status> + where + T: OtapBatchStore + 'static, + { + let mut ctx = self.prepare_request(request.headers())?; + let recv_stream = request.into_body(); + let body = GrpcStreamingBody::new( + recv_stream, + ctx.request_encoding, + Rc::clone(self), + self.max_decoding_message_size as usize, + ); + + let mut status_stream = stream_batch_statuses::( + body, + self.effect_handler.clone(), + ack_registry, + otap_batch, + self.max_in_flight_per_connection, + ); + + let mut send_stream = respond + .send_response(ctx.response, false) + .map_err(|e| Status::internal(format!("failed to send response headers: {e}")))?; + + let mut request_timeout = RequestTimeout::new(self.request_timeout); + + loop { + let next_item = match request_timeout.next_with(&mut status_stream).await { + Ok(item) => item, + Err(()) => { + if let Some(duration) = self.request_timeout { + log::debug!("Request timed out after {:?}", duration); + } + send_error_trailers( + send_stream, + Status::deadline_exceeded("request timed out"), + ); + return Ok(()); + } + }; + + match next_item { + Some(Ok(status)) => { + let bytes = ctx.response_encoder.encode(&status)?; + if let Err(e) = send_stream.send_data(bytes, false) { + log::debug!("send_data failed: {e}"); + return Ok(()); + } + } + Some(Err(status)) => { + log::error!("Stream aborted with status {}", status); + send_error_trailers(send_stream, status); + return Ok(()); + } + None => break, + } + } + + send_ok_trailers(send_stream); + Ok(()) + } + + /// Serves a unary OTLP Export call (Logs, Metrics, or Traces). + /// + /// The request body is optionally compressed and contains a single Export request which + /// is forwarded into the pipeline. Depending on `wait_for_result` we either: + /// - wait for an Ack or Nack and convert it to a gRPC status, or + /// - return an empty `Export*ServiceResponse` immediately after enqueueing. + async fn serve_otlp_unary( + self: &Rc, + request: Request, + mut respond: SendResponse, + signal: SignalType, + ack_registry: Option, + ) -> Result<(), Status> { + let (parts, body) = request.into_parts(); + let mut ctx = self.prepare_request(&parts.headers)?; + let mut recv_stream = GrpcStreamingBody::new( + body, + ctx.request_encoding, + Rc::clone(self), + self.max_decoding_message_size as usize, + ); + let mut request_timeout = RequestTimeout::new(self.request_timeout); + + let request_bytes = match request_timeout + .with_future(recv_stream.next_message_bytes()) + .await + { + Ok(Ok(Some(bytes))) => bytes, + Ok(Ok(None)) => { + respond_with_error( + respond, + Status::invalid_argument("missing request body"), + &self.request_accept_header, + ); + return Ok(()); + } + Ok(Err(status)) => { + respond_with_error(respond, status, &self.request_accept_header); + return Ok(()); + } + Err(()) => { + if let Some(duration) = self.request_timeout { + log::debug!("Request timed out after {:?}", duration); + } + respond_with_error( + respond, + Status::deadline_exceeded("request timed out"), + &self.request_accept_header, + ); + return Ok(()); + } + }; + + // Wrap the raw request protobuf bytes in pipeline pdata. + let mut otap_pdata = OtapPdata::new( + Context::default(), + otlp_proto_bytes(signal, request_bytes).into(), + ); + + // Optional Ack tracking depending on `wait_for_result`. + let wait_token = if let Some(state) = ack_registry.as_ref() { + match state.allocate() { + Some(token) => { + self.effect_handler.subscribe_to( + Interests::ACKS | Interests::NACKS, + token.to_calldata(), + &mut otap_pdata, + ); + Some((state.clone(), token)) + } + None => { + respond_with_error( + respond, + Status::resource_exhausted("too many concurrent requests"), + &self.request_accept_header, + ); + return Ok(()); + } + } + } else { + None + }; + + if let Err(err) = self.effect_handler.send_message(otap_pdata).await { + log::error!("Failed to send to pipeline: {err}"); + + if let Some((state, token)) = wait_token { + state.cancel(token); + } + + respond_with_error( + respond, + Status::internal("failed to send to pipeline"), + &self.request_accept_header, + ); + return Ok(()); + } + + // If we are not waiting for results we can respond immediately. + if let Some((state, token)) = wait_token { + let ack_future = AckCompletionFuture::new(token, state); + let ack_result = match request_timeout.with_future(ack_future).await { + Ok(result) => result, + Err(()) => { + if let Some(duration) = self.request_timeout { + log::debug!("Request timed out after {:?}", duration); + } + respond_with_error( + respond, + Status::deadline_exceeded("request timed out"), + &self.request_accept_header, + ); + return Ok(()); + } + }; + + match ack_result { + AckPollResult::Ack => {} + AckPollResult::Nack(reason) => { + respond_with_error( + respond, + Status::unavailable(format!("Pipeline processing failed: {reason}")), + &self.request_accept_header, + ); + return Ok(()); + } + AckPollResult::Cancelled => { + respond_with_error( + respond, + Status::internal("request cancelled"), + &self.request_accept_header, + ); + return Ok(()); + } + } + } + + let mut send_stream = respond + .send_response(ctx.response, false) + .map_err(|e| Status::internal(format!("failed to send response headers: {e}")))?; + + let payload = encode_otlp_response(signal, &mut ctx.response_encoder)?; + if let Err(e) = send_stream.send_data(payload, false) { + log::debug!("send_data failed: {e}"); + return Ok(()); + } + + send_ok_trailers(send_stream); + Ok(()) + } +} + +/// Sends trailers for a successful gRPC response (status code 0). +fn send_ok_trailers(mut stream: h2::SendStream) { + let mut trailers = HeaderMap::new(); + let _ = trailers.insert("grpc-status", HeaderValue::from_static("0")); + if let Err(e) = stream.send_trailers(trailers) { + log::debug!("send_trailers failed: {e}"); + } +} + +/// Sends trailers for a failed gRPC response with the provided status code and message. +fn send_error_trailers(mut stream: h2::SendStream, status: Status) { + let mut trailers = HeaderMap::new(); + let _ = trailers.insert( + "grpc-status", + HeaderValue::from_str(&(status.code() as i32).to_string()) + .unwrap_or_else(|_| HeaderValue::from_static("2")), + ); + if !status.message().is_empty() { + if let Ok(value) = HeaderValue::from_str(status.message()) { + let _ = trailers.insert("grpc-message", value); + } + } + if let Err(e) = stream.send_trailers(trailers) { + log::debug!("send_trailers failed: {e}"); + } +} + +/// Sends a unary gRPC error response with an empty body and error trailers. +/// +/// We always respond with HTTP 200 and encode the error in `grpc-status` and `grpc-message` +/// trailers as per the gRPC spec. +pub(crate) fn respond_with_error( + mut respond: SendResponse, + status: Status, + accept_header: &HeaderValue, +) { + let response = match Response::builder() + .status(HttpStatusCode::OK) + .header("content-type", "application/grpc") + .header("grpc-accept-encoding", accept_header.clone()) + .body(()) + { + Ok(response) => response, + Err(e) => { + log::debug!("failed to build error response: {e}"); + return; + } + }; + + match respond.send_response(response, false) { + Ok(stream) => send_error_trailers(stream, status), + Err(e) => log::debug!("failed to send error response: {e}"), + } +} + +/// Wraps raw OTLP protobuf bytes into the typed enum used by the pipeline. +fn otlp_proto_bytes(signal: SignalType, bytes: Bytes) -> OtlpProtoBytes { + match signal { + SignalType::Logs => OtlpProtoBytes::ExportLogsRequest(bytes), + SignalType::Metrics => OtlpProtoBytes::ExportMetricsRequest(bytes), + SignalType::Traces => OtlpProtoBytes::ExportTracesRequest(bytes), + } +} + +/// Builds the empty `Export*ServiceResponse` payload for a successful OTLP call. +fn encode_otlp_response( + signal: SignalType, + encoder: &mut GrpcResponseFrameEncoder, +) -> Result { + match signal { + SignalType::Logs => encoder.encode(&ExportLogsServiceResponse { + partial_success: None, + }), + SignalType::Metrics => encoder.encode(&ExportMetricsServiceResponse { + partial_success: None, + }), + SignalType::Traces => encoder.encode(&ExportTraceServiceResponse { + partial_success: None, + }), + } +} diff --git a/rust/otap-dataflow/crates/otap/src/otel_receiver/security.md b/rust/otap-dataflow/crates/otap/src/otel_receiver/security.md new file mode 100644 index 0000000000..b7336ac580 --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otel_receiver/security.md @@ -0,0 +1,35 @@ +# Attack Vectors and Mitigations + +## Scope and Thread Model + +This document summarizes the security analysis of the OTEL Receiver component +in the OTAP Dataflow engine. The receiver implements OTLP and OTAP Arrow over a +custom H2 gRPC server. It runs inside a single-threaded runtime per core, +avoids Send and Sync, avoids Arc and Mutex, and minimizes heap allocations. +Because it is internet-facing, it must defend against malformed H2 messages, +flow control abuse, gRPC-level resource exhaustion, compression bombs, and +protocol downgrade attacks. + +The component processes untrusted network traffic, including arbitrary HTTP/2 +frames, arbitrary gRPC frames, compressed payloads, and long-lived streaming +connections. Therefore, an attacker with network access is assumed capable of +sending intentionally malformed or adversarial payloads. + +## Summary of Major Attack Vectors and Mitigations + +The table below summarizes each category: + +| Vector | Description | Mitigation | +| ------------------------------------------ | ----------------------------------------------------------------------------- | -------------------------------------------------------------- | +| Oversized gRPC frames | Huge declared length in 5-byte gRPC prefix that forces unlimited buffering | Enforced global `max_decoding_message_size` and rejected early | +| Compression bombs | zstd, gzip, deflate payloads inflating to large memory footprint | Decompressors capped at `max_decoding_message_size` | +| HTTP/2 flow control bypass | Releasing capacity on read (not consume) allowed unlimited inflight buffering | Release only on actual consumption, reactivating flow control | +| Slowloris H2 handshake | Idle or extremely slow H2 prefaces tying up unlimited connection slots | Handshake timeout | +| Infinite idle H2 connections | Long-lived connections with no active streams | Keepalive and idle cutoff logic | +| Ack registry exhaustion | Leaked registry slots during pipeline errors | Slots cancelled when enqueue fails | +| CPU starvation inside reactor | Long decode or decompress without preemption | Hard size limits and bounded internal buffers | +| Protocol downgrade or invalid content-type | Invalid gRPC Content-Type or compression headers | Strict header validation and early rejection | +| Unknown or unapproved compression methods | "zstdarrowX", raw zstd variants, unknown tokens | Strict whitelist matching `AcceptedGrpcEncodings` | +| Stream flooding | Opening too many H2 streams | Per-connection stream admission and `max_in_flight limit` | +| Pipeline overload | Downstream queue saturation | Ack registry backpressure and overloaded_status | +| Malformed Arrow data | Invalid protobuf or Arrow content stalling stream | Immediate error, closing stream safely | diff --git a/rust/otap-dataflow/crates/otap/src/otel_receiver/status.rs b/rust/otap-dataflow/crates/otap/src/otel_receiver/status.rs new file mode 100644 index 0000000000..9271fa34cb --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otel_receiver/status.rs @@ -0,0 +1,98 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Minimal gRPC-style status type to avoid pulling in tonic for the OTEL receiver. +//! The receiver only needs a handful of status codes plus message formatting to +//! drive trailer generation and logging. + +use std::error::Error; +use std::fmt; + +/// Subset of gRPC status codes used by the lightweight OTEL receiver. +/// +/// This lets us avoid pulling in `tonic` only to represent status values. +#[repr(i32)] +#[derive(Clone, Copy, Debug, PartialEq, Eq)] +pub(crate) enum StatusCode { + #[allow(unused)] + Ok = 0, + #[allow(unused)] + Cancelled = 1, + #[allow(unused)] + Unknown = 2, + InvalidArgument = 3, + DeadlineExceeded = 4, + ResourceExhausted = 8, + Unimplemented = 12, + Internal = 13, + Unavailable = 14, +} + +#[derive(Clone, Debug, PartialEq, Eq)] +pub(crate) struct Status { + code: StatusCode, + message: String, +} + +impl Status { + pub fn new(code: StatusCode, message: impl Into) -> Self { + Self { + code, + message: message.into(), + } + } + + pub fn code(&self) -> StatusCode { + self.code + } + + pub fn message(&self) -> &str { + &self.message + } + + #[allow(unused)] + pub fn into_parts(self) -> (StatusCode, String) { + (self.code, self.message) + } + + pub fn invalid_argument(message: impl Into) -> Self { + Self::new(StatusCode::InvalidArgument, message) + } + + pub fn unimplemented(message: impl Into) -> Self { + Self::new(StatusCode::Unimplemented, message) + } + + pub fn internal(message: impl Into) -> Self { + Self::new(StatusCode::Internal, message) + } + + pub fn deadline_exceeded(message: impl Into) -> Self { + Self::new(StatusCode::DeadlineExceeded, message) + } + + pub fn resource_exhausted(message: impl Into) -> Self { + Self::new(StatusCode::ResourceExhausted, message) + } + + pub fn unavailable(message: impl Into) -> Self { + Self::new(StatusCode::Unavailable, message) + } + + #[allow(unused)] + pub fn cancelled(message: impl Into) -> Self { + Self::new(StatusCode::Cancelled, message) + } +} + +impl fmt::Display for Status { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + if self.message.is_empty() { + write!(f, "{:?}", self.code) + } else { + write!(f, "{:?}: {}", self.code, self.message) + } + } +} + +impl Error for Status {} diff --git a/rust/otap-dataflow/crates/otap/src/otel_receiver/stream.rs b/rust/otap-dataflow/crates/otap/src/otel_receiver/stream.rs new file mode 100644 index 0000000000..394f783a89 --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otel_receiver/stream.rs @@ -0,0 +1,643 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Glue between incoming OTAP Arrow streams and the ACK registry. +//! +//! `StatusStream` fans batches from a gRPC request into the downstream pipeline and +//! fans ACK/NACK notifications back into `BatchStatus` responses. The bounded +//! registry supplies backpressure—when no tokens remain we immediately return an +//! overloaded status and stop enqueueing more work. Each stream also tracks how many +//! batches are currently `in_flight`, so a single connection can’t monopolize every +//! token. Both limits interact with the rest of the system by gating how much work +//! we hand the pipeline; excess demand is reflected back to the client. Everything +//! runs on the single-threaded runtime, so the implementation uses `Rc`/`RefCell` and futures +//! that never cross threads, operations such as enqueueing batches, polling inflight +//! futures, and reclaiming registry slots are all O(1), keeping the hot path +//! predictable under load. + +use super::ack::{ + AckPollResult, AckRegistry, AckToken, nack_status, overloaded_status, success_status, +}; +use super::grpc::RequestStream; +use crate::otel_receiver::status::Status; +use crate::pdata::{Context, OtapPdata}; +use futures::Stream; +use futures::future::{LocalBoxFuture, poll_fn}; +use futures::stream::FuturesUnordered; +use log::error; +use otap_df_engine::local::receiver as local; +use otap_df_engine::{Interests, ProducerEffectHandlerExtension}; +use otap_df_pdata::Consumer; +use otap_df_pdata::otap::{OtapArrowRecords, OtapBatchStore, from_record_messages}; +use otap_df_pdata::proto::opentelemetry::arrow::v1::{BatchArrowRecords, BatchStatus}; +use std::future::Future; +use std::marker::PhantomData; +use std::pin::Pin; +use std::task::{Context as TaskContext, Poll}; + +/// Builds the stream of OTAP batch statuses for a single gRPC request. +pub(crate) fn stream_batch_statuses( + input_stream: S, + effect_handler: local::EffectHandler, + ack_registry: Option, + otap_batch: F, + max_in_flight_per_connection: usize, +) -> StatusStream +where + S: RequestStream + Unpin + 'static, + T: OtapBatchStore + 'static, + F: Fn(T) -> OtapArrowRecords + Send + Copy + 'static + Unpin, +{ + let state = StatusStreamState::new( + input_stream, + effect_handler, + ack_registry, + otap_batch, + max_in_flight_per_connection, + ); + StatusStream::new(state) +} + +/// Drives an inbound OTAP stream while waiting for ACK or NACK outcomes. +/// +/// Each `StatusStream` instance is bound to a single gRPC stream and lives +/// entirely on the local executor. It pushes decoded batches into the pipeline +/// and yields `BatchStatus` responses as soon as the corresponding ACK futures +/// complete. +pub(crate) struct StatusStream +where + S: RequestStream + Unpin + 'static, + T: OtapBatchStore + 'static, + F: Fn(T) -> OtapArrowRecords + Send + Copy + 'static + Unpin, +{ + state: Option>, + pending: Option, StreamStep)>>, + finished: bool, +} + +impl StatusStream +where + S: RequestStream + Unpin + 'static, + T: OtapBatchStore + 'static, + F: Fn(T) -> OtapArrowRecords + Send + Copy + 'static + Unpin, +{ + /// Wraps the prepared state in the facade consumed by the router. + fn new(state: StatusStreamState) -> Self { + Self { + state: Some(state), + pending: None, + finished: false, + } + } + + /// Drives the state machine until it produces the next `StreamStep`. + /// + /// Internally this means "fill" until we either enqueue more work or hit an + /// error, and if that doesn't yield anything new we "drain" by awaiting the + /// next inflight ACK future. + fn drive_next( + state: StatusStreamState, + ) -> LocalBoxFuture<'static, (StatusStreamState, StreamStep)> { + Box::pin(async move { + let mut state = state; + let step = state.next_item().await; + (state, step) + }) + } +} + +impl Stream for StatusStream +where + S: RequestStream + Unpin + 'static, + T: OtapBatchStore + 'static, + F: Fn(T) -> OtapArrowRecords + Send + Copy + 'static + Unpin, +{ + type Item = Result; + + /// Implements the `Stream` contract by repeatedly driving the state machine. + fn poll_next(self: Pin<&mut Self>, cx: &mut TaskContext<'_>) -> Poll> { + let this = self.get_mut(); + + if this.finished { + return Poll::Ready(None); + } + + if this.pending.is_none() { + // Lazily grab ownership of the state the first time we are polled. If it + // is already `None` we know the stream is complete. + let state = match this.state.take() { + Some(state) => state, + None => { + this.finished = true; + return Poll::Ready(None); + } + }; + // Kick off an async step that will either enqueue more work or drain an inflight future. + this.pending = Some(Self::drive_next(state)); + } + + match this + .pending + .as_mut() + .expect("pending future must exist") + .as_mut() + .poll(cx) + { + Poll::Pending => Poll::Pending, + Poll::Ready((state, step)) => { + this.pending = None; + match step { + StreamStep::Yield(item) => { + // Save the updated state and yield the status/error to the caller. + this.state = Some(state); + Poll::Ready(Some(item)) + } + StreamStep::Done => { + // No more work; mark finished and drop the state. + this.finished = true; + this.state = None; + Poll::Ready(None) + } + } + } + } + } +} + +/// Mutable state for a single `StatusStream`. +/// +/// Holds: +/// - the decoded gRPC input stream, +/// - the local effect handler into the pipeline, +/// - an optional `AckRegistry` for wait for result mode, +/// - a bounded set of in flight ACK wait futures, and +/// - a per connection `max_in_flight` limit that prevents a single client +/// from monopolizing all wait slots. +struct StatusStreamState +where + S: RequestStream + Unpin + 'static, + T: OtapBatchStore + 'static, + F: Fn(T) -> OtapArrowRecords + Send + Copy + 'static + Unpin, +{ + input_stream: S, + consumer: Consumer, + effect_handler: local::EffectHandler, + state: Option, + otap_batch: F, + in_flight: InFlightSet, + max_in_flight: usize, + finished: bool, + _marker: PhantomData T>, +} + +/// What the stream should do next. +enum StreamStep { + /// Emit a `BatchStatus` (or gRPC error) downstream. + Yield(Result), + /// Tear down the stream. + /// No more messages will be produced. + Done, +} + +/// Result of attempting to enqueue a batch into the pipeline. +enum PreparedBatch { + /// The batch was queued and we should continue filling/draining. + Enqueued, + /// The batch triggered an immediate status (success/failure) or termination. + Immediate(StreamStep), +} + +impl StatusStreamState +where + S: RequestStream + Unpin + 'static, + T: OtapBatchStore + 'static, + F: Fn(T) -> OtapArrowRecords + Send + Copy + 'static + Unpin, +{ + /// Creates state for a single inbound connection/request. + fn new( + input_stream: S, + effect_handler: local::EffectHandler, + state: Option, + otap_batch: F, + max_in_flight_per_connection: usize, + ) -> Self { + Self { + input_stream, + consumer: Consumer::default(), + effect_handler, + state, + otap_batch, + in_flight: InFlightSet::with_capacity(max_in_flight_per_connection.max(1)), + max_in_flight: max_in_flight_per_connection.max(1), + finished: false, + _marker: PhantomData, + } + } + + /// Pulls the next work item by either filling or draining the pipeline. + async fn next_item(&mut self) -> StreamStep { + if let Some(step) = self.fill_inflight().await { + return step; + } + + match poll_fn(|cx| self.in_flight.poll_next(cx)).await { + Some(step) => { + if matches!(step, StreamStep::Done) { + self.finished = true; + } + step + } + None => StreamStep::Done, + } + } + + /// Attempts to enqueue additional batches while respecting capacity limits. + /// + /// At most `max_in_flight` iterations and each operation is O(1), so the loop + /// remains bounded even when the inbound stream is eager. + async fn fill_inflight(&mut self) -> Option { + while !self.finished && self.in_flight.len() < self.max_in_flight { + match self.input_stream.next_message().await { + Ok(Some(batch)) => match self.enqueue_batch(batch).await { + PreparedBatch::Enqueued => continue, + PreparedBatch::Immediate(step) => return Some(step), + }, + Ok(None) => { + self.finished = true; + break; + } + Err(status) => { + self.finished = true; + return Some(StreamStep::Yield(Err(status))); + } + } + } + None + } + + /// Converts an incoming `BatchArrowRecords` into pipeline work plus wait token. + /// + /// Aside from the actual pipeline send (which is async), all bookkeeping here + /// is O(1): decoding, registry allocation, and inflight bookkeeping simply + /// index into fixed-size structures. + async fn enqueue_batch(&mut self, mut batch: BatchArrowRecords) -> PreparedBatch { + let batch_id = batch.batch_id; + + // Decode the batch. Because this receiver pulls everything onto a single + // thread, there is no concurrent mutation of `batch` after this point. + let batch = match self.consumer.consume_bar(&mut batch) { + Ok(batch) => batch, + Err(e) => { + error!("Error decoding OTAP Batch: {e:?}. Closing stream"); + self.finished = true; + return PreparedBatch::Immediate(StreamStep::Done); + } + }; + + let batch = from_record_messages::(batch); + let otap_batch_as_otap_arrow_records = (self.otap_batch)(batch); + let mut otap_pdata = + OtapPdata::new(Context::default(), otap_batch_as_otap_arrow_records.into()); + + let wait_token = if let Some(state) = self.state.clone() { + match state.allocate() { + None => { + error!("Too many concurrent requests"); + return PreparedBatch::Immediate(StreamStep::Yield(Ok(overloaded_status( + batch_id, + )))); + } + Some(token) => { + self.effect_handler.subscribe_to( + Interests::ACKS | Interests::NACKS, + token.to_calldata(), + &mut otap_pdata, + ); + Some((state, token)) + } + } + } else { + None + }; + + // Push the batch into the downstream pipeline. This is the only `.await` + // in the method and will yield until the local channel accepts the data. + if let Err(e) = self.effect_handler.send_message(otap_pdata).await { + error!("Failed to send to pipeline: {e}"); + + if let Some((state, token)) = wait_token { + state.cancel(token); + } + + self.finished = true; + return PreparedBatch::Immediate(StreamStep::Done); + }; + + if let Some((state, token)) = wait_token { + if let Err(_future) = self + .in_flight + .push(AckWaitFuture::new(batch_id, token, state)) + { + error!("In-flight future set unexpectedly full"); + return PreparedBatch::Immediate(StreamStep::Yield(Ok(overloaded_status( + batch_id, + )))); + } + PreparedBatch::Enqueued + } else { + PreparedBatch::Immediate(StreamStep::Yield(Ok(success_status(batch_id)))) + } + } +} + +/// Bounded collection of ACK wait futures that enforces per stream inflight limits. +/// +/// All operations are O(1) and polling is delegated to `FuturesUnordered`. +struct InFlightSet { + futures: FuturesUnordered, + capacity: usize, +} + +impl InFlightSet { + /// Creates a set that can hold up to `capacity` futures. + fn with_capacity(capacity: usize) -> Self { + Self { + futures: FuturesUnordered::new(), + capacity, + } + } + + /// Returns the number of currently tracked futures. + fn len(&self) -> usize { + self.futures.len() + } + + /// Attempts to push a future, returning it back if the set is full. + fn push(&mut self, future: F) -> Result<(), F> { + if self.len() >= self.capacity { + Err(future) + } else { + self.futures.push(future); + Ok(()) + } + } + + /// Polls the underlying futures, forwarding readiness to the caller. + fn poll_next(&mut self, cx: &mut TaskContext<'_>) -> Poll::Output>> + where + F: Future + Unpin, + { + Pin::new(&mut self.futures).poll_next(cx) + } +} + +/// Future that resolves once a specific batch receives an ACK or NACK. +struct AckWaitFuture { + batch_id: i64, + token: AckToken, + state: AckRegistry, + completed: bool, +} + +impl AckWaitFuture { + /// Builds a wait future tied to the provided registry token. + fn new(batch_id: i64, token: AckToken, state: AckRegistry) -> Self { + Self { + batch_id, + token, + state, + completed: false, + } + } +} + +impl Future for AckWaitFuture { + type Output = StreamStep; + + /// Resolves once the registry slot finishes with ACK/NACK/cancelled. + fn poll(self: Pin<&mut Self>, cx: &mut TaskContext<'_>) -> Poll { + let this = self.get_mut(); + match this.state.poll_slot(this.token, cx) { + Poll::Pending => Poll::Pending, + Poll::Ready(AckPollResult::Ack) => { + this.completed = true; + Poll::Ready(StreamStep::Yield(Ok(success_status(this.batch_id)))) + } + Poll::Ready(AckPollResult::Nack(reason)) => { + this.completed = true; + Poll::Ready(StreamStep::Yield(Ok(nack_status(this.batch_id, reason)))) + } + Poll::Ready(AckPollResult::Cancelled) => { + this.completed = true; + Poll::Ready(StreamStep::Done) + } + } + } +} + +impl Drop for AckWaitFuture { + /// Ensures the registry slot is released if the future is dropped early. + fn drop(&mut self) { + if !self.completed { + self.state.cancel(self.token); + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::otap_mock::create_otap_batch; + use crate::otel_receiver::ack::{AckRegistry, AckToken}; + use crate::otel_receiver::grpc::RequestStream; + use crate::otel_receiver::status::Status; + use crate::pdata::OtapPdata; + use async_trait::async_trait; + use futures::StreamExt; + use otap_df_channel::mpsc; + use otap_df_config::PortName; + use otap_df_engine::control::pipeline_ctrl_msg_channel; + use otap_df_engine::local::message::LocalSender; + use otap_df_engine::local::receiver as local; + use otap_df_engine::testing::test_node; + use otap_df_pdata::Producer; + use otap_df_pdata::otap::{Logs, Metrics, OtapArrowRecords, OtapBatchStore, Traces}; + use otap_df_pdata::proto::opentelemetry::arrow::v1::{ + ArrowPayloadType, BatchArrowRecords, StatusCode as ProtoStatusCode, + }; + use otap_df_telemetry::reporter::MetricsReporter; + use std::collections::{HashMap, HashSet, VecDeque}; + use tokio::task::yield_now; + use tokio::time::Duration; + + struct FakeArrowStream { + batches: VecDeque, + } + + impl FakeArrowStream { + fn new(batches: VecDeque) -> Self { + Self { batches } + } + } + + #[async_trait(?Send)] + impl RequestStream for FakeArrowStream { + async fn next_message(&mut self) -> Result, Status> { + Ok(self.batches.pop_front()) + } + } + + fn build_test_effect_handler( + channel_capacity: usize, + ) -> (local::EffectHandler, mpsc::Receiver) { + let (tx, rx) = mpsc::Channel::new(channel_capacity); + let mut senders = HashMap::new(); + let default_port: PortName = PortName::from("default"); + let _ = senders.insert(default_port.clone(), LocalSender::MpscSender(tx)); + let (ctrl_tx, _ctrl_rx) = pipeline_ctrl_msg_channel(4); + let (_metrics_rx, metrics_reporter) = MetricsReporter::create_new_and_receiver(1); + let effect_handler = local::EffectHandler::new( + test_node("otel_receiver_status_test"), + senders, + Some(default_port), + ctrl_tx, + metrics_reporter, + ); + (effect_handler, rx) + } + + fn arrow_batches( + payload_type: ArrowPayloadType, + batch_count: usize, + ) -> VecDeque { + let mut queue = VecDeque::with_capacity(batch_count); + let mut producer = Producer::new(); + for batch_index in 0..batch_count { + let mut batch = create_otap_batch(batch_index as i64, payload_type); + let bar = producer + .produce_bar(&mut batch) + .expect("failed to encode arrow batch"); + queue.push_back(bar); + } + queue + } + + async fn drive_ack_pipeline( + pdata_rx: mpsc::Receiver, + ack_registry: AckRegistry, + total_batches: usize, + ) -> (usize, usize) { + let mut success = 0; + let mut failure = 0; + for idx in 0..total_batches { + let pdata = pdata_rx + .recv() + .await + .expect("pdata channel closed unexpectedly"); + let calldata = pdata + .current_calldata() + .expect("missing calldata for wait_for_result"); + let token = AckToken::from_calldata(&calldata).expect("invalid ack token"); + + if idx % 11 == 0 { + tokio::time::sleep(Duration::from_micros(((idx % 5) + 1) as u64 * 20)).await; + let _ = ack_registry.complete(token, Err(format!("failure #{idx}"))); + failure += 1; + } else { + if idx % 7 == 0 { + tokio::time::sleep(Duration::from_micros(((idx % 3) + 1) as u64 * 10)).await; + } else if idx % 3 == 0 { + yield_now().await; + } + let _ = ack_registry.complete(token, Ok(())); + success += 1; + } + } + (success, failure) + } + + async fn run_status_stream_load_test( + payload_type: ArrowPayloadType, + otap_batch: fn(T) -> OtapArrowRecords, + ) where + T: OtapBatchStore + 'static, + { + const TOTAL_BATCHES: usize = 1024; + const MAX_CONCURRENT_REQUESTS: usize = 256; + const MAX_IN_FLIGHT: usize = 64; + + let stream = FakeArrowStream::new(arrow_batches(payload_type, TOTAL_BATCHES)); + let (effect_handler, pdata_rx) = build_test_effect_handler(TOTAL_BATCHES); + let ack_registry = AckRegistry::new(MAX_CONCURRENT_REQUESTS); + + let mut status_stream = stream_batch_statuses::<_, T, _>( + stream, + effect_handler, + Some(ack_registry.clone()), + otap_batch, + MAX_IN_FLIGHT, + ); + + let ack_task = drive_ack_pipeline(pdata_rx, ack_registry.clone(), TOTAL_BATCHES); + + let status_task = async { + let mut successes = 0; + let mut failures = 0; + let mut ids = HashSet::with_capacity(TOTAL_BATCHES); + while let Some(next) = status_stream.next().await { + let status = next.expect("receiver should not emit tonic errors"); + assert!( + ids.insert(status.batch_id), + "duplicate status for batch {}", + status.batch_id + ); + match status.status_code { + code if code == ProtoStatusCode::Ok as i32 => { + assert_eq!(status.status_message, "Successfully received"); + successes += 1; + } + code if code == ProtoStatusCode::Unavailable as i32 => { + assert!( + status + .status_message + .starts_with("Pipeline processing failed:"), + "unexpected failure message {}", + status.status_message + ); + failures += 1; + } + other => panic!("unexpected status code {other}"), + } + } + assert_eq!(ids.len(), TOTAL_BATCHES); + (successes, failures) + }; + + let ((expected_successes, expected_failures), (actual_successes, actual_failures)) = + tokio::join!(ack_task, status_task); + + assert_eq!(actual_successes, expected_successes); + assert_eq!(actual_failures, expected_failures); + assert_eq!(actual_successes + actual_failures, TOTAL_BATCHES); + } + + #[tokio::test(flavor = "current_thread")] + async fn stream_batch_statuses_handles_large_ack_nack_load() { + run_status_stream_load_test::( + ArrowPayloadType::MultivariateMetrics, + OtapArrowRecords::Metrics, + ) + .await; + } + + #[tokio::test(flavor = "current_thread")] + async fn stream_batch_statuses_handles_large_logs_load() { + run_status_stream_load_test::(ArrowPayloadType::Logs, OtapArrowRecords::Logs).await; + } + + #[tokio::test(flavor = "current_thread")] + async fn stream_batch_statuses_handles_large_traces_load() { + run_status_stream_load_test::(ArrowPayloadType::Spans, OtapArrowRecords::Traces) + .await; + } +} diff --git a/rust/otap-dataflow/crates/otap/src/otel_receiver/test_common.rs b/rust/otap-dataflow/crates/otap/src/otel_receiver/test_common.rs new file mode 100644 index 0000000000..bc58d5f50d --- /dev/null +++ b/rust/otap-dataflow/crates/otap/src/otel_receiver/test_common.rs @@ -0,0 +1,161 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +use crate::pdata::OtapPdata; +use otap_df_pdata::OtlpProtoBytes; +use otap_df_pdata::proto::OtlpProtoMessage; +use otap_df_pdata::proto::opentelemetry::collector::logs::v1::ExportLogsServiceRequest; +use otap_df_pdata::proto::opentelemetry::collector::metrics::v1::ExportMetricsServiceRequest; +use otap_df_pdata::proto::opentelemetry::collector::trace::v1::ExportTraceServiceRequest; +use otap_df_pdata::proto::opentelemetry::logs::v1::LogsData; +use otap_df_pdata::proto::opentelemetry::metrics::v1::MetricsData; +use otap_df_pdata::proto::opentelemetry::trace::v1::TracesData; +use prost::Message; +use weaver_forge::registry::{ResolvedGroup, ResolvedRegistry}; +use weaver_resolved_schema::attribute::Attribute; +use weaver_semconv::attribute::{AttributeType, PrimitiveOrArrayTypeSpec, RequirementLevel}; +use weaver_semconv::group::{GroupType, InstrumentSpec, SpanKindSpec}; + +/// Build a small semantic-conventions registry for generating fake signals in tests. +pub fn build_test_registry() -> ResolvedRegistry { + let string_attribute = Attribute { + name: "service.name".to_string(), + r#type: AttributeType::PrimitiveOrArray(PrimitiveOrArrayTypeSpec::String), + brief: "service name".to_string(), + examples: None, + tag: None, + requirement_level: RequirementLevel::default(), + sampling_relevant: None, + note: String::new(), + stability: None, + deprecated: None, + prefix: false, + tags: None, + annotations: None, + value: None, + role: None, + }; + let int_attribute = Attribute { + name: "example.counter".to_string(), + r#type: AttributeType::PrimitiveOrArray(PrimitiveOrArrayTypeSpec::Int), + brief: "example counter".to_string(), + examples: None, + tag: None, + requirement_level: RequirementLevel::default(), + sampling_relevant: None, + note: String::new(), + stability: None, + deprecated: None, + prefix: false, + tags: None, + annotations: None, + value: None, + role: None, + }; + + let span_group = ResolvedGroup { + id: "fake_span".to_string(), + r#type: GroupType::Span, + brief: "test span group".to_string(), + note: String::new(), + prefix: String::new(), + extends: None, + stability: None, + deprecated: None, + attributes: vec![string_attribute.clone()], + span_kind: Some(SpanKindSpec::Server), + events: vec!["fake.event".to_string()], + metric_name: None, + instrument: None, + unit: None, + name: None, + lineage: None, + display_name: None, + body: None, + entity_associations: Vec::new(), + annotations: None, + }; + + let metric_group = ResolvedGroup { + id: "fake_metric".to_string(), + r#type: GroupType::Metric, + brief: "test metric group".to_string(), + note: String::new(), + prefix: String::new(), + extends: None, + stability: None, + deprecated: None, + attributes: vec![string_attribute.clone(), int_attribute], + span_kind: None, + events: Vec::new(), + metric_name: Some("requests.count".to_string()), + instrument: Some(InstrumentSpec::Counter), + unit: Some("1".to_string()), + name: None, + lineage: None, + display_name: None, + body: None, + entity_associations: Vec::new(), + annotations: None, + }; + + let event_group = ResolvedGroup { + id: "fake_event".to_string(), + r#type: GroupType::Event, + brief: "test event group".to_string(), + note: String::new(), + prefix: String::new(), + extends: None, + stability: None, + deprecated: None, + attributes: vec![string_attribute], + span_kind: None, + events: Vec::new(), + metric_name: None, + instrument: None, + unit: None, + name: Some("app.log".to_string()), + lineage: None, + display_name: None, + body: None, + entity_associations: Vec::new(), + annotations: None, + }; + + ResolvedRegistry { + registry_url: "test://otlp_fake_registry".to_string(), + groups: vec![span_group, metric_group, event_group], + } +} + +/// Decode an `OtapPdata` payload back into an OTLP message variant for equivalence checks. +pub fn decode_pdata_to_message(pdata: &OtapPdata) -> OtlpProtoMessage { + let proto_bytes: OtlpProtoBytes = pdata + .clone() + .payload() + .try_into() + .expect("convert to proto bytes"); + match proto_bytes { + OtlpProtoBytes::ExportLogsRequest(bytes) => { + let request = + ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode logs payload"); + OtlpProtoMessage::Logs(LogsData { + resource_logs: request.resource_logs, + }) + } + OtlpProtoBytes::ExportMetricsRequest(bytes) => { + let request = ExportMetricsServiceRequest::decode(bytes.as_ref()) + .expect("decode metrics payload"); + OtlpProtoMessage::Metrics(MetricsData { + resource_metrics: request.resource_metrics, + }) + } + OtlpProtoBytes::ExportTracesRequest(bytes) => { + let request = + ExportTraceServiceRequest::decode(bytes.as_ref()).expect("decode traces payload"); + OtlpProtoMessage::Traces(TracesData { + resource_spans: request.resource_spans, + }) + } + } +} diff --git a/rust/otap-dataflow/crates/otap/src/otlp_exporter.rs b/rust/otap-dataflow/crates/otap/src/otlp_exporter.rs index 9872c543f9..19928d0ffe 100644 --- a/rust/otap-dataflow/crates/otap/src/otlp_exporter.rs +++ b/rust/otap-dataflow/crates/otap/src/otlp_exporter.rs @@ -1,12 +1,22 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 +//! Asynchronous OTLP exporter implementation. +//! +//! The exporter receives pipeline messages on a single-threaded Tokio runtime. Each payload is +//! encoded (when necessary) and handed off to a gRPC export RPC. We keep the gRPC futures in a +//! lightweight in-flight queue which enforces the configured concurrency limit. As soon as a +//! request finishes we forward the Ack/Nack to the pipeline controller so the dataflow can make +//! progress. + use crate::OTAP_EXPORTER_FACTORIES; -use crate::compression::CompressionMethod; use crate::metrics::ExporterPDataMetrics; +use crate::otap_grpc::GrpcClientSettings; use crate::otap_grpc::otlp::client::{LogsServiceClient, MetricsServiceClient, TraceServiceClient}; use crate::pdata::{Context, OtapPdata}; use async_trait::async_trait; +use bytes::Bytes; +use futures::ready; use linkme::distributed_slice; use otap_df_config::SignalType; use otap_df_config::node::NodeUserConfig; @@ -25,11 +35,16 @@ use otap_df_pdata::otlp::logs::LogsProtoBytesEncoder; use otap_df_pdata::otlp::metrics::MetricsProtoBytesEncoder; use otap_df_pdata::otlp::traces::TracesProtoBytesEncoder; use otap_df_pdata::otlp::{ProtoBuffer, ProtoBytesEncoder}; -use otap_df_pdata::{OtapPayload, OtapPayloadHelpers, OtlpProtoBytes}; +use otap_df_pdata::{OtapArrowRecords, OtapPayload, OtapPayloadHelpers, OtlpProtoBytes}; use otap_df_telemetry::metrics::MetricSet; use serde::Deserialize; +use std::future::Future; +use std::mem; +use std::pin::Pin; use std::sync::Arc; +use std::task::{Context as TaskContext, Poll}; use std::time::Duration; +use tonic::codec::CompressionEncoding; use tonic::transport::Channel; /// The URN for the OTLP exporter @@ -39,14 +54,16 @@ pub const OTLP_EXPORTER_URN: &str = "urn:otel:otlp:exporter"; #[derive(Debug, Deserialize)] #[serde(deny_unknown_fields)] pub struct Config { - /// The gRPC endpoint to connect to - pub grpc_endpoint: String, - /// The compression method to use for the gRPC connection - pub compression_method: Option, - /// Timeout for RPC requests. If not specified, no timeout is applied. - /// Format: humantime format (e.g., "30s", "5m", "1h", "500ms") - #[serde(default, with = "humantime_serde")] - pub timeout: Option, + /// Shared gRPC client settings reused across OTLP exports. + #[serde(flatten)] + pub grpc: GrpcClientSettings, + /// Maximum number of concurrent in-flight export RPCs. + #[serde(default = "default_max_in_flight")] + pub max_in_flight: usize, +} + +const fn default_max_in_flight() -> usize { + 32 } /// Exporter that sends OTLP data via gRPC @@ -104,7 +121,7 @@ impl Exporter for OTLPExporter { effect_handler .info(&format!( "Exporting OTLP traffic to endpoint: {}", - self.config.grpc_endpoint + self.config.grpc.grpc_endpoint )) .await; @@ -113,166 +130,198 @@ impl Exporter for OTLPExporter { .start_periodic_telemetry(Duration::from_secs(1)) .await?; - let mut endpoint = - Channel::from_shared(self.config.grpc_endpoint.clone()).map_err(|e| { - let source_detail = format_error_sources(&e); - Error::ExporterError { - exporter: exporter_id.clone(), - kind: ExporterErrorKind::Connect, - error: format!("grpc channel error {e}"), - source_detail, - } - })?; - - // Apply timeout if configured - if let Some(timeout) = self.config.timeout { - endpoint = endpoint.timeout(timeout); - } + let endpoint = self.config.grpc.build_endpoint().map_err(|e| { + let source_detail = format_error_sources(&e); + Error::ExporterError { + exporter: exporter_id.clone(), + kind: ExporterErrorKind::Connect, + error: format!("grpc channel error {e}"), + source_detail, + } + })?; let channel = endpoint.connect_lazy(); - // start a grpc client and connect to the server - let mut metrics_client = MetricsServiceClient::new(channel.clone()); - let mut logs_client = LogsServiceClient::new(channel.clone()); - let mut trace_client = TraceServiceClient::new(channel.clone()); - - if let Some(ref compression) = self.config.compression_method { - let encoding = compression.map_to_compression_encoding(); - - logs_client = logs_client - .send_compressed(encoding) - .accept_compressed(encoding); - metrics_client = metrics_client - .send_compressed(encoding) - .accept_compressed(encoding); - trace_client = trace_client - .send_compressed(encoding) - .accept_compressed(encoding); - } + let compression = self.config.grpc.compression_encoding(); + let max_in_flight = self.config.max_in_flight.max(1); - // reuse the encoder and the buffer across pdatas let mut logs_encoder = LogsProtoBytesEncoder::new(); let mut metrics_encoder = MetricsProtoBytesEncoder::new(); let mut traces_encoder = TracesProtoBytesEncoder::new(); let mut proto_buffer = ProtoBuffer::new(); + let mut client_pool = ClientPool::new(channel, compression); + let mut inflight = InFlightQueue::new(); + let mut pending_msg: Option> = None; + // Main receive/export loop: respect the concurrency budget, push new exports, + // and feed completions back to the pipeline. loop { - match msg_chan.recv().await? { - Message::Control(NodeControlMsg::Shutdown { deadline, .. }) => { - _ = timer_cancel_handle.cancel().await; - return Ok(TerminalState::new(deadline, [self.pdata_metrics])); + if inflight.len() >= max_in_flight && pending_msg.is_some() { + if let Some(completed) = inflight.next_completion().await { + let client = process_completed_export( + completed, + &effect_handler, + &mut self.pdata_metrics, + ) + .await; + client_pool.release(client); } - Message::Control(NodeControlMsg::CollectTelemetry { - mut metrics_reporter, - }) => { - _ = metrics_reporter.report(&mut self.pdata_metrics); + continue; + } + + tokio::select! { + biased; + Some(completed) = inflight.next_completion(), if !inflight.is_empty() => { + // A gRPC export finished; surface the result to the pipeline before + // accepting more work so we honour backpressure and ordered Acks. + let client = process_completed_export( + completed, + &effect_handler, + &mut self.pdata_metrics, + ) + .await; + client_pool.release(client); } - Message::PData(pdata) => { - // Capture signal type before moving pdata into try_from - let signal_type = pdata.signal_type(); - - // Keep context for Ack/Nack delivery - let (context, payload) = pdata.into_parts(); - self.pdata_metrics.inc_consumed(signal_type); - - match (signal_type, payload) { - // use optimized direct encoding OTAP -> OTLP bytes directly - (SignalType::Logs, OtapPayload::OtapArrowRecords(otap_batch)) => { - match handle_otap_export( - otap_batch, - context, - &mut proto_buffer, - &mut logs_encoder, - &mut logs_client, - &effect_handler, - ) - .await - { - Ok(()) => self.pdata_metrics.logs_exported.inc(), - _ => self.pdata_metrics.logs_failed.inc(), + msg = async { + if let Some(msg) = pending_msg.take() { + Ok(msg) + } else { + msg_chan.recv().await + } + } => { + let msg = msg?; + + match msg { + Message::Control(NodeControlMsg::Shutdown { deadline, .. }) => { + debug_assert!( + pending_msg.is_none(), + "pending message should have been drained before shutdown" + ); + while !inflight.is_empty() { + if let Some(completed) = inflight.next_completion().await { + let client = process_completed_export( + completed, + &effect_handler, + &mut self.pdata_metrics, + ) + .await; + client_pool.release(client); + } } + _ = timer_cancel_handle.cancel().await; + return Ok(TerminalState::new(deadline, [self.pdata_metrics])); } - (SignalType::Metrics, OtapPayload::OtapArrowRecords(otap_batch)) => { - match handle_otap_export( - otap_batch, - context, - &mut proto_buffer, - &mut metrics_encoder, - &mut metrics_client, - &effect_handler, - ) - .await - { - Ok(()) => self.pdata_metrics.metrics_exported.inc(), - _ => self.pdata_metrics.metrics_failed.inc(), - } + Message::Control(NodeControlMsg::CollectTelemetry { metrics_reporter }) => { + _ = metrics_reporter.report(&mut self.pdata_metrics); } - (SignalType::Traces, OtapPayload::OtapArrowRecords(otap_batch)) => { - match handle_otap_export( - otap_batch, - context, - &mut proto_buffer, - &mut traces_encoder, - &mut trace_client, - &effect_handler, - ) - .await - { - Ok(()) => self.pdata_metrics.traces_exported.inc(), - _ => self.pdata_metrics.traces_failed.inc(), + Message::PData(pdata) => { + if inflight.len() >= max_in_flight { + pending_msg = Some(Message::PData(pdata)); + continue; } - } - (_, OtapPayload::OtlpBytes(service_req)) => { - _ = match service_req { - OtlpProtoBytes::ExportLogsRequest(bytes) => { - match handle_otlp_export( - bytes, + + let signal_type = pdata.signal_type(); + let (context, payload) = pdata.into_parts(); + self.pdata_metrics.inc_consumed(signal_type); + + // Dispatch based on signal type and the concrete payload representation. + match (signal_type, payload) { + (SignalType::Logs, OtapPayload::OtapArrowRecords(otap_batch)) => { + match prepare_otap_export( + otap_batch, context, - &mut logs_client, - &effect_handler, - |b| OtlpProtoBytes::ExportLogsRequest(b.to_vec()).into(), - ) - .await - { - Ok(()) => self.pdata_metrics.logs_exported.inc(), - _ => self.pdata_metrics.logs_failed.inc(), + &mut proto_buffer, + &mut logs_encoder, + exporter_id.clone(), + SignalType::Logs, + ) { + Ok(prepared) => { + let future = + make_export_future(prepared, &mut client_pool); + inflight.push(future); + } + Err(_) => { + self.pdata_metrics.logs_failed.inc(); + } } } - OtlpProtoBytes::ExportMetricsRequest(bytes) => { - match handle_otlp_export( - bytes, + (SignalType::Metrics, OtapPayload::OtapArrowRecords(otap_batch)) => { + match prepare_otap_export( + otap_batch, context, - &mut metrics_client, - &effect_handler, - |b| OtlpProtoBytes::ExportMetricsRequest(b.to_vec()).into(), - ) - .await - { - Ok(()) => self.pdata_metrics.metrics_exported.inc(), - _ => self.pdata_metrics.metrics_failed.inc(), + &mut proto_buffer, + &mut metrics_encoder, + exporter_id.clone(), + SignalType::Metrics, + ) { + Ok(prepared) => { + let future = + make_export_future(prepared, &mut client_pool); + inflight.push(future); + } + Err(_) => { + self.pdata_metrics.metrics_failed.inc(); + } } } - OtlpProtoBytes::ExportTracesRequest(bytes) => { - match handle_otlp_export( - bytes, + (SignalType::Traces, OtapPayload::OtapArrowRecords(otap_batch)) => { + match prepare_otap_export( + otap_batch, context, - &mut trace_client, - &effect_handler, - |b| OtlpProtoBytes::ExportTracesRequest(b.to_vec()).into(), - ) - .await - { - Ok(()) => self.pdata_metrics.traces_exported.inc(), - _ => self.pdata_metrics.traces_failed.inc(), + &mut proto_buffer, + &mut traces_encoder, + exporter_id.clone(), + SignalType::Traces, + ) { + Ok(prepared) => { + let future = + make_export_future(prepared, &mut client_pool); + inflight.push(future); + } + Err(_) => { + self.pdata_metrics.traces_failed.inc(); + } } } - }; + (_, OtapPayload::OtlpBytes(service_req)) => { + let prepared = match service_req { + OtlpProtoBytes::ExportLogsRequest(bytes) => { + prepare_otlp_export( + bytes, + context, + SignalType::Logs, + |b| OtlpProtoBytes::ExportLogsRequest(b).into(), + ) + } + OtlpProtoBytes::ExportMetricsRequest(bytes) => { + prepare_otlp_export( + bytes, + context, + SignalType::Metrics, + |b| OtlpProtoBytes::ExportMetricsRequest(b).into(), + ) + } + OtlpProtoBytes::ExportTracesRequest(bytes) => { + prepare_otlp_export( + bytes, + context, + SignalType::Traces, + |b| OtlpProtoBytes::ExportTracesRequest(b).into(), + ) + } + }; + + let future = + make_export_future(prepared, &mut client_pool); + inflight.push(future); + } + } + } + _ => { + // ignore unhandled messages } } } - _ => { - // ignore unhandled messages - } } } } @@ -311,73 +360,325 @@ async fn handle_export_result( } } -/// Generic function for encoding OTAP records to protobuf, exporting via gRPC, -/// and handling Ack/Nack delivery. -async fn handle_otap_export( - mut otap_batch: otap_df_pdata::otap::OtapArrowRecords, +struct PreparedExport { + bytes: Bytes, + context: Context, + saved_payload: OtapPayload, + signal_type: SignalType, +} + +fn prepare_otap_export( + mut otap_batch: OtapArrowRecords, context: Context, proto_buffer: &mut ProtoBuffer, encoder: &mut Enc, - client: &mut crate::otap_grpc::otlp::client::OtlpServiceClient, - effect_handler: &EffectHandler, -) -> Result<(), Error> -where - T2: tonic::client::GrpcService, - T2::Error: Into>, - T2::ResponseBody: Send + 'static, - ::Error: - Into> + Send, - S: crate::otap_grpc::otlp::client::ServiceDescriptor, - Resp: prost::Message + Default + Send + 'static, -{ - // Encode OTAP records to protobuf + exporter: NodeId, + signal_type: SignalType, +) -> Result { proto_buffer.clear(); encoder .encode(&mut otap_batch, proto_buffer) .map_err(|e| Error::ExporterError { - exporter: effect_handler.exporter_id(), + exporter, kind: ExporterErrorKind::Other, error: format!("encoding error: {}", e), source_detail: "".to_string(), })?; - let bytes = proto_buffer.as_ref().to_vec(); + let mut owned_buffer = ProtoBuffer::new(); + mem::swap(proto_buffer, &mut owned_buffer); + let next_capacity = owned_buffer.capacity(); + let bytes = Bytes::from(owned_buffer.into_bytes()); + *proto_buffer = ProtoBuffer::with_capacity(next_capacity); + if !context.may_return_payload() { - // drop before the export, payload not requested let _drop = otap_batch.take_payload(); } let saved_payload: OtapPayload = otap_batch.into(); - // Export and handle result with Ack/Nack - let result = client.export(bytes).await; - handle_export_result(result, context, saved_payload, effect_handler).await + Ok(PreparedExport { + bytes, + context, + saved_payload, + signal_type, + }) } -/// Generic function for exporting OTLP bytes via gRPC and handling Ack/Nack delivery. -async fn handle_otlp_export( - bytes: Vec, +fn prepare_otlp_export( + bytes: Bytes, context: Context, - client: &mut crate::otap_grpc::otlp::client::OtlpServiceClient, - effect_handler: &EffectHandler, - save_payload_fn: impl FnOnce(&[u8]) -> OtapPayload, -) -> Result<(), Error> -where - T2: tonic::client::GrpcService, - T2::Error: Into>, - T2::ResponseBody: Send + 'static, - ::Error: - Into> + Send, - S: crate::otap_grpc::otlp::client::ServiceDescriptor, - Resp: prost::Message + Default + Send + 'static, -{ + signal_type: SignalType, + save_payload_fn: impl FnOnce(Bytes) -> OtapPayload, +) -> PreparedExport { let saved_payload = if context.may_return_payload() { - save_payload_fn(&bytes) + save_payload_fn(bytes.clone()) } else { - save_payload_fn(&[]) + save_payload_fn(Bytes::new()) }; - let result = client.export(bytes).await; - handle_export_result(result, context, saved_payload, effect_handler).await + PreparedExport { + bytes, + context, + saved_payload, + signal_type, + } +} + +/// Applies the Ack/Nack side effects for a completed gRPC export and returns the reusable client. +async fn process_completed_export( + completed: CompletedExport, + effect_handler: &EffectHandler, + pdata_metrics: &mut MetricSet, +) -> SignalClient { + let CompletedExport { + result, + context, + saved_payload, + signal_type, + client, + } = completed; + + match handle_export_result(result, context, saved_payload, effect_handler).await { + Ok(()) => pdata_metrics.add_exported(signal_type, 1), + Err(_) => pdata_metrics.add_failed(signal_type, 1), + } + + client +} + +/// Builds an export future for the provided payload, borrowing a signal-specific client from the pool. +fn make_export_future(prepared: PreparedExport, client_pool: &mut ClientPool) -> ExportFuture { + let PreparedExport { + bytes, + context, + saved_payload, + signal_type, + } = prepared; + + let future: ExportTaskFuture = match signal_type { + SignalType::Logs => { + let mut client = client_pool.take_logs(); + Box::pin(async move { + let result = client.export(bytes).await.map(|_| ()); + (result, SignalClient::Logs(client)) + }) + } + SignalType::Metrics => { + let mut client = client_pool.take_metrics(); + Box::pin(async move { + let result = client.export(bytes).await.map(|_| ()); + (result, SignalClient::Metrics(client)) + }) + } + SignalType::Traces => { + let mut client = client_pool.take_traces(); + Box::pin(async move { + let result = client.export(bytes).await.map(|_| ()); + (result, SignalClient::Traces(client)) + }) + } + }; + + ExportFuture::new(future, context, saved_payload, signal_type) +} + +type ExportTaskFuture = + Pin, SignalClient)> + 'static>>; + +struct ExportFuture { + future: ExportTaskFuture, + outcome_data: Option<(Context, OtapPayload)>, + signal_type: SignalType, +} + +impl ExportFuture { + fn new( + future: ExportTaskFuture, + context: Context, + saved_payload: OtapPayload, + signal_type: SignalType, + ) -> Self { + Self { + future, + outcome_data: Some((context, saved_payload)), + signal_type, + } + } +} + +impl Unpin for ExportFuture {} + +impl Future for ExportFuture { + type Output = CompletedExport; + + fn poll(mut self: Pin<&mut Self>, cx: &mut TaskContext<'_>) -> Poll { + let (result, client) = ready!(self.future.as_mut().poll(cx)); + let (context, saved_payload) = self + .outcome_data + .take() + .expect("outcome data already taken"); + Poll::Ready(CompletedExport { + result, + context, + saved_payload, + signal_type: self.signal_type, + client, + }) + } +} + +/// FIFO-ish wrapper around the in-flight export RPCs. +struct InFlightQueue { + futures: Vec, +} + +impl InFlightQueue { + fn new() -> Self { + Self { + futures: Vec::new(), + } + } + + fn len(&self) -> usize { + self.futures.len() + } + + fn is_empty(&self) -> bool { + self.futures.is_empty() + } + + fn push(&mut self, future: ExportFuture) { + self.futures.push(future); + } + + fn poll_next(&mut self, cx: &mut TaskContext<'_>) -> Poll> { + let mut index = 0; + while index < self.futures.len() { + let poll_result = { + let future = Pin::new(&mut self.futures[index]); + future.poll(cx) + }; + + match poll_result { + Poll::Ready(completed) => { + let _ = self.futures.swap_remove(index); + return Poll::Ready(Some(completed)); + } + Poll::Pending => { + index += 1; + } + } + } + + if self.futures.is_empty() { + Poll::Ready(None) + } else { + Poll::Pending + } + } + + /// Returns a future that resolves once the next export finishes. + fn next_completion(&mut self) -> NextCompletion<'_> { + NextCompletion { queue: self } + } +} + +/// Drives the in-flight queue until the next export finishes. +struct NextCompletion<'a> { + queue: &'a mut InFlightQueue, +} + +impl Future for NextCompletion<'_> { + type Output = Option; + + fn poll(self: Pin<&mut Self>, cx: &mut TaskContext<'_>) -> Poll { + let this = self.get_mut(); + this.queue.poll_next(cx) + } +} + +/// Keeps a small stash of gRPC clients so each export can reuse an existing connection. +struct ClientPool { + base_channel: Channel, + compression: Option, + logs: Vec>, + metrics: Vec>, + traces: Vec>, +} + +impl ClientPool { + fn new(base_channel: Channel, compression: Option) -> Self { + Self { + base_channel, + compression, + logs: Vec::new(), + metrics: Vec::new(), + traces: Vec::new(), + } + } + + fn take_logs(&mut self) -> LogsServiceClient { + self.logs.pop().unwrap_or_else(|| self.make_logs_client()) + } + + fn take_metrics(&mut self) -> MetricsServiceClient { + self.metrics + .pop() + .unwrap_or_else(|| self.make_metrics_client()) + } + + fn take_traces(&mut self) -> TraceServiceClient { + self.traces + .pop() + .unwrap_or_else(|| self.make_traces_client()) + } + + fn release(&mut self, client: SignalClient) { + match client { + SignalClient::Logs(client) => self.logs.push(client), + SignalClient::Metrics(client) => self.metrics.push(client), + SignalClient::Traces(client) => self.traces.push(client), + } + } + + fn make_logs_client(&self) -> LogsServiceClient { + let mut client = LogsServiceClient::new(self.base_channel.clone()); + if let Some(encoding) = self.compression { + client = client.send_compressed(encoding); + } + client + } + + fn make_metrics_client(&self) -> MetricsServiceClient { + let mut client = MetricsServiceClient::new(self.base_channel.clone()); + if let Some(encoding) = self.compression { + client = client.send_compressed(encoding); + } + client + } + + fn make_traces_client(&self) -> TraceServiceClient { + let mut client = TraceServiceClient::new(self.base_channel.clone()); + if let Some(encoding) = self.compression { + client = client.send_compressed(encoding); + } + client + } +} + +enum SignalClient { + Logs(LogsServiceClient), + Metrics(MetricsServiceClient), + Traces(TraceServiceClient), +} + +/// Captures everything we need once a single export RPC has completed. +struct CompletedExport { + result: Result<(), tonic::Status>, + context: Context, + saved_payload: OtapPayload, + signal_type: SignalType, + client: SignalClient, } #[cfg(test)] @@ -401,11 +702,12 @@ mod tests { exporter::{TestContext, TestRuntime}, test_node, }; - use otap_df_pdata::proto::opentelemetry::collector::{ - logs::v1::{ExportLogsServiceRequest, logs_service_server::LogsServiceServer}, - metrics::v1::{ExportMetricsServiceRequest, metrics_service_server::MetricsServiceServer}, - trace::v1::{ExportTraceServiceRequest, trace_service_server::TraceServiceServer}, - }; + use otap_df_pdata::proto::opentelemetry::collector::logs::v1::ExportLogsServiceRequest; + use otap_df_pdata::proto::opentelemetry::collector::logs::v1::logs_service_server::LogsServiceServer; + use otap_df_pdata::proto::opentelemetry::collector::metrics::v1::ExportMetricsServiceRequest; + use otap_df_pdata::proto::opentelemetry::collector::metrics::v1::metrics_service_server::MetricsServiceServer; + use otap_df_pdata::proto::opentelemetry::collector::trace::v1::ExportTraceServiceRequest; + use otap_df_pdata::proto::opentelemetry::collector::trace::v1::trace_service_server::TraceServiceServer; use otap_df_telemetry::metrics::MetricSetSnapshot; use otap_df_telemetry::registry::MetricsRegistryHandle; use otap_df_telemetry::reporter::MetricsReporter; @@ -473,21 +775,21 @@ mod tests { /// Test closure that simulates a typical test scenario by sending timer ticks, config, /// data message, and shutdown control messages. - fn scenario() - -> impl FnOnce(TestContext) -> std::pin::Pin>> { + fn scenario() -> impl FnOnce(TestContext) -> Pin>> { |ctx| { Box::pin(async move { // Send a data message let req = ExportLogsServiceRequest::default(); let mut req_bytes = vec![]; req.encode(&mut req_bytes).unwrap(); - let logs_pdata = - OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(req_bytes).into()) - .test_subscribe_to( - Interests::ACKS | Interests::NACKS, - TestCallData::default().into(), - 123, - ); + let logs_pdata = OtapPdata::new_default( + OtlpProtoBytes::ExportLogsRequest(Bytes::from(req_bytes)).into(), + ) + .test_subscribe_to( + Interests::ACKS | Interests::NACKS, + TestCallData::default().into(), + 123, + ); ctx.send_pdata(logs_pdata) .await .expect("Failed to send log message"); @@ -495,13 +797,14 @@ mod tests { let req = ExportMetricsServiceRequest::default(); let mut req_bytes = vec![]; req.encode(&mut req_bytes).unwrap(); - let metrics_pdata = - OtapPdata::new_default(OtlpProtoBytes::ExportMetricsRequest(req_bytes).into()) - .test_subscribe_to( - Interests::ACKS | Interests::NACKS, - TestCallData::default().into(), - 123, - ); + let metrics_pdata = OtapPdata::new_default( + OtlpProtoBytes::ExportMetricsRequest(Bytes::from(req_bytes)).into(), + ) + .test_subscribe_to( + Interests::ACKS | Interests::NACKS, + TestCallData::default().into(), + 123, + ); ctx.send_pdata(metrics_pdata) .await .expect("Failed to send metric message"); @@ -509,13 +812,14 @@ mod tests { let req = ExportTraceServiceRequest::default(); let mut req_bytes = vec![]; req.encode(&mut req_bytes).unwrap(); - let traces_pdata = - OtapPdata::new_default(OtlpProtoBytes::ExportTracesRequest(req_bytes).into()) - .test_subscribe_to( - Interests::ACKS | Interests::NACKS, - TestCallData::default().into(), - 123, - ); + let traces_pdata = OtapPdata::new_default( + OtlpProtoBytes::ExportTracesRequest(Bytes::from(req_bytes)).into(), + ) + .test_subscribe_to( + Interests::ACKS | Interests::NACKS, + TestCallData::default().into(), + 123, + ); ctx.send_pdata(traces_pdata) .await .expect("Failed to send metric message"); @@ -531,10 +835,8 @@ mod tests { /// Validation closure that checks the expected counter values fn validation_procedure( mut receiver: tokio::sync::mpsc::Receiver, - ) -> impl FnOnce( - TestContext, - Result<(), Error>, - ) -> std::pin::Pin>> { + ) -> impl FnOnce(TestContext, Result<(), Error>) -> Pin>> + { |_, exporter_result| { Box::pin(async move { assert!(exporter_result.is_ok()); @@ -616,9 +918,11 @@ mod tests { let exporter = ExporterWrapper::local( OTLPExporter { config: Config { - grpc_endpoint, - compression_method: None, - timeout: None, + grpc: GrpcClientSettings { + grpc_endpoint: grpc_endpoint.clone(), + ..Default::default() + }, + max_in_flight: 32, }, pdata_metrics: pipeline_ctx.register_metrics::(), }, @@ -681,9 +985,11 @@ mod tests { let mut exporter = ExporterWrapper::local( OTLPExporter { config: Config { - grpc_endpoint, - compression_method: None, - timeout: None, + grpc: GrpcClientSettings { + grpc_endpoint: grpc_endpoint.clone(), + ..Default::default() + }, + max_in_flight: 32, }, pdata_metrics: pipeline_ctx.register_metrics::(), }, @@ -741,7 +1047,7 @@ mod tests { // send a request while the server isn't running and check how we handle it let pdata = OtapPdata::new_default(OtapPayload::OtlpBytes( - OtlpProtoBytes::ExportLogsRequest(req_bytes.clone()), + OtlpProtoBytes::ExportLogsRequest(req_bytes.clone().into()), )) .test_subscribe_to( Interests::ACKS | Interests::NACKS, @@ -762,7 +1068,7 @@ mod tests { // send a pdata let pdata = OtapPdata::new_default(OtapPayload::OtlpBytes( - OtlpProtoBytes::ExportLogsRequest(req_bytes.clone()), + OtlpProtoBytes::ExportLogsRequest(req_bytes.clone().into()), )) .test_subscribe_to( Interests::ACKS | Interests::NACKS, @@ -783,7 +1089,7 @@ mod tests { // send a request while the server isn't running and check that we still handle it correctly let pdata = OtapPdata::new_default(OtapPayload::OtlpBytes( - OtlpProtoBytes::ExportLogsRequest(req_bytes.clone()), + OtlpProtoBytes::ExportLogsRequest(req_bytes.clone().into()), )) .test_subscribe_to( Interests::ACKS | Interests::NACKS, @@ -807,7 +1113,7 @@ mod tests { // send another pdata. This ensures the client can reconnect after it was shut down let pdata = OtapPdata::new_default(OtapPayload::OtlpBytes( - OtlpProtoBytes::ExportLogsRequest(req_bytes.clone()), + OtlpProtoBytes::ExportLogsRequest(req_bytes.clone().into()), )) .test_subscribe_to( Interests::ACKS | Interests::NACKS, diff --git a/rust/otap-dataflow/crates/otap/src/otlp_receiver.rs b/rust/otap-dataflow/crates/otap/src/otlp_receiver.rs index faaedf672f..8fb3c08c94 100644 --- a/rust/otap-dataflow/crates/otap/src/otlp_receiver.rs +++ b/rust/otap-dataflow/crates/otap/src/otlp_receiver.rs @@ -1,22 +1,40 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 +//! # OTLP receiver architecture +//! +//! The receiver is registered in `OTAP_RECEIVER_FACTORIES` so the dataflow engine can spin it up +//! from configuration. `OTLPReceiver::from_config` turns user provided configuration into the +//! shared `GrpcServerSettings`. +//! +//! Once the TCP listener is bound the receiver builds three OTLP-specific gRPC servers (logs, +//! metrics, traces). Each server is backed by the shared codecs in `otap_grpc::otlp::server`, +//! producing lazily-decoded [`OtapPdata`](crate::pdata::OtapPdata) that are pushed straight into +//! the pipeline. The `SignalAckRoutingState` bundle aggregates the per-signal subscription maps that +//! connect incoming requests with their ACK/NACK responses when `wait_for_result` is enabled. +//! +//! A `tokio::select!` drives two responsibilities concurrently: +//! - poll control messages from the engine (shutdown, telemetry collection, ACK/NACK forwarding) +//! - serve the gRPC endpoints with the tuned concurrency settings derived from downstream channel +//! capacity. +//! +//! Periodic telemetry snapshots update the `OtlpReceiverMetrics` counters, which focus on ACK/NACK +//! behaviour today. Additional instrumentation can be layered on the same metric set without +//! changing the control loop. + use crate::OTAP_RECEIVER_FACTORIES; -use crate::otap_grpc::otlp::server::{ - LogsServiceServer, MetricsServiceServer, RouteResponse, Settings, SharedState, - TraceServiceServer, -}; +use crate::otap_grpc::otlp::server::{LogsServiceServer, MetricsServiceServer, TraceServiceServer}; use crate::pdata::OtapPdata; -use crate::compression::CompressionMethod; +use crate::otap_grpc::common::SignalAckRoutingState; +use crate::otap_grpc::{GrpcServerSettings, common}; use async_trait::async_trait; use linkme::distributed_slice; -use otap_df_config::SignalType; use otap_df_config::node::NodeUserConfig; use otap_df_engine::ReceiverFactory; use otap_df_engine::config::ReceiverConfig; use otap_df_engine::context::PipelineContext; -use otap_df_engine::control::{AckMsg, NackMsg, NodeControlMsg}; +use otap_df_engine::control::NodeControlMsg; use otap_df_engine::error::{Error, ReceiverErrorKind, format_error_sources}; use otap_df_engine::node::NodeId; use otap_df_engine::receiver::ReceiverWrapper; @@ -27,12 +45,9 @@ use otap_df_telemetry::metrics::MetricSet; use otap_df_telemetry_macros::metric_set; use serde::Deserialize; use serde_json::Value; -use std::net::SocketAddr; use std::ops::Add; use std::sync::Arc; use std::time::{Duration, Instant}; -use tokio_stream::wrappers::TcpListenerStream; -use tonic::codec::EnabledCompressionEncodings; use tonic::transport::Server; /// URN for the OTLP Receiver @@ -42,62 +57,33 @@ pub const OTLP_RECEIVER_URN: &str = "urn:otel:otlp:receiver"; #[derive(Debug, Deserialize)] #[serde(deny_unknown_fields)] pub struct Config { - /// The endpoint details: protocol, name, port. - listening_addr: SocketAddr, - - /// Compression methods accepted - /// TODO: this should be (CompressionMethod, CompressionMethod), with separate settings - /// (as tonic supports) for request and response compression. - compression_method: Option, - - /// Maximum number of concurrent (in-flight) requests (default: 1000) - #[serde(default = "default_max_concurrent_requests")] - max_concurrent_requests: usize, - - /// Whether to wait for the result (default: true) - /// - /// When enabled, the receiver will not send a response until the - /// immediate downstream component has acknowledged receipt of the - /// data. This does not guarantee that data has been fully - /// processed or successfully exported to the final destination, - /// since components are able acknowledge early. - /// - /// Note when wait_for_result=false, it is impossible to - /// see a failure, errors are effectively suppressed. - #[serde(default = "default_wait_for_result")] - wait_for_result: bool, - - /// Timeout for RPC requests. If not specified, no timeout is applied. - /// Format: humantime format (e.g., "30s", "5m", "1h", "500ms") - #[serde(default, with = "humantime_serde")] - pub timeout: Option, -} - -const fn default_max_concurrent_requests() -> usize { - 1000 -} - -const fn default_wait_for_result() -> bool { - // See https://github.com/open-telemetry/otel-arrow/issues/1311 - // This matches the OTel Collector default for wait_for_result, presently. - false + /// Shared gRPC server settings reused across gRPC-based receivers. + #[serde(flatten)] + pub grpc: GrpcServerSettings, } -/// Receiver implementation that receives OTLP grpc service requests and decodes the data into OTAP. +/// gRPC receiver that ingests OTLP signals and forwards them into the OTAP pipeline. +/// +/// The implementation mirrors the OTAP Arrow receiver layout: a shared [`GrpcServerConfig`] drives +/// listener creation, per-signal tonic services are registered on a single server, and the receiver +/// is wrapped in a [`ReceiverFactory`] so the dataflow engine can build it from configuration. +/// +/// Several optimisations keep the hot path inexpensive: +/// - Incoming request bodies stay in their serialized OTLP form thanks to the custom +/// [`OtlpBytesCodec`](crate::otap_grpc::otlp::server::OtlpBytesCodec), allowing downstream stages +/// to decode lazily. +/// - `tune_max_concurrent_requests` clamps the gRPC concurrency to the downstream channel capacity, +/// preventing backlog buildup while still honouring user settings. +/// - `SignalAckRoutingState` maintains per-signal ACK subscription slots so `wait_for_result` +/// lookups avoid extra bookkeeping and route responses directly back to callers. +/// - Metrics are wired through a `MetricSet`, letting periodic snapshots flush ACK/NACK counters +/// without rebuilding instruments. pub struct OTLPReceiver { config: Config, metrics: MetricSet, } -/// State shared between gRPC server task and the effect handler. -struct SharedStates { - logs: Option, - metrics: Option, - traces: Option, -} - -/// Declares the OTLP receiver as a shared receiver factory -/// +/// Declares the OTLP receiver as a shared receiver factory. #[allow(unsafe_code)] #[distributed_slice(OTAP_RECEIVER_FACTORIES)] pub static OTLP_RECEIVER: ReceiverFactory = ReceiverFactory { @@ -106,8 +92,11 @@ pub static OTLP_RECEIVER: ReceiverFactory = ReceiverFactory { node: NodeId, node_config: Arc, receiver_config: &ReceiverConfig| { + let mut receiver = OTLPReceiver::from_config(pipeline, &node_config.config)?; + receiver.tune_max_concurrent_requests(receiver_config.output_pdata_channel.capacity); + Ok(ReceiverWrapper::shared( - OTLPReceiver::from_config(pipeline, &node_config.config)?, + receiver, node, node_config, receiver_config, @@ -133,55 +122,8 @@ impl OTLPReceiver { Ok(Self { config, metrics }) } - fn route_ack_response(&self, states: &SharedStates, ack: AckMsg) -> RouteResponse { - let calldata = ack.calldata; - let resp = Ok(()); - let state = match ack.accepted.signal_type() { - SignalType::Logs => states.logs.as_ref(), - SignalType::Metrics => states.metrics.as_ref(), - SignalType::Traces => states.traces.as_ref(), - }; - - state - .map(|s| s.route_response(calldata, resp)) - .unwrap_or(RouteResponse::None) - } - - fn route_nack_response( - &self, - states: &SharedStates, - mut nack: NackMsg, - ) -> RouteResponse { - let calldata = std::mem::take(&mut nack.calldata); - let signal_type = nack.refused.signal_type(); - let resp = Err(nack); - let state = match signal_type { - SignalType::Logs => states.logs.as_ref(), - SignalType::Metrics => states.metrics.as_ref(), - SignalType::Traces => states.traces.as_ref(), - }; - - state - .map(|s| s.route_response(calldata, resp)) - .unwrap_or(RouteResponse::None) - } - - fn handle_ack_response(&mut self, resp: RouteResponse) { - match resp { - RouteResponse::Sent => self.metrics.acks_sent.inc(), - RouteResponse::Expired => self.metrics.acks_nacks_invalid_or_expired.inc(), - RouteResponse::Invalid => self.metrics.acks_nacks_invalid_or_expired.inc(), - RouteResponse::None => {} - } - } - - fn handle_nack_response(&mut self, resp: RouteResponse) { - match resp { - RouteResponse::Sent => self.metrics.nacks_sent.inc(), - RouteResponse::Expired => self.metrics.acks_nacks_invalid_or_expired.inc(), - RouteResponse::Invalid => self.metrics.acks_nacks_invalid_or_expired.inc(), - RouteResponse::None => {} - } + fn tune_max_concurrent_requests(&mut self, downstream_capacity: usize) { + common::tune_max_concurrent_requests(&mut self.config.grpc, downstream_capacity); } } @@ -220,39 +162,23 @@ impl shared::Receiver for OTLPReceiver { effect_handler: shared::EffectHandler, ) -> Result { // Make the receiver mutable so we can update metrics on telemetry collection. - let listener = effect_handler.tcp_listener(self.config.listening_addr)?; - let listener_stream = TcpListenerStream::new(listener); - - let mut compression = EnabledCompressionEncodings::default(); - let _ = self - .config - .compression_method - .as_ref() - .map(|method| compression.enable(method.map_to_compression_encoding())); - - let settings = Settings { - max_concurrent_requests: self.config.max_concurrent_requests, - wait_for_result: self.config.wait_for_result, - accept_compression_encodings: compression, - send_compression_encodings: compression, - }; + let config = &self.config.grpc; + let listener = effect_handler.tcp_listener(config.listening_addr)?; + // Wrap the raw listener to enforce keepalive/nodelay socket tuning on accepted streams. + let incoming = config.build_tcp_incoming(listener); + let settings = config.build_settings(); let logs_server = LogsServiceServer::new(effect_handler.clone(), &settings); let metrics_server = MetricsServiceServer::new(effect_handler.clone(), &settings); let traces_server = TraceServiceServer::new(effect_handler.clone(), &settings); - let states = SharedStates { - logs: logs_server.common.state(), - metrics: metrics_server.common.state(), - traces: traces_server.common.state(), - }; - - let mut server_builder = Server::builder(); - - // Apply timeout if configured - if let Some(timeout) = self.config.timeout { - server_builder = server_builder.timeout(timeout); - } + // Gather the per-signal subscription maps so ACK/NACK routing stays signal-aware. + let ack_routing_states = SignalAckRoutingState::new( + logs_server.common.state(), + metrics_server.common.state(), + traces_server.common.state(), + ); + let mut server_builder = common::apply_server_tuning(Server::builder(), config); let server = server_builder .add_service(logs_server) @@ -276,15 +202,27 @@ impl shared::Receiver for OTLPReceiver { _ = telemetry_cancel_handle.cancel().await; return Ok(TerminalState::new(deadline, [snapshot])); }, - Ok(NodeControlMsg::CollectTelemetry { mut metrics_reporter }) => { + Ok(NodeControlMsg::CollectTelemetry { metrics_reporter }) => { // Report current receiver metrics. _ = metrics_reporter.report(&mut self.metrics); }, Ok(NodeControlMsg::Ack(ack)) => { - self.handle_ack_response(self.route_ack_response(&states, ack)); + let resp = common::route_ack_response(&ack_routing_states, ack); + common::handle_route_response( + resp, + &mut self.metrics, + |metrics| metrics.acks_sent.inc(), + |metrics| metrics.acks_nacks_invalid_or_expired.inc(), + ); }, Ok(NodeControlMsg::Nack(nack)) => { - self.handle_nack_response(self.route_nack_response(&states, nack)); + let resp = common::route_nack_response(&ack_routing_states, nack); + common::handle_route_response( + resp, + &mut self.metrics, + |metrics| metrics.nacks_sent.inc(), + |metrics| metrics.acks_nacks_invalid_or_expired.inc(), + ); }, Err(e) => { return Err(Error::ChannelRecvError(e)); @@ -299,7 +237,7 @@ impl shared::Receiver for OTLPReceiver { }, // Run server - result = server.serve_with_incoming(listener_stream) => { + result = server.serve_with_incoming(incoming) => { if let Err(error) = result { let source_detail = format_error_sources(&error); return Err(Error::ReceiverError { @@ -323,6 +261,7 @@ impl shared::Receiver for OTLPReceiver { mod tests { use super::*; + use crate::compression::CompressionMethod; use otap_df_config::node::NodeUserConfig; use otap_df_engine::context::ControllerContext; use otap_df_engine::control::NackMsg; @@ -352,10 +291,21 @@ mod tests { use otap_df_pdata::proto::opentelemetry::trace::v1::{ResourceSpans, ScopeSpans}; use otap_df_telemetry::registry::MetricsRegistryHandle; use prost::Message; + use std::net::SocketAddr; use std::pin::Pin; use std::time::{Duration, Instant}; use tokio::time::timeout; + fn test_config(addr: SocketAddr) -> Config { + let grpc = GrpcServerSettings { + listening_addr: addr, + max_concurrent_requests: 1000, + wait_for_result: true, + ..Default::default() + }; + Config { grpc } + } + fn create_logs_service_request() -> ExportLogsServiceRequest { ExportLogsServiceRequest { resource_logs: vec![ResourceLogs { @@ -442,13 +392,57 @@ mod tests { let receiver = OTLPReceiver::from_config(pipeline_ctx.clone(), &config_with_max_concurrent_requests) .unwrap(); - assert_eq!(receiver.config.max_concurrent_requests, 5000); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 5000); let config_default = json!({ "listening_addr": "127.0.0.1:4317" }); let receiver = OTLPReceiver::from_config(pipeline_ctx.clone(), &config_default).unwrap(); - assert_eq!(receiver.config.max_concurrent_requests, 1000); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 0); + assert!(receiver.config.grpc.request_compression.is_none()); + assert!(receiver.config.grpc.response_compression.is_none()); + assert!( + receiver + .config + .grpc + .preferred_response_compression() + .is_none() + ); + assert!(receiver.config.grpc.tcp_nodelay); + assert_eq!( + receiver.config.grpc.tcp_keepalive, + Some(Duration::from_secs(45)) + ); + assert_eq!( + receiver.config.grpc.tcp_keepalive_interval, + Some(Duration::from_secs(15)) + ); + assert_eq!(receiver.config.grpc.tcp_keepalive_retries, Some(5)); + assert_eq!(receiver.config.grpc.transport_concurrency_limit, None); + assert!(receiver.config.grpc.load_shed); + assert_eq!( + receiver.config.grpc.initial_stream_window_size, + Some(8 * 1024 * 1024) + ); + assert_eq!( + receiver.config.grpc.initial_connection_window_size, + Some(24 * 1024 * 1024) + ); + assert!(!receiver.config.grpc.http2_adaptive_window); + assert_eq!(receiver.config.grpc.max_frame_size, Some(16 * 1024)); + assert_eq!( + receiver.config.grpc.max_decoding_message_size, + Some(4 * 1024 * 1024) + ); + assert_eq!( + receiver.config.grpc.http2_keepalive_interval, + Some(Duration::from_secs(30)) + ); + assert_eq!( + receiver.config.grpc.http2_keepalive_timeout, + Some(Duration::from_secs(10)) + ); + assert_eq!(receiver.config.grpc.max_concurrent_streams, None); let config_full = json!({ "listening_addr": "127.0.0.1:4317", @@ -456,7 +450,90 @@ mod tests { "max_concurrent_requests": 2500 }); let receiver = OTLPReceiver::from_config(pipeline_ctx.clone(), &config_full).unwrap(); - assert_eq!(receiver.config.max_concurrent_requests, 2500); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 2500); + assert_eq!( + receiver.config.grpc.request_compression, + Some(vec![CompressionMethod::Gzip]) + ); + assert!(receiver.config.grpc.response_compression.is_none()); + + let config_with_server_overrides = json!({ + "listening_addr": "127.0.0.1:4317", + "max_concurrent_requests": 512, + "tcp_nodelay": false, + "tcp_keepalive": "60s", + "tcp_keepalive_interval": "20s", + "tcp_keepalive_retries": 3, + "transport_concurrency_limit": 256, + "load_shed": false, + "initial_stream_window_size": "4MiB", + "initial_connection_window_size": "16MiB", + "max_frame_size": "8MiB", + "max_decoding_message_size": "6MiB", + "http2_keepalive_interval": "45s", + "http2_keepalive_timeout": "20s", + "max_concurrent_streams": 1024, + "http2_adaptive_window": true + }); + let receiver = + OTLPReceiver::from_config(pipeline_ctx.clone(), &config_with_server_overrides).unwrap(); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 512); + assert!(!receiver.config.grpc.tcp_nodelay); + assert_eq!( + receiver.config.grpc.tcp_keepalive, + Some(Duration::from_secs(60)) + ); + assert_eq!( + receiver.config.grpc.tcp_keepalive_interval, + Some(Duration::from_secs(20)) + ); + assert_eq!(receiver.config.grpc.tcp_keepalive_retries, Some(3)); + assert_eq!(receiver.config.grpc.transport_concurrency_limit, Some(256)); + assert!(!receiver.config.grpc.load_shed); + assert_eq!( + receiver.config.grpc.initial_stream_window_size, + Some(4 * 1024 * 1024) + ); + assert_eq!( + receiver.config.grpc.initial_connection_window_size, + Some(16 * 1024 * 1024) + ); + assert_eq!(receiver.config.grpc.max_frame_size, Some(8 * 1024 * 1024)); + assert_eq!( + receiver.config.grpc.max_decoding_message_size, + Some(6 * 1024 * 1024) + ); + assert_eq!( + receiver.config.grpc.http2_keepalive_interval, + Some(Duration::from_secs(45)) + ); + assert_eq!( + receiver.config.grpc.http2_keepalive_timeout, + Some(Duration::from_secs(20)) + ); + assert_eq!(receiver.config.grpc.max_concurrent_streams, Some(1024)); + assert!(receiver.config.grpc.http2_adaptive_window); + + let config_with_compression_list = json!({ + "listening_addr": "127.0.0.1:4317", + "compression_method": ["gzip", "zstd", "gzip"] + }); + let receiver = + OTLPReceiver::from_config(pipeline_ctx.clone(), &config_with_compression_list).unwrap(); + assert_eq!( + receiver.config.grpc.request_compression, + Some(vec![CompressionMethod::Gzip, CompressionMethod::Zstd]) + ); + assert!(receiver.config.grpc.response_compression.is_none()); + + let config_with_compression_none = json!({ + "listening_addr": "127.0.0.1:4317", + "compression_method": "none" + }); + let receiver = + OTLPReceiver::from_config(pipeline_ctx.clone(), &config_with_compression_none).unwrap(); + assert_eq!(receiver.config.grpc.request_compression, Some(vec![])); + assert!(receiver.config.grpc.response_compression.is_none()); let config_with_timeout = json!({ "listening_addr": "127.0.0.1:4317", @@ -464,14 +541,54 @@ mod tests { }); let receiver = OTLPReceiver::from_config(pipeline_ctx.clone(), &config_with_timeout).unwrap(); - assert_eq!(receiver.config.timeout, Some(Duration::from_secs(30))); + assert_eq!(receiver.config.grpc.timeout, Some(Duration::from_secs(30))); let config_with_timeout_ms = json!({ "listening_addr": "127.0.0.1:4317", "timeout": "500ms" }); let receiver = OTLPReceiver::from_config(pipeline_ctx, &config_with_timeout_ms).unwrap(); - assert_eq!(receiver.config.timeout, Some(Duration::from_millis(500))); + assert_eq!( + receiver.config.grpc.timeout, + Some(Duration::from_millis(500)) + ); + } + + #[test] + fn test_tune_max_concurrent_requests() { + use serde_json::json; + + let metrics_registry_handle = MetricsRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = + controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + // Defaults clamp to downstream capacity. + let config_default = json!({ + "listening_addr": "127.0.0.1:4317" + }); + let mut receiver = + OTLPReceiver::from_config(pipeline_ctx.clone(), &config_default).unwrap(); + receiver.tune_max_concurrent_requests(128); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 128); + + // User provided smaller value is preserved. + let config_small = json!({ + "listening_addr": "127.0.0.1:4317", + "max_concurrent_requests": 32 + }); + let mut receiver = OTLPReceiver::from_config(pipeline_ctx.clone(), &config_small).unwrap(); + receiver.tune_max_concurrent_requests(128); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 32); + + // Config value of zero adopts downstream capacity. + let config_zero = json!({ + "listening_addr": "127.0.0.1:4317", + "max_concurrent_requests": 0 + }); + let mut receiver = OTLPReceiver::from_config(pipeline_ctx, &config_zero).unwrap(); + receiver.tune_max_concurrent_requests(256); + assert_eq!(receiver.config.grpc.max_concurrent_requests, 256); } fn scenario( @@ -651,13 +768,7 @@ mod tests { let receiver = ReceiverWrapper::shared( OTLPReceiver { - config: Config { - wait_for_result: true, - listening_addr: addr, - compression_method: None, - max_concurrent_requests: 1000, - timeout: None, - }, + config: test_config(addr), metrics: pipeline_ctx.register_metrics::(), }, test_node(test_runtime.config().name.clone()), @@ -689,13 +800,7 @@ mod tests { let receiver = ReceiverWrapper::shared( OTLPReceiver { - config: Config { - wait_for_result: true, - listening_addr: addr, - compression_method: None, - max_concurrent_requests: 1000, - timeout: None, - }, + config: test_config(addr), metrics: pipeline_ctx.register_metrics::(), }, test_node(test_runtime.config().name.clone()), diff --git a/rust/otap-dataflow/crates/otap/src/parquet_exporter.rs b/rust/otap-dataflow/crates/otap/src/parquet_exporter.rs index 076acb7e77..8048f013b4 100644 --- a/rust/otap-dataflow/crates/otap/src/parquet_exporter.rs +++ b/rust/otap-dataflow/crates/otap/src/parquet_exporter.rs @@ -226,9 +226,7 @@ impl Exporter for ParquetExporter { } } } - Message::Control(NodeControlMsg::CollectTelemetry { - mut metrics_reporter, - }) => { + Message::Control(NodeControlMsg::CollectTelemetry { metrics_reporter }) => { if let Some(metrics) = self.pdata_metrics.as_mut() { _ = metrics_reporter.report(metrics); } diff --git a/rust/otap-dataflow/crates/otap/src/perf_exporter/config.rs b/rust/otap-dataflow/crates/otap/src/perf_exporter/config.rs index 58f232e342..d128bbb4d0 100644 --- a/rust/otap-dataflow/crates/otap/src/perf_exporter/config.rs +++ b/rust/otap-dataflow/crates/otap/src/perf_exporter/config.rs @@ -24,7 +24,7 @@ pub struct Config { #[serde(default = "default_cpu_usage")] cpu_usage: bool, - #[serde(default = "default_mem_usage")] + #[serde(default = "default_mem_usage", alias = "memory_usage")] mem_usage: bool, #[serde(default = "default_disk_usage")] @@ -32,6 +32,9 @@ pub struct Config { #[serde(default = "default_io_usage")] io_usage: bool, + + #[serde(default = "default_display_throughput")] + display_throughput: bool, } fn default_frequency() -> u64 { @@ -56,6 +59,10 @@ fn default_io_usage() -> bool { true } +fn default_display_throughput() -> bool { + false +} + fn default_smoothing_factor() -> f32 { 0.3 } @@ -71,6 +78,7 @@ impl Config { mem_usage: bool, disk_usage: bool, io_usage: bool, + display_throughput: bool, ) -> Self { Self { frequency, @@ -80,6 +88,7 @@ impl Config { mem_usage, disk_usage, io_usage, + display_throughput, } } /// check the frequency interval @@ -117,6 +126,12 @@ impl Config { pub const fn smoothing_factor(&self) -> f32 { self.smoothing_factor } + + /// check if throughput logging is enabled + #[must_use] + pub const fn display_throughput(&self) -> bool { + self.display_throughput + } } impl Default for Config { @@ -129,6 +144,7 @@ impl Default for Config { mem_usage: default_mem_usage(), disk_usage: default_disk_usage(), io_usage: default_io_usage(), + display_throughput: default_display_throughput(), } } } diff --git a/rust/otap-dataflow/crates/otap/src/perf_exporter/exporter.rs b/rust/otap-dataflow/crates/otap/src/perf_exporter/exporter.rs index e29aad3dae..a72ffc18b5 100644 --- a/rust/otap-dataflow/crates/otap/src/perf_exporter/exporter.rs +++ b/rust/otap-dataflow/crates/otap/src/perf_exporter/exporter.rs @@ -24,6 +24,14 @@ use crate::pdata::OtapPdata; use crate::perf_exporter::config::Config; use crate::perf_exporter::metrics::PerfExporterPdataMetrics; use async_trait::async_trait; +#[cfg(feature = "jemalloc-metrics")] +use jemalloc_ctl::thread::{allocatedp, deallocatedp}; +#[cfg(target_os = "linux")] +use nix::sys::resource::{UsageWho, getrusage}; +#[cfg(target_os = "linux")] +use nix::sys::time::TimeValLike; +#[cfg(target_os = "linux")] +use nix::unistd::{SysconfVar, gettid, sysconf}; use otap_df_config::SignalType; use otap_df_config::node::NodeUserConfig; use otap_df_engine::ConsumerEffectHandlerExtension; @@ -40,6 +48,7 @@ use otap_df_engine::{ExporterFactory, distributed_slice}; use otap_df_pdata::otap::OtapArrowRecords; use otap_df_telemetry::metrics::{MetricSet, MetricSetHandler}; use serde_json::Value; +use std::collections::VecDeque; use std::sync::Arc; use std::time::Instant; use tokio::time::Duration; @@ -52,6 +61,60 @@ pub struct PerfExporter { config: Config, metrics: MetricSet, pdata_metrics: MetricSet, + last_telemetry: Instant, + thread_usage: ThreadUsageTracker, + last_window: VecDeque, +} + +#[derive(Clone, Copy)] +struct IntervalSample { + elapsed_secs: f64, + logs: u64, + metrics: u64, + spans: u64, + end: Instant, +} + +#[derive(Clone, Copy, Debug)] +struct ThreadUsageSample { + user_cpu_ns: u64, + system_cpu_ns: u64, + rss_bytes: u64, + minor_faults: u64, + major_faults: u64, + #[cfg(feature = "jemalloc-metrics")] + jemalloc_allocated: u64, + #[cfg(feature = "jemalloc-metrics")] + jemalloc_deallocated: u64, +} + +impl ThreadUsageSample {} + +#[derive(Debug)] +struct ThreadUsageTracker { + last: Option, + page_size: u64, +} + +impl ThreadUsageTracker { + fn new() -> Self { + Self { + last: None, + page_size: detect_page_size(), + } + } + + fn record_delta(&mut self) -> Option { + let current = capture_thread_usage(self.page_size)?; + let previous = self.last.replace(current); + Some(ThreadUsageDelta { previous, current }) + } +} + +#[derive(Clone, Copy, Debug)] +struct ThreadUsageDelta { + previous: Option, + current: ThreadUsageSample, } /// Declares the OTAP Perf exporter as a local exporter factory @@ -81,11 +144,15 @@ impl PerfExporter { pub fn new(pipeline_ctx: PipelineContext, config: Config) -> Self { let metrics = pipeline_ctx.register_metrics::(); let pdata_metrics = pipeline_ctx.register_metrics::(); + let last_telemetry = Instant::now(); PerfExporter { config, metrics, pdata_metrics, + last_telemetry, + thread_usage: ThreadUsageTracker::new(), + last_window: VecDeque::new(), } } @@ -119,6 +186,77 @@ impl PerfExporter { } } +fn detect_page_size() -> u64 { + #[cfg(target_os = "linux")] + { + sysconf(SysconfVar::PAGE_SIZE) + .ok() + .flatten() + .and_then(|value| u64::try_from(value).ok()) + .filter(|value| *value > 0) + .unwrap_or(4096) + } + #[cfg(not(target_os = "linux"))] + { + 4096 + } +} + +#[cfg(target_os = "linux")] +fn capture_thread_usage(page_size: u64) -> Option { + let usage = getrusage(UsageWho::RUSAGE_THREAD).ok()?; + let user_cpu_ns = usage.user_time().num_nanoseconds().max(0) as u64; + let system_cpu_ns = usage.system_time().num_nanoseconds().max(0) as u64; + let rss_bytes = read_thread_rss_bytes(page_size).unwrap_or(0); + let minor_faults = usage.minor_page_faults().max(0) as u64; + let major_faults = usage.major_page_faults().max(0) as u64; + #[cfg(feature = "jemalloc-metrics")] + let jemalloc = capture_jemalloc_thread_alloc().unwrap_or((0, 0)); + + Some(ThreadUsageSample { + user_cpu_ns, + system_cpu_ns, + rss_bytes, + minor_faults, + major_faults, + #[cfg(feature = "jemalloc-metrics")] + jemalloc_allocated: jemalloc.0, + #[cfg(feature = "jemalloc-metrics")] + jemalloc_deallocated: jemalloc.1, + }) +} + +#[cfg(not(target_os = "linux"))] +fn capture_thread_usage(_page_size: u64) -> Option { + None +} + +#[cfg(feature = "jemalloc-metrics")] +#[allow(dead_code)] +fn capture_jemalloc_thread_alloc() -> Option<(u64, u64)> { + let allocated = allocatedp::read().ok()?; + let deallocated = deallocatedp::read().ok()?; + Some((allocated.get(), deallocated.get())) +} + +#[cfg(not(feature = "jemalloc-metrics"))] +#[allow(dead_code)] +fn capture_jemalloc_thread_alloc() -> Option<(u64, u64)> { + None +} + +#[cfg(target_os = "linux")] +fn read_thread_rss_bytes(page_size: u64) -> Option { + let tid = gettid(); + let statm_path = format!("/proc/self/task/{}/statm", tid); + let contents = std::fs::read_to_string(statm_path).ok()?; + let mut parts = contents.split_whitespace(); + let _ = parts.next()?; + let resident_pages: u64 = parts.next()?.parse().ok()?; + + Some(resident_pages.saturating_mul(page_size)) +} + #[async_trait(?Send)] impl local::Exporter for PerfExporter { async fn start( @@ -140,9 +278,188 @@ impl local::Exporter for PerfExporter { loop { let msg = msg_chan.recv().await?; match msg { - Message::Control(NodeControlMsg::CollectTelemetry { - mut metrics_reporter, - }) => { + Message::Control(NodeControlMsg::CollectTelemetry { metrics_reporter }) => { + let now = Instant::now(); + let elapsed_secs = now + .duration_since(self.last_telemetry) + .as_secs_f64() + .max(1e-6); + + if self.config.display_throughput() { + // Capture the last interval sample before clearing metrics. + self.last_window.push_back(IntervalSample { + elapsed_secs, + logs: self.metrics.logs.get(), + metrics: self.metrics.metrics.get(), + spans: self.metrics.spans.get(), + end: now, + }); + + // Keep only the last 10s worth of samples. + let window = Duration::from_secs(10); + while let Some(front) = self.last_window.front() { + if now.duration_since(front.end) > window { + _ = self.last_window.pop_front(); + } else { + break; + } + } + + let (total_logs, total_metrics, total_spans, total_secs) = self + .last_window + .iter() + .fold((0u64, 0u64, 0u64, 0f64), |(l, m, s, d), sample| { + ( + l + sample.logs, + m + sample.metrics, + s + sample.spans, + d + sample.elapsed_secs, + ) + }); + + let total_secs = total_secs.max(1e-6); + let logs_per_sec = total_logs as f64 / total_secs; + let metrics_per_sec = total_metrics as f64 / total_secs; + let spans_per_sec = total_spans as f64 / total_secs; + + effect_handler + .info(&format!( + "PerfExporter throughput (avg last {:.2}s): logs {:.2}/s, metrics {:.2}/s, spans {:.2}/s", + total_secs, logs_per_sec, metrics_per_sec, spans_per_sec, + )) + .await; + } + + if self.config.cpu_usage() || self.config.mem_usage() { + if let Some(delta) = self.thread_usage.record_delta() { + let mut status_segments = Vec::new(); + + if self.config.cpu_usage() { + if let Some(previous) = delta.previous { + let cpu_user_delta = delta + .current + .user_cpu_ns + .saturating_sub(previous.user_cpu_ns); + let cpu_system_delta = delta + .current + .system_cpu_ns + .saturating_sub(previous.system_cpu_ns); + let cpu_total_delta = + cpu_user_delta.saturating_add(cpu_system_delta); + + self.metrics.thread_cpu_user_ns.add(cpu_user_delta); + self.metrics.thread_cpu_system_ns.add(cpu_system_delta); + self.metrics.thread_cpu_total_ns.add(cpu_total_delta); + + let cpu_total_pct = + (cpu_total_delta as f64 / (elapsed_secs * 1e9)) * 100.0; + let cpu_user_pct = + (cpu_user_delta as f64 / (elapsed_secs * 1e9)) * 100.0; + let cpu_system_pct = + (cpu_system_delta as f64 / (elapsed_secs * 1e9)) * 100.0; + + status_segments.push(format!( + "CPU total {:.2}% (user {:.2}%, sys {:.2}%)", + cpu_total_pct, cpu_user_pct, cpu_system_pct, + )); + } + } + + if self.config.mem_usage() { + let rss_bytes = delta.current.rss_bytes; + self.metrics.thread_rss_bytes.set(rss_bytes); + + let mut rss_delta = 0; + let mut fault_bytes = 0; + #[cfg(feature = "jemalloc-metrics")] + let mut alloc_delta = 0; + #[cfg(feature = "jemalloc-metrics")] + let mut dealloc_delta = 0; + + if let Some(previous) = delta.previous { + rss_delta = rss_bytes.abs_diff(previous.rss_bytes); + + let minor_faults = delta + .current + .minor_faults + .saturating_sub(previous.minor_faults); + let major_faults = delta + .current + .major_faults + .saturating_sub(previous.major_faults); + + let minor_fault_bytes = + minor_faults.saturating_mul(self.thread_usage.page_size); + let major_fault_bytes = + major_faults.saturating_mul(self.thread_usage.page_size); + fault_bytes = + minor_fault_bytes.saturating_add(major_fault_bytes); + + self.metrics.thread_rss_delta_bytes.add(rss_delta); + self.metrics.thread_minor_fault_bytes.add(minor_fault_bytes); + self.metrics.thread_major_fault_bytes.add(major_fault_bytes); + self.metrics.thread_fault_bytes.add(fault_bytes); + + #[cfg(feature = "jemalloc-metrics")] + { + alloc_delta = delta + .current + .jemalloc_allocated + .saturating_sub(previous.jemalloc_allocated); + dealloc_delta = delta + .current + .jemalloc_deallocated + .saturating_sub(previous.jemalloc_deallocated); + + self.metrics.thread_alloc_bytes.add(alloc_delta); + self.metrics.thread_dealloc_bytes.add(dealloc_delta); + } + } + + let rss_mib = rss_bytes as f64 / (1024.0 * 1024.0); + let rss_throughput_mib = + rss_delta as f64 / (1024.0 * 1024.0) / elapsed_secs; + let fault_throughput_mib = + fault_bytes as f64 / (1024.0 * 1024.0) / elapsed_secs; + #[cfg(feature = "jemalloc-metrics")] + let alloc_throughput_mib = + alloc_delta as f64 / (1024.0 * 1024.0) / elapsed_secs; + #[cfg(feature = "jemalloc-metrics")] + let dealloc_throughput_mib = + dealloc_delta as f64 / (1024.0 * 1024.0) / elapsed_secs; + + #[cfg(feature = "jemalloc-metrics")] + status_segments.push(format!( + "Memory {:.2} MiB RSS, {:.2} MiB/s faults, {:.2} MiB/s Δ (alloc {:.2} MiB/s, free {:.2} MiB/s, {} B RSS change, {} B fault bytes)", + rss_mib, + fault_throughput_mib, + rss_throughput_mib, + alloc_throughput_mib, + dealloc_throughput_mib, + rss_delta, + fault_bytes, + )); + #[cfg(not(feature = "jemalloc-metrics"))] + status_segments.push(format!( + "Memory {:.2} MiB RSS, {:.2} MiB/s faults, {:.2} MiB/s Δ ({} B RSS change, {} B fault bytes)", + rss_mib, fault_throughput_mib, rss_throughput_mib, rss_delta, fault_bytes, + )); + } + + if !status_segments.is_empty() { + effect_handler + .info(&format!( + "PerfExporter thread stats over {:.2}s: {}", + elapsed_secs, + status_segments.join(" | "), + )) + .await; + } + } + } + + self.last_telemetry = now; + _ = metrics_reporter.report(&mut self.metrics); _ = metrics_reporter.report(&mut self.pdata_metrics); } @@ -382,7 +699,7 @@ mod tests { #[test] fn test_exporter_local() { let test_runtime = TestRuntime::new(); - let config = Config::new(1000, 0.3, true, true, true, true, true); + let config = Config::new(1000, 0.3, true, true, true, true, true, false); let node_config = Arc::new(NodeUserConfig::new_exporter_config(OTAP_PERF_EXPORTER_URN)); let metrics_registry_handle = MetricsRegistryHandle::new(); let controller_ctx = ControllerContext::new(metrics_registry_handle.clone()); diff --git a/rust/otap-dataflow/crates/otap/src/perf_exporter/metrics.rs b/rust/otap-dataflow/crates/otap/src/perf_exporter/metrics.rs index d5c601fa31..28a2b66270 100644 --- a/rust/otap-dataflow/crates/otap/src/perf_exporter/metrics.rs +++ b/rust/otap-dataflow/crates/otap/src/perf_exporter/metrics.rs @@ -3,7 +3,7 @@ //! Metrics for the OTAP PerfExporter node. -use otap_df_telemetry::instrument::Counter; +use otap_df_telemetry::instrument::{Counter, Gauge}; use otap_df_telemetry_macros::metric_set; /// Pdata-oriented metrics for the OTAP PerfExporter. @@ -22,4 +22,34 @@ pub struct PerfExporterPdataMetrics { /// Number of metrics received. #[metric(unit = "{metric}")] pub metrics: Counter, + /// Nanoseconds of user CPU time consumed by this thread since the last telemetry tick. + #[metric(unit = "{ns}")] + pub thread_cpu_user_ns: Counter, + /// Nanoseconds of system CPU time consumed by this thread since the last telemetry tick. + #[metric(unit = "{ns}")] + pub thread_cpu_system_ns: Counter, + /// Nanoseconds of total CPU time consumed by this thread since the last telemetry tick. + #[metric(unit = "{ns}")] + pub thread_cpu_total_ns: Counter, + /// Resident set size of this thread in bytes at the last telemetry tick. + #[metric(unit = "{byte}")] + pub thread_rss_bytes: Gauge, + /// Absolute change in RSS for this thread since the previous telemetry tick (bytes). + #[metric(unit = "{byte}")] + pub thread_rss_delta_bytes: Counter, + /// Bytes accounted for by minor page faults for this thread since the last telemetry tick. + #[metric(unit = "{byte}")] + pub thread_minor_fault_bytes: Counter, + /// Bytes accounted for by major page faults for this thread since the last telemetry tick. + #[metric(unit = "{byte}")] + pub thread_major_fault_bytes: Counter, + /// Total bytes touched by page faults for this thread since the last telemetry tick. + #[metric(unit = "{byte}")] + pub thread_fault_bytes: Counter, + /// Bytes allocated by this thread since the last telemetry tick (jemalloc). + #[metric(unit = "{byte}")] + pub thread_alloc_bytes: Counter, + /// Bytes deallocated by this thread since the last telemetry tick (jemalloc). + #[metric(unit = "{byte}")] + pub thread_dealloc_bytes: Counter, } diff --git a/rust/otap-dataflow/crates/otap/src/retry_processor.rs b/rust/otap-dataflow/crates/otap/src/retry_processor.rs index 7c0479c621..c51d026b42 100644 --- a/rust/otap-dataflow/crates/otap/src/retry_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/retry_processor.rs @@ -632,9 +632,7 @@ impl Processor for RetryProcessor { } Ok(()) } - NodeControlMsg::CollectTelemetry { - mut metrics_reporter, - } => metrics_reporter + NodeControlMsg::CollectTelemetry { metrics_reporter } => metrics_reporter .report(&mut self.metrics) .map_err(|e| Error::InternalError { message: e.to_string(), diff --git a/rust/otap-dataflow/crates/otap/src/signal_type_router.rs b/rust/otap-dataflow/crates/otap/src/signal_type_router.rs index d5be165789..23893bbc7d 100644 --- a/rust/otap-dataflow/crates/otap/src/signal_type_router.rs +++ b/rust/otap-dataflow/crates/otap/src/signal_type_router.rs @@ -161,10 +161,7 @@ impl local::Processor for SignalTypeRouter { ) -> Result<(), EngineError> { match msg { Message::Control(ctrl) => { - if let NodeControlMsg::CollectTelemetry { - mut metrics_reporter, - } = ctrl - { + if let NodeControlMsg::CollectTelemetry { metrics_reporter } = ctrl { if let Some(m) = self.metrics.as_mut() { let _ = metrics_reporter.report(m); } diff --git a/rust/otap-dataflow/crates/otap/src/syslog_cef_receiver.rs b/rust/otap-dataflow/crates/otap/src/syslog_cef_receiver.rs index e5986e33e7..049713d555 100644 --- a/rust/otap-dataflow/crates/otap/src/syslog_cef_receiver.rs +++ b/rust/otap-dataflow/crates/otap/src/syslog_cef_receiver.rs @@ -149,7 +149,7 @@ impl local::Receiver for SyslogCefReceiver { let _ = timer_cancel_handle.cancel().await; break; } - Ok(NodeControlMsg::CollectTelemetry { mut metrics_reporter }) => { + Ok(NodeControlMsg::CollectTelemetry { metrics_reporter }) => { let mut m = self.metrics.borrow_mut(); let _ = metrics_reporter.report(&mut m); } @@ -375,7 +375,7 @@ impl local::Receiver for SyslogCefReceiver { let _ = timer_cancel_handle.cancel().await; break; } - Ok(NodeControlMsg::CollectTelemetry { mut metrics_reporter }) => { + Ok(NodeControlMsg::CollectTelemetry { metrics_reporter }) => { let mut m = self.metrics.borrow_mut(); let _ = metrics_reporter.report(&mut m); } diff --git a/rust/otap-dataflow/crates/otap/src/testing/mod.rs b/rust/otap-dataflow/crates/otap/src/testing/mod.rs index 86e4dee33f..d613abb767 100644 --- a/rust/otap-dataflow/crates/otap/src/testing/mod.rs +++ b/rust/otap-dataflow/crates/otap/src/testing/mod.rs @@ -65,7 +65,7 @@ pub fn create_test_pdata() -> OtapPdata { let mut otlp_bytes = vec![]; otlp_service_req.encode(&mut otlp_bytes).unwrap(); - OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(otlp_bytes).into()) + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(otlp_bytes.into()).into()) } /// Simple exporter test where there is NO subscribe_to() in the context. diff --git a/rust/otap-dataflow/crates/pdata/Cargo.toml b/rust/otap-dataflow/crates/pdata/Cargo.toml index 77028a2b24..35dcb39574 100644 --- a/rust/otap-dataflow/crates/pdata/Cargo.toml +++ b/rust/otap-dataflow/crates/pdata/Cargo.toml @@ -34,6 +34,7 @@ thiserror = { workspace = true } tokio = { workspace = true } tonic = { workspace = true } tonic-prost = { workspace = true } +bytes = { workspace = true } [dev-dependencies] tokio-stream = { workspace = true } diff --git a/rust/otap-dataflow/crates/pdata/src/otlp/common.rs b/rust/otap-dataflow/crates/pdata/src/otlp/common.rs index b9d18a290c..fcbb819a55 100644 --- a/rust/otap-dataflow/crates/pdata/src/otlp/common.rs +++ b/rust/otap-dataflow/crates/pdata/src/otlp/common.rs @@ -314,6 +314,14 @@ impl ProtoBuffer { Self { buffer: Vec::new() } } + /// Construct a new buffer with at least the provided capacity. + #[must_use] + pub fn with_capacity(capacity: usize) -> Self { + Self { + buffer: Vec::with_capacity(capacity), + } + } + #[must_use] pub fn into_bytes(self) -> Vec { self.buffer @@ -363,6 +371,12 @@ impl ProtoBuffer { self.buffer.len() } + /// Returns the current capacity of the underlying buffer. + #[must_use] + pub fn capacity(&self) -> usize { + self.buffer.capacity() + } + #[must_use] pub const fn is_empty(&self) -> bool { self.buffer.is_empty() diff --git a/rust/otap-dataflow/crates/pdata/src/otlp/mod.rs b/rust/otap-dataflow/crates/pdata/src/otlp/mod.rs index eed0fa613a..3f29623070 100644 --- a/rust/otap-dataflow/crates/pdata/src/otlp/mod.rs +++ b/rust/otap-dataflow/crates/pdata/src/otlp/mod.rs @@ -4,6 +4,7 @@ //! This module contains traits and utilities for OTLP (OpenTelemetry //! Protocol) message types. +use bytes::Bytes; pub use otap_df_pdata_otlp_macros::Message; // Required for derived code pub use otap_df_pdata_otlp_macros::qualified; // Required for derived code @@ -30,11 +31,11 @@ mod tests; #[derive(Clone, Debug)] pub enum OtlpProtoBytes { /// protobuf serialized ExportLogsServiceRequest - ExportLogsRequest(Vec), + ExportLogsRequest(Bytes), /// protobuf serialized ExportMetricsServiceRequest - ExportMetricsRequest(Vec), + ExportMetricsRequest(Bytes), /// protobuf serialized ExportTracesServiceRequest - ExportTracesRequest(Vec), + ExportTracesRequest(Bytes), } impl OtlpProtoBytes { @@ -44,7 +45,7 @@ impl OtlpProtoBytes { match self { OtlpProtoBytes::ExportLogsRequest(bytes) | OtlpProtoBytes::ExportMetricsRequest(bytes) - | OtlpProtoBytes::ExportTracesRequest(bytes) => bytes.as_slice(), + | OtlpProtoBytes::ExportTracesRequest(bytes) => bytes.as_ref(), } } } diff --git a/rust/otap-dataflow/crates/pdata/src/payload.rs b/rust/otap-dataflow/crates/pdata/src/payload.rs index 16d4a77348..e060252aa1 100644 --- a/rust/otap-dataflow/crates/pdata/src/payload.rs +++ b/rust/otap-dataflow/crates/pdata/src/payload.rs @@ -49,7 +49,7 @@ //! otlp_service_req.encode(&mut buf).unwrap(); //! //! // Create a new OtapPayload from OTLP bytes -//! let payload: OtapPayload = OtlpProtoBytes::ExportLogsRequest(buf).into(); +//! let payload: OtapPayload = OtlpProtoBytes::ExportLogsRequest(buf.into()).into(); //! //! // Convert to OTAP records //! let otap_arrow_records: OtapArrowRecords = payload.try_into().unwrap(); @@ -229,7 +229,7 @@ impl OtapPayloadHelpers for OtlpProtoBytes { fn num_items(&self) -> usize { match self { Self::ExportLogsRequest(bytes) => { - let logs_data_view = RawLogsData::new(bytes); + let logs_data_view = RawLogsData::new(bytes.as_ref()); use crate::views::logs::{LogsDataView, ResourceLogsView, ScopeLogsView}; logs_data_view .resources() @@ -241,7 +241,7 @@ impl OtapPayloadHelpers for OtlpProtoBytes { .sum() } Self::ExportTracesRequest(bytes) => { - let traces_data_view = RawTraceData::new(bytes); + let traces_data_view = RawTraceData::new(bytes.as_ref()); use crate::views::trace::{ResourceSpansView, ScopeSpansView, TracesView}; traces_data_view .resources() @@ -304,20 +304,26 @@ impl TryFrom for OtlpProtoBytes { let mut buffer = ProtoBuffer::new(); logs_encoder.encode(&mut value, &mut buffer)?; - Ok(Self::ExportLogsRequest(buffer.into_bytes())) + Ok(Self::ExportLogsRequest(bytes::Bytes::from( + buffer.into_bytes(), + ))) } OtapArrowRecords::Metrics(_) => { let mut metrics_encoder = MetricsProtoBytesEncoder::new(); let mut buffer = ProtoBuffer::new(); metrics_encoder.encode(&mut value, &mut buffer)?; - Ok(Self::ExportMetricsRequest(buffer.into_bytes())) + Ok(Self::ExportMetricsRequest(bytes::Bytes::from( + buffer.into_bytes(), + ))) } OtapArrowRecords::Traces(_) => { let mut traces_encoder = TracesProtoBytesEncoder::new(); let mut buffer = ProtoBuffer::new(); traces_encoder.encode(&mut value, &mut buffer)?; - Ok(Self::ExportTracesRequest(buffer.into_bytes())) + Ok(Self::ExportTracesRequest(bytes::Bytes::from( + buffer.into_bytes(), + ))) } } } @@ -329,19 +335,19 @@ impl TryFrom for OtapArrowRecords { fn try_from(value: OtlpProtoBytes) -> Result { match value { OtlpProtoBytes::ExportLogsRequest(bytes) => { - let logs_data_view = RawLogsData::new(&bytes); + let logs_data_view = RawLogsData::new(bytes.as_ref()); let otap_batch = encode_logs_otap_batch(&logs_data_view)?; Ok(otap_batch) } OtlpProtoBytes::ExportTracesRequest(bytes) => { - let trace_data_view = RawTraceData::new(&bytes); + let trace_data_view = RawTraceData::new(bytes.as_ref()); let otap_batch = encode_spans_otap_batch(&trace_data_view)?; Ok(otap_batch) } OtlpProtoBytes::ExportMetricsRequest(bytes) => { - let metrics_data_view = RawMetricsData::new(&bytes); + let metrics_data_view = RawMetricsData::new(bytes.as_ref()); let otap_batch = encode_metrics_otap_batch(&metrics_data_view)?; Ok(otap_batch) @@ -387,7 +393,7 @@ mod test { let otlp_service_req = create_test_logs(); otlp_service_req.encode(&mut otlp_bytes).unwrap(); - let pdata: OtapPayload = OtlpProtoBytes::ExportLogsRequest(otlp_bytes).into(); + let pdata: OtapPayload = OtlpProtoBytes::ExportLogsRequest(otlp_bytes.into()).into(); // test can go OtlpProtoBytes -> OtapBatch & back let otap_batch: OtapArrowRecords = pdata.try_into().unwrap(); @@ -413,7 +419,7 @@ mod test { fn roundtrip_otlp_otap_logs(otlp_service_req: ExportLogsServiceRequest) { let mut otlp_bytes = vec![]; otlp_service_req.encode(&mut otlp_bytes).unwrap(); - let pdata: OtapPayload = OtlpProtoBytes::ExportLogsRequest(otlp_bytes).into(); + let pdata: OtapPayload = OtlpProtoBytes::ExportLogsRequest(otlp_bytes.into()).into(); // test can go OtlpProtoBytes (written by prost) -> OtapBatch & back (using prost) let otap_batch: OtapArrowRecords = pdata.try_into().unwrap(); @@ -439,7 +445,7 @@ mod test { fn roundtrip_otlp_otap_traces(otlp_service_req: ExportTraceServiceRequest) { let mut otlp_bytes = vec![]; otlp_service_req.encode(&mut otlp_bytes).unwrap(); - let pdata: OtapPayload = OtlpProtoBytes::ExportTracesRequest(otlp_bytes).into(); + let pdata: OtapPayload = OtlpProtoBytes::ExportTracesRequest(otlp_bytes.into()).into(); // test can go OtlpBytes (written by prost) -> OtapBatch & back (using prost) let otap_batch: OtapArrowRecords = pdata.try_into().unwrap(); @@ -465,7 +471,7 @@ mod test { fn roundtrip_otlp_otap_metrics(otlp_service_request: ExportMetricsServiceRequest) { let mut otlp_bytes = vec![]; otlp_service_request.encode(&mut otlp_bytes).unwrap(); - let pdata: OtapPayload = OtlpProtoBytes::ExportMetricsRequest(otlp_bytes).into(); + let pdata: OtapPayload = OtlpProtoBytes::ExportMetricsRequest(otlp_bytes.into()).into(); // test can go OtlpBytes (written by prost) to OTAP & back (using prost) let otap_batch: OtapArrowRecords = pdata.try_into().unwrap(); @@ -1204,9 +1210,9 @@ mod test { #[test] fn test_signal_type() { // Test signal_type for OtlpProtoBytes variants - let logs_bytes = OtlpProtoBytes::ExportLogsRequest(vec![]); - let metrics_bytes = OtlpProtoBytes::ExportMetricsRequest(vec![]); - let traces_bytes = OtlpProtoBytes::ExportTracesRequest(vec![]); + let logs_bytes = OtlpProtoBytes::ExportLogsRequest(bytes::Bytes::new()); + let metrics_bytes = OtlpProtoBytes::ExportMetricsRequest(bytes::Bytes::new()); + let traces_bytes = OtlpProtoBytes::ExportTracesRequest(bytes::Bytes::new()); assert_eq!(logs_bytes.signal_type(), SignalType::Logs); assert_eq!(metrics_bytes.signal_type(), SignalType::Metrics); @@ -1222,7 +1228,7 @@ mod test { assert_eq!(traces_records.signal_type(), SignalType::Traces); // Test signal_type for OtapPdata variants - let pdata_logs: OtapPayload = OtlpProtoBytes::ExportLogsRequest(vec![]).into(); + let pdata_logs: OtapPayload = OtlpProtoBytes::ExportLogsRequest(bytes::Bytes::new()).into(); let pdata_metrics: OtapPayload = OtapArrowRecords::Metrics(Default::default()).into(); assert_eq!(pdata_logs.signal_type(), SignalType::Logs); assert_eq!(pdata_metrics.signal_type(), SignalType::Metrics); diff --git a/rust/otap-dataflow/crates/telemetry/src/registry.rs b/rust/otap-dataflow/crates/telemetry/src/registry.rs index df4d994523..061aa5fb2f 100644 --- a/rust/otap-dataflow/crates/telemetry/src/registry.rs +++ b/rust/otap-dataflow/crates/telemetry/src/registry.rs @@ -7,6 +7,7 @@ //! dynamic dispatch. use crate::attributes::AttributeSetHandler; +use crate::descriptor::Instrument; use crate::descriptor::MetricsDescriptor; use crate::descriptor::MetricsField; use crate::metrics::{MetricSet, MetricSetHandler}; @@ -207,20 +208,28 @@ impl MetricsRegistry { entry .metric_values .iter_mut() - .zip(metrics_values) - .for_each(|(e, v)| { - #[cfg(feature = "unchecked-arithmetic")] - { - // SAFETY: Metric values are expected to be well-behaved and not overflow - // in typical telemetry scenarios. This is a performance optimization for - // hot path metric accumulation. - *e = e.wrapping_add(*v); - } - #[cfg(not(feature = "unchecked-arithmetic"))] - { - *e += v; - } - }); + .zip(metrics_values.iter().copied()) + .zip(entry.metrics_descriptor.metrics.iter()) + .for_each( + |((current_value, new_value), field)| match field.instrument { + Instrument::Gauge => { + *current_value = new_value; + } + Instrument::Counter | Instrument::UpDownCounter | Instrument::Histogram => { + #[cfg(feature = "unchecked-arithmetic")] + { + // SAFETY: Metric values are expected to be well-behaved and not overflow + // in typical telemetry scenarios. This is a performance optimization for + // hot path metric accumulation. + *current_value = current_value.wrapping_add(new_value); + } + #[cfg(not(feature = "unchecked-arithmetic"))] + { + *current_value += new_value; + } + } + }, + ); } else { // TODO: consider logging missing key } @@ -439,6 +448,59 @@ mod tests { } } + #[derive(Debug)] + struct GaugeMetricSet { + values: Vec, + } + + impl GaugeMetricSet { + fn new() -> Self { + Self { values: vec![0, 0] } + } + } + + impl Default for GaugeMetricSet { + fn default() -> Self { + Self::new() + } + } + + static GAUGE_METRICS_DESCRIPTOR: MetricsDescriptor = MetricsDescriptor { + name: "gauge_metrics", + metrics: &[ + MetricsField { + name: "counter", + unit: "1", + brief: "Counter field", + instrument: Instrument::Counter, + }, + MetricsField { + name: "gauge", + unit: "1", + brief: "Gauge field", + instrument: Instrument::Gauge, + }, + ], + }; + + impl MetricSetHandler for GaugeMetricSet { + fn descriptor(&self) -> &'static MetricsDescriptor { + &GAUGE_METRICS_DESCRIPTOR + } + + fn snapshot_values(&self) -> Vec { + self.values.clone() + } + + fn clear_values(&mut self) { + self.values.iter_mut().for_each(|v| *v = 0); + } + + fn needs_flush(&self) -> bool { + self.values.iter().any(|&v| v != 0) + } + } + #[derive(Debug)] struct MockAttributeSet { // Store the attribute values as owned data that we can return references to @@ -520,6 +582,27 @@ mod tests { assert_eq!(accumulated_values, vec![15, 35]); } + #[test] + fn test_accumulate_snapshot_overwrites_gauge() { + let handle = MetricsRegistryHandle::new(); + let attrs = MockAttributeSet::new("test_value".to_string()); + + let metric_set: MetricSet = handle.register(attrs); + let metrics_key = metric_set.key; + + handle.accumulate_snapshot(metrics_key, &[5, 100]); + handle.accumulate_snapshot(metrics_key, &[7, 50]); + + let mut accumulated_values = Vec::new(); + handle.visit_metrics_and_reset(|_desc, _attrs, iter| { + for (field, value) in iter { + accumulated_values.push((field.name, value)); + } + }); + + assert_eq!(accumulated_values, vec![("counter", 12), ("gauge", 50)]); + } + #[test] fn test_accumulate_snapshot_invalid_key() { let handle = MetricsRegistryHandle::new(); diff --git a/rust/otap-dataflow/crates/telemetry/src/reporter.rs b/rust/otap-dataflow/crates/telemetry/src/reporter.rs index 867800f645..0eaefef8c2 100644 --- a/rust/otap-dataflow/crates/telemetry/src/reporter.rs +++ b/rust/otap-dataflow/crates/telemetry/src/reporter.rs @@ -37,7 +37,7 @@ impl MetricsReporter { /// Report multivariate metrics and reset the metrics if successful. pub fn report( - &mut self, + &self, metrics: &mut MetricSet, ) -> Result<(), Error> { if !metrics.needs_flush() { diff --git a/rust/otap-dataflow/src/main.rs b/rust/otap-dataflow/src/main.rs index e7fd203e14..4fef745512 100644 --- a/rust/otap-dataflow/src/main.rs +++ b/rust/otap-dataflow/src/main.rs @@ -4,6 +4,9 @@ //! Create and run a multi-core pipeline use clap::Parser; +#[cfg(feature = "jemalloc-allocator")] +use jemallocator::Jemalloc; +#[cfg(not(feature = "jemalloc-allocator"))] use mimalloc_rust::*; use otap_df_config::pipeline::PipelineConfig; use otap_df_config::pipeline_group::{CoreAllocation, CoreRange, Quota}; @@ -12,8 +15,13 @@ use otap_df_controller::Controller; use otap_df_otap::OTAP_PIPELINE_FACTORY; use std::path::PathBuf; +#[cfg(feature = "jemalloc-allocator")] #[global_allocator] -static GLOBAL_MIMALLOC: GlobalMiMalloc = GlobalMiMalloc; +static GLOBAL_ALLOC: Jemalloc = Jemalloc; + +#[cfg(not(feature = "jemalloc-allocator"))] +#[global_allocator] +static GLOBAL_ALLOC: GlobalMiMalloc = GlobalMiMalloc; #[derive(Parser)] #[command( @@ -84,6 +92,7 @@ fn parse_core_id_range(s: &str) -> Result { } fn main() -> Result<(), Box> { + init_logging(); let args = Args::parse(); // For now, we predefine pipeline group and pipeline IDs. @@ -145,6 +154,13 @@ fn main() -> Result<(), Box> { } } +fn init_logging() { + let env = env_logger::Env::default().default_filter_or("info"); + let _ = env_logger::Builder::from_env(env) + .format_timestamp_millis() + .try_init(); +} + fn system_info() -> String { // Your custom logic here - this could read files, check system state, etc. let available_cores = std::thread::available_parallelism() diff --git a/tools/pipeline_perf_test/test_suites/integration/templates/configs/backend/config.yaml.j2 b/tools/pipeline_perf_test/test_suites/integration/templates/configs/backend/config.yaml.j2 index 1aaac7cd7c..e31b12b28c 100644 --- a/tools/pipeline_perf_test/test_suites/integration/templates/configs/backend/config.yaml.j2 +++ b/tools/pipeline_perf_test/test_suites/integration/templates/configs/backend/config.yaml.j2 @@ -8,14 +8,13 @@ settings: {# Determine plugin URN and config for receiver #} {%- if receiver_type == "otap" -%} - {% set plugin_urn = "urn:otel:otap:receiver" %} + {% set plugin_urn = "urn:otel:otel:receiver" %} {% set compression = compression_method | default("zstd") %} {% set extra_config = { - "response_stream_channel_size": 100, "compression_method": compression } %} {%- elif receiver_type == "otlp" -%} - {% set plugin_urn = "urn:otel:otlp:receiver" %} + {% set plugin_urn = "urn:otel:otel:receiver" %} {% set compression = compression_method | default("gzip") %} {% set extra_config = { "compression_method": compression @@ -53,4 +52,4 @@ nodes: cpu_usage: false mem_usage: false disk_usage: false - io_usage: false \ No newline at end of file + io_usage: false diff --git a/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/backpressure/otap-attr-otap.yaml b/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/backpressure/otap-attr-otap.yaml index 37f55fac39..4eae843102 100644 --- a/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/backpressure/otap-attr-otap.yaml +++ b/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/backpressure/otap-attr-otap.yaml @@ -6,7 +6,7 @@ settings: nodes: receiver: kind: receiver - plugin_urn: "urn:otel:otap:receiver" + plugin_urn: "urn:otel:otap2:receiver" out_ports: out_port: destinations: @@ -14,7 +14,6 @@ nodes: dispatch_strategy: round_robin config: listening_addr: "0.0.0.0:4317" - response_stream_channel_size: 100 compression_method: zstd wait_for_result: true diff --git a/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/backpressure/otap-attr-otlp.yaml b/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/backpressure/otap-attr-otlp.yaml index a2acc438e3..f3293b6924 100644 --- a/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/backpressure/otap-attr-otlp.yaml +++ b/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/backpressure/otap-attr-otlp.yaml @@ -6,7 +6,7 @@ settings: nodes: receiver: kind: receiver - plugin_urn: "urn:otel:otap:receiver" + plugin_urn: "urn:otel:otap2:receiver" out_ports: out_port: destinations: @@ -14,7 +14,6 @@ nodes: dispatch_strategy: round_robin config: listening_addr: "0.0.0.0:4317" - response_stream_channel_size: 100 compression_method: zstd wait_for_result: true diff --git a/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otap-attr-otap.yaml b/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otap-attr-otap.yaml index 677561daae..0dc99bfea5 100644 --- a/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otap-attr-otap.yaml +++ b/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otap-attr-otap.yaml @@ -6,7 +6,7 @@ settings: nodes: receiver: kind: receiver - plugin_urn: "urn:otel:otap:receiver" + plugin_urn: "urn:otel:otel:receiver" out_ports: out_port: destinations: @@ -14,7 +14,6 @@ nodes: dispatch_strategy: round_robin config: listening_addr: "0.0.0.0:4317" - response_stream_channel_size: 100 compression_method: zstd attr: diff --git a/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otap-attr-otlp.yaml b/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otap-attr-otlp.yaml index 23476f56f4..d5734ce43d 100644 --- a/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otap-attr-otlp.yaml +++ b/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otap-attr-otlp.yaml @@ -6,7 +6,7 @@ settings: nodes: receiver: kind: receiver - plugin_urn: "urn:otel:otap:receiver" + plugin_urn: "urn:otel:otel:receiver" out_ports: out_port: destinations: @@ -14,7 +14,6 @@ nodes: dispatch_strategy: round_robin config: listening_addr: "0.0.0.0:4317" - response_stream_channel_size: 100 compression_method: zstd attr: diff --git a/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otlp-attr-otap.yaml b/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otlp-attr-otap.yaml index 76262d6ba4..65ca454a66 100644 --- a/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otlp-attr-otap.yaml +++ b/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otlp-attr-otap.yaml @@ -6,7 +6,7 @@ settings: nodes: receiver: kind: receiver - plugin_urn: "urn:otel:otlp:receiver" + plugin_urn: "urn:otel:otel:receiver" out_ports: out_port: destinations: diff --git a/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otlp-attr-otlp.yaml b/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otlp-attr-otlp.yaml index 8e50dc8055..a7c9dda3b9 100644 --- a/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otlp-attr-otlp.yaml +++ b/tools/pipeline_perf_test/test_suites/integration/templates/configs/engine/continuous/otlp-attr-otlp.yaml @@ -6,7 +6,7 @@ settings: nodes: receiver: kind: receiver - plugin_urn: "urn:otel:otlp:receiver" + plugin_urn: "urn:otel:otel:receiver" out_ports: out_port: destinations: