|
| 1 | +use std::pin::Pin; |
| 2 | +use std::task::{Context, Poll}; |
| 3 | + |
| 4 | +use crate::metrics::proto::MetricsSetProto; |
| 5 | +use crate::protobuf::StageKey; |
| 6 | +use crate::protobuf::{AppMetadata, FlightAppMetadata}; |
| 7 | +use arrow_flight::{error::FlightError, FlightData}; |
| 8 | +use dashmap::DashMap; |
| 9 | +use futures::stream::Stream; |
| 10 | +use prost::Message; |
| 11 | +use std::sync::Arc; |
| 12 | + |
| 13 | +/// MetricsCollectingStream wraps a FlightData stream and extracts metrics from app_metadata |
| 14 | +/// while passing through all the other FlightData unchanged. |
| 15 | +pub struct MetricsCollectingStream<S> |
| 16 | +where |
| 17 | + S: Stream<Item = Result<FlightData, FlightError>> + Send + Unpin, |
| 18 | +{ |
| 19 | + inner: S, |
| 20 | + metrics_collection: Arc<DashMap<StageKey, Vec<MetricsSetProto>>>, |
| 21 | +} |
| 22 | + |
| 23 | +impl<S> MetricsCollectingStream<S> |
| 24 | +where |
| 25 | + S: Stream<Item = Result<FlightData, FlightError>> + Send + Unpin, |
| 26 | +{ |
| 27 | + #[allow(dead_code)] |
| 28 | + pub fn new( |
| 29 | + stream: S, |
| 30 | + metrics_collection: Arc<DashMap<StageKey, Vec<MetricsSetProto>>>, |
| 31 | + ) -> Self { |
| 32 | + Self { |
| 33 | + inner: stream, |
| 34 | + metrics_collection, |
| 35 | + } |
| 36 | + } |
| 37 | + |
| 38 | + fn extract_metrics_from_flight_data( |
| 39 | + &self, |
| 40 | + flight_data: &mut FlightData, |
| 41 | + ) -> Result<(), FlightError> { |
| 42 | + if !flight_data.app_metadata.is_empty() { |
| 43 | + return match FlightAppMetadata::decode(flight_data.app_metadata.as_ref()) { |
| 44 | + Ok(metadata) => { |
| 45 | + if let Some(content) = metadata.content { |
| 46 | + match content { |
| 47 | + AppMetadata::MetricsCollection(task_metrics_set) => { |
| 48 | + for task_metrics in task_metrics_set.tasks { |
| 49 | + if let Some(stage_key) = task_metrics.stage_key { |
| 50 | + self.metrics_collection |
| 51 | + .insert(stage_key, task_metrics.metrics); |
| 52 | + } else { |
| 53 | + return Err(FlightError::ProtocolError("expected Some StageKey in MetricsCollectingStream, got None".to_string())); |
| 54 | + } |
| 55 | + } |
| 56 | + } |
| 57 | + } |
| 58 | + } |
| 59 | + flight_data.app_metadata.clear(); |
| 60 | + Ok(()) |
| 61 | + } |
| 62 | + Err(e) => Err(FlightError::ProtocolError(format!( |
| 63 | + "failed to decode app_metadata: {}", |
| 64 | + e |
| 65 | + ))), |
| 66 | + }; |
| 67 | + } |
| 68 | + Ok(()) |
| 69 | + } |
| 70 | +} |
| 71 | + |
| 72 | +impl<S> Stream for MetricsCollectingStream<S> |
| 73 | +where |
| 74 | + S: Stream<Item = Result<FlightData, FlightError>> + Send + Unpin, |
| 75 | +{ |
| 76 | + type Item = Result<FlightData, FlightError>; |
| 77 | + |
| 78 | + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> { |
| 79 | + match Pin::new(&mut self.inner).poll_next(cx) { |
| 80 | + Poll::Ready(Some(Ok(mut flight_data))) => { |
| 81 | + // Extract metrics from app_metadata if present. |
| 82 | + match self.extract_metrics_from_flight_data(&mut flight_data) { |
| 83 | + Ok(_) => Poll::Ready(Some(Ok(flight_data))), |
| 84 | + Err(e) => Poll::Ready(Some(Err(e))), |
| 85 | + } |
| 86 | + } |
| 87 | + Poll::Ready(Some(Err(err))) => Poll::Ready(Some(Err(err))), |
| 88 | + Poll::Ready(None) => Poll::Ready(None), |
| 89 | + Poll::Pending => Poll::Pending, |
| 90 | + } |
| 91 | + } |
| 92 | +} |
| 93 | + |
| 94 | +#[cfg(test)] |
| 95 | +mod tests { |
| 96 | + use super::*; |
| 97 | + use crate::metrics::proto::{ |
| 98 | + MetricProto, MetricValueProto, MetricsSetProto, NamedCount, NamedGauge, |
| 99 | + }; |
| 100 | + use crate::protobuf::{ |
| 101 | + AppMetadata, FlightAppMetadata, MetricsCollection, StageKey, TaskMetrics, |
| 102 | + }; |
| 103 | + use arrow_flight::FlightData; |
| 104 | + use futures::stream::{self, StreamExt}; |
| 105 | + use prost::{bytes::Bytes, Message}; |
| 106 | + |
| 107 | + #[tokio::test] |
| 108 | + async fn test_metrics_collecting_stream_extracts_and_removes_metadata() { |
| 109 | + let test_metrics_set = MetricsSetProto { |
| 110 | + metrics: vec![ |
| 111 | + MetricProto { |
| 112 | + metric: Some(MetricValueProto::Count(NamedCount { |
| 113 | + name: "test_count".to_string(), |
| 114 | + value: 42, |
| 115 | + })), |
| 116 | + labels: vec![], |
| 117 | + partition: Some(0), |
| 118 | + }, |
| 119 | + MetricProto { |
| 120 | + metric: Some(MetricValueProto::Gauge(NamedGauge { |
| 121 | + name: "test_gauge".to_string(), |
| 122 | + value: 99, |
| 123 | + })), |
| 124 | + labels: vec![], |
| 125 | + partition: Some(0), |
| 126 | + }, |
| 127 | + ], |
| 128 | + }; |
| 129 | + |
| 130 | + let stage_keys = vec![ |
| 131 | + StageKey { |
| 132 | + query_id: "test_query".to_string(), |
| 133 | + stage_id: 1, |
| 134 | + task_number: 1, |
| 135 | + }, |
| 136 | + StageKey { |
| 137 | + query_id: "test_query_2".to_string(), |
| 138 | + stage_id: 2, |
| 139 | + task_number: 2, |
| 140 | + }, |
| 141 | + ]; |
| 142 | + |
| 143 | + let app_metadatas = stage_keys |
| 144 | + .iter() |
| 145 | + .map(|stage_key| FlightAppMetadata { |
| 146 | + content: Some(AppMetadata::MetricsCollection(MetricsCollection { |
| 147 | + tasks: vec![TaskMetrics { |
| 148 | + stage_key: Some(stage_key.clone()), |
| 149 | + metrics: vec![test_metrics_set.clone()], |
| 150 | + }], |
| 151 | + })), |
| 152 | + }) |
| 153 | + .collect::<Vec<_>>(); |
| 154 | + |
| 155 | + // Create test FlightData messages - some with metadata, some without |
| 156 | + let flight_data_with_metadata = FlightData { |
| 157 | + flight_descriptor: None, |
| 158 | + data_header: Bytes::new(), |
| 159 | + app_metadata: app_metadatas[0].encode_to_vec().into(), |
| 160 | + data_body: vec![1, 2, 3].into(), |
| 161 | + }; |
| 162 | + |
| 163 | + let flight_data_without_metadata = FlightData { |
| 164 | + flight_descriptor: None, |
| 165 | + data_header: Bytes::new(), |
| 166 | + app_metadata: Bytes::new(), |
| 167 | + data_body: vec![4, 5, 6].into(), |
| 168 | + }; |
| 169 | + |
| 170 | + let flight_data_with_metadata2 = FlightData { |
| 171 | + flight_descriptor: None, |
| 172 | + data_header: Bytes::new(), |
| 173 | + app_metadata: app_metadatas[1].encode_to_vec().into(), |
| 174 | + data_body: vec![7, 8, 9].into(), |
| 175 | + }; |
| 176 | + |
| 177 | + let input_stream = stream::iter(vec![ |
| 178 | + Ok(flight_data_with_metadata), |
| 179 | + Ok(flight_data_without_metadata), |
| 180 | + Ok(flight_data_with_metadata2), |
| 181 | + ]); |
| 182 | + |
| 183 | + let metrics_collection = Arc::new(DashMap::new()); |
| 184 | + let mut collecting_stream = |
| 185 | + MetricsCollectingStream::new(input_stream, metrics_collection.clone()); |
| 186 | + |
| 187 | + // Collect all messages from the stream. All should have empty app_metadata. |
| 188 | + let mut collected_messages = vec![]; |
| 189 | + while let Some(result) = collecting_stream.next().await { |
| 190 | + collected_messages.push(result.unwrap()); |
| 191 | + } |
| 192 | + assert_eq!(collected_messages.len(), 3); |
| 193 | + for msg in &collected_messages { |
| 194 | + assert!( |
| 195 | + msg.app_metadata.is_empty(), |
| 196 | + "app_metadata should be empty after collection" |
| 197 | + ); |
| 198 | + } |
| 199 | + |
| 200 | + // Verify the data in the messages. |
| 201 | + assert_eq!(collected_messages[0].data_body, vec![1, 2, 3]); |
| 202 | + assert_eq!(collected_messages[1].data_body, vec![4, 5, 6]); |
| 203 | + assert_eq!(collected_messages[2].data_body, vec![7, 8, 9]); |
| 204 | + |
| 205 | + // Verify metrics were collected |
| 206 | + assert_eq!(metrics_collection.len(), 2); |
| 207 | + for stage_key in stage_keys { |
| 208 | + let collected_metrics = metrics_collection.get(&stage_key).unwrap(); |
| 209 | + assert_eq!(collected_metrics.len(), 1); |
| 210 | + assert_eq!(collected_metrics[0].metrics.len(), 2); // We have 2 metrics: Count and Gauge |
| 211 | + // Verify the first metric value (Count) |
| 212 | + if let Some(MetricValueProto::Count(count)) = &collected_metrics[0].metrics[0].metric { |
| 213 | + assert_eq!(count.name, "test_count"); |
| 214 | + assert_eq!(count.value, 42); |
| 215 | + } else { |
| 216 | + panic!("expected Count metric"); |
| 217 | + } |
| 218 | + |
| 219 | + // Verify the second metric value (Gauge) |
| 220 | + if let Some(MetricValueProto::Gauge(gauge)) = &collected_metrics[0].metrics[1].metric { |
| 221 | + assert_eq!(gauge.name, "test_gauge"); |
| 222 | + assert_eq!(gauge.value, 99); |
| 223 | + } else { |
| 224 | + panic!("expected Gauge metric"); |
| 225 | + } |
| 226 | + } |
| 227 | + } |
| 228 | + |
| 229 | + #[tokio::test] |
| 230 | + async fn test_metrics_collecting_stream_error_missing_stage_key() { |
| 231 | + let metrics_collection = Arc::new(DashMap::new()); |
| 232 | + let task_metrics_with_no_stage_key = TaskMetrics { |
| 233 | + stage_key: None, |
| 234 | + metrics: vec![MetricsSetProto { |
| 235 | + metrics: vec![MetricProto { |
| 236 | + metric: Some(MetricValueProto::Count(NamedCount { |
| 237 | + name: "test_count".to_string(), |
| 238 | + value: 42, |
| 239 | + })), |
| 240 | + labels: vec![], |
| 241 | + partition: Some(0), |
| 242 | + }], |
| 243 | + }], |
| 244 | + }; |
| 245 | + |
| 246 | + let invalid_app_metadata = FlightAppMetadata { |
| 247 | + content: Some(AppMetadata::MetricsCollection(MetricsCollection { |
| 248 | + tasks: vec![task_metrics_with_no_stage_key], |
| 249 | + })), |
| 250 | + }; |
| 251 | + |
| 252 | + let invalid_flight_data = FlightData { |
| 253 | + flight_descriptor: None, |
| 254 | + data_header: Bytes::new(), |
| 255 | + app_metadata: invalid_app_metadata.encode_to_vec().into(), |
| 256 | + data_body: vec![1, 2, 3].into(), |
| 257 | + }; |
| 258 | + |
| 259 | + let error_stream = stream::iter(vec![Ok(invalid_flight_data)]); |
| 260 | + let mut collecting_stream = MetricsCollectingStream::new(error_stream, metrics_collection); |
| 261 | + |
| 262 | + let result = collecting_stream.next().await.unwrap(); |
| 263 | + assert!(result.is_err()); |
| 264 | + if let Err(FlightError::ProtocolError(msg)) = result { |
| 265 | + assert!(msg.contains("expected Some StageKey in MetricsCollectingStream, got None")); |
| 266 | + } else { |
| 267 | + panic!("expected FlightError::ProtocolError with stage key error"); |
| 268 | + } |
| 269 | + } |
| 270 | + |
| 271 | + #[tokio::test] |
| 272 | + async fn test_metrics_collecting_stream_error_invalid_metadata() { |
| 273 | + let metrics_collection = Arc::new(DashMap::new()); |
| 274 | + |
| 275 | + let flight_data_with_invalid_metadata = FlightData { |
| 276 | + flight_descriptor: None, |
| 277 | + data_header: Bytes::new(), |
| 278 | + app_metadata: vec![0xFF, 0xFF, 0xFF, 0xFF].into(), // Invalid protobuf data |
| 279 | + data_body: vec![4, 5, 6].into(), |
| 280 | + }; |
| 281 | + |
| 282 | + let error_stream = stream::iter(vec![Ok(flight_data_with_invalid_metadata)]); |
| 283 | + let mut collecting_stream = MetricsCollectingStream::new(error_stream, metrics_collection); |
| 284 | + |
| 285 | + let result = collecting_stream.next().await.unwrap(); |
| 286 | + assert!(result.is_err()); |
| 287 | + if let Err(FlightError::ProtocolError(msg)) = result { |
| 288 | + assert!(msg.contains("failed to decode app_metadata")); |
| 289 | + } else { |
| 290 | + panic!("expected FlightError::ProtocolError with decode error"); |
| 291 | + } |
| 292 | + } |
| 293 | + |
| 294 | + #[tokio::test] |
| 295 | + async fn test_metrics_collecting_stream_error_propagation() { |
| 296 | + let metrics_collection = Arc::new(DashMap::new()); |
| 297 | + |
| 298 | + // Create a stream that emits an error - should be propagated through |
| 299 | + let stream_error = FlightError::ProtocolError("stream error from inner stream".to_string()); |
| 300 | + let error_stream = stream::iter(vec![Err(stream_error)]); |
| 301 | + let mut collecting_stream = MetricsCollectingStream::new(error_stream, metrics_collection); |
| 302 | + |
| 303 | + let result = collecting_stream.next().await.unwrap(); |
| 304 | + assert!(result.is_err()); |
| 305 | + if let Err(FlightError::ProtocolError(msg)) = result { |
| 306 | + assert!(msg.contains("stream error from inner stream")); |
| 307 | + } else { |
| 308 | + panic!("expected FlightError::ProtocolError with inner stream error"); |
| 309 | + } |
| 310 | + } |
| 311 | +} |
0 commit comments