From d892f232b8bb9e7541a6e77fc8f31714124bc5d0 Mon Sep 17 00:00:00 2001 From: Matthew Kim Date: Wed, 18 Mar 2026 11:01:55 -0400 Subject: [PATCH 1/8] feat: replace fixed MetricDataPoint fields with dynamic tag HashMap --- .../src/otlp/arrow_metrics.rs | 520 +++++++----------- .../quickwit-opentelemetry/src/otlp/mod.rs | 2 +- .../src/otlp/otel_metrics.rs | 408 ++++++++------ 3 files changed, 425 insertions(+), 505 deletions(-) diff --git a/quickwit/quickwit-opentelemetry/src/otlp/arrow_metrics.rs b/quickwit/quickwit-opentelemetry/src/otlp/arrow_metrics.rs index 1811a63a909..8a4b3142501 100644 --- a/quickwit/quickwit-opentelemetry/src/otlp/arrow_metrics.rs +++ b/quickwit/quickwit-opentelemetry/src/otlp/arrow_metrics.rs @@ -12,272 +12,139 @@ // See the License for the specific language governing permissions and // limitations under the License. -//! Arrow-based batch building for metrics with dictionary encoding. +//! Arrow-based batch building for metrics with dynamic schema discovery. //! //! This module provides Arrow RecordBatch construction with dictionary-encoded -//! string columns for efficient storage of metrics with low cardinality tags. +//! string columns for efficient storage of metrics with dynamic tag keys. +//! The schema is discovered at `finish()` time by scanning all accumulated +//! data points for the union of tag keys. +use std::collections::BTreeSet; use std::io::Cursor; use std::sync::Arc; use arrow::array::{ - ArrayBuilder, ArrayRef, Float64Builder, RecordBatch, StringBuilder, StringDictionaryBuilder, - UInt8Builder, UInt64Builder, + ArrayRef, Float64Builder, RecordBatch, StringDictionaryBuilder, UInt64Builder, UInt8Builder, }; -use arrow::datatypes::{DataType, Field, Fields, Int32Type, Schema as ArrowSchema}; +use arrow::datatypes::{DataType, Field, Int32Type, Schema as ArrowSchema}; use arrow::ipc::reader::StreamReader; use arrow::ipc::writer::StreamWriter; -use parquet::variant::{VariantArrayBuilder, VariantBuilderExt, VariantType}; use quickwit_proto::bytes::Bytes; use quickwit_proto::ingest::{DocBatchV2, DocFormat}; use quickwit_proto::types::DocUid; use super::otel_metrics::{MetricDataPoint, MetricType}; -/// Creates the Arrow schema for metrics with dictionary-encoded string columns. -/// -/// Dictionary encoding stores unique string values once and references them by -/// integer index, providing significant compression for low cardinality tag values. -pub fn metrics_arrow_schema() -> ArrowSchema { - ArrowSchema::new(vec![ - // Dictionary-encoded string columns for low cardinality fields - Field::new( - "metric_name", - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), - false, - ), - // MetricType enum stored as UInt8 (only ~5 possible values) - Field::new("metric_type", DataType::UInt8, false), - Field::new("metric_unit", DataType::Utf8, true), - // Measurement timestamp in seconds since Unix epoch. - Field::new("timestamp_secs", DataType::UInt64, false), - Field::new("start_timestamp_secs", DataType::UInt64, true), - Field::new("value", DataType::Float64, false), - // Dictionary-encoded tag columns (low cardinality expected) - Field::new( - "tag_service", - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), - true, - ), - Field::new( - "tag_env", - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), - true, - ), - Field::new( - "tag_datacenter", - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), - true, - ), - Field::new( - "tag_region", - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), - true, - ), - Field::new( - "tag_host", - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), - true, - ), - // VARIANT fields for semi-structured attributes - // VariantArrayBuilder produces BinaryView fields, not Binary - Field::new( - "attributes", - DataType::Struct(Fields::from(vec![ - Field::new("metadata", DataType::BinaryView, false), - Field::new("value", DataType::BinaryView, false), - ])), - true, - ) - .with_extension_type(VariantType), - // Service name (low cardinality) - Field::new( - "service_name", - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), - false, - ), - Field::new( - "resource_attributes", - DataType::Struct(Fields::from(vec![ - Field::new("metadata", DataType::BinaryView, false), - Field::new("value", DataType::BinaryView, false), - ])), - true, - ) - .with_extension_type(VariantType), - ]) -} - /// Builder for creating Arrow RecordBatch from MetricDataPoints. /// -/// Uses dictionary encoding for low cardinality string columns -/// (tags, service names, metric names) to achieve significant compression. -/// Uses VARIANT encoding for semi-structured attributes. +/// Accumulates data points and discovers the schema dynamically at `finish()` +/// time. Uses dictionary encoding for string columns (metric_name, all tags). pub struct ArrowMetricsBatchBuilder { - metric_name: StringDictionaryBuilder, - metric_type: UInt8Builder, - metric_unit: StringBuilder, - timestamp_secs: UInt64Builder, - start_timestamp_secs: UInt64Builder, - value: Float64Builder, - tag_service: StringDictionaryBuilder, - tag_env: StringDictionaryBuilder, - tag_datacenter: StringDictionaryBuilder, - tag_region: StringDictionaryBuilder, - tag_host: StringDictionaryBuilder, - attributes: VariantArrayBuilder, - service_name: StringDictionaryBuilder, - resource_attributes: VariantArrayBuilder, + data_points: Vec, } impl ArrowMetricsBatchBuilder { /// Creates a new builder with pre-allocated capacity. pub fn with_capacity(capacity: usize) -> Self { Self { - metric_name: StringDictionaryBuilder::new(), - metric_type: UInt8Builder::with_capacity(capacity), - metric_unit: StringBuilder::with_capacity(capacity, capacity * 8), - timestamp_secs: UInt64Builder::with_capacity(capacity), - start_timestamp_secs: UInt64Builder::with_capacity(capacity), - value: Float64Builder::with_capacity(capacity), - tag_service: StringDictionaryBuilder::new(), - tag_env: StringDictionaryBuilder::new(), - tag_datacenter: StringDictionaryBuilder::new(), - tag_region: StringDictionaryBuilder::new(), - tag_host: StringDictionaryBuilder::new(), - attributes: VariantArrayBuilder::new(capacity), - service_name: StringDictionaryBuilder::new(), - resource_attributes: VariantArrayBuilder::new(capacity), + data_points: Vec::with_capacity(capacity), } } /// Appends a MetricDataPoint to the batch. - pub fn append(&mut self, data_point: &MetricDataPoint) { - self.metric_name.append_value(&data_point.metric_name); - self.metric_type.append_value(data_point.metric_type as u8); + pub fn append(&mut self, data_point: MetricDataPoint) { + self.data_points.push(data_point); + } - match &data_point.metric_unit { - Some(unit) => self.metric_unit.append_value(unit), - None => self.metric_unit.append_null(), + /// Finalizes and returns the RecordBatch. + /// + /// Performs two passes: + /// 1. Schema discovery: scans all data points to collect the union of tag keys. + /// 2. Array building: creates per-column builders and populates them. + pub fn finish(self) -> RecordBatch { + let num_rows = self.data_points.len(); + + // Pass 1: discover all tag keys across all data points. + let mut tag_keys: BTreeSet<&str> = BTreeSet::new(); + for dp in &self.data_points { + for key in dp.tags.keys() { + tag_keys.insert(key.as_str()); + } } + let sorted_tag_keys: Vec<&str> = tag_keys.into_iter().collect(); - self.timestamp_secs.append_value(data_point.timestamp_secs); - match data_point.start_timestamp_secs { - Some(ts) => self.start_timestamp_secs.append_value(ts), - None => self.start_timestamp_secs.append_null(), + // Build the Arrow schema dynamically + let mut fields = Vec::with_capacity(4 + sorted_tag_keys.len()); + fields.push(Field::new( + "metric_name", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + false, + )); + fields.push(Field::new("metric_type", DataType::UInt8, false)); + fields.push(Field::new("timestamp_secs", DataType::UInt64, false)); + fields.push(Field::new("value", DataType::Float64, false)); + + for &tag_key in &sorted_tag_keys { + fields.push(Field::new( + tag_key, + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + true, + )); } - self.value.append_value(data_point.value); - - append_optional_dict(&mut self.tag_service, &data_point.tag_service); - append_optional_dict(&mut self.tag_env, &data_point.tag_env); - append_optional_dict(&mut self.tag_datacenter, &data_point.tag_datacenter); - append_optional_dict(&mut self.tag_region, &data_point.tag_region); - append_optional_dict(&mut self.tag_host, &data_point.tag_host); - - if data_point.attributes.is_empty() { - self.attributes.append_null(); - } else { - append_variant_object(&mut self.attributes, &data_point.attributes); + + let schema = Arc::new(ArrowSchema::new(fields)); + + // Pass 2: build arrays + let mut metric_name_builder: StringDictionaryBuilder = + StringDictionaryBuilder::new(); + let mut metric_type_builder = UInt8Builder::with_capacity(num_rows); + let mut timestamp_secs_builder = UInt64Builder::with_capacity(num_rows); + let mut value_builder = Float64Builder::with_capacity(num_rows); + + let mut tag_builders: Vec> = sorted_tag_keys + .iter() + .map(|_| StringDictionaryBuilder::new()) + .collect(); + + for dp in &self.data_points { + metric_name_builder.append_value(&dp.metric_name); + metric_type_builder.append_value(dp.metric_type as u8); + timestamp_secs_builder.append_value(dp.timestamp_secs); + value_builder.append_value(dp.value); + + for (tag_idx, tag_key) in sorted_tag_keys.iter().enumerate() { + match dp.tags.get(*tag_key) { + Some(tag_val) => tag_builders[tag_idx].append_value(tag_val), + None => tag_builders[tag_idx].append_null(), + } + } } - self.service_name.append_value(&data_point.service_name); + let mut arrays: Vec = Vec::with_capacity(4 + sorted_tag_keys.len()); + arrays.push(Arc::new(metric_name_builder.finish())); + arrays.push(Arc::new(metric_type_builder.finish())); + arrays.push(Arc::new(timestamp_secs_builder.finish())); + arrays.push(Arc::new(value_builder.finish())); - if data_point.resource_attributes.is_empty() { - self.resource_attributes.append_null(); - } else { - append_variant_object( - &mut self.resource_attributes, - &data_point.resource_attributes, - ); + for tag_builder in &mut tag_builders { + arrays.push(Arc::new(tag_builder.finish())); } - } - /// Finalizes and returns the RecordBatch. - pub fn finish(mut self) -> RecordBatch { - // Build variant arrays and convert to ArrayRef - let attributes_array = self.attributes.build(); - let resource_attributes_array = self.resource_attributes.build(); - - let arrays: Vec = vec![ - Arc::new(self.metric_name.finish()), - Arc::new(self.metric_type.finish()), - Arc::new(self.metric_unit.finish()), - Arc::new(self.timestamp_secs.finish()), - Arc::new(self.start_timestamp_secs.finish()), - Arc::new(self.value.finish()), - Arc::new(self.tag_service.finish()), - Arc::new(self.tag_env.finish()), - Arc::new(self.tag_datacenter.finish()), - Arc::new(self.tag_region.finish()), - Arc::new(self.tag_host.finish()), - ArrayRef::from(attributes_array), - Arc::new(self.service_name.finish()), - ArrayRef::from(resource_attributes_array), - ]; - - RecordBatch::try_new(Arc::new(metrics_arrow_schema()), arrays) + RecordBatch::try_new(schema, arrays) .expect("record batch should match Arrow schema") } /// Returns the number of rows appended so far. pub fn len(&self) -> usize { - self.timestamp_secs.len() + self.data_points.len() } /// Returns true if no rows have been appended. pub fn is_empty(&self) -> bool { - self.len() == 0 + self.data_points.is_empty() } } -/// Helper to append optional string values to dictionary builder. -fn append_optional_dict(builder: &mut StringDictionaryBuilder, value: &Option) { - match value { - Some(s) => builder.append_value(s), - None => builder.append_null(), - } -} - -/// Helper to append a HashMap as a VARIANT object to the builder. -fn append_variant_object( - builder: &mut VariantArrayBuilder, - map: &std::collections::HashMap, -) { - // Use a macro-like approach with fold to build the object - // We need to chain with_field calls which consume and return the builder - let obj_builder = builder.new_object(); - - // Build object by folding over the map entries - let final_builder = map.iter().fold(obj_builder, |b, (key, value)| { - match value { - serde_json::Value::Null => b.with_field(key.as_str(), ()), - serde_json::Value::Bool(v) => b.with_field(key.as_str(), *v), - serde_json::Value::Number(n) => { - if let Some(i) = n.as_i64() { - b.with_field(key.as_str(), i) - } else if let Some(f) = n.as_f64() { - b.with_field(key.as_str(), f) - } else { - b.with_field(key.as_str(), ()) - } - } - serde_json::Value::String(s) => b.with_field(key.as_str(), s.as_str()), - serde_json::Value::Array(arr) => { - // For arrays, serialize to JSON string as fallback - let json_str = serde_json::to_string(arr).unwrap_or_default(); - b.with_field(key.as_str(), json_str.as_str()) - } - serde_json::Value::Object(obj) => { - // For nested objects, serialize to JSON string as fallback - let json_str = serde_json::to_string(obj).unwrap_or_default(); - b.with_field(key.as_str(), json_str.as_str()) - } - } - }); - - final_builder.finish(); -} - /// Error type for Arrow IPC operations. #[derive(Debug, thiserror::Error)] pub enum ArrowIpcError { @@ -462,32 +329,26 @@ impl ArrowDocBatchV2Builder { mod tests { use std::collections::HashMap; - use serde_json::Value as JsonValue; - use super::*; fn make_test_data_point() -> MetricDataPoint { + let mut tags = HashMap::new(); + tags.insert("service".to_string(), "api".to_string()); + tags.insert("env".to_string(), "prod".to_string()); + tags.insert("datacenter".to_string(), "us-east-1a".to_string()); + tags.insert("region".to_string(), "us-east-1".to_string()); + tags.insert("host".to_string(), "server-001".to_string()); + tags.insert("endpoint".to_string(), "/health".to_string()); + tags.insert("metric_unit".to_string(), "%".to_string()); + tags.insert("start_timestamp_secs".to_string(), "1704067190".to_string()); + tags.insert("service_name".to_string(), "api-service".to_string()); + MetricDataPoint { metric_name: "cpu.usage".to_string(), metric_type: MetricType::Gauge, - metric_unit: Some("%".to_string()), timestamp_secs: 1704067200, - start_timestamp_secs: Some(1704067190), value: 85.5, - tag_service: Some("api".to_string()), - tag_env: Some("prod".to_string()), - tag_datacenter: Some("us-east-1a".to_string()), - tag_region: Some("us-east-1".to_string()), - tag_host: Some("server-001".to_string()), - attributes: HashMap::from([( - "endpoint".to_string(), - JsonValue::String("/health".to_string()), - )]), - service_name: "api-service".to_string(), - resource_attributes: HashMap::from([( - "k8s.pod".to_string(), - JsonValue::String("pod-123".to_string()), - )]), + tags, } } @@ -495,38 +356,36 @@ mod tests { fn test_arrow_batch_builder_single_row() { let dp = make_test_data_point(); let mut builder = ArrowMetricsBatchBuilder::with_capacity(1); - builder.append(&dp); + builder.append(dp); assert_eq!(builder.len(), 1); assert!(!builder.is_empty()); let batch = builder.finish(); assert_eq!(batch.num_rows(), 1); - assert_eq!(batch.num_columns(), 14); + // 4 fixed columns + 9 tag columns + assert_eq!(batch.num_columns(), 13); } #[test] fn test_arrow_batch_builder_multiple_rows() { let mut builder = ArrowMetricsBatchBuilder::with_capacity(100); - for i in 0..100 { + for idx in 0..100 { + let mut tags = HashMap::new(); + tags.insert("service".to_string(), format!("service-{}", idx % 10)); + tags.insert("env".to_string(), "prod".to_string()); + tags.insert("host".to_string(), format!("host-{}", idx % 5)); + tags.insert("service_name".to_string(), "test-service".to_string()); + let dp = MetricDataPoint { metric_name: "test.metric".to_string(), metric_type: MetricType::Gauge, - metric_unit: None, - timestamp_secs: 1704067200 + i as u64, - start_timestamp_secs: None, - value: i as f64 * 0.1, - tag_service: Some(format!("service-{}", i % 10)), // 10 unique values - tag_env: Some("prod".to_string()), // 1 unique value - tag_datacenter: None, - tag_region: None, - tag_host: Some(format!("host-{}", i % 5)), // 5 unique values - attributes: HashMap::new(), - service_name: "test-service".to_string(), - resource_attributes: HashMap::new(), + timestamp_secs: 1704067200 + idx as u64, + value: idx as f64 * 0.1, + tags, }; - builder.append(&dp); + builder.append(dp); } assert_eq!(builder.len(), 100); @@ -539,89 +398,108 @@ mod tests { let mut builder = ArrowMetricsBatchBuilder::with_capacity(1000); // Create 1000 data points with only 10 unique service values - for i in 0..1000 { + for idx in 0..1000 { + let mut tags = HashMap::new(); + tags.insert("service".to_string(), format!("service-{}", idx % 10)); + tags.insert("env".to_string(), "prod".to_string()); + tags.insert("datacenter".to_string(), format!("dc-{}", idx % 4)); + tags.insert("service_name".to_string(), format!("svc-{}", idx % 5)); + let dp = MetricDataPoint { metric_name: "test.metric".to_string(), metric_type: MetricType::Gauge, - metric_unit: None, - timestamp_secs: 1704067200 + i as u64, - start_timestamp_secs: None, - value: i as f64, - tag_service: Some(format!("service-{}", i % 10)), - tag_env: Some("prod".to_string()), - tag_datacenter: Some(format!("dc-{}", i % 4)), - tag_region: None, - tag_host: None, - attributes: HashMap::new(), - service_name: format!("svc-{}", i % 5), - resource_attributes: HashMap::new(), + timestamp_secs: 1704067200 + idx as u64, + value: idx as f64, + tags, }; - builder.append(&dp); + builder.append(dp); } let batch = builder.finish(); assert_eq!(batch.num_rows(), 1000); // Verify the batch was created successfully with dictionary encoding - // The dictionary arrays should have far fewer unique values than rows let schema = batch.schema(); - // Check that tag_service uses dictionary encoding - let tag_service_field = schema.field_with_name("tag_service").unwrap(); + // Check that the service tag uses dictionary encoding + let service_field = schema.field_with_name("service").unwrap(); assert!(matches!( - tag_service_field.data_type(), + service_field.data_type(), DataType::Dictionary(_, _) )); } #[test] fn test_null_handling() { + let mut tags = HashMap::new(); + tags.insert("service_name".to_string(), "unknown".to_string()); + let dp = MetricDataPoint { metric_name: "minimal.metric".to_string(), metric_type: MetricType::Gauge, - metric_unit: None, // null timestamp_secs: 1704067200, - start_timestamp_secs: None, // null value: 0.0, - tag_service: None, // null - tag_env: None, // null - tag_datacenter: None, - tag_region: None, - tag_host: None, - attributes: HashMap::new(), // empty -> null - service_name: "unknown".to_string(), - resource_attributes: HashMap::new(), // empty -> null + tags, }; let mut builder = ArrowMetricsBatchBuilder::with_capacity(1); - builder.append(&dp); + builder.append(dp); let batch = builder.finish(); assert_eq!(batch.num_rows(), 1); - // The batch should handle nulls correctly + // 4 fixed columns + 1 tag column (service_name) + assert_eq!(batch.num_columns(), 5); } #[test] - fn test_schema_field_count() { - let schema = metrics_arrow_schema(); - assert_eq!(schema.fields().len(), 14); + fn test_dynamic_schema_discovery() { + let mut builder = ArrowMetricsBatchBuilder::with_capacity(2); - // Verify field names + // First data point has tags: env, host + let mut tags1 = HashMap::new(); + tags1.insert("env".to_string(), "prod".to_string()); + tags1.insert("host".to_string(), "server-1".to_string()); + + builder.append(MetricDataPoint { + metric_name: "metric.a".to_string(), + metric_type: MetricType::Gauge, + timestamp_secs: 1704067200, + value: 1.0, + tags: tags1, + }); + + // Second data point has tags: env, region (different set) + let mut tags2 = HashMap::new(); + tags2.insert("env".to_string(), "staging".to_string()); + tags2.insert("region".to_string(), "us-west".to_string()); + + builder.append(MetricDataPoint { + metric_name: "metric.b".to_string(), + metric_type: MetricType::Sum, + timestamp_secs: 1704067201, + value: 2.0, + tags: tags2, + }); + + let batch = builder.finish(); + assert_eq!(batch.num_rows(), 2); + // 4 fixed + 3 tag columns (env, host, region) - sorted alphabetically + assert_eq!(batch.num_columns(), 7); + + let schema = batch.schema(); let field_names: Vec<&str> = schema.fields().iter().map(|f| f.name().as_str()).collect(); - assert!(field_names.contains(&"metric_name")); - assert!(field_names.contains(&"metric_type")); - assert!(field_names.contains(&"metric_unit")); - assert!(field_names.contains(&"timestamp_secs")); - assert!(field_names.contains(&"start_timestamp_secs")); - assert!(field_names.contains(&"value")); - assert!(field_names.contains(&"tag_service")); - assert!(field_names.contains(&"tag_env")); - assert!(field_names.contains(&"tag_datacenter")); - assert!(field_names.contains(&"tag_region")); - assert!(field_names.contains(&"tag_host")); - assert!(field_names.contains(&"attributes")); - assert!(field_names.contains(&"service_name")); - assert!(field_names.contains(&"resource_attributes")); + assert_eq!( + field_names, + vec![ + "metric_name", + "metric_type", + "timestamp_secs", + "value", + "env", + "host", + "region", + ] + ); } #[test] @@ -638,24 +516,20 @@ mod tests { fn test_ipc_round_trip() { // Build a RecordBatch let mut builder = ArrowMetricsBatchBuilder::with_capacity(10); - for i in 0..10 { + for idx in 0..10 { + let mut tags = HashMap::new(); + tags.insert("service".to_string(), format!("service-{}", idx % 3)); + tags.insert("env".to_string(), "prod".to_string()); + tags.insert("service_name".to_string(), "test-service".to_string()); + let dp = MetricDataPoint { metric_name: "test.metric".to_string(), metric_type: MetricType::Gauge, - metric_unit: None, - timestamp_secs: 1704067200 + i as u64, - start_timestamp_secs: None, - value: i as f64 * 0.1, - tag_service: Some(format!("service-{}", i % 3)), - tag_env: Some("prod".to_string()), - tag_datacenter: None, - tag_region: None, - tag_host: None, - attributes: HashMap::new(), - service_name: "test-service".to_string(), - resource_attributes: HashMap::new(), + timestamp_secs: 1704067200 + idx as u64, + value: idx as f64 * 0.1, + tags, }; - builder.append(&dp); + builder.append(dp); } let original_batch = builder.finish(); @@ -682,24 +556,18 @@ mod tests { let mut doc_uid_generator = DocUidGenerator::default(); let mut doc_uids = Vec::new(); - for i in 0..5 { + for idx in 0..5 { + let mut tags = HashMap::new(); + tags.insert("service_name".to_string(), "test".to_string()); + let dp = MetricDataPoint { metric_name: "test.metric".to_string(), metric_type: MetricType::Gauge, - metric_unit: None, - timestamp_secs: 1704067200 + i as u64, - start_timestamp_secs: None, - value: i as f64, - tag_service: None, - tag_env: None, - tag_datacenter: None, - tag_region: None, - tag_host: None, - attributes: HashMap::new(), - service_name: "test".to_string(), - resource_attributes: HashMap::new(), + timestamp_secs: 1704067200 + idx as u64, + value: idx as f64, + tags, }; - builder.append(&dp); + builder.append(dp); doc_uids.push(doc_uid_generator.next_doc_uid()); } let batch = builder.finish(); diff --git a/quickwit/quickwit-opentelemetry/src/otlp/mod.rs b/quickwit/quickwit-opentelemetry/src/otlp/mod.rs index 67abd6c1d68..9927c4fd800 100644 --- a/quickwit/quickwit-opentelemetry/src/otlp/mod.rs +++ b/quickwit/quickwit-opentelemetry/src/otlp/mod.rs @@ -37,7 +37,7 @@ mod traces; pub use arrow_metrics::{ ArrowDocBatchV2Builder, ArrowIpcError, ArrowMetricsBatchBuilder, ipc_to_json_values, - ipc_to_record_batch, metrics_arrow_schema, record_batch_to_ipc, + ipc_to_record_batch, record_batch_to_ipc, }; pub use logs::{ JsonLogIterator, OTEL_LOGS_INDEX_ID, OtlpGrpcLogsService, OtlpLogsError, parse_otlp_logs_json, diff --git a/quickwit/quickwit-opentelemetry/src/otlp/otel_metrics.rs b/quickwit/quickwit-opentelemetry/src/otlp/otel_metrics.rs index bcb67c5fadb..1a19e5fea53 100644 --- a/quickwit/quickwit-opentelemetry/src/otlp/otel_metrics.rs +++ b/quickwit/quickwit-opentelemetry/src/otlp/otel_metrics.rs @@ -134,59 +134,14 @@ impl From for tonic::Status { } } -/// Represents a single metric data point document +/// Represents a single metric data point document. #[derive(Debug, Clone)] pub struct MetricDataPoint { - // Metric identity pub metric_name: String, pub metric_type: MetricType, - pub metric_unit: Option, - - // Timestamps (seconds granularity) pub timestamp_secs: u64, - pub start_timestamp_secs: Option, - - // Value (f64 only) pub value: f64, - - // Explicit tag columns - pub tag_service: Option, - pub tag_env: Option, - pub tag_datacenter: Option, - pub tag_region: Option, - pub tag_host: Option, - - // Dynamic tags (remaining attributes) - pub attributes: HashMap, - - // Resource metadata - pub service_name: String, - pub resource_attributes: HashMap, -} - -struct ExplicitTags { - service: Option, - env: Option, - datacenter: Option, - region: Option, - host: Option, -} - -fn extract_string_tag(attributes: &mut HashMap, key: &str) -> Option { - attributes.remove(key).and_then(|v| match v { - JsonValue::String(s) => Some(s), - _ => None, - }) -} - -fn extract_explicit_tags(attributes: &mut HashMap) -> ExplicitTags { - ExplicitTags { - service: extract_string_tag(attributes, "service"), - env: extract_string_tag(attributes, "env"), - datacenter: extract_string_tag(attributes, "datacenter"), - region: extract_string_tag(attributes, "region"), - host: extract_string_tag(attributes, "host"), - } + pub tags: HashMap, } /// Convert nanoseconds to seconds @@ -194,6 +149,16 @@ fn nanos_to_secs(nanos: u64) -> u64 { nanos / 1_000_000_000 } +/// Convert a `serde_json::Value` to a plain `String`. +fn json_value_to_string(value: JsonValue) -> String { + match value { + JsonValue::String(s) => s, + JsonValue::Number(n) => n.to_string(), + JsonValue::Bool(b) => b.to_string(), + other => serde_json::to_string(&other).unwrap_or_default(), + } +} + struct ParsedMetrics { doc_batch: DocBatchV2, num_data_points: u64, @@ -243,7 +208,7 @@ impl OtlpGrpcMetricsService { Status::internal("failed to parse metric records") })??; - if num_data_points == num_parse_errors { + if num_data_points > 0 && num_data_points == num_parse_errors { return Err(tonic::Status::internal(error_message)); } @@ -274,15 +239,19 @@ impl OtlpGrpcMetricsService { request: ExportMetricsServiceRequest, parent_span: RuntimeSpan, ) -> tonic::Result { - let data_points = parse_otlp_metrics(request)?; - let num_data_points = data_points.len() as u64; - - // Build Arrow RecordBatch from data points - let mut arrow_builder = ArrowMetricsBatchBuilder::with_capacity(num_data_points as usize); + let ParseOtlpResult { + data_points, + num_rejected, + } = parse_otlp_metrics(request); + let num_data_points = data_points.len() as u64 + num_rejected; + + // Build Arrow RecordBatch from valid data points + let mut arrow_builder = + ArrowMetricsBatchBuilder::with_capacity(data_points.len()); let mut doc_uid_generator = DocUidGenerator::default(); - let mut doc_uids = Vec::with_capacity(num_data_points as usize); + let mut doc_uids = Vec::with_capacity(data_points.len()); - for data_point in &data_points { + for data_point in data_points { arrow_builder.append(data_point); doc_uids.push(doc_uid_generator.next_doc_uid()); } @@ -300,13 +269,22 @@ impl OtlpGrpcMetricsService { let current_span = RuntimeSpan::current(); current_span.record("num_data_points", num_data_points); current_span.record("num_bytes", doc_batch.num_bytes()); - current_span.record("num_parse_errors", 0u64); + current_span.record("num_parse_errors", num_rejected); + + let error_message = if num_rejected > 0 { + format!( + "{num_rejected} data point(s) rejected (unsupported temporality or missing \ + required fields)" + ) + } else { + String::new() + }; let parsed_metrics = ParsedMetrics { doc_batch, num_data_points, - num_parse_errors: 0, - error_message: String::new(), + num_parse_errors: num_rejected, + error_message, }; Ok(parsed_metrics) } @@ -381,16 +359,19 @@ impl MetricsService for OtlpGrpcMetricsService { } } -fn parse_otlp_metrics( - request: ExportMetricsServiceRequest, -) -> Result, OtlpMetricsError> { +struct ParseOtlpResult { + data_points: Vec, + num_rejected: u64, +} + +fn parse_otlp_metrics(request: ExportMetricsServiceRequest) -> ParseOtlpResult { let mut data_points = Vec::new(); + let mut num_rejected: u64 = 0; for resource_metrics in request.resource_metrics { let mut resource_attributes = extract_attributes( resource_metrics .resource - .clone() .map(|rsrc| rsrc.attributes) .unwrap_or_default(), ); @@ -401,25 +382,23 @@ fn parse_otlp_metrics( for scope_metrics in resource_metrics.scope_metrics { for metric in scope_metrics.metrics { - parse_metric( - &metric, - &service_name, - &resource_attributes, - &mut data_points, - )?; + parse_metric(&metric, &service_name, &mut data_points, &mut num_rejected); } } } - Ok(data_points) + ParseOtlpResult { + data_points, + num_rejected, + } } fn parse_metric( metric: &Metric, service_name: &str, - resource_attributes: &HashMap, data_points: &mut Vec, -) -> Result<(), OtlpMetricsError> { + num_rejected: &mut u64, +) { let metric_name = metric.name.clone(); let metric_unit = if metric.unit.is_empty() { None @@ -430,36 +409,47 @@ fn parse_metric( match &metric.data { Some(metric::Data::Gauge(gauge)) => { for dp in &gauge.data_points { - let data_point = create_number_data_point( + match create_number_data_point( &metric_name, MetricType::Gauge, &metric_unit, dp, service_name, - resource_attributes, - )?; - data_points.push(data_point); + ) { + Ok(Some(data_point)) => data_points.push(data_point), + Ok(None) => *num_rejected += 1, + Err(err) => { + warn!(error = %err, metric_name, "skipping invalid gauge data point"); + *num_rejected += 1; + } + } } } Some(metric::Data::Sum(sum)) => { - // Only support DELTA temporality if sum.aggregation_temporality == AggregationTemporality::Cumulative as i32 { - return Err(OtlpMetricsError::InvalidArgument( - "cumulative aggregation temporality is not supported, only delta is supported" - .to_string(), - )); + warn!( + metric_name, + "skipping sum metric with cumulative temporality (only delta is supported)" + ); + *num_rejected += sum.data_points.len() as u64; + return; } for dp in &sum.data_points { - let data_point = create_number_data_point( + match create_number_data_point( &metric_name, MetricType::Sum, &metric_unit, dp, service_name, - resource_attributes, - )?; - data_points.push(data_point); + ) { + Ok(Some(data_point)) => data_points.push(data_point), + Ok(None) => *num_rejected += 1, + Err(err) => { + warn!(error = %err, metric_name, "skipping invalid sum data point"); + *num_rejected += 1; + } + } } } Some(metric::Data::Histogram(_)) => { @@ -475,8 +465,6 @@ fn parse_metric( warn!("metric has no data, skipping"); } } - - Ok(()) } fn create_number_data_point( @@ -485,8 +473,20 @@ fn create_number_data_point( metric_unit: &Option, dp: &NumberDataPoint, service_name: &str, - resource_attributes: &HashMap, -) -> Result { +) -> Result, OtlpMetricsError> { + // Convert timestamps to seconds + let timestamp_secs = nanos_to_secs(dp.time_unix_nano); + + // Validate: skip data points with empty metric_name or zero timestamp + if metric_name.is_empty() { + warn!("skipping data point with empty metric_name"); + return Ok(None); + } + if timestamp_secs == 0 { + warn!("skipping data point with zero timestamp_secs"); + return Ok(None); + } + // Extract value as f64 let value = match &dp.value { Some( @@ -500,34 +500,39 @@ fn create_number_data_point( None => 0.0, }; - // Extract attributes and explicit tags - let mut attributes = extract_attributes(dp.attributes.clone()); - let explicit_tags = extract_explicit_tags(&mut attributes); + // Extract attributes and convert all values to strings for tags + let attributes = extract_attributes(dp.attributes.clone()); + let mut tags = HashMap::with_capacity(attributes.len() + 3); - // Convert timestamps to seconds - let timestamp_secs = nanos_to_secs(dp.time_unix_nano); - let start_timestamp_secs = if dp.start_time_unix_nano != 0 { - Some(nanos_to_secs(dp.start_time_unix_nano)) - } else { - None - }; + for (key, json_val) in attributes { + tags.insert(key, json_value_to_string(json_val)); + } + + // Add metric_unit and start_timestamp_secs using or_insert_with so a + // data-point attribute with the same name is not silently overwritten. + if let Some(unit) = metric_unit { + tags.entry("metric_unit".to_string()) + .or_insert_with(|| unit.clone()); + } - Ok(MetricDataPoint { + if dp.start_time_unix_nano != 0 { + let start_ts = nanos_to_secs(dp.start_time_unix_nano); + tags.entry("start_timestamp_secs".to_string()) + .or_insert_with(|| start_ts.to_string()); + } + + // Fall back to the resource-level service.name if no data-point-level + // "service" tag was set. Data-point attributes take precedence. + tags.entry("service".to_string()) + .or_insert_with(|| service_name.to_string()); + + Ok(Some(MetricDataPoint { metric_name: metric_name.to_string(), metric_type, - metric_unit: metric_unit.clone(), timestamp_secs, - start_timestamp_secs, value, - tag_service: explicit_tags.service, - tag_env: explicit_tags.env, - tag_datacenter: explicit_tags.datacenter, - tag_region: explicit_tags.region, - tag_host: explicit_tags.host, - attributes, - service_name: service_name.to_string(), - resource_attributes: resource_attributes.clone(), - }) + tags, + })) } #[cfg(test)] @@ -562,42 +567,6 @@ mod tests { assert_eq!(nanos_to_secs(2_000_000_000), 2); } - #[test] - fn test_extract_explicit_tags() { - let mut attributes = HashMap::from([ - ("service".to_string(), JsonValue::String("api".to_string())), - ("env".to_string(), JsonValue::String("prod".to_string())), - ( - "datacenter".to_string(), - JsonValue::String("us-east".to_string()), - ), - ( - "region".to_string(), - JsonValue::String("us-east-1".to_string()), - ), - ( - "host".to_string(), - JsonValue::String("server-1".to_string()), - ), - ( - "custom_tag".to_string(), - JsonValue::String("custom_value".to_string()), - ), - ]); - - let explicit_tags = extract_explicit_tags(&mut attributes); - - assert_eq!(explicit_tags.service, Some("api".to_string())); - assert_eq!(explicit_tags.env, Some("prod".to_string())); - assert_eq!(explicit_tags.datacenter, Some("us-east".to_string())); - assert_eq!(explicit_tags.region, Some("us-east-1".to_string())); - assert_eq!(explicit_tags.host, Some("server-1".to_string())); - - // custom_tag should remain in attributes - assert_eq!(attributes.len(), 1); - assert!(attributes.contains_key("custom_tag")); - } - fn make_test_gauge_request() -> ExportMetricsServiceRequest { use quickwit_proto::opentelemetry::proto::common::v1::{AnyValue, KeyValue, any_value}; use quickwit_proto::opentelemetry::proto::metrics::v1::{ @@ -747,35 +716,44 @@ mod tests { #[test] fn test_parse_gauge_metrics() { let request = make_test_gauge_request(); - let data_points = parse_otlp_metrics(request).unwrap(); + let data_points = parse_otlp_metrics(request).data_points; assert_eq!(data_points.len(), 1); let dp = &data_points[0]; assert_eq!(dp.metric_name, "cpu.usage"); assert_eq!(dp.metric_type, MetricType::Gauge); - assert_eq!(dp.metric_unit, Some("%".to_string())); + assert_eq!(dp.tags.get("metric_unit").map(|s| s.as_str()), Some("%")); assert_eq!(dp.timestamp_secs, 2); - assert_eq!(dp.start_timestamp_secs, Some(1)); + assert_eq!( + dp.tags.get("start_timestamp_secs").map(|s| s.as_str()), + Some("1") + ); assert_eq!(dp.value, 85.5); - assert_eq!(dp.tag_service, Some("api".to_string())); - assert_eq!(dp.tag_env, Some("prod".to_string())); - assert_eq!(dp.service_name, "test-service"); + // Data-point attribute "service" takes precedence over resource-level service.name. + assert_eq!(dp.tags.get("service").map(|s| s.as_str()), Some("api")); + assert_eq!(dp.tags.get("env").map(|s| s.as_str()), Some("prod")); } #[test] fn test_parse_sum_delta_metrics() { let request = make_test_sum_delta_request(); - let data_points = parse_otlp_metrics(request).unwrap(); + let data_points = parse_otlp_metrics(request).data_points; assert_eq!(data_points.len(), 1); let dp = &data_points[0]; assert_eq!(dp.metric_name, "http.requests"); assert_eq!(dp.metric_type, MetricType::Sum); - assert_eq!(dp.metric_unit, Some("1".to_string())); + assert_eq!(dp.tags.get("metric_unit").map(|s| s.as_str()), Some("1")); assert_eq!(dp.timestamp_secs, 2); assert_eq!(dp.value, 100.0); // int converted to f64 - assert_eq!(dp.tag_host, Some("server-1".to_string())); - assert_eq!(dp.service_name, "counter-service"); + assert_eq!( + dp.tags.get("host").map(|s| s.as_str()), + Some("server-1") + ); + assert_eq!( + dp.tags.get("service").map(|s| s.as_str()), + Some("counter-service") + ); } #[test] @@ -783,11 +761,10 @@ mod tests { let request = make_test_sum_cumulative_request(); let result = parse_otlp_metrics(request); - assert!(result.is_err()); - match result.unwrap_err() { - OtlpMetricsError::InvalidArgument(_) => {} - err => panic!("unexpected error type: {:?}", err), - } + // Cumulative sums are skipped (not a hard error) so other metrics in the same + // request can still be processed. The rejected count is incremented instead. + assert_eq!(result.data_points.len(), 0); + assert_eq!(result.num_rejected, 1); } /// Test parsing metrics with various attribute types @@ -860,19 +837,17 @@ mod tests { }], }; - let data_points = parse_otlp_metrics(request).unwrap(); + let data_points = parse_otlp_metrics(request).data_points; assert_eq!(data_points.len(), 1); let dp = &data_points[0]; - assert_eq!(dp.service_name, "test"); - - // Verify resource attributes contain the non-service.name attributes - assert!(dp.resource_attributes.contains_key("int_attr")); - assert!(dp.resource_attributes.contains_key("bool_attr")); - assert!(dp.resource_attributes.contains_key("double_attr")); + assert_eq!(dp.tags.get("service").map(|s| s.as_str()), Some("test")); - // Verify data point attributes - assert!(dp.attributes.contains_key("string_tag")); + // Verify data point attributes are in tags as strings + assert_eq!( + dp.tags.get("string_tag").map(|s| s.as_str()), + Some("value") + ); } /// Test metrics with empty and missing values @@ -908,17 +883,94 @@ mod tests { }], }; - let data_points = parse_otlp_metrics(request).unwrap(); + let data_points = parse_otlp_metrics(request).data_points; assert_eq!(data_points.len(), 1); let dp = &data_points[0]; assert_eq!(dp.metric_name, "minimal.metric"); - assert_eq!(dp.service_name, "unknown_service"); // Default value - assert!(dp.metric_unit.is_none()); - assert!(dp.start_timestamp_secs.is_none()); - assert!(dp.tag_service.is_none()); - assert!(dp.tag_env.is_none()); - assert!(dp.attributes.is_empty()); - assert!(dp.resource_attributes.is_empty()); + assert_eq!( + dp.tags.get("service").map(|s| s.as_str()), + Some("unknown_service") + ); + // No metric_unit tag when unit is empty + assert!(dp.tags.get("metric_unit").is_none()); + // No start_timestamp_secs tag when start time is 0 + assert!(dp.tags.get("start_timestamp_secs").is_none()); + // Only "service" should be in tags (no attributes, no unit, no start time) + assert_eq!(dp.tags.len(), 1); + } + + /// Test that data points with empty metric_name are skipped + #[test] + fn test_skip_empty_metric_name() { + use quickwit_proto::opentelemetry::proto::metrics::v1::{ + Gauge, ResourceMetrics, ScopeMetrics, number_data_point, + }; + + let request = ExportMetricsServiceRequest { + resource_metrics: vec![ResourceMetrics { + resource: None, + scope_metrics: vec![ScopeMetrics { + scope: None, + metrics: vec![Metric { + name: String::new(), // Empty name + description: String::new(), + unit: String::new(), + data: Some(metric::Data::Gauge(Gauge { + data_points: vec![NumberDataPoint { + attributes: Vec::new(), + start_time_unix_nano: 0, + time_unix_nano: 1_000_000_000, + exemplars: Vec::new(), + flags: 0, + value: Some(number_data_point::Value::AsDouble(1.0)), + }], + })), + }], + schema_url: String::new(), + }], + schema_url: String::new(), + }], + }; + + let data_points = parse_otlp_metrics(request).data_points; + assert_eq!(data_points.len(), 0); + } + + /// Test that data points with zero timestamp are skipped + #[test] + fn test_skip_zero_timestamp() { + use quickwit_proto::opentelemetry::proto::metrics::v1::{ + Gauge, ResourceMetrics, ScopeMetrics, number_data_point, + }; + + let request = ExportMetricsServiceRequest { + resource_metrics: vec![ResourceMetrics { + resource: None, + scope_metrics: vec![ScopeMetrics { + scope: None, + metrics: vec![Metric { + name: "test.metric".to_string(), + description: String::new(), + unit: String::new(), + data: Some(metric::Data::Gauge(Gauge { + data_points: vec![NumberDataPoint { + attributes: Vec::new(), + start_time_unix_nano: 0, + time_unix_nano: 0, // Zero timestamp + exemplars: Vec::new(), + flags: 0, + value: Some(number_data_point::Value::AsDouble(1.0)), + }], + })), + }], + schema_url: String::new(), + }], + schema_url: String::new(), + }], + }; + + let data_points = parse_otlp_metrics(request).data_points; + assert_eq!(data_points.len(), 0); } } From 4e928fea88d1217d8416aa2939538671345d62ea Mon Sep 17 00:00:00 2001 From: Matthew Kim Date: Wed, 18 Mar 2026 11:02:04 -0400 Subject: [PATCH 2/8] feat: replace ParquetField enum with constants and dynamic validation --- quickwit/Cargo.lock | 1 + quickwit/quickwit-parquet-engine/Cargo.toml | 1 + .../src/schema/fields.rs | 180 ++++-------------- .../quickwit-parquet-engine/src/schema/mod.rs | 2 +- .../src/schema/parquet.rs | 68 +++---- 5 files changed, 76 insertions(+), 176 deletions(-) diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index d099964a383..684303cb6a8 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7611,6 +7611,7 @@ dependencies = [ name = "quickwit-parquet-engine" version = "0.8.0" dependencies = [ + "anyhow", "arrow", "parquet", "proptest", diff --git a/quickwit/quickwit-parquet-engine/Cargo.toml b/quickwit/quickwit-parquet-engine/Cargo.toml index f886083eded..9842ceb6f00 100644 --- a/quickwit/quickwit-parquet-engine/Cargo.toml +++ b/quickwit/quickwit-parquet-engine/Cargo.toml @@ -11,6 +11,7 @@ authors.workspace = true license.workspace = true [dependencies] +anyhow = { workspace = true } arrow = { workspace = true } parquet = { workspace = true } quickwit-common = { workspace = true } diff --git a/quickwit/quickwit-parquet-engine/src/schema/fields.rs b/quickwit/quickwit-parquet-engine/src/schema/fields.rs index b6a52c738aa..a8646823017 100644 --- a/quickwit/quickwit-parquet-engine/src/schema/fields.rs +++ b/quickwit/quickwit-parquet-engine/src/schema/fields.rs @@ -12,151 +12,55 @@ // See the License for the specific language governing permissions and // limitations under the License. -//! Parquet field definitions with column metadata. +//! Parquet field definitions with sort order constants and validation. -use arrow::datatypes::{DataType, Field, Fields}; -use parquet::variant::VariantType; +use anyhow::{bail, Result}; +use arrow::datatypes::DataType; -/// All fields in the parquet schema. -#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] -pub enum ParquetField { - MetricName, - MetricType, - MetricUnit, - TimestampSecs, - StartTimestampSecs, - Value, - TagService, - TagEnv, - TagDatacenter, - TagRegion, - TagHost, - Attributes, - ServiceName, - ResourceAttributes, -} +/// Required field names that must exist in every batch. +pub const REQUIRED_FIELDS: &[&str] = &["metric_name", "metric_type", "timestamp_secs", "value"]; -impl ParquetField { - /// Field name as stored in Parquet. - pub fn name(&self) -> &'static str { - match self { - Self::MetricName => "metric_name", - Self::MetricType => "metric_type", - Self::MetricUnit => "metric_unit", - Self::TimestampSecs => "timestamp_secs", - Self::StartTimestampSecs => "start_timestamp_secs", - Self::Value => "value", - Self::TagService => "tag_service", - Self::TagEnv => "tag_env", - Self::TagDatacenter => "tag_datacenter", - Self::TagRegion => "tag_region", - Self::TagHost => "tag_host", - Self::Attributes => "attributes", - Self::ServiceName => "service_name", - Self::ResourceAttributes => "resource_attributes", - } - } +/// Sort order column names. Columns not present in a batch are skipped. +pub const SORT_ORDER: &[&str] = &[ + "metric_name", + "service", + "env", + "datacenter", + "region", + "host", + "timestamp_secs", +]; - /// Whether this field is nullable. - pub fn is_nullable(&self) -> bool { - matches!( - self, - Self::MetricUnit - | Self::StartTimestampSecs - | Self::TagService - | Self::TagEnv - | Self::TagDatacenter - | Self::TagRegion - | Self::TagHost - | Self::Attributes - | Self::ResourceAttributes - ) +/// Arrow type for required fields by name. +pub fn required_field_type(name: &str) -> Option { + match name { + "metric_name" => Some(DataType::Dictionary( + Box::new(DataType::Int32), + Box::new(DataType::Utf8), + )), + "metric_type" => Some(DataType::UInt8), + "timestamp_secs" => Some(DataType::UInt64), + "value" => Some(DataType::Float64), + _ => None, } +} - /// Arrow DataType for this field. - /// Use dictionary encoding for high-cardinality strings. - pub fn arrow_type(&self) -> DataType { - match self { - // Dictionary-encoded strings for high cardinality - Self::MetricName | Self::ServiceName => { - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)) +/// Validate that a batch schema contains all required fields with correct types. +pub fn validate_required_fields(schema: &arrow::datatypes::Schema) -> Result<()> { + for &name in REQUIRED_FIELDS { + match schema.index_of(name) { + Ok(idx) => { + let expected_type = required_field_type(name).unwrap(); + let actual_type = schema.field(idx).data_type(); + if *actual_type != expected_type { + bail!( + "field '{}' has type {:?}, expected {:?}", + name, actual_type, expected_type + ); + } } - // Dictionary-encoded optional tags - Self::TagService - | Self::TagEnv - | Self::TagDatacenter - | Self::TagRegion - | Self::TagHost => { - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)) - } - // Enum stored as UInt8 - Self::MetricType => DataType::UInt8, - // Timestamps as UInt64 seconds - Self::TimestampSecs | Self::StartTimestampSecs => DataType::UInt64, - // Metric value - Self::Value => DataType::Float64, - // Plain string for metric unit - Self::MetricUnit => DataType::Utf8, - // VARIANT type for semi-structured attributes - // Uses the Parquet Variant binary encoding format - Self::Attributes | Self::ResourceAttributes => { - // VARIANT is stored as a struct with metadata and value BinaryView fields - // VariantArrayBuilder produces BinaryView, not Binary - DataType::Struct(Fields::from(vec![ - Field::new("metadata", DataType::BinaryView, false), - Field::new("value", DataType::BinaryView, false), - ])) - } - } - } - - /// Convert to Arrow Field. - pub fn to_arrow_field(&self) -> Field { - let field = Field::new(self.name(), self.arrow_type(), self.is_nullable()); - - // Add VARIANT extension type metadata for attributes fields - match self { - Self::Attributes | Self::ResourceAttributes => field.with_extension_type(VariantType), - _ => field, + Err(_) => bail!("missing required field '{}'", name), } } - - /// All fields in schema order. - pub fn all() -> &'static [ParquetField] { - &[ - Self::MetricName, - Self::MetricType, - Self::MetricUnit, - Self::TimestampSecs, - Self::StartTimestampSecs, - Self::Value, - Self::TagService, - Self::TagEnv, - Self::TagDatacenter, - Self::TagRegion, - Self::TagHost, - Self::Attributes, - Self::ServiceName, - Self::ResourceAttributes, - ] - } - - /// Sort order for metrics data (used for pruning). - /// Order: metric_name, common tags (service, env, datacenter, region, host), timestamp. - pub fn sort_order() -> &'static [ParquetField] { - &[ - Self::MetricName, - Self::TagService, - Self::TagEnv, - Self::TagDatacenter, - Self::TagRegion, - Self::TagHost, - Self::TimestampSecs, - ] - } - - /// Get the column index in the schema. - pub fn column_index(&self) -> usize { - Self::all().iter().position(|f| f == self).unwrap() - } + Ok(()) } diff --git a/quickwit/quickwit-parquet-engine/src/schema/mod.rs b/quickwit/quickwit-parquet-engine/src/schema/mod.rs index 4cbe31d357d..f9b5c06d9c4 100644 --- a/quickwit/quickwit-parquet-engine/src/schema/mod.rs +++ b/quickwit/quickwit-parquet-engine/src/schema/mod.rs @@ -20,5 +20,5 @@ mod fields; mod parquet; -pub use fields::ParquetField; +pub use fields::{REQUIRED_FIELDS, SORT_ORDER, required_field_type, validate_required_fields}; pub use parquet::ParquetSchema; diff --git a/quickwit/quickwit-parquet-engine/src/schema/parquet.rs b/quickwit/quickwit-parquet-engine/src/schema/parquet.rs index 83ef99e351c..d0991f89807 100644 --- a/quickwit/quickwit-parquet-engine/src/schema/parquet.rs +++ b/quickwit/quickwit-parquet-engine/src/schema/parquet.rs @@ -14,13 +14,7 @@ //! Parquet schema construction for metrics. -use std::sync::Arc; - -use arrow::datatypes::{Schema as ArrowSchema, SchemaRef}; -use parquet::arrow::ArrowSchemaConverter; -use parquet::schema::types::SchemaDescriptor; - -use super::fields::ParquetField; +use arrow::datatypes::SchemaRef; /// Parquet schema for storage. #[derive(Debug, Clone)] @@ -29,15 +23,11 @@ pub struct ParquetSchema { } impl ParquetSchema { - /// Create a new ParquetSchema. - pub fn new() -> Self { - let fields: Vec<_> = ParquetField::all() - .iter() - .map(|f| f.to_arrow_field()) - .collect(); - - let arrow_schema = Arc::new(ArrowSchema::new(fields)); - Self { arrow_schema } + /// Create a ParquetSchema from an Arrow schema. + pub fn from_arrow_schema(schema: SchemaRef) -> Self { + Self { + arrow_schema: schema, + } } /// Get the Arrow schema. @@ -45,11 +35,6 @@ impl ParquetSchema { &self.arrow_schema } - /// Convert to Parquet schema descriptor. - pub fn parquet_schema(&self) -> Result { - ArrowSchemaConverter::new().convert(&self.arrow_schema) - } - /// Get field by name. pub fn field(&self, name: &str) -> Option<&arrow::datatypes::Field> { self.arrow_schema.field_with_name(name).ok() @@ -61,33 +46,42 @@ impl ParquetSchema { } } -impl Default for ParquetSchema { - fn default() -> Self { - Self::new() - } -} - #[cfg(test)] mod tests { + use std::sync::Arc; + + use arrow::datatypes::{DataType, Field, Schema as ArrowSchema}; + use super::*; + fn create_test_schema() -> SchemaRef { + Arc::new(ArrowSchema::new(vec![ + Field::new( + "metric_name", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + false, + ), + Field::new("metric_type", DataType::UInt8, false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("value", DataType::Float64, false), + Field::new( + "service", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + true, + ), + ])) + } + #[test] fn test_schema_creation() { - let schema = ParquetSchema::new(); - assert_eq!(schema.num_fields(), 14); + let schema = ParquetSchema::from_arrow_schema(create_test_schema()); + assert_eq!(schema.num_fields(), 5); } #[test] fn test_field_lookup() { - let schema = ParquetSchema::new(); + let schema = ParquetSchema::from_arrow_schema(create_test_schema()); let field = schema.field("metric_name").unwrap(); assert!(!field.is_nullable()); } - - #[test] - fn test_parquet_conversion() { - let schema = ParquetSchema::new(); - let parquet_schema = schema.parquet_schema(); - assert!(parquet_schema.is_ok()); - } } From d8d71ed515260592ce0d142ba018ce0224dd8e6d Mon Sep 17 00:00:00 2001 From: Matthew Kim Date: Wed, 18 Mar 2026 11:02:13 -0400 Subject: [PATCH 3/8] feat: derive sort order and bloom filters from batch schema --- .../src/storage/config.rs | 247 ++++++----- .../src/storage/split_writer.rs | 10 +- .../src/storage/writer.rs | 401 ++++++------------ 3 files changed, 244 insertions(+), 414 deletions(-) diff --git a/quickwit/quickwit-parquet-engine/src/storage/config.rs b/quickwit/quickwit-parquet-engine/src/storage/config.rs index f87845d2ba9..a10a423cf9c 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/config.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/config.rs @@ -14,12 +14,13 @@ //! Parquet writer configuration for metrics storage. +use arrow::datatypes::{DataType, Schema as ArrowSchema}; use parquet::basic::Compression as ParquetCompression; use parquet::file::metadata::SortingColumn; use parquet::file::properties::{EnabledStatistics, WriterProperties, WriterPropertiesBuilder}; use parquet::schema::types::ColumnPath; -use crate::schema::ParquetField; +use crate::schema::SORT_ORDER; /// Default row group size: 128K rows for efficient columnar scans. const DEFAULT_ROW_GROUP_SIZE: usize = 128 * 1024; @@ -117,8 +118,9 @@ impl ParquetWriterConfig { self } - /// Convert to Parquet WriterProperties. - pub fn to_writer_properties(&self) -> WriterProperties { + /// Convert to Parquet WriterProperties using the given Arrow schema to configure + /// per-column settings like dictionary encoding and bloom filters. + pub fn to_writer_properties(&self, schema: &ArrowSchema) -> WriterProperties { let mut builder = WriterProperties::builder() .set_max_row_group_size(self.row_group_size) .set_data_page_size_limit(self.data_page_size) @@ -126,7 +128,7 @@ impl ParquetWriterConfig { // Enable column index for efficient pruning on sorted data (64 bytes default) .set_column_index_truncate_length(Some(64)) // Set sorting columns metadata for readers to use during pruning - .set_sorting_columns(Some(Self::sorting_columns())) + .set_sorting_columns(Some(Self::sorting_columns(schema))) // Enable row group level statistics (min/max/null_count) for query pruning // This allows DataFusion to skip row groups based on timestamp ranges .set_statistics_enabled(EnabledStatistics::Chunk); @@ -142,77 +144,58 @@ impl ParquetWriterConfig { Compression::Uncompressed => builder.set_compression(ParquetCompression::UNCOMPRESSED), }; - // Apply RLE_DICTIONARY encoding and bloom filters for dictionary columns - builder = Self::configure_dictionary_columns(builder); + // Apply dictionary encoding and bloom filters based on schema column types + builder = Self::configure_columns(builder, schema); builder.build() } - /// Configure dictionary encoding and bloom filters for high-cardinality columns. + /// Configure dictionary encoding and bloom filters based on the Arrow schema. /// - /// Dictionary-encoded columns benefit from: - /// - Dictionary encoding: Enabled by default, uses RLE for dictionary indices - /// - Bloom filters: Enable efficient equality filtering without scanning - /// - /// Note: Dictionary encoding is ON by default in Parquet. When enabled, the dictionary - /// indices are automatically encoded using RLE (run-length encoding), which efficiently - /// compresses runs of repeated values. This is ideal for sorted data where consecutive - /// rows often share the same dictionary index. - fn configure_dictionary_columns( + /// - Dictionary encoding is enabled on all Dictionary(Int32, Utf8) columns. + /// - Bloom filters are enabled on metric_name and sort order tag columns. + fn configure_columns( mut builder: WriterPropertiesBuilder, + schema: &ArrowSchema, ) -> WriterPropertiesBuilder { - // Dictionary-encoded columns - ensure dictionary encoding is explicitly enabled - // (default is true, but being explicit documents intent) - let dictionary_columns = [ - ParquetField::MetricName, - ParquetField::TagService, - ParquetField::TagEnv, - ParquetField::TagDatacenter, - ParquetField::TagRegion, - ParquetField::TagHost, - ParquetField::ServiceName, - ]; - - // Columns that benefit from bloom filters (used in WHERE clauses) - // Note: We enable bloom filters on filtering columns, not timestamp_secs or value - let bloom_filter_columns = [ - (ParquetField::MetricName, BLOOM_FILTER_NDV_METRIC_NAME), - (ParquetField::TagService, BLOOM_FILTER_NDV_TAGS), - (ParquetField::TagEnv, BLOOM_FILTER_NDV_TAGS), - (ParquetField::TagDatacenter, BLOOM_FILTER_NDV_TAGS), - (ParquetField::TagHost, BLOOM_FILTER_NDV_TAGS), - (ParquetField::ServiceName, BLOOM_FILTER_NDV_TAGS), - ]; - - // Ensure dictionary encoding is enabled on dictionary columns - // (dictionary encoding uses RLE for indices automatically) - for field in dictionary_columns { + for field in schema.fields() { let col_path = ColumnPath::new(vec![field.name().to_string()]); - builder = builder.set_column_dictionary_enabled(col_path, true); - } - // Enable bloom filters on filtering columns - for (field, ndv) in bloom_filter_columns { - let col_path = ColumnPath::new(vec![field.name().to_string()]); - builder = builder - .set_column_bloom_filter_enabled(col_path.clone(), true) - .set_column_bloom_filter_fpp(col_path.clone(), BLOOM_FILTER_FPP) - .set_column_bloom_filter_ndv(col_path, ndv); - } + // Enable dictionary encoding on all Dictionary(_, _) columns + if matches!(field.data_type(), DataType::Dictionary(_, _)) { + builder = builder.set_column_dictionary_enabled(col_path.clone(), true); + } + // Enable bloom filters on dictionary-typed metric_name and sort order tag columns. + // Exclude non-dictionary columns, like timestamp_secs. + let is_bloom_column = matches!(field.data_type(), DataType::Dictionary(_, _)) + && (field.name() == "metric_name" + || SORT_ORDER.contains(&field.name().as_str())); + if is_bloom_column { + let ndv = if field.name() == "metric_name" { + BLOOM_FILTER_NDV_METRIC_NAME + } else { + BLOOM_FILTER_NDV_TAGS + }; + builder = builder + .set_column_bloom_filter_enabled(col_path.clone(), true) + .set_column_bloom_filter_fpp(col_path.clone(), BLOOM_FILTER_FPP) + .set_column_bloom_filter_ndv(col_path, ndv); + } + } builder } - /// Get the sorting columns for parquet metadata. - /// Order: metric_name, tag_service, tag_env, tag_datacenter, tag_region, tag_host, - /// timestamp_secs. - fn sorting_columns() -> Vec { - ParquetField::sort_order() + /// Get the sorting columns for parquet metadata, computed from the schema + /// and SORT_ORDER. Only columns present in the schema are included. + fn sorting_columns(schema: &ArrowSchema) -> Vec { + SORT_ORDER .iter() - .map(|field| SortingColumn { - column_idx: field.column_index() as i32, + .filter_map(|name| schema.index_of(name).ok()) + .map(|idx| SortingColumn { + column_idx: idx as i32, descending: false, - nulls_first: true, + nulls_first: false, }) .collect() } @@ -220,8 +203,41 @@ impl ParquetWriterConfig { #[cfg(test)] mod tests { + use std::sync::Arc; + + use arrow::datatypes::Field; + use super::*; + /// Create a test schema with required fields + some tag columns. + fn create_test_schema() -> ArrowSchema { + ArrowSchema::new(vec![ + Field::new( + "metric_name", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + false, + ), + Field::new("metric_type", DataType::UInt8, false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("value", DataType::Float64, false), + Field::new( + "service", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + true, + ), + Field::new( + "env", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + true, + ), + Field::new( + "host", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + true, + ), + ]) + } + #[test] fn test_default_config() { let config = ParquetWriterConfig::default(); @@ -245,93 +261,89 @@ mod tests { #[test] fn test_to_writer_properties_zstd() { let config = ParquetWriterConfig::default(); - let props = config.to_writer_properties(); - // WriterProperties doesn't expose compression directly, but we can verify it builds + let schema = create_test_schema(); + let props = config.to_writer_properties(&schema); assert!(props.max_row_group_size() == 128 * 1024); } #[test] fn test_to_writer_properties_snappy() { let config = ParquetWriterConfig::new().with_compression(Compression::Snappy); - let props = config.to_writer_properties(); + let schema = create_test_schema(); + let props = config.to_writer_properties(&schema); assert!(props.max_row_group_size() == 128 * 1024); } #[test] fn test_to_writer_properties_uncompressed() { let config = ParquetWriterConfig::new().with_compression(Compression::Uncompressed); - let props = config.to_writer_properties(); + let schema = create_test_schema(); + let props = config.to_writer_properties(&schema); assert!(props.max_row_group_size() == 128 * 1024); } #[test] fn test_bloom_filter_configuration() { let config = ParquetWriterConfig::default(); - let props = config.to_writer_properties(); + let schema = create_test_schema(); + let props = config.to_writer_properties(&schema); // Verify bloom filter is enabled on metric_name column let metric_name_path = ColumnPath::new(vec!["metric_name".to_string()]); let bloom_props = props.bloom_filter_properties(&metric_name_path); assert!( bloom_props.is_some(), - "Bloom filter should be enabled for metric_name" + "bloom filter should be enabled for metric_name" ); let bloom_props = bloom_props.unwrap(); assert!( (bloom_props.fpp - BLOOM_FILTER_FPP).abs() < 0.001, - "Bloom filter FPP should be {}", + "bloom filter FPP should be {}", BLOOM_FILTER_FPP ); assert_eq!( bloom_props.ndv, BLOOM_FILTER_NDV_METRIC_NAME, - "Bloom filter NDV for metric_name should be {}", + "bloom filter NDV for metric_name should be {}", BLOOM_FILTER_NDV_METRIC_NAME ); - // Verify bloom filter is enabled on tag columns - let tag_service_path = ColumnPath::new(vec!["tag_service".to_string()]); - let bloom_props = props.bloom_filter_properties(&tag_service_path); + // Verify bloom filter is enabled on service tag column (in SORT_ORDER) + let service_path = ColumnPath::new(vec!["service".to_string()]); + let bloom_props = props.bloom_filter_properties(&service_path); assert!( bloom_props.is_some(), - "Bloom filter should be enabled for tag_service" + "bloom filter should be enabled for service" ); let bloom_props = bloom_props.unwrap(); assert_eq!( bloom_props.ndv, BLOOM_FILTER_NDV_TAGS, - "Bloom filter NDV for tag columns should be {}", + "bloom filter NDV for tag columns should be {}", BLOOM_FILTER_NDV_TAGS ); - // Verify bloom filter is NOT enabled on timestamp_secs (not a filtering column) - let timestamp_path = ColumnPath::new(vec!["timestamp_secs".to_string()]); - let bloom_props = props.bloom_filter_properties(×tamp_path); - assert!( - bloom_props.is_none(), - "Bloom filter should NOT be enabled for timestamp_secs" - ); - // Verify bloom filter is NOT enabled on value column let value_path = ColumnPath::new(vec!["value".to_string()]); let bloom_props = props.bloom_filter_properties(&value_path); assert!( bloom_props.is_none(), - "Bloom filter should NOT be enabled for value" + "bloom filter should NOT be enabled for value" ); } #[test] fn test_statistics_enabled() { let config = ParquetWriterConfig::default(); - let props = config.to_writer_properties(); + let schema = create_test_schema(); + let props = config.to_writer_properties(&schema); // Verify statistics are enabled at Chunk (row group) level let metric_name_path = ColumnPath::new(vec!["metric_name".to_string()]); assert_eq!( props.statistics_enabled(&metric_name_path), EnabledStatistics::Chunk, - "Statistics should be enabled at Chunk level" + "statistics should be enabled at Chunk level" ); // Verify for timestamp column as well (important for time range pruning) @@ -339,31 +351,24 @@ mod tests { assert_eq!( props.statistics_enabled(×tamp_path), EnabledStatistics::Chunk, - "Statistics should be enabled at Chunk level for timestamp" + "statistics should be enabled at Chunk level for timestamp" ); } #[test] fn test_dictionary_encoding_enabled() { let config = ParquetWriterConfig::default(); - let props = config.to_writer_properties(); - - // Verify dictionary encoding is enabled for dictionary columns - let dictionary_columns = [ - "metric_name", - "tag_service", - "tag_env", - "tag_datacenter", - "tag_region", - "tag_host", - "service_name", - ]; + let schema = create_test_schema(); + let props = config.to_writer_properties(&schema); + + // Verify dictionary encoding is enabled for dictionary-typed columns + let dictionary_columns = ["metric_name", "service", "env", "host"]; for col_name in dictionary_columns { let col_path = ColumnPath::new(vec![col_name.to_string()]); assert!( props.dictionary_enabled(&col_path), - "Dictionary encoding should be enabled for {}", + "dictionary encoding should be enabled for {}", col_name ); } @@ -371,37 +376,31 @@ mod tests { #[test] fn test_sorting_columns_order() { - let sorting_cols = ParquetWriterConfig::sorting_columns(); + let schema = create_test_schema(); + let sorting_cols = ParquetWriterConfig::sorting_columns(&schema); - // Verify we have the expected number of sorting columns + // The test schema has metric_name (idx 0), timestamp_secs (idx 2), + // service (idx 4), env (idx 5), host (idx 6). + // SORT_ORDER is: metric_name, service, env, datacenter, region, host, timestamp_secs + // Only present columns are included, so: metric_name, service, env, host, timestamp_secs assert_eq!( sorting_cols.len(), - 7, - "Should have 7 sorting columns: metric_name, 5 tags, timestamp" + 5, + "should have 5 sorting columns from the test schema" ); - // Verify sort order matches expected: metric_name, tag_service, tag_env, - // tag_datacenter, tag_region, tag_host, timestamp_secs - let expected_order = [ - ParquetField::MetricName, - ParquetField::TagService, - ParquetField::TagEnv, - ParquetField::TagDatacenter, - ParquetField::TagRegion, - ParquetField::TagHost, - ParquetField::TimestampSecs, - ]; - - for (i, expected_field) in expected_order.iter().enumerate() { - assert_eq!( - sorting_cols[i].column_idx, - expected_field.column_index() as i32, - "Sorting column {} should be {}", - i, - expected_field.name() - ); - assert!(!sorting_cols[i].descending, "Sorting should be ascending"); - assert!(sorting_cols[i].nulls_first, "Nulls should be first"); + // Verify all are ascending with nulls first + for col in &sorting_cols { + assert!(!col.descending, "sorting should be ascending"); + assert!(!col.nulls_first, "nulls should be last"); } + + // Verify order matches SORT_ORDER filtered by schema presence: + // metric_name (idx 0), service (idx 4), env (idx 5), host (idx 6), timestamp_secs (idx 2) + assert_eq!(sorting_cols[0].column_idx, 0); // metric_name + assert_eq!(sorting_cols[1].column_idx, 4); // service + assert_eq!(sorting_cols[2].column_idx, 5); // env + assert_eq!(sorting_cols[3].column_idx, 6); // host + assert_eq!(sorting_cols[4].column_idx, 2); // timestamp_secs } } diff --git a/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs b/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs index 8505eb0f75c..ae5fa340722 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs @@ -25,7 +25,6 @@ use tracing::{debug, info, instrument}; use super::config::ParquetWriterConfig; use super::writer::{ParquetWriteError, ParquetWriter}; -use crate::schema::{ParquetField, ParquetSchema}; use crate::split::{MetricsSplitMetadata, ParquetSplit, SplitId, TAG_SERVICE, TimeRange}; /// Writer that produces complete ParquetSplit with metadata from RecordBatch data. @@ -40,16 +39,11 @@ impl ParquetSplitWriter { /// Create a new ParquetSplitWriter. /// /// # Arguments - /// * `schema` - The metrics schema for validation /// * `config` - Parquet writer configuration /// * `base_path` - Directory where split files will be written - pub fn new( - schema: ParquetSchema, - config: ParquetWriterConfig, - base_path: impl Into, - ) -> Self { + pub fn new(config: ParquetWriterConfig, base_path: impl Into) -> Self { Self { - writer: ParquetWriter::new(schema, config), + writer: ParquetWriter::new(config), base_path: base_path.into(), } } diff --git a/quickwit/quickwit-parquet-engine/src/storage/writer.rs b/quickwit/quickwit-parquet-engine/src/storage/writer.rs index 50a375f42b8..134702103eb 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/writer.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/writer.rs @@ -27,7 +27,7 @@ use thiserror::Error; use tracing::{debug, instrument}; use super::config::ParquetWriterConfig; -use crate::schema::{ParquetField, ParquetSchema}; +use crate::schema::{SORT_ORDER, validate_required_fields}; /// Errors that can occur during parquet writing. #[derive(Debug, Error)] @@ -44,21 +44,20 @@ pub enum ParquetWriteError { #[error("Arrow error: {0}")] ArrowError(#[from] arrow::error::ArrowError), - /// Schema mismatch between RecordBatch and ParquetSchema. - #[error("Schema mismatch: expected {expected} fields, got {got}")] - SchemaMismatch { expected: usize, got: usize }, + /// Schema validation failed. + #[error("Schema validation failed: {0}")] + SchemaValidation(String), } /// Writer for metrics data to Parquet format. pub struct ParquetWriter { config: ParquetWriterConfig, - schema: ParquetSchema, } impl ParquetWriter { /// Create a new ParquetWriter. - pub fn new(schema: ParquetSchema, config: ParquetWriterConfig) -> Self { - Self { config, schema } + pub fn new(config: ParquetWriterConfig) -> Self { + Self { config } } /// Get the writer configuration. @@ -66,58 +65,58 @@ impl ParquetWriter { &self.config } - /// Get the metrics schema. - pub fn schema(&self) -> &ParquetSchema { - &self.schema - } - - /// Validate that a RecordBatch matches the expected schema. - fn validate_batch(&self, batch: &RecordBatch) -> Result<(), ParquetWriteError> { - let expected = self.schema.num_fields(); - let got = batch.num_columns(); - if expected != got { - return Err(ParquetWriteError::SchemaMismatch { expected, got }); - } - Ok(()) - } - /// Sort a RecordBatch by the metrics sort order. - /// Order: metric_name, tag_service, tag_env, tag_datacenter, tag_region, tag_host, - /// timestamp_secs. This sorting enables efficient pruning during query execution. + /// Columns from SORT_ORDER that are present in the batch schema are used; + /// missing columns are skipped. fn sort_batch(&self, batch: &RecordBatch) -> Result { - // Build sort columns from the defined sort order - let sort_columns: Vec = ParquetField::sort_order() + let schema = batch.schema(); + let mut sort_columns: Vec = SORT_ORDER .iter() - .map(|field| { - let col_idx = field.column_index(); - SortColumn { - values: Arc::clone(batch.column(col_idx)), - options: Some(SortOptions { - descending: false, - nulls_first: true, - }), - } + .filter_map(|name| schema.index_of(name).ok()) + .map(|idx| SortColumn { + values: Arc::clone(batch.column(idx)), + options: Some(SortOptions { + descending: false, + nulls_first: false, + }), }) .collect(); - // Compute sorted indices - let indices = lexsort_to_indices(&sort_columns, None)?; + if sort_columns.is_empty() { + return Ok(batch.clone()); + } - // Reorder the batch using the sorted indices - let sorted_batch = take_record_batch(batch, &indices)?; - Ok(sorted_batch) + // Append the original row index as a tiebreaker so that rows with + // identical sort keys keep their arrival order (stable sort semantics). + // lexsort_to_indices uses an unstable sort internally; the tiebreaker + // makes it behave stably at a small cost (one u32 comparison per + // equal-key pair, 4 bytes × num_rows of extra allocation). + let row_indices = Arc::new(arrow::array::UInt32Array::from_iter_values( + 0..batch.num_rows() as u32, + )); + sort_columns.push(SortColumn { + values: row_indices, + options: Some(SortOptions { + descending: false, + nulls_first: false, + }), + }); + + let indices = lexsort_to_indices(&sort_columns, None)?; + Ok(take_record_batch(batch, &indices)?) } /// Write a RecordBatch to Parquet bytes in memory. /// The batch is sorted before writing by: metric_name, common tags, timestamp. #[instrument(skip(self, batch), fields(batch_rows = batch.num_rows()))] pub fn write_to_bytes(&self, batch: &RecordBatch) -> Result, ParquetWriteError> { - self.validate_batch(batch)?; + validate_required_fields(&batch.schema()) + .map_err(|e| ParquetWriteError::SchemaValidation(e.to_string()))?; // Sort the batch before writing for efficient pruning let sorted_batch = self.sort_batch(batch)?; - let props = self.config.to_writer_properties(); + let props = self.config.to_writer_properties(&sorted_batch.schema()); let buffer = Cursor::new(Vec::new()); let mut writer = ArrowWriter::try_new(buffer, sorted_batch.schema(), Some(props))?; @@ -125,7 +124,7 @@ impl ParquetWriter { let buffer = writer.into_inner()?; let bytes = buffer.into_inner(); - debug!(bytes_written = bytes.len(), "Completed write to bytes"); + debug!(bytes_written = bytes.len(), "completed write to bytes"); Ok(bytes) } @@ -139,12 +138,13 @@ impl ParquetWriter { batch: &RecordBatch, path: &Path, ) -> Result { - self.validate_batch(batch)?; + validate_required_fields(&batch.schema()) + .map_err(|e| ParquetWriteError::SchemaValidation(e.to_string()))?; // Sort the batch before writing for efficient pruning let sorted_batch = self.sort_batch(batch)?; - let props = self.config.to_writer_properties(); + let props = self.config.to_writer_properties(&sorted_batch.schema()); let file = File::create(path)?; let mut writer = ArrowWriter::try_new(file, sorted_batch.schema(), Some(props))?; @@ -152,7 +152,7 @@ impl ParquetWriter { let file = writer.into_inner()?; let bytes_written = file.metadata()?.len(); - debug!(bytes_written, "Completed write to file"); + debug!(bytes_written, "completed write to file"); Ok(bytes_written) } } @@ -161,152 +161,26 @@ impl ParquetWriter { mod tests { use std::sync::Arc; - use arrow::array::{ - ArrayRef, DictionaryArray, Float64Array, StringArray, UInt8Array, UInt64Array, - }; + use arrow::array::{ArrayRef, DictionaryArray, Float64Array, StringArray, UInt64Array, UInt8Array}; use arrow::datatypes::{DataType, Field, Int32Type, Schema}; - use parquet::variant::{VariantArrayBuilder, VariantBuilderExt}; + use crate::test_helpers::create_test_batch_with_tags; use super::*; - /// Create dictionary array for string fields with Int32 keys. - fn create_dict_array(values: Vec<&str>) -> ArrayRef { - let string_array = StringArray::from(values); - Arc::new( - DictionaryArray::::try_new( - arrow::array::Int32Array::from(vec![0i32]), - Arc::new(string_array), - ) - .unwrap(), - ) - } - - /// Create nullable dictionary array for optional string fields. - fn create_nullable_dict_array(value: Option<&str>) -> ArrayRef { - match value { - Some(v) => { - let string_array = StringArray::from(vec![v]); - Arc::new( - DictionaryArray::::try_new( - arrow::array::Int32Array::from(vec![0i32]), - Arc::new(string_array), - ) - .unwrap(), - ) - } - None => { - let string_array = StringArray::from(vec![None::<&str>]); - Arc::new( - DictionaryArray::::try_new( - arrow::array::Int32Array::from(vec![None::]), - Arc::new(string_array), - ) - .unwrap(), - ) - } - } - } - - /// Create a VARIANT array for testing. - fn create_variant_array(fields: Option<&[(&str, &str)]>) -> ArrayRef { - let mut builder = VariantArrayBuilder::new(1); - match fields { - Some(kv_pairs) => { - let mut obj = builder.new_object(); - for (key, value) in kv_pairs { - obj = obj.with_field(key, *value); - } - obj.finish(); - } - None => { - builder.append_null(); - } - } - ArrayRef::from(builder.build()) - } - fn create_test_batch() -> RecordBatch { - let schema = ParquetSchema::new(); - - // Create arrays for all 14 fields in ParquetSchema matching fields.rs: - // MetricName: Dictionary(Int32, Utf8) - let metric_name: ArrayRef = create_dict_array(vec!["test.metric"]); - - // MetricType: UInt8 - let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8])); // gauge - - // MetricUnit: Utf8 (nullable) - let metric_unit: ArrayRef = Arc::new(StringArray::from(vec![Some("bytes")])); - - // TimestampSecs: UInt64 - let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(vec![1704067200u64])); - - // StartTimestampSecs: UInt64 (nullable) - let start_timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(vec![None::])); - - // Value: Float64 - let value: ArrayRef = Arc::new(Float64Array::from(vec![42.0])); - - // TagService: Dictionary(Int32, Utf8) (nullable) - let tag_service: ArrayRef = create_nullable_dict_array(Some("web")); - - // TagEnv: Dictionary(Int32, Utf8) (nullable) - let tag_env: ArrayRef = create_nullable_dict_array(Some("prod")); - - // TagDatacenter: Dictionary(Int32, Utf8) (nullable) - let tag_datacenter: ArrayRef = create_nullable_dict_array(Some("us-east-1")); - - // TagRegion: Dictionary(Int32, Utf8) (nullable) - let tag_region: ArrayRef = create_nullable_dict_array(None); - - // TagHost: Dictionary(Int32, Utf8) (nullable) - let tag_host: ArrayRef = create_nullable_dict_array(Some("host-001")); - - // Attributes: VARIANT (nullable) - let attributes: ArrayRef = create_variant_array(Some(&[("key", "value")])); - - // ServiceName: Dictionary(Int32, Utf8) - let service_name: ArrayRef = create_dict_array(vec!["my-service"]); - - // ResourceAttributes: VARIANT (nullable) - let resource_attributes: ArrayRef = create_variant_array(None); - - RecordBatch::try_new( - schema.arrow_schema().clone(), - vec![ - metric_name, - metric_type, - metric_unit, - timestamp_secs, - start_timestamp_secs, - value, - tag_service, - tag_env, - tag_datacenter, - tag_region, - tag_host, - attributes, - service_name, - resource_attributes, - ], - ) - .unwrap() + create_test_batch_with_tags(1, &["service", "env"]) } #[test] fn test_writer_creation() { - let schema = ParquetSchema::new(); let config = ParquetWriterConfig::default(); - let writer = ParquetWriter::new(schema, config); - - assert_eq!(writer.schema().num_fields(), 14); + let _writer = ParquetWriter::new(config); } #[test] fn test_write_to_bytes() { - let schema = ParquetSchema::new(); let config = ParquetWriterConfig::default(); - let writer = ParquetWriter::new(schema, config); + let writer = ParquetWriter::new(config); let batch = create_test_batch(); let bytes = writer.write_to_bytes(&batch).unwrap(); @@ -318,9 +192,8 @@ mod tests { #[test] fn test_write_to_file() { - let schema = ParquetSchema::new(); let config = ParquetWriterConfig::default(); - let writer = ParquetWriter::new(schema, config); + let writer = ParquetWriter::new(config); let batch = create_test_batch(); let temp_dir = std::env::temp_dir(); @@ -334,12 +207,11 @@ mod tests { } #[test] - fn test_schema_mismatch() { - let schema = ParquetSchema::new(); + fn test_schema_validation_missing_field() { let config = ParquetWriterConfig::default(); - let writer = ParquetWriter::new(schema, config); + let writer = ParquetWriter::new(config); - // Create a batch with wrong number of columns + // Create a batch missing required fields let wrong_schema = Arc::new(Schema::new(vec![Field::new( "single_field", DataType::Utf8, @@ -352,22 +224,42 @@ mod tests { .unwrap(); let result = writer.write_to_bytes(&wrong_batch); - assert!(matches!( - result, - Err(ParquetWriteError::SchemaMismatch { - expected: 14, - got: 1 - }) - )); + assert!(matches!(result, Err(ParquetWriteError::SchemaValidation(_)))); + } + + #[test] + fn test_schema_validation_wrong_type() { + let config = ParquetWriterConfig::default(); + let writer = ParquetWriter::new(config); + + // Create a batch where metric_name has wrong type (Utf8 instead of Dictionary) + let wrong_schema = Arc::new(Schema::new(vec![ + Field::new("metric_name", DataType::Utf8, false), + Field::new("metric_type", DataType::UInt8, false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("value", DataType::Float64, false), + ])); + let wrong_batch = RecordBatch::try_new( + wrong_schema, + vec![ + Arc::new(StringArray::from(vec!["test"])) as ArrayRef, + Arc::new(UInt8Array::from(vec![0u8])) as ArrayRef, + Arc::new(UInt64Array::from(vec![100u64])) as ArrayRef, + Arc::new(Float64Array::from(vec![1.0])) as ArrayRef, + ], + ) + .unwrap(); + + let result = writer.write_to_bytes(&wrong_batch); + assert!(matches!(result, Err(ParquetWriteError::SchemaValidation(_)))); } #[test] fn test_write_with_snappy_compression() { use super::super::config::Compression; - let schema = ParquetSchema::new(); let config = ParquetWriterConfig::new().with_compression(Compression::Snappy); - let writer = ParquetWriter::new(schema, config); + let writer = ParquetWriter::new(config); let batch = create_test_batch(); let bytes = writer.write_to_bytes(&batch).unwrap(); @@ -382,9 +274,25 @@ mod tests { use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; - let schema = ParquetSchema::new(); let config = ParquetWriterConfig::default(); - let writer = ParquetWriter::new(schema.clone(), config); + let writer = ParquetWriter::new(config); + + // Create a schema with required fields + service tag for sort verification + let schema = Arc::new(Schema::new(vec![ + Field::new( + "metric_name", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + false, + ), + Field::new("metric_type", DataType::UInt8, false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("value", DataType::Float64, false), + Field::new( + "service", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + true, + ), + ])); // Create unsorted batch with multiple rows: // Row 0: metric_b, service_a, timestamp=300 @@ -393,11 +301,6 @@ mod tests { // Expected sorted order: metric_a/service_a/200, metric_a/service_b/100, // metric_b/service_a/300 - // Build arrays for 3 rows (original unsorted order in comments above) - let timestamps = [300u64, 100u64, 200u64]; - let values = [1.0, 2.0, 3.0]; - - // metric_name: Dictionary(Int32, Utf8) let metric_name: ArrayRef = { let keys = arrow::array::Int32Array::from(vec![0i32, 1, 1]); let values = StringArray::from(vec!["metric_b", "metric_a"]); @@ -405,88 +308,18 @@ mod tests { }; let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8, 0, 0])); - let metric_unit: ArrayRef = Arc::new(StringArray::from(vec![ - Some("bytes"), - Some("bytes"), - Some("bytes"), - ])); - let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps.to_vec())); - let start_timestamp_secs: ArrayRef = - Arc::new(UInt64Array::from(vec![None::, None, None])); - let value: ArrayRef = Arc::new(Float64Array::from(values.to_vec())); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(vec![300u64, 100u64, 200u64])); + let value: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0])); - // tag_service: Dictionary(Int32, Utf8) (nullable) - let tag_service: ArrayRef = { + let service: ArrayRef = { let keys = arrow::array::Int32Array::from(vec![Some(0i32), Some(1), Some(0)]); let values = StringArray::from(vec!["service_a", "service_b"]); Arc::new(DictionaryArray::::try_new(keys, Arc::new(values)).unwrap()) }; - let tag_env: ArrayRef = { - let keys = arrow::array::Int32Array::from(vec![Some(0i32), Some(0), Some(0)]); - let values = StringArray::from(vec!["prod"]); - Arc::new(DictionaryArray::::try_new(keys, Arc::new(values)).unwrap()) - }; - - let tag_datacenter: ArrayRef = { - let keys = arrow::array::Int32Array::from(vec![None::, None, None]); - let values = StringArray::from(vec![None::<&str>]); - Arc::new(DictionaryArray::::try_new(keys, Arc::new(values)).unwrap()) - }; - - let tag_region: ArrayRef = { - let keys = arrow::array::Int32Array::from(vec![None::, None, None]); - let values = StringArray::from(vec![None::<&str>]); - Arc::new(DictionaryArray::::try_new(keys, Arc::new(values)).unwrap()) - }; - - let tag_host: ArrayRef = { - let keys = arrow::array::Int32Array::from(vec![None::, None, None]); - let values = StringArray::from(vec![None::<&str>]); - Arc::new(DictionaryArray::::try_new(keys, Arc::new(values)).unwrap()) - }; - - // Build VARIANT arrays for 3 rows - let attributes: ArrayRef = { - let mut builder = VariantArrayBuilder::new(3); - for _ in 0..3 { - builder.append_null(); - } - ArrayRef::from(builder.build()) - }; - - let service_name: ArrayRef = { - let keys = arrow::array::Int32Array::from(vec![0i32, 1, 0]); - let values = StringArray::from(vec!["service_a", "service_b"]); - Arc::new(DictionaryArray::::try_new(keys, Arc::new(values)).unwrap()) - }; - - let resource_attributes: ArrayRef = { - let mut builder = VariantArrayBuilder::new(3); - for _ in 0..3 { - builder.append_null(); - } - ArrayRef::from(builder.build()) - }; - let batch = RecordBatch::try_new( - schema.arrow_schema().clone(), - vec![ - metric_name, - metric_type, - metric_unit, - timestamp_secs, - start_timestamp_secs, - value, - tag_service, - tag_env, - tag_datacenter, - tag_region, - tag_host, - attributes, - service_name, - resource_attributes, - ], + schema, + vec![metric_name, metric_type, timestamp_secs, value, service], ) .unwrap(); @@ -509,25 +342,29 @@ mod tests { assert_eq!(result.num_rows(), 3); // Extract metric names and timestamps to verify sort order + let metric_idx = result.schema().index_of("metric_name").unwrap(); + let ts_idx = result.schema().index_of("timestamp_secs").unwrap(); + let service_idx = result.schema().index_of("service").unwrap(); + let metric_col = result - .column(ParquetField::MetricName.column_index()) + .column(metric_idx) .as_any() .downcast_ref::>() .unwrap(); let ts_col = result - .column(ParquetField::TimestampSecs.column_index()) + .column(ts_idx) .as_any() .downcast_ref::() .unwrap(); let service_col = result - .column(ParquetField::TagService.column_index()) + .column(service_idx) .as_any() .downcast_ref::>() .unwrap(); // Get string values from dictionary - let get_metric = |i: usize| -> &str { - let key = metric_col.keys().value(i); + let get_metric = |row: usize| -> &str { + let key = metric_col.keys().value(row); metric_col .values() .as_any() @@ -535,8 +372,8 @@ mod tests { .unwrap() .value(key as usize) }; - let get_service = |i: usize| -> &str { - let key = service_col.keys().value(i); + let get_service = |row: usize| -> &str { + let key = service_col.keys().value(row); service_col .values() .as_any() @@ -545,7 +382,7 @@ mod tests { .value(key as usize) }; - // Expected sort order: metric_name ASC, tag_service ASC, timestamp ASC + // Expected sort order: metric_name ASC, service ASC, timestamp_secs ASC // Row 0: metric_a, service_a, 200 (original row 2) // Row 1: metric_a, service_b, 100 (original row 1) // Row 2: metric_b, service_a, 300 (original row 0) From ad115bff5da04e0c153277dae8f5b1508b930f41 Mon Sep 17 00:00:00 2001 From: Matthew Kim Date: Wed, 18 Mar 2026 11:02:23 -0400 Subject: [PATCH 4/8] feat: union schema accumulation and schema-agnostic ingest validation --- .../src/index/accumulator.rs | 251 ++++++++++-------- .../src/ingest/processor.rs | 179 ++----------- quickwit/quickwit-parquet-engine/src/lib.rs | 3 + 3 files changed, 160 insertions(+), 273 deletions(-) diff --git a/quickwit/quickwit-parquet-engine/src/index/accumulator.rs b/quickwit/quickwit-parquet-engine/src/index/accumulator.rs index 29065847136..d9eb01f4411 100644 --- a/quickwit/quickwit-parquet-engine/src/index/accumulator.rs +++ b/quickwit/quickwit-parquet-engine/src/index/accumulator.rs @@ -14,15 +14,18 @@ //! Batch accumulator for producing splits from RecordBatches. +use std::collections::BTreeMap; +use std::sync::Arc; use std::time::Instant; +use arrow::array::{new_null_array, ArrayRef}; use arrow::compute::concat_batches; +use arrow::datatypes::{DataType, Field, Schema as ArrowSchema, SchemaRef}; use arrow::record_batch::RecordBatch; use tracing::{debug, info}; use super::config::ParquetIndexingConfig; use crate::metrics::PARQUET_ENGINE_METRICS; -use crate::schema::ParquetSchema; /// Error type for index operations. #[derive(Debug, thiserror::Error)] @@ -38,11 +41,14 @@ pub enum IndexingError { /// Batches are accumulated until either `max_rows` or `max_bytes` threshold is reached, /// at which point they are concatenated and returned for downstream processing (writing to /// Parquet by the ParquetPackager actor). +/// +/// Consecutive batches may have different column sets. The accumulator tracks the union +/// schema incrementally and aligns all batches to it on flush. pub struct ParquetBatchAccumulator { /// Configuration for accumulation thresholds. config: ParquetIndexingConfig, - /// Metrics schema for concatenation. - schema: ParquetSchema, + /// Union of all fields seen across pending batches. + union_fields: BTreeMap, /// Pending batches waiting to be flushed. pending_batches: Vec, /// Total rows in pending batches. @@ -57,11 +63,9 @@ impl ParquetBatchAccumulator { /// # Arguments /// * `config` - Configuration for accumulation thresholds pub fn new(config: ParquetIndexingConfig) -> Self { - let schema = ParquetSchema::new(); - Self { config, - schema, + union_fields: BTreeMap::new(), pending_batches: Vec::new(), pending_rows: 0, pending_bytes: 0, @@ -83,6 +87,13 @@ impl ParquetBatchAccumulator { .index_rows_total .inc_by(batch_rows as u64); + // Merge fields into union schema before pushing (we need the schema reference) + for field in batch.schema().fields() { + self.union_fields + .entry(field.name().clone()) + .or_insert_with(|| (field.data_type().clone(), field.is_nullable())); + } + self.pending_batches.push(batch); self.pending_rows += batch_rows; self.pending_bytes += batch_bytes; @@ -92,7 +103,7 @@ impl ParquetBatchAccumulator { batch_bytes, total_pending_rows = self.pending_rows, total_pending_bytes = self.pending_bytes, - "Added batch to accumulator" + "added batch to accumulator" ); let flushed = if self.should_flush() { @@ -101,7 +112,7 @@ impl ParquetBatchAccumulator { pending_bytes = self.pending_bytes, max_rows = self.config.max_rows, max_bytes = self.config.max_bytes, - "Threshold exceeded, triggering flush" + "threshold exceeded, triggering flush" ); self.flush_internal()? } else { @@ -119,6 +130,7 @@ impl ParquetBatchAccumulator { /// Discard all pending data without producing output. pub fn discard(&mut self) { self.pending_batches.clear(); + self.union_fields.clear(); self.pending_rows = 0; self.pending_bytes = 0; } @@ -136,14 +148,33 @@ impl ParquetBatchAccumulator { return Ok(None); } - // Concatenate all pending batches into one - let combined = concat_batches(self.schema.arrow_schema(), self.pending_batches.iter())?; + // Build the union schema from accumulated fields. + // All fields are marked nullable=true regardless of their source schema: + // any field that appears in some batches but not others will be null-filled + // for the missing batches, so non-nullable would cause Arrow to reject the concat. + let fields: Vec = self + .union_fields + .iter() + .map(|(name, (data_type, _nullable))| Field::new(name, data_type.clone(), true)) + .collect(); + let union_schema: SchemaRef = Arc::new(ArrowSchema::new(fields)); + + // Align each pending batch to the union schema + let aligned: Vec = self + .pending_batches + .iter() + .map(|batch| align_batch_to_schema(batch, &union_schema)) + .collect::, _>>()?; + + // Concatenate all aligned batches + let combined = concat_batches(&union_schema, aligned.iter())?; let num_rows = combined.num_rows(); - info!(num_rows, "Flushed accumulated batches"); + info!(num_rows, "flushed accumulated batches"); // Reset state self.pending_batches.clear(); + self.union_fields.clear(); self.pending_rows = 0; self.pending_bytes = 0; @@ -173,6 +204,24 @@ impl ParquetBatchAccumulator { } } +/// Align a RecordBatch to a target schema, inserting null columns where needed. +fn align_batch_to_schema( + batch: &RecordBatch, + target_schema: &SchemaRef, +) -> Result { + let num_rows = batch.num_rows(); + let batch_schema = batch.schema(); + let columns: Vec = target_schema + .fields() + .iter() + .map(|field| match batch_schema.index_of(field.name()) { + Ok(idx) => Arc::clone(batch.column(idx)), + Err(_) => new_null_array(field.data_type(), num_rows), + }) + .collect(); + RecordBatch::try_new(target_schema.clone(), columns) +} + /// Estimate the memory size of a RecordBatch. fn estimate_batch_bytes(batch: &RecordBatch) -> usize { batch @@ -184,107 +233,10 @@ fn estimate_batch_bytes(batch: &RecordBatch) -> usize { #[cfg(test)] mod tests { - use std::sync::Arc; - - use arrow::array::{ - ArrayRef, DictionaryArray, Float64Array, Int32Array, StringArray, UInt8Array, UInt64Array, - }; - use arrow::datatypes::Int32Type; - use parquet::variant::{VariantArrayBuilder, VariantBuilderExt}; + use crate::test_helpers::{create_test_batch, create_test_batch_with_tags}; use super::*; - /// Create dictionary array for string fields with Int32 keys. - fn create_dict_array(values: &[&str]) -> ArrayRef { - let keys: Vec = (0..values.len()).map(|i| i as i32).collect(); - let string_array = StringArray::from(values.to_vec()); - Arc::new( - DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) - .unwrap(), - ) - } - - /// Create nullable dictionary array for optional string fields. - fn create_nullable_dict_array(values: &[Option<&str>]) -> ArrayRef { - let keys: Vec> = values - .iter() - .enumerate() - .map(|(i, v)| v.map(|_| i as i32)) - .collect(); - let string_values: Vec<&str> = values.iter().filter_map(|v| *v).collect(); - let string_array = StringArray::from(string_values); - Arc::new( - DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) - .unwrap(), - ) - } - - /// Create a VARIANT array for testing with specified number of rows. - fn create_variant_array(num_rows: usize, fields: Option<&[(&str, &str)]>) -> ArrayRef { - let mut builder = VariantArrayBuilder::new(num_rows); - for _ in 0..num_rows { - match fields { - Some(kv_pairs) => { - let mut obj = builder.new_object(); - for (key, value) in kv_pairs { - obj = obj.with_field(key, *value); - } - obj.finish(); - } - None => { - builder.append_null(); - } - } - } - ArrayRef::from(builder.build()) - } - - /// Create a test batch matching the metrics schema. - fn create_test_batch(num_rows: usize) -> RecordBatch { - let schema = ParquetSchema::new(); - - let metric_names: Vec<&str> = vec!["cpu.usage"; num_rows]; - let metric_name: ArrayRef = create_dict_array(&metric_names); - let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; num_rows])); - let metric_unit: ArrayRef = Arc::new(StringArray::from(vec![Some("bytes"); num_rows])); - let timestamps: Vec = (0..num_rows).map(|i| 100 + i as u64).collect(); - let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); - let start_timestamp_secs: ArrayRef = - Arc::new(UInt64Array::from(vec![None::; num_rows])); - let values: Vec = (0..num_rows).map(|i| 42.0 + i as f64).collect(); - let value: ArrayRef = Arc::new(Float64Array::from(values)); - let tag_service: ArrayRef = create_nullable_dict_array(&vec![Some("web"); num_rows]); - let tag_env: ArrayRef = create_nullable_dict_array(&vec![Some("prod"); num_rows]); - let tag_datacenter: ArrayRef = - create_nullable_dict_array(&vec![Some("us-east-1"); num_rows]); - let tag_region: ArrayRef = create_nullable_dict_array(&vec![None; num_rows]); - let tag_host: ArrayRef = create_nullable_dict_array(&vec![Some("host-001"); num_rows]); - let attributes: ArrayRef = create_variant_array(num_rows, None); - let service_name: ArrayRef = create_dict_array(&vec!["my-service"; num_rows]); - let resource_attributes: ArrayRef = create_variant_array(num_rows, None); - - RecordBatch::try_new( - schema.arrow_schema().clone(), - vec![ - metric_name, - metric_type, - metric_unit, - timestamp_secs, - start_timestamp_secs, - value, - tag_service, - tag_env, - tag_datacenter, - tag_region, - tag_host, - attributes, - service_name, - resource_attributes, - ], - ) - .unwrap() - } - #[test] fn test_accumulator_below_threshold() { let config = ParquetIndexingConfig::default().with_max_rows(1000); @@ -351,6 +303,87 @@ mod tests { assert_eq!(combined.num_rows(), 50); } + #[test] + fn test_accumulator_merges_different_tag_sets() { + let config = ParquetIndexingConfig::default().with_max_rows(1000); + let mut accumulator = ParquetBatchAccumulator::new(config); + + // First batch has "service" tag + let batch1 = create_test_batch_with_tags(3, &["service"]); + let _ = accumulator.add_batch(batch1).unwrap(); + + // Second batch has "host" tag + let batch2 = create_test_batch_with_tags(2, &["host"]); + let _ = accumulator.add_batch(batch2).unwrap(); + + let combined = accumulator.flush().unwrap().unwrap(); + assert_eq!(combined.num_rows(), 5); + + // Union schema should have all 4 required fields + both tags + let schema = combined.schema(); + assert!(schema.index_of("metric_name").is_ok()); + assert!(schema.index_of("metric_type").is_ok()); + assert!(schema.index_of("timestamp_secs").is_ok()); + assert!(schema.index_of("value").is_ok()); + assert!(schema.index_of("service").is_ok()); + assert!(schema.index_of("host").is_ok()); + assert_eq!(schema.fields().len(), 6); + + // First 3 rows should have null "host", last 2 rows should have null "service" + let host_idx = schema.index_of("host").unwrap(); + let host_col = combined.column(host_idx); + assert_eq!(host_col.null_count(), 3); // first batch had no host + + let service_idx = schema.index_of("service").unwrap(); + let service_col = combined.column(service_idx); + assert_eq!(service_col.null_count(), 2); // second batch had no service + + // No duplicate column names — each name appears exactly once. + let field_names: Vec<&str> = schema.fields().iter().map(|f| f.name().as_str()).collect(); + let unique_count = field_names.iter().collect::>().len(); + assert_eq!( + unique_count, + field_names.len(), + "duplicate columns in union schema: {field_names:?}" + ); + } + + #[test] + fn test_accumulator_no_duplicates_with_overlapping_tags() { + let config = ParquetIndexingConfig::default().with_max_rows(1000); + let mut accumulator = ParquetBatchAccumulator::new(config); + + // Both batches share "service"; second also has "host". + // "service" must appear exactly once in the flushed schema. + let batch1 = create_test_batch_with_tags(3, &["service"]); + let batch2 = create_test_batch_with_tags(2, &["service", "host"]); + let _ = accumulator.add_batch(batch1).unwrap(); + let _ = accumulator.add_batch(batch2).unwrap(); + + let combined = accumulator.flush().unwrap().unwrap(); + assert_eq!(combined.num_rows(), 5); + + let schema = combined.schema(); + let field_names: Vec<&str> = schema.fields().iter().map(|f| f.name().as_str()).collect(); + let unique_count = field_names.iter().collect::>().len(); + assert_eq!( + unique_count, + field_names.len(), + "duplicate columns in union schema: {field_names:?}" + ); + + // 4 required + service + host = 6 + assert_eq!(schema.fields().len(), 6); + + // Rows from batch1 have no "host" → 3 nulls; batch2 has "host" for all 2 rows → 0 nulls. + let host_idx = schema.index_of("host").unwrap(); + assert_eq!(combined.column(host_idx).null_count(), 3); + + // "service" present in both batches → 0 nulls total. + let service_idx = schema.index_of("service").unwrap(); + assert_eq!(combined.column(service_idx).null_count(), 0); + } + #[test] fn test_estimate_batch_bytes() { let batch = create_test_batch(100); diff --git a/quickwit/quickwit-parquet-engine/src/ingest/processor.rs b/quickwit/quickwit-parquet-engine/src/ingest/processor.rs index 80dd4f322e5..6097d7c3c32 100644 --- a/quickwit/quickwit-parquet-engine/src/ingest/processor.rs +++ b/quickwit/quickwit-parquet-engine/src/ingest/processor.rs @@ -21,7 +21,7 @@ use arrow::record_batch::RecordBatch; use tracing::{debug, instrument, warn}; use crate::metrics::PARQUET_ENGINE_METRICS; -use crate::schema::ParquetSchema; +use crate::schema::validate_required_fields; /// Error type for ingest operations. #[derive(Debug, thiserror::Error)] @@ -34,9 +34,9 @@ pub enum IngestError { #[error("IPC decode error: {0}")] IpcDecode(#[from] arrow::error::ArrowError), - /// RecordBatch schema doesn't match expected metrics schema. - #[error("Schema mismatch: expected {expected} fields, got {actual}")] - SchemaMismatch { expected: usize, actual: usize }, + /// RecordBatch schema validation failed. + #[error("Schema validation failed: {0}")] + SchemaValidation(String), /// Expected exactly one RecordBatch in IPC stream. #[error("Expected 1 RecordBatch in IPC stream, got {0}")] @@ -53,17 +53,10 @@ pub enum IngestError { /// Processor that converts Arrow IPC bytes to RecordBatch. /// -/// Validates that the decoded batch matches the expected metrics schema. -pub struct ParquetIngestProcessor { - schema: ParquetSchema, -} +/// Validates that the decoded batch contains all required fields. +pub struct ParquetIngestProcessor; impl ParquetIngestProcessor { - /// Create a new ParquetIngestProcessor. - pub fn new(schema: ParquetSchema) -> Self { - Self { schema } - } - /// Convert Arrow IPC bytes to RecordBatch. /// /// Returns error if IPC is malformed or schema doesn't match. @@ -95,65 +88,15 @@ impl ParquetIngestProcessor { debug!( num_rows = batch.num_rows(), - "Successfully decoded IPC to RecordBatch" + "successfully decoded IPC to RecordBatch" ); Ok(batch) } - /// Validate that the RecordBatch schema matches expected metrics schema. + /// Validate that the RecordBatch schema contains all required fields. fn validate_schema(&self, batch: &RecordBatch) -> Result<(), IngestError> { - let expected_fields = self.schema.num_fields(); - let actual_fields = batch.schema().fields().len(); - - if expected_fields != actual_fields { - warn!( - expected = expected_fields, - actual = actual_fields, - "Schema mismatch: field count differs" - ); - return Err(IngestError::SchemaMismatch { - expected: expected_fields, - actual: actual_fields, - }); - } - - // Verify field names match (in order) - let schema_fields = self.schema.arrow_schema().fields(); - let batch_schema = batch.schema(); - let batch_fields = batch_schema.fields(); - - for (expected, actual) in schema_fields.iter().zip(batch_fields.iter()) { - if expected.name() != actual.name() { - warn!( - expected_name = expected.name(), - actual_name = actual.name(), - "Schema mismatch: field name differs" - ); - return Err(IngestError::SchemaMismatch { - expected: expected_fields, - actual: actual_fields, - }); - } - if expected.data_type() != actual.data_type() { - warn!( - field = expected.name(), - expected_type = ?expected.data_type(), - actual_type = ?actual.data_type(), - "Schema mismatch: field type differs" - ); - return Err(IngestError::SchemaMismatch { - expected: expected_fields, - actual: actual_fields, - }); - } - } - - Ok(()) - } - - /// Get a reference to the schema. - pub fn schema(&self) -> &ParquetSchema { - &self.schema + validate_required_fields(batch.schema().as_ref()) + .map_err(|e| IngestError::SchemaValidation(e.to_string())) } } @@ -168,11 +111,7 @@ fn ipc_to_record_batch(ipc_bytes: &[u8]) -> Result { return Err(IngestError::UnexpectedBatchCount(batches.len())); } - // Safe: we verified exactly 1 batch above, but use ok_or for defensive programming - batches - .into_iter() - .next() - .ok_or(IngestError::UnexpectedBatchCount(0)) + Ok(batches.into_iter().next().expect("len verified to be 1 above")) } /// Serialize a RecordBatch to Arrow IPC stream format. @@ -190,100 +129,13 @@ pub fn record_batch_to_ipc(batch: &RecordBatch) -> Result, IngestError> #[cfg(test)] mod tests { - use std::sync::Arc; - - use arrow::array::{ - ArrayRef, DictionaryArray, Float64Array, Int32Array, StringArray, UInt8Array, UInt64Array, - }; - use arrow::datatypes::Int32Type; - use parquet::variant::VariantArrayBuilder; + use crate::test_helpers::create_test_batch; use super::*; - /// Create dictionary array for string fields with Int32 keys. - fn create_dict_array(values: &[&str]) -> ArrayRef { - let keys: Vec = (0..values.len()).map(|i| i as i32).collect(); - let string_array = StringArray::from(values.to_vec()); - Arc::new( - DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) - .unwrap(), - ) - } - - /// Create nullable dictionary array for optional string fields. - fn create_nullable_dict_array(values: &[Option<&str>]) -> ArrayRef { - let keys: Vec> = values - .iter() - .enumerate() - .map(|(i, v)| v.map(|_| i as i32)) - .collect(); - let string_values: Vec<&str> = values.iter().filter_map(|v| *v).collect(); - let string_array = StringArray::from(string_values); - Arc::new( - DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) - .unwrap(), - ) - } - - /// Create a VARIANT array for testing with specified number of rows (all nulls). - fn create_variant_array(num_rows: usize) -> ArrayRef { - let mut builder = VariantArrayBuilder::new(num_rows); - for _ in 0..num_rows { - builder.append_null(); - } - ArrayRef::from(builder.build()) - } - - /// Create a test batch matching the metrics schema. - fn create_test_batch(num_rows: usize) -> RecordBatch { - let schema = ParquetSchema::new(); - - let metric_names: Vec<&str> = vec!["cpu.usage"; num_rows]; - let metric_name: ArrayRef = create_dict_array(&metric_names); - let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; num_rows])); - let metric_unit: ArrayRef = Arc::new(StringArray::from(vec![Some("bytes"); num_rows])); - let timestamps: Vec = (0..num_rows).map(|i| 100 + i as u64).collect(); - let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); - let start_timestamp_secs: ArrayRef = - Arc::new(UInt64Array::from(vec![None::; num_rows])); - let values: Vec = (0..num_rows).map(|i| 42.0 + i as f64).collect(); - let value: ArrayRef = Arc::new(Float64Array::from(values)); - let tag_service: ArrayRef = create_nullable_dict_array(&vec![Some("web"); num_rows]); - let tag_env: ArrayRef = create_nullable_dict_array(&vec![Some("prod"); num_rows]); - let tag_datacenter: ArrayRef = - create_nullable_dict_array(&vec![Some("us-east-1"); num_rows]); - let tag_region: ArrayRef = create_nullable_dict_array(&vec![None; num_rows]); - let tag_host: ArrayRef = create_nullable_dict_array(&vec![Some("host-001"); num_rows]); - let attributes: ArrayRef = create_variant_array(num_rows); - let service_name: ArrayRef = create_dict_array(&vec!["my-service"; num_rows]); - let resource_attributes: ArrayRef = create_variant_array(num_rows); - - RecordBatch::try_new( - schema.arrow_schema().clone(), - vec![ - metric_name, - metric_type, - metric_unit, - timestamp_secs, - start_timestamp_secs, - value, - tag_service, - tag_env, - tag_datacenter, - tag_region, - tag_host, - attributes, - service_name, - resource_attributes, - ], - ) - .unwrap() - } - #[test] fn test_process_ipc() { - let schema = ParquetSchema::new(); - let processor = ParquetIngestProcessor::new(schema); + let processor = ParquetIngestProcessor; // Create a valid batch let batch = create_test_batch(10); @@ -295,13 +147,12 @@ mod tests { let recovered = result.unwrap(); assert_eq!(recovered.num_rows(), 10); - assert_eq!(recovered.num_columns(), 14); + assert_eq!(recovered.num_columns(), 6); // 4 required + 2 tags } #[test] fn test_process_ipc_invalid_bytes() { - let schema = ParquetSchema::new(); - let processor = ParquetIngestProcessor::new(schema); + let processor = ParquetIngestProcessor; let result = processor.process_ipc(&[0u8; 10]); assert!(result.is_err()); diff --git a/quickwit/quickwit-parquet-engine/src/lib.rs b/quickwit/quickwit-parquet-engine/src/lib.rs index eac11bfa8e9..c6d8640e801 100644 --- a/quickwit/quickwit-parquet-engine/src/lib.rs +++ b/quickwit/quickwit-parquet-engine/src/lib.rs @@ -26,3 +26,6 @@ pub mod metrics; pub mod schema; pub mod split; pub mod storage; + +#[cfg(test)] +pub mod test_helpers; From d0a995e82f51e3c8963d8dc2fd6b1792e3f5ff70 Mon Sep 17 00:00:00 2001 From: Matthew Kim Date: Wed, 18 Mar 2026 11:02:30 -0400 Subject: [PATCH 5/8] feat: dynamic column lookup in split writer --- .../src/storage/split_writer.rs | 318 +++++------------- 1 file changed, 83 insertions(+), 235 deletions(-) diff --git a/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs b/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs index ae5fa340722..a9e081455d0 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs @@ -83,7 +83,7 @@ impl ParquetSplitWriter { debug!( start_secs = time_range.start_secs, end_secs = time_range.end_secs, - "Extracted time range from batch" + "extracted time range from batch" ); // Extract distinct metric names from batch @@ -116,7 +116,7 @@ impl ParquetSplitWriter { split_id = %split_id, file_path = %file_path.display(), size_bytes, - "Split file written successfully" + "split file written successfully" ); Ok(ParquetSplit::new(metadata)) @@ -125,7 +125,11 @@ impl ParquetSplitWriter { /// Extracts the time range (min/max timestamp_secs) from a RecordBatch. fn extract_time_range(batch: &RecordBatch) -> Result { - let timestamp_col = batch.column(ParquetField::TimestampSecs.column_index()); + let timestamp_idx = batch + .schema() + .index_of("timestamp_secs") + .map_err(|_| ParquetWriteError::SchemaValidation("missing timestamp_secs column".into()))?; + let timestamp_col = batch.column(timestamp_idx); let timestamp_array = timestamp_col.as_primitive::(); let min_val = min(timestamp_array); @@ -145,7 +149,11 @@ fn extract_time_range(batch: &RecordBatch) -> Result Result, ParquetWriteError> { - let metric_col = batch.column(ParquetField::MetricName.column_index()); + let metric_idx = batch + .schema() + .index_of("metric_name") + .map_err(|_| ParquetWriteError::SchemaValidation("missing metric_name column".into()))?; + let metric_col = batch.column(metric_idx); let mut names = HashSet::new(); // The column is Dictionary(Int32, Utf8) @@ -174,7 +182,11 @@ fn extract_metric_names(batch: &RecordBatch) -> Result, ParquetW /// Extracts distinct service names from a RecordBatch. fn extract_service_names(batch: &RecordBatch) -> Result, ParquetWriteError> { - let service_col = batch.column(ParquetField::ServiceName.column_index()); + let service_idx = match batch.schema().index_of("service").ok() { + Some(idx) => idx, + None => return Ok(HashSet::new()), + }; + let service_col = batch.column(service_idx); let mut names = HashSet::new(); // The column is Dictionary(Int32, Utf8) @@ -205,84 +217,58 @@ fn extract_service_names(batch: &RecordBatch) -> Result, Parquet mod tests { use std::sync::Arc; - use arrow::array::{ - ArrayRef, DictionaryArray, Float64Array, Int32Array, StringArray, UInt8Array, UInt64Array, - }; - use arrow::datatypes::Int32Type; - use parquet::variant::{VariantArrayBuilder, VariantBuilderExt}; + use arrow::array::{ArrayRef, Float64Array, UInt8Array, UInt64Array}; + use arrow::datatypes::{DataType, Field, Schema as ArrowSchema}; + use crate::test_helpers::{create_dict_array, create_nullable_dict_array}; use super::*; - #[test] - fn test_column_indices_match_schema() { - let schema = ParquetSchema::new(); - for field in ParquetField::all() { - let expected: usize = - schema - .arrow_schema() - .index_of(field.name()) - .unwrap_or_else(|_| { - panic!("field {:?} should exist in arrow schema", field.name()) - }); - assert_eq!( - field.column_index(), - expected, - "column_index() for {:?} does not match arrow schema position", - field.name() - ); + /// Create a test batch with required fields, optional service column, and specified tag columns. + fn create_test_batch_with_options( + num_rows: usize, + metric_names: &[&str], + timestamps: &[u64], + service_names: Option<&[&str]>, + tags: &[&str], + ) -> RecordBatch { + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + + let mut fields = vec![ + Field::new("metric_name", dict_type.clone(), false), + Field::new("metric_type", DataType::UInt8, false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("value", DataType::Float64, false), + ]; + if service_names.is_some() { + fields.push(Field::new("service", dict_type.clone(), true)); } - } + for tag in tags { + fields.push(Field::new(*tag, dict_type.clone(), true)); + } + let schema = Arc::new(ArrowSchema::new(fields)); - /// Create a VARIANT array for testing with specified number of rows. - fn create_variant_array(num_rows: usize, fields: Option<&[(&str, &str)]>) -> ArrayRef { - let mut builder = VariantArrayBuilder::new(num_rows); - for _ in 0..num_rows { - match fields { - Some(kv_pairs) => { - let mut obj = builder.new_object(); - for (key, value) in kv_pairs { - obj = obj.with_field(key, *value); - } - obj.finish(); - } - None => { - builder.append_null(); - } - } + let metric_name: ArrayRef = create_dict_array(metric_names); + let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; num_rows])); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps.to_vec())); + let values: Vec = (0..num_rows).map(|i| 42.0 + i as f64).collect(); + let value: ArrayRef = Arc::new(Float64Array::from(values)); + + let mut columns: Vec = vec![metric_name, metric_type, timestamp_secs, value]; + + if let Some(svc_names) = service_names { + columns.push(create_dict_array(svc_names)); } - ArrayRef::from(builder.build()) - } - /// Create dictionary array for string fields with Int32 keys. - fn create_dict_array(values: &[&str]) -> ArrayRef { - let keys: Vec = (0..values.len()).map(|i| i as i32).collect(); - let string_array = StringArray::from(values.to_vec()); - Arc::new( - DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) - .unwrap(), - ) - } + for tag in tags { + let tag_values: Vec> = vec![Some(tag); num_rows]; + columns.push(create_nullable_dict_array(&tag_values)); + } - /// Create nullable dictionary array for optional string fields. - fn create_nullable_dict_array(values: &[Option<&str>]) -> ArrayRef { - let keys: Vec> = values - .iter() - .enumerate() - .map(|(i, v)| v.map(|_| i as i32)) - .collect(); - let string_values: Vec<&str> = values.iter().filter_map(|v| *v).collect(); - let string_array = StringArray::from(string_values); - Arc::new( - DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) - .unwrap(), - ) + RecordBatch::try_new(schema, columns).unwrap() } - /// Create a test batch with specified number of rows and test data. + /// Create a simple test batch with default values. fn create_test_batch(num_rows: usize) -> RecordBatch { - let schema = ParquetSchema::new(); - - // Generate test data let metric_names: Vec<&str> = (0..num_rows) .map(|i| { if i % 2 == 0 { @@ -293,82 +279,23 @@ mod tests { }) .collect(); let timestamps: Vec = (0..num_rows).map(|i| 100 + i as u64 * 10).collect(); - - // MetricName: Dictionary(Int32, Utf8) - let metric_name: ArrayRef = create_dict_array(&metric_names); - - // MetricType: UInt8 - let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; num_rows])); - - // MetricUnit: Utf8 (nullable) - let metric_unit: ArrayRef = Arc::new(StringArray::from(vec![Some("bytes"); num_rows])); - - // TimestampSecs: UInt64 - let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); - - // StartTimestampSecs: UInt64 (nullable) - let start_timestamp_secs: ArrayRef = - Arc::new(UInt64Array::from(vec![None::; num_rows])); - - // Value: Float64 - let values: Vec = (0..num_rows).map(|i| 42.0 + i as f64).collect(); - let value: ArrayRef = Arc::new(Float64Array::from(values)); - - // TagService: Dictionary(Int32, Utf8) (nullable) - let tag_service: ArrayRef = create_nullable_dict_array(&vec![Some("web"); num_rows]); - - // TagEnv: Dictionary(Int32, Utf8) (nullable) - let tag_env: ArrayRef = create_nullable_dict_array(&vec![Some("prod"); num_rows]); - - // TagDatacenter: Dictionary(Int32, Utf8) (nullable) - let tag_datacenter: ArrayRef = - create_nullable_dict_array(&vec![Some("us-east-1"); num_rows]); - - // TagRegion: Dictionary(Int32, Utf8) (nullable) - let tag_region: ArrayRef = create_nullable_dict_array(&vec![None; num_rows]); - - // TagHost: Dictionary(Int32, Utf8) (nullable) - let tag_host: ArrayRef = create_nullable_dict_array(&vec![Some("host-001"); num_rows]); - - // Attributes: VARIANT (nullable) - let attributes: ArrayRef = create_variant_array(num_rows, Some(&[("key", "value")])); - - // ServiceName: Dictionary(Int32, Utf8) - let service_names: Vec<&str> = (0..num_rows).map(|_| "my-service").collect(); - let service_name: ArrayRef = create_dict_array(&service_names); - - // ResourceAttributes: VARIANT (nullable) - let resource_attributes: ArrayRef = create_variant_array(num_rows, None); - - RecordBatch::try_new( - schema.arrow_schema().clone(), - vec![ - metric_name, - metric_type, - metric_unit, - timestamp_secs, - start_timestamp_secs, - value, - tag_service, - tag_env, - tag_datacenter, - tag_region, - tag_host, - attributes, - service_name, - resource_attributes, - ], + let service_names: Vec<&str> = vec!["my-service"; num_rows]; + + create_test_batch_with_options( + num_rows, + &metric_names, + ×tamps, + Some(&service_names), + &["service", "host"], ) - .unwrap() } #[test] fn test_write_split_creates_file() { - let schema = ParquetSchema::new(); let config = ParquetWriterConfig::default(); let temp_dir = tempfile::tempdir().unwrap(); - let writer = ParquetSplitWriter::new(schema, config, temp_dir.path()); + let writer = ParquetSplitWriter::new(config, temp_dir.path()); let batch = create_test_batch(10); let split = writer.write_split(&batch, "test-index").unwrap(); @@ -387,14 +314,19 @@ mod tests { #[test] fn test_write_split_extracts_time_range() { - let schema = ParquetSchema::new(); let config = ParquetWriterConfig::default(); let temp_dir = tempfile::tempdir().unwrap(); - let writer = ParquetSplitWriter::new(schema, config, temp_dir.path()); + let writer = ParquetSplitWriter::new(config, temp_dir.path()); // Create batch with timestamps [100, 150, 200] - let batch = create_test_batch_with_timestamps(&[100, 150, 200]); + let batch = create_test_batch_with_options( + 3, + &["test.metric", "test.metric", "test.metric"], + &[100, 150, 200], + Some(&["my-service", "my-service", "my-service"]), + &[], + ); let split = writer.write_split(&batch, "test-index").unwrap(); // Verify time range @@ -404,14 +336,19 @@ mod tests { #[test] fn test_write_split_extracts_metric_names() { - let schema = ParquetSchema::new(); let config = ParquetWriterConfig::default(); let temp_dir = tempfile::tempdir().unwrap(); - let writer = ParquetSplitWriter::new(schema, config, temp_dir.path()); + let writer = ParquetSplitWriter::new(config, temp_dir.path()); // Create batch with specific metric names - let batch = create_test_batch_with_metric_names(&["cpu.usage", "memory.used", "cpu.usage"]); + let batch = create_test_batch_with_options( + 3, + &["cpu.usage", "memory.used", "cpu.usage"], + &[100, 100, 100], + Some(&["my-service", "my-service", "my-service"]), + &[], + ); let split = writer.write_split(&batch, "test-index").unwrap(); // Verify metric names (distinct values) @@ -419,93 +356,4 @@ mod tests { assert!(split.metadata.metric_names.contains("memory.used")); assert_eq!(split.metadata.metric_names.len(), 2); } - - /// Create a test batch with specific timestamps. - fn create_test_batch_with_timestamps(timestamps: &[u64]) -> RecordBatch { - let schema = ParquetSchema::new(); - let num_rows = timestamps.len(); - - let metric_names: Vec<&str> = vec!["test.metric"; num_rows]; - let metric_name: ArrayRef = create_dict_array(&metric_names); - let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; num_rows])); - let metric_unit: ArrayRef = Arc::new(StringArray::from(vec![Some("bytes"); num_rows])); - let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps.to_vec())); - let start_timestamp_secs: ArrayRef = - Arc::new(UInt64Array::from(vec![None::; num_rows])); - let value: ArrayRef = Arc::new(Float64Array::from(vec![42.0; num_rows])); - let tag_service: ArrayRef = create_nullable_dict_array(&vec![Some("web"); num_rows]); - let tag_env: ArrayRef = create_nullable_dict_array(&vec![Some("prod"); num_rows]); - let tag_datacenter: ArrayRef = - create_nullable_dict_array(&vec![Some("us-east-1"); num_rows]); - let tag_region: ArrayRef = create_nullable_dict_array(&vec![None; num_rows]); - let tag_host: ArrayRef = create_nullable_dict_array(&vec![Some("host-001"); num_rows]); - let attributes: ArrayRef = create_variant_array(num_rows, None); - let service_name: ArrayRef = create_dict_array(&vec!["my-service"; num_rows]); - let resource_attributes: ArrayRef = create_variant_array(num_rows, None); - - RecordBatch::try_new( - schema.arrow_schema().clone(), - vec![ - metric_name, - metric_type, - metric_unit, - timestamp_secs, - start_timestamp_secs, - value, - tag_service, - tag_env, - tag_datacenter, - tag_region, - tag_host, - attributes, - service_name, - resource_attributes, - ], - ) - .unwrap() - } - - /// Create a test batch with specific metric names. - fn create_test_batch_with_metric_names(metric_names: &[&str]) -> RecordBatch { - let schema = ParquetSchema::new(); - let num_rows = metric_names.len(); - - let metric_name: ArrayRef = create_dict_array(metric_names); - let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; num_rows])); - let metric_unit: ArrayRef = Arc::new(StringArray::from(vec![Some("bytes"); num_rows])); - let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(vec![100u64; num_rows])); - let start_timestamp_secs: ArrayRef = - Arc::new(UInt64Array::from(vec![None::; num_rows])); - let value: ArrayRef = Arc::new(Float64Array::from(vec![42.0; num_rows])); - let tag_service: ArrayRef = create_nullable_dict_array(&vec![Some("web"); num_rows]); - let tag_env: ArrayRef = create_nullable_dict_array(&vec![Some("prod"); num_rows]); - let tag_datacenter: ArrayRef = - create_nullable_dict_array(&vec![Some("us-east-1"); num_rows]); - let tag_region: ArrayRef = create_nullable_dict_array(&vec![None; num_rows]); - let tag_host: ArrayRef = create_nullable_dict_array(&vec![Some("host-001"); num_rows]); - let attributes: ArrayRef = create_variant_array(num_rows, None); - let service_name: ArrayRef = create_dict_array(&vec!["my-service"; num_rows]); - let resource_attributes: ArrayRef = create_variant_array(num_rows, None); - - RecordBatch::try_new( - schema.arrow_schema().clone(), - vec![ - metric_name, - metric_type, - metric_unit, - timestamp_secs, - start_timestamp_secs, - value, - tag_service, - tag_env, - tag_datacenter, - tag_region, - tag_host, - attributes, - service_name, - resource_attributes, - ], - ) - .unwrap() - } } From 35c39421f257aaec47dffc49037a56479ad4ff5e Mon Sep 17 00:00:00 2001 From: Matthew Kim Date: Wed, 18 Mar 2026 11:02:39 -0400 Subject: [PATCH 6/8] feat: remove ParquetSchema dependency from indexing actors --- .../src/actors/indexing_pipeline.rs | 2 - .../src/actors/parquet_doc_processor.rs | 257 +++++------------- .../src/actors/parquet_e2e_test.rs | 103 ++----- .../src/actors/parquet_indexer.rs | 101 +------ .../src/actors/parquet_packager.rs | 102 +------ .../src/models/processed_parquet_batch.rs | 104 +------ 6 files changed, 110 insertions(+), 559 deletions(-) diff --git a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs index 16daf18102f..cfb2ade9361 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs @@ -637,10 +637,8 @@ impl IndexingPipeline { .spawn(parquet_uploader); // ParquetPackager - let parquet_schema = quickwit_parquet_engine::schema::ParquetSchema::new(); let writer_config = quickwit_parquet_engine::storage::ParquetWriterConfig::default(); let split_writer = quickwit_parquet_engine::storage::ParquetSplitWriter::new( - parquet_schema, writer_config, self.params.indexing_directory.path(), ); diff --git a/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs b/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs index 2a83c9610c9..aa26f67fc94 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs @@ -24,7 +24,6 @@ use quickwit_common::rate_limited_tracing::rate_limited_warn; use quickwit_common::runtimes::RuntimeType; use quickwit_metastore::checkpoint::SourceCheckpointDelta; use quickwit_parquet_engine::ingest::{IngestError, ParquetIngestProcessor}; -use quickwit_parquet_engine::schema::ParquetSchema; use quickwit_proto::types::{IndexId, SourceId}; use serde::Serialize; use tokio::runtime::Handle; @@ -143,9 +142,11 @@ impl ParquetDocProcessor { source_id: SourceId, indexer_mailbox: Mailbox, ) -> Self { - let schema = ParquetSchema::new(); - let processor = ParquetIngestProcessor::new(schema); - let counters = ParquetDocProcessorCounters::new(index_id.clone(), source_id.clone()); + let processor = ParquetIngestProcessor; + let counters = ParquetDocProcessorCounters::new( + index_id.clone(), + source_id.clone(), + ); info!( index_id = %index_id, @@ -305,8 +306,9 @@ impl Handler for ParquetDocProcessor { // Without this, a batch of consistently malformed data blocks offset progress // forever. if !checkpoint_forwarded && !checkpoint_delta.is_empty() { - let empty_batch = - RecordBatch::new_empty(self.processor.schema().arrow_schema().clone()); + let empty_batch = RecordBatch::new_empty(std::sync::Arc::new( + arrow::datatypes::Schema::empty(), + )); let processed_batch = ProcessedParquetBatch::new(empty_batch, checkpoint_delta, force_commit); ctx.send_message(&self.indexer_mailbox, processed_batch) @@ -402,10 +404,10 @@ mod tests { use std::sync::Arc as StdArc; use arrow::array::{ - ArrayRef, BinaryViewArray, DictionaryArray, Float64Array, Int32Array, StringArray, - StructArray, UInt8Array, UInt64Array, + ArrayRef, DictionaryArray, Float64Array, Int32Array, StringArray, UInt8Array, + UInt64Array, }; - use arrow::datatypes::{DataType, Field, Int32Type}; + use arrow::datatypes::{DataType, Field, Int32Type, Schema as ArrowSchema}; use arrow::record_batch::RecordBatch; let universe = Universe::with_accelerated_time(); @@ -419,99 +421,42 @@ mod tests { let (metrics_doc_processor_mailbox, metrics_doc_processor_handle) = universe.spawn_builder().spawn(metrics_doc_processor); - // Create a test batch matching the metrics schema - let schema = ParquetSchema::new(); + // Create a test batch with the 4 required fields plus a tag column let num_rows = 3; - - // Helper to create dictionary arrays - fn create_dict_array(values: &[&str]) -> ArrayRef { - let keys: Vec = (0..values.len()).map(|i| i as i32).collect(); - let string_array = StringArray::from(values.to_vec()); - StdArc::new( - DictionaryArray::::try_new( - Int32Array::from(keys), - StdArc::new(string_array), - ) - .unwrap(), - ) - } - - fn create_nullable_dict_array(values: &[Option<&str>]) -> ArrayRef { - let keys: Vec> = values - .iter() - .enumerate() - .map(|(i, v)| v.map(|_| i as i32)) - .collect(); - let string_values: Vec<&str> = values.iter().filter_map(|v| *v).collect(); - let string_array = StringArray::from(string_values); - StdArc::new( - DictionaryArray::::try_new( - Int32Array::from(keys), - StdArc::new(string_array), - ) - .unwrap(), - ) - } - - let metric_name: ArrayRef = create_dict_array(&vec!["cpu.usage"; num_rows]); - let metric_type: ArrayRef = StdArc::new(UInt8Array::from(vec![0u8; num_rows])); - let metric_unit: ArrayRef = StdArc::new(StringArray::from(vec![Some("bytes"); num_rows])); - let timestamp_secs: ArrayRef = StdArc::new(UInt64Array::from(vec![100u64, 101u64, 102u64])); - let start_timestamp_secs: ArrayRef = - StdArc::new(UInt64Array::from(vec![None::; num_rows])); - let value: ArrayRef = StdArc::new(Float64Array::from(vec![42.0, 43.0, 44.0])); - let tag_service: ArrayRef = create_nullable_dict_array(&vec![Some("web"); num_rows]); - let tag_env: ArrayRef = create_nullable_dict_array(&vec![Some("prod"); num_rows]); - let tag_datacenter: ArrayRef = - create_nullable_dict_array(&vec![Some("us-east-1"); num_rows]); - let tag_region: ArrayRef = create_nullable_dict_array(&vec![None; num_rows]); - let tag_host: ArrayRef = create_nullable_dict_array(&vec![Some("host-001"); num_rows]); - - // Create empty Variant (Struct with metadata and value BinaryView fields) - let metadata_array = StdArc::new(BinaryViewArray::from(vec![b"" as &[u8]; num_rows])); - let value_array = StdArc::new(BinaryViewArray::from(vec![b"" as &[u8]; num_rows])); - let attributes: ArrayRef = StdArc::new(StructArray::from(vec![ - ( - StdArc::new(Field::new("metadata", DataType::BinaryView, false)), - metadata_array.clone() as ArrayRef, + let schema = StdArc::new(ArrowSchema::new(vec![ + Field::new( + "metric_name", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + false, ), - ( - StdArc::new(Field::new("value", DataType::BinaryView, false)), - value_array.clone() as ArrayRef, + Field::new("metric_type", DataType::UInt8, false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("value", DataType::Float64, false), + Field::new( + "service", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + true, ), ])); - let service_name: ArrayRef = create_dict_array(&vec!["my-service"; num_rows]); - - let resource_attributes: ArrayRef = StdArc::new(StructArray::from(vec![ - ( - StdArc::new(Field::new("metadata", DataType::BinaryView, false)), - metadata_array as ArrayRef, - ), - ( - StdArc::new(Field::new("value", DataType::BinaryView, false)), - value_array as ArrayRef, - ), - ])); + let metric_name: ArrayRef = { + let keys = Int32Array::from(vec![0i32; num_rows]); + let vals = StringArray::from(vec!["cpu.usage"]); + StdArc::new(DictionaryArray::::try_new(keys, StdArc::new(vals)).unwrap()) + }; + let metric_type: ArrayRef = StdArc::new(UInt8Array::from(vec![0u8; num_rows])); + let timestamp_secs: ArrayRef = + StdArc::new(UInt64Array::from(vec![100u64, 101u64, 102u64])); + let value: ArrayRef = StdArc::new(Float64Array::from(vec![42.0, 43.0, 44.0])); + let service: ArrayRef = { + let keys = Int32Array::from(vec![0i32; num_rows]); + let vals = StringArray::from(vec!["web"]); + StdArc::new(DictionaryArray::::try_new(keys, StdArc::new(vals)).unwrap()) + }; let batch = RecordBatch::try_new( - schema.arrow_schema().clone(), - vec![ - metric_name, - metric_type, - metric_unit, - timestamp_secs, - start_timestamp_secs, - value, - tag_service, - tag_env, - tag_datacenter, - tag_region, - tag_host, - attributes, - service_name, - resource_attributes, - ], + schema, + vec![metric_name, metric_type, timestamp_secs, value, service], ) .unwrap(); @@ -625,10 +570,10 @@ mod tests { use std::sync::Arc as StdArc; use arrow::array::{ - ArrayRef, BinaryViewArray, DictionaryArray, Float64Array, Int32Array, StringArray, - StructArray, UInt8Array, UInt64Array, + ArrayRef, DictionaryArray, Float64Array, Int32Array, StringArray, UInt8Array, + UInt64Array, }; - use arrow::datatypes::{DataType, Field, Int32Type}; + use arrow::datatypes::{DataType, Field, Int32Type, Schema as ArrowSchema}; use arrow::record_batch::RecordBatch; use quickwit_parquet_engine::storage::{ParquetSplitWriter, ParquetWriterConfig}; use quickwit_proto::metastore::MockMetastoreService; @@ -657,9 +602,8 @@ mod tests { let (uploader_mailbox, _uploader_handle) = universe.spawn_builder().spawn(uploader); // Create ParquetPackager - let parquet_schema = ParquetSchema::new(); let writer_config = ParquetWriterConfig::default(); - let split_writer = ParquetSplitWriter::new(parquet_schema, writer_config, temp_dir.path()); + let split_writer = ParquetSplitWriter::new(writer_config, temp_dir.path()); let packager = ParquetPackager::new(split_writer, uploader_mailbox); let (packager_mailbox, packager_handle) = universe.spawn_builder().spawn(packager); @@ -681,100 +625,43 @@ mod tests { let (metrics_doc_processor_mailbox, metrics_doc_processor_handle) = universe.spawn_builder().spawn(metrics_doc_processor); - // Create a test batch - let schema = ParquetSchema::new(); + // Create a test batch with the 4 required fields plus a tag column let num_rows = 5; + let schema = StdArc::new(ArrowSchema::new(vec![ + Field::new( + "metric_name", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + false, + ), + Field::new("metric_type", DataType::UInt8, false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("value", DataType::Float64, false), + Field::new( + "service", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + true, + ), + ])); - fn create_dict_array(values: &[&str]) -> ArrayRef { - let keys: Vec = (0..values.len()).map(|i| i as i32).collect(); - let string_array = StringArray::from(values.to_vec()); - StdArc::new( - DictionaryArray::::try_new( - Int32Array::from(keys), - StdArc::new(string_array), - ) - .unwrap(), - ) - } - - fn create_nullable_dict_array(values: &[Option<&str>]) -> ArrayRef { - let keys: Vec> = values - .iter() - .enumerate() - .map(|(i, v)| v.map(|_| i as i32)) - .collect(); - let string_values: Vec<&str> = values.iter().filter_map(|v| *v).collect(); - let string_array = StringArray::from(string_values); - StdArc::new( - DictionaryArray::::try_new( - Int32Array::from(keys), - StdArc::new(string_array), - ) - .unwrap(), - ) - } - - let metric_name: ArrayRef = create_dict_array(&vec!["cpu.usage"; num_rows]); + let metric_name: ArrayRef = { + let keys = Int32Array::from(vec![0i32; num_rows]); + let vals = StringArray::from(vec!["cpu.usage"]); + StdArc::new(DictionaryArray::::try_new(keys, StdArc::new(vals)).unwrap()) + }; let metric_type: ArrayRef = StdArc::new(UInt8Array::from(vec![0u8; num_rows])); - let metric_unit: ArrayRef = StdArc::new(StringArray::from(vec![Some("bytes"); num_rows])); let timestamps: Vec = (0..num_rows).map(|i| 100 + i as u64).collect(); let timestamp_secs: ArrayRef = StdArc::new(UInt64Array::from(timestamps)); - let start_timestamp_secs: ArrayRef = - StdArc::new(UInt64Array::from(vec![None::; num_rows])); let values: Vec = (0..num_rows).map(|i| 42.0 + i as f64).collect(); let value: ArrayRef = StdArc::new(Float64Array::from(values)); - let tag_service: ArrayRef = create_nullable_dict_array(&vec![Some("web"); num_rows]); - let tag_env: ArrayRef = create_nullable_dict_array(&vec![Some("prod"); num_rows]); - let tag_datacenter: ArrayRef = - create_nullable_dict_array(&vec![Some("us-east-1"); num_rows]); - let tag_region: ArrayRef = create_nullable_dict_array(&vec![None; num_rows]); - let tag_host: ArrayRef = create_nullable_dict_array(&vec![Some("host-001"); num_rows]); - - // Create empty Variant (Struct with metadata and value BinaryView fields) - let metadata_array = StdArc::new(BinaryViewArray::from(vec![b"" as &[u8]; num_rows])); - let value_array = StdArc::new(BinaryViewArray::from(vec![b"" as &[u8]; num_rows])); - let attributes: ArrayRef = StdArc::new(StructArray::from(vec![ - ( - StdArc::new(Field::new("metadata", DataType::BinaryView, false)), - metadata_array.clone() as ArrayRef, - ), - ( - StdArc::new(Field::new("value", DataType::BinaryView, false)), - value_array.clone() as ArrayRef, - ), - ])); - - let service_name: ArrayRef = create_dict_array(&vec!["my-service"; num_rows]); - - let resource_attributes: ArrayRef = StdArc::new(StructArray::from(vec![ - ( - StdArc::new(Field::new("metadata", DataType::BinaryView, false)), - metadata_array as ArrayRef, - ), - ( - StdArc::new(Field::new("value", DataType::BinaryView, false)), - value_array as ArrayRef, - ), - ])); + let service: ArrayRef = { + let keys = Int32Array::from(vec![0i32; num_rows]); + let vals = StringArray::from(vec!["web"]); + StdArc::new(DictionaryArray::::try_new(keys, StdArc::new(vals)).unwrap()) + }; let batch = RecordBatch::try_new( - schema.arrow_schema().clone(), - vec![ - metric_name, - metric_type, - metric_unit, - timestamp_secs, - start_timestamp_secs, - value, - tag_service, - tag_env, - tag_datacenter, - tag_region, - tag_host, - attributes, - service_name, - resource_attributes, - ], + schema, + vec![metric_name, metric_type, timestamp_secs, value, service], ) .unwrap(); diff --git a/quickwit/quickwit-indexing/src/actors/parquet_e2e_test.rs b/quickwit/quickwit-indexing/src/actors/parquet_e2e_test.rs index 0823ce4ad04..d613fc96003 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_e2e_test.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_e2e_test.rs @@ -24,15 +24,13 @@ use std::time::Duration; use arrow::array::{ ArrayRef, DictionaryArray, Float64Array, Int32Array, StringArray, UInt8Array, UInt64Array, }; -use arrow::datatypes::Int32Type; +use arrow::datatypes::{DataType, Field, Int32Type, Schema as ArrowSchema}; use arrow::record_batch::RecordBatch; use bytes::Bytes; -use parquet::variant::{VariantArrayBuilder, VariantBuilderExt}; use quickwit_actors::{ActorHandle, Universe}; use quickwit_common::test_utils::wait_until_predicate; use quickwit_metastore::checkpoint::SourceCheckpointDelta; use quickwit_parquet_engine::ingest::record_batch_to_ipc; -use quickwit_parquet_engine::schema::ParquetSchema; use quickwit_parquet_engine::storage::{ParquetSplitWriter, ParquetWriterConfig}; use quickwit_proto::metastore::{EmptyResponse, MockMetastoreService}; use quickwit_proto::types::IndexUid; @@ -65,48 +63,6 @@ async fn wait_for_published_splits( .map_err(|_| anyhow::anyhow!("Timeout waiting for {} published splits", expected_splits)) } -fn create_dict_array(values: &[&str]) -> ArrayRef { - let keys: Vec = (0..values.len()).map(|i| i as i32).collect(); - let string_array = StringArray::from(values.to_vec()); - Arc::new( - DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) - .unwrap(), - ) -} - -fn create_nullable_dict_array(values: &[Option<&str>]) -> ArrayRef { - let keys: Vec> = values - .iter() - .enumerate() - .map(|(i, v)| v.map(|_| i as i32)) - .collect(); - let string_values: Vec<&str> = values.iter().filter_map(|v| *v).collect(); - let string_array = StringArray::from(string_values); - Arc::new( - DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) - .unwrap(), - ) -} - -fn create_variant_array(num_rows: usize, fields: Option<&[(&str, &str)]>) -> ArrayRef { - let mut builder = VariantArrayBuilder::new(num_rows); - for _ in 0..num_rows { - match fields { - Some(kv_pairs) => { - let mut obj = builder.new_object(); - for &(key, value) in kv_pairs { - obj = obj.with_field(key, value); - } - obj.finish(); - } - None => { - builder.append_null(); - } - } - } - ArrayRef::from(builder.build()) -} - fn create_test_batch( num_rows: usize, metric_name: &str, @@ -114,44 +70,46 @@ fn create_test_batch( base_timestamp: u64, base_value: f64, ) -> RecordBatch { - let schema = ParquetSchema::new(); - - let metric_names: Vec<&str> = vec![metric_name; num_rows]; - let metric_name_arr: ArrayRef = create_dict_array(&metric_names); + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new( + "metric_name", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + false, + ), + Field::new("metric_type", DataType::UInt8, false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("value", DataType::Float64, false), + Field::new( + "service", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + true, + ), + ])); + + let metric_name_arr: ArrayRef = { + let keys = Int32Array::from(vec![0i32; num_rows]); + let vals = StringArray::from(vec![metric_name]); + Arc::new(DictionaryArray::::try_new(keys, Arc::new(vals)).unwrap()) + }; let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; num_rows])); - let metric_unit: ArrayRef = Arc::new(StringArray::from(vec![Some("count"); num_rows])); let timestamps: Vec = (0..num_rows).map(|i| base_timestamp + i as u64).collect(); let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); - let start_timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(vec![None::; num_rows])); let values: Vec = (0..num_rows).map(|i| base_value + i as f64).collect(); let value: ArrayRef = Arc::new(Float64Array::from(values)); - let tag_service: ArrayRef = create_nullable_dict_array(&vec![Some(service); num_rows]); - let tag_env: ArrayRef = create_nullable_dict_array(&vec![Some("prod"); num_rows]); - let tag_datacenter: ArrayRef = create_nullable_dict_array(&vec![Some("us-east-1"); num_rows]); - let tag_region: ArrayRef = create_nullable_dict_array(&vec![None; num_rows]); - let tag_host: ArrayRef = create_nullable_dict_array(&vec![Some("host-001"); num_rows]); - let attributes: ArrayRef = create_variant_array(num_rows, None); - let service_names: Vec<&str> = vec![service; num_rows]; - let service_name: ArrayRef = create_dict_array(&service_names); - let resource_attributes: ArrayRef = create_variant_array(num_rows, None); + let service_arr: ArrayRef = { + let keys = Int32Array::from(vec![0i32; num_rows]); + let vals = StringArray::from(vec![service]); + Arc::new(DictionaryArray::::try_new(keys, Arc::new(vals)).unwrap()) + }; RecordBatch::try_new( - schema.arrow_schema().clone(), + schema, vec![ metric_name_arr, metric_type, - metric_unit, timestamp_secs, - start_timestamp_secs, value, - tag_service, - tag_env, - tag_datacenter, - tag_region, - tag_host, - attributes, - service_name, - resource_attributes, + service_arr, ], ) .unwrap() @@ -213,9 +171,8 @@ async fn test_metrics_pipeline_e2e() { let (uploader_mailbox, _uploader_handle) = universe.spawn_builder().spawn(uploader); // ParquetPackager between indexer and uploader - let parquet_schema = ParquetSchema::new(); let writer_config = ParquetWriterConfig::default(); - let split_writer = ParquetSplitWriter::new(parquet_schema, writer_config, temp_dir.path()); + let split_writer = ParquetSplitWriter::new(writer_config, temp_dir.path()); let packager = ParquetPackager::new(split_writer, uploader_mailbox); let (packager_mailbox, packager_handle) = universe.spawn_builder().spawn(packager); diff --git a/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs b/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs index 2738a42163f..a8dfe450f37 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs @@ -530,15 +530,8 @@ mod tests { use std::sync::atomic::Ordering; use std::time::Duration; - use arrow::array::{ - ArrayRef, DictionaryArray, Float64Array, Int32Array, StringArray, UInt8Array, UInt64Array, - }; - use arrow::datatypes::Int32Type; - use arrow::record_batch::RecordBatch; - use parquet::variant::{VariantArrayBuilder, VariantBuilderExt}; use quickwit_actors::{ActorHandle, Universe}; use quickwit_common::test_utils::wait_until_predicate; - use quickwit_parquet_engine::schema::ParquetSchema; use quickwit_parquet_engine::storage::{ParquetSplitWriter, ParquetWriterConfig}; use quickwit_proto::metastore::{EmptyResponse, MockMetastoreService}; use quickwit_storage::RamStorage; @@ -546,6 +539,7 @@ mod tests { use super::*; use crate::actors::{ ParquetPackager, ParquetPublisher, ParquetUploader, SplitsUpdateMailbox, UploaderType, + parquet_test_helpers::create_test_batch, }; /// Create a test ParquetUploader and return its mailbox. @@ -598,9 +592,8 @@ mod tests { temp_dir: &std::path::Path, uploader_mailbox: Mailbox, ) -> (Mailbox, ActorHandle) { - let schema = ParquetSchema::new(); let writer_config = ParquetWriterConfig::default(); - let split_writer = ParquetSplitWriter::new(schema, writer_config, temp_dir); + let split_writer = ParquetSplitWriter::new(writer_config, temp_dir); let packager = ParquetPackager::new(split_writer, uploader_mailbox); universe.spawn_builder().spawn(packager) @@ -624,96 +617,6 @@ mod tests { .map_err(|_| anyhow::anyhow!("Timeout waiting for {} staged splits", expected_splits)) } - /// Create dictionary array for string fields with Int32 keys. - fn create_dict_array(values: &[&str]) -> ArrayRef { - let keys: Vec = (0..values.len()).map(|i| i as i32).collect(); - let string_array = StringArray::from(values.to_vec()); - Arc::new( - DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) - .unwrap(), - ) - } - - /// Create nullable dictionary array for optional string fields. - fn create_nullable_dict_array(values: &[Option<&str>]) -> ArrayRef { - let keys: Vec> = values - .iter() - .enumerate() - .map(|(i, v)| v.map(|_| i as i32)) - .collect(); - let string_values: Vec<&str> = values.iter().filter_map(|v| *v).collect(); - let string_array = StringArray::from(string_values); - Arc::new( - DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) - .unwrap(), - ) - } - - /// Create a VARIANT array for testing with specified number of rows. - fn create_variant_array(num_rows: usize, fields: Option<&[(&str, &str)]>) -> ArrayRef { - let mut builder = VariantArrayBuilder::new(num_rows); - for _ in 0..num_rows { - match fields { - Some(kv_pairs) => { - let mut obj = builder.new_object(); - for &(key, value) in kv_pairs { - obj = obj.with_field(key, value); - } - obj.finish(); - } - None => { - builder.append_null(); - } - } - } - ArrayRef::from(builder.build()) - } - - /// Create a test batch matching the metrics schema. - fn create_test_batch(num_rows: usize) -> RecordBatch { - let schema = ParquetSchema::new(); - - let metric_names: Vec<&str> = vec!["cpu.usage"; num_rows]; - let metric_name: ArrayRef = create_dict_array(&metric_names); - let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; num_rows])); - let metric_unit: ArrayRef = Arc::new(StringArray::from(vec![Some("bytes"); num_rows])); - let timestamps: Vec = (0..num_rows).map(|i| 100 + i as u64).collect(); - let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); - let start_timestamp_secs: ArrayRef = - Arc::new(UInt64Array::from(vec![None::; num_rows])); - let values: Vec = (0..num_rows).map(|i| 42.0 + i as f64).collect(); - let value: ArrayRef = Arc::new(Float64Array::from(values)); - let tag_service: ArrayRef = create_nullable_dict_array(&vec![Some("web"); num_rows]); - let tag_env: ArrayRef = create_nullable_dict_array(&vec![Some("prod"); num_rows]); - let tag_datacenter: ArrayRef = - create_nullable_dict_array(&vec![Some("us-east-1"); num_rows]); - let tag_region: ArrayRef = create_nullable_dict_array(&vec![None; num_rows]); - let tag_host: ArrayRef = create_nullable_dict_array(&vec![Some("host-001"); num_rows]); - let attributes: ArrayRef = create_variant_array(num_rows, None); - let service_name: ArrayRef = create_dict_array(&vec!["my-service"; num_rows]); - let resource_attributes: ArrayRef = create_variant_array(num_rows, None); - - RecordBatch::try_new( - schema.arrow_schema().clone(), - vec![ - metric_name, - metric_type, - metric_unit, - timestamp_secs, - start_timestamp_secs, - value, - tag_service, - tag_env, - tag_datacenter, - tag_region, - tag_host, - attributes, - service_name, - resource_attributes, - ], - ) - .unwrap() - } #[tokio::test] async fn test_metrics_indexer_receives_batch() { diff --git a/quickwit/quickwit-indexing/src/actors/parquet_packager.rs b/quickwit/quickwit-indexing/src/actors/parquet_packager.rs index a23b88789e4..d0d39d05606 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_packager.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_packager.rs @@ -235,109 +235,18 @@ mod tests { use std::sync::atomic::Ordering as AtomicOrdering; use std::time::Duration; - use arrow::array::{ - ArrayRef, DictionaryArray, Float64Array, Int32Array, StringArray, UInt8Array, UInt64Array, - }; - use arrow::datatypes::Int32Type; - use arrow::record_batch::RecordBatch; - use parquet::variant::{VariantArrayBuilder, VariantBuilderExt}; use quickwit_actors::{ActorHandle, Universe}; use quickwit_common::test_utils::wait_until_predicate; use quickwit_metastore::checkpoint::{IndexCheckpointDelta, SourceCheckpointDelta}; - use quickwit_parquet_engine::schema::ParquetSchema; use quickwit_parquet_engine::storage::ParquetWriterConfig; use quickwit_proto::metastore::{EmptyResponse, MockMetastoreService}; use quickwit_storage::RamStorage; use super::*; - use crate::actors::{ParquetPublisher, SplitsUpdateMailbox, UploaderType}; - - fn create_dict_array(values: &[&str]) -> ArrayRef { - let keys: Vec = (0..values.len()).map(|i| i as i32).collect(); - let string_array = StringArray::from(values.to_vec()); - Arc::new( - DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) - .unwrap(), - ) - } - - fn create_nullable_dict_array(values: &[Option<&str>]) -> ArrayRef { - let keys: Vec> = values - .iter() - .enumerate() - .map(|(i, v)| v.map(|_| i as i32)) - .collect(); - let string_values: Vec<&str> = values.iter().filter_map(|v| *v).collect(); - let string_array = StringArray::from(string_values); - Arc::new( - DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) - .unwrap(), - ) - } - - fn create_variant_array(num_rows: usize, fields: Option<&[(&str, &str)]>) -> ArrayRef { - let mut builder = VariantArrayBuilder::new(num_rows); - for _ in 0..num_rows { - match fields { - Some(kv_pairs) => { - let mut obj = builder.new_object(); - for &(key, value) in kv_pairs { - obj = obj.with_field(key, value); - } - obj.finish(); - } - None => { - builder.append_null(); - } - } - } - ArrayRef::from(builder.build()) - } - - fn create_test_batch(num_rows: usize) -> RecordBatch { - let schema = ParquetSchema::new(); - - let metric_names: Vec<&str> = vec!["cpu.usage"; num_rows]; - let metric_name: ArrayRef = create_dict_array(&metric_names); - let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; num_rows])); - let metric_unit: ArrayRef = Arc::new(StringArray::from(vec![Some("bytes"); num_rows])); - let timestamps: Vec = (0..num_rows).map(|i| 100 + i as u64).collect(); - let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); - let start_timestamp_secs: ArrayRef = - Arc::new(UInt64Array::from(vec![None::; num_rows])); - let values: Vec = (0..num_rows).map(|i| 42.0 + i as f64).collect(); - let value: ArrayRef = Arc::new(Float64Array::from(values)); - let tag_service: ArrayRef = create_nullable_dict_array(&vec![Some("web"); num_rows]); - let tag_env: ArrayRef = create_nullable_dict_array(&vec![Some("prod"); num_rows]); - let tag_datacenter: ArrayRef = - create_nullable_dict_array(&vec![Some("us-east-1"); num_rows]); - let tag_region: ArrayRef = create_nullable_dict_array(&vec![None; num_rows]); - let tag_host: ArrayRef = create_nullable_dict_array(&vec![Some("host-001"); num_rows]); - let attributes: ArrayRef = create_variant_array(num_rows, None); - let service_name: ArrayRef = create_dict_array(&vec!["my-service"; num_rows]); - let resource_attributes: ArrayRef = create_variant_array(num_rows, None); - - RecordBatch::try_new( - schema.arrow_schema().clone(), - vec![ - metric_name, - metric_type, - metric_unit, - timestamp_secs, - start_timestamp_secs, - value, - tag_service, - tag_env, - tag_datacenter, - tag_region, - tag_host, - attributes, - service_name, - resource_attributes, - ], - ) - .unwrap() - } + use crate::actors::{ + ParquetPublisher, SplitsUpdateMailbox, UploaderType, + parquet_test_helpers::create_test_batch, + }; fn create_test_uploader( universe: &Universe, @@ -366,9 +275,8 @@ mod tests { temp_dir: &std::path::Path, uploader_mailbox: Mailbox, ) -> (Mailbox, ActorHandle) { - let schema = ParquetSchema::new(); let writer_config = ParquetWriterConfig::default(); - let split_writer = ParquetSplitWriter::new(schema, writer_config, temp_dir); + let split_writer = ParquetSplitWriter::new(writer_config, temp_dir); let packager = ParquetPackager::new(split_writer, uploader_mailbox); universe.spawn_builder().spawn(packager) diff --git a/quickwit/quickwit-indexing/src/models/processed_parquet_batch.rs b/quickwit/quickwit-indexing/src/models/processed_parquet_batch.rs index 0db83abcc02..e09b5d5d9a9 100644 --- a/quickwit/quickwit-indexing/src/models/processed_parquet_batch.rs +++ b/quickwit/quickwit-indexing/src/models/processed_parquet_batch.rs @@ -96,112 +96,10 @@ impl fmt::Debug for ProcessedParquetBatch { #[cfg(test)] mod tests { - use std::sync::Arc; - - use arrow::array::{ - ArrayRef, BinaryViewArray, DictionaryArray, Float64Array, Int32Array, StringArray, - StructArray, UInt8Array, UInt64Array, - }; - use arrow::datatypes::{DataType, Field, Int32Type}; - use quickwit_parquet_engine::schema::ParquetSchema; + use crate::actors::parquet_test_helpers::create_test_batch; use super::*; - /// Create dictionary array for string fields with Int32 keys. - fn create_dict_array(values: &[&str]) -> ArrayRef { - let keys: Vec = (0..values.len()).map(|i| i as i32).collect(); - let string_array = StringArray::from(values.to_vec()); - Arc::new( - DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) - .unwrap(), - ) - } - - /// Create nullable dictionary array for optional string fields. - fn create_nullable_dict_array(values: &[Option<&str>]) -> ArrayRef { - let keys: Vec> = values - .iter() - .enumerate() - .map(|(i, v)| v.map(|_| i as i32)) - .collect(); - let string_values: Vec<&str> = values.iter().filter_map(|v| *v).collect(); - let string_array = StringArray::from(string_values); - Arc::new( - DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) - .unwrap(), - ) - } - - /// Create a test batch matching the metrics schema. - fn create_test_batch(num_rows: usize) -> RecordBatch { - let schema = ParquetSchema::new(); - - let metric_names: Vec<&str> = vec!["cpu.usage"; num_rows]; - let metric_name: ArrayRef = create_dict_array(&metric_names); - let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; num_rows])); - let metric_unit: ArrayRef = Arc::new(StringArray::from(vec![Some("bytes"); num_rows])); - let timestamps: Vec = (0..num_rows).map(|i| 100 + i as u64).collect(); - let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); - let start_timestamp_secs: ArrayRef = - Arc::new(UInt64Array::from(vec![None::; num_rows])); - let values: Vec = (0..num_rows).map(|i| 42.0 + i as f64).collect(); - let value: ArrayRef = Arc::new(Float64Array::from(values)); - let tag_service: ArrayRef = create_nullable_dict_array(&vec![Some("web"); num_rows]); - let tag_env: ArrayRef = create_nullable_dict_array(&vec![Some("prod"); num_rows]); - let tag_datacenter: ArrayRef = - create_nullable_dict_array(&vec![Some("us-east-1"); num_rows]); - let tag_region: ArrayRef = create_nullable_dict_array(&vec![None; num_rows]); - let tag_host: ArrayRef = create_nullable_dict_array(&vec![Some("host-001"); num_rows]); - - // Create empty Variant (Struct with metadata and value BinaryView fields) - let metadata_array = Arc::new(BinaryViewArray::from(vec![b"" as &[u8]; num_rows])); - let value_array = Arc::new(BinaryViewArray::from(vec![b"" as &[u8]; num_rows])); - let attributes: ArrayRef = Arc::new(StructArray::from(vec![ - ( - Arc::new(Field::new("metadata", DataType::BinaryView, false)), - metadata_array.clone() as ArrayRef, - ), - ( - Arc::new(Field::new("value", DataType::BinaryView, false)), - value_array.clone() as ArrayRef, - ), - ])); - - let service_name: ArrayRef = create_dict_array(&vec!["my-service"; num_rows]); - - let resource_attributes: ArrayRef = Arc::new(StructArray::from(vec![ - ( - Arc::new(Field::new("metadata", DataType::BinaryView, false)), - metadata_array as ArrayRef, - ), - ( - Arc::new(Field::new("value", DataType::BinaryView, false)), - value_array as ArrayRef, - ), - ])); - - RecordBatch::try_new( - schema.arrow_schema().clone(), - vec![ - metric_name, - metric_type, - metric_unit, - timestamp_secs, - start_timestamp_secs, - value, - tag_service, - tag_env, - tag_datacenter, - tag_region, - tag_host, - attributes, - service_name, - resource_attributes, - ], - ) - .unwrap() - } - #[test] fn test_processed_parquet_batch_new() { let batch = create_test_batch(10); From 33c4070f2020320bcb6f5423a9a0094c1e3a700e Mon Sep 17 00:00:00 2001 From: Matthew Kim Date: Wed, 18 Mar 2026 11:02:50 -0400 Subject: [PATCH 7/8] refactor: deduplicate test batch helpers --- quickwit/quickwit-indexing/Cargo.toml | 1 + .../src/actors/parquet_doc_processor.rs | 98 +------------------ .../src/actors/parquet_indexer.rs | 3 +- .../src/actors/parquet_packager.rs | 7 +- .../src/models/processed_parquet_batch.rs | 2 +- quickwit/quickwit-opentelemetry/Cargo.toml | 1 - quickwit/quickwit-parquet-engine/src/lib.rs | 2 +- .../src/test_helpers.rs | 96 ++++++++++++++++++ 8 files changed, 108 insertions(+), 102 deletions(-) create mode 100644 quickwit/quickwit-parquet-engine/src/test_helpers.rs diff --git a/quickwit/quickwit-indexing/Cargo.toml b/quickwit/quickwit-indexing/Cargo.toml index 677b95e32aa..5d3a6504d29 100644 --- a/quickwit/quickwit-indexing/Cargo.toml +++ b/quickwit/quickwit-indexing/Cargo.toml @@ -123,6 +123,7 @@ sqlx = { workspace = true, features = ["runtime-tokio", "postgres"] } tempfile = { workspace = true } quickwit-actors = { workspace = true, features = ["testsuite"] } +quickwit-parquet-engine = { workspace = true, features = ["testsuite"] } quickwit-cluster = { workspace = true, features = ["testsuite"] } quickwit-common = { workspace = true, features = ["testsuite"] } quickwit-config = { workspace = true, features = ["testsuite"] } diff --git a/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs b/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs index aa26f67fc94..11265b52c9f 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs @@ -401,14 +401,8 @@ mod tests { #[tokio::test] async fn test_metrics_doc_processor_valid_arrow_ipc() { - use std::sync::Arc as StdArc; + use quickwit_parquet_engine::test_helpers::create_test_batch_with_tags; - use arrow::array::{ - ArrayRef, DictionaryArray, Float64Array, Int32Array, StringArray, UInt8Array, - UInt64Array, - }; - use arrow::datatypes::{DataType, Field, Int32Type, Schema as ArrowSchema}; - use arrow::record_batch::RecordBatch; let universe = Universe::with_accelerated_time(); let (indexer_mailbox, _indexer_inbox) = universe.create_test_mailbox::(); @@ -421,46 +415,7 @@ mod tests { let (metrics_doc_processor_mailbox, metrics_doc_processor_handle) = universe.spawn_builder().spawn(metrics_doc_processor); - // Create a test batch with the 4 required fields plus a tag column - let num_rows = 3; - let schema = StdArc::new(ArrowSchema::new(vec![ - Field::new( - "metric_name", - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), - false, - ), - Field::new("metric_type", DataType::UInt8, false), - Field::new("timestamp_secs", DataType::UInt64, false), - Field::new("value", DataType::Float64, false), - Field::new( - "service", - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), - true, - ), - ])); - - let metric_name: ArrayRef = { - let keys = Int32Array::from(vec![0i32; num_rows]); - let vals = StringArray::from(vec!["cpu.usage"]); - StdArc::new(DictionaryArray::::try_new(keys, StdArc::new(vals)).unwrap()) - }; - let metric_type: ArrayRef = StdArc::new(UInt8Array::from(vec![0u8; num_rows])); - let timestamp_secs: ArrayRef = - StdArc::new(UInt64Array::from(vec![100u64, 101u64, 102u64])); - let value: ArrayRef = StdArc::new(Float64Array::from(vec![42.0, 43.0, 44.0])); - let service: ArrayRef = { - let keys = Int32Array::from(vec![0i32; num_rows]); - let vals = StringArray::from(vec!["web"]); - StdArc::new(DictionaryArray::::try_new(keys, StdArc::new(vals)).unwrap()) - }; - - let batch = RecordBatch::try_new( - schema, - vec![metric_name, metric_type, timestamp_secs, value, service], - ) - .unwrap(); - - // Serialize to Arrow IPC + let batch = create_test_batch_with_tags(3, &["service"]); let ipc_bytes = record_batch_to_ipc(&batch).unwrap(); // Create RawDocBatch with the IPC bytes @@ -569,13 +524,8 @@ mod tests { async fn test_metrics_doc_processor_with_indexer() { use std::sync::Arc as StdArc; - use arrow::array::{ - ArrayRef, DictionaryArray, Float64Array, Int32Array, StringArray, UInt8Array, - UInt64Array, - }; - use arrow::datatypes::{DataType, Field, Int32Type, Schema as ArrowSchema}; - use arrow::record_batch::RecordBatch; use quickwit_parquet_engine::storage::{ParquetSplitWriter, ParquetWriterConfig}; + use quickwit_parquet_engine::test_helpers::create_test_batch_with_tags; use quickwit_proto::metastore::MockMetastoreService; use quickwit_storage::RamStorage; @@ -625,47 +575,7 @@ mod tests { let (metrics_doc_processor_mailbox, metrics_doc_processor_handle) = universe.spawn_builder().spawn(metrics_doc_processor); - // Create a test batch with the 4 required fields plus a tag column - let num_rows = 5; - let schema = StdArc::new(ArrowSchema::new(vec![ - Field::new( - "metric_name", - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), - false, - ), - Field::new("metric_type", DataType::UInt8, false), - Field::new("timestamp_secs", DataType::UInt64, false), - Field::new("value", DataType::Float64, false), - Field::new( - "service", - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), - true, - ), - ])); - - let metric_name: ArrayRef = { - let keys = Int32Array::from(vec![0i32; num_rows]); - let vals = StringArray::from(vec!["cpu.usage"]); - StdArc::new(DictionaryArray::::try_new(keys, StdArc::new(vals)).unwrap()) - }; - let metric_type: ArrayRef = StdArc::new(UInt8Array::from(vec![0u8; num_rows])); - let timestamps: Vec = (0..num_rows).map(|i| 100 + i as u64).collect(); - let timestamp_secs: ArrayRef = StdArc::new(UInt64Array::from(timestamps)); - let values: Vec = (0..num_rows).map(|i| 42.0 + i as f64).collect(); - let value: ArrayRef = StdArc::new(Float64Array::from(values)); - let service: ArrayRef = { - let keys = Int32Array::from(vec![0i32; num_rows]); - let vals = StringArray::from(vec!["web"]); - StdArc::new(DictionaryArray::::try_new(keys, StdArc::new(vals)).unwrap()) - }; - - let batch = RecordBatch::try_new( - schema, - vec![metric_name, metric_type, timestamp_secs, value, service], - ) - .unwrap(); - - // Serialize to Arrow IPC + let batch = create_test_batch_with_tags(5, &["service"]); let ipc_bytes = record_batch_to_ipc(&batch).unwrap(); // Create RawDocBatch with force_commit to trigger split production diff --git a/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs b/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs index a8dfe450f37..89d3d7c8e21 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs @@ -537,9 +537,10 @@ mod tests { use quickwit_storage::RamStorage; use super::*; + use quickwit_parquet_engine::test_helpers::create_test_batch; + use crate::actors::{ ParquetPackager, ParquetPublisher, ParquetUploader, SplitsUpdateMailbox, UploaderType, - parquet_test_helpers::create_test_batch, }; /// Create a test ParquetUploader and return its mailbox. diff --git a/quickwit/quickwit-indexing/src/actors/parquet_packager.rs b/quickwit/quickwit-indexing/src/actors/parquet_packager.rs index d0d39d05606..f0e5fe8dc76 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_packager.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_packager.rs @@ -243,10 +243,9 @@ mod tests { use quickwit_storage::RamStorage; use super::*; - use crate::actors::{ - ParquetPublisher, SplitsUpdateMailbox, UploaderType, - parquet_test_helpers::create_test_batch, - }; + use quickwit_parquet_engine::test_helpers::create_test_batch; + + use crate::actors::{ParquetPublisher, SplitsUpdateMailbox, UploaderType}; fn create_test_uploader( universe: &Universe, diff --git a/quickwit/quickwit-indexing/src/models/processed_parquet_batch.rs b/quickwit/quickwit-indexing/src/models/processed_parquet_batch.rs index e09b5d5d9a9..c70afe35976 100644 --- a/quickwit/quickwit-indexing/src/models/processed_parquet_batch.rs +++ b/quickwit/quickwit-indexing/src/models/processed_parquet_batch.rs @@ -96,7 +96,7 @@ impl fmt::Debug for ProcessedParquetBatch { #[cfg(test)] mod tests { - use crate::actors::parquet_test_helpers::create_test_batch; + use quickwit_parquet_engine::test_helpers::create_test_batch; use super::*; diff --git a/quickwit/quickwit-opentelemetry/Cargo.toml b/quickwit/quickwit-opentelemetry/Cargo.toml index d6e897990a2..a7432ac5403 100644 --- a/quickwit/quickwit-opentelemetry/Cargo.toml +++ b/quickwit/quickwit-opentelemetry/Cargo.toml @@ -15,7 +15,6 @@ anyhow = { workspace = true } arrow = { workspace = true } async-trait = { workspace = true } once_cell = { workspace = true } -parquet = { workspace = true } prost = { workspace = true } serde = { workspace = true } serde_json = { workspace = true } diff --git a/quickwit/quickwit-parquet-engine/src/lib.rs b/quickwit/quickwit-parquet-engine/src/lib.rs index c6d8640e801..309ebf4f442 100644 --- a/quickwit/quickwit-parquet-engine/src/lib.rs +++ b/quickwit/quickwit-parquet-engine/src/lib.rs @@ -27,5 +27,5 @@ pub mod schema; pub mod split; pub mod storage; -#[cfg(test)] +#[cfg(any(test, feature = "testsuite"))] pub mod test_helpers; diff --git a/quickwit/quickwit-parquet-engine/src/test_helpers.rs b/quickwit/quickwit-parquet-engine/src/test_helpers.rs new file mode 100644 index 00000000000..dd6892dbe23 --- /dev/null +++ b/quickwit/quickwit-parquet-engine/src/test_helpers.rs @@ -0,0 +1,96 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Shared test helpers for building Arrow RecordBatches in unit tests. + +use std::sync::Arc; + +use arrow::array::{ + ArrayRef, DictionaryArray, Float64Array, Int32Array, StringArray, UInt8Array, UInt64Array, +}; +use arrow::datatypes::{DataType, Field, Int32Type, Schema as ArrowSchema}; +use arrow::record_batch::RecordBatch; + +/// Creates a dictionary-encoded string array with compact 0-based keys. +pub fn create_dict_array(values: &[&str]) -> ArrayRef { + let keys: Vec = (0..values.len()).map(|i| i as i32).collect(); + let string_array = StringArray::from(values.to_vec()); + Arc::new( + DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) + .unwrap(), + ) +} + +/// Creates a nullable dictionary-encoded string array. +/// +/// Each `Some(value)` gets a key into the dictionary; `None` values produce +/// a null key. +pub fn create_nullable_dict_array(values: &[Option<&str>]) -> ArrayRef { + let keys: Vec> = values + .iter() + .enumerate() + .map(|(i, v)| v.map(|_| i as i32)) + .collect(); + let string_values: Vec<&str> = values.iter().filter_map(|v| *v).collect(); + let string_array = StringArray::from(string_values); + Arc::new( + DictionaryArray::::try_new(Int32Array::from(keys), Arc::new(string_array)) + .unwrap(), + ) +} + +/// Creates a RecordBatch with the 4 required fields plus the specified +/// nullable dictionary-encoded tag columns. +/// +/// - `metric_name`: all rows set to `"cpu.usage"` +/// - `metric_type`: all rows `0` (Gauge) +/// - `timestamp_secs`: sequential, starting at `100` +/// - `value`: sequential `f64` starting at `42.0` +/// - each tag column: all rows set to the column name as the value +pub fn create_test_batch_with_tags(num_rows: usize, tags: &[&str]) -> RecordBatch { + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + + let mut fields = vec![ + Field::new("metric_name", dict_type.clone(), false), + Field::new("metric_type", DataType::UInt8, false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("value", DataType::Float64, false), + ]; + for tag in tags { + fields.push(Field::new(*tag, dict_type.clone(), true)); + } + let schema = Arc::new(ArrowSchema::new(fields)); + + let metric_names: Vec<&str> = vec!["cpu.usage"; num_rows]; + let metric_name: ArrayRef = create_dict_array(&metric_names); + let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; num_rows])); + let timestamps: Vec = (0..num_rows).map(|i| 100 + i as u64).collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let values: Vec = (0..num_rows).map(|i| 42.0 + i as f64).collect(); + let value: ArrayRef = Arc::new(Float64Array::from(values)); + + let mut columns: Vec = vec![metric_name, metric_type, timestamp_secs, value]; + for tag in tags { + let tag_values: Vec> = vec![Some(tag); num_rows]; + columns.push(create_nullable_dict_array(&tag_values)); + } + + RecordBatch::try_new(schema, columns).unwrap() +} + +/// Creates a RecordBatch with the 4 required fields and default tags +/// (`service`, `host`). +pub fn create_test_batch(num_rows: usize) -> RecordBatch { + create_test_batch_with_tags(num_rows, &["service", "host"]) +} From 7a5979fe95d390f94e523203ae83719358aba4e5 Mon Sep 17 00:00:00 2001 From: Matthew Kim Date: Mon, 30 Mar 2026 14:12:46 -0400 Subject: [PATCH 8/8] lint --- quickwit/Cargo.lock | 1 - .../src/actors/parquet_doc_processor.rs | 10 +++------- .../src/actors/parquet_indexer.rs | 4 +--- .../src/actors/parquet_packager.rs | 3 +-- .../src/otlp/arrow_metrics.rs | 5 ++--- .../src/otlp/otel_metrics.rs | 13 +++---------- .../src/index/accumulator.rs | 15 ++++++++++----- .../src/ingest/processor.rs | 8 +++++--- .../quickwit-parquet-engine/src/schema/fields.rs | 6 ++++-- .../src/storage/config.rs | 3 +-- .../src/storage/split_writer.rs | 5 +++-- .../src/storage/writer.rs | 16 ++++++++++++---- 12 files changed, 45 insertions(+), 44 deletions(-) diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 684303cb6a8..f686efc5e11 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7591,7 +7591,6 @@ dependencies = [ "arrow", "async-trait", "once_cell", - "parquet", "prost 0.14.3", "quickwit-common", "quickwit-config", diff --git a/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs b/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs index 11265b52c9f..eb51621a30f 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs @@ -143,10 +143,7 @@ impl ParquetDocProcessor { indexer_mailbox: Mailbox, ) -> Self { let processor = ParquetIngestProcessor; - let counters = ParquetDocProcessorCounters::new( - index_id.clone(), - source_id.clone(), - ); + let counters = ParquetDocProcessorCounters::new(index_id.clone(), source_id.clone()); info!( index_id = %index_id, @@ -306,9 +303,8 @@ impl Handler for ParquetDocProcessor { // Without this, a batch of consistently malformed data blocks offset progress // forever. if !checkpoint_forwarded && !checkpoint_delta.is_empty() { - let empty_batch = RecordBatch::new_empty(std::sync::Arc::new( - arrow::datatypes::Schema::empty(), - )); + let empty_batch = + RecordBatch::new_empty(std::sync::Arc::new(arrow::datatypes::Schema::empty())); let processed_batch = ProcessedParquetBatch::new(empty_batch, checkpoint_delta, force_commit); ctx.send_message(&self.indexer_mailbox, processed_batch) diff --git a/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs b/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs index 89d3d7c8e21..254ff2dc719 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs @@ -533,12 +533,11 @@ mod tests { use quickwit_actors::{ActorHandle, Universe}; use quickwit_common::test_utils::wait_until_predicate; use quickwit_parquet_engine::storage::{ParquetSplitWriter, ParquetWriterConfig}; + use quickwit_parquet_engine::test_helpers::create_test_batch; use quickwit_proto::metastore::{EmptyResponse, MockMetastoreService}; use quickwit_storage::RamStorage; use super::*; - use quickwit_parquet_engine::test_helpers::create_test_batch; - use crate::actors::{ ParquetPackager, ParquetPublisher, ParquetUploader, SplitsUpdateMailbox, UploaderType, }; @@ -618,7 +617,6 @@ mod tests { .map_err(|_| anyhow::anyhow!("Timeout waiting for {} staged splits", expected_splits)) } - #[tokio::test] async fn test_metrics_indexer_receives_batch() { let universe = Universe::with_accelerated_time(); diff --git a/quickwit/quickwit-indexing/src/actors/parquet_packager.rs b/quickwit/quickwit-indexing/src/actors/parquet_packager.rs index f0e5fe8dc76..b0950141b10 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_packager.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_packager.rs @@ -239,12 +239,11 @@ mod tests { use quickwit_common::test_utils::wait_until_predicate; use quickwit_metastore::checkpoint::{IndexCheckpointDelta, SourceCheckpointDelta}; use quickwit_parquet_engine::storage::ParquetWriterConfig; + use quickwit_parquet_engine::test_helpers::create_test_batch; use quickwit_proto::metastore::{EmptyResponse, MockMetastoreService}; use quickwit_storage::RamStorage; use super::*; - use quickwit_parquet_engine::test_helpers::create_test_batch; - use crate::actors::{ParquetPublisher, SplitsUpdateMailbox, UploaderType}; fn create_test_uploader( diff --git a/quickwit/quickwit-opentelemetry/src/otlp/arrow_metrics.rs b/quickwit/quickwit-opentelemetry/src/otlp/arrow_metrics.rs index 8a4b3142501..d36c2f1b98b 100644 --- a/quickwit/quickwit-opentelemetry/src/otlp/arrow_metrics.rs +++ b/quickwit/quickwit-opentelemetry/src/otlp/arrow_metrics.rs @@ -24,7 +24,7 @@ use std::io::Cursor; use std::sync::Arc; use arrow::array::{ - ArrayRef, Float64Builder, RecordBatch, StringDictionaryBuilder, UInt64Builder, UInt8Builder, + ArrayRef, Float64Builder, RecordBatch, StringDictionaryBuilder, UInt8Builder, UInt64Builder, }; use arrow::datatypes::{DataType, Field, Int32Type, Schema as ArrowSchema}; use arrow::ipc::reader::StreamReader; @@ -130,8 +130,7 @@ impl ArrowMetricsBatchBuilder { arrays.push(Arc::new(tag_builder.finish())); } - RecordBatch::try_new(schema, arrays) - .expect("record batch should match Arrow schema") + RecordBatch::try_new(schema, arrays).expect("record batch should match Arrow schema") } /// Returns the number of rows appended so far. diff --git a/quickwit/quickwit-opentelemetry/src/otlp/otel_metrics.rs b/quickwit/quickwit-opentelemetry/src/otlp/otel_metrics.rs index 1a19e5fea53..17eab282bc1 100644 --- a/quickwit/quickwit-opentelemetry/src/otlp/otel_metrics.rs +++ b/quickwit/quickwit-opentelemetry/src/otlp/otel_metrics.rs @@ -246,8 +246,7 @@ impl OtlpGrpcMetricsService { let num_data_points = data_points.len() as u64 + num_rejected; // Build Arrow RecordBatch from valid data points - let mut arrow_builder = - ArrowMetricsBatchBuilder::with_capacity(data_points.len()); + let mut arrow_builder = ArrowMetricsBatchBuilder::with_capacity(data_points.len()); let mut doc_uid_generator = DocUidGenerator::default(); let mut doc_uids = Vec::with_capacity(data_points.len()); @@ -746,10 +745,7 @@ mod tests { assert_eq!(dp.tags.get("metric_unit").map(|s| s.as_str()), Some("1")); assert_eq!(dp.timestamp_secs, 2); assert_eq!(dp.value, 100.0); // int converted to f64 - assert_eq!( - dp.tags.get("host").map(|s| s.as_str()), - Some("server-1") - ); + assert_eq!(dp.tags.get("host").map(|s| s.as_str()), Some("server-1")); assert_eq!( dp.tags.get("service").map(|s| s.as_str()), Some("counter-service") @@ -844,10 +840,7 @@ mod tests { assert_eq!(dp.tags.get("service").map(|s| s.as_str()), Some("test")); // Verify data point attributes are in tags as strings - assert_eq!( - dp.tags.get("string_tag").map(|s| s.as_str()), - Some("value") - ); + assert_eq!(dp.tags.get("string_tag").map(|s| s.as_str()), Some("value")); } /// Test metrics with empty and missing values diff --git a/quickwit/quickwit-parquet-engine/src/index/accumulator.rs b/quickwit/quickwit-parquet-engine/src/index/accumulator.rs index d9eb01f4411..817f3f16d93 100644 --- a/quickwit/quickwit-parquet-engine/src/index/accumulator.rs +++ b/quickwit/quickwit-parquet-engine/src/index/accumulator.rs @@ -18,7 +18,7 @@ use std::collections::BTreeMap; use std::sync::Arc; use std::time::Instant; -use arrow::array::{new_null_array, ArrayRef}; +use arrow::array::{ArrayRef, new_null_array}; use arrow::compute::concat_batches; use arrow::datatypes::{DataType, Field, Schema as ArrowSchema, SchemaRef}; use arrow::record_batch::RecordBatch; @@ -233,9 +233,8 @@ fn estimate_batch_bytes(batch: &RecordBatch) -> usize { #[cfg(test)] mod tests { - use crate::test_helpers::{create_test_batch, create_test_batch_with_tags}; - use super::*; + use crate::test_helpers::{create_test_batch, create_test_batch_with_tags}; #[test] fn test_accumulator_below_threshold() { @@ -340,7 +339,10 @@ mod tests { // No duplicate column names — each name appears exactly once. let field_names: Vec<&str> = schema.fields().iter().map(|f| f.name().as_str()).collect(); - let unique_count = field_names.iter().collect::>().len(); + let unique_count = field_names + .iter() + .collect::>() + .len(); assert_eq!( unique_count, field_names.len(), @@ -365,7 +367,10 @@ mod tests { let schema = combined.schema(); let field_names: Vec<&str> = schema.fields().iter().map(|f| f.name().as_str()).collect(); - let unique_count = field_names.iter().collect::>().len(); + let unique_count = field_names + .iter() + .collect::>() + .len(); assert_eq!( unique_count, field_names.len(), diff --git a/quickwit/quickwit-parquet-engine/src/ingest/processor.rs b/quickwit/quickwit-parquet-engine/src/ingest/processor.rs index 6097d7c3c32..21412414b2b 100644 --- a/quickwit/quickwit-parquet-engine/src/ingest/processor.rs +++ b/quickwit/quickwit-parquet-engine/src/ingest/processor.rs @@ -111,7 +111,10 @@ fn ipc_to_record_batch(ipc_bytes: &[u8]) -> Result { return Err(IngestError::UnexpectedBatchCount(batches.len())); } - Ok(batches.into_iter().next().expect("len verified to be 1 above")) + Ok(batches + .into_iter() + .next() + .expect("len verified to be 1 above")) } /// Serialize a RecordBatch to Arrow IPC stream format. @@ -129,9 +132,8 @@ pub fn record_batch_to_ipc(batch: &RecordBatch) -> Result, IngestError> #[cfg(test)] mod tests { - use crate::test_helpers::create_test_batch; - use super::*; + use crate::test_helpers::create_test_batch; #[test] fn test_process_ipc() { diff --git a/quickwit/quickwit-parquet-engine/src/schema/fields.rs b/quickwit/quickwit-parquet-engine/src/schema/fields.rs index a8646823017..9f46dcf3b8c 100644 --- a/quickwit/quickwit-parquet-engine/src/schema/fields.rs +++ b/quickwit/quickwit-parquet-engine/src/schema/fields.rs @@ -14,7 +14,7 @@ //! Parquet field definitions with sort order constants and validation. -use anyhow::{bail, Result}; +use anyhow::{Result, bail}; use arrow::datatypes::DataType; /// Required field names that must exist in every batch. @@ -55,7 +55,9 @@ pub fn validate_required_fields(schema: &arrow::datatypes::Schema) -> Result<()> if *actual_type != expected_type { bail!( "field '{}' has type {:?}, expected {:?}", - name, actual_type, expected_type + name, + actual_type, + expected_type ); } } diff --git a/quickwit/quickwit-parquet-engine/src/storage/config.rs b/quickwit/quickwit-parquet-engine/src/storage/config.rs index a10a423cf9c..13b8d395a44 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/config.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/config.rs @@ -169,8 +169,7 @@ impl ParquetWriterConfig { // Enable bloom filters on dictionary-typed metric_name and sort order tag columns. // Exclude non-dictionary columns, like timestamp_secs. let is_bloom_column = matches!(field.data_type(), DataType::Dictionary(_, _)) - && (field.name() == "metric_name" - || SORT_ORDER.contains(&field.name().as_str())); + && (field.name() == "metric_name" || SORT_ORDER.contains(&field.name().as_str())); if is_bloom_column { let ndv = if field.name() == "metric_name" { BLOOM_FILTER_NDV_METRIC_NAME diff --git a/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs b/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs index a9e081455d0..466e2c9cdcb 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs @@ -219,11 +219,12 @@ mod tests { use arrow::array::{ArrayRef, Float64Array, UInt8Array, UInt64Array}; use arrow::datatypes::{DataType, Field, Schema as ArrowSchema}; - use crate::test_helpers::{create_dict_array, create_nullable_dict_array}; use super::*; + use crate::test_helpers::{create_dict_array, create_nullable_dict_array}; - /// Create a test batch with required fields, optional service column, and specified tag columns. + /// Create a test batch with required fields, optional service column, and specified tag + /// columns. fn create_test_batch_with_options( num_rows: usize, metric_names: &[&str], diff --git a/quickwit/quickwit-parquet-engine/src/storage/writer.rs b/quickwit/quickwit-parquet-engine/src/storage/writer.rs index 134702103eb..6f29c0be4cc 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/writer.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/writer.rs @@ -161,11 +161,13 @@ impl ParquetWriter { mod tests { use std::sync::Arc; - use arrow::array::{ArrayRef, DictionaryArray, Float64Array, StringArray, UInt64Array, UInt8Array}; + use arrow::array::{ + ArrayRef, DictionaryArray, Float64Array, StringArray, UInt8Array, UInt64Array, + }; use arrow::datatypes::{DataType, Field, Int32Type, Schema}; - use crate::test_helpers::create_test_batch_with_tags; use super::*; + use crate::test_helpers::create_test_batch_with_tags; fn create_test_batch() -> RecordBatch { create_test_batch_with_tags(1, &["service", "env"]) @@ -224,7 +226,10 @@ mod tests { .unwrap(); let result = writer.write_to_bytes(&wrong_batch); - assert!(matches!(result, Err(ParquetWriteError::SchemaValidation(_)))); + assert!(matches!( + result, + Err(ParquetWriteError::SchemaValidation(_)) + )); } #[test] @@ -251,7 +256,10 @@ mod tests { .unwrap(); let result = writer.write_to_bytes(&wrong_batch); - assert!(matches!(result, Err(ParquetWriteError::SchemaValidation(_)))); + assert!(matches!( + result, + Err(ParquetWriteError::SchemaValidation(_)) + )); } #[test]