From f21574bb0997481cf220d795c31ec27be2c6077e Mon Sep 17 00:00:00 2001 From: streamer45 Date: Thu, 22 Jan 2026 09:24:09 +0100 Subject: [PATCH] feat: revamp metrics --- apps/skit/src/server.rs | 14 +- apps/skit/src/session.rs | 3 + crates/core/src/lib.rs | 1 + crates/core/src/metrics.rs | 58 + crates/engine/src/dynamic_actor.rs | 64 +- crates/engine/src/dynamic_pin_distributor.rs | 1 + crates/engine/src/graph_builder.rs | 13 +- crates/engine/src/lib.rs | 17 +- crates/engine/src/oneshot.rs | 88 + crates/engine/src/tests/connection_types.rs | 9 +- crates/engine/src/tests/oneshot_linear.rs | 1 + crates/nodes/src/audio/codecs/flac.rs | 5 +- crates/nodes/src/audio/codecs/mp3.rs | 5 +- crates/nodes/src/audio/codecs/opus.rs | 14 +- crates/nodes/src/containers/wav.rs | 5 +- crates/nodes/src/core/pacer.rs | 3 +- crates/nodes/src/transport/moq/peer.rs | 3 +- crates/nodes/src/transport/moq/push.rs | 3 +- docs/src/content/docs/guides/observability.md | 2 +- samples/grafana-dashboard.json | 2821 ++++++++--------- 20 files changed, 1621 insertions(+), 1509 deletions(-) create mode 100644 crates/core/src/metrics.rs diff --git a/apps/skit/src/server.rs b/apps/skit/src/server.rs index 27d3c6da..d26f0766 100644 --- a/apps/skit/src/server.rs +++ b/apps/skit/src/server.rs @@ -1890,9 +1890,7 @@ impl InstrumentedOneshotStream { impl Drop for InstrumentedOneshotStream { fn drop(&mut self) { if !self.recorded { - // If the client disconnects early, the response body stream is dropped without EOF. - // Record as error so we still get visibility into partial/aborted oneshot executions. - self.record("error"); + self.record("incomplete"); } } } @@ -2060,6 +2058,9 @@ async fn process_oneshot_pipeline_handler( .with_description( "Oneshot pipeline runtime from request start until response stream ends", ) + .with_boundaries( + streamkit_core::metrics::HISTOGRAM_BOUNDARIES_PIPELINE_DURATION.to_vec(), + ) .build() }) .clone(); @@ -2262,7 +2263,12 @@ async fn metrics_middleware(req: axum::http::Request, next: Next) -> Respo let meter = global::meter("skit_server"); ( meter.u64_counter("http.server.requests").build(), - meter.f64_histogram("http.server.duration").build(), + meter + .f64_histogram("http.server.duration") + .with_boundaries( + streamkit_core::metrics::HISTOGRAM_BOUNDARIES_HTTP_DURATION.to_vec(), + ) + .build(), ) }) .clone(); diff --git a/apps/skit/src/session.rs b/apps/skit/src/session.rs index 99eba42d..680d8331 100644 --- a/apps/skit/src/session.rs +++ b/apps/skit/src/session.rs @@ -354,6 +354,9 @@ impl Default for SessionManager { .f64_histogram("session.duration") .with_description("Session lifetime duration in seconds") .with_unit("s") + .with_boundaries( + streamkit_core::metrics::HISTOGRAM_BOUNDARIES_SESSION_DURATION.to_vec(), + ) .build(), } } diff --git a/crates/core/src/lib.rs b/crates/core/src/lib.rs index 5024f941..1bae09fa 100644 --- a/crates/core/src/lib.rs +++ b/crates/core/src/lib.rs @@ -53,6 +53,7 @@ pub mod control; pub mod error; pub mod frame_pool; pub mod helpers; +pub mod metrics; pub mod moq_gateway; pub mod node; pub mod node_config; diff --git a/crates/core/src/metrics.rs b/crates/core/src/metrics.rs new file mode 100644 index 00000000..e8624856 --- /dev/null +++ b/crates/core/src/metrics.rs @@ -0,0 +1,58 @@ +// SPDX-FileCopyrightText: © 2025 StreamKit Contributors +// +// SPDX-License-Identifier: MPL-2.0 + +//! Shared metrics configuration and histogram boundaries. +//! +//! Defines standard histogram bucket boundaries for OpenTelemetry metrics +//! to ensure accurate percentile calculations across the codebase. + +/// Sub-millisecond boundaries for per-packet codec operations (10μs to 1s) +/// Used by: opus encode/decode +pub const HISTOGRAM_BOUNDARIES_CODEC_PACKET: &[f64] = + &[0.00001, 0.00005, 0.0001, 0.0005, 0.001, 0.005, 0.01, 0.05, 0.1, 0.5, 1.0]; + +/// Millisecond-to-minute boundaries for per-file operations (1ms to 60s) +/// Used by: mp3/flac/wav decode/demux +pub const HISTOGRAM_BOUNDARIES_FILE_OPERATION: &[f64] = + &[0.001, 0.005, 0.01, 0.05, 0.1, 0.5, 1.0, 5.0, 10.0, 30.0, 60.0]; + +/// Node execution boundaries (10μs to 60s) +/// Used by: node.execution.duration +pub const HISTOGRAM_BOUNDARIES_NODE_EXECUTION: &[f64] = + &[0.00001, 0.0001, 0.001, 0.01, 0.1, 1.0, 10.0, 60.0]; + +/// Backpressure wait time boundaries (1μs to 10s) +/// Used by: pin_distributor.send_wait_seconds +pub const HISTOGRAM_BOUNDARIES_BACKPRESSURE: &[f64] = + &[0.000001, 0.00001, 0.0001, 0.001, 0.01, 0.1, 1.0, 10.0]; + +/// Pacer lateness boundaries (1ms to 10s) +/// Used by: pacer.lateness_seconds +pub const HISTOGRAM_BOUNDARIES_PACER_LATENESS: &[f64] = + &[0.001, 0.01, 0.05, 0.1, 0.25, 0.5, 1.0, 2.0, 5.0, 10.0]; + +/// Clock offset boundaries in milliseconds (0.1ms to 10s) +/// Used by: moq.push.clock_offset_ms +pub const HISTOGRAM_BOUNDARIES_CLOCK_OFFSET_MS: &[f64] = + &[0.1, 1.0, 5.0, 10.0, 20.0, 50.0, 100.0, 200.0, 500.0, 1000.0, 5000.0, 10000.0]; + +/// Frame gap boundaries in milliseconds (1ms to 1s, with common frame rates) +/// Used by: moq.peer.inter_frame_ms +pub const HISTOGRAM_BOUNDARIES_FRAME_GAP_MS: &[f64] = + &[1.0, 5.0, 10.0, 16.0, 20.0, 33.0, 50.0, 100.0, 200.0, 500.0, 1000.0]; + +/// Pipeline duration boundaries (10ms to 5 minutes) +/// Used by: oneshot_pipeline.duration +pub const HISTOGRAM_BOUNDARIES_PIPELINE_DURATION: &[f64] = + &[0.01, 0.05, 0.1, 0.5, 1.0, 5.0, 10.0, 30.0, 60.0, 120.0, 300.0]; + +/// HTTP request duration boundaries (1ms to 60s) +/// Used by: http.server.duration +pub const HISTOGRAM_BOUNDARIES_HTTP_DURATION: &[f64] = + &[0.001, 0.005, 0.01, 0.05, 0.1, 0.5, 1.0, 5.0, 10.0, 30.0, 60.0]; + +/// Session lifetime boundaries (1s to 24 hours) +/// Used by: session.duration +pub const HISTOGRAM_BOUNDARIES_SESSION_DURATION: &[f64] = + &[1.0, 10.0, 60.0, 300.0, 600.0, 1800.0, 3600.0, 7200.0, 21600.0, 86400.0]; diff --git a/crates/engine/src/dynamic_actor.rs b/crates/engine/src/dynamic_actor.rs index 767b19d1..ebd88818 100644 --- a/crates/engine/src/dynamic_actor.rs +++ b/crates/engine/src/dynamic_actor.rs @@ -52,6 +52,8 @@ pub struct DynamicEngine { HashMap>, /// Map of node pin metadata: NodeId -> Pin Metadata (for runtime type validation) pub(super) node_pin_metadata: HashMap, + /// Map of node_id -> node_kind for labeling metrics + pub(super) node_kinds: HashMap, pub(super) batch_size: usize, /// Session ID for gateway registration (if applicable) pub(super) session_id: Option, @@ -75,11 +77,11 @@ pub struct DynamicEngine { pub(super) nodes_active_gauge: opentelemetry::metrics::Gauge, pub(super) node_state_transitions_counter: opentelemetry::metrics::Counter, pub(super) engine_operations_counter: opentelemetry::metrics::Counter, - // Node-level packet metrics - pub(super) node_packets_received_gauge: opentelemetry::metrics::Gauge, - pub(super) node_packets_sent_gauge: opentelemetry::metrics::Gauge, - pub(super) node_packets_discarded_gauge: opentelemetry::metrics::Gauge, - pub(super) node_packets_errored_gauge: opentelemetry::metrics::Gauge, + // Node-level packet metrics (counters, not gauges - for proper rate() calculation) + pub(super) node_packets_received_counter: opentelemetry::metrics::Counter, + pub(super) node_packets_sent_counter: opentelemetry::metrics::Counter, + pub(super) node_packets_discarded_counter: opentelemetry::metrics::Counter, + pub(super) node_packets_errored_counter: opentelemetry::metrics::Counter, // Node state metric (1=running, 0=not running) pub(super) node_state_gauge: opentelemetry::metrics::Gauge, } @@ -363,16 +365,50 @@ impl DynamicEngine { "Node stats updated" ); - // Store the current stats - self.node_stats.insert(update.node_id.clone(), update.stats.clone()); + let node_kind = + self.node_kinds.get(&update.node_id).map_or("unknown", std::string::String::as_str); + let labels = &[ + KeyValue::new("node_id", update.node_id.clone()), + KeyValue::new("node_kind", node_kind.to_string()), + ]; + + let prev_stats = self.node_stats.get(&update.node_id); + + let delta_received = prev_stats.map_or(update.stats.received, |prev| { + if update.stats.received < prev.received { + update.stats.received + } else { + update.stats.received - prev.received + } + }); + let delta_sent = prev_stats.map_or(update.stats.sent, |prev| { + if update.stats.sent < prev.sent { + update.stats.sent + } else { + update.stats.sent - prev.sent + } + }); + let delta_discarded = prev_stats.map_or(update.stats.discarded, |prev| { + if update.stats.discarded < prev.discarded { + update.stats.discarded + } else { + update.stats.discarded - prev.discarded + } + }); + let delta_errored = prev_stats.map_or(update.stats.errored, |prev| { + if update.stats.errored < prev.errored { + update.stats.errored + } else { + update.stats.errored - prev.errored + } + }); - // Record metrics with node_id label - let labels = &[KeyValue::new("node_id", update.node_id.clone())]; + self.node_packets_received_counter.add(delta_received, labels); + self.node_packets_sent_counter.add(delta_sent, labels); + self.node_packets_discarded_counter.add(delta_discarded, labels); + self.node_packets_errored_counter.add(delta_errored, labels); - self.node_packets_received_gauge.record(update.stats.received, labels); - self.node_packets_sent_gauge.record(update.stats.sent, labels); - self.node_packets_discarded_gauge.record(update.stats.discarded, labels); - self.node_packets_errored_gauge.record(update.stats.errored, labels); + self.node_stats.insert(update.node_id.clone(), update.stats.clone()); // Broadcast to all subscribers self.stats_subscribers.retain(|subscriber| { @@ -862,6 +898,7 @@ impl DynamicEngine { self.node_stats.remove(node_id); self.node_pin_metadata.remove(node_id); self.pin_management_txs.remove(node_id); + self.node_kinds.remove(node_id); self.nodes_active_gauge.record(self.live_nodes.len() as u64, &[]); } @@ -881,6 +918,7 @@ impl DynamicEngine { tracing::info!(name = %node_id, kind = %kind, "Adding node to graph"); match self.registry.create_node(&kind, params.as_ref()) { Ok(node) => { + self.node_kinds.insert(node_id.clone(), kind.clone()); // Delegate initialization to helper function // Pass by reference to avoid unnecessary clones if let Err(e) = self diff --git a/crates/engine/src/dynamic_pin_distributor.rs b/crates/engine/src/dynamic_pin_distributor.rs index 333ae716..1dd777f6 100644 --- a/crates/engine/src/dynamic_pin_distributor.rs +++ b/crates/engine/src/dynamic_pin_distributor.rs @@ -92,6 +92,7 @@ impl PinDistributorActor { let send_wait_histogram = meter .f64_histogram("pin_distributor.send_wait_seconds") .with_description("Time spent waiting for downstream capacity (backpressure)") + .with_boundaries(streamkit_core::metrics::HISTOGRAM_BOUNDARIES_BACKPRESSURE.to_vec()) .build(); let queue_depth_gauge = meter .u64_gauge("pin_distributor.queue_depth") diff --git a/crates/engine/src/graph_builder.rs b/crates/engine/src/graph_builder.rs index 1df2b348..75c4cf0a 100644 --- a/crates/engine/src/graph_builder.rs +++ b/crates/engine/src/graph_builder.rs @@ -13,6 +13,7 @@ use streamkit_core::node::{InitContext, NodeContext, OutputRouting, OutputSender use streamkit_core::packet_meta::{can_connect, packet_type_registry}; use streamkit_core::pins::PinUpdate; use streamkit_core::state::{NodeState, NodeStateUpdate, StopReason}; +use streamkit_core::stats::NodeStatsUpdate; use streamkit_core::types::{Packet, PacketType}; use streamkit_core::PinCardinality; use tokio::sync::mpsc; @@ -62,6 +63,7 @@ pub async fn wire_and_spawn_graph( batch_size: usize, media_channel_capacity: usize, state_tx: Option>, + stats_tx: Option>, cancellation_token: Option, audio_pool: Option>, ) -> Result, StreamKitError> { @@ -356,9 +358,9 @@ pub async fn wire_and_spawn_graph( output_sender: OutputSender::new(name.clone(), OutputRouting::Direct(direct_outputs)), batch_size, state_tx: node_state_tx.clone(), - stats_tx: None, // Stateless pipelines don't track stats - telemetry_tx: None, // Stateless pipelines don't emit telemetry - session_id: None, // Stateless pipelines don't have sessions + stats_tx: stats_tx.clone(), // Used by oneshot metrics recording + telemetry_tx: None, // Stateless pipelines don't emit telemetry + session_id: None, // Stateless pipelines don't have sessions cancellation_token: cancellation_token.clone(), pin_management_rx: None, // Stateless pipelines don't support dynamic pins audio_pool: audio_pool.clone(), @@ -384,12 +386,13 @@ pub async fn wire_and_spawn_graph( let meter = global::meter("skit_engine"); let histogram = meter .f64_histogram("node.execution.duration") + .with_boundaries(streamkit_core::metrics::HISTOGRAM_BOUNDARIES_NODE_EXECUTION.to_vec()) .build(); let status = if result.is_ok() { "ok" } else { "error" }; let labels = [ - KeyValue::new("node.name", name.clone()), - KeyValue::new("node.kind", kind.clone()), + KeyValue::new("node_id", name.clone()), + KeyValue::new("node_kind", kind.clone()), KeyValue::new("status", status), ]; histogram.record(duration.as_secs_f64(), &labels); diff --git a/crates/engine/src/lib.rs b/crates/engine/src/lib.rs index 1a96d045..bff31e87 100644 --- a/crates/engine/src/lib.rs +++ b/crates/engine/src/lib.rs @@ -235,6 +235,7 @@ impl Engine { pin_distributors: HashMap::new(), pin_management_txs: HashMap::new(), node_pin_metadata: HashMap::new(), + node_kinds: HashMap::new(), batch_size: config.packet_batch_size, session_id: config.session_id, audio_pool: self.audio_pool.clone(), @@ -257,20 +258,20 @@ impl Engine { .u64_counter("engine.operations") .with_description("Engine control operations") .build(), - node_packets_received_gauge: meter - .u64_gauge("node.packets.received") + node_packets_received_counter: meter + .u64_counter("node.packets.received") .with_description("Total packets received by node") .build(), - node_packets_sent_gauge: meter - .u64_gauge("node.packets.sent") + node_packets_sent_counter: meter + .u64_counter("node.packets.sent") .with_description("Total packets sent by node") .build(), - node_packets_discarded_gauge: meter - .u64_gauge("node.packets.discarded") + node_packets_discarded_counter: meter + .u64_counter("node.packets.discarded") .with_description("Total packets discarded by node") .build(), - node_packets_errored_gauge: meter - .u64_gauge("node.packets.errored") + node_packets_errored_counter: meter + .u64_counter("node.packets.errored") .with_description("Total packet processing errors by node") .build(), node_state_gauge: meter diff --git a/crates/engine/src/oneshot.rs b/crates/engine/src/oneshot.rs index 6e612807..bc7f077d 100644 --- a/crates/engine/src/oneshot.rs +++ b/crates/engine/src/oneshot.rs @@ -25,16 +25,19 @@ use crate::constants::{ DEFAULT_BATCH_SIZE, DEFAULT_ONESHOT_IO_CAPACITY, DEFAULT_ONESHOT_MEDIA_CAPACITY, + DEFAULT_STATE_CHANNEL_CAPACITY, }; // Note: The constants are used in OneshotEngineConfig::default() use crate::{graph_builder, Engine}; use bytes::Bytes; use futures::Stream; +use opentelemetry::{global, KeyValue}; use std::collections::HashMap; use streamkit_api::Pipeline; use streamkit_core::control::NodeControlMessage; use streamkit_core::error::StreamKitError; use streamkit_core::node::ProcessorNode; +use streamkit_core::stats::{NodeStats, NodeStatsUpdate}; use tokio::sync::mpsc; use tokio_util::sync::CancellationToken; @@ -372,9 +375,12 @@ impl Engine { let node_kinds: HashMap = definition.nodes.iter().map(|(name, def)| (name.clone(), def.kind.clone())).collect(); + let node_kinds_for_metrics = node_kinds.clone(); let audio_pool = self.audio_pool.clone(); + let (stats_tx, stats_rx) = mpsc::channel(DEFAULT_STATE_CHANNEL_CAPACITY); + let live_nodes = graph_builder::wire_and_spawn_graph( nodes, &definition.connections, @@ -382,12 +388,15 @@ impl Engine { config.packet_batch_size, config.media_channel_capacity, None, // No state tracking for oneshot pipelines + Some(stats_tx), Some(cancellation_token.clone()), Some(audio_pool), ) .await?; tracing::info!("Pipeline graph successfully spawned"); + spawn_oneshot_metrics_recorder(stats_rx, node_kinds_for_metrics); + // --- 5.5. Start file readers (if any) --- if !source_node_ids.is_empty() { tracing::info!( @@ -428,3 +437,82 @@ impl Engine { Ok(OneshotPipelineResult { data_stream: output_stream_rx, content_type }) } } + +fn spawn_oneshot_metrics_recorder( + mut stats_rx: mpsc::Receiver, + node_kinds: HashMap, +) { + let meter = global::meter("skit_engine"); + let node_packets_received_counter = meter + .u64_counter("node.packets.received") + .with_description("Total packets received by node") + .build(); + let node_packets_sent_counter = meter + .u64_counter("node.packets.sent") + .with_description("Total packets sent by node") + .build(); + let node_packets_discarded_counter = meter + .u64_counter("node.packets.discarded") + .with_description("Total packets discarded by node") + .build(); + let node_packets_errored_counter = meter + .u64_counter("node.packets.errored") + .with_description("Total packet processing errors by node") + .build(); + + let node_kinds = std::sync::Arc::new(node_kinds); + + tokio::spawn(async move { + // Track previous stats per node to compute deltas + let mut prev_stats: HashMap = HashMap::new(); + + while let Some(update) = stats_rx.recv().await { + let node_kind = + node_kinds.get(&update.node_id).map_or("unknown", std::string::String::as_str); + + let labels = &[ + KeyValue::new("node_id", update.node_id.clone()), + KeyValue::new("node_kind", node_kind.to_string()), + ]; + + let prev = prev_stats.get(&update.node_id); + let delta_received = prev.map_or(update.stats.received, |p| { + if update.stats.received < p.received { + update.stats.received + } else { + update.stats.received - p.received + } + }); + let delta_sent = prev.map_or(update.stats.sent, |p| { + if update.stats.sent < p.sent { + update.stats.sent + } else { + update.stats.sent - p.sent + } + }); + let delta_discarded = prev.map_or(update.stats.discarded, |p| { + if update.stats.discarded < p.discarded { + update.stats.discarded + } else { + update.stats.discarded - p.discarded + } + }); + let delta_errored = prev.map_or(update.stats.errored, |p| { + if update.stats.errored < p.errored { + update.stats.errored + } else { + update.stats.errored - p.errored + } + }); + + // Add deltas to counters (not absolute values) + node_packets_received_counter.add(delta_received, labels); + node_packets_sent_counter.add(delta_sent, labels); + node_packets_discarded_counter.add(delta_discarded, labels); + node_packets_errored_counter.add(delta_errored, labels); + + // Update previous stats for this node + prev_stats.insert(update.node_id.clone(), update.stats); + } + }); +} diff --git a/crates/engine/src/tests/connection_types.rs b/crates/engine/src/tests/connection_types.rs index 938ce4c4..3996f5b9 100644 --- a/crates/engine/src/tests/connection_types.rs +++ b/crates/engine/src/tests/connection_types.rs @@ -28,6 +28,7 @@ fn create_test_engine() -> DynamicEngine { pin_distributors: HashMap::new(), pin_management_txs: HashMap::new(), node_pin_metadata: HashMap::new(), + node_kinds: HashMap::new(), batch_size: 32, session_id: None, audio_pool: std::sync::Arc::new(streamkit_core::FramePool::::audio_default()), @@ -41,10 +42,10 @@ fn create_test_engine() -> DynamicEngine { nodes_active_gauge: meter.u64_gauge("test.nodes").build(), node_state_transitions_counter: meter.u64_counter("test.transitions").build(), engine_operations_counter: meter.u64_counter("test.operations").build(), - node_packets_received_gauge: meter.u64_gauge("test.received").build(), - node_packets_sent_gauge: meter.u64_gauge("test.sent").build(), - node_packets_discarded_gauge: meter.u64_gauge("test.discarded").build(), - node_packets_errored_gauge: meter.u64_gauge("test.errored").build(), + node_packets_received_counter: meter.u64_counter("test.received").build(), + node_packets_sent_counter: meter.u64_counter("test.sent").build(), + node_packets_discarded_counter: meter.u64_counter("test.discarded").build(), + node_packets_errored_counter: meter.u64_counter("test.errored").build(), node_state_gauge: meter.u64_gauge("test.state").build(), } } diff --git a/crates/engine/src/tests/oneshot_linear.rs b/crates/engine/src/tests/oneshot_linear.rs index 6538cf58..cf2028f3 100644 --- a/crates/engine/src/tests/oneshot_linear.rs +++ b/crates/engine/src/tests/oneshot_linear.rs @@ -72,6 +72,7 @@ async fn test_oneshot_rejects_fanout() { None, None, None, + None, ) .await else { diff --git a/crates/nodes/src/audio/codecs/flac.rs b/crates/nodes/src/audio/codecs/flac.rs index 4b211aa6..114fe868 100644 --- a/crates/nodes/src/audio/codecs/flac.rs +++ b/crates/nodes/src/audio/codecs/flac.rs @@ -91,7 +91,10 @@ impl ProcessorNode for FlacDecoderNode { let meter = global::meter("skit_nodes"); let packets_processed_counter = meter.u64_counter("flac_packets_processed").build(); - let decode_duration_histogram = meter.f64_histogram("flac_decode_duration").build(); + let decode_duration_histogram = meter + .f64_histogram("flac_decode_duration") + .with_boundaries(streamkit_core::metrics::HISTOGRAM_BOUNDARIES_FILE_OPERATION.to_vec()) + .build(); // Create channels for communication with the blocking task. // This must be bounded to provide backpressure and prevent unbounded buffering. diff --git a/crates/nodes/src/audio/codecs/mp3.rs b/crates/nodes/src/audio/codecs/mp3.rs index d7da1b86..346fe4ea 100644 --- a/crates/nodes/src/audio/codecs/mp3.rs +++ b/crates/nodes/src/audio/codecs/mp3.rs @@ -93,7 +93,10 @@ impl ProcessorNode for Mp3DecoderNode { let meter = global::meter("skit_nodes"); let packets_processed_counter = meter.u64_counter("mp3_packets_processed").build(); - let decode_duration_histogram = meter.f64_histogram("mp3_decode_duration").build(); + let decode_duration_histogram = meter + .f64_histogram("mp3_decode_duration") + .with_boundaries(streamkit_core::metrics::HISTOGRAM_BOUNDARIES_FILE_OPERATION.to_vec()) + .build(); // Create channels for communication with the blocking task. // This must be bounded to provide backpressure and prevent unbounded buffering. diff --git a/crates/nodes/src/audio/codecs/opus.rs b/crates/nodes/src/audio/codecs/opus.rs index 3482d37f..fed47acd 100644 --- a/crates/nodes/src/audio/codecs/opus.rs +++ b/crates/nodes/src/audio/codecs/opus.rs @@ -84,7 +84,10 @@ impl ProcessorNode for OpusDecoderNode { let meter = global::meter("skit_nodes"); let packets_processed_counter = meter.u64_counter("opus_packets_processed").build(); - let decode_duration_histogram = meter.f64_histogram("opus_decode_duration").build(); + let decode_duration_histogram = meter + .f64_histogram("opus_decode_duration") + .with_boundaries(streamkit_core::metrics::HISTOGRAM_BOUNDARIES_CODEC_PACKET.to_vec()) + .build(); // Create channels for communication with the blocking task // Now includes metadata with each packet @@ -436,7 +439,10 @@ impl ProcessorNode for OpusEncoderNode { let meter = global::meter("skit_nodes"); let packets_processed_counter = meter.u64_counter("opus_packets_processed").build(); - let encode_duration_histogram = meter.f64_histogram("opus_encode_duration").build(); + let encode_duration_histogram = meter + .f64_histogram("opus_encode_duration") + .with_boundaries(streamkit_core::metrics::HISTOGRAM_BOUNDARIES_CODEC_PACKET.to_vec()) + .build(); // Create channels for communication with the blocking task // Now includes channel count with each frame @@ -461,8 +467,6 @@ impl ProcessorNode for OpusEncoderNode { // Use blocking_recv - efficient for spawn_blocking context while let Some((samples, channels)) = encode_rx.blocking_recv() { - let encode_start_time = Instant::now(); - // Initialize or recreate encoder if channel count changed if current_channels != Some(channels) { let opus_channels = @@ -496,6 +500,8 @@ impl ProcessorNode for OpusEncoderNode { }; } + let encode_start_time = Instant::now(); + let result = { // Encoder must exist at this point because we just initialized it above // based on the channel count. If it doesn't exist, something went wrong. diff --git a/crates/nodes/src/containers/wav.rs b/crates/nodes/src/containers/wav.rs index 6d04586b..014ec70f 100644 --- a/crates/nodes/src/containers/wav.rs +++ b/crates/nodes/src/containers/wav.rs @@ -93,7 +93,10 @@ impl ProcessorNode for WavDemuxerNode { let meter = global::meter("skit_nodes"); let packets_processed_counter = meter.u64_counter("wav_packets_processed").build(); - let demux_duration_histogram = meter.f64_histogram("wav_demux_duration").build(); + let demux_duration_histogram = meter + .f64_histogram("wav_demux_duration") + .with_boundaries(streamkit_core::metrics::HISTOGRAM_BOUNDARIES_FILE_OPERATION.to_vec()) + .build(); // Create channels for communication with the blocking task. // This must be bounded to provide backpressure and prevent unbounded buffering. diff --git a/crates/nodes/src/core/pacer.rs b/crates/nodes/src/core/pacer.rs index f8ede39a..db6bcbe0 100644 --- a/crates/nodes/src/core/pacer.rs +++ b/crates/nodes/src/core/pacer.rs @@ -187,12 +187,13 @@ impl ProcessorNode for PacerNode { let lateness_histogram = meter .f64_histogram("pacer.lateness_seconds") .with_description("Pacer observed send lateness vs. packet timestamp") + .with_boundaries(streamkit_core::metrics::HISTOGRAM_BOUNDARIES_PACER_LATENESS.to_vec()) .build(); let queue_gauge = meter .u64_gauge("pacer.queue_depth") .with_description("Pacer buffered packet count") .build(); - let metric_labels = [KeyValue::new("node", node_name.clone())]; + let metric_labels = [KeyValue::new("node_id", node_name.clone())]; // Internal bounded queue for backpressure control let mut packet_queue: VecDeque = diff --git a/crates/nodes/src/transport/moq/peer.rs b/crates/nodes/src/transport/moq/peer.rs index 49e344e9..abbdd59e 100644 --- a/crates/nodes/src/transport/moq/peer.rs +++ b/crates/nodes/src/transport/moq/peer.rs @@ -1150,9 +1150,10 @@ impl MoqPeerNode { let gap_histogram = meter .f64_histogram("moq.peer.inter_frame_ms") .with_description("Gap between consecutive frames sent to subscribers") + .with_boundaries(streamkit_core::metrics::HISTOGRAM_BOUNDARIES_FRAME_GAP_MS.to_vec()) .build(); let metric_labels = [ - opentelemetry::KeyValue::new("node", node_id), + opentelemetry::KeyValue::new("node_id", node_id), opentelemetry::KeyValue::new("broadcast", broadcast_name), ]; let mut last_ts_ms: Option = None; diff --git a/crates/nodes/src/transport/moq/push.rs b/crates/nodes/src/transport/moq/push.rs index bbaa0441..13fa7628 100644 --- a/crates/nodes/src/transport/moq/push.rs +++ b/crates/nodes/src/transport/moq/push.rs @@ -205,9 +205,10 @@ impl ProcessorNode for MoqPushNode { let clock_offset_histogram = meter .f64_histogram("moq.push.clock_offset_ms") .with_description("Offset between outgoing MoQ timestamp and upstream packet timestamp") + .with_boundaries(streamkit_core::metrics::HISTOGRAM_BOUNDARIES_CLOCK_OFFSET_MS.to_vec()) .build(); let metric_labels = [ - KeyValue::new("node", node_name.clone()), + KeyValue::new("node_id", node_name.clone()), KeyValue::new("broadcast", self.config.broadcast.clone()), ]; diff --git a/docs/src/content/docs/guides/observability.md b/docs/src/content/docs/guides/observability.md index cd7a258e..8b607d77 100644 --- a/docs/src/content/docs/guides/observability.md +++ b/docs/src/content/docs/guides/observability.md @@ -43,7 +43,7 @@ Point `telemetry.otlp_endpoint` at your Prometheus OTLP endpoint (see the Promet ### Grafana dashboard -Import [`samples/grafana-dashboard.json`](https://github.com/streamer45/streamkit/blob/main/samples/grafana-dashboard.json) into Grafana and select the same Prometheus (or other OTLP-backed) datasource you're sending metrics to. +Import [`samples/grafana-dashboard.json`](https://github.com/streamer45/streamkit/blob/main/samples/grafana-dashboard.json) into Grafana and select the same Prometheus (or other OTLP-backed) datasource you're sending metrics to. This streamlined dashboard focuses on high-signal health indicators with collapsed advanced sections for debugging. ![Grafana Dashboard](/screenshots/grafana_dashboard.png) diff --git a/samples/grafana-dashboard.json b/samples/grafana-dashboard.json index cc34b54c..3be07001 100644 --- a/samples/grafana-dashboard.json +++ b/samples/grafana-dashboard.json @@ -31,9 +31,9 @@ }, "id": 1, "panels": [], - "title": "System Overview", + "title": "Health Overview", "type": "row", - "description": "Resource usage and key runtime counts." + "description": "Core health indicators - if everything is green here, your system is healthy" }, { "datasource": { @@ -49,7 +49,7 @@ "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", - "axisLabel": "CPU %", + "axisLabel": "Error %", "axisPlacement": "auto", "barAlignment": 0, "drawStyle": "line", @@ -74,10 +74,11 @@ "mode": "none" }, "thresholdsStyle": { - "mode": "off" + "mode": "line" } }, "mappings": [], + "max": 100, "min": 0, "thresholds": { "mode": "absolute", @@ -86,19 +87,43 @@ "color": "green", "value": null }, + { + "color": "yellow", + "value": 1 + }, + { + "color": "orange", + "value": 5 + }, { "color": "red", - "value": 80 + "value": 10 } ] }, "unit": "percent" }, - "overrides": [] + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "HTTP 5xx" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "red", + "mode": "fixed" + } + } + ] + } + ] }, "gridPos": { "h": 8, - "w": 8, + "w": 6, "x": 0, "y": 1 }, @@ -110,12 +135,12 @@ "max" ], "displayMode": "table", - "placement": "right", + "placement": "bottom", "showLegend": true }, "tooltip": { "mode": "multi", - "sort": "none" + "sort": "desc" } }, "targets": [ @@ -125,8 +150,8 @@ "uid": "${DS_PROMETHEUS}" }, "editorMode": "code", - "expr": "system_cpu_utilization_percent", - "legendFormat": "System CPU", + "expr": "100 * sum(rate(http_server_requests_total{http_route=~\"/api/.*\", http_status_code=~\"5..\"}[5m])) / clamp_min(sum(rate(http_server_requests_total{http_route=~\"/api/.*\"}[5m])), 1e-9)", + "legendFormat": "HTTP 5xx", "range": true, "refId": "A" }, @@ -136,15 +161,130 @@ "uid": "${DS_PROMETHEUS}" }, "editorMode": "code", - "expr": "process_cpu_utilization_percent", - "legendFormat": "Process CPU", + "expr": "100 * sum(rate(oneshot_pipeline_duration_count{status=\"error\"}[5m])) / clamp_min(sum(rate(oneshot_pipeline_duration_count{status=~\"ok|error\"}[5m])), 1e-9)", + "legendFormat": "Oneshot Error", + "range": true, + "refId": "B" + } + ], + "title": "Error Rates (SLO)", + "type": "timeseries", + "description": "Error rates for API endpoints and oneshot pipelines. Should be <1% for healthy operation." + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "Seconds", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "tooltip": false, + "viz": false, + "legend": false + }, + "insertNulls": false, + "lineInterpolation": "smooth", + "lineWidth": 2, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "yellow", + "value": 1 + }, + { + "color": "red", + "value": 5 + } + ] + }, + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 6, + "x": 6, + "y": 1 + }, + "id": 3, + "options": { + "legend": { + "calcs": [ + "lastNotNull", + "max" + ], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "histogram_quantile(0.95, sum(rate(http_server_duration_bucket{http_route=~\"/api/.*\", http_status_code=~\"2..\"}[5m])) by (le))", + "legendFormat": "HTTP p95", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "histogram_quantile(0.95, sum(rate(oneshot_pipeline_duration_bucket{status=\"ok\"}[5m])) by (le))", + "legendFormat": "Oneshot p95", "range": true, "refId": "B" } ], - "title": "CPU Usage", + "title": "Latency p95 (SLO)", "type": "timeseries", - "description": "Process and host CPU utilization percentage." + "description": "95th percentile latency for API requests and oneshot pipelines. Lower is better." }, { "datasource": { @@ -160,7 +300,7 @@ "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", - "axisLabel": "GiB", + "axisLabel": "", "axisPlacement": "auto", "barAlignment": 0, "drawStyle": "line", @@ -189,6 +329,7 @@ } }, "mappings": [], + "min": 0, "thresholds": { "mode": "absolute", "steps": [ @@ -198,28 +339,22 @@ } ] }, - "unit": "gbytes" + "unit": "percent" }, "overrides": [ { "matcher": { - "id": "byName", - "options": "System Memory Total" + "id": "byFrameRefID", + "options": "B" }, "properties": [ { - "id": "custom.lineStyle", - "value": { - "dash": [ - 6, - 4 - ], - "fill": "dash" - } + "id": "unit", + "value": "gbytes" }, { - "id": "custom.lineWidth", - "value": 1 + "id": "custom.axisPlacement", + "value": "right" } ] } @@ -227,11 +362,11 @@ }, "gridPos": { "h": 8, - "w": 8, - "x": 8, + "w": 6, + "x": 12, "y": 1 }, - "id": 3, + "id": 4, "options": { "legend": { "calcs": [ @@ -239,7 +374,7 @@ "max" ], "displayMode": "table", - "placement": "right", + "placement": "bottom", "showLegend": true }, "tooltip": { @@ -254,8 +389,8 @@ "uid": "${DS_PROMETHEUS}" }, "editorMode": "code", - "expr": "process_memory_usage_bytes / 1024 / 1024 / 1024", - "legendFormat": "Process Memory", + "expr": "process_cpu_utilization_percent", + "legendFormat": "Process CPU", "range": true, "refId": "A" }, @@ -265,26 +400,15 @@ "uid": "${DS_PROMETHEUS}" }, "editorMode": "code", - "expr": "system_memory_usage_bytes / 1024 / 1024 / 1024", - "legendFormat": "System Memory Used", + "expr": "process_memory_usage_bytes / 1024 / 1024 / 1024", + "legendFormat": "Process Memory", "range": true, "refId": "B" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "editorMode": "code", - "expr": "system_memory_total_bytes / 1024 / 1024 / 1024", - "legendFormat": "System Memory Total", - "range": true, - "refId": "C" } ], - "title": "Memory Usage", + "title": "Process Health", "type": "timeseries", - "description": "Working set usage compared to host capacity (GiB)." + "description": "Skit process CPU and memory usage. High CPU or memory growth indicates resource constraints." }, { "datasource": { @@ -306,7 +430,7 @@ }, { "color": "green", - "value": 10 + "value": 1 }, { "color": "yellow", @@ -323,11 +447,11 @@ }, "gridPos": { "h": 8, - "w": 8, - "x": 16, + "w": 6, + "x": 18, "y": 1 }, - "id": 4, + "id": 5, "options": { "colorMode": "value", "graphMode": "area", @@ -358,17 +482,6 @@ "range": true, "refId": "A" }, - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "editorMode": "code", - "expr": "websocket_connections_active", - "legendFormat": "WebSocket Connections", - "range": true, - "refId": "B" - }, { "datasource": { "type": "prometheus", @@ -378,7 +491,7 @@ "expr": "engine_nodes_active", "legendFormat": "Active Nodes", "range": true, - "refId": "C" + "refId": "B" }, { "datasource": { @@ -386,15 +499,15 @@ "uid": "${DS_PROMETHEUS}" }, "editorMode": "code", - "expr": "sum(plugins_loaded)", - "legendFormat": "Loaded Plugins", + "expr": "websocket_connections_active", + "legendFormat": "WebSockets", "range": true, - "refId": "D" + "refId": "C" } ], "title": "Active Components", "type": "stat", - "description": "Active sessions, WebSocket consumers, engine nodes, and loaded plugins." + "description": "Current active sessions, engine nodes, and WebSocket connections." }, { "collapsed": false, @@ -404,11 +517,11 @@ "x": 0, "y": 9 }, - "id": 5, + "id": 6, "panels": [], - "title": "HTTP API Performance", + "title": "HTTP API", "type": "row", - "description": "Request activity and responsiveness for the HTTP surface." + "description": "HTTP server performance and request handling" }, { "datasource": { @@ -446,7 +559,7 @@ "spanNulls": false, "stacking": { "group": "A", - "mode": "normal" + "mode": "none" }, "thresholdsStyle": { "mode": "off" @@ -468,11 +581,11 @@ }, "gridPos": { "h": 8, - "w": 12, + "w": 8, "x": 0, "y": 10 }, - "id": 6, + "id": 7, "options": { "legend": { "calcs": [ @@ -495,15 +608,15 @@ "uid": "${DS_PROMETHEUS}" }, "editorMode": "code", - "expr": "sum(rate(http_server_requests_total[1m])) by (http_method)", - "legendFormat": "{{http_method}}", + "expr": "sum(rate(http_server_requests_total[1m]))", + "legendFormat": "Total RPS", "range": true, "refId": "A" } ], "title": "HTTP Request Rate", "type": "timeseries", - "description": "HTTP requests per second grouped by method." + "description": "Total HTTP requests per second across all endpoints." }, { "datasource": { @@ -536,7 +649,7 @@ "spanNulls": false, "stacking": { "group": "A", - "mode": "percent" + "mode": "normal" }, "thresholdsStyle": { "mode": "off" @@ -560,11 +673,11 @@ }, "gridPos": { "h": 8, - "w": 6, - "x": 12, + "w": 8, + "x": 8, "y": 10 }, - "id": 7, + "id": 8, "options": { "legend": { "calcs": [ @@ -586,7 +699,7 @@ "uid": "${DS_PROMETHEUS}" }, "editorMode": "code", - "expr": "sum(rate(http_server_requests_total[1m])) by (http_status_code)", + "expr": "sum(rate(http_server_requests_total{http_route=~\"/api/.*\"}[1m])) by (http_status_code) / ignoring(http_status_code) group_left sum(rate(http_server_requests_total{http_route=~\"/api/.*\"}[1m]))", "legendFormat": "{{http_status_code}}", "range": true, "refId": "A" @@ -594,7 +707,7 @@ ], "title": "HTTP Status Mix", "type": "timeseries", - "description": "Share of HTTP response codes over time." + "description": "Distribution of HTTP status codes for API routes (/api/*). Watch for increases in 4xx or 5xx." }, { "datasource": { @@ -654,11 +767,11 @@ }, "gridPos": { "h": 8, - "w": 6, - "x": 18, + "w": 8, + "x": 16, "y": 10 }, - "id": 8, + "id": 9, "options": { "legend": { "calcs": [ @@ -711,7 +824,7 @@ ], "title": "HTTP Latency Percentiles", "type": "timeseries", - "description": "Rolling p50/p95/p99 latency derived from request duration histogram." + "description": "HTTP request duration percentiles. p50 = median, p95/p99 = tail latency." }, { "collapsed": false, @@ -721,11 +834,11 @@ "x": 0, "y": 18 }, - "id": 9, + "id": 10, "panels": [], - "title": "Node Performance", + "title": "Engine & Data Plane", "type": "row", - "description": "Execution performance across StreamKit engine nodes." + "description": "Pipeline execution performance and backpressure" }, { "datasource": { @@ -789,7 +902,7 @@ "x": 0, "y": 19 }, - "id": 10, + "id": 11, "options": { "legend": { "calcs": [ @@ -812,8 +925,8 @@ "uid": "${DS_PROMETHEUS}" }, "editorMode": "code", - "expr": "sum(rate(node_packets_sent[1m]))", - "legendFormat": "sent (sum)", + "expr": "sum(rate(node_packets_sent_total[1m]))", + "legendFormat": "Sent", "range": true, "refId": "A" }, @@ -823,15 +936,15 @@ "uid": "${DS_PROMETHEUS}" }, "editorMode": "code", - "expr": "sum(rate(node_packets_received[1m]))", - "legendFormat": "received (sum)", + "expr": "sum(rate(node_packets_received_total[1m]))", + "legendFormat": "Received", "range": true, "refId": "B" } ], - "title": "Node Throughput (packets/sec)", + "title": "Node Throughput", "type": "timeseries", - "description": "Aggregate packet send/receive rates across all nodes." + "description": "Aggregate packet processing rate across all nodes. Shows pipeline activity." }, { "datasource": { @@ -872,7 +985,7 @@ "mode": "none" }, "thresholdsStyle": { - "mode": "off" + "mode": "line" } }, "mappings": [], @@ -885,7 +998,7 @@ }, { "color": "red", - "value": 1 + "value": 0.1 } ] }, @@ -899,7 +1012,7 @@ "x": 12, "y": 19 }, - "id": 11, + "id": 12, "options": { "legend": { "calcs": [ @@ -922,8 +1035,8 @@ "uid": "${DS_PROMETHEUS}" }, "editorMode": "code", - "expr": "sum(rate(node_packets_errored[1m]))", - "legendFormat": "errors (sum)", + "expr": "sum(rate(node_packets_errored_total[1m]))", + "legendFormat": "Errors", "range": true, "refId": "A" }, @@ -933,15 +1046,15 @@ "uid": "${DS_PROMETHEUS}" }, "editorMode": "code", - "expr": "sum(rate(node_packets_discarded[1m]))", - "legendFormat": "discarded (sum)", + "expr": "sum(rate(node_packets_discarded_total[1m]))", + "legendFormat": "Discarded", "range": true, "refId": "B" } ], - "title": "Node Errors & Drops (per sec)", + "title": "Node Errors & Drops", "type": "timeseries", - "description": "Aggregate error and discard rates across all nodes." + "description": "Processing errors and discarded packets. Should be near zero for healthy operation." }, { "datasource": { @@ -951,7 +1064,39 @@ "fieldConfig": { "defaults": { "color": { - "mode": "thresholds" + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "Seconds", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "tooltip": false, + "viz": false, + "legend": false + }, + "insertNulls": false, + "lineInterpolation": "smooth", + "lineWidth": 2, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "line" + } }, "mappings": [], "thresholds": { @@ -963,44 +1108,40 @@ }, { "color": "yellow", - "value": 100 + "value": 0.1 }, { "color": "red", - "value": 1000 + "value": 1 } ] }, - "unit": "short" + "unit": "s" }, "overrides": [] }, "gridPos": { "h": 8, - "w": 24, + "w": 12, "x": 0, "y": 27 }, - "id": 12, + "id": 13, "options": { - "displayMode": "gradient", - "maxVizHeight": 300, - "minVizHeight": 16, - "minVizWidth": 8, - "namePlacement": "auto", - "orientation": "horizontal", - "reduceOptions": { - "values": false, + "legend": { "calcs": [ - "lastNotNull" + "lastNotNull", + "max" ], - "fields": "" + "displayMode": "table", + "placement": "right", + "showLegend": true }, - "showUnfilled": true, - "sizing": "auto", - "valueMode": "color" + "tooltip": { + "mode": "multi", + "sort": "desc" + } }, - "pluginVersion": "12.0.0", "targets": [ { "datasource": { @@ -1008,15 +1149,26 @@ "uid": "${DS_PROMETHEUS}" }, "editorMode": "code", - "expr": "topk(10, node_packets_sent)", - "legendFormat": "{{node_id}}", + "expr": "histogram_quantile(0.95, sum(rate(pin_distributor_send_wait_seconds_bucket[5m])) by (le))", + "legendFormat": "p95 Wait", "range": true, "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "histogram_quantile(0.99, sum(rate(pin_distributor_send_wait_seconds_bucket[5m])) by (le))", + "legendFormat": "p99 Wait", + "range": true, + "refId": "B" } ], - "title": "Top Nodes by Packets Sent (Total)", - "type": "bargauge", - "description": "Cumulative packets sent, showing only the top talkers." + "title": "Pin Distributor Backpressure", + "type": "timeseries", + "description": "Time spent waiting for downstream capacity. High values indicate bottlenecks." }, { "datasource": { @@ -1032,7 +1184,7 @@ "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", - "axisLabel": "State", + "axisLabel": "Drops/s", "axisPlacement": "auto", "barAlignment": 0, "drawStyle": "line", @@ -1044,7 +1196,7 @@ "legend": false }, "insertNulls": false, - "lineInterpolation": "stepAfter", + "lineInterpolation": "smooth", "lineWidth": 2, "pointSize": 5, "scaleDistribution": { @@ -1057,38 +1209,24 @@ "mode": "none" }, "thresholdsStyle": { - "mode": "off" + "mode": "line" } }, - "mappings": [ - { - "options": { - "0": { - "color": "red", - "index": 0, - "text": "Stopped/Failed" - }, - "1": { - "color": "green", - "index": 1, - "text": "Running" - } - }, - "type": "value" - } - ], - "max": 1, - "min": 0, + "mappings": [], "thresholds": { "mode": "absolute", "steps": [ { - "color": "red", + "color": "green", "value": null }, { - "color": "green", - "value": 1 + "color": "yellow", + "value": 10 + }, + { + "color": "red", + "value": 100 } ] }, @@ -1100,13 +1238,14 @@ "h": 8, "w": 12, "x": 12, - "y": 35 + "y": 27 }, "id": 14, "options": { "legend": { "calcs": [ - "lastNotNull" + "lastNotNull", + "sum" ], "displayMode": "table", "placement": "right", @@ -1114,7 +1253,7 @@ }, "tooltip": { "mode": "multi", - "sort": "none" + "sort": "desc" } }, "targets": [ @@ -1124,1427 +1263,1181 @@ "uid": "${DS_PROMETHEUS}" }, "editorMode": "code", - "expr": "node_state{state=\"running\"}", - "legendFormat": "{{node_id}}", + "expr": "sum(rate(pin_distributor_best_effort_drops_total[1m]))", + "legendFormat": "Best-Effort Drops", "range": true, "refId": "A" } ], - "title": "Node State (1=Running, 0=Other)", + "title": "Best-Effort Drops", "type": "timeseries", - "description": "Binary running state per node (1 = running)." + "description": "Packets dropped on best-effort connections due to slow consumers." }, { - "collapsed": false, + "collapsed": true, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 43 + "y": 35 }, "id": 15, - "panels": [], - "title": "MoQ Transport Nodes", - "type": "row", - "description": "MoQ subscriber/publisher/peer behaviour and health." - }, - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" + "panels": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" }, - "custom": { - "axisBorderShow": false, - "axisCenteredZero": false, - "axisColorMode": "text", - "axisLabel": "Packets/s", - "axisPlacement": "auto", - "barAlignment": 0, - "drawStyle": "line", - "fillOpacity": 20, - "gradientMode": "none", - "hideFrom": { - "tooltip": false, - "viz": false, - "legend": false - }, - "insertNulls": false, - "lineInterpolation": "smooth", - "lineWidth": 2, - "pointSize": 5, - "scaleDistribution": { - "type": "linear" - }, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "none" + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "Packets/s", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "tooltip": false, + "viz": false, + "legend": false + }, + "insertNulls": false, + "lineInterpolation": "smooth", + "lineWidth": 2, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "pps" }, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ + "overrides": [ { - "color": "green", - "value": null + "matcher": { + "id": "byRegexp", + "options": "/received/" + }, + "properties": [ + { + "id": "custom.lineStyle", + "value": { + "dash": [ + 10, + 10 + ], + "fill": "dash" + } + } + ] } ] }, - "unit": "pps" - }, - "overrides": [ - { - "matcher": { - "id": "byRegexp", - "options": "/received/" + "gridPos": { + "h": 8, + "w": 24, + "x": 0, + "y": 36 + }, + "id": 16, + "options": { + "legend": { + "calcs": [ + "lastNotNull", + "mean" + ], + "displayMode": "table", + "placement": "right", + "showLegend": true }, - "properties": [ - { - "id": "custom.lineStyle", - "value": { - "dash": [ - 10, - 10 - ], - "fill": "dash" - } - } - ] - } - ] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 44 - }, - "id": 16, - "options": { - "legend": { - "calcs": [ - "lastNotNull", - "mean" + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(node_packets_sent_total{node_id=~\"(.*moq::(peer|subscriber|publisher).*|moq_(peer|subscriber|publisher).*)\"}[1m])", + "legendFormat": "{{node_id}} sent", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(node_packets_received_total{node_id=~\"(.*moq::(peer|subscriber|publisher).*|moq_(peer|subscriber|publisher).*)\"}[1m])", + "legendFormat": "{{node_id}} received", + "range": true, + "refId": "B" + } ], - "displayMode": "table", - "placement": "right", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" + "title": "MoQ Transport Throughput", + "type": "timeseries", + "description": "Packet rates for MoQ transport nodes (peer/subscriber/publisher)." } + ], + "title": "MoQ Transport (Advanced)", + "type": "row", + "description": "Media over QUIC transport metrics - collapsed by default" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 36 }, - "targets": [ + "id": 17, + "panels": [ { "datasource": { "type": "prometheus", "uid": "${DS_PROMETHEUS}" }, - "editorMode": "code", - "expr": "rate(node_packets_sent{node_id=~\"(.*moq::(peer|subscriber|publisher).*|moq_(peer|subscriber|publisher).*)\"}[1m])", - "legendFormat": "{{node_id}} sent", - "range": true, - "refId": "A" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "yellow", + "value": 100 + }, + { + "color": "red", + "value": 1000 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 24, + "x": 0, + "y": 37 + }, + "id": 18, + "options": { + "displayMode": "gradient", + "maxVizHeight": 300, + "minVizHeight": 16, + "minVizWidth": 8, + "namePlacement": "auto", + "orientation": "horizontal", + "reduceOptions": { + "values": false, + "calcs": [ + "lastNotNull" + ], + "fields": "" + }, + "showUnfilled": true, + "sizing": "auto", + "valueMode": "color" }, - "editorMode": "code", - "expr": "rate(node_packets_received{node_id=~\"(.*moq::(peer|subscriber|publisher).*|moq_(peer|subscriber|publisher).*)\"}[1m])", - "legendFormat": "{{node_id}} received", - "range": true, - "refId": "B" + "pluginVersion": "12.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "topk(10, sum by(node_kind) (node_packets_sent_total))", + "legendFormat": "{{node_kind}}", + "range": true, + "refId": "A" + } + ], + "title": "Top Node Types by Packets Sent", + "type": "bargauge", + "description": "Top 10 node types by total packets sent (aggregated by node kind)." } ], - "title": "MoQ Transport Throughput (packets/sec)", - "type": "timeseries", - "description": "Send and receive rates for MoQ subscriber/publisher/peer nodes." + "title": "Per-Node Details (Advanced)", + "type": "row", + "description": "Deep-dive into individual node performance" }, { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "thresholds" - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - }, - { - "color": "yellow", - "value": 1000 - }, - { - "color": "red", - "value": 10000 - } - ] - }, - "unit": "short" - }, - "overrides": [] - }, + "collapsed": true, "gridPos": { - "h": 8, - "w": 6, - "x": 12, - "y": 44 - }, - "id": 17, - "options": { - "colorMode": "value", - "graphMode": "none", - "justifyMode": "center", - "orientation": "horizontal", - "percentChangeColorMode": "standard", - "reduceOptions": { - "values": false, - "calcs": [ - "lastNotNull" - ], - "fields": "" - }, - "showPercentChange": false, - "textMode": "valueAndName", - "wideLayout": false + "h": 1, + "w": 24, + "x": 0, + "y": 37 }, - "pluginVersion": "12.0.0", - "targets": [ + "id": 19, + "panels": [ { "datasource": { "type": "prometheus", "uid": "${DS_PROMETHEUS}" }, - "editorMode": "code", - "expr": "sum(node_packets_sent{node_id=~\"(.*moq::(peer|subscriber|publisher).*|moq_(peer|subscriber|publisher).*)\"}) by (node_id)", - "legendFormat": "{{node_id}} sent", - "range": true, - "refId": "A" + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "seconds", + "axisPlacement": "auto", + "drawStyle": "line", + "lineWidth": 2, + "fillOpacity": 15, + "showPoints": "never", + "spanNulls": false, + "lineInterpolation": "smooth", + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 38 + }, + "id": 20, + "options": { + "legend": { + "calcs": [ + "lastNotNull", + "max" + ], + "displayMode": "list", + "placement": "right", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "refId": "A", + "range": true, + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "histogram_quantile(0.95, sum by(le, node_id) (rate(pacer_lateness_seconds_bucket[5m])))", + "legendFormat": "{{node_id}}" + } + ], + "title": "Pacer Lateness p95", + "type": "timeseries", + "description": "Audio pacing lateness (95th percentile). Shows timing drift." }, { "datasource": { "type": "prometheus", "uid": "${DS_PROMETHEUS}" }, - "editorMode": "code", - "expr": "sum(node_packets_received{node_id=~\"(.*moq::(peer|subscriber|publisher).*|moq_(peer|subscriber|publisher).*)\"}) by (node_id)", - "legendFormat": "{{node_id}} received", - "range": true, - "refId": "B" - } - ], - "title": "MoQ Transport Packets (Total)", - "type": "stat", - "description": "Total packets sent/received by each MoQ subscriber/publisher/peer node." - }, - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisBorderShow": false, - "axisCenteredZero": false, - "axisColorMode": "text", - "axisLabel": "State", - "axisPlacement": "auto", - "barAlignment": 0, - "drawStyle": "line", - "fillOpacity": 20, - "gradientMode": "none", - "hideFrom": { - "tooltip": false, - "viz": false, - "legend": false - }, - "insertNulls": false, - "lineInterpolation": "stepAfter", - "lineWidth": 2, - "pointSize": 5, - "scaleDistribution": { - "type": "linear" + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "seconds", + "axisPlacement": "auto", + "drawStyle": "line", + "lineWidth": 2, + "fillOpacity": 10, + "showPoints": "never", + "spanNulls": true, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "s" }, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "none" + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 38 + }, + "id": 21, + "options": { + "legend": { + "calcs": [ + "lastNotNull", + "max" + ], + "displayMode": "list", + "placement": "right", + "showLegend": true }, - "thresholdsStyle": { - "mode": "off" + "tooltip": { + "mode": "multi", + "sort": "desc" } }, - "mappings": [ + "targets": [ { - "options": { - "0": { - "color": "red", - "index": 0, - "text": "Stopped" - }, - "1": { - "color": "green", - "index": 1, - "text": "Running" - } + "refId": "A", + "range": true, + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" }, - "type": "value" + "editorMode": "code", + "expr": "avg by(node_id, pin_name) (pin_distributor_queue_depth_seconds)", + "legendFormat": "{{node_id}}/{{pin_name}}" } ], - "max": 1, - "min": 0, - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "red", - "value": null - }, - { - "color": "green", - "value": 1 - } - ] - }, - "unit": "short" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 6, - "x": 18, - "y": 44 - }, - "id": 18, - "options": { - "legend": { - "calcs": [ - "lastNotNull" - ], - "displayMode": "table", - "placement": "right", - "showLegend": true + "title": "Pin Distributor Queue Depth (seconds)", + "type": "timeseries", + "description": "Estimated backlog in media seconds at pin distributors." }, - "tooltip": { - "mode": "multi", - "sort": "none" - } - }, - "targets": [ { "datasource": { "type": "prometheus", "uid": "${DS_PROMETHEUS}" }, - "editorMode": "code", - "expr": "node_state{node_id=~\"(.*moq::(peer|subscriber|publisher).*|moq_(peer|subscriber|publisher).*)\", state=\"running\"}", - "legendFormat": "{{node_id}}", - "range": true, - "refId": "A" - } - ], - "title": "MoQ Transport State", - "type": "timeseries", - "description": "Running state of each MoQ subscriber/publisher/peer node." - }, - { - "collapsed": false, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 52 - }, - "id": 19, - "panels": [], - "title": "SLOs", - "type": "row", - "description": "High-signal latency and error-rate panels." - }, - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "unit": "percent", - "min": 0, - "max": 100 - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 53 - }, - "id": 20, - "options": { - "legend": { - "calcs": [ - "lastNotNull", - "max" + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "packets", + "axisPlacement": "auto", + "drawStyle": "line", + "lineWidth": 2, + "fillOpacity": 10, + "showPoints": "never", + "spanNulls": true, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 46 + }, + "id": 22, + "options": { + "legend": { + "calcs": [ + "lastNotNull", + "max" + ], + "displayMode": "list", + "placement": "right", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "refId": "A", + "range": true, + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "avg by(node_id) (pacer_queue_depth)", + "legendFormat": "{{node_id}}" + } ], - "displayMode": "table", - "placement": "right", - "showLegend": true + "title": "Pacer Queue Depth", + "type": "timeseries", + "description": "Pacer buffered packet count." }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ { "datasource": { "type": "prometheus", "uid": "${DS_PROMETHEUS}" }, - "editorMode": "code", - "expr": "100 * sum(rate(http_server_requests_total{http_route=~\"/api/.*\", http_status_code=~\"5..\"}[5m])) / clamp_min(sum(rate(http_server_requests_total{http_route=~\"/api/.*\"}[5m])), 1e-9)", - "legendFormat": "5xx", - "range": true, - "refId": "A" + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "ms", + "axisPlacement": "auto", + "drawStyle": "line", + "lineWidth": 2, + "fillOpacity": 10, + "showPoints": "never", + "spanNulls": true, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "ms" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 46 + }, + "id": 23, + "options": { + "legend": { + "calcs": [ + "lastNotNull", + "max" + ], + "displayMode": "list", + "placement": "right", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "refId": "A", + "range": true, + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "histogram_quantile(0.95, sum by(le, node_id, broadcast) (rate(moq_push_clock_offset_ms_bucket[5m])))", + "legendFormat": "{{node_id}}/{{broadcast}}" + } + ], + "title": "MoQ Push Clock Offset p95", + "type": "timeseries", + "description": "Offset between outgoing MoQ timestamp and upstream packet timestamp." }, { "datasource": { "type": "prometheus", "uid": "${DS_PROMETHEUS}" }, - "editorMode": "code", - "expr": "100 * sum(rate(http_server_requests_total{http_route=~\"/api/.*\", http_status_code=~\"4..\"}[5m])) / clamp_min(sum(rate(http_server_requests_total{http_route=~\"/api/.*\"}[5m])), 1e-9)", - "legendFormat": "4xx", - "range": true, - "refId": "B" + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "ms", + "axisPlacement": "auto", + "drawStyle": "line", + "lineWidth": 2, + "fillOpacity": 10, + "showPoints": "never", + "spanNulls": true, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "ms" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 54 + }, + "id": 24, + "options": { + "legend": { + "calcs": [ + "lastNotNull", + "max" + ], + "displayMode": "list", + "placement": "right", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "refId": "A", + "range": true, + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "histogram_quantile(0.95, sum by(le, node_id, broadcast) (rate(moq_peer_inter_frame_ms_bucket[5m])))", + "legendFormat": "{{node_id}}/{{broadcast}}" + } + ], + "title": "MoQ Peer Inter-frame Gap p95", + "type": "timeseries", + "description": "Gap between consecutive frames sent to subscribers." } ], - "title": "HTTP Error Rate (API)", - "type": "timeseries", - "description": "HTTP error rate for /api/* endpoints (4xx and 5xx)." + "title": "Timing & Queue Depths (Advanced)", + "type": "row", + "description": "Detailed timing and queue metrics" }, { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "unit": "s", - "min": 0 - }, - "overrides": [] - }, + "collapsed": true, "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 53 - }, - "id": 21, - "options": { - "legend": { - "calcs": [ - "lastNotNull", - "max" - ], - "displayMode": "table", - "placement": "right", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } + "h": 1, + "w": 24, + "x": 0, + "y": 38 }, - "targets": [ + "id": 25, + "panels": [ { "datasource": { "type": "prometheus", "uid": "${DS_PROMETHEUS}" }, - "editorMode": "code", - "expr": "histogram_quantile(0.95, sum(rate(http_server_duration_bucket{http_route=~\"/api/.*\", http_status_code=~\"2..\"}[5m])) by (le))", - "legendFormat": "p95", - "range": true, - "refId": "A" + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "Milliseconds", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "tooltip": false, + "viz": false, + "legend": false + }, + "insertNulls": false, + "lineInterpolation": "smooth", + "lineWidth": 2, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "ms" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": ".*Throughput.*" + }, + "properties": [ + { + "id": "unit", + "value": "pps" + }, + { + "id": "custom.axisPlacement", + "value": "right" + }, + { + "id": "custom.axisLabel", + "value": "Packets/s" + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 39 + }, + "id": 26, + "options": { + "legend": { + "calcs": [ + "lastNotNull", + "mean" + ], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "histogram_quantile(0.95, sum(rate(opus_decode_duration_bucket[5m])) by (le)) * 1000", + "legendFormat": "Decode p95", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "histogram_quantile(0.95, sum(rate(opus_encode_duration_bucket[5m])) by (le)) * 1000", + "legendFormat": "Encode p95", + "range": true, + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum(rate(opus_decode_duration_count[1m]))", + "legendFormat": "Decode Throughput", + "range": true, + "refId": "C" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum(rate(opus_encode_duration_count[1m]))", + "legendFormat": "Encode Throughput", + "range": true, + "refId": "D" + } + ], + "title": "Opus Codec Performance", + "type": "timeseries", + "description": "Opus p95 duration per packet (left axis, typical: 1-10ms) and processing throughput (right axis)." }, { "datasource": { "type": "prometheus", "uid": "${DS_PROMETHEUS}" }, - "editorMode": "code", - "expr": "histogram_quantile(0.99, sum(rate(http_server_duration_bucket{http_route=~\"/api/.*\", http_status_code=~\"2..\"}[5m])) by (le))", - "legendFormat": "p99", - "range": true, - "refId": "B" - } - ], - "title": "HTTP Latency (API, 2xx)", - "type": "timeseries", - "description": "p95/p99 latency for successful /api/* requests (2xx only)." - }, - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "unit": "s", - "min": 0 - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 61 - }, - "id": 22, - "options": { - "legend": { - "calcs": [ - "lastNotNull", - "max" - ], - "displayMode": "table", - "placement": "right", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "editorMode": "code", - "expr": "histogram_quantile(0.95, sum(rate(oneshot_pipeline_duration_bucket{status=\"ok\"}[5m])) by (le))", - "legendFormat": "p95", - "range": true, - "refId": "A" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "editorMode": "code", - "expr": "histogram_quantile(0.99, sum(rate(oneshot_pipeline_duration_bucket{status=\"ok\"}[5m])) by (le))", - "legendFormat": "p99", - "range": true, - "refId": "B" - } - ], - "title": "Oneshot Pipeline Latency (ok)", - "type": "timeseries", - "description": "p95/p99 runtime for oneshot pipeline executions (status=ok). Samples are emitted when a oneshot pipeline finishes (long-running jobs won't appear until completion)." - }, - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "unit": "percent", - "min": 0, - "max": 100 - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 61 - }, - "id": 23, - "options": { - "legend": { - "calcs": [ - "lastNotNull", - "max" - ], - "displayMode": "table", - "placement": "right", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "editorMode": "code", - "expr": "100 * sum(rate(oneshot_pipeline_duration_count{status=\"error\"}[5m])) / clamp_min(sum(rate(oneshot_pipeline_duration_count[5m])), 1e-9)", - "legendFormat": "error %", - "range": true, - "refId": "A" - } - ], - "title": "Oneshot Pipeline Error Rate", - "type": "timeseries", - "description": "Percentage of oneshot pipeline executions that end with status=error. Samples are emitted when a oneshot pipeline finishes." - }, - { - "collapsed": false, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 69 - }, - "id": 24, - "panels": [], - "title": "Pin Distributor (Routing)", - "type": "row", - "description": "Packet fan-out routing performance and backpressure at the output-pin distributor layer." - }, - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisBorderShow": false, - "axisCenteredZero": false, - "axisColorMode": "text", - "axisLabel": "Seconds", - "axisPlacement": "auto", - "barAlignment": 0, - "drawStyle": "line", - "fillOpacity": 20, - "gradientMode": "none", - "hideFrom": { - "tooltip": false, - "viz": false, - "legend": false - }, - "insertNulls": false, - "lineInterpolation": "smooth", - "lineWidth": 2, - "pointSize": 5, - "scaleDistribution": { - "type": "linear" - }, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "none" - }, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - } - ] - }, - "unit": "s" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 70 - }, - "id": 25, - "options": { - "legend": { - "calcs": [ - "lastNotNull", - "max" - ], - "displayMode": "table", - "placement": "right", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "editorMode": "code", - "expr": "histogram_quantile(0.95, sum(rate(pin_distributor_send_wait_seconds_bucket[5m])) by (le))", - "legendFormat": "p95 (global)", - "range": true, - "refId": "A" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "editorMode": "code", - "expr": "histogram_quantile(0.99, sum(rate(pin_distributor_send_wait_seconds_bucket[5m])) by (le))", - "legendFormat": "p99 (global)", - "range": true, - "refId": "B" - } - ], - "title": "Pin Distributor Backpressure Wait (p95, p99)", - "type": "timeseries", - "description": "p95/p99 of time spent blocked on downstream capacity in output-pin distributors (synchronized backpressure)." - }, - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisBorderShow": false, - "axisCenteredZero": false, - "axisColorMode": "text", - "axisLabel": "Outputs", - "axisPlacement": "auto", - "barAlignment": 0, - "drawStyle": "line", - "fillOpacity": 20, - "gradientMode": "none", - "hideFrom": { - "tooltip": false, - "viz": false, - "legend": false - }, - "insertNulls": false, - "lineInterpolation": "smooth", - "lineWidth": 2, - "pointSize": 5, - "scaleDistribution": { - "type": "linear" - }, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "none" - }, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - } - ] - }, - "unit": "short" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 70 - }, - "id": 26, - "options": { - "legend": { - "calcs": [ - "lastNotNull", - "max" - ], - "displayMode": "table", - "placement": "right", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "editorMode": "code", - "expr": "sum(pin_distributor_outputs_active)", - "legendFormat": "outputs (sum)", - "range": true, - "refId": "A" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "editorMode": "code", - "expr": "max(pin_distributor_outputs_active)", - "legendFormat": "outputs (max pin)", - "range": true, - "refId": "B" - } - ], - "title": "Pin Distributor Outputs Active", - "type": "timeseries", - "description": "Active downstream outputs per distributor. Sum = total fan-out edges; max = most fanned-out single output pin." - }, - { - "id": 27, - "title": "Pacer Lateness p95 (s)", - "type": "timeseries", - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisLabel": "seconds", - "axisPlacement": "auto", - "drawStyle": "line", - "lineWidth": 2, - "fillOpacity": 15, - "showPoints": "never", - "spanNulls": false, - "lineInterpolation": "smooth", - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - } - ] - }, - "unit": "s" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 80 - }, - "options": { - "legend": { - "calcs": [ - "lastNotNull", - "max" - ], - "displayMode": "list", - "placement": "right", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "refId": "A", - "range": true, - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "editorMode": "code", - "expr": "histogram_quantile(0.95, sum by(le, node) (rate(pacer_lateness_seconds_bucket[5m])))", - "legendFormat": "{{node}}" - } - ] - }, - { - "id": 28, - "title": "Pacer Queue Depth", - "type": "timeseries", - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisLabel": "packets", - "axisPlacement": "auto", - "drawStyle": "line", - "lineWidth": 2, - "fillOpacity": 10, - "showPoints": "never", - "spanNulls": true, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - } - ] - }, - "unit": "none" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 80 - }, - "options": { - "legend": { - "calcs": [ - "lastNotNull", - "max" - ], - "displayMode": "list", - "placement": "right", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "refId": "A", - "range": true, - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "editorMode": "code", - "expr": "avg by(node) (pacer_queue_depth)", - "legendFormat": "{{node}}" - } - ] - }, - { - "id": 29, - "title": "MoQ Push Clock Offset p95 (ms)", - "type": "timeseries", - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisLabel": "milliseconds", - "axisPlacement": "auto", - "drawStyle": "line", - "lineWidth": 2, - "fillOpacity": 15, - "showPoints": "never", - "spanNulls": false, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - } - ] - }, - "unit": "ms" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 88 - }, - "options": { - "legend": { - "calcs": [ - "lastNotNull", - "max" - ], - "displayMode": "list", - "placement": "right", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "refId": "A", - "range": true, - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "editorMode": "code", - "expr": "histogram_quantile(0.95, sum by(le, node, broadcast) (rate(moq_push_clock_offset_ms_bucket[5m])))", - "legendFormat": "{{node}}/{{broadcast}}" - } - ] - }, - { - "id": 30, - "title": "MoQ Peer Inter-frame Gap p95 (ms)", - "type": "timeseries", - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisLabel": "milliseconds", - "axisPlacement": "auto", - "drawStyle": "line", - "lineWidth": 2, - "fillOpacity": 15, - "showPoints": "never", - "spanNulls": false, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - } - ] - }, - "unit": "ms" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 88 - }, - "options": { - "legend": { - "calcs": [ - "lastNotNull", - "max" - ], - "displayMode": "list", - "placement": "right", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "refId": "A", - "range": true, - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "editorMode": "code", - "expr": "histogram_quantile(0.95, sum by(le, node, broadcast) (rate(moq_peer_inter_frame_ms_bucket[5m])))", - "legendFormat": "{{node}}/{{broadcast}}" - } - ] - }, - { - "id": 31, - "title": "Pin Distributor Queue Depth", - "type": "timeseries", - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisLabel": "packets", - "axisPlacement": "auto", - "drawStyle": "line", - "lineWidth": 2, - "fillOpacity": 10, - "showPoints": "never", - "spanNulls": true, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "Milliseconds", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "tooltip": false, + "viz": false, + "legend": false + }, + "insertNulls": false, + "lineInterpolation": "smooth", + "lineWidth": 2, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "ms" + }, + "overrides": [ { - "color": "green", - "value": null + "matcher": { + "id": "byRegexp", + "options": ".*Throughput.*" + }, + "properties": [ + { + "id": "unit", + "value": "pps" + }, + { + "id": "custom.axisPlacement", + "value": "right" + }, + { + "id": "custom.axisLabel", + "value": "Packets/s" + } + ] } ] }, - "unit": "none" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 96 - }, - "options": { - "legend": { - "calcs": [ - "lastNotNull", - "max" - ], - "displayMode": "list", - "placement": "right", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "refId": "A", - "range": true, - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "editorMode": "code", - "expr": "avg by(node_id, pin_name) (pin_distributor_queue_depth)", - "legendFormat": "{{node_id}}/{{pin_name}}" - } - ] - }, - { - "id": 32, - "title": "Pin Distributor Queue Depth (bytes)", - "type": "timeseries", - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisLabel": "bytes", - "axisPlacement": "auto", - "drawStyle": "line", - "lineWidth": 2, - "fillOpacity": 10, - "showPoints": "never", - "spanNulls": true, - "thresholdsStyle": { - "mode": "off" + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 39 + }, + "id": 27, + "options": { + "legend": { + "calcs": [ + "lastNotNull", + "mean" + ], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" } }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - } - ] - }, - "unit": "bytes" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 104 - }, - "options": { - "legend": { - "calcs": [ - "lastNotNull", - "max" + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "histogram_quantile(0.95, sum(rate(mp3_decode_duration_bucket[5m])) by (le)) * 1000", + "legendFormat": "Decode p95", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum(rate(mp3_decode_duration_count[1m]))", + "legendFormat": "Decode Throughput", + "range": true, + "refId": "B" + } ], - "displayMode": "list", - "placement": "right", - "showLegend": true + "title": "MP3 Codec Performance", + "type": "timeseries", + "description": "MP3 p95 decode duration per packet (left axis) and processing throughput (right axis)." }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ { - "refId": "A", - "range": true, "datasource": { "type": "prometheus", "uid": "${DS_PROMETHEUS}" }, - "editorMode": "code", - "expr": "avg by(node_id, pin_name) (pin_distributor_queue_depth_bytes)", - "legendFormat": "{{node_id}}/{{pin_name}}" - } - ] - }, - { - "id": 33, - "title": "Pin Distributor Queue Depth (seconds)", - "type": "timeseries", - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisLabel": "seconds", - "axisPlacement": "auto", - "drawStyle": "line", - "lineWidth": 2, - "fillOpacity": 10, - "showPoints": "never", - "spanNulls": true, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "Milliseconds", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "tooltip": false, + "viz": false, + "legend": false + }, + "insertNulls": false, + "lineInterpolation": "smooth", + "lineWidth": 2, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "ms" + }, + "overrides": [ { - "color": "green", - "value": null + "matcher": { + "id": "byRegexp", + "options": ".*Throughput.*" + }, + "properties": [ + { + "id": "unit", + "value": "pps" + }, + { + "id": "custom.axisPlacement", + "value": "right" + }, + { + "id": "custom.axisLabel", + "value": "Packets/s" + } + ] } ] }, - "unit": "s" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 104 - }, - "options": { - "legend": { - "calcs": [ - "lastNotNull", - "max" + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 47 + }, + "id": 28, + "options": { + "legend": { + "calcs": [ + "lastNotNull", + "mean" + ], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "histogram_quantile(0.95, sum(rate(flac_decode_duration_bucket[5m])) by (le)) * 1000", + "legendFormat": "Decode p95", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum(rate(flac_decode_duration_count[1m]))", + "legendFormat": "Decode Throughput", + "range": true, + "refId": "B" + } ], - "displayMode": "list", - "placement": "right", - "showLegend": true + "title": "FLAC Codec Performance", + "type": "timeseries", + "description": "FLAC p95 decode duration per packet (left axis) and processing throughput (right axis)." }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ { - "refId": "A", - "range": true, "datasource": { "type": "prometheus", "uid": "${DS_PROMETHEUS}" }, - "editorMode": "code", - "expr": "avg by(node_id, pin_name) (pin_distributor_queue_depth_seconds)", - "legendFormat": "{{node_id}}/{{pin_name}}" - } - ] - }, - { - "id": 34, - "title": "Pin Distributor Send Wait p95 (s)", - "type": "timeseries", - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisLabel": "seconds", - "axisPlacement": "auto", - "drawStyle": "line", - "lineWidth": 2, - "fillOpacity": 15, - "showPoints": "never", - "spanNulls": false, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "Milliseconds", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "none", + "hideFrom": { + "tooltip": false, + "viz": false, + "legend": false + }, + "insertNulls": false, + "lineInterpolation": "smooth", + "lineWidth": 2, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "ms" + }, + "overrides": [ { - "color": "green", - "value": null + "matcher": { + "id": "byRegexp", + "options": ".*Throughput.*" + }, + "properties": [ + { + "id": "unit", + "value": "pps" + }, + { + "id": "custom.axisPlacement", + "value": "right" + }, + { + "id": "custom.axisLabel", + "value": "Packets/s" + } + ] } ] }, - "unit": "s" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 112 - }, - "options": { - "legend": { - "calcs": [ - "lastNotNull", - "max" - ], - "displayMode": "list", - "placement": "right", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "refId": "A", - "range": true, - "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 47 + }, + "id": 29, + "options": { + "legend": { + "calcs": [ + "lastNotNull", + "mean" + ], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } }, - "editorMode": "code", - "expr": "histogram_quantile(0.95, sum by(le, node_id, pin_name) (rate(pin_distributor_send_wait_seconds_bucket[5m])))", - "legendFormat": "{{node_id}}/{{pin_name}}" + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "histogram_quantile(0.95, sum(rate(wav_demux_duration_bucket[5m])) by (le)) * 1000", + "legendFormat": "Demux p95", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "sum(rate(wav_demux_duration_count[1m]))", + "legendFormat": "Demux Throughput", + "range": true, + "refId": "B" + } + ], + "title": "WAV Container Performance", + "type": "timeseries", + "description": "WAV p95 demux duration per packet (left axis) and processing throughput (right axis)." } - ] + ], + "title": "Codecs (Advanced)", + "type": "row", + "description": "Audio codec and container performance metrics" } ], "refresh": "10s", @@ -2601,7 +2494,7 @@ "timezone": "browser", "title": "StreamKit Performance Dashboard", "uid": "streamkit-perf", - "version": 2, + "version": 1, "weekStart": "", - "description": "Operational overview of StreamKit runtime health and throughput." + "description": "Streamlined operational dashboard focusing on high-signal health indicators. Expanded rows show core metrics, collapsed rows provide deep-dive debugging." }