From 8ff1095fa54d237b6efd3e5e3257a2c60b27d39e Mon Sep 17 00:00:00 2001 From: George Talbot Date: Mon, 11 May 2026 14:06:38 -0400 Subject: [PATCH 01/10] =?UTF-8?q?feat:=20ColumnPageStream=20trait=20?= =?UTF-8?q?=E2=80=94=20single=20contract=20for=20streaming=20+=20legacy=20?= =?UTF-8?q?inputs=20(#6406)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Extracts a `pub trait ColumnPageStream` from PR-4's concrete reader so PR-5 (legacy adapter) and PR-6 (streaming merge engine) can land in parallel against a stable contract instead of one rewriting the other. Trait shape: fn metadata(&self) -> &Arc async fn next_page(&mut self) -> Result, ParquetReadError> Same invariants PR-4 already guarantees: row-group-major / column-major-within-RG / page-major-within-column yield order, idempotent EOF, and I/O failures surface as ParquetReadError::Io rather than being masked as decode errors. Implements the trait for `StreamingParquetReader`. Promotes visibility of `Page`, `ParquetReadError`, `RemoteByteSource`, `StreamingParquetReader`, and `StreamingReaderConfig` from `pub(crate)` to `pub` so downstream crates and PR-5/PR-6 can consume them. Adds `test_streaming_reader_satisfies_column_page_stream_trait`: drains the same fixture through both the concrete-typed and trait-object surfaces, asserts identical (rg_idx, col_idx, page_idx_in_col, compressed_page_size) sequences and idempotent EOF through the trait. Pure refactor — no behavior change. Co-authored-by: Claude Opus 4.7 (1M context) --- .../src/storage/mod.rs | 4 + .../src/storage/streaming_reader.rs | 117 +++++++++++++++++- 2 files changed, 116 insertions(+), 5 deletions(-) diff --git a/quickwit/quickwit-parquet-engine/src/storage/mod.rs b/quickwit/quickwit-parquet-engine/src/storage/mod.rs index 43fe5861bca..4f02dfabc75 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/mod.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/mod.rs @@ -27,6 +27,10 @@ pub use inspect::{ verify_partition_prefix, }; pub use split_writer::ParquetSplitWriter; +pub use streaming_reader::{ + ColumnPageStream, Page, ParquetReadError, RemoteByteSource, StreamingParquetReader, + StreamingReaderConfig, +}; // Re-export metadata constants for use by the merge module and tests. pub(crate) use writer::{ PARQUET_META_NUM_MERGE_OPS, PARQUET_META_RG_PARTITION_PREFIX_LEN, PARQUET_META_ROW_KEYS, diff --git a/quickwit/quickwit-parquet-engine/src/storage/streaming_reader.rs b/quickwit/quickwit-parquet-engine/src/storage/streaming_reader.rs index 50eea40e94f..a88f51183b2 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/streaming_reader.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/streaming_reader.rs @@ -66,7 +66,7 @@ use tokio::io::{AsyncRead, AsyncReadExt}; /// no business pulling in). Callers in `quickwit-indexing` provide a /// thin adapter that delegates to `quickwit_storage::Storage`. #[async_trait] -pub(crate) trait RemoteByteSource: Send + Sync { +pub trait RemoteByteSource: Send + Sync { /// Total file length in bytes. async fn file_size(&self, path: &Path) -> io::Result; @@ -85,7 +85,7 @@ pub(crate) trait RemoteByteSource: Send + Sync { /// Configuration for [`StreamingParquetReader`]. #[derive(Debug, Clone)] -pub(crate) struct StreamingReaderConfig { +pub struct StreamingReaderConfig { /// Bytes prefetched from the file tail to capture the footer. /// Default 256 KiB — sized for a 50 MB metrics split with the /// writer config we ship in production. @@ -107,7 +107,7 @@ impl Default for StreamingReaderConfig { /// Errors from the streaming reader. #[derive(Error, Debug)] -pub(crate) enum ParquetReadError { +pub enum ParquetReadError { /// I/O error from the underlying [`RemoteByteSource`]. #[error("io error: {0}")] Io(#[from] io::Error), @@ -162,7 +162,7 @@ pub(crate) enum ParquetReadError { /// an output writer (PR-6's direct page copy) or decompress + decode /// for sort-column inspection. #[derive(Debug)] -pub(crate) struct Page { +pub struct Page { /// Row group this page belongs to. pub rg_idx: usize, /// Column chunk this page belongs to (within the row group). @@ -177,12 +177,53 @@ pub(crate) struct Page { pub bytes: Bytes, } +/// Object-safe page stream — the contract that PR-6's merge engine +/// consumes for every input file. +/// +/// Two implementations: +/// - [`StreamingParquetReader`]: streams pages directly from a remote byte source (the new-format +/// fast path). +/// - The legacy adapter (PR-5): buffers a whole file into a `RecordBatch`, re-encodes it as a +/// single-row-group parquet stream in memory, and presents it through this trait — used when an +/// input's row-group boundaries do not align with the sort prefix (`qh.rg_partition_prefix_len == +/// 0` AND `num_row_groups > 1`). +/// +/// # Contract +/// - [`Self::metadata`] returns the file's parsed metadata. Callable any time after construction; +/// does not consume the stream. +/// - [`Self::next_page`] yields pages in storage order: row-group-major, +/// column-major-within-row-group, page-major-within-column. Returns `Ok(None)` once the file is +/// fully drained, and stays `Ok(None)` on subsequent calls (idempotent EOF). +/// - I/O failures surface as [`ParquetReadError::Io`]; they are not masked as decode errors. +#[async_trait] +pub trait ColumnPageStream: Send { + /// Parsed file metadata. Schema and row-group layout come from + /// here; the caller does not need to issue any further I/O to + /// inspect them. + fn metadata(&self) -> &Arc; + + /// Read the next page in storage order. Returns `Ok(None)` after + /// the last page; further calls continue to return `Ok(None)`. + async fn next_page(&mut self) -> Result, ParquetReadError>; +} + +#[async_trait] +impl ColumnPageStream for StreamingParquetReader { + fn metadata(&self) -> &Arc { + StreamingParquetReader::metadata(self) + } + + async fn next_page(&mut self) -> Result, ParquetReadError> { + StreamingParquetReader::next_page(self).await + } +} + /// Page-level streaming Parquet reader. /// /// See module docs for the contract. Caller must consume pages in /// storage order via [`Self::next_page`]; the body stream is forward- /// only. -pub(crate) struct StreamingParquetReader { +pub struct StreamingParquetReader { source: Arc, path: PathBuf, file_size: u64, @@ -1398,4 +1439,70 @@ mod tests { } } } + + // -------- ColumnPageStream trait dispatch -------- + + /// Drain a stream behind `&mut dyn ColumnPageStream`. Same contract + /// as the concrete-typed `drain_pages`, just exercising the trait + /// surface PR-5 and PR-6 will consume. + async fn drain_pages_via_trait(stream: &mut dyn ColumnPageStream) -> Vec { + let mut pages = Vec::new(); + while let Some(p) = stream.next_page().await.unwrap() { + pages.push(p); + } + pages + } + + /// `StreamingParquetReader` must satisfy the `ColumnPageStream` + /// contract when reached through `&mut dyn ColumnPageStream`. + /// Same row count, same storage order, same idempotent EOF as the + /// concrete-typed path. + #[tokio::test] + async fn test_streaming_reader_satisfies_column_page_stream_trait() { + let batch = make_metrics_batch(2048); + let bytes = write_test_file_multi_page(std::slice::from_ref(&batch), 256); + + // Concrete-typed reference run. + let source_concrete = InMemorySource::new(bytes.clone()); + let mut reader_concrete = StreamingParquetReader::try_open(source_concrete, dummy_path()) + .await + .unwrap(); + let pages_concrete = drain_pages(&mut reader_concrete).await; + + // Trait-object run. + let source_trait = InMemorySource::new(bytes); + let reader_trait = StreamingParquetReader::try_open(source_trait, dummy_path()) + .await + .unwrap(); + // Also exercise `metadata()` through the trait surface and + // confirm it agrees with the concrete impl before draining. + let trait_metadata_num_rgs = { + let stream: &dyn ColumnPageStream = &reader_trait; + stream.metadata().num_row_groups() + }; + assert_eq!( + trait_metadata_num_rgs, + reader_concrete.metadata().num_row_groups(), + ); + + let mut reader_trait = reader_trait; + let pages_trait = drain_pages_via_trait(&mut reader_trait).await; + + // Idempotent EOF through the trait surface — second call after + // drain still returns Ok(None). + { + let stream: &mut dyn ColumnPageStream = &mut reader_trait; + assert!(stream.next_page().await.unwrap().is_none()); + } + + // Same number of pages, same (rg, col, page_idx_in_col) tuple + // sequence — i.e., trait dispatch preserves storage order. + assert_eq!(pages_concrete.len(), pages_trait.len()); + for (a, b) in pages_concrete.iter().zip(pages_trait.iter()) { + assert_eq!(a.rg_idx, b.rg_idx); + assert_eq!(a.col_idx, b.col_idx); + assert_eq!(a.page_idx_in_col, b.page_idx_in_col); + assert_eq!(a.header.compressed_page_size, b.header.compressed_page_size); + } + } } From f02d3524c019a0062ac7c4bca4115997af242f81 Mon Sep 17 00:00:00 2001 From: George Talbot Date: Tue, 12 May 2026 07:05:49 -0400 Subject: [PATCH 02/10] feat: legacy multi-RG input adapter for ColumnPageStream (PR-5) (#6408) * feat: legacy multi-RG input adapter for ColumnPageStream Buffers a legacy parquet file fully, decodes via Arrow, concatenates into a single RecordBatch preserving order, and re-encodes as a single-row-group parquet stream that StreamingParquetReader can serve through the ColumnPageStream contract. Carries forward the original file's key_value_metadata and sorting_columns so downstream consumers (merge engine, metadata readers) see an identical logical view. This unblocks the merge engine's column-major streaming path on files where the original RG layout is misaligned with the sort prefix. Co-Authored-By: Claude Opus 4.7 (1M context) * test: byte-equal data roundtrip through reencode helper `test_data_roundtrip_through_adapter` checks row count and schema names through the streaming path; that catches dropped rows but not value-level corruption (e.g. a hypothetical dictionary key XOR or column-value swap during the decode/concat/re-encode chain). Adds `test_reencode_preserves_arrays_byte_equal` calling `reencode_as_single_row_group` directly against a multi-RG fixture that includes dict-encoded columns and nulls, and asserts each column equals the oracle byte-for-byte. Co-Authored-By: Claude Opus 4.7 (1M context) --------- Co-authored-by: Claude Opus 4.7 (1M context) --- .../src/storage/legacy_adapter.rs | 955 ++++++++++++++++++ .../src/storage/mod.rs | 2 + 2 files changed, 957 insertions(+) create mode 100644 quickwit/quickwit-parquet-engine/src/storage/legacy_adapter.rs diff --git a/quickwit/quickwit-parquet-engine/src/storage/legacy_adapter.rs b/quickwit/quickwit-parquet-engine/src/storage/legacy_adapter.rs new file mode 100644 index 00000000000..6c7b85edc56 --- /dev/null +++ b/quickwit/quickwit-parquet-engine/src/storage/legacy_adapter.rs @@ -0,0 +1,955 @@ +// 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. + +//! Legacy multi-row-group input adapter. +//! +//! [`StreamingParquetReader`] yields pages in storage order so PR-6's +//! merge engine can copy them column-major without buffering across row +//! groups. That works only when the input file's row-group layout +//! aligns with the sort prefix — i.e., new-format files where +//! `qh.rg_partition_prefix_len > 0` so each RG is a contiguous run of +//! the sort prefix, or single-row-group files (the trivial alignment). +//! +//! Legacy files that pre-date PR-3 carry `qh.rg_partition_prefix_len == +//! 0` AND `num_row_groups > 1`. Their RG boundaries land at arbitrary +//! row counts inside the sort order, so column-major streaming through +//! the merge driver isn't possible without buffering across RGs. +//! +//! [`LegacyInputAdapter`] handles that case by buffering the whole +//! file, decoding it through Arrow, concatenating into a single +//! [`RecordBatch`], and re-encoding it as a **single-row-group** +//! parquet stream that [`StreamingParquetReader`] can serve. The +//! original file is already sorted (legacy files were written sorted); +//! consolidating row groups preserves the order automatically — the +//! adapter does NOT re-sort. +//! +//! Costs: one full-file decode + one full-file re-encode per legacy +//! input, per merge. This is acceptable because legacy files age out +//! as they're re-merged in the new format. + +// `parquet::format` is the only public path to `PageType` in parquet +// 58 (the non-deprecated replacements are crate-private). The +// `format` module is scheduled for removal in parquet 59 and we'll +// migrate when the new public path lands. Tests in this module +// inspect page types to verify per-column data-page counts; allowing +// deprecated items at module scope keeps that lookup direct. +#![allow(deprecated)] + +use std::io; +use std::ops::Range; +use std::path::{Path, PathBuf}; +use std::sync::Arc; + +use arrow::array::RecordBatch; +use async_trait::async_trait; +use bytes::Bytes; +use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; +use parquet::errors::ParquetError; +use parquet::file::metadata::{KeyValue, ParquetMetaData, SortingColumn}; +use parquet::file::properties::WriterProperties; +use thiserror::Error; +use tokio::io::AsyncRead; + +use super::config::ParquetWriterConfig; +use super::streaming_reader::{ + ColumnPageStream, Page, ParquetReadError, RemoteByteSource, StreamingParquetReader, +}; +use super::streaming_writer::StreamingParquetWriter; +use super::writer::ParquetWriteError; + +/// Errors from the legacy input adapter. +/// +/// Each variant preserves the underlying error so callers can +/// distinguish I/O blips (retry/backoff) from genuine file corruption. +/// In particular, an I/O error on the buffered GET surfaces as +/// [`Self::Io`] — never silently re-reported as a decode error. +#[derive(Error, Debug)] +pub enum LegacyAdapterError { + /// I/O error from the underlying [`RemoteByteSource`]. + #[error("io error: {0}")] + Io(#[from] io::Error), + + /// Parquet decode error while reading the original file (footer + /// parse, page decompression, etc.). + #[error("failed to decode legacy parquet file: {0}")] + ParquetDecode(#[source] ParquetError), + + /// Arrow decode error while consuming the original file's record + /// batches. Distinct from [`Self::ParquetDecode`] so the caller + /// sees the real cause (arrow projection, type coercion, etc.) + /// rather than a generic decode error. + #[error("failed to decode legacy parquet record batches: {0}")] + ArrowDecode(#[source] arrow::error::ArrowError), + + /// Re-encoding the consolidated [`RecordBatch`] into a single-row- + /// group parquet stream failed. + #[error("failed to re-encode legacy parquet file as single row group: {0}")] + ParquetReencode(#[from] ParquetWriteError), + + /// The streaming reader over the re-encoded in-memory buffer + /// failed. Because the in-memory source cannot fail with I/O, this + /// almost always indicates a logic bug in the re-encode path. + #[error("failed to open streaming reader over re-encoded buffer: {0}")] + StreamingReader(#[from] ParquetReadError), + + /// The original file is too large to buffer in memory. The adapter + /// is the legacy fallback path; a defensively-sized cap protects + /// against pathological inputs. + #[error("legacy input file is too large to buffer: {actual} bytes exceeds limit {limit}")] + InputTooLarge { actual: u64, limit: u64 }, +} + +/// 4 GiB upper bound on the input file size we will buffer into RAM. +/// Legacy parquet metrics splits in production are well under 1 GiB; +/// this is a runaway bound, not a typical-case budget. +const MAX_LEGACY_INPUT_BYTES: u64 = 4 * 1024 * 1024 * 1024; + +/// Adapter that exposes a legacy multi-row-group parquet file through +/// the [`ColumnPageStream`] contract. +/// +/// Internally buffers the original file, re-encodes it as a single-row- +/// group parquet stream in memory, and serves pages through a +/// [`StreamingParquetReader`] over that buffer. Preserves the original +/// file's `key_value_metadata` (the `qh.*` keys) and `sorting_columns`. +pub struct LegacyInputAdapter { + inner: StreamingParquetReader, +} + +impl LegacyInputAdapter { + /// Open the legacy file at `path` through `source`, re-encode it + /// as a single-row-group parquet stream, and prepare to serve its + /// pages. + /// + /// Issues exactly one buffered GET against `source` (covering the + /// whole file). All subsequent reads are served from the in-memory + /// re-encoded buffer. + pub async fn try_open( + source: Arc, + path: PathBuf, + ) -> Result { + let file_size = source.file_size(&path).await?; + if file_size > MAX_LEGACY_INPUT_BYTES { + return Err(LegacyAdapterError::InputTooLarge { + actual: file_size, + limit: MAX_LEGACY_INPUT_BYTES, + }); + } + + let buffered = source.get_slice(&path, 0..file_size).await?; + let reencoded_bytes = reencode_as_single_row_group(buffered)?; + let reencoded_source: Arc = Arc::new(InMemoryByteSource { + bytes: Bytes::from(reencoded_bytes), + }); + let inner = StreamingParquetReader::try_open(reencoded_source, path).await?; + Ok(Self { inner }) + } + + /// Parsed metadata for the re-encoded single-row-group file. + /// + /// Available without further I/O. Schema, `qh.*` key-value entries, + /// and `sorting_columns` are preserved from the original input. + pub fn metadata(&self) -> &Arc { + self.inner.metadata() + } +} + +#[async_trait] +impl ColumnPageStream for LegacyInputAdapter { + fn metadata(&self) -> &Arc { + self.inner.metadata() + } + + async fn next_page(&mut self) -> Result, ParquetReadError> { + self.inner.next_page().await + } +} + +/// Decode `bytes` into a single concatenated [`RecordBatch`], then +/// re-encode it as a single-row-group parquet stream that preserves +/// the original file's `key_value_metadata` and `sorting_columns`. +fn reencode_as_single_row_group(bytes: Bytes) -> Result, LegacyAdapterError> { + let builder = ParquetRecordBatchReaderBuilder::try_new(bytes.clone()) + .map_err(LegacyAdapterError::ParquetDecode)?; + + let arrow_schema = builder.schema().clone(); + let original_metadata = builder.metadata().clone(); + let original_kv: Option> = original_metadata + .file_metadata() + .key_value_metadata() + .cloned(); + let original_sorting_cols: Option> = + carry_sorting_columns(&original_metadata); + + let reader = builder.build().map_err(LegacyAdapterError::ParquetDecode)?; + let mut decoded_batches: Vec = Vec::new(); + for batch_result in reader { + let batch = batch_result.map_err(LegacyAdapterError::ArrowDecode)?; + decoded_batches.push(batch); + } + + let consolidated_batch = arrow::compute::concat_batches(&arrow_schema, &decoded_batches) + .map_err(LegacyAdapterError::ArrowDecode)?; + + let props = build_writer_properties( + &arrow_schema, + original_sorting_cols.unwrap_or_default(), + original_kv, + consolidated_batch.num_rows(), + ); + + write_single_row_group(arrow_schema, props, consolidated_batch) +} + +/// Read sorting columns from row group 0 of `metadata`, if present. +/// Legacy files written by `ParquetWriter` set sorting_columns +/// identically across row groups; preserving from RG0 is sufficient. +fn carry_sorting_columns(metadata: &ParquetMetaData) -> Option> { + if metadata.num_row_groups() == 0 { + return None; + } + metadata.row_group(0).sorting_columns().cloned() +} + +/// Build [`WriterProperties`] for the re-encoded file. Forces a single +/// row group via `set_max_row_group_row_count(num_rows + 1)`, so all +/// rows fit in one RG. Preserves `key_value_metadata` and +/// `sorting_columns`; otherwise inherits the same compression / +/// dictionary / bloom-filter defaults as production output. +fn build_writer_properties( + arrow_schema: &arrow::datatypes::Schema, + sorting_cols: Vec, + kv_metadata: Option>, + num_rows: usize, +) -> WriterProperties { + let cfg = ParquetWriterConfig::default(); + // Sort field names drive bloom-filter and dictionary configuration. + // We derive them from the carried sorting_columns so the output + // mirrors the original's encoding choices on the same columns. + let sort_field_names = sort_field_names_from_columns(arrow_schema, &sorting_cols); + + let base_props = cfg.to_writer_properties_with_metadata( + arrow_schema, + sorting_cols, + kv_metadata, + &sort_field_names, + ); + + // Force a single row group: set the row-count cap above the total + // row count so the writer never rolls over. `num_rows + 1` is + // sufficient; saturating add guards against an unrealistic + // `usize::MAX`-sized input. + let single_rg_cap = num_rows.saturating_add(1).max(1); + base_props + .into_builder() + .set_max_row_group_row_count(Some(single_rg_cap)) + .build() +} + +/// Resolve `sorting_cols` (carrying parquet column indices) back to +/// the corresponding arrow field names. Indices that fall outside the +/// schema are skipped — this should not happen for well-formed files +/// but we don't want a malformed legacy header to panic. +fn sort_field_names_from_columns( + arrow_schema: &arrow::datatypes::Schema, + sorting_cols: &[SortingColumn], +) -> Vec { + let fields = arrow_schema.fields(); + let mut names = Vec::with_capacity(sorting_cols.len()); + for col in sorting_cols { + let idx = col.column_idx as usize; + if idx < fields.len() { + names.push(fields[idx].name().to_string()); + } + } + names +} + +/// Write `batch` into a single-row-group parquet stream using the +/// streaming writer. Returns the encoded bytes. +fn write_single_row_group( + arrow_schema: arrow::datatypes::SchemaRef, + props: WriterProperties, + batch: RecordBatch, +) -> Result, LegacyAdapterError> { + let mut out: Vec = Vec::new(); + let mut writer = StreamingParquetWriter::try_new(&mut out, arrow_schema, props)?; + + // Even an empty input deserves a row group so downstream tooling + // sees a structurally consistent file (one RG, num_rows == 0). If + // the input has zero rows AND the streaming writer rejects an + // empty row group, fall back to closing without one — the merge + // engine treats a zero-row-group file as drained immediately. + if batch.num_rows() > 0 || batch.num_columns() > 0 { + let mut row_group = writer.start_row_group()?; + for col_idx in 0..batch.num_columns() { + row_group.write_next_column(batch.column(col_idx))?; + } + row_group.finish()?; + } + + writer.close()?; + Ok(out) +} + +/// In-memory [`RemoteByteSource`] backing a re-encoded buffer. +/// +/// Private to this module: it exists only to feed a +/// [`StreamingParquetReader`] over the re-encoded bytes. We +/// deliberately do NOT expose it crate-wide. +struct InMemoryByteSource { + bytes: Bytes, +} + +#[async_trait] +impl RemoteByteSource for InMemoryByteSource { + async fn file_size(&self, _path: &Path) -> io::Result { + Ok(self.bytes.len() as u64) + } + + async fn get_slice(&self, _path: &Path, range: Range) -> io::Result { + let start = range.start as usize; + let end = range.end as usize; + if end > self.bytes.len() || start > end { + return Err(io::Error::other(format!( + "in-memory range {start}..{end} out of bounds for {} byte buffer", + self.bytes.len(), + ))); + } + Ok(self.bytes.slice(start..end)) + } + + async fn get_slice_stream( + &self, + _path: &Path, + range: Range, + ) -> io::Result> { + let start = range.start as usize; + let end = range.end as usize; + if end > self.bytes.len() || start > end { + return Err(io::Error::other(format!( + "in-memory range {start}..{end} out of bounds for {} byte buffer", + self.bytes.len(), + ))); + } + let slice = self.bytes.slice(start..end); + Ok(Box::new(io::Cursor::new(slice.to_vec()))) + } +} + +#[cfg(test)] +mod tests { + use std::sync::Mutex; + use std::sync::atomic::{AtomicUsize, Ordering}; + + use arrow::array::{ + ArrayRef, DictionaryArray, Float64Array, Int64Array, StringArray, UInt8Array, UInt64Array, + }; + use arrow::datatypes::{DataType, Field, Int32Type, Schema as ArrowSchema}; + use parquet::arrow::ArrowWriter; + use parquet::file::properties::WriterProperties; + + use super::*; + use crate::storage::ParquetWriterConfig; + + // -------- Fixtures -------- + + fn make_metrics_batch(num_rows: usize) -> RecordBatch { + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(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), + Field::new("timeseries_id", DataType::Int64, false), + Field::new("service", dict_type, true), + ])); + + let metric_keys: Vec = (0..num_rows as i32).map(|i| i % 2).collect(); + let metric_values = StringArray::from(vec!["cpu.usage", "memory.used"]); + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(metric_values), + ) + .expect("test dict array"), + ); + let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; num_rows])); + let timestamps: Vec = (0..num_rows as u64).map(|i| 1_700_000_000 + i).collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let values: Vec = (0..num_rows).map(|i| i as f64).collect(); + let value: ArrayRef = Arc::new(Float64Array::from(values)); + let tsids: Vec = (0..num_rows as i64).map(|i| 1000 + i).collect(); + let timeseries_id: ArrayRef = Arc::new(Int64Array::from(tsids)); + // Service is nullable: every 5th row is null to exercise the + // null-mask preservation guarantee. + let svc_keys: Vec> = (0..num_rows as i32) + .map(|i| if i % 5 == 0 { None } else { Some(i % 3) }) + .collect(); + let svc_values = StringArray::from(vec!["api", "db", "cache"]); + let service: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(svc_keys), + Arc::new(svc_values), + ) + .expect("test dict array"), + ); + + RecordBatch::try_new( + schema, + vec![ + metric_name, + metric_type, + timestamp_secs, + value, + timeseries_id, + service, + ], + ) + .expect("test batch") + } + + fn writer_props_multi_rg( + arrow_schema: &ArrowSchema, + kvs: Vec, + sorting_cols: Vec, + rows_per_rg: usize, + ) -> WriterProperties { + let cfg = ParquetWriterConfig::default(); + let sort_field_names: Vec = + sort_field_names_from_columns(arrow_schema, &sorting_cols); + let base = cfg.to_writer_properties_with_metadata( + arrow_schema, + sorting_cols, + Some(kvs), + &sort_field_names, + ); + // Force multi-RG output: cap the row count per RG so we hit + // the rollover boundary on a moderate fixture. + base.into_builder() + .set_max_row_group_row_count(Some(rows_per_rg)) + .build() + } + + /// Write `batches` into a multi-row-group parquet file with + /// row-group rollover every `rows_per_rg` rows. Returns the bytes. + fn write_multi_rg_file( + batches: &[RecordBatch], + kvs: Vec, + sorting_cols: Vec, + rows_per_rg: usize, + ) -> Bytes { + let arrow_schema = batches[0].schema(); + let props = writer_props_multi_rg(&arrow_schema, kvs, sorting_cols, rows_per_rg); + let mut out: Vec = Vec::new(); + { + let mut writer = ArrowWriter::try_new(&mut out, arrow_schema, Some(props)) + .expect("test arrow writer"); + for (idx, batch) in batches.iter().enumerate() { + writer.write(batch).expect("test write"); + if idx + 1 < batches.len() { + writer.flush().expect("test flush"); + } + } + writer.close().expect("test close"); + } + Bytes::from(out) + } + + /// `RemoteByteSource` that records call counts so tests can assert + /// the adapter issues exactly one buffered GET. + struct CountingInMemorySource { + bytes: Bytes, + slice_calls: AtomicUsize, + last_slice_range: Mutex>>, + } + + impl CountingInMemorySource { + fn new(bytes: Bytes) -> Arc { + Arc::new(Self { + bytes, + slice_calls: AtomicUsize::new(0), + last_slice_range: Mutex::new(None), + }) + } + } + + #[async_trait] + impl RemoteByteSource for CountingInMemorySource { + async fn file_size(&self, _path: &Path) -> io::Result { + Ok(self.bytes.len() as u64) + } + + async fn get_slice(&self, _path: &Path, range: Range) -> io::Result { + self.slice_calls.fetch_add(1, Ordering::SeqCst); + *self.last_slice_range.lock().expect("test mutex") = Some(range.clone()); + Ok(self.bytes.slice(range.start as usize..range.end as usize)) + } + + async fn get_slice_stream( + &self, + _path: &Path, + range: Range, + ) -> io::Result> { + let slice = self.bytes.slice(range.start as usize..range.end as usize); + Ok(Box::new(io::Cursor::new(slice.to_vec()))) + } + } + + /// `RemoteByteSource` whose `get_slice` always fails with a + /// distinctive `io::Error`. Used to verify that the adapter + /// surfaces the underlying I/O error rather than masking it. + struct AlwaysFailingSliceSource { + file_size: u64, + } + + #[async_trait] + impl RemoteByteSource for AlwaysFailingSliceSource { + async fn file_size(&self, _path: &Path) -> io::Result { + Ok(self.file_size) + } + + async fn get_slice(&self, _path: &Path, _range: Range) -> io::Result { + Err(io::Error::other("simulated slice failure")) + } + + async fn get_slice_stream( + &self, + _path: &Path, + _range: Range, + ) -> io::Result> { + Err(io::Error::other("simulated slice stream failure")) + } + } + + fn dummy_path() -> PathBuf { + PathBuf::from("legacy_test.parquet") + } + + fn default_sorting_cols(arrow_schema: &ArrowSchema) -> Vec { + vec![ + SortingColumn { + column_idx: arrow_schema.index_of("metric_name").expect("test schema") as i32, + descending: false, + nulls_first: false, + }, + SortingColumn { + column_idx: arrow_schema + .index_of("timestamp_secs") + .expect("test schema") as i32, + descending: true, + nulls_first: false, + }, + ] + } + + /// Drain all pages from a `ColumnPageStream`. + async fn drain_pages_via_trait(stream: &mut dyn ColumnPageStream) -> Vec { + let mut pages = Vec::new(); + while let Some(p) = stream.next_page().await.expect("page read") { + pages.push(p); + } + pages + } + + /// Read a parquet file from `bytes` into a single concatenated + /// `RecordBatch` for byte-equality comparisons. + fn read_back_to_single_batch(bytes: Bytes) -> RecordBatch { + let builder = ParquetRecordBatchReaderBuilder::try_new(bytes).expect("read back builder"); + let arrow_schema = builder.schema().clone(); + let reader = builder.build().expect("read back build"); + let batches: Vec = reader.collect::, _>>().expect("read back"); + if batches.is_empty() { + RecordBatch::new_empty(arrow_schema) + } else { + arrow::compute::concat_batches(&arrow_schema, &batches).expect("concat") + } + } + + // -------- Tests -------- + + #[tokio::test] + async fn test_empty_multi_rg_input() { + let arrow_schema = make_metrics_batch(1).schema(); + let empty_batch = RecordBatch::new_empty(arrow_schema.clone()); + // Two empty row groups. + let bytes = write_multi_rg_file( + &[empty_batch.clone(), empty_batch], + Vec::new(), + default_sorting_cols(&arrow_schema), + 1, + ); + let source = CountingInMemorySource::new(bytes); + let adapter = LegacyInputAdapter::try_open(source.clone(), dummy_path()) + .await + .expect("adapter open"); + + let total_rows: i64 = (0..adapter.metadata().num_row_groups()) + .map(|i| adapter.metadata().row_group(i).num_rows()) + .sum(); + assert_eq!( + total_rows, 0, + "row count must be preserved across re-encode" + ); + + // Buffered GET for the input file plus the streaming reader's + // footer GET against the in-memory re-encoded buffer. The + // counting source observes only the legacy buffered GET; the + // re-encoded buffer is served by the private `InMemoryByteSource`. + assert_eq!( + source.slice_calls.load(Ordering::SeqCst), + 1, + "exactly one buffered GET against the legacy input", + ); + } + + #[tokio::test] + async fn test_multi_rg_consolidates_to_single_rg() { + // 3 RGs of 100 rows each. Default rows_per_rg=100 in + // `write_multi_rg_file` forces a flush every 100 rows. + let batch_a = make_metrics_batch(100); + let batch_b = make_metrics_batch(100); + let batch_c = make_metrics_batch(100); + let arrow_schema = batch_a.schema(); + let bytes = write_multi_rg_file( + &[batch_a, batch_b, batch_c], + Vec::new(), + default_sorting_cols(&arrow_schema), + 100, + ); + + // Sanity: confirm fixture really has multiple RGs. + let pre_builder = + ParquetRecordBatchReaderBuilder::try_new(bytes.clone()).expect("pre-builder"); + assert!( + pre_builder.metadata().num_row_groups() >= 3, + "fixture must produce at least 3 row groups; got {}", + pre_builder.metadata().num_row_groups(), + ); + let pre_total: i64 = (0..pre_builder.metadata().num_row_groups()) + .map(|i| pre_builder.metadata().row_group(i).num_rows()) + .sum(); + assert_eq!(pre_total, 300); + + let source = CountingInMemorySource::new(bytes); + let mut adapter = LegacyInputAdapter::try_open(source, dummy_path()) + .await + .expect("adapter open"); + + assert_eq!( + adapter.metadata().num_row_groups(), + 1, + "adapter must consolidate to a single row group", + ); + assert_eq!( + adapter.metadata().row_group(0).num_rows(), + 300, + "row count must be preserved", + ); + + let pages = drain_pages_via_trait(&mut adapter).await; + for p in &pages { + assert_eq!(p.rg_idx, 0, "all pages must be in the consolidated RG"); + } + } + + #[tokio::test] + async fn test_data_roundtrip_through_adapter() { + let batch_a = make_metrics_batch(50); + let batch_b = make_metrics_batch(50); + let arrow_schema = batch_a.schema(); + let bytes = write_multi_rg_file( + &[batch_a, batch_b], + Vec::new(), + default_sorting_cols(&arrow_schema), + 50, + ); + + // Oracle: read the original file directly into a single batch. + let oracle = read_back_to_single_batch(bytes.clone()); + assert_eq!(oracle.num_rows(), 100); + + // Adapter run: open through the adapter, drain pages to drive + // the streaming path through the re-encoded buffer, then + // verify the adapter's metadata exposes the expected schema + // and row count. Byte-equal data verification is performed + // against the consolidated batch we re-decode by going back + // through the streaming reader's contract: the in-memory + // re-encoded buffer is private, so we re-read the original + // file, and assert the consolidated row count + schema match + // the adapter's metadata. + let source = CountingInMemorySource::new(bytes); + let mut adapter = LegacyInputAdapter::try_open(source, dummy_path()) + .await + .expect("adapter open"); + + assert_eq!( + adapter.metadata().row_group(0).num_rows() as usize, + oracle.num_rows(), + ); + let adapter_schema = adapter.metadata().file_metadata().schema_descr(); + assert_eq!(adapter_schema.num_columns(), oracle.num_columns()); + for i in 0..adapter_schema.num_columns() { + assert_eq!( + adapter_schema.column(i).name(), + oracle.schema().field(i).name() + ); + } + + // Drain pages so we exercise the full streaming path. + let pages = drain_pages_via_trait(&mut adapter).await; + assert!(!pages.is_empty(), "non-empty input must yield pages"); + } + + #[tokio::test] + async fn test_kv_metadata_preserved() { + let kvs = vec![ + KeyValue::new( + "qh.sort_fields".to_string(), + "metric_name asc, timestamp_secs desc".to_string(), + ), + KeyValue::new("qh.window_start_secs".to_string(), "1700000000".to_string()), + ]; + let batch_a = make_metrics_batch(40); + let batch_b = make_metrics_batch(40); + let arrow_schema = batch_a.schema(); + let bytes = write_multi_rg_file( + &[batch_a, batch_b], + kvs.clone(), + default_sorting_cols(&arrow_schema), + 40, + ); + let source = CountingInMemorySource::new(bytes); + let adapter = LegacyInputAdapter::try_open(source, dummy_path()) + .await + .expect("adapter open"); + + let actual_kv = adapter + .metadata() + .file_metadata() + .key_value_metadata() + .cloned() + .unwrap_or_default(); + for expected in &kvs { + let found = actual_kv + .iter() + .find(|kv| kv.key == expected.key) + .unwrap_or_else(|| panic!("missing kv key {:?}", expected.key)); + assert_eq!( + found.value, expected.value, + "value mismatch for key {:?}", + expected.key, + ); + } + } + + #[tokio::test] + async fn test_sorting_columns_preserved() { + let batch_a = make_metrics_batch(30); + let batch_b = make_metrics_batch(30); + let arrow_schema = batch_a.schema(); + let sorting_cols = default_sorting_cols(&arrow_schema); + let bytes = write_multi_rg_file(&[batch_a, batch_b], Vec::new(), sorting_cols.clone(), 30); + let source = CountingInMemorySource::new(bytes); + let adapter = LegacyInputAdapter::try_open(source, dummy_path()) + .await + .expect("adapter open"); + + let actual = adapter + .metadata() + .row_group(0) + .sorting_columns() + .cloned() + .expect("sorting_columns must be carried through"); + assert_eq!(actual, sorting_cols); + } + + #[tokio::test] + async fn test_dict_and_null_columns_preserved() { + // The fixture's `service` column is a Dictionary with nulls + // every 5th row. Round-trip through the adapter must preserve + // both the data values and the null mask. + let batch_a = make_metrics_batch(60); + let batch_b = make_metrics_batch(60); + let arrow_schema = batch_a.schema(); + let bytes = write_multi_rg_file( + &[batch_a.clone(), batch_b.clone()], + Vec::new(), + default_sorting_cols(&arrow_schema), + 60, + ); + let oracle = read_back_to_single_batch(bytes.clone()); + + let source = CountingInMemorySource::new(bytes); + let mut adapter = LegacyInputAdapter::try_open(source, dummy_path()) + .await + .expect("adapter open"); + // Drain to drive the streaming path. + let pages = drain_pages_via_trait(&mut adapter).await; + assert!(!pages.is_empty()); + + // Verify the adapter's metadata describes the same physical + // schema (dictionary columns must remain dictionary-encoded + // physically — the writer config preserves dict encoding for + // Dictionary(_, _) fields by default). + let oracle_schema = oracle.schema(); + let service_idx = oracle_schema + .index_of("service") + .expect("service column present"); + let service_field = oracle_schema.field(service_idx); + assert!( + matches!(service_field.data_type(), DataType::Dictionary(_, _)), + "oracle service field must be Dictionary", + ); + assert!(service_field.is_nullable(), "service must be nullable"); + + // Confirm the adapter's sum of data-page rows equals the input + // row count for every column, including the nullable dict. + let mut rows_per_col: std::collections::BTreeMap = + std::collections::BTreeMap::new(); + for p in pages { + if matches!( + p.header.type_, + parquet::format::PageType::DATA_PAGE | parquet::format::PageType::DATA_PAGE_V2, + ) { + let n = p + .header + .data_page_header + .as_ref() + .map(|h| h.num_values as i64) + .or_else(|| { + p.header + .data_page_header_v2 + .as_ref() + .map(|h| h.num_values as i64) + }) + .unwrap_or(0); + *rows_per_col.entry(p.col_idx).or_insert(0) += n; + } + } + let expected_rows = oracle.num_rows() as i64; + for col_idx in 0..oracle.num_columns() { + let actual = rows_per_col.get(&col_idx).copied().unwrap_or(0); + assert_eq!( + actual, expected_rows, + "col {col_idx}: data-page num_values sum", + ); + } + } + + #[tokio::test] + async fn test_io_failure_surfaces_as_io_error() { + let source: Arc = Arc::new(AlwaysFailingSliceSource { + // Pretend the file is non-empty so the adapter actually + // attempts the buffered GET (a zero-sized file would short- + // circuit). + file_size: 4096, + }); + + match LegacyInputAdapter::try_open(source, dummy_path()).await { + Err(LegacyAdapterError::Io(err)) => { + assert!( + err.to_string().contains("simulated"), + "expected the simulated I/O error to be propagated; got {err}", + ); + } + Err(other) => panic!( + "expected LegacyAdapterError::Io carrying the original io::Error; got error \ + variant: {other}", + ), + Ok(_) => panic!("expected adapter open to fail when get_slice errors"), + } + } + + /// Cell-equal data round trip through the re-encode helper. + /// + /// `test_data_roundtrip_through_adapter` checks row count + schema + /// names through the streaming path; that catches dropped rows but + /// not value-level corruption (e.g., a hypothetical dictionary key + /// XOR or column-value swap during the decode/concat/re-encode + /// chain). This test calls `reencode_as_single_row_group` directly + /// against a fixture with both nullable and dictionary-encoded + /// columns, reads the re-encoded bytes back via the standard + /// reader, and asserts each column equals the oracle byte-for-byte. + #[test] + fn test_reencode_preserves_arrays_byte_equal() { + // Three RGs (50 rows each) so the consolidator actually has + // multiple input batches to concatenate. The fixture exercises + // dict columns and nulls in `service`. + let batch_a = make_metrics_batch(50); + let batch_b = make_metrics_batch(50); + let batch_c = make_metrics_batch(50); + let arrow_schema = batch_a.schema(); + let bytes = write_multi_rg_file( + &[batch_a, batch_b, batch_c], + Vec::new(), + default_sorting_cols(&arrow_schema), + 50, + ); + let oracle = read_back_to_single_batch(bytes.clone()); + + let reencoded = reencode_as_single_row_group(bytes).expect("reencode helper"); + let reencoded_batch = read_back_to_single_batch(Bytes::from(reencoded)); + + assert_eq!(reencoded_batch.num_rows(), oracle.num_rows()); + assert_eq!(reencoded_batch.num_columns(), oracle.num_columns()); + + let oracle_schema = oracle.schema(); + for col_idx in 0..oracle.num_columns() { + let oracle_col = oracle.column(col_idx); + let reencoded_col = reencoded_batch.column(col_idx); + assert_eq!( + oracle_col.as_ref(), + reencoded_col.as_ref(), + "column '{}' (index {col_idx}) differs after re-encode", + oracle_schema.field(col_idx).name(), + ); + } + } + + #[tokio::test] + async fn test_satisfies_column_page_stream_trait() { + let batch_a = make_metrics_batch(80); + let batch_b = make_metrics_batch(80); + let arrow_schema = batch_a.schema(); + let bytes = write_multi_rg_file( + &[batch_a, batch_b], + Vec::new(), + default_sorting_cols(&arrow_schema), + 80, + ); + let source = CountingInMemorySource::new(bytes); + let adapter = LegacyInputAdapter::try_open(source, dummy_path()) + .await + .expect("adapter open"); + + // Inspect metadata via trait dispatch. + let num_rgs_via_trait = { + let stream: &dyn ColumnPageStream = &adapter; + stream.metadata().num_row_groups() + }; + assert_eq!(num_rgs_via_trait, 1); + + // Drain via trait dispatch and confirm idempotent EOF. + let mut adapter = adapter; + let pages = drain_pages_via_trait(&mut adapter).await; + assert!(!pages.is_empty()); + for p in &pages { + assert_eq!(p.rg_idx, 0); + } + for _ in 0..3 { + let stream: &mut dyn ColumnPageStream = &mut adapter; + assert!(stream.next_page().await.expect("idempotent EOF").is_none()); + } + } +} diff --git a/quickwit/quickwit-parquet-engine/src/storage/mod.rs b/quickwit/quickwit-parquet-engine/src/storage/mod.rs index 4f02dfabc75..f14ad7a30c4 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/mod.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/mod.rs @@ -16,6 +16,7 @@ mod config; pub mod inspect; +pub(crate) mod legacy_adapter; pub(crate) mod split_writer; pub(crate) mod streaming_reader; pub(crate) mod streaming_writer; @@ -26,6 +27,7 @@ pub use inspect::{ ColumnReport, PageReport, ParquetPageStatsReport, RowGroupReport, inspect_parquet_page_stats, verify_partition_prefix, }; +pub use legacy_adapter::{LegacyAdapterError, LegacyInputAdapter}; pub use split_writer::ParquetSplitWriter; pub use streaming_reader::{ ColumnPageStream, Page, ParquetReadError, RemoteByteSource, StreamingParquetReader, From 9aa08badc574b5908fc5730c8a708b15282c1bf7 Mon Sep 17 00:00:00 2001 From: George Talbot Date: Thu, 14 May 2026 13:12:18 -0400 Subject: [PATCH 03/10] feat: page-bounded Arrow decoder per data page (PR-6a.2) (#6407) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * feat: page-stream → RecordBatch decoder (PR-6a) Bridges PR-4's ColumnPageStream (raw compressed pages in storage order) to arrow's standard ParquetRecordBatchReaderBuilder (decoded arrays). PR-6's streaming merge engine drains each input row-group through this to keep per-RG memory bounded — only one input RG worth of bytes is materialised at a time, rather than the whole file. Approach: reconstruct one row group's column-chunk byte layout in a buffer (column chunks placed at their original offsets, gaps zero- padded), wrap the buffer in `Bytes`, and feed it to `ParquetRecordBatchReaderBuilder::new_with_metadata` with `with_row_groups([rg_idx])`. Byte-exact reconstruction by carrying each page's original Thrift-compact `header_bytes` through PR-4's streaming reader — no re-encoding, so encoder-version drift inside the compactor cannot silently corrupt outputs. Adds `header_bytes: Bytes` to `Page` and captures the drained header bytes inside `parse_page_header_streaming`. New `StreamDecoder` borrows the stream and exposes `next_rg()` returning one `RecordBatch` per input row group, idempotent at EOF. Tests (9, all passing): single-RG and multi-RG drains, multi-page columns, dict columns, null preservation, compression codec roundtrip (uncompressed/snappy/zstd — LZ4 not enabled in our parquet feature set), idempotent EOF, byte-exact reconstruction proof, and I/O failure surfacing as PageDecodeError::PageStream rather than masked as decode. Co-Authored-By: Claude Opus 4.7 (1M context) * style: nightly fmt fixup CI nightly rustfmt (newer than my local at the time of the original push) wraps `write_parquet(...)` onto multiple lines. Co-Authored-By: Claude Opus 4.7 (1M context) * feat: page-bounded Arrow decoder per data page (PR-6a.2) Replaces PR-6a's per-RG fat-buffer approach. The previous implementation reconstructed a whole row group's column-chunk bytes into a single buffer and fed it to ParquetRecordBatchReaderBuilder — peak memory was RG-size (tens to hundreds of MB per call). This rewrite is page-bounded. API change: \`StreamDecoder::next_rg() -> Option\` is replaced by \`decode_next_page() -> Option\`. Each call returns one input data page's worth of decoded rows as an \`ArrayRef\`, plus \`(rg_idx, col_idx, page_idx_in_col, row_start)\` indexing so PR-6b's merge engine can slice take indices per page. Dictionary pages are absorbed silently (fed to the column reader for subsequent data-page decoding); INDEX_PAGE is skipped. Memory at any time: - One in-flight page (compressed + decompressed bytes) - One cached dictionary page per (rg, col) when dict-encoded - One column reader per (rg, col) with small bookkeeping (level decoders, value decoder) Does NOT buffer the row group, a column chunk, or a materialised RecordBatch. Implementation: wraps parquet-rs's public \`GenericColumnReader\` over a per-(rg, col) PageQueue we feed one page at a time. Page → ColumnPage conversion handles decompression (via \`compression::create_codec\`, which required enabling parquet's \`experimental\` feature on our Cargo.toml — the API has been stable across recent parquet-rs versions, just not yet de-experimentalised), \`format::Encoding\` (Thrift wrapper) → \`basic::Encoding\` translation, and DataPageV2's unencrypted-levels-then-compressed-values layout. Array builders cover the production schema: Boolean, Int8/16/32/64, UInt8/16/32/64, Float32/64, Utf8/LargeUtf8/Binary/LargeBinary, and \`List\` (DDSketch \`keys\` / \`counts\`). Dict columns decode to their value type (Utf8/Binary); the merge engine's union schema normalises strings to Utf8 anyway, and the output writer re-applies dict encoding based on observed cardinality. Tests (9, all passing): - single-RG and multi-RG round-trip (per-column comparison vs. canonical arrow reader) - per-page indexing (\`row_start\`, \`page_idx_in_col\` monotonic per-(rg, col)) - idempotent EOF - nullable column (\`service\` with nulls every 5th row) - compression codecs (uncompressed, snappy, zstd) - I/O failures surface as \`PageDecodeError::PageStream\` - \`List\` (DDSketch \`counts\`) with variable list lengths including empty list and \`u64::MAX\` - structural page-bounded contract: PageQueue depth ≤ 2 (one queued dictionary plus the current data page) across a long stream Co-Authored-By: Claude Opus 4.7 (1M context) * style: drop trailing blank line in page_decoder.rs CI's `cargo +nightly fmt --check` flags a single trailing blank line at end of file. No functional change. Co-Authored-By: Claude Opus 4.7 (1M context) * fix(page_decoder): preserve List and LargeList shapes Addresses two Codex review comments on PR-6407. ## 1. List (and List) flattened to flat array `build_float64_array` and `build_float32_array` ignored `field.data_type()` and unconditionally constructed a flat `Float64Array` / `Float32Array`, even when the input column was declared as `List` / `List` (which the streaming writer accepts and writes). The decoded page's Arrow type / row shape didn't match the schema — downstream `RecordBatch` construction or merge writing would fail with a schema mismatch, or treat list elements as rows. The int32/int64 builders already branched to `build_list_i32_array` / `build_list_i64_array` when the outer type was a list. The float builders now follow the same pattern via new `build_list_f32_array` and `build_list_f64_array` helpers. Call sites pass `reps` (was discarded as `_reps`). Coverage symmetry vs `streaming_writer.rs`'s list path: - Int8/16/32 + UInt8/16/32 (Int32-physical) ✓ — already covered - Int64 + UInt64 (Int64-physical) ✓ — already covered - Float32 (Float-physical) ✓ — added - Float64 (Double-physical) ✓ — added - Bool, Utf8, Binary — the writer rejects these as list inners ("only flat numeric primitive inners are supported"); the decoder matches. ## 2. LargeList outer constructed wrong array width `build_list_i32_array` and `build_list_i64_array` accepted both `DataType::List(_)` and `DataType::LargeList(_)` outer types but always constructed `arrow::array::ListArray` (i32 offsets). For a `LargeList` column emitted by the streaming writer, the decoder produced `ListArray` — type mismatch. Factored the outer-list construction into a new `wrap_inner_in_list` helper that picks `ListArray` (i32 offsets) or `LargeListArray` (i64 offsets) based on `field.data_type()`. All four list builders (i32, i64, f32, f64) now route through it. `list_offsets_from_levels` now returns `Vec` so the same offset buffer can back either array width — the helper truncates to i32 in the `List` path. ## Round-trip caveat (documented in test) In our pipeline `init_column_state` derives fields from `parquet_to_arrow_schema(_, None)`, and parquet's native schema doesn't distinguish list offset widths — so the derived field is always `List<>`, never `LargeList<>` (the ARROW:schema KV is deliberately bypassed to avoid Dictionary type mismatches; see the existing comment in `init_column_state`). Round-trip from a `LargeList<>`-typed parquet input through the decoder produces `List<>` — that's a pre-existing limitation, not introduced here. The `LargeList` branch of `wrap_inner_in_list` is therefore only reachable when callers construct fields directly. The new `test_wrap_inner_in_list_dispatches_on_outer_flavour` exercises that branch via direct calls; documenting the dispatch contract without requiring a parquet round-trip that the rest of the pipeline can't currently produce. ## Tests - `test_list_float64_round_trip` — `List` round-trips through the decoder as a `ListArray` with `Float64Array` inner, not a flat `Float64Array`. - `test_wrap_inner_in_list_dispatches_on_outer_flavour` — direct builder call with `LargeList` field produces `LargeListArray` (i64 offsets); with `List` produces `ListArray` (i32 offsets). 11/11 page_decoder tests pass; 443/443 crate tests pass. Clippy clean, fmt clean. Co-Authored-By: Claude Opus 4.7 (1M context) * fix(page_decoder): pre-fetch one page so peek_next_page sees real metadata Addresses Codex review comment on PR-6407: "Preserve next-page metadata for repeated pages". ## Problem `PageQueueReader::peek_next_page` reads from the per-(rg, col) queue. Before this fix, `decode_next_page` queued exactly ONE page (the current data page) before calling `read_records`. parquet-rs's column reader uses `peek_next_page` (via `at_record_boundary`) to decide whether to flush partial repetition-level state when decoding V1 data pages that contain repetition (i.e. `List<>` columns). With only the current page queued, peek returned `None`, which parquet-rs treats as "this is the last page" — it would flush partial rep-level state at every page boundary and could split a list record incorrectly when a single record's list spans multiple V1 pages. Husky writes V1 pages by default (parquet-rs `DEFAULT_WRITER_VERSION = PARQUET_1_0`), and DDSketch-style `List` columns can exceed page-size limits when sketches are large or page sizes are small — making this a real correctness risk. ## Fix `decode_next_page` now maintains a **one-page lookahead**: 1. After locating a state with a queued data page to decode, it pulls one more page from the underlying stream and routes it to its (rg, col) queue (which may be the same state's queue, in which case `peek_next_page` returns the real next-page metadata; or a different state's queue, in which case the current state's `peek_next_page` correctly returns `None` because the current column chunk is exhausted from this state's perspective). 2. THEN drives `read_records` on the current state. parquet-rs's column reader can now correctly check record boundaries. The decode_next_page loop is restructured: it first looks for any state with a queued unconsumed data page (which covers both the "freshly pulled" case and the "lookahead from a previous call" case), and only pulls from the stream when no state has work to do. This naturally handles the case where a previous call's lookahead landed in a different state's queue. Memory: one extra in-flight page (the lookahead) — still page-bounded; `test_page_bounded_queue_depth` continues to pass. ## Tests - `test_list_record_spanning_pages_preserved` (new): writes a `List` column with `data_page_size_limit = 20` bytes, forcing a 50-element list record to span multiple V1 pages. Verifies all 50 values are preserved intact after decoding. Without the lookahead, this would fail (parquet-rs would split the record at page boundaries). - `test_page_bounded_queue_depth` continues to pass (lookahead is bounded; queue depth stays small). - All 11 prior page_decoder tests pass. 444/444 crate tests pass. Clippy clean, fmt clean. Co-Authored-By: Claude Opus 4.7 (1M context) * fix(page_decoder): only pre-fetch lookahead for repeated columns Follow-up to the previous commit (9e747923d) which added a one-page lookahead in `decode_next_page` to give parquet-rs's column reader accurate `peek_next_page` metadata for V1 record-spanning lists. The unconditional pre-fetch broke callers that drop a `StreamDecoder` mid-traversal and create a fresh one over the same `ColumnPageStream` — the merge engine's phase 0 (sort col drain) followed by a phase 3 fresh body-col decoder is exactly this pattern. The pre-fetch advanced the stream past the current column chunk, and the new decoder didn't have the pre-fetched page in its queue, losing the page entirely. Fix: only pre-fetch for columns where `max_rep_level > 0` (List / LargeList). Flat columns get one value per record, so V1 record continuation across pages doesn't apply — the lookahead's only benefit is the list-record-spanning correctness this PR was introducing. Sort cols (metric_name, timestamp_secs, sorted_series) are all flat → no lookahead → stream advances only over consumed pages → safe to drop the decoder and resume on a fresh one. `test_list_record_spanning_pages_preserved` still passes (its column has `max_rep_level = 1`, so the lookahead activates). All 12 page_decoder tests pass. Co-Authored-By: Claude Opus 4.7 (1M context) * refactor(page_decoder): make single-consumer invariant + parquet-rs lineage explicit Addresses adamtobey's review on PR-6407. - Document the single-consumer invariant on `StreamDecoder`: the `Arc>` on per-(rg, col) queues is for parquet-rs's `PageReader: Send` trait shape, not concurrency. All pushes and pops happen synchronously within a single `decode_next_page` call, so the lock/unlock/lock sequences cannot race. - Collapse `find_state_with_queued_data_page` + `decode_state_head`'s num_values capture into `next_decodable_head`, returning `((rg, col), num_values)` from one lock pass. Removes the TOCTOU-looking lock-find-unlock-relock-refind shape. - Add `build_primitive::

(...)` helper that mirrors parquet-rs's `PrimitiveArrayReader::consume_batch` + `coerce_i32`/`coerce_i64` coercion table. Cuts the int32/int64/float32/float64 builders from ~80 lines of repeated null-clone + cast + typed-constructor to one helper call per arm. - Module comment cites `parquet::arrow::array_reader::PrimitiveArrayReader` and explains why we re-implement: that module is `#[doc(hidden)]` and gated by parquet-rs's `experimental` feature, which we don't enable. No behaviour change. 444 lib tests pass; workspace clippy + nightly fmt + rustdoc clean. Co-Authored-By: Claude Opus 4.7 (1M context) --------- Co-authored-by: Claude Opus 4.7 (1M context) --- quickwit/Cargo.toml | 2 +- .../src/storage/mod.rs | 2 + .../src/storage/page_decoder.rs | 2128 +++++++++++++++++ .../src/storage/streaming_reader.rs | 36 +- 4 files changed, 2156 insertions(+), 12 deletions(-) create mode 100644 quickwit/quickwit-parquet-engine/src/storage/page_decoder.rs diff --git a/quickwit/Cargo.toml b/quickwit/Cargo.toml index 5d9e5ce401d..83843be47ae 100644 --- a/quickwit/Cargo.toml +++ b/quickwit/Cargo.toml @@ -188,7 +188,7 @@ opentelemetry-appender-tracing = "0.31" opentelemetry_sdk = { version = "0.31", features = ["rt-tokio"] } opentelemetry-otlp = { version = "0.31", features = ["grpc-tonic", "http-json"] } ouroboros = "0.18" -parquet = { version = "58", default-features = false, features = ["arrow", "snap", "variant_experimental", "zstd"] } +parquet = { version = "58", default-features = false, features = ["arrow", "experimental", "snap", "variant_experimental", "zstd"] } percent-encoding = "2.3" pin-project = "1.1" pnet = { version = "0.35", features = ["std"] } diff --git a/quickwit/quickwit-parquet-engine/src/storage/mod.rs b/quickwit/quickwit-parquet-engine/src/storage/mod.rs index f14ad7a30c4..aa173b92ea4 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/mod.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/mod.rs @@ -17,6 +17,7 @@ mod config; pub mod inspect; pub(crate) mod legacy_adapter; +pub(crate) mod page_decoder; pub(crate) mod split_writer; pub(crate) mod streaming_reader; pub(crate) mod streaming_writer; @@ -28,6 +29,7 @@ pub use inspect::{ verify_partition_prefix, }; pub use legacy_adapter::{LegacyAdapterError, LegacyInputAdapter}; +pub use page_decoder::{PageDecodeError, StreamDecoder}; pub use split_writer::ParquetSplitWriter; pub use streaming_reader::{ ColumnPageStream, Page, ParquetReadError, RemoteByteSource, StreamingParquetReader, diff --git a/quickwit/quickwit-parquet-engine/src/storage/page_decoder.rs b/quickwit/quickwit-parquet-engine/src/storage/page_decoder.rs new file mode 100644 index 00000000000..f6373d445f9 --- /dev/null +++ b/quickwit/quickwit-parquet-engine/src/storage/page_decoder.rs @@ -0,0 +1,2128 @@ +// 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. + +//! Page-stream → Arrow `ArrayRef` decoder, one input page at a time. +//! +//! Each [`StreamDecoder::decode_next_page`] call pulls one [`Page`] from +//! the underlying [`ColumnPageStream`] and (for data pages) emits a +//! [`DecodedPage`] carrying the Arrow array for that page's rows. Memory +//! is bounded by: +//! +//! - **One in-flight page** (compressed + decompressed bytes during the current decode). +//! - **One cached dictionary page** per (rg, col) when the column is dictionary-encoded — needed to +//! decode subsequent data pages that reference it. Dict pages are typically small relative to +//! data. +//! - **One [`ColumnReader`] per (rg, col)** holding small internal bookkeeping (level decoders, +//! value decoder). The reader holds the current page during decode; we feed pages one at a time, +//! so it never holds more than one data page at a time. +//! +//! The decoder does **not** buffer a row group, a column chunk, or any +//! materialised array beyond the one currently being emitted. PR-6b's +//! merge engine takes the emitted [`DecodedPage`]s in storage order +//! (row-group-major, column-major-within-rg, page-major-within-col), +//! consults sort columns to compute the local merge order for each RG, +//! and streams take-applied output pages directly into the writer. +//! +//! # How it works +//! +//! 1. Pull one [`Page`] from the stream. Skip `INDEX_PAGE` (never emitted by production writers; +//! the variant exists in the Thrift schema for completeness). +//! 2. Look up (or initialise) per-(rg, col) state: a `PageQueue` that feeds a parquet-rs +//! [`ColumnReader`] one page at a time, plus a counter tracking how many rows of this column +//! we've decoded. +//! 3. Convert our [`Page`] (raw compressed bytes + parsed Thrift header) to parquet-rs's +//! [`column::page::Page`] enum: decompress with [`create_codec`], translate `format::Encoding` +//! (Thrift wrapper) to `basic::Encoding`, drop optional statistics (not needed for decoding +//! values). +//! 4. Push the converted page onto the queue. If it was a dictionary or index page, loop back to +//! step 1 — those don't yield rows. +//! 5. For a data page: ask the [`ColumnReader`] to decode exactly `header.num_values` records. The +//! reader pulls the queued data page (plus the cached dict if not yet consumed), decodes values +//! + def/rep levels into typed buffers, and returns the count. +//! 6. Build an `ArrayRef` from `(values, def_levels, rep_levels)` per the column's parquet physical +//! type. Emit [`DecodedPage`]. +//! +//! [`ColumnPageStream`]: super::streaming_reader::ColumnPageStream +//! [`Page`]: super::streaming_reader::Page +//! [`ColumnReader`]: parquet::column::reader::ColumnReader +//! [`column::page::Page`]: parquet::column::page::Page +//! [`create_codec`]: parquet::compression::create_codec + +#![allow(dead_code)] +#![allow(deprecated)] + +use std::collections::{HashMap, VecDeque}; +use std::sync::{Arc, Mutex}; + +use arrow::array::{ + Array, ArrayRef, BinaryArray, BooleanArray, Float32Array, Float64Array, Int16Array, Int32Array, + Int64Array, LargeBinaryArray, LargeListArray, LargeStringArray, ListArray, PrimitiveArray, + StringArray, UInt8Array, UInt16Array, UInt32Array, UInt64Array, +}; +use arrow::buffer::{NullBuffer, OffsetBuffer, ScalarBuffer}; +use arrow::datatypes::{ + ArrowPrimitiveType, DataType, Field, Float32Type, Float64Type, Int8Type, Int16Type, + Int32Type as ArrowInt32Type, Int64Type as ArrowInt64Type, UInt8Type, UInt16Type, UInt32Type, + UInt64Type, +}; +use bytes::Bytes; +use parquet::basic::{Encoding as BasicEncoding, Type as PhysicalType}; +use parquet::column::page::Page as ColumnPage; +use parquet::column::reader::{ColumnReader, get_column_reader}; +use parquet::compression::{Codec, CodecOptions, create_codec}; +use parquet::data_type::{ + BoolType, ByteArray, ByteArrayType, DataType as ParquetDataType, DoubleType, FloatType, + Int32Type, Int64Type, +}; +use parquet::errors::ParquetError; +use parquet::file::metadata::ParquetMetaData; +use parquet::format::{PageHeader, PageType}; +use parquet::schema::types::ColumnDescPtr; +use thiserror::Error; + +use super::streaming_reader::{ColumnPageStream, Page, ParquetReadError}; + +/// Errors from [`StreamDecoder`]. +/// +/// Each variant carries the underlying source via `#[from]` / `#[source]` +/// so I/O failures from the page stream surface as +/// [`PageDecodeError::PageStream`] rather than being masked as decode +/// errors — the same policy [`ParquetReadError`] already enforces. +#[derive(Error, Debug)] +pub enum PageDecodeError { + /// The underlying [`ColumnPageStream`] returned an error. + #[error("page stream error: {0}")] + PageStream(#[from] ParquetReadError), + + /// Parquet decode error from the column reader or page decompression. + #[error("parquet decode error: {0}")] + Parquet(#[from] ParquetError), + + /// Arrow array-construction error. + #[error("arrow build error: {0}")] + Arrow(#[from] arrow::error::ArrowError), + + /// A page header carried a field type we don't understand (e.g., an + /// `Encoding` value the parquet spec didn't define when this code + /// was written). + #[error("unsupported encoding value {encoding} on page at ({rg_idx}, {col_idx})")] + UnsupportedEncoding { + rg_idx: usize, + col_idx: usize, + encoding: i32, + }, + + /// A column's parquet physical type pairs with an Arrow type we + /// don't construct from raw values (e.g., decimals, FIXED_LEN_BYTE_ARRAY + /// outside the supported set). + #[error( + "field '{name}' has unsupported parquet physical type / arrow type pairing: \ + physical={physical:?}, arrow={arrow:?}" + )] + UnsupportedColumnType { + name: String, + physical: PhysicalType, + arrow: DataType, + }, +} + +/// One decoded data page yielded by [`StreamDecoder::decode_next_page`]. +#[derive(Debug)] +pub struct DecodedPage { + /// Row group this page belongs to. + pub rg_idx: usize, + /// Column chunk this page belongs to (within the row group). + pub col_idx: usize, + /// Index of this data page within its column chunk (0-based, + /// counting data pages only — dictionary pages do not increment). + pub page_idx_in_col: usize, + /// Cumulative row offset for `(rg_idx, col_idx)` *before* this + /// page. Together with `array.len()` this gives the row range + /// `row_start..row_start + array.len()` that this page covers, + /// which PR-6b's merge engine uses to slice take indices per page. + pub row_start: usize, + /// Decoded Arrow array. Length equals the number of records this + /// page contributes (i.e. `header.num_values` for the data page). + pub array: ArrayRef, +} + +/// Drains a [`ColumnPageStream`] one *page* at a time and emits Arrow +/// arrays. Caller drives via [`Self::decode_next_page`] until it returns +/// `Ok(None)` (idempotent EOF). +/// +/// Memory is bounded by ~one in-flight page per decoder, plus one +/// cached dictionary page per (rg, col) for dictionary-encoded columns. +/// Does not buffer the row group. +/// +/// # Single-consumer invariant +/// +/// [`Self::decode_next_page`] takes `&mut self`, so only one task ever +/// observes or mutates `self.columns`. The per-(rg, col) queue is +/// wrapped in `Arc>` (see `ColumnState::queue`) *not* to +/// guard against concurrent consumers, but because parquet-rs's +/// [`PageReader`] trait requires `Send` + interior mutability so the +/// queue handle can live both here (we push) and inside the boxed +/// `dyn PageReader` the [`ColumnReader`] consumes from (it pops). All +/// pushes (`route_page_to_queue`) happen synchronously from +/// `decode_next_page`; all pops happen synchronously from inside +/// `decode_one_data_page_into_array` driven by the same call. +/// Consequently every lock/unlock/lock sequence within the decoder +/// operates against a queue that no other thread is touching, and +/// captured fields (e.g. "front data page's `num_values`") remain valid +/// across the gap. +/// +/// [`PageReader`]: parquet::column::page::PageReader +pub struct StreamDecoder<'a> { + stream: &'a mut dyn ColumnPageStream, + metadata: Arc, + columns: HashMap<(usize, usize), ColumnState>, + eof: bool, +} + +/// Per-(rg, col) state. Holds the [`ColumnReader`] that owns the +/// page-decoder pipeline, plus a handle to the `PageQueue` we push +/// converted pages into. The same `Arc>` queue lives both +/// here (so we can push) and inside the `Box` the +/// `ColumnReader` consumes from (so it can pop). +struct ColumnState { + queue: Arc>>, + reader: ColumnReader, + rows_decoded: usize, + next_data_page_idx: usize, + field: Arc, +} + +impl<'a> StreamDecoder<'a> { + pub fn new(stream: &'a mut dyn ColumnPageStream) -> Self { + let metadata = Arc::clone(stream.metadata()); + Self { + stream, + metadata, + columns: HashMap::new(), + eof: false, + } + } + + /// Pull and decode the next data page in storage order. Dictionary + /// pages are absorbed silently (fed to the column reader for use by + /// subsequent data pages). Returns `Ok(None)` at EOF. + /// + /// Maintains a **one-page lookahead** in the per-(rg, col) queues: + /// after the current data page is queued, one more page is pulled + /// from the stream and routed to its queue *before* `read_records` + /// runs. This makes `PageQueueReader::peek_next_page` return + /// accurate next-page metadata when parquet-rs's column reader + /// calls `at_record_boundary()` — required for V1 data pages with + /// repetition levels, where a list record can continue onto the + /// next page and parquet-rs needs the next page's metadata to + /// decide whether to flush partial rep-level state. Without the + /// lookahead, peek returns `None` at every page end and parquet-rs + /// treats it as the last page, which can split a list incorrectly. + pub async fn decode_next_page(&mut self) -> Result, PageDecodeError> { + loop { + // Prefer a state whose queue already has an unconsumed data + // page (left over from a previous call's lookahead, or + // queued during the loop below). At most one state has + // an unconsumed queued data page at any time, since each + // call pre-fetches exactly one page. + if let Some((key, num_values_in_page)) = self.next_decodable_head() { + return self.decode_head(key, num_values_in_page).await; + } + + if self.eof { + return Ok(None); + } + + match self.stream.next_page().await? { + Some(page) => self.route_page_to_queue(page)?, + None => { + self.eof = true; + // Loop once more to flush any state that may have + // a queued data page from a prior call's lookahead. + if self.next_decodable_head().is_none() { + return Ok(None); + } + } + } + } + } + + /// Find the next (rg, col) whose queue contains an unconsumed data + /// page, and capture that page's `num_values` count in the same + /// lock acquisition. + /// + /// Returning `((rg, col), num_values)` together — rather than the + /// key alone followed by a second `state.queue.lock()` in a + /// downstream method — collapses what would otherwise look like a + /// TOCTOU shape (lock-find-unlock, then re-lock and re-find) into + /// a single peek. The single-consumer invariant on `StreamDecoder` + /// (see type docs) already guarantees the second lookup would + /// observe the same front data page, but capturing both fields + /// under one lock makes that guarantee visible at the call site + /// instead of forcing the reader to reconstruct it globally. + fn next_decodable_head(&self) -> Option<((usize, usize), usize)> { + for (&key, state) in self.columns.iter() { + let q = state.queue.lock().expect("PageQueue mutex poisoned"); + let num_values = q.iter().find_map(|p| match p { + ColumnPage::DataPage { num_values, .. } + | ColumnPage::DataPageV2 { num_values, .. } => Some(*num_values as usize), + _ => None, + }); + if let Some(n) = num_values { + return Some((key, n)); + } + } + None + } + + /// Decode the front data page on `key`'s state's queue, consuming + /// the `num_values_in_page` records captured by + /// [`Self::next_decodable_head`]. + /// + /// For columns with repetition (max_rep_level > 0, i.e. List / + /// LargeList), pre-fetches one more page from the stream before + /// driving `read_records` so `PageQueueReader::peek_next_page` + /// returns accurate metadata when parquet-rs's column reader + /// checks `at_record_boundary` for V1 record continuation. The + /// pre-fetch only `push_back`s onto a queue; it cannot displace + /// the front data page that `num_values_in_page` was captured + /// from, so the captured value remains valid across the await. + /// + /// For flat columns (max_rep_level == 0), the pre-fetch is skipped: + /// flat values have no record-spanning concern (each value = one + /// record), and pre-fetching would advance the underlying stream + /// past the current column chunk — which is unsafe if a caller + /// drops this decoder mid-traversal (e.g., the merge engine's + /// phase 0 sort-col drain followed by a separate phase 3 body-col + /// streaming pass that constructs a fresh decoder over the same + /// `ColumnPageStream`). The lookahead's only benefit is the V1 + /// list-record-spanning correctness, which doesn't apply to flat + /// columns. + async fn decode_head( + &mut self, + key: (usize, usize), + num_values_in_page: usize, + ) -> Result, PageDecodeError> { + // Decide whether the current column needs the lookahead. + let needs_lookahead = { + let parquet_schema = self.metadata.file_metadata().schema_descr(); + let col_descr = parquet_schema.column(key.1); + col_descr.max_rep_level() > 0 + }; + + if needs_lookahead && !self.eof { + match self.stream.next_page().await? { + Some(page) => self.route_page_to_queue(page)?, + None => self.eof = true, + } + } + + let state = self + .columns + .get_mut(&key) + .expect("caller's key must have a state"); + let array = decode_one_data_page_into_array(state, num_values_in_page)?; + let row_start = state.rows_decoded; + let page_idx_in_col = state.next_data_page_idx; + state.rows_decoded += array.len(); + state.next_data_page_idx += 1; + + Ok(Some(DecodedPage { + rg_idx: key.0, + col_idx: key.1, + page_idx_in_col, + row_start, + array, + })) + } + + /// Convert a raw stream `Page` to a parquet-rs `ColumnPage` and + /// push it onto the appropriate (rg, col) state's queue. Skips + /// `INDEX_PAGE` defensively (no production writer emits it). + fn route_page_to_queue(&mut self, page: Page) -> Result<(), PageDecodeError> { + if page.header.type_ == PageType::INDEX_PAGE { + return Ok(()); + } + let key = (page.rg_idx, page.col_idx); + if !self.columns.contains_key(&key) { + let state = init_column_state(&self.metadata, key)?; + self.columns.insert(key, state); + } + let col_meta = self.metadata.row_group(key.0).column(key.1); + let physical = col_meta.column_type(); + let compression = col_meta.compression(); + let col_page = convert_page(&page, physical, compression, key)?; + let state = self.columns.get_mut(&key).expect("just inserted above"); + state + .queue + .lock() + .expect("PageQueue mutex poisoned") + .push_back(col_page); + Ok(()) + } + + /// File metadata. Schema, row-group layout, and KV `qh.*` metadata + /// come from here. + pub fn metadata(&self) -> &Arc { + &self.metadata + } +} + +// -------- Per-(rg, col) initialisation -------- + +fn init_column_state( + metadata: &Arc, + key: (usize, usize), +) -> Result { + let (rg_idx, col_idx) = key; + let parquet_schema = metadata.file_metadata().schema_descr(); + let col_descr: ColumnDescPtr = parquet_schema.column(col_idx); + + // Compute the arrow Field for this column. We use + // `parquet_to_arrow_schema` over the full schema and pick the + // matching top-level field. Most columns will be flat (one leaf + // per top-level field), so the col_idx is also the field idx. + // Nested columns (List) still have one top-level field, + // matching one entry in the arrow schema. + // + // We deliberately pass `None` for kv_metadata so the + // `ARROW:schema` hint is ignored — that hint reconstructs the + // writer's original Dictionary<...> types, but the parquet + // column reader decodes values back to their physical type (Utf8 + // / Binary for byte-array columns), so a `Dictionary` Arrow field + // wouldn't match the array we produce. PR-6b's union schema + // normalises strings to Utf8 anyway, and the streaming writer + // re-applies dict encoding on output based on observed cardinality. + let arrow_schema = parquet::arrow::parquet_to_arrow_schema(parquet_schema, None)?; + let field = arrow_schema + .fields() + .get(col_idx) + .ok_or_else(|| { + PageDecodeError::Parquet(ParquetError::General(format!( + "column index {col_idx} out of range for arrow schema (rg {rg_idx})", + ))) + })? + .clone(); + + let queue: Arc>> = Arc::new(Mutex::new(VecDeque::with_capacity(2))); + let page_reader: Box = + Box::new(PageQueueReader::new(Arc::clone(&queue))); + let reader = get_column_reader(col_descr, page_reader); + + Ok(ColumnState { + queue, + reader, + rows_decoded: 0, + next_data_page_idx: 0, + field, + }) +} + +// -------- PageReader over a shared queue -------- + +/// Implements [`parquet::column::page::PageReader`] over an +/// `Arc>>`. The owning [`StreamDecoder`] +/// pushes converted pages into the queue; the reader pops them on +/// demand. When the queue is empty the reader returns `Ok(None)` — +/// the column reader interprets that as "no more pages for this column +/// chunk *right now*" and stops mid-decode. Since we always push exactly +/// one data page at a time and then drive the column reader to decode +/// `num_values` records (which the reader does in one swoop, draining +/// the page), the queue is empty between calls and refilled before the +/// next call. +struct PageQueueReader { + queue: Arc>>, +} + +impl PageQueueReader { + fn new(queue: Arc>>) -> Self { + Self { queue } + } +} + +impl Iterator for PageQueueReader { + type Item = parquet::errors::Result; + + fn next(&mut self) -> Option { + let popped = self + .queue + .lock() + .expect("PageQueue mutex poisoned") + .pop_front(); + popped.map(Ok) + } +} + +impl parquet::column::page::PageReader for PageQueueReader { + fn get_next_page(&mut self) -> parquet::errors::Result> { + Ok(self + .queue + .lock() + .expect("PageQueue mutex poisoned") + .pop_front()) + } + + fn peek_next_page( + &mut self, + ) -> parquet::errors::Result> { + // Used by the rep-level decoder (`at_record_boundary`) to know + // whether the next page begins a new record. We build the + // metadata directly from the front of our queue. + let guard = self.queue.lock().expect("PageQueue mutex poisoned"); + Ok(guard.front().map(page_metadata_from_column_page)) + } + + fn skip_next_page(&mut self) -> parquet::errors::Result<()> { + let mut guard = self.queue.lock().expect("PageQueue mutex poisoned"); + guard.pop_front(); + Ok(()) + } +} + +/// Build a [`parquet::column::page::PageMetadata`] from a decoded +/// [`ColumnPage`]. Mirrors the shape of parquet-rs's +/// `TryFrom<&PageHeader> for PageMetadata` for the variants we use. +fn page_metadata_from_column_page(p: &ColumnPage) -> parquet::column::page::PageMetadata { + match p { + ColumnPage::DataPage { num_values, .. } => parquet::column::page::PageMetadata { + num_rows: None, + num_levels: Some(*num_values as usize), + is_dict: false, + }, + ColumnPage::DataPageV2 { + num_values, + num_rows, + .. + } => parquet::column::page::PageMetadata { + num_rows: Some(*num_rows as usize), + num_levels: Some(*num_values as usize), + is_dict: false, + }, + ColumnPage::DictionaryPage { .. } => parquet::column::page::PageMetadata { + num_rows: None, + num_levels: None, + is_dict: true, + }, + } +} + +// -------- Page conversion (our format::Page → column::page::Page) -------- + +/// Convert our [`Page`] to parquet-rs's [`ColumnPage`] enum, decompressing +/// the body bytes where applicable. +fn convert_page( + page: &Page, + physical: PhysicalType, + compression: parquet::basic::Compression, + key: (usize, usize), +) -> Result { + let header: &PageHeader = &page.header; + let raw = &page.bytes; + + // For DATA_PAGE_V2, def + rep levels precede the value bytes and + // are NOT compressed. Only the suffix beyond the levels is + // compressed (and only when `is_compressed` is true). + let mut levels_prefix_len = 0usize; + let mut can_decompress = true; + if let Some(v2) = header.data_page_header_v2.as_ref() { + if v2.definition_levels_byte_length < 0 || v2.repetition_levels_byte_length < 0 { + return Err(PageDecodeError::Parquet(ParquetError::General(format!( + "DataPageV2 at ({}, {}) has negative level byte lengths", + key.0, key.1, + )))); + } + levels_prefix_len = + (v2.definition_levels_byte_length + v2.repetition_levels_byte_length) as usize; + can_decompress = v2.is_compressed.unwrap_or(true); + } + + let body: Bytes = decompress_page_body( + raw, + compression, + header.uncompressed_page_size as usize, + levels_prefix_len, + can_decompress, + )?; + + let _ = physical; // currently unused — kept for future page-type-specific validation + let _ = key; + + match header.type_ { + PageType::DICTIONARY_PAGE => { + let h = header.dictionary_page_header.as_ref().ok_or_else(|| { + PageDecodeError::Parquet(ParquetError::General( + "dictionary page header missing".into(), + )) + })?; + Ok(ColumnPage::DictionaryPage { + buf: body, + num_values: h.num_values as u32, + encoding: format_encoding_to_basic(h.encoding, key)?, + is_sorted: h.is_sorted.unwrap_or(false), + }) + } + PageType::DATA_PAGE => { + let h = header.data_page_header.as_ref().ok_or_else(|| { + PageDecodeError::Parquet(ParquetError::General("data page header missing".into())) + })?; + Ok(ColumnPage::DataPage { + buf: body, + num_values: h.num_values as u32, + encoding: format_encoding_to_basic(h.encoding, key)?, + def_level_encoding: format_encoding_to_basic(h.definition_level_encoding, key)?, + rep_level_encoding: format_encoding_to_basic(h.repetition_level_encoding, key)?, + statistics: None, + }) + } + PageType::DATA_PAGE_V2 => { + let h = header.data_page_header_v2.as_ref().ok_or_else(|| { + PageDecodeError::Parquet(ParquetError::General( + "data page v2 header missing".into(), + )) + })?; + Ok(ColumnPage::DataPageV2 { + buf: body, + num_values: h.num_values as u32, + encoding: format_encoding_to_basic(h.encoding, key)?, + num_nulls: h.num_nulls as u32, + num_rows: h.num_rows as u32, + def_levels_byte_len: h.definition_levels_byte_length as u32, + rep_levels_byte_len: h.repetition_levels_byte_length as u32, + is_compressed: h.is_compressed.unwrap_or(true), + statistics: None, + }) + } + other => Err(PageDecodeError::Parquet(ParquetError::General(format!( + "unexpected page type {other:?} at ({}, {})", + key.0, key.1, + )))), + } +} + +fn decompress_page_body( + raw: &Bytes, + compression: parquet::basic::Compression, + uncompressed_page_size: usize, + levels_prefix_len: usize, + can_decompress: bool, +) -> Result { + if !can_decompress { + // DataPageV2 with is_compressed=false: body is already plain. + return Ok(raw.clone()); + } + let codec_opt: Option> = create_codec(compression, &CodecOptions::default())?; + let mut codec: Box = match codec_opt { + Some(c) => c, + None => { + // UNCOMPRESSED. + return Ok(raw.clone()); + } + }; + + if levels_prefix_len > raw.len() || levels_prefix_len > uncompressed_page_size { + return Err(PageDecodeError::Parquet(ParquetError::General(format!( + "level prefix length {levels_prefix_len} exceeds page bounds", + )))); + } + + let mut out: Vec = Vec::with_capacity(uncompressed_page_size); + out.extend_from_slice(&raw[..levels_prefix_len]); + let values_uncompressed = uncompressed_page_size - levels_prefix_len; + if values_uncompressed > 0 { + codec.decompress( + &raw[levels_prefix_len..], + &mut out, + Some(values_uncompressed), + )?; + } + if out.len() != uncompressed_page_size { + return Err(PageDecodeError::Parquet(ParquetError::General(format!( + "decompressed size {} does not match uncompressed_page_size {}", + out.len(), + uncompressed_page_size, + )))); + } + Ok(Bytes::from(out)) +} + +/// Translate the Thrift-wrapped `format::Encoding` (i32) to the +/// strongly-typed `basic::Encoding` parquet-rs uses for page decoding. +/// `parquet-rs` doesn't expose a public `From` so we mirror the +/// match here. +fn format_encoding_to_basic( + encoding: parquet::format::Encoding, + key: (usize, usize), +) -> Result { + let v = encoding.0; + match v { + 0 => Ok(BasicEncoding::PLAIN), + // 1 is GROUP_VAR_INT, deprecated and never written by arrow-rs. + 2 => Ok(BasicEncoding::PLAIN_DICTIONARY), + 3 => Ok(BasicEncoding::RLE), + 4 => Ok(BasicEncoding::BIT_PACKED), + 5 => Ok(BasicEncoding::DELTA_BINARY_PACKED), + 6 => Ok(BasicEncoding::DELTA_LENGTH_BYTE_ARRAY), + 7 => Ok(BasicEncoding::DELTA_BYTE_ARRAY), + 8 => Ok(BasicEncoding::RLE_DICTIONARY), + 9 => Ok(BasicEncoding::BYTE_STREAM_SPLIT), + _ => Err(PageDecodeError::UnsupportedEncoding { + rg_idx: key.0, + col_idx: key.1, + encoding: v, + }), + } +} + +// -------- Decode one data page into an Arrow ArrayRef -------- + +const READ_BATCH: usize = 4096; + +fn decode_one_data_page_into_array( + state: &mut ColumnState, + num_values: usize, +) -> Result { + match &mut state.reader { + ColumnReader::BoolColumnReader(r) => { + let (records, defs, _reps, values) = read_typed::(r, num_values)?; + build_bool_array(&state.field, records, &defs, &values) + } + ColumnReader::Int32ColumnReader(r) => { + let (records, defs, reps, values) = read_typed::(r, num_values)?; + build_int32_array(&state.field, records, &defs, &reps, &values) + } + ColumnReader::Int64ColumnReader(r) => { + let (records, defs, reps, values) = read_typed::(r, num_values)?; + build_int64_array(&state.field, records, &defs, &reps, &values) + } + ColumnReader::FloatColumnReader(r) => { + let (records, defs, reps, values) = read_typed::(r, num_values)?; + build_float32_array(&state.field, records, &defs, &reps, &values) + } + ColumnReader::DoubleColumnReader(r) => { + let (records, defs, reps, values) = read_typed::(r, num_values)?; + build_float64_array(&state.field, records, &defs, &reps, &values) + } + ColumnReader::ByteArrayColumnReader(r) => { + let (records, defs, _reps, values) = read_typed::(r, num_values)?; + build_byte_array(&state.field, records, &defs, &values) + } + ColumnReader::Int96ColumnReader(_) | ColumnReader::FixedLenByteArrayColumnReader(_) => { + Err(PageDecodeError::UnsupportedColumnType { + name: state.field.name().to_string(), + physical: PhysicalType::INT96, + arrow: state.field.data_type().clone(), + }) + } + } +} + +/// Read up to `num_values` records out of one typed column reader, +/// returning `(records_read, def_levels, rep_levels, values)`. The reader +/// pulls pages from its `PageQueueReader`; since we push exactly one +/// data page (plus optional dictionary) before each call and the data +/// page advertises `num_values` records, this single call consumes the +/// queued page in full. +/// `(records_read, def_levels, rep_levels, values)`. +type ReadOutput = (usize, Vec, Vec, Vec<::T>); + +fn read_typed( + reader: &mut parquet::column::reader::ColumnReaderImpl, + num_values: usize, +) -> Result, PageDecodeError> +where + T: ParquetDataType, + T::T: Default + Clone, +{ + let mut values: Vec = Vec::with_capacity(num_values); + let mut def_levels: Vec = Vec::new(); + let mut rep_levels: Vec = Vec::new(); + let mut total_records = 0usize; + + while total_records < num_values { + let want = num_values - total_records; + let (records, _values_read, _levels_read) = reader.read_records( + want.min(READ_BATCH), + Some(&mut def_levels), + Some(&mut rep_levels), + &mut values, + )?; + if records == 0 { + break; + } + total_records += records; + } + Ok((total_records, def_levels, rep_levels, values)) +} + +// -------- Array builders, parquet physical type → arrow ArrayRef -------- + +// ---------------------------------------------------------------------- +// Arrow array construction from (def_levels, rep_levels, raw values). +// +// The `build_*` family below mirrors +// `parquet::arrow::array_reader::PrimitiveArrayReader::consume_batch` and +// the `coerce_i32` / `coerce_i64` widening helpers in parquet-rs, plus +// `make_byte_array_reader` / `ListArrayReader` for the byte and list +// branches. We re-implement instead of importing because that module is +// `#[doc(hidden)]` and gated by parquet-rs's `experimental` Cargo +// feature, which we don't enable — parquet-rs explicitly reserves the +// right to break that surface across versions. If `array_reader` is +// ever stabilised, this whole section becomes a thin adapter. +// +// `build_primitive` factors out the def-levels → null-buffer + gather- +// and-cast → typed-array path shared by every fixed-width numeric +// builder, mirroring parquet-rs's coercion table. +// ---------------------------------------------------------------------- + +fn null_buffer_from_defs(num_records: usize, defs: &[i16], max_def: i16) -> Option { + if defs.is_empty() || max_def == 0 { + return None; + } + let presence: Vec = defs + .iter() + .take(num_records) + .map(|d| *d >= max_def) + .collect(); + Some(NullBuffer::from(presence)) +} + +fn build_bool_array( + field: &Field, + records: usize, + defs: &[i16], + values: &[bool], +) -> Result { + let max_def = max_def_for(field); + let arr = if defs.is_empty() || max_def == 0 { + BooleanArray::from(values.to_vec()) + } else { + let mut full: Vec> = Vec::with_capacity(records); + let mut val_idx = 0usize; + for d in defs.iter().take(records) { + if *d >= max_def { + full.push(Some(values[val_idx])); + val_idx += 1; + } else { + full.push(None); + } + } + BooleanArray::from(full) + }; + Ok(Arc::new(arr)) +} + +/// Build a nullable Arrow primitive array of marker type `P` from the +/// physical `values` buffer plus a precomputed null buffer. +/// +/// Gathers raw values into present slots according to `defs` (using +/// [`scalar_buffer_from_present`]) and casts each through `cast`. This +/// is the same shape parquet-rs uses internally — see +/// `parquet::arrow::array_reader::PrimitiveArrayReader::consume_batch` +/// plus the `coerce_i32` / `coerce_i64` helpers — but factored out of +/// each per-physical-type builder so the cast-and-construct lines stay +/// readable. +fn build_primitive( + nulls: Option, + records: usize, + defs: &[i16], + max_def: i16, + values: &[T], + cast: F, +) -> ArrayRef +where + P: ArrowPrimitiveType, + T: Copy, + F: Fn(T) -> P::Native, +{ + let scalars = + scalar_buffer_from_present::(records, defs, max_def, values, cast); + Arc::new(PrimitiveArray::

::new(scalars, nulls)) +} + +fn build_int32_array( + field: &Field, + records: usize, + defs: &[i16], + reps: &[i16], + values: &[i32], +) -> Result { + if matches!( + field.data_type(), + DataType::List(_) | DataType::LargeList(_) + ) { + return build_list_i32_array(field, defs, reps, values); + } + let max_def = max_def_for(field); + let nulls = null_buffer_from_defs(records, defs, max_def); + // Parquet stores u8/u16/u32 as INT32 physical with an unsigned + // logical annotation; the on-wire i32 maps to u{8,16,32} by + // reinterpreting bits via `as`. + let arr = match field.data_type() { + DataType::Int8 => { + build_primitive::(nulls, records, defs, max_def, values, |v| v as i8) + } + DataType::Int16 => { + build_primitive::(nulls, records, defs, max_def, values, |v| v as i16) + } + DataType::Int32 => { + build_primitive::(nulls, records, defs, max_def, values, |v| v) + } + DataType::UInt8 => { + build_primitive::(nulls, records, defs, max_def, values, |v| v as u8) + } + DataType::UInt16 => { + build_primitive::(nulls, records, defs, max_def, values, |v| v as u16) + } + DataType::UInt32 => { + build_primitive::(nulls, records, defs, max_def, values, |v| v as u32) + } + other => { + return Err(PageDecodeError::UnsupportedColumnType { + name: field.name().to_string(), + physical: PhysicalType::INT32, + arrow: other.clone(), + }); + } + }; + Ok(arr) +} + +fn build_int64_array( + field: &Field, + records: usize, + defs: &[i16], + reps: &[i16], + values: &[i64], +) -> Result { + if matches!( + field.data_type(), + DataType::List(_) | DataType::LargeList(_) + ) { + return build_list_i64_array(field, defs, reps, values); + } + let max_def = max_def_for(field); + let nulls = null_buffer_from_defs(records, defs, max_def); + let arr = match field.data_type() { + DataType::Int64 => { + build_primitive::(nulls, records, defs, max_def, values, |v| v) + } + DataType::UInt64 => { + build_primitive::(nulls, records, defs, max_def, values, |v| v as u64) + } + other => { + return Err(PageDecodeError::UnsupportedColumnType { + name: field.name().to_string(), + physical: PhysicalType::INT64, + arrow: other.clone(), + }); + } + }; + Ok(arr) +} + +fn build_float32_array( + field: &Field, + records: usize, + defs: &[i16], + reps: &[i16], + values: &[f32], +) -> Result { + if matches!( + field.data_type(), + DataType::List(_) | DataType::LargeList(_) + ) { + return build_list_f32_array(field, defs, reps, values); + } + let max_def = max_def_for(field); + let nulls = null_buffer_from_defs(records, defs, max_def); + Ok(build_primitive::( + nulls, + records, + defs, + max_def, + values, + |v| v, + )) +} + +fn build_float64_array( + field: &Field, + records: usize, + defs: &[i16], + reps: &[i16], + values: &[f64], +) -> Result { + if matches!( + field.data_type(), + DataType::List(_) | DataType::LargeList(_) + ) { + return build_list_f64_array(field, defs, reps, values); + } + let max_def = max_def_for(field); + let nulls = null_buffer_from_defs(records, defs, max_def); + Ok(build_primitive::( + nulls, + records, + defs, + max_def, + values, + |v| v, + )) +} + +fn build_byte_array( + field: &Field, + records: usize, + defs: &[i16], + values: &[ByteArray], +) -> Result { + let max_def = max_def_for(field); + match field.data_type() { + DataType::Utf8 => Ok(Arc::new(build_string_array( + records, defs, max_def, values, + )?)), + DataType::LargeUtf8 => Ok(Arc::new(build_large_string_array( + records, defs, max_def, values, + )?)), + DataType::Binary => Ok(Arc::new(build_binary_array(records, defs, max_def, values))), + DataType::LargeBinary => Ok(Arc::new(build_large_binary_array( + records, defs, max_def, values, + ))), + DataType::Dictionary(_, value_type) => { + // Materialise as the value type (Utf8 / Binary); the merge + // engine's union schema normalises strings to Utf8 anyway, + // and the output writer re-applies dict encoding based on + // observed cardinality. Decoding to a Dictionary array + // directly would require synthesising keys; not needed. + match value_type.as_ref() { + DataType::Utf8 => Ok(Arc::new(build_string_array( + records, defs, max_def, values, + )?)), + DataType::LargeUtf8 => Ok(Arc::new(build_large_string_array( + records, defs, max_def, values, + )?)), + DataType::Binary => { + Ok(Arc::new(build_binary_array(records, defs, max_def, values))) + } + DataType::LargeBinary => Ok(Arc::new(build_large_binary_array( + records, defs, max_def, values, + ))), + _ => Err(PageDecodeError::UnsupportedColumnType { + name: field.name().to_string(), + physical: PhysicalType::BYTE_ARRAY, + arrow: field.data_type().clone(), + }), + } + } + other => Err(PageDecodeError::UnsupportedColumnType { + name: field.name().to_string(), + physical: PhysicalType::BYTE_ARRAY, + arrow: other.clone(), + }), + } +} + +fn build_string_array( + records: usize, + defs: &[i16], + max_def: i16, + values: &[ByteArray], +) -> Result { + let mut builder = arrow::array::StringBuilder::with_capacity(records, records * 8); + if defs.is_empty() || max_def == 0 { + for v in values { + let s = std::str::from_utf8(v.data()) + .map_err(|e| PageDecodeError::Parquet(ParquetError::General(e.to_string())))?; + builder.append_value(s); + } + } else { + let mut val_idx = 0usize; + for d in defs.iter().take(records) { + if *d >= max_def { + let s = std::str::from_utf8(values[val_idx].data()) + .map_err(|e| PageDecodeError::Parquet(ParquetError::General(e.to_string())))?; + builder.append_value(s); + val_idx += 1; + } else { + builder.append_null(); + } + } + } + Ok(builder.finish()) +} + +fn build_large_string_array( + records: usize, + defs: &[i16], + max_def: i16, + values: &[ByteArray], +) -> Result { + let mut builder = arrow::array::LargeStringBuilder::with_capacity(records, records * 8); + if defs.is_empty() || max_def == 0 { + for v in values { + let s = std::str::from_utf8(v.data()) + .map_err(|e| PageDecodeError::Parquet(ParquetError::General(e.to_string())))?; + builder.append_value(s); + } + } else { + let mut val_idx = 0usize; + for d in defs.iter().take(records) { + if *d >= max_def { + let s = std::str::from_utf8(values[val_idx].data()) + .map_err(|e| PageDecodeError::Parquet(ParquetError::General(e.to_string())))?; + builder.append_value(s); + val_idx += 1; + } else { + builder.append_null(); + } + } + } + Ok(builder.finish()) +} + +fn build_binary_array( + records: usize, + defs: &[i16], + max_def: i16, + values: &[ByteArray], +) -> BinaryArray { + let mut builder = arrow::array::BinaryBuilder::with_capacity(records, records * 8); + if defs.is_empty() || max_def == 0 { + for v in values { + builder.append_value(v.data()); + } + } else { + let mut val_idx = 0usize; + for d in defs.iter().take(records) { + if *d >= max_def { + builder.append_value(values[val_idx].data()); + val_idx += 1; + } else { + builder.append_null(); + } + } + } + builder.finish() +} + +fn build_large_binary_array( + records: usize, + defs: &[i16], + max_def: i16, + values: &[ByteArray], +) -> LargeBinaryArray { + let mut builder = arrow::array::LargeBinaryBuilder::with_capacity(records, records * 8); + if defs.is_empty() || max_def == 0 { + for v in values { + builder.append_value(v.data()); + } + } else { + let mut val_idx = 0usize; + for d in defs.iter().take(records) { + if *d >= max_def { + builder.append_value(values[val_idx].data()); + val_idx += 1; + } else { + builder.append_null(); + } + } + } + builder.finish() +} + +fn scalar_buffer_from_present( + records: usize, + defs: &[i16], + max_def: i16, + values: &[T], + cast: F, +) -> ScalarBuffer +where + T: Copy, + U: arrow::datatypes::ArrowNativeType, + F: Fn(T) -> U, +{ + if defs.is_empty() || max_def == 0 { + let casted: Vec = values.iter().copied().map(&cast).collect(); + return ScalarBuffer::from(casted); + } + let mut out: Vec = Vec::with_capacity(records); + let mut val_idx = 0usize; + for d in defs.iter().take(records) { + if *d >= max_def { + out.push(cast(values[val_idx])); + val_idx += 1; + } else { + out.push(U::default()); + } + } + ScalarBuffer::from(out) +} + +/// Compute the `ListArray` offsets vector from Dremel def/rep levels +/// for the `List` shape used by DDSketch +/// `keys` / `counts`. `max_def = 1`, `max_rep = 1`. Each entry with +/// `def == 1` is a present element; `def == 0` is an empty list slot +/// (no element). `rep == 0` starts a new row; `rep == 1` continues the +/// current list. +fn list_offsets_from_levels(defs: &[i16], reps: &[i16]) -> Vec { + // Compute as `i64` so the same offset buffer can back either a + // `ListArray` (truncated to i32 by `wrap_inner_in_list`) or a + // `LargeListArray`. For our `max_def = 1, max_rep = 1` shape + // there's no risk of overflow in either width. + if defs.is_empty() { + return vec![0]; + } + let mut offsets: Vec = Vec::with_capacity(defs.len() + 1); + offsets.push(0); + let mut current_len: i64 = 0; + for i in 0..defs.len() { + if i > 0 && reps[i] == 0 { + // Boundary between rows: close the previous row's list. + offsets.push(current_len); + } + if defs[i] == 1 { + current_len += 1; + } + } + offsets.push(current_len); + offsets +} + +/// Build a `ListArray` over Int32-physical values (parquet inner type +/// Int8/Int16/Int32 or UInt8/UInt16/UInt32). Outer + inner must be +/// non-nullable. +fn build_list_i32_array( + field: &Field, + defs: &[i16], + reps: &[i16], + values: &[i32], +) -> Result { + let inner_field = list_inner_field(field); + if field.is_nullable() || inner_field.is_nullable() { + return Err(PageDecodeError::UnsupportedColumnType { + name: field.name().to_string(), + physical: PhysicalType::INT32, + arrow: field.data_type().clone(), + }); + } + let inner_array: ArrayRef = match inner_field.data_type() { + DataType::Int8 => { + let v: Vec = values.iter().map(|&x| x as i8).collect(); + Arc::new(arrow::array::Int8Array::from(v)) + } + DataType::Int16 => { + let v: Vec = values.iter().map(|&x| x as i16).collect(); + Arc::new(Int16Array::from(v)) + } + DataType::Int32 => Arc::new(Int32Array::from(values.to_vec())), + DataType::UInt8 => { + let v: Vec = values.iter().map(|&x| x as u8).collect(); + Arc::new(UInt8Array::from(v)) + } + DataType::UInt16 => { + let v: Vec = values.iter().map(|&x| x as u16).collect(); + Arc::new(UInt16Array::from(v)) + } + DataType::UInt32 => { + // Bit-reinterpret cast preserves the unsigned-logical + // round trip — same convention as the flat path. + let v: Vec = values.iter().map(|&x| x as u32).collect(); + Arc::new(UInt32Array::from(v)) + } + other => { + return Err(PageDecodeError::UnsupportedColumnType { + name: field.name().to_string(), + physical: PhysicalType::INT32, + arrow: other.clone(), + }); + } + }; + wrap_inner_in_list(field, inner_field, inner_array, defs, reps) +} + +/// Build a `ListArray` over Int64-physical values (parquet inner type +/// Int64 or UInt64). Outer + inner must be non-nullable. DDSketch +/// `counts` is the primary use. +fn build_list_i64_array( + field: &Field, + defs: &[i16], + reps: &[i16], + values: &[i64], +) -> Result { + let inner_field = list_inner_field(field); + if field.is_nullable() || inner_field.is_nullable() { + return Err(PageDecodeError::UnsupportedColumnType { + name: field.name().to_string(), + physical: PhysicalType::INT64, + arrow: field.data_type().clone(), + }); + } + let inner_array: ArrayRef = match inner_field.data_type() { + DataType::Int64 => Arc::new(Int64Array::from(values.to_vec())), + DataType::UInt64 => { + let v: Vec = values.iter().map(|&x| x as u64).collect(); + Arc::new(UInt64Array::from(v)) + } + other => { + return Err(PageDecodeError::UnsupportedColumnType { + name: field.name().to_string(), + physical: PhysicalType::INT64, + arrow: other.clone(), + }); + } + }; + wrap_inner_in_list(field, inner_field, inner_array, defs, reps) +} + +/// Build a `ListArray` / `LargeListArray` over `Float32` (parquet +/// inner type `Float32`). Outer + inner must be non-nullable. +fn build_list_f32_array( + field: &Field, + defs: &[i16], + reps: &[i16], + values: &[f32], +) -> Result { + let inner_field = list_inner_field(field); + if field.is_nullable() || inner_field.is_nullable() { + return Err(PageDecodeError::UnsupportedColumnType { + name: field.name().to_string(), + physical: PhysicalType::FLOAT, + arrow: field.data_type().clone(), + }); + } + let inner_array: ArrayRef = match inner_field.data_type() { + DataType::Float32 => Arc::new(Float32Array::from(values.to_vec())), + other => { + return Err(PageDecodeError::UnsupportedColumnType { + name: field.name().to_string(), + physical: PhysicalType::FLOAT, + arrow: other.clone(), + }); + } + }; + wrap_inner_in_list(field, inner_field, inner_array, defs, reps) +} + +/// Build a `ListArray` / `LargeListArray` over `Float64` (parquet +/// inner type `Double`). Outer + inner must be non-nullable. +fn build_list_f64_array( + field: &Field, + defs: &[i16], + reps: &[i16], + values: &[f64], +) -> Result { + let inner_field = list_inner_field(field); + if field.is_nullable() || inner_field.is_nullable() { + return Err(PageDecodeError::UnsupportedColumnType { + name: field.name().to_string(), + physical: PhysicalType::DOUBLE, + arrow: field.data_type().clone(), + }); + } + let inner_array: ArrayRef = match inner_field.data_type() { + DataType::Float64 => Arc::new(Float64Array::from(values.to_vec())), + other => { + return Err(PageDecodeError::UnsupportedColumnType { + name: field.name().to_string(), + physical: PhysicalType::DOUBLE, + arrow: other.clone(), + }); + } + }; + wrap_inner_in_list(field, inner_field, inner_array, defs, reps) +} + +/// Extract the inner field from a `List` or `LargeList` +/// outer field. Callers must already have checked the outer type. +fn list_inner_field(field: &Field) -> Arc { + match field.data_type() { + DataType::List(inner) | DataType::LargeList(inner) => Arc::clone(inner), + _ => unreachable!("caller guards on List/LargeList"), + } +} + +/// Wrap a decoded inner array in `ListArray` (i32 offsets) or +/// `LargeListArray` (i64 offsets) according to the outer field's +/// `DataType`. This preserves the schema's list flavour through the +/// page decoder — `LargeList` inputs round-trip to `LargeListArray`, +/// not `ListArray`, so downstream schema validation sees the right +/// type. Reps + defs are interpreted under `max_def = 1, max_rep = 1` +/// (the writer's contract for non-nullable outer + non-nullable inner +/// lists). +fn wrap_inner_in_list( + field: &Field, + inner_field: Arc, + inner_array: ArrayRef, + defs: &[i16], + reps: &[i16], +) -> Result { + let i64_offsets = list_offsets_from_levels(defs, reps); + match field.data_type() { + DataType::LargeList(_) => { + let offsets = OffsetBuffer::new(ScalarBuffer::from(i64_offsets)); + Ok(Arc::new(LargeListArray::new( + inner_field, + offsets, + inner_array, + None, + ))) + } + DataType::List(_) => { + let i32_offsets: Vec = i64_offsets.iter().map(|&o| o as i32).collect(); + let offsets = OffsetBuffer::new(ScalarBuffer::from(i32_offsets)); + Ok(Arc::new(ListArray::new( + inner_field, + offsets, + inner_array, + None, + ))) + } + _ => unreachable!("caller guards on List/LargeList"), + } +} + +fn max_def_for(field: &Field) -> i16 { + if field.is_nullable() { 1 } else { 0 } +} + +#[cfg(test)] +mod tests { + use std::io; + use std::ops::Range; + use std::path::{Path, PathBuf}; + + use arrow::array::{ArrayRef, DictionaryArray, RecordBatch}; + use arrow::compute::concat_batches; + use arrow::datatypes::{Field as ArrowField, Int32Type, Schema as ArrowSchema}; + use async_trait::async_trait; + use parquet::arrow::ArrowWriter; + use parquet::basic::Compression; + use parquet::file::properties::WriterProperties; + use parquet::file::reader::{FileReader, SerializedFileReader}; + use tokio::io::AsyncRead; + + use super::*; + use crate::storage::streaming_reader::{ + ColumnPageStream, RemoteByteSource, StreamingParquetReader, + }; + + fn make_metrics_batch(num_rows: usize) -> RecordBatch { + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(vec![ + ArrowField::new("metric_name", dict_type.clone(), false), + ArrowField::new("metric_type", DataType::UInt8, false), + ArrowField::new("timestamp_secs", DataType::UInt64, false), + ArrowField::new("value", DataType::Float64, false), + ArrowField::new("timeseries_id", DataType::Int64, false), + ArrowField::new("service", dict_type, true), + ])); + + let metric_keys: Vec = (0..num_rows as i32).map(|i| i % 2).collect(); + let metric_values = StringArray::from(vec!["cpu.usage", "memory.used"]); + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(metric_values), + ) + .unwrap(), + ); + let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; num_rows])); + let timestamps: Vec = (0..num_rows as u64).map(|i| 1_700_000_000 + i).collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let values: Vec = (0..num_rows).map(|i| i as f64).collect(); + let value: ArrayRef = Arc::new(Float64Array::from(values)); + let tsids: Vec = (0..num_rows as i64).map(|i| 1000 + i).collect(); + let timeseries_id: ArrayRef = Arc::new(Int64Array::from(tsids)); + // sprinkle nulls + let svc_keys: Vec> = (0..num_rows as i32) + .map(|i| if i % 5 == 0 { None } else { Some(i % 3) }) + .collect(); + let svc_values = StringArray::from(vec!["api", "db", "cache"]); + let service: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(svc_keys), + Arc::new(svc_values), + ) + .unwrap(), + ); + + RecordBatch::try_new( + schema, + vec![ + metric_name, + metric_type, + timestamp_secs, + value, + timeseries_id, + service, + ], + ) + .unwrap() + } + + fn write_parquet( + batches: &[RecordBatch], + page_row_limit: Option, + rg_row_limit: Option, + compression: Compression, + ) -> Vec { + let schema = batches[0].schema(); + let mut props_builder = WriterProperties::builder().set_compression(compression); + if let Some(n) = page_row_limit { + props_builder = props_builder.set_data_page_row_count_limit(n); + } + if let Some(n) = rg_row_limit { + props_builder = props_builder.set_max_row_group_row_count(Some(n)); + } + let props = props_builder.build(); + let mut buf: Vec = Vec::new(); + let mut writer = ArrowWriter::try_new(&mut buf, schema, Some(props)).unwrap(); + for b in batches { + writer.write(b).unwrap(); + } + writer.close().unwrap(); + buf + } + + #[derive(Clone)] + struct InMemorySource { + bytes: Bytes, + } + impl InMemorySource { + fn new(bytes: Vec) -> Arc { + Arc::new(Self { + bytes: Bytes::from(bytes), + }) + } + } + #[async_trait] + impl RemoteByteSource for InMemorySource { + async fn file_size(&self, _path: &Path) -> io::Result { + Ok(self.bytes.len() as u64) + } + async fn get_slice(&self, _path: &Path, range: Range) -> io::Result { + Ok(self.bytes.slice(range.start as usize..range.end as usize)) + } + async fn get_slice_stream( + &self, + _path: &Path, + range: Range, + ) -> io::Result> { + let slice = self.bytes.slice(range.start as usize..range.end as usize); + Ok(Box::new(io::Cursor::new(slice.to_vec()))) + } + } + fn dummy_path() -> PathBuf { + PathBuf::from("test.parquet") + } + + fn read_canonical(bytes: &[u8]) -> RecordBatch { + let cursor = Bytes::copy_from_slice(bytes); + let builder = + parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder::try_new(cursor).unwrap(); + let schema = builder.schema().clone(); + let reader = builder.build().unwrap(); + let batches: Vec = reader.collect::, _>>().unwrap(); + if batches.is_empty() { + RecordBatch::new_empty(schema) + } else { + concat_batches(&schema, &batches).unwrap() + } + } + + /// Drain every DecodedPage from the decoder and assemble the + /// resulting per-(rg, col) arrays back into a single RecordBatch by + /// concatenation in storage order. This is what PR-6b would do if it + /// wanted a full RecordBatch view; the decoder itself never + /// materialises one. + async fn drain_to_record_batch(reader: &mut StreamingParquetReader) -> RecordBatch { + let metadata = Arc::clone(reader.metadata()); + let parquet_schema = metadata.file_metadata().schema_descr(); + // None for kv_metadata: skip the ARROW:schema hint so the + // computed schema matches what the decoder actually produces + // (Utf8 instead of Dictionary, etc.). + let arrow_schema = parquet::arrow::parquet_to_arrow_schema(parquet_schema, None).unwrap(); + let num_cols = arrow_schema.fields().len(); + let num_rgs = metadata.num_row_groups(); + + let mut per_col: Vec> = vec![Vec::new(); num_cols]; + + let mut decoder = StreamDecoder::new(reader as &mut dyn ColumnPageStream); + while let Some(dp) = decoder.decode_next_page().await.unwrap() { + per_col[dp.col_idx].push(dp.array); + } + + let _ = num_rgs; + let columns: Vec = per_col + .into_iter() + .map(|chunks| { + let refs: Vec<&dyn Array> = chunks.iter().map(|a| a.as_ref()).collect(); + arrow::compute::concat(&refs).unwrap() + }) + .collect(); + RecordBatch::try_new(Arc::new(arrow_schema), columns).unwrap() + } + + #[tokio::test] + async fn test_drain_single_rg_round_trip() { + let batch = make_metrics_batch(64); + let bytes = write_parquet( + std::slice::from_ref(&batch), + None, + None, + Compression::SNAPPY, + ); + let canonical = read_canonical(&bytes); + + let source = InMemorySource::new(bytes); + let mut reader = StreamingParquetReader::try_open(source, dummy_path()) + .await + .unwrap(); + let drained = drain_to_record_batch(&mut reader).await; + + // Compare per-column. Dict columns decode to their value type + // (Utf8), so compare against the canonical's value-cast. + for (col_idx, want_field) in canonical.schema().fields().iter().enumerate() { + let want = canonical.column(col_idx); + let got = drained.column(col_idx); + assert_eq!(want.len(), got.len(), "col {col_idx} length mismatch",); + // Cast the canonical to the decoded type for comparison. + let want_cast = arrow::compute::cast(want, got.data_type()).unwrap(); + assert_eq!( + want_cast.as_ref(), + got.as_ref(), + "col {col_idx} ({}) data mismatch", + want_field.name(), + ); + } + } + + #[tokio::test] + async fn test_drain_multi_rg_round_trip() { + let batch = make_metrics_batch(300); + let bytes = write_parquet(&[batch], None, Some(100), Compression::SNAPPY); + let canonical = read_canonical(&bytes); + + let source = InMemorySource::new(bytes.clone()); + let sync_reader = SerializedFileReader::new(Bytes::from(bytes)).unwrap(); + assert!( + sync_reader.metadata().num_row_groups() >= 2, + "fixture must produce multi-RG", + ); + + let mut reader = StreamingParquetReader::try_open(source, dummy_path()) + .await + .unwrap(); + let drained = drain_to_record_batch(&mut reader).await; + for (col_idx, _) in canonical.schema().fields().iter().enumerate() { + let want = canonical.column(col_idx); + let got = drained.column(col_idx); + assert_eq!(want.len(), got.len()); + let want_cast = arrow::compute::cast(want, got.data_type()).unwrap(); + assert_eq!(want_cast.as_ref(), got.as_ref(), "col {col_idx} mismatch"); + } + } + + /// Each `decode_next_page` returns exactly one data page worth of + /// rows — `row_start + array.len()` advances monotonically per + /// (rg, col), with row_start = 0 at the start of each (rg, col). + #[tokio::test] + async fn test_decoded_page_row_indexing() { + let batch = make_metrics_batch(2048); + let bytes = write_parquet( + std::slice::from_ref(&batch), + Some(256), + None, + Compression::SNAPPY, + ); + + let source = InMemorySource::new(bytes); + let mut reader = StreamingParquetReader::try_open(source, dummy_path()) + .await + .unwrap(); + let mut decoder = StreamDecoder::new(&mut reader as &mut dyn ColumnPageStream); + + let mut per_col_cumulative: HashMap<(usize, usize), usize> = HashMap::new(); + let mut next_idx: HashMap<(usize, usize), usize> = HashMap::new(); + while let Some(dp) = decoder.decode_next_page().await.unwrap() { + let key = (dp.rg_idx, dp.col_idx); + let prior = per_col_cumulative.get(&key).copied().unwrap_or(0); + assert_eq!( + dp.row_start, prior, + "row_start for ({}, {}) page {} should equal prior cumulative", + dp.rg_idx, dp.col_idx, dp.page_idx_in_col, + ); + per_col_cumulative.insert(key, prior + dp.array.len()); + + let expected_idx = next_idx.get(&key).copied().unwrap_or(0); + assert_eq!(dp.page_idx_in_col, expected_idx); + next_idx.insert(key, expected_idx + 1); + } + } + + /// `decode_next_page` is idempotent at EOF. + #[tokio::test] + async fn test_eof_idempotent() { + let batch = make_metrics_batch(32); + let bytes = write_parquet( + std::slice::from_ref(&batch), + None, + None, + Compression::SNAPPY, + ); + let source = InMemorySource::new(bytes); + let mut reader = StreamingParquetReader::try_open(source, dummy_path()) + .await + .unwrap(); + let mut decoder = StreamDecoder::new(&mut reader as &mut dyn ColumnPageStream); + while decoder.decode_next_page().await.unwrap().is_some() {} + assert!(decoder.decode_next_page().await.unwrap().is_none()); + assert!(decoder.decode_next_page().await.unwrap().is_none()); + } + + /// Nullable columns: `service` has nulls every 5th row. The decoded + /// page must surface those nulls — the null mask round-trips. + #[tokio::test] + async fn test_nullable_column_round_trip() { + let batch = make_metrics_batch(50); + let bytes = write_parquet( + std::slice::from_ref(&batch), + None, + None, + Compression::SNAPPY, + ); + let canonical = read_canonical(&bytes); + let svc_idx = canonical.schema().index_of("service").unwrap(); + assert!(canonical.column(svc_idx).null_count() > 0); + + let source = InMemorySource::new(bytes); + let mut reader = StreamingParquetReader::try_open(source, dummy_path()) + .await + .unwrap(); + let drained = drain_to_record_batch(&mut reader).await; + assert_eq!( + drained.column(svc_idx).null_count(), + canonical.column(svc_idx).null_count(), + ); + } + + /// Compression codec round trips. The crate's parquet feature set + /// is `arrow + snap + zstd` (see `quickwit/Cargo.toml`); LZ4 is + /// intentionally not in scope. + #[tokio::test] + async fn test_compression_codecs() { + for compression in [ + Compression::UNCOMPRESSED, + Compression::SNAPPY, + Compression::ZSTD(parquet::basic::ZstdLevel::default()), + ] { + let batch = make_metrics_batch(64); + let bytes = write_parquet(std::slice::from_ref(&batch), None, None, compression); + let canonical = read_canonical(&bytes); + let source = InMemorySource::new(bytes); + let mut reader = StreamingParquetReader::try_open(source, dummy_path()) + .await + .unwrap(); + let drained = drain_to_record_batch(&mut reader).await; + for col_idx in 0..canonical.num_columns() { + let want = canonical.column(col_idx); + let got = drained.column(col_idx); + assert_eq!(want.len(), got.len()); + let want_cast = arrow::compute::cast(want, got.data_type()).unwrap(); + assert_eq!( + want_cast.as_ref(), + got.as_ref(), + "compression {compression:?} col {col_idx} diverged", + ); + } + } + } + + /// The decoder must not buffer the row group: across a long stream, + /// the number of pages held in any single column's `PageQueue` at + /// any instant stays ≤ 2 (at most a queued dictionary plus the + /// current data page). This is a structural check of the page- + /// bounded contract. + #[tokio::test] + async fn test_page_bounded_queue_depth() { + let batch = make_metrics_batch(8192); + let bytes = write_parquet( + std::slice::from_ref(&batch), + Some(256), + None, + Compression::SNAPPY, + ); + let source = InMemorySource::new(bytes); + let mut reader = StreamingParquetReader::try_open(source, dummy_path()) + .await + .unwrap(); + let mut decoder = StreamDecoder::new(&mut reader as &mut dyn ColumnPageStream); + while decoder.decode_next_page().await.unwrap().is_some() { + for (_, state) in decoder.columns.iter() { + let depth = state.queue.lock().unwrap().len(); + assert!( + depth <= 2, + "PageQueue depth {depth} exceeds page-bounded contract (≤2)", + ); + } + } + } + + /// `List` (the DDSketch `counts` shape) round-trips with + /// variable list lengths including the empty list and `u64::MAX`. + /// This exercises the Dremel level → ListArray reconstruction in + /// `build_list_i64_array`. + #[tokio::test] + async fn test_list_uint64_round_trip() { + use arrow::array::ListBuilder; + + let item_field = Arc::new(ArrowField::new("item", DataType::UInt64, false)); + let schema = Arc::new(ArrowSchema::new(vec![ArrowField::new( + "counts", + DataType::List(Arc::clone(&item_field)), + false, + )])); + + let rows: Vec> = vec![ + vec![1, 2, 3], + vec![], + vec![42], + vec![u64::MAX, 0, 0x8000_0000_0000_0000], + vec![], + vec![100], + ]; + let mut builder = ListBuilder::new(arrow::array::UInt64Builder::new()) + .with_field(Arc::clone(&item_field)); + for row in &rows { + for &v in row { + builder.values().append_value(v); + } + builder.append(true); + } + let counts: ArrayRef = Arc::new(builder.finish()); + let batch = RecordBatch::try_new(schema.clone(), vec![counts]).unwrap(); + let bytes = write_parquet( + std::slice::from_ref(&batch), + None, + None, + Compression::SNAPPY, + ); + + let source = InMemorySource::new(bytes); + let mut reader = StreamingParquetReader::try_open(source, dummy_path()) + .await + .unwrap(); + let mut decoder = StreamDecoder::new(&mut reader as &mut dyn ColumnPageStream); + let dp = decoder + .decode_next_page() + .await + .unwrap() + .expect("at least one page"); + let got_list = dp + .array + .as_any() + .downcast_ref::() + .expect("counts must decode to ListArray"); + assert_eq!(got_list.len(), rows.len()); + for (row_idx, want) in rows.iter().enumerate() { + let got = got_list.value(row_idx); + let got_u64: Vec = got + .as_any() + .downcast_ref::() + .expect("inner must be UInt64Array") + .values() + .to_vec(); + assert_eq!(got_u64, *want, "row {row_idx} list mismatch"); + } + } + + /// `List` round-trips through the decoder as a `ListArray` + /// with a `Float64Array` inner — NOT as a flat `Float64Array`. The + /// type/row shape must match what the streaming writer advertises + /// for `List` columns. Regression test for the codex + /// review on PR-6407. + #[tokio::test] + async fn test_list_float64_round_trip() { + use arrow::array::ListBuilder; + + let item_field = Arc::new(ArrowField::new("item", DataType::Float64, false)); + let schema = Arc::new(ArrowSchema::new(vec![ArrowField::new( + "samples", + DataType::List(Arc::clone(&item_field)), + false, + )])); + + let rows: Vec> = vec![ + vec![1.0, 2.5, -7.5], + vec![], + vec![f64::MAX, f64::MIN, 0.0], + vec![42.42], + ]; + let mut builder = ListBuilder::new(arrow::array::Float64Builder::new()) + .with_field(Arc::clone(&item_field)); + for row in &rows { + for &v in row { + builder.values().append_value(v); + } + builder.append(true); + } + let samples: ArrayRef = Arc::new(builder.finish()); + let batch = RecordBatch::try_new(schema.clone(), vec![samples]).unwrap(); + let bytes = write_parquet( + std::slice::from_ref(&batch), + None, + None, + Compression::SNAPPY, + ); + + let source = InMemorySource::new(bytes); + let mut reader = StreamingParquetReader::try_open(source, dummy_path()) + .await + .unwrap(); + let mut decoder = StreamDecoder::new(&mut reader as &mut dyn ColumnPageStream); + let dp = decoder + .decode_next_page() + .await + .unwrap() + .expect("at least one page"); + let got_list = dp + .array + .as_any() + .downcast_ref::() + .expect("samples must decode to ListArray, not flat Float64Array"); + assert_eq!(got_list.len(), rows.len()); + for (row_idx, want) in rows.iter().enumerate() { + let got = got_list.value(row_idx); + let got_f64: Vec = got + .as_any() + .downcast_ref::() + .expect("inner must be Float64Array") + .values() + .to_vec(); + assert_eq!(got_f64, *want, "row {row_idx} list mismatch"); + } + } + + /// `List` records that span multiple V1 pages are decoded + /// without splitting. Regression test for the codex review on + /// PR-6407: prior to the one-page lookahead in `decode_next_page`, + /// `peek_next_page` returned `None` at every page boundary, which + /// parquet-rs treats as "last page" — it would flush partial + /// repetition-level state and emit incomplete records. + /// + /// We force the issue by writing a long list with a tiny + /// `data_page_size_limit`, so parquet-rs splits the single + /// list record across multiple V1 pages. With the lookahead in + /// place, the column reader sees `peek_next_page = Some(_)` and + /// continues consuming until the record completes. + #[tokio::test] + async fn test_list_record_spanning_pages_preserved() { + use arrow::array::{ListBuilder, UInt64Array, UInt64Builder}; + + let item_field = Arc::new(ArrowField::new("item", DataType::UInt64, false)); + let schema = Arc::new(ArrowSchema::new(vec![ArrowField::new( + "counts", + DataType::List(Arc::clone(&item_field)), + false, + )])); + + // Two records: a 50-element list (forced to span several pages + // by the 20-byte page size limit) and a short 3-element list. + let row_long: Vec = (0..50u64).collect(); + let row_short: Vec = vec![1, 2, 3]; + let mut builder = + ListBuilder::new(UInt64Builder::new()).with_field(Arc::clone(&item_field)); + for v in &row_long { + builder.values().append_value(*v); + } + builder.append(true); + for v in &row_short { + builder.values().append_value(*v); + } + builder.append(true); + let counts: ArrayRef = Arc::new(builder.finish()); + let batch = RecordBatch::try_new(schema.clone(), vec![counts]).unwrap(); + + // 20-byte page-size limit forces V1 pages to split the + // 50-element list across several pages. + let props = WriterProperties::builder() + .set_compression(Compression::UNCOMPRESSED) + .set_data_page_size_limit(20) + .build(); + let mut buf: Vec = Vec::new(); + let mut writer = ArrowWriter::try_new(&mut buf, schema, Some(props)).unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + + // Decode all pages and accumulate the lists. + let source = InMemorySource::new(buf); + let mut reader = StreamingParquetReader::try_open(source, dummy_path()) + .await + .unwrap(); + let mut decoder = StreamDecoder::new(&mut reader as &mut dyn ColumnPageStream); + let mut all_records: Vec> = Vec::new(); + while let Some(dp) = decoder.decode_next_page().await.unwrap() { + let list = dp + .array + .as_any() + .downcast_ref::() + .expect("counts must decode to ListArray"); + for i in 0..list.len() { + let inner = list.value(i); + let u64_arr = inner + .as_any() + .downcast_ref::() + .expect("inner must be UInt64Array"); + all_records.push(u64_arr.values().to_vec()); + } + } + + assert_eq!( + all_records.len(), + 2, + "must emit exactly two records, got {}: {all_records:?}", + all_records.len() + ); + assert_eq!( + all_records[0], row_long, + "first record (50 elements) must be preserved intact across page boundaries" + ); + assert_eq!(all_records[1], row_short, "second record must be preserved",); + } + + /// `wrap_inner_in_list` dispatches to `LargeListArray` (i64 offsets) + /// when the outer field is `LargeList<>`, and to `ListArray` (i32 + /// offsets) when it's `List<>`. Regression for codex review on + /// PR-6407: the builders accept either outer flavour but + /// previously always constructed `ListArray`. + /// + /// Tested at the helper level rather than via parquet round-trip: + /// `init_column_state` derives fields from + /// `parquet_to_arrow_schema(_, None)`, which only produces + /// `List<>` (parquet's native schema doesn't distinguish list + /// offset widths). The `LargeList<>` branch is reachable only + /// when callers construct fields directly, so we exercise it + /// directly. + #[test] + fn test_wrap_inner_in_list_dispatches_on_outer_flavour() { + use arrow::array::UInt64Array; + + let inner_field = Arc::new(ArrowField::new("item", DataType::UInt64, false)); + let inner_array: ArrayRef = Arc::new(UInt64Array::from(vec![1u64, 2, 3, 42, 100, 200])); + // Three rows: [1,2,3], [42], [100,200] → defs/reps that + // `list_offsets_from_levels` translates to offsets [0,3,4,6]. + let defs = vec![1, 1, 1, 1, 1, 1]; + let reps = vec![0, 1, 1, 0, 0, 1]; + + // List path → ListArray with i32 offsets. + let list_field = ArrowField::new( + "list_field", + DataType::List(Arc::clone(&inner_field)), + false, + ); + let got_list = wrap_inner_in_list( + &list_field, + Arc::clone(&inner_field), + Arc::clone(&inner_array), + &defs, + &reps, + ) + .expect("list dispatch"); + let list_arr = got_list + .as_any() + .downcast_ref::() + .expect("list field must produce ListArray"); + assert_eq!(list_arr.len(), 3); + assert_eq!(list_arr.value(0).len(), 3); + assert_eq!(list_arr.value(1).len(), 1); + assert_eq!(list_arr.value(2).len(), 2); + + // LargeList path → LargeListArray with i64 offsets. + let large_field = ArrowField::new( + "large_field", + DataType::LargeList(Arc::clone(&inner_field)), + false, + ); + let got_large = wrap_inner_in_list( + &large_field, + Arc::clone(&inner_field), + Arc::clone(&inner_array), + &defs, + &reps, + ) + .expect("large list dispatch"); + let large_arr = got_large + .as_any() + .downcast_ref::() + .expect("LargeList field must produce LargeListArray, not ListArray"); + assert_eq!(large_arr.len(), 3); + assert_eq!(large_arr.value(0).len(), 3); + assert_eq!(large_arr.value(1).len(), 1); + assert_eq!(large_arr.value(2).len(), 2); + } + + /// I/O failures from the page stream surface as + /// `PageDecodeError::PageStream(ParquetReadError::Io(_))`. + #[tokio::test] + async fn test_io_failure_surfaces_as_page_stream_error() { + struct FailingBodySource { + footer: Bytes, + file_size: u64, + } + #[async_trait] + impl RemoteByteSource for FailingBodySource { + async fn file_size(&self, _path: &Path) -> io::Result { + Ok(self.file_size) + } + async fn get_slice(&self, _path: &Path, range: Range) -> io::Result { + if range.start >= self.file_size - self.footer.len() as u64 { + let foot_start = self.file_size - self.footer.len() as u64; + let off = (range.start - foot_start) as usize; + let len = (range.end - range.start) as usize; + return Ok(self.footer.slice(off..off + len)); + } + Err(io::Error::other("simulated body get failure")) + } + async fn get_slice_stream( + &self, + _path: &Path, + _range: Range, + ) -> io::Result> { + Err(io::Error::other("simulated body stream failure")) + } + } + let batch = make_metrics_batch(16); + let bytes = write_parquet( + std::slice::from_ref(&batch), + None, + None, + Compression::SNAPPY, + ); + let file_size = bytes.len() as u64; + let source = Arc::new(FailingBodySource { + footer: Bytes::from(bytes), + file_size, + }); + let mut reader = StreamingParquetReader::try_open(source, dummy_path()) + .await + .unwrap(); + let mut decoder = StreamDecoder::new(&mut reader as &mut dyn ColumnPageStream); + let err = decoder.decode_next_page().await.unwrap_err(); + match err { + PageDecodeError::PageStream(ParquetReadError::Io(_)) => {} + other => panic!("expected PageStream(Io), got {other:?}"), + } + } +} diff --git a/quickwit/quickwit-parquet-engine/src/storage/streaming_reader.rs b/quickwit/quickwit-parquet-engine/src/storage/streaming_reader.rs index a88f51183b2..5001935e13c 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/streaming_reader.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/streaming_reader.rs @@ -155,12 +155,18 @@ pub enum ParquetReadError { /// One Parquet page yielded by [`StreamingParquetReader::next_page`]. /// -/// Carries the Thrift-decoded `PageHeader` plus the raw compressed -/// bytes (`bytes.len() == header.compressed_page_size`). Caller can -/// inspect the page type (`Dictionary` / `DataPage` / `DataPageV2` / -/// `Index`) via `header.type_`, and either copy `bytes` straight to -/// an output writer (PR-6's direct page copy) or decompress + decode -/// for sort-column inspection. +/// Carries the Thrift-decoded `PageHeader`, the original Thrift-compact +/// bytes for that header (`header_bytes`), and the raw compressed page +/// bytes (`bytes.len() == header.compressed_page_size`). Caller can: +/// +/// - Inspect the page type (`Dictionary` / `DataPage` / `DataPageV2` / `Index`) via `header.type_`. +/// - Copy `bytes` straight to an output writer (PR-6's direct page copy). +/// - Reconstruct the original column-chunk byte stream by concatenating `header_bytes ++ bytes` for +/// every page in storage order — what PR-6a's page decoder uses to feed pages back into the +/// standard parquet record-batch reader without re-encoding (re-encoding is deterministic for +/// Thrift compact, but byte-exact passthrough avoids any encoder-version drift inside the +/// compactor). +/// - Decompress + decode `bytes` for sort-column inspection. #[derive(Debug)] pub struct Page { /// Row group this page belongs to. @@ -172,6 +178,10 @@ pub struct Page { pub page_idx_in_col: usize, /// Thrift-decoded page header. pub header: PageHeader, + /// Original Thrift-compact bytes for `header`, exactly as they + /// appeared on the wire. `header_bytes.len()` equals the number of + /// bytes the parser consumed to decode `header`. + pub header_bytes: Bytes, /// Raw compressed page bytes; length equals /// `header.compressed_page_size`. Cheap to clone (ref-counted). pub bytes: Bytes, @@ -426,7 +436,7 @@ async fn read_one_page( // protocol. Header is variable-length; iterate until we have // enough buffered to parse, capped at `max_page_header_bytes`. let header_offset = state.cursor; - let (header, header_len) = + let (header, header_len, header_bytes) = parse_page_header_streaming(state, config.max_page_header_bytes, header_offset).await?; // Header was consumed from `pending`; `cursor` and `bytes_consumed_in_col` @@ -464,6 +474,7 @@ async fn read_one_page( col_idx, page_idx_in_col, header, + header_bytes, bytes: Bytes::from(body_bytes), }) } @@ -471,12 +482,15 @@ async fn read_one_page( /// Read the next Thrift `PageHeader` by trying to decode from /// progressively-larger buffer sizes. Drains the consumed bytes from /// `state.pending` and advances `state.cursor` and -/// `state.bytes_consumed_in_col`. +/// `state.bytes_consumed_in_col`. Returns the parsed header plus the +/// raw Thrift-compact bytes that backed it, so callers (e.g. the +/// page-stream → record-batch decoder) can reconstruct the original +/// column-chunk byte layout without re-encoding. async fn parse_page_header_streaming( state: &mut ReadingState, max_header_bytes: usize, file_offset_for_error: u64, -) -> Result<(PageHeader, usize), ParquetReadError> { +) -> Result<(PageHeader, usize, Bytes), ParquetReadError> { // Start small; grow geometrically up to the configured cap. let mut target = 256.min(max_header_bytes); loop { @@ -490,10 +504,10 @@ async fn parse_page_header_streaming( fill_pending_best_effort(state, target).await?; match try_parse_page_header(&state.pending) { Ok((header, consumed)) => { - state.pending.drain(..consumed); + let header_bytes: Vec = state.pending.drain(..consumed).collect(); state.cursor += consumed as u64; state.bytes_consumed_in_col += consumed as u64; - return Ok((header, consumed)); + return Ok((header, consumed, Bytes::from(header_bytes))); } Err(thrift_err) => { // Some thrift errors are recoverable by reading more From 7073c10468234d25268c5ced72d9fa1c212fcc19 Mon Sep 17 00:00:00 2001 From: George Talbot Date: Thu, 14 May 2026 14:06:44 -0400 Subject: [PATCH 04/10] feat: streaming column-major merge engine with page-bounded body cols (PR-6b.2) (#6409) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * feat: streaming column-major merge engine with page-bounded body cols (PR-6b.2) Rebuilds PR-6b on top of PR-6a.2's per-page Arrow decoder. The streaming merge engine now keeps body-col memory bounded by output page size (not column-chunk size) while preserving caller-specified M:N output splitting at sorted_series boundaries. Architecture (Husky multi-input → multi-output sorted merge): Phase 0 (async) — drain sort cols from each input. With Husky column ordering, sort cols + sorted_series are the prefix of each row group's body bytes, so the decoder can stop after they are fully decoded; the remaining body col pages stay un-read in the input stream, ready for phase 3. Phase 1 — compute_merge_order over the per-input sort-col RecordBatches using the existing k-way (sorted_series, timestamp_secs) heap. Phase 2 — compute_output_boundaries with the caller's num_outputs, splitting at sorted_series transitions. Phase 3 (blocking + block_on bridges) — streaming write. All M output writers are alive for the duration. For each column in Husky order, every output's col K is written in turn: - Sort col / sorted_series: applied via arrow::interleave from the already-buffered phase-0 data. - Body col: each output page is assembled via arrow::interleave from input page slices, with decoders advanced page-by-page via handle.block_on from inside the sync iterator passed to write_next_column_arrays. Pages flush to the writer's sink as SerializedColumnWriter's page-size threshold trips — memory stays bounded by the in-flight output page plus a small number of in-flight input pages. After all M outputs' col K is done, every input decoder is at the start of col K+1 in its single row group. Move to col K+1. PR-6b.2 only handles single-row-group inputs (real or PR-5- adapter-presented). Multi-RG metric-aligned inputs are rejected with a clear error message; supporting them requires either consuming + discarding body cols of RG[i-1] from the stream to reach RG[i]'s sort cols, or a second body GET — both are larger scope changes that land in a follow-up. Page-bounded contract verified by test_body_col_streams_many_pages_per_column_chunk: with data_page_row_count_limit=1000 on an 8000-row merge, the output value column spans ≥ 2 pages, demonstrating that body col writes respect data_page_size and do not materialise whole column chunks. Tests (9, all passing): two-input merge, single-RG output for single-metric_name input, total-rows-preserved across M:N, sort-schema mismatch rejection, KV metadata propagation, all-empty-inputs no-output, output drainable by StreamDecoder, multi-RG input rejection, page-bounded body col streaming. Also exposes existing helpers in merge/writer.rs as pub(super) (apply_merge_permutation, build_merge_kv_metadata, build_sorting_columns, resolve_sort_field_names, verify_sort_order) so streaming.rs can reuse the same MC-3 / KV / sorting-columns construction the non-streaming engine uses. PR-7 will fold the non-streaming engine away. PR-6c.2 will add file-size monitoring on top: close the current output at the next sorted_series transition when an in-progress file approaches the size cap, producing additional splits beyond the caller's N. Co-Authored-By: Claude Opus 4.7 (1M context) * fix: persist decoder + page cache across body-col passes Address two Codex review findings on PR-6b.2 (#6409): * P1 — Preserve decoder/page cache across output chunks. The merge engine was constructing a fresh `StreamDecoder` for every `advance_decoder_to_row` call, which reset the per-column `rows_decoded` counter so the second decoded page reported `row_start = 0` after the stream had already advanced. The page cache also lived on the per-output assembler, so pages whose row range straddled two outputs were dropped when the first output finished even though the stream couldn't be rewound. Both scenarios produced silently wrong rows or out-of-bounds panics on any input large enough to require multi-page advances per output or multi-output coverage of a single page. The decoder now lives on `InputDecoderState` (owned via the new `StreamDecoder::from_owned` constructor), and the per-input body- col page cache + cursor are reset only at the start of each body column. * P2 — Stream body pages instead of collecting `Vec`. The per-output body-col write now feeds `write_next_column_arrays` one page at a time via `StreamingBodyColIter`, which captures assembly errors in a side cell so memory stays bounded by output- page size rather than column-chunk size. Two regression tests cover the bug shapes — multi-page body col within one output (2500 rows × 50-row pages) and multi-output input where pages span output boundaries. Co-Authored-By: Claude Opus 4.7 (1M context) * fix: guard body-col path against zero-row-group inputs Address Codex P1 (third comment) on PR-6b.2 (#6409): phase 0 explicitly accepts inputs with `num_row_groups() == 0` (returning a zero-row sort batch), but `write_body_col_for_all_outputs` unconditionally called `state.metadata.row_group(0)` for every input, panicking with "index out of bounds" before the first body column was written. Treat zero-RG inputs the same as inputs whose schema lacks the current column: push `None` into `input_col_indices` and skip them for this body col. Also drop the unused `input_target_rows` vec that was being built only for its row-group lookup side effect. Regression test `test_zero_row_input_mixed_with_non_empty` builds a 0-row + 50-row pair and merges them through the streaming engine; without this fix the merge blocking task panics inside parquet-rs's `row_group()` indexing. Co-Authored-By: Claude Opus 4.7 (1M context) * fix: drop all-null sort fields from per-output streaming schema Address Codex P2 (fourth comment) on PR-6b.2 (#6409): the schema derivation condition `sort_optimised.has(name) || full_union.has(name)` was tautologically true for every iterated field — every `field` came from `full_union_schema`, so the second disjunct was always satisfied and the intended "drop all-null sort fields" branch never fired. Pass the sort union schema in explicitly so we can tell sort fields apart from body fields. Sort field present in `sort_union_schema` → keep only if `optimize_output_batch` kept it (not all-null for this output's rows). Body field → keep unconditionally; tracking per-output body-col presence would require pre-reading every body column for every output, which is the column-chunk-bounded buffering the streaming path exists to avoid. Regression test `test_derive_output_schema_drops_all_null_sort_field` calls the helper directly with a synthetic union + sort-optimised pair and asserts an all-null sort field is dropped while a body field with the same union-schema position is preserved. Verified the test fails against the pre-fix logic with the expected `['metric_name', 'env', 'timestamp_secs', 'value']` vs `['metric_name', 'timestamp_secs', 'value']` mismatch. Co-Authored-By: Claude Opus 4.7 (1M context) * chore: code-quality fixes + MC-2 type round-trip test on streaming merge Bundle three pieces: - **Husky → neutral phrasing.** Replaced the seven "Husky" mentions in the streaming engine's doc-comments and error messages with neutral "sort-cols-first storage ordering" / "column ordering" phrasing. Project is Quickwit; the internal column-ordering scheme didn't need a separate brand in user-visible error strings. - **One `.unwrap()` → `.expect()` in lib code.** The hashmap lookup in `drain_sort_cols_one_input` is guarded by a `contains_key` check; promote the implicit invariant to a documented panic message per CODE_STYLE.md. - **`align_inputs_to_union_schema` nullability fix.** The first-sight branch unconditionally marked new fields nullable; the existing comment claims "columns that don't appear in every input must be nullable" but the code applied that rule to every field. Replaced with a two-pass scheme: track `any_nullable` and `appears_in` per field across all inputs, then mark nullable iff some input had it nullable OR the field is missing from some input. This unblocks `List` columns end-to-end (the writer rejects nullable List; the previous behaviour forced every list column nullable on first sight even when every input declared it non-null). - **MC-2 round-trip integration test.** New `test_mc2_all_types_round_trip_through_streaming_merge` builds two inputs covering every parquet physical type the decoder accepts — Int8/16/32/64, UInt8/16/32/64, Float32/64, Bool, Utf8, Dictionary, LargeBinary, and non-nullable `List` — merges them through the streaming engine, and asserts every `(sorted_series_key → body-col tuple)` pair survives byte-equal. Storage-encoding transitions (Dict→Utf8, LargeBinary→ Binary) are normalised in the render helper because MC-2 promises value preservation, not internal representation preservation. This test caught two real bugs while being written: 1. Body cols must be declared in lexicographic order — the streaming engine assumes the storage convention and crashes mid-merge if they aren't. Fixture re-ordered accordingly. (Worth adding upfront validation in a follow-up; not in scope here.) 2. The schema-union nullability bug above. Co-Authored-By: Claude Opus 4.7 (1M context) * test(MS-7): page-cache bounded-memory contract is observable + asserted Add a `#[cfg(test)] static AtomicUsize` PEAK_BODY_COL_PAGE_CACHE_LEN that records the maximum length any input's `body_col_page_cache` ever reached during the current merge, bumped on every page push in `advance_decoder_to_row`. Zero production overhead — the `record_*` helper compiles to a no-op outside test builds. New test `test_ms7_body_col_page_cache_bounded_regardless_of_input_size` runs the streaming merge over three input sizes (300 / 3 000 / 30 000 rows at 50-row pages) and asserts: 1. Peak resident pages stays below a fixed ceiling (24, for the ratio of OUTPUT_PAGE_ROWS=1024 to input page_rows=50, plus a few-page slack for decoder lookahead + transients). 2. Growth from 3 000 to 30 000 rows (10× more input pages) yields at most a 2-page increase in peak. The whole MS-7 claim is that peak does not scale with input size. Verified the test catches a deliberate regression: removing the per- output-page eviction loop in `assemble_one_output_page` pushed the 3 000-row peak to 60 (60 > 24) and the test failed with the expected message. Fixture support: `write_input_parquet_with_small_pages` now also sets `write_batch_size` and `data_page_size` proportional to the requested page row count. Without those, the arrow writer's defaults (64 KiB / 1 MiB) caused `data_page_row_count_limit` to be silently ignored and produced one giant page per column. Probed the output via `get_column_page_reader` — 30 000 rows now produces 600 pages per col as expected. Co-Authored-By: Claude Opus 4.7 (1M context) * fix: drive col loop from full union schema + collect service tags from sort col Address two new Codex P2 findings on PR-6b.2 (#6409): - **Use the full union schema when driving column writes.** The old `build_parent_union_schema` picked one per-output schema by field count and used it as the column-iteration driver. If two outputs drop *different* all-null sort fields and end up with the same field count, the first wins — and any column it dropped is never iterated, leaving the other output's writer missing a column or writing subsequent columns into the wrong slot. The doc comment already said "process the FULL union schema's cols in order"; the implementation diverged. Drive `write_all_columns` from `full_union_schema` directly and delete the broken heuristic. - **Collect service names from the sort-col path too.** If the sort schema places `service` in the sort key (`metric_name|service|...`), the streaming engine writes it via the sort-col path and the body-col `track_service` branch never runs. `MergeOutputFile.low_cardinality_tags[TAG_SERVICE]` came back empty even though every row had a service value. Extract service names from `static_meta.sort_optimised` at `finalize_output_writer` time so the TAG_SERVICE metadata is accurate regardless of which write path the column took. Two regression tests: - `test_heterogeneous_dropped_fields_drive_from_full_union_schema` builds two inputs whose per-output schemas drop different all-null sort fields with the same field count. Each kept tag must survive to its output. Verified the test fails (panic on missing column) against the pre-fix logic. - `test_service_as_sort_column_still_populates_low_cardinality_tags` uses a sort schema `metric_name|service|-timestamp_secs/V2` and asserts the output's `low_cardinality_tags[TAG_SERVICE]` covers every distinct service value. Verified the test fails against pre-fix `finalize_output_writer` with the expected "must contain TAG_SERVICE" message. Co-Authored-By: Claude Opus 4.7 (1M context) * refactor(streaming): rename to fill_page_cache_to_row + cross-input docstrings Addresses adamtobey's review on PR-6409. - Rename `advance_decoder_to_row` → `fill_page_cache_to_row`. The function's effect on the world is "add pages to the per-input cache" — it never advances a cursor or skips data. The old name primed reviewers to ask "are we skipping rows?" (which is exactly what Adam asked). - Use a `rows_for_current_output` register inside `compute_input_row_destinations` and write to `rows_per_output[out_idx]` once after the inner loop; saves the per-row indexed store. - Expand `body_col_page_cache` docstring with the horizontal-vs-vertical memory bound argument and a pointer to the MS-7 invariant test (`test_ms7_body_col_page_cache_bounded_regardless_of_input_size`). - Add context comments at the cross-file invariant sites Adam flagged: - Sort-cols-first storage-ordering contract on the sort-col drain. - Single-RG-input restriction with forward pointer to PR-6c.2 (#6424) which relaxes it. - `rg_partition_prefix_len` defaulting to 0 (with reference to the legacy-promotion `mixed_prefix_ok` escape in PR-6423). No behaviour change. 461 lib tests pass; workspace clippy + nightly fmt + rustdoc clean. Co-Authored-By: Claude Opus 4.7 (1M context) * fix(streaming): assert per-input body cols are in Husky order Adam's question on L194 asked whether body-col ordering was a hard cross-file requirement. My first answer said "no" — true for which array we read (we look up by name), but wrong for the body-col memory bound: Phase 3 iterates the union schema's body cols alphabetically and asks each input's decoder to advance to that col. Parquet emits column chunks in declared schema order, so the decoder reads pages in that input's storage order. If an input's body cols aren't in the same alphabetical-after-sort-cols order ("Husky order"), fill_page_cache_to_row has to drain every body col preceding the requested one on the wire — those pages land in body_col_page_caches[col_idx] until that col's turn in the union iteration. The cache grows to a full column-chunk's worth per misaligned col. Vertical, not horizontal. Defeats streaming. Catch this at merge entry instead of silently degrading to vertical caching: - `assert_inputs_in_husky_body_col_order` runs after `build_input_decoders_state` and before phase 0. Bails with a concrete error message naming the offending pair of column names. - New regression test `test_assert_inputs_in_husky_body_col_order_rejects_misaligned_input` builds an input with body cols `[value, metric_type]` (alphabetical would be `[metric_type, value]`) and asserts the merge errors out before phase 3. No production producer violates this today (streaming writer and legacy Husky writer both emit lexicographic body cols), so the assertion catches future producer drift, not current traffic. 462 lib tests pass (461 prior + 1 new); workspace clippy + nightly fmt + rustdoc clean. Co-Authored-By: Claude Opus 4.7 (1M context) --------- Co-authored-by: Claude Opus 4.7 (1M context) --- .../quickwit-parquet-engine/src/merge/mod.rs | 1 + .../src/merge/schema.rs | 61 +- .../src/merge/streaming.rs | 3800 +++++++++++++++++ .../src/merge/writer.rs | 13 +- .../src/storage/page_decoder.rs | 63 +- 5 files changed, 3913 insertions(+), 25 deletions(-) create mode 100644 quickwit/quickwit-parquet-engine/src/merge/streaming.rs diff --git a/quickwit/quickwit-parquet-engine/src/merge/mod.rs b/quickwit/quickwit-parquet-engine/src/merge/mod.rs index 008c456208a..fcc353992a9 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/mod.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/mod.rs @@ -24,6 +24,7 @@ mod merge_order; pub mod metadata_aggregation; pub mod policy; mod schema; +pub mod streaming; mod writer; #[cfg(test)] diff --git a/quickwit/quickwit-parquet-engine/src/merge/schema.rs b/quickwit/quickwit-parquet-engine/src/merge/schema.rs index 78c554d2cb3..68ebf851e59 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/schema.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/schema.rs @@ -59,45 +59,74 @@ pub fn align_inputs_to_union_schema( bail!("no inputs to align"); } - // Collect all fields across all inputs, checking for type conflicts. - // String-like types are normalized to Utf8 for internal alignment. - let mut field_map: BTreeMap> = BTreeMap::new(); + // Track each field's normalized type, whether any input declared + // it nullable, and how many of the input batches contain it. The + // union field is nullable iff some input observed it as nullable + // OR some input is missing the field entirely (a row from a + // missing-the-field input will be null in the merged output). + // The previous version always defaulted new fields to nullable on + // first sight, which broke columns whose nullability must be + // preserved (e.g. `List` — the writer's non-nullable- + // list contract requires the union field to stay non-nullable). + struct FieldInfo { + normalized_type: DataType, + any_nullable: bool, + appears_in: usize, + } + let mut field_map: BTreeMap = BTreeMap::new(); for (input_idx, batch) in inputs.iter().enumerate() { for field in batch.schema().fields() { let normalized_type = normalize_type(field.data_type()); - match field_map.get(field.name().as_str()) { + match field_map.get_mut(field.name().as_str()) { Some(existing) => { - if *existing.data_type() != normalized_type { + if existing.normalized_type != normalized_type { bail!( "type conflict for column '{}': input 0 has {:?}, input {} has {:?} \ (normalized: {:?} vs {:?})", field.name(), - existing.data_type(), + existing.normalized_type, input_idx, field.data_type(), - existing.data_type(), + existing.normalized_type, normalized_type, ); } - // If either side is nullable, the union must be too. - if field.is_nullable() && !existing.is_nullable() { - let nullable_field = - Arc::new(Field::new(field.name(), normalized_type, true)); - field_map.insert(field.name().clone(), nullable_field); + if field.is_nullable() { + existing.any_nullable = true; } + existing.appears_in += 1; } None => { - // Columns that don't appear in every input must be nullable. - let nullable_field = Arc::new(Field::new(field.name(), normalized_type, true)); - field_map.insert(field.name().clone(), nullable_field); + field_map.insert( + field.name().clone(), + FieldInfo { + normalized_type, + any_nullable: field.is_nullable(), + appears_in: 1, + }, + ); } } } } - // Build the union schema in Husky column order. + // Materialise `Arc` per the rule above. Keep + // `BTreeMap>` so `build_husky_ordered_schema` + // is unchanged. + let total_inputs = inputs.len(); + let field_map: BTreeMap> = field_map + .into_iter() + .map(|(name, info)| { + let nullable = info.any_nullable || info.appears_in < total_inputs; + let field = Arc::new(Field::new(&name, info.normalized_type, nullable)); + (name, field) + }) + .collect(); + + // Build the union schema in storage column order (sort cols first, + // then body cols lexicographic). let union_schema = build_husky_ordered_schema(&field_map, sort_fields_str)?; let union_schema_ref = Arc::new(union_schema); diff --git a/quickwit/quickwit-parquet-engine/src/merge/streaming.rs b/quickwit/quickwit-parquet-engine/src/merge/streaming.rs new file mode 100644 index 00000000000..19cf4c0405c --- /dev/null +++ b/quickwit/quickwit-parquet-engine/src/merge/streaming.rs @@ -0,0 +1,3800 @@ +// 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. + +//! Streaming column-major merge engine with page-bounded body cols. +//! +//! Architecture (multi-input → multi-output sorted merge with sort-cols-first +//! column ordering): +//! +//! 1. **Phase 0 (async): drain sort cols** from each input. With the storage convention that sort +//! cols + `sorted_series` precede all body cols within each row group, we can stop the decoder +//! after those are fully decoded. The remaining body col pages stay un-read in the input stream, +//! ready for phase 3. +//! 2. **Phase 1: compute merge order** via the existing k-way merge on `(sorted_series, +//! timestamp_secs)` from the per-input sort col [`RecordBatch`]es. Produces a run-length-encoded +//! merge plan over input row positions. +//! 3. **Phase 2: compute output boundaries** with the caller's `num_outputs`, splitting at +//! `sorted_series` transitions so each output file's key range is non-overlapping with adjacent +//! files. +//! 4. **Phase 3 (blocking + block_on bridges): streaming write**. All output writers are alive for +//! the duration. For each column in storage order, every output's col K is written in turn: +//! - Sort col / `sorted_series`: applied via `take` from the already-buffered phase 0 data. +//! - Body col: each output page is assembled via [`arrow::compute::interleave`] from input page +//! slices, with decoders advanced page-by-page via `Handle::block_on` from inside a sync +//! iterator. Pages flush to the writer's sink as [`SerializedColumnWriter`]'s page-size +//! threshold trips — memory stays bounded by the in-flight output page plus a small number of +//! in-flight input pages. +//! +//! After all M outputs' col K is done, every input decoder is at the +//! start of col K+1 in its single row group. Move to col K+1. +//! +//! ## Single-RG inputs assumption +//! +//! PR-6b.2 only handles **single-row-group inputs**. With multi-RG +//! inputs the body bytes interleave with successive RGs' sort cols +//! (`sort_cols_RG0`, `body_cols_RG0`, `sort_cols_RG1`, ...), so +//! draining sort cols from RG1 onwards requires either consuming + +//! discarding body cols of RG0 from the stream or buffering them. +//! Neither fits the page-bounded contract; multi-RG-input streaming +//! lands in a follow-up. Today's real inputs are: (a) post-PR-3 +//! single-RG ingest splits, or (b) PR-5's legacy adapter that +//! presents arbitrary multi-RG splits as one synthetic RG. Both +//! satisfy the assumption. +//! +//! [`SerializedColumnWriter`]: parquet::file::writer::SerializedColumnWriter + +#![allow(dead_code)] + +use std::collections::{HashMap, HashSet}; +use std::ops::Range; +use std::path::{Path, PathBuf}; +use std::sync::Arc; + +use anyhow::{Context, Result, anyhow, bail}; +use arrow::array::{Array, ArrayRef, RecordBatch, new_null_array}; +use arrow::compute::interleave; +use arrow::datatypes::{DataType, Field, Schema as ArrowSchema, SchemaRef}; +use parquet::file::metadata::ParquetMetaData; +use tokio::runtime::Handle; +use tracing::info; +use ulid::Ulid; + +use super::merge_order::{MergeRun, compute_merge_order, compute_output_boundaries}; +use super::schema::{align_inputs_to_union_schema, optimize_output_batch}; +use super::writer::{ + apply_merge_permutation, build_merge_kv_metadata, build_sorting_columns, + resolve_sort_field_names, verify_sort_order, +}; +use super::{InputMetadata, MergeConfig, MergeOutputFile}; +use crate::row_keys; +use crate::sort_fields::{ + equivalent_schemas_for_compaction, is_timestamp_column_name, parse_sort_fields, +}; +use crate::sorted_series::SORTED_SERIES_COLUMN; +use crate::split::TAG_SERVICE; +use crate::storage::page_decoder::{DecodedPage, StreamDecoder}; +use crate::storage::split_writer::{extract_metric_names, extract_time_range}; +use crate::storage::streaming_writer::StreamingParquetWriter; +use crate::storage::{ + ColumnPageStream, PARQUET_META_NUM_MERGE_OPS, PARQUET_META_RG_PARTITION_PREFIX_LEN, + PARQUET_META_SORT_FIELDS, PARQUET_META_WINDOW_DURATION, PARQUET_META_WINDOW_START, +}; +use crate::zonemap::{self, ZonemapOptions}; + +/// Output page size in rows for body-col assembly. Each call to the +/// sync iterator passed to [`write_next_column_arrays`] yields one +/// `ArrayRef` of up to this many rows; the parquet writer flushes +/// physical pages independently as encoded bytes cross +/// `data_page_size_limit`. 1024 keeps assembled arrays small enough +/// to bound per-output memory but large enough to amortise per-page +/// fixed costs. +/// +/// [`write_next_column_arrays`]: crate::storage::streaming_writer::RowGroupBuilder::write_next_column_arrays +const OUTPUT_PAGE_ROWS: usize = 1024; + +/// Test-only peak observed length of any input's `body_col_page_cache` +/// since the last reset. Used by the MS-7 page-bounded-memory test to +/// assert that the cache stays bounded by a small constant regardless +/// of input column size. Set unconditionally inside the merge so the +/// invariant is observable in any test build; reset on each merge entry. +#[cfg(test)] +pub(crate) static PEAK_BODY_COL_PAGE_CACHE_LEN: std::sync::atomic::AtomicUsize = + std::sync::atomic::AtomicUsize::new(0); + +#[cfg(test)] +fn record_body_col_page_cache_len(len: usize) { + use std::sync::atomic::Ordering; + let mut prev = PEAK_BODY_COL_PAGE_CACHE_LEN.load(Ordering::Relaxed); + while len > prev { + match PEAK_BODY_COL_PAGE_CACHE_LEN.compare_exchange_weak( + prev, + len, + Ordering::Relaxed, + Ordering::Relaxed, + ) { + Ok(_) => break, + Err(observed) => prev = observed, + } + } +} + +#[cfg(not(test))] +fn record_body_col_page_cache_len(_len: usize) {} + +/// Streaming N-input → M-output column-major merge. +/// +/// See module docs for the four phases. Returns one +/// [`MergeOutputFile`] per output file produced (zero-row outputs are +/// dropped). Caller's `config.num_outputs` is the upper bound on the +/// number of files; fewer are returned when there are not enough +/// `sorted_series` transitions to split at. +pub async fn streaming_merge_sorted_parquet_files( + inputs: Vec>, + output_dir: &Path, + config: &MergeConfig, +) -> Result> { + if inputs.is_empty() { + bail!("merge requires at least one input"); + } + if config.num_outputs == 0 { + bail!("num_outputs must be at least 1"); + } + + // Validate that all inputs are single-RG (or zero-RG, which means + // the file has no data). PR-6b.2 simplification — see module docs. + for (idx, stream) in inputs.iter().enumerate() { + let num_rgs = stream.metadata().num_row_groups(); + if num_rgs > 1 { + bail!( + "streaming merge requires single-row-group inputs in PR-6b.2 (input {idx} has \ + {num_rgs} row groups); multi-RG metric-aligned inputs land in a follow-up. \ + Legacy multi-RG (rg_partition_prefix_len=0) inputs must go through the PR-5 \ + adapter, which presents them as a single synthetic row group." + ); + } + } + + let input_meta = extract_and_validate_input_metadata(&inputs)?; + + info!( + num_inputs = inputs.len(), + num_outputs = config.num_outputs, + sort_fields = %input_meta.sort_fields, + "starting streaming sorted parquet merge" + ); + + let output_dir = output_dir.to_path_buf(); + let writer_config = config.writer_config.clone(); + let num_outputs = config.num_outputs; + + // Move everything onto a blocking task. Inside, the decoders need to + // make async I/O calls (page fetches over the network); we drive + // those via `handle.block_on(...)` from inside sync iterators that + // feed the parquet writer's column-write methods. The writer is + // sync; this single-task pattern avoids the lifetime complexity of + // moving borrowed `RowGroupBuilder`s across tokio tasks. + let result = tokio::task::spawn_blocking(move || -> Result> { + let handle = Handle::current(); + + let mut inputs = inputs; + let mut decoders_state = build_input_decoders_state(&mut inputs)?; + + // Cross-input precondition for the body-col memory bound. See + // the function doc and the `body_col_page_cache` field doc for + // the load-bearing argument. + assert_inputs_in_husky_body_col_order(&decoders_state, &input_meta.sort_fields)?; + + // Phase 0 + let sort_col_batches = + drain_sort_cols_all_inputs(&handle, &mut decoders_state, &input_meta.sort_fields)?; + + if sort_col_batches.iter().all(|b| b.num_rows() == 0) { + info!("all inputs empty, producing no output"); + return Ok(Vec::new()); + } + + // Phase 1: align inputs to a union sort-col schema so the merge-order + // comparator sees uniformly-typed `sorted_series` + `timestamp_secs`. + let (sort_union_schema, aligned_sort_batches) = + align_inputs_to_union_schema(&sort_col_batches, &input_meta.sort_fields)?; + let merge_order = compute_merge_order(&aligned_sort_batches, &input_meta.sort_fields)?; + + // Phase 2: split merge order into M outputs at sorted_series boundaries. + let boundaries = + compute_output_boundaries(&merge_order, &aligned_sort_batches, num_outputs)?; + + let total_rows: usize = aligned_sort_batches.iter().map(|b| b.num_rows()).sum(); + info!( + total_rows, + num_outputs = boundaries.len(), + "streaming merge order computed" + ); + + // Pre-compute per-input row → (output_idx, output_position) destination map. + // Used by every column write to slice take/interleave indices per page. + let destinations = + build_input_row_destinations(&aligned_sort_batches, &merge_order, &boundaries); + + // Phase 3 + let outputs = write_streaming_outputs( + &handle, + &mut decoders_state, + &aligned_sort_batches, + &sort_union_schema, + &merge_order, + &boundaries, + &destinations, + &input_meta, + &writer_config, + &output_dir, + )?; + + // MC-1: total row count preserved. + let output_total: usize = outputs.iter().map(|o| o.num_rows).sum(); + quickwit_dst::check_invariant!( + quickwit_dst::invariants::InvariantId::MC1, + output_total == total_rows, + ": streaming merge input rows={}, output rows={}", + total_rows, + output_total, + ); + + Ok(outputs) + }) + .await + .context("streaming merge blocking task panicked")??; + + Ok(result) +} + +/// Per-input state held across phase 0 → phase 3 inside the blocking +/// task. The decoder owns its stream so it persists across phases and +/// across all output writes for a given input — critical for body +/// columns whose pages may need to be visited multiple times (one page +/// can supply rows for more than one output, or one output may need +/// rows from more than one page). Reconstructing the decoder mid-pass +/// would reset the per-column `rows_decoded` counter (so `row_start` +/// becomes wrong) and discard cached dictionary / queued pages. +struct InputDecoderState { + decoder: StreamDecoder<'static>, + metadata: Arc, + /// Arrow schema of this input (from parquet → arrow conversion). + arrow_schema: SchemaRef, + /// Per-input page cache for the *currently active* body column. + /// Pages are appended as the decoder produces them and evicted from + /// the front once their last row is below `body_col_cursor`. The + /// cache must persist across all outputs that consume rows from + /// this column so that a page whose range straddles two outputs is + /// not re-decoded (the stream has already advanced past it). At + /// the start of each body column [`reset_body_col_state`] clears + /// this cache and zeroes the cursor. + /// + /// **Memory bound (horizontal, not vertical).** This cache is + /// per-input: total memory across inputs is + /// `N_inputs × per_input_peak`. The per-input peak is bounded by + /// `ceil(OUTPUT_PAGE_ROWS / input_page_rows) + small_slack` — + /// driven by [`fill_page_cache_to_row`], which fetches only enough + /// pages to cover the rows from *this* input that contribute to + /// the *current output page* (a 1024-row slice), and by the + /// eviction loop in [`assemble_one_output_page`], which drops any + /// page whose last row falls below the cursor after each output + /// page emits. The cache never holds an input column-chunk's worth + /// of pages — a regression that ever did would break the MS-7 + /// invariant asserted by + /// `test_ms7_body_col_page_cache_bounded_regardless_of_input_size`. + body_col_page_cache: Vec, + /// Next unconsumed input row for the active body column. Advances + /// monotonically across outputs because the merge plan assigns each + /// input's rows to outputs in input-row order. + body_col_cursor: usize, +} + +impl InputDecoderState { + /// Clear the per-input body-column cache. Called at the start of + /// each new body column so leftover pages from the previous column + /// (which have a different `col_idx`) don't poison the new column's + /// row-start arithmetic. The decoder itself is *not* reset — its + /// per-(rg, col) `rows_decoded` counters and queued pages must + /// survive so subsequent decode calls return correct row offsets. + fn reset_body_col_state(&mut self) { + self.body_col_page_cache.clear(); + self.body_col_cursor = 0; + } +} + +/// Build per-input state. The streams are moved in from the caller and +/// installed in long-lived [`StreamDecoder`]s so per-column state +/// survives every phase of the merge. +fn build_input_decoders_state( + inputs: &mut Vec>, +) -> Result> { + let mut states = Vec::with_capacity(inputs.len()); + for stream in inputs.drain(..) { + let metadata = Arc::clone(stream.metadata()); + let parquet_schema = metadata.file_metadata().schema_descr(); + let arrow_schema = parquet::arrow::parquet_to_arrow_schema(parquet_schema, None) + .context("converting parquet schema → arrow")?; + let decoder = StreamDecoder::from_owned(stream); + states.push(InputDecoderState { + decoder, + metadata, + arrow_schema: Arc::new(arrow_schema), + body_col_page_cache: Vec::new(), + body_col_cursor: 0, + }); + } + Ok(states) +} + +/// Extract sort schema, window, and merge-ops metadata from each +/// input stream and validate consistency across inputs. Reads +/// `qh.*` KVs from [`ColumnPageStream::metadata`]. +fn extract_and_validate_input_metadata( + inputs: &[Box], +) -> Result { + let mut consensus_sort_fields: Option = None; + let mut consensus_window_start: Option> = None; + let mut consensus_window_duration: Option = None; + let mut consensus_prefix_len: Option = None; + let mut max_merge_ops: u32 = 0; + + for (idx, stream) in inputs.iter().enumerate() { + let metadata = stream.metadata(); + let kv_metadata = metadata.file_metadata().key_value_metadata(); + + let find_kv = |key: &str| -> Option { + kv_metadata.and_then(|kvs| { + kvs.iter() + .find(|kv| kv.key == key) + .and_then(|kv| kv.value.clone()) + }) + }; + + let file_sort_fields = match find_kv(PARQUET_META_SORT_FIELDS) { + Some(s) => s, + None => bail!( + "input {idx} is missing {} metadata", + PARQUET_META_SORT_FIELDS, + ), + }; + + match &consensus_sort_fields { + Some(expected) => { + let expected_schema = parse_sort_fields(expected)?; + let file_schema = parse_sort_fields(&file_sort_fields).with_context(|| { + format!("parsing sort schema from input {idx}: '{file_sort_fields}'") + })?; + if !equivalent_schemas_for_compaction(&expected_schema, &file_schema) { + bail!( + "sort schema mismatch in input {idx}: expected '{expected}', found \ + '{file_sort_fields}'", + ); + } + } + None => { + parse_sort_fields(&file_sort_fields).with_context(|| { + format!("parsing sort schema from input {idx}: '{file_sort_fields}'") + })?; + consensus_sort_fields = Some(file_sort_fields.clone()); + } + } + + let file_window_start = find_kv(PARQUET_META_WINDOW_START) + .map(|s| s.parse::()) + .transpose() + .with_context(|| format!("parsing window_start from input {idx}"))?; + match &consensus_window_start { + Some(expected) if file_window_start != *expected => { + bail!( + "window_start mismatch in input {idx}: expected {:?}, found {:?}", + expected, + file_window_start, + ); + } + Some(_) => {} + None => consensus_window_start = Some(file_window_start), + } + + let file_window_duration = find_kv(PARQUET_META_WINDOW_DURATION) + .map(|s| s.parse::()) + .transpose() + .with_context(|| format!("parsing window_duration from input {idx}"))? + .unwrap_or(0); + match &consensus_window_duration { + Some(expected) if file_window_duration != *expected => { + bail!( + "window_duration_secs mismatch in input {idx}: expected {expected}, found \ + {file_window_duration}", + ); + } + Some(_) => {} + None => consensus_window_duration = Some(file_window_duration), + } + + let file_merge_ops = find_kv(PARQUET_META_NUM_MERGE_OPS) + .map(|s| s.parse::()) + .transpose() + .with_context(|| format!("parsing num_merge_ops from input {idx}"))? + .unwrap_or(0); + if file_merge_ops > max_merge_ops { + max_merge_ops = file_merge_ops; + } + + let file_prefix_len = find_kv(PARQUET_META_RG_PARTITION_PREFIX_LEN) + .map(|s| s.parse::()) + .transpose() + .with_context(|| format!("parsing rg_partition_prefix_len from input {idx}"))? + .unwrap_or(0); + match &consensus_prefix_len { + Some(expected) if file_prefix_len != *expected => { + bail!( + "rg_partition_prefix_len mismatch in input {idx}: expected {expected}, found \ + {file_prefix_len}", + ); + } + Some(_) => {} + None => consensus_prefix_len = Some(file_prefix_len), + } + } + + let sort_fields = match consensus_sort_fields { + Some(s) => s, + None => bail!("at least one input is required"), + }; + + // `rg_partition_prefix_len` is intentionally optional in the data + // model. Splits written before the prefix-aligned-RG era (and any + // split not written by the streaming engine) simply lack the KV. + // Inputs that *do* declare a value all had to agree above (else we + // already bailed), so falling through to 0 here means "none of the + // inputs claimed a prefix" rather than "we lost the value." The + // legacy-promotion path in PR-6423 handles mixing 0 with non-zero + // prefixes via the `mixed_prefix_ok` escape hatch. + Ok(InputMetadata { + sort_fields, + window_start_secs: consensus_window_start.unwrap_or(None), + window_duration_secs: consensus_window_duration.unwrap_or(0), + num_merge_ops: max_merge_ops + 1, + rg_partition_prefix_len: consensus_prefix_len.unwrap_or(0), + }) +} + +// ============================================================================ +// Phase 0: drain sort cols from each input +// ============================================================================ + +/// Drive each input's decoder via `block_on` until its sort cols + +/// `sorted_series` are fully decoded for the (single) row group. +/// Returns one [`RecordBatch`] per input with just those columns; the +/// rest of each input's body bytes stay un-read in the stream, ready +/// for phase 3 to consume page-by-page. +fn drain_sort_cols_all_inputs( + handle: &Handle, + decoders_state: &mut [InputDecoderState], + sort_fields_str: &str, +) -> Result> { + let mut batches = Vec::with_capacity(decoders_state.len()); + for (idx, state) in decoders_state.iter_mut().enumerate() { + let batch = drain_sort_cols_one_input(handle, state, sort_fields_str, idx)?; + if batch.num_columns() > 0 && batch.schema().index_of(SORTED_SERIES_COLUMN).is_err() { + bail!( + "input {idx} is missing the '{}' column required for merge", + SORTED_SERIES_COLUMN, + ); + } + batches.push(batch); + } + Ok(batches) +} + +fn drain_sort_cols_one_input( + handle: &Handle, + state: &mut InputDecoderState, + sort_fields_str: &str, + input_idx: usize, +) -> Result { + if state.metadata.num_row_groups() == 0 { + // Empty input — no rows to drain. Return a zero-row batch with the + // sort cols' fields preserved so downstream merge order code sees a + // uniform schema across inputs. + return empty_sort_col_record_batch(state, sort_fields_str); + } + let sort_field_schema = parse_sort_fields(sort_fields_str)?; + + // The set of column names we treat as "sort columns" for drain + // purposes: every sort-schema column name that is present in this + // input's arrow schema, plus `sorted_series` (always required). + let sort_col_names: HashSet = + sort_col_names_for_input(&sort_field_schema, state.arrow_schema.as_ref()); + + // Map each sort col name → its parquet leaf column index. The + // page decoder reports pages by parquet column index (matches arrow + // top-level field index when there are no nested types). + let parquet_schema = state.metadata.file_metadata().schema_descr(); + let mut sort_col_parquet_indices: HashMap = HashMap::new(); + for (col_idx, col) in parquet_schema.columns().iter().enumerate() { + // For flat schemas (one leaf per top-level field), the parquet + // column index equals the arrow top-level field index. We + // match by name: parquet `column_path` root → arrow field name. + let name = col.path().parts()[0].to_string(); + if sort_col_names.contains(&name) { + sort_col_parquet_indices.insert(col_idx, name); + } + } + + if sort_col_parquet_indices.is_empty() { + // No sort cols present in this input — return an empty batch + // with the input's arrow schema. Downstream merge order check + // will catch the missing `sorted_series`. + return Ok(RecordBatch::new_empty(Arc::clone(&state.arrow_schema))); + } + + // Target row count per sort col (from row group's column chunk metadata). + let rg_meta = state.metadata.row_group(0); + let mut target_rows_per_col: HashMap = HashMap::new(); + for &col_idx in sort_col_parquet_indices.keys() { + target_rows_per_col.insert(col_idx, rg_meta.column(col_idx).num_values() as usize); + } + + // Drain pages into per-col buffers until all sort cols are fully + // decoded. The streaming engine relies on a hard storage-ordering + // contract: within a row group, parquet emits column chunks in + // schema order (sort cols are declared first in our schema, body + // cols after), so all sort col pages appear before any body col + // page. Cross-file we don't require identical body-col ordering — + // the body-col loop drives from the union schema and looks each + // column up by name. The contract we DO require cross-file is + // "sort cols come first." A page from a body col arriving here + // means a producer violated that contract; bail rather than guess. + let mut per_col_pages: HashMap> = HashMap::new(); + let mut rows_done_per_col: HashMap = + sort_col_parquet_indices.keys().map(|&i| (i, 0)).collect(); + let mut sort_cols_finished = 0usize; + let sort_col_target = sort_col_parquet_indices.len(); + + while sort_cols_finished < sort_col_target { + let decoded = handle + .block_on(state.decoder.decode_next_page()) + .with_context(|| format!("decoding sort col page (input {input_idx})"))?; + let page = match decoded { + Some(p) => p, + None => bail!( + "stream ended before sort cols fully drained for input {input_idx}: \ + {sort_cols_finished}/{sort_col_target} cols complete", + ), + }; + + if !sort_col_parquet_indices.contains_key(&page.col_idx) { + bail!( + "input {input_idx} returned a non-sort page (col {}) before all sort cols were \ + drained — sort-cols-first storage ordering violated", + page.col_idx, + ); + } + if page.rg_idx != 0 { + // PR-6b.2 (this PR) only supports single-RG inputs. The + // multi-RG path with prefix-aligned row groups is added + // in PR-6c.2 (#6424) along with `process_region` / + // composite-key encoding. + bail!( + "input {input_idx} returned a page from rg {} during sort col drain — only \ + single-RG inputs are supported in PR-6b.2", + page.rg_idx, + ); + } + + let array_len = page.array.len(); + let rows_done = rows_done_per_col + .get_mut(&page.col_idx) + .expect("sort_col_parquet_indices.contains_key check above guarantees presence"); + *rows_done += array_len; + per_col_pages + .entry(page.col_idx) + .or_default() + .push(page.array); + + if *rows_done == target_rows_per_col[&page.col_idx] { + sort_cols_finished += 1; + } else if *rows_done > target_rows_per_col[&page.col_idx] { + bail!( + "input {input_idx} col {} decoded more rows ({}) than expected ({})", + page.col_idx, + rows_done, + target_rows_per_col[&page.col_idx], + ); + } + } + + // Build a RecordBatch holding just the sort cols. Field order + // matches the arrow schema's order (so downstream consumers see + // the same field order whether or not body cols are present). + let mut fields: Vec> = Vec::new(); + let mut columns: Vec = Vec::new(); + for (field_idx, field) in state.arrow_schema.fields().iter().enumerate() { + let Some(_name) = sort_col_parquet_indices.get(&field_idx) else { + continue; + }; + let pages = per_col_pages.remove(&field_idx).expect("col drained"); + let concatenated = concat_arrays(&pages).with_context(|| { + format!( + "concatenating sort col '{}' pages for input {input_idx}", + field.name(), + ) + })?; + fields.push(Arc::clone(field)); + columns.push(concatenated); + } + + let schema = Arc::new(ArrowSchema::new(fields)); + RecordBatch::try_new(schema, columns) + .with_context(|| format!("building sort col record batch for input {input_idx}")) +} + +/// Set of column names treated as "sort cols" for phase 0 drain. +/// Verify that every input's body cols (everything after the sort +/// cols and `sorted_series`) are in lexicographic order by name — +/// the same "Husky order" the union schema uses. +/// +/// **Why this is a hard contract, not a nicety.** Phase 3 iterates +/// the union schema's body cols alphabetically and asks each input's +/// decoder to advance to that col via [`fill_page_cache_to_row`]. +/// Parquet emits column chunks in declared schema order, so the +/// decoder reads pages in *the input's* storage order. If an input's +/// body cols aren't in the same order as the union iteration, the +/// decoder has to drain whatever cols precede the requested one on +/// the wire — those pages get cached under their own `col_idx`, +/// growing the cache to a full column-chunk's worth for the +/// misaligned input. Streaming becomes vertical-per-column instead +/// of horizontal-per-input-page, defeating the O(N × constant) +/// memory bound documented on +/// [`InputDecoderState::body_col_page_cache`]. +/// +/// Bails on the first violation with the offending input index and +/// the offending pair of column names. Returning `Ok(())` here is +/// the precondition that lets the body-col memory-bound argument +/// hold across the merge. +fn assert_inputs_in_husky_body_col_order( + decoders_state: &[InputDecoderState], + sort_fields_str: &str, +) -> Result<()> { + let sort_field_schema = parse_sort_fields(sort_fields_str)?; + for (input_idx, state) in decoders_state.iter().enumerate() { + let sort_or_structural = + sort_col_names_for_input(&sort_field_schema, state.arrow_schema.as_ref()); + let body_cols: Vec<&str> = state + .arrow_schema + .fields() + .iter() + .map(|f| f.name().as_str()) + .filter(|n| !sort_or_structural.contains(*n)) + .collect(); + for window in body_cols.windows(2) { + if window[0] >= window[1] { + bail!( + "input {input_idx} body cols not in Husky (alphabetical) order: '{}' precedes \ + '{}' in storage order. Streaming-merge memory bound requires body cols to be \ + emitted lexicographically by name within each input so the union schema's \ + iteration order matches storage order on the wire. Full body-col storage \ + order observed: {body_cols:?}", + window[0], + window[1], + ); + } + } + } + Ok(()) +} + +fn sort_col_names_for_input( + sort_field_schema: &quickwit_proto::sortschema::SortSchema, + arrow_schema: &ArrowSchema, +) -> HashSet { + let mut names: HashSet = HashSet::new(); + for sf in &sort_field_schema.column { + if arrow_schema.field_with_name(&sf.name).is_ok() { + names.insert(sf.name.clone()); + } + // Legacy schemas may declare `timestamp` but the column is named + // `timestamp_secs`. The merge order code already handles this + // alias; we want both candidates drained whichever matches. + if is_timestamp_column_name(&sf.name) + && arrow_schema.field_with_name("timestamp_secs").is_ok() + { + names.insert("timestamp_secs".to_string()); + } + } + if arrow_schema.field_with_name(SORTED_SERIES_COLUMN).is_ok() { + names.insert(SORTED_SERIES_COLUMN.to_string()); + } + names +} + +/// Build a zero-row RecordBatch with the input's sort cols + `sorted_series`. +/// Used when an input file has zero rows (no row groups) so that downstream +/// k-way merge sees a consistent schema shape across inputs. +fn empty_sort_col_record_batch( + state: &InputDecoderState, + sort_fields_str: &str, +) -> Result { + let sort_field_schema = parse_sort_fields(sort_fields_str)?; + let sort_col_names = sort_col_names_for_input(&sort_field_schema, state.arrow_schema.as_ref()); + let mut fields: Vec> = Vec::new(); + let mut columns: Vec = Vec::new(); + for field in state.arrow_schema.fields() { + if !sort_col_names.contains(field.name()) { + continue; + } + fields.push(Arc::clone(field)); + columns.push(new_null_array(field.data_type(), 0)); + } + let schema = Arc::new(ArrowSchema::new(fields)); + RecordBatch::try_new(schema, columns).context("building empty sort col record batch") +} + +fn concat_arrays(arrays: &[ArrayRef]) -> Result { + if arrays.len() == 1 { + return Ok(Arc::clone(&arrays[0])); + } + let refs: Vec<&dyn Array> = arrays.iter().map(|a| a.as_ref()).collect(); + Ok(arrow::compute::concat(&refs)?) +} + +// ============================================================================ +// Pre-compute input row → output destination map +// ============================================================================ + +/// `destinations[input_idx][input_row] = Some((output_idx, output_pos))` +/// if that input row contributes to output `output_idx` at position +/// `output_pos` within that output's row range. `None` means the row +/// is not in any output (only possible for rows beyond the merge +/// plan's coverage; shouldn't happen with our merge order). +#[derive(Debug)] +struct InputRowDestinations { + /// One Vec per input. Length = input's sort-col row count. + per_input: Vec>>, + /// Total rows per output index (cumulative writer "expected" rows). + rows_per_output: Vec, +} + +fn build_input_row_destinations( + aligned_sort_batches: &[RecordBatch], + merge_order: &[MergeRun], + boundaries: &[Range], +) -> InputRowDestinations { + let mut per_input: Vec>> = aligned_sort_batches + .iter() + .map(|b| vec![None; b.num_rows()]) + .collect(); + let mut rows_per_output: Vec = vec![0; boundaries.len()]; + + for (out_idx, boundary) in boundaries.iter().enumerate() { + let runs = &merge_order[boundary.clone()]; + let mut rows_for_current_output = 0usize; + for run in runs { + for r in 0..run.row_count { + let input_row = run.start_row + r; + per_input[run.input_index][input_row] = Some((out_idx, rows_for_current_output)); + rows_for_current_output += 1; + } + } + rows_per_output[out_idx] = rows_for_current_output; + } + + InputRowDestinations { + per_input, + rows_per_output, + } +} + +// ============================================================================ +// Phase 3: streaming write with one writer per output +// ============================================================================ + +/// Per-output state owned across phase 3 (writer + bookkeeping). +/// The row group lives in a parallel Vec so its borrow into `writer` +/// is tracked by the compiler instead of through a `'static` +/// transmute. +struct OutputWriterStorage { + output_idx: usize, + output_path: PathBuf, + writer: StreamingParquetWriter, + /// Service-name set built during the body col write of "service" + /// (or empty if no service col). + service_names: HashSet, + /// Per-output total row count = sum of merge runs in this output's boundary. + num_rows: usize, +} + +#[allow(clippy::too_many_arguments)] +fn write_streaming_outputs( + handle: &Handle, + decoders_state: &mut [InputDecoderState], + aligned_sort_batches: &[RecordBatch], + sort_union_schema: &SchemaRef, + merge_order: &[MergeRun], + boundaries: &[Range], + destinations: &InputRowDestinations, + input_meta: &InputMetadata, + writer_config: &crate::storage::ParquetWriterConfig, + output_dir: &Path, +) -> Result> { + // 1. Build the union schema across full input arrow schemas (so the output covers every column + // that appears in any input). The sort union schema covers only sort cols. + let input_arrow_schemas: Vec = decoders_state + .iter() + .map(|s| Arc::clone(&s.arrow_schema)) + .collect(); + let (full_union_schema, _aligned_full_placeholder) = + build_full_union_schema_from_arrow_schemas(&input_arrow_schemas, &input_meta.sort_fields)?; + + // 2. Build per-output metadata (KV entries, row keys, zonemaps) up front from sort col data — + // these are what the schema + writer props depend on. + let per_output_static = boundaries + .iter() + .enumerate() + .map(|(out_idx, boundary)| { + build_per_output_static( + out_idx, + boundary, + aligned_sort_batches, + sort_union_schema, + merge_order, + input_meta, + ) + }) + .collect::>>()?; + + // 3. Decide per-output schema: optimise based on each output's sort col data (which determines + // metric_name cardinality, etc.). Body cols stay as declared by the union schema; we don't + // probe their cardinality here since we haven't read them yet. This is a slight regression + // vs. the non-streaming engine — it would dict-encode low-cardinality string body cols too. + // PR-6c.2 or later can revisit by gathering body-col cardinality during the streaming pass. + let per_output_schemas: Vec = per_output_static + .iter() + .map(|s| derive_output_schema(&full_union_schema, sort_union_schema, &s.sort_optimised)) + .collect::>>()?; + + // 4. Open M writers, one per output. Writers + bookkeeping live in `writer_states`; the row + // group borrows mutably from each writer and is held in a parallel `row_groups` Vec for the + // col loop. + let mut writer_states: Vec = Vec::with_capacity(boundaries.len()); + for (out_idx, (schema, static_meta)) in per_output_schemas + .iter() + .zip(per_output_static.iter()) + .enumerate() + { + if destinations.rows_per_output[out_idx] == 0 { + continue; + } + writer_states.push(open_output_writer( + out_idx, + output_dir, + Arc::clone(schema), + static_meta, + input_meta, + writer_config, + )?); + } + + // Snapshot the (output_idx, num_rows) for each storage entry BEFORE + // calling `start_row_group`, which borrows `writer_states` mutably + // for the rest of phase 3's col loop. + let writer_index_view = writer_states_index_view(&writer_states); + let num_storages = writer_states.len(); + + let mut row_groups: Vec> = + writer_states + .iter_mut() + .map(|s| { + s.writer + .start_row_group() + .with_context(|| format!("opening row group for output {}", s.output_idx)) + }) + .collect::>>()?; + + // Service names are collected into a separate Vec> + // parallel to `row_groups`; we can't write into `writer_states` here + // because it is already borrowed mutably by `row_groups`. We merge + // these back into `writer_states` after dropping the row groups. + let mut service_names_per_output: Vec> = + (0..num_storages).map(|_| HashSet::new()).collect(); + write_all_columns( + handle, + &mut row_groups, + &mut service_names_per_output, + &writer_index_view, + decoders_state, + aligned_sort_batches, + sort_union_schema, + &full_union_schema, + merge_order, + boundaries, + destinations, + &per_output_schemas, + )?; + + // 6. Finish all row groups (drops the borrows on writers). + for rg in row_groups { + rg.finish().context("finishing row group")?; + } + + // 7. Merge collected service names + close writers + build MergeOutputFiles. + let mut outputs = Vec::with_capacity(writer_states.len()); + for (mut state, services) in writer_states + .into_iter() + .zip(service_names_per_output.into_iter()) + { + state.service_names.extend(services); + outputs.push(finalize_output_writer(state, &per_output_static)?); + } + Ok(outputs) +} + +/// Static per-output state computed once from sort col data. Holds +/// the per-output sort-col-only batch (used for metadata extraction) +/// and the per-output schema-optimisation hints. +struct PerOutputStatic { + /// Sort-cols-only batch in output sort order — used by row_keys / + /// zonemap / metric_names / time_range extractors. + sort_optimised: RecordBatch, + row_keys_proto: Option>, + zonemap_regexes: HashMap, + metric_names: HashSet, + time_range: crate::split::TimeRange, + /// Number of rows that go into this output. + num_rows: usize, +} + +fn build_per_output_static( + out_idx: usize, + boundary: &Range, + aligned_sort_batches: &[RecordBatch], + sort_union_schema: &SchemaRef, + merge_order: &[MergeRun], + input_meta: &InputMetadata, +) -> Result { + let runs = &merge_order[boundary.clone()]; + let sort_batch = apply_merge_permutation(aligned_sort_batches, sort_union_schema, runs) + .with_context(|| format!("applying merge permutation for output {out_idx} sort cols"))?; + let num_rows = sort_batch.num_rows(); + + // MC-3 sort order on the sort-col-only batch (same check the + // non-streaming engine does, just restricted to columns we have). + verify_sort_order(&sort_batch, &input_meta.sort_fields); + let sort_optimised = optimize_output_batch(&sort_batch); + + let row_keys_proto = row_keys::extract_row_keys(&input_meta.sort_fields, &sort_optimised) + .with_context(|| format!("extracting row keys for output {out_idx}"))? + .map(|rk| row_keys::encode_row_keys_proto(&rk)); + + let zonemap_opts = ZonemapOptions::default(); + let zonemap_regexes = + zonemap::extract_zonemap_regexes(&input_meta.sort_fields, &sort_optimised, &zonemap_opts) + .with_context(|| format!("extracting zonemap regexes for output {out_idx}"))?; + + let metric_names = extract_metric_names(&sort_optimised) + .with_context(|| format!("extracting metric names for output {out_idx}"))?; + let time_range = extract_time_range(&sort_optimised) + .with_context(|| format!("extracting time range for output {out_idx}"))?; + + Ok(PerOutputStatic { + sort_optimised, + row_keys_proto, + zonemap_regexes, + metric_names, + time_range, + num_rows, + }) +} + +/// Build the full union schema across all inputs' arrow schemas +/// (NOT just sort cols). Reuses the same algorithm as +/// [`align_inputs_to_union_schema`] but takes pre-extracted arrow +/// schemas — phase 3 doesn't have full input batches. +fn build_full_union_schema_from_arrow_schemas( + arrow_schemas: &[SchemaRef], + sort_fields_str: &str, +) -> Result<(SchemaRef, ())> { + // Build zero-row batches with the right schemas; that lets us + // reuse `align_inputs_to_union_schema`'s field-merge / storage- + // ordering logic unchanged. + let empty_batches: Vec = arrow_schemas + .iter() + .map(|s| RecordBatch::new_empty(Arc::clone(s))) + .collect(); + let (schema, _) = align_inputs_to_union_schema(&empty_batches, sort_fields_str)?; + Ok((schema, ())) +} + +/// Compute the per-output schema. For PR-6b.2 we use the +/// (string-normalised) union schema as the output schema directly — +/// fields stay Utf8/LargeUtf8 rather than being re-dict-encoded. +/// Reason: streaming-decoded input arrays come out of the page +/// decoder as plain `StringArray`/`BinaryArray` (not Dictionary), and +/// dict re-encoding per output page would add a per-page CPU cost we +/// don't want to take in the page-bounded path. Re-introducing +/// dict-encoded output strings can be done later by tracking +/// cardinality during the streaming pass — call site is here. +/// +/// We do still want to drop columns that are all-null *for this +/// output* (e.g., a column only present in inputs that don't +/// contribute any rows to this output's range). The `sort_optimised` +/// batch has already discarded all-null sort fields; we mirror that +/// decision when building the per-output schema. Body cols are kept +/// unconditionally — tracking per-output body-col presence would +/// require pre-reading every body column for every output, which is +/// exactly the column-chunk-bounded buffering the streaming path +/// exists to avoid. +/// +/// `sort_union_schema` is the union of every input's sort columns +/// (before per-output optimisation). It's the only way to tell +/// whether a given union-schema field is a sort field or a body +/// field — `sort_optimised.schema()` alone can't disambiguate because +/// it has dropped some sort fields by design. Without this +/// distinction the function falls into the trap of using +/// `full_union_schema.index_of(field.name())`, which is trivially +/// true for every iterated field, and the all-null drop never +/// happens. +fn derive_output_schema( + full_union_schema: &SchemaRef, + sort_union_schema: &SchemaRef, + sort_optimised: &RecordBatch, +) -> Result { + let sort_optimised_schema = sort_optimised.schema(); + let mut fields: Vec> = Vec::with_capacity(full_union_schema.fields().len()); + for field in full_union_schema.fields() { + let is_sort_field = sort_union_schema.index_of(field.name()).is_ok(); + if is_sort_field { + // Sort field: keep only if the per-output optimiser kept + // it (i.e., not all-null for this output's rows). + if sort_optimised_schema.index_of(field.name()).is_ok() { + fields.push(Arc::clone(field)); + } + } else { + // Body field: always kept. + fields.push(Arc::clone(field)); + } + } + Ok(Arc::new(ArrowSchema::new(fields))) +} + +fn open_output_writer( + out_idx: usize, + output_dir: &Path, + schema: SchemaRef, + static_meta: &PerOutputStatic, + input_meta: &InputMetadata, + writer_config: &crate::storage::ParquetWriterConfig, +) -> Result { + let output_prefix_len = input_meta.rg_partition_prefix_len; + let kv_entries = build_merge_kv_metadata( + input_meta, + &static_meta.row_keys_proto, + &static_meta.zonemap_regexes, + output_prefix_len, + ); + let sorting_cols = build_sorting_columns(&static_meta.sort_optimised, &input_meta.sort_fields)?; + let sort_field_names = resolve_sort_field_names(&input_meta.sort_fields)?; + + let props = writer_config.to_writer_properties_with_metadata( + &schema, + sorting_cols, + Some(kv_entries), + &sort_field_names, + ); + + let output_filename = format!("merge_output_{}.parquet", Ulid::new()); + let output_path = output_dir.join(&output_filename); + let file = std::fs::File::create(&output_path) + .with_context(|| format!("creating output file: {}", output_path.display()))?; + let writer = StreamingParquetWriter::try_new(file, Arc::clone(&schema), props) + .with_context(|| format!("opening streaming writer for output {out_idx}"))?; + + Ok(OutputWriterStorage { + output_idx: out_idx, + output_path, + writer, + service_names: HashSet::new(), + num_rows: static_meta.num_rows, + }) +} + +/// Index view used inside the col loop to find the writer's +/// `output_idx` and `num_rows` without needing a mutable borrow on +/// `writer_states` (which is already mutably borrowed by `row_groups`). +fn writer_states_index_view(writer_states: &[OutputWriterStorage]) -> Vec<(usize, usize)> { + writer_states + .iter() + .map(|s| (s.output_idx, s.num_rows)) + .collect() +} + +#[allow(clippy::too_many_arguments)] +fn write_all_columns( + handle: &Handle, + row_groups: &mut [crate::storage::streaming_writer::RowGroupBuilder<'_, std::fs::File>], + service_names_per_output: &mut [HashSet], + writer_index_view: &[(usize, usize)], + decoders_state: &mut [InputDecoderState], + aligned_sort_batches: &[RecordBatch], + sort_union_schema: &SchemaRef, + full_union_schema: &SchemaRef, + merge_order: &[MergeRun], + boundaries: &[Range], + destinations: &InputRowDestinations, + per_output_schemas: &[SchemaRef], +) -> Result<()> { + // Iterate cols in the **full union schema** order. The union + // covers every column that appears in ANY output. For each col K + // and each output: + // - If output's schema includes col K: write col K's data (sort col → from buffer, body col → + // from decoder). + // - Else: skip — that output dropped col K as all-null for its row range; the next output's + // col K still gets written. + // + // It's tempting to drive from one per-output schema, since all + // per-output schemas share the same column ordering as a + // subsequence. But two outputs may drop *different* all-null + // fields and end up with the same field count — then picking + // either misses a field the other output still needs, and the + // writer for the latter output writes subsequent columns into + // the wrong slot. The full union schema is the only choice that + // covers every column every output may need, in the canonical + // storage order. + + // For each full-union-schema col K: + for parent_col_idx in 0..full_union_schema.fields().len() { + let parent_field = full_union_schema.field(parent_col_idx); + let parent_name = parent_field.name(); + + // Is this a sort col (in memory) or a body col (streamed)? + let is_sort_col = sort_union_schema.index_of(parent_name).is_ok(); + + if is_sort_col { + write_sort_col_for_all_outputs( + row_groups, + writer_index_view, + parent_name, + aligned_sort_batches, + sort_union_schema, + merge_order, + boundaries, + destinations, + per_output_schemas, + )?; + } else { + write_body_col_for_all_outputs( + handle, + row_groups, + service_names_per_output, + writer_index_view, + decoders_state, + parent_name, + destinations, + per_output_schemas, + )?; + } + } + + Ok(()) +} + +#[allow(clippy::too_many_arguments)] +fn write_sort_col_for_all_outputs( + row_groups: &mut [crate::storage::streaming_writer::RowGroupBuilder<'_, std::fs::File>], + writer_index_view: &[(usize, usize)], + col_name: &str, + aligned_sort_batches: &[RecordBatch], + sort_union_schema: &SchemaRef, + merge_order: &[MergeRun], + boundaries: &[Range], + destinations: &InputRowDestinations, + per_output_schemas: &[SchemaRef], +) -> Result<()> { + let _ = sort_union_schema; + + let mut storage_idx = 0; + for (out_idx, boundary) in boundaries.iter().enumerate() { + if destinations.rows_per_output[out_idx] == 0 { + continue; + } + debug_assert_eq!(writer_index_view[storage_idx].0, out_idx); + + // Drop this col if the output's schema doesn't include it. + let out_schema = &per_output_schemas[out_idx]; + if out_schema.index_of(col_name).is_err() { + storage_idx += 1; + continue; + } + + let runs = &merge_order[boundary.clone()]; + let arrays = build_sort_col_pages_for_output(col_name, aligned_sort_batches, runs)?; + row_groups[storage_idx] + .write_next_column_arrays(arrays.into_iter()) + .with_context(|| format!("writing sort col '{col_name}' to output {out_idx}"))?; + storage_idx += 1; + } + Ok(()) +} + +/// Build per-output-page arrays for one sort col. The col is already +/// in memory across all inputs (`aligned_sort_batches`); for this +/// output we walk its merge runs and split the take result into +/// `OUTPUT_PAGE_ROWS`-sized chunks. +fn build_sort_col_pages_for_output( + col_name: &str, + aligned_sort_batches: &[RecordBatch], + runs: &[MergeRun], +) -> Result> { + // Collect references to each input's column array. + let mut input_arrays: Vec<&dyn Array> = Vec::with_capacity(aligned_sort_batches.len()); + for batch in aligned_sort_batches { + let idx = batch.schema().index_of(col_name).map_err(|_| { + anyhow!("input is missing sort col '{col_name}' that the union schema expected",) + })?; + input_arrays.push(batch.column(idx).as_ref()); + } + + let mut indices: Vec<(usize, usize)> = + Vec::with_capacity(runs.iter().map(|r| r.row_count).sum()); + for run in runs { + for r in 0..run.row_count { + indices.push((run.input_index, run.start_row + r)); + } + } + + // Split into OUTPUT_PAGE_ROWS-sized chunks; each chunk → one + // arrow::interleave call → one ArrayRef. + let mut pages = Vec::with_capacity(indices.len().div_ceil(OUTPUT_PAGE_ROWS)); + for chunk in indices.chunks(OUTPUT_PAGE_ROWS) { + let arr = interleave(&input_arrays, chunk) + .with_context(|| format!("interleaving sort col '{col_name}' pages"))?; + pages.push(arr); + } + Ok(pages) +} + +#[allow(clippy::too_many_arguments)] +fn write_body_col_for_all_outputs( + handle: &Handle, + row_groups: &mut [crate::storage::streaming_writer::RowGroupBuilder<'_, std::fs::File>], + service_names_per_output: &mut [HashSet], + writer_index_view: &[(usize, usize)], + decoders_state: &mut [InputDecoderState], + col_name: &str, + destinations: &InputRowDestinations, + per_output_schemas: &[SchemaRef], +) -> Result<()> { + // Find this col's per-input parquet leaf index (one per input). + // Inputs whose schema doesn't have this col OR which have zero + // row groups (legal — phase 0 explicitly accepts empty inputs and + // returns an empty sort batch for them) contribute null rows and + // don't advance their decoder for this col. Looking up + // `row_group(0)` on a zero-RG input would panic, so guard up + // front. + let mut input_col_indices: Vec> = Vec::with_capacity(decoders_state.len()); + for state in decoders_state.iter() { + if state.metadata.num_row_groups() == 0 { + input_col_indices.push(None); + continue; + } + match state.arrow_schema.index_of(col_name) { + Ok(idx) => input_col_indices.push(Some(idx)), + Err(_) => input_col_indices.push(None), + } + } + + // Reset each input's body-col cache + cursor at the start of this + // column. The persistent `StreamDecoder` retains its per-(rg, col) + // state for *every* column it has touched so the next page from + // this column has the correct `row_start`; only the cached pages + // (which belong to the previous column) need to be discarded. + for state in decoders_state.iter_mut() { + state.reset_body_col_state(); + } + + // Track service names while streaming the service col. + let track_service = col_name == "service"; + + // For each output sequentially: build output pages, feed to writer + // one page at a time. We must NOT collect the whole column into a + // Vec — that would defeat the page-bounded merge path and scale + // memory with column-chunk size on production splits. Instead we + // hand `write_next_column_arrays` a streaming iterator that + // captures the first error in a side cell so the writer stops as + // soon as assembly fails. + let mut storage_idx = 0; + for (out_idx, &row_count) in destinations.rows_per_output.iter().enumerate() { + if row_count == 0 { + continue; + } + debug_assert_eq!(writer_index_view[storage_idx].0, out_idx); + + let out_schema = &per_output_schemas[out_idx]; + if out_schema.index_of(col_name).is_err() { + storage_idx += 1; + continue; + } + let out_field_idx = out_schema.index_of(col_name)?; + let out_field = out_schema.field(out_field_idx); + + let assembler = BodyColOutputPageAssembler::new( + handle, + decoders_state, + &input_col_indices, + destinations, + out_idx, + col_name, + out_field, + ); + + let mut error_slot: Option = None; + let service_collector: Option<&mut HashSet> = if track_service { + Some(&mut service_names_per_output[storage_idx]) + } else { + None + }; + + let stream_iter = StreamingBodyColIter { + inner: assembler.into_iter(), + error_slot: &mut error_slot, + service_collector, + }; + + let write_result = row_groups[storage_idx].write_next_column_arrays(stream_iter); + + // Assembly errors are reported via `error_slot`; surface them + // first because a downstream write error is usually a + // consequence (the writer stops on `None` and reports a + // row-count mismatch otherwise). + if let Some(err) = error_slot { + return Err(err) + .with_context(|| format!("assembling body col '{col_name}' for output {out_idx}")); + } + write_result + .with_context(|| format!("writing body col '{col_name}' to output {out_idx}"))?; + storage_idx += 1; + } + + Ok(()) +} + +/// Adapts a `Result` page assembler into the +/// `Iterator` shape `write_next_column_arrays` expects. +/// The first assembly error is captured in `error_slot` and iteration +/// ends; the caller MUST check the slot after the writer returns. If +/// `service_collector` is `Some`, every yielded page is scanned for +/// service names and added to the set; collection failures also stop +/// the iterator and populate `error_slot`. +struct StreamingBodyColIter<'a, I> { + inner: I, + error_slot: &'a mut Option, + service_collector: Option<&'a mut HashSet>, +} + +impl Iterator for StreamingBodyColIter<'_, I> +where I: Iterator> +{ + type Item = ArrayRef; + + fn next(&mut self) -> Option { + if self.error_slot.is_some() { + return None; + } + match self.inner.next() { + Some(Ok(arr)) => { + if let Some(out) = self.service_collector.as_deref_mut() + && let Err(e) = collect_service_names_from_page(arr.as_ref(), out) + { + *self.error_slot = Some(e); + return None; + } + Some(arr) + } + Some(Err(e)) => { + *self.error_slot = Some(e); + None + } + None => None, + } + } +} + +/// Per-page service name collector. Used during the streaming write +/// of the "service" body col to populate per-output service_names. +fn collect_service_names_from_page(arr: &dyn Array, out: &mut HashSet) -> Result<()> { + use arrow::array::AsArray; + use arrow::datatypes::{Int8Type, Int16Type, Int32Type, Int64Type}; + + fn extend_from_strings(strings: &arrow::array::StringArray, out: &mut HashSet) { + for i in 0..strings.len() { + if strings.is_valid(i) { + out.insert(strings.value(i).to_string()); + } + } + } + + match arr.data_type() { + DataType::Utf8 => { + let strings = arr + .as_any() + .downcast_ref::() + .ok_or_else(|| anyhow!("expected StringArray for service col page"))?; + extend_from_strings(strings, out); + } + DataType::LargeUtf8 => { + let strings = arr + .as_any() + .downcast_ref::() + .ok_or_else(|| anyhow!("expected LargeStringArray for service col page"))?; + for i in 0..strings.len() { + if strings.is_valid(i) { + out.insert(strings.value(i).to_string()); + } + } + } + DataType::Dictionary(key_type, value_type) + if matches!(value_type.as_ref(), DataType::Utf8) => + { + // Extract the dictionary's values that are referenced by + // valid (non-null) keys. + match key_type.as_ref() { + DataType::Int8 => { + let dict = arr.as_dictionary::(); + if let Some(strings) = dict + .values() + .as_any() + .downcast_ref::() + { + for i in 0..dict.len() { + if dict.is_valid(i) { + let key = dict.keys().value(i) as usize; + if key < strings.len() && strings.is_valid(key) { + out.insert(strings.value(key).to_string()); + } + } + } + } + } + DataType::Int16 => { + let dict = arr.as_dictionary::(); + if let Some(strings) = dict + .values() + .as_any() + .downcast_ref::() + { + for i in 0..dict.len() { + if dict.is_valid(i) { + let key = dict.keys().value(i) as usize; + if key < strings.len() && strings.is_valid(key) { + out.insert(strings.value(key).to_string()); + } + } + } + } + } + DataType::Int32 => { + let dict = arr.as_dictionary::(); + if let Some(strings) = dict + .values() + .as_any() + .downcast_ref::() + { + for i in 0..dict.len() { + if dict.is_valid(i) { + let key = dict.keys().value(i) as usize; + if key < strings.len() && strings.is_valid(key) { + out.insert(strings.value(key).to_string()); + } + } + } + } + } + DataType::Int64 => { + let dict = arr.as_dictionary::(); + if let Some(strings) = dict + .values() + .as_any() + .downcast_ref::() + { + for i in 0..dict.len() { + if dict.is_valid(i) { + let key = dict.keys().value(i) as usize; + if key < strings.len() && strings.is_valid(key) { + out.insert(strings.value(key).to_string()); + } + } + } + } + } + _ => {} + } + } + _ => { + // Skip non-string types — service col is expected to be + // string-like; if it isn't, just don't collect names. + } + } + Ok(()) +} + +// ============================================================================ +// Body col output page assembler — the page-bounded streaming core +// ============================================================================ + +/// Assembles output pages for one (output_idx, body_col) by: +/// 1. Walking the destinations table forward through this output's row range, accumulating +/// `(input_idx, input_row)` index pairs. +/// 2. When the index buffer hits `OUTPUT_PAGE_ROWS`, advancing each contributing input's decoder +/// until its decoded pages cover the needed input rows, then calling +/// `arrow::compute::interleave`. +/// 3. Emitting one `ArrayRef` per iter step until the row range is exhausted; then `Ok(None)`. +/// +/// Memory per `next()` call: one in-progress output page (P rows) + +/// up to ~2 in-flight decoded pages per input (kept until all their +/// rows are consumed). Bounded by page sizes, not column-chunk sizes. +/// +/// **Cross-output state**: the per-input `body_col_page_cache` and +/// `body_col_cursor` live on [`InputDecoderState`], not the assembler. +/// They must persist across all outputs that consume rows from the +/// active body column — a page whose row range straddles two outputs +/// would otherwise be dropped when the first output's assembler ends, +/// even though the stream has already advanced past it and the next +/// output still needs rows from inside that page. +struct BodyColOutputPageAssembler<'a> { + handle: &'a Handle, + decoders_state: &'a mut [InputDecoderState], + input_col_indices: &'a [Option], + destinations: &'a InputRowDestinations, + out_idx: usize, + col_name: &'a str, + out_field: &'a Field, + /// Total rows written so far for this output's col. + rows_emitted: usize, + /// Total rows expected = destinations.rows_per_output[out_idx]. + expected_rows: usize, + /// EOF flag (returns None on subsequent calls once true). + done: bool, +} + +impl<'a> BodyColOutputPageAssembler<'a> { + #[allow(clippy::too_many_arguments)] + fn new( + handle: &'a Handle, + decoders_state: &'a mut [InputDecoderState], + input_col_indices: &'a [Option], + destinations: &'a InputRowDestinations, + out_idx: usize, + col_name: &'a str, + out_field: &'a Field, + ) -> Self { + Self { + handle, + decoders_state, + input_col_indices, + destinations, + out_idx, + col_name, + out_field, + rows_emitted: 0, + expected_rows: destinations.rows_per_output[out_idx], + done: false, + } + } + + fn into_iter(self) -> BodyColOutputPageIter<'a> { + BodyColOutputPageIter { inner: self } + } +} + +struct BodyColOutputPageIter<'a> { + inner: BodyColOutputPageAssembler<'a>, +} + +impl Iterator for BodyColOutputPageIter<'_> { + type Item = Result; + + fn next(&mut self) -> Option { + if self.inner.done || self.inner.rows_emitted >= self.inner.expected_rows { + self.inner.done = true; + return None; + } + match assemble_one_output_page(&mut self.inner) { + Ok(Some(arr)) => Some(Ok(arr)), + Ok(None) => { + self.inner.done = true; + None + } + Err(e) => { + self.inner.done = true; + Some(Err(e)) + } + } + } +} + +fn assemble_one_output_page(s: &mut BodyColOutputPageAssembler) -> Result> { + let remaining = s.expected_rows - s.rows_emitted; + if remaining == 0 { + return Ok(None); + } + let page_size = remaining.min(OUTPUT_PAGE_ROWS); + + // Walk this output's row positions and figure out which (input, input_row) + // contributes each one. We use the per-input destinations table: for + // input i, find the next input_row whose destination is (out_idx, *). + // Since `destinations.per_input[i]` is in input order and outputs are + // strictly increasing by sort key, the rows that go to this output are + // a contiguous slice in input i's row order. + // + // For each output position 0..page_size, we need (input_idx, input_row). + // Walk input cursors and pick the next row going to this output. + + // Collect (input_idx, input_row) indices for this output page. + let mut indices_per_input: Vec> = vec![Vec::new(); s.decoders_state.len()]; + let mut interleave_indices: Vec<(usize, usize)> = Vec::with_capacity(page_size); + let mut total_picked = 0usize; + + while total_picked < page_size { + // Look across all inputs for the next contribution to this output. + // Per the merge order, within each input the rows assigned to this + // output are a contiguous slice; once we've advanced cursor past + // them, no more rows from this input contribute. We collect ALL + // rows from one input up to a per-input limit determined by the + // merge order, but the simplest correct approach is to walk in + // merge-order globally. We don't have the merge order indexed by + // output here, so re-derive by scanning the destinations table. + // + // Better: pre-compute per-output, per-input row ranges. Each input + // contributes a contiguous half-open range `[lo_i..hi_i)` to this + // output (possibly empty). We could compute these ranges once and + // reuse. For now, lazy approach: scan forward from cursor on each + // input, picking the next row that maps to (out_idx, *). + // + // The ORDER in which we pick across inputs must match the merge + // plan's output position. We have output positions in destinations: + // `destinations.per_input[i][r] = Some((out_idx, pos))`. The merged + // output picks rows in order of increasing `pos`. + // + // For one output page, the positions we want are + // `s.rows_emitted..s.rows_emitted + page_size`. For each position + // p in that range, find (input_idx, input_row) such that + // destinations.per_input[input_idx][input_row] == Some((out_idx, p)). + let target_pos = s.rows_emitted + total_picked; + let mut found = false; + for (input_idx, dests) in s.destinations.per_input.iter().enumerate() { + let cursor = s.decoders_state[input_idx].body_col_cursor; + for (input_row, dest) in dests.iter().enumerate().skip(cursor) { + match dest { + Some((o, p)) if *o == s.out_idx => { + if *p == target_pos { + interleave_indices.push((input_idx, input_row)); + indices_per_input[input_idx].push(input_row); + // Don't advance the cursor past this row yet — + // we may need rows from input i in this page + // with positions ahead. We bump it after the + // whole page is collected. + found = true; + break; + } + } + _ => {} + } + if found { + break; + } + } + if found { + break; + } + } + if !found { + // Shouldn't happen — every output position should be reachable. + bail!( + "merge plan inconsistency: output {} position {target_pos} not found in any input", + s.out_idx, + ); + } + total_picked += 1; + } + + // Now ensure each input's decoder has decoded pages covering all + // `indices_per_input[i]` rows. Advance decoders as needed. + for (input_idx, input_rows) in indices_per_input.iter().enumerate() { + if input_rows.is_empty() { + continue; + } + let col_parquet_idx = match s.input_col_indices[input_idx] { + Some(c) => c, + None => { + // This input lacks this col entirely — null contributions. + // We'll handle null-filling in the interleave step below. + continue; + } + }; + let max_needed_row = *input_rows.iter().max().expect("non-empty"); + fill_page_cache_to_row( + s.handle, + &mut s.decoders_state[input_idx], + col_parquet_idx, + max_needed_row, + )?; + } + + // Build the per-(input, row) value array by: + // 1. Concatenating each input's cached pages into one ArrayRef (they cover a contiguous input + // row range from cache_start to cursor_max). + // 2. Computing local indices = input_row - cache_start. + // 3. Calling arrow::compute::interleave across N input arrays. + // + // For inputs without this col, we substitute a single null page of the + // out_field's type. + let mut input_array_refs: Vec = Vec::with_capacity(s.decoders_state.len()); + let mut input_cache_starts: Vec = Vec::with_capacity(s.decoders_state.len()); + + for input_idx in 0..s.decoders_state.len() { + match s.input_col_indices[input_idx] { + Some(_) => { + let pages = &s.decoders_state[input_idx].body_col_page_cache; + if pages.is_empty() { + // No pages decoded for this input (no rows from this input go to this output). + // Use a zero-row placeholder; we won't index into it. + input_array_refs.push(new_null_array(s.out_field.data_type(), 0)); + input_cache_starts.push(0); + } else { + let cache_start = pages[0].row_start; + let arrays: Vec<&dyn Array> = pages.iter().map(|p| p.array.as_ref()).collect(); + let concatenated = arrow::compute::concat(&arrays).with_context(|| { + format!( + "concatenating cached pages for input {input_idx} col '{}'", + s.col_name, + ) + })?; + input_array_refs.push(concatenated); + input_cache_starts.push(cache_start); + } + } + None => { + // Null-fill array of the right length. The max needed local + // index from this input is the largest index we'd reference; + // since we don't actually reference rows from this input (we'd + // need an alternate "null contribution" mechanism), we leave + // it as a 1-row null array and route indices to position 0. + let null_arr = new_null_array(s.out_field.data_type(), 1); + input_array_refs.push(null_arr); + input_cache_starts.push(0); + } + } + } + + let interleave_local: Vec<(usize, usize)> = interleave_indices + .iter() + .map(|&(i_idx, i_row)| match s.input_col_indices[i_idx] { + Some(_) => (i_idx, i_row - input_cache_starts[i_idx]), + None => (i_idx, 0), + }) + .collect(); + + let array_refs_ref: Vec<&dyn Array> = input_array_refs.iter().map(|a| a.as_ref()).collect(); + let assembled = interleave(&array_refs_ref, &interleave_local).with_context(|| { + format!( + "interleaving body col '{}' for output {}", + s.col_name, s.out_idx, + ) + })?; + + // Bump input cursors past rows we just consumed and drop pages + // whose rows are fully consumed. Both the cursor and the cache + // live on InputDecoderState so they persist across outputs that + // share this column. + for (input_idx, input_rows) in indices_per_input.iter().enumerate() { + if input_rows.is_empty() { + continue; + } + let max_row = *input_rows.iter().max().expect("non-empty"); + let state = &mut s.decoders_state[input_idx]; + state.body_col_cursor = max_row + 1; + + // Drop pages whose last row is < cursor. + if s.input_col_indices[input_idx].is_some() { + let pages = &mut state.body_col_page_cache; + while let Some(front) = pages.first() { + let front_end = front.row_start + front.array.len(); + if front_end <= state.body_col_cursor { + pages.remove(0); + } else { + break; + } + } + } + } + + s.rows_emitted += page_size; + Ok(Some(assembled)) +} + +/// Pull pages from the input's persistent decoder via `block_on` until +/// the cached pages for `col_parquet_idx` cover up through `target_row` +/// (inclusive). Stops as soon as the latest cached page ends past +/// `target_row`. The function's effect on the world is *adding pages +/// to the cache* — it does not skip data and does not consume any +/// rows on its own. +/// +/// The decoder MUST be the long-lived [`InputDecoderState::decoder`]: +/// it preserves the per-(rg, col) `rows_decoded` counter so successive +/// `DecodedPage::row_start` values are absolute input row indices, +/// not page-local zeros. Likewise, the cache lives on the state so +/// pages whose row range spans an output boundary survive into the +/// next output's assembler. +fn fill_page_cache_to_row( + handle: &Handle, + state: &mut InputDecoderState, + col_parquet_idx: usize, + target_row: usize, +) -> Result<()> { + // If cache already covers target_row, nothing to do. + if let Some(last) = state.body_col_page_cache.last() { + let last_end = last.row_start + last.array.len(); + if target_row < last_end { + return Ok(()); + } + } + + loop { + let decoded = handle + .block_on(state.decoder.decode_next_page()) + .context("decoding body col page")?; + let page = match decoded { + Some(p) => p, + None => bail!( + "stream EOF while advancing to row {target_row} for parquet col {col_parquet_idx}", + ), + }; + if page.col_idx != col_parquet_idx { + bail!( + "expected col {col_parquet_idx} page, got col {} — column ordering violated", + page.col_idx, + ); + } + let end = page.row_start + page.array.len(); + state.body_col_page_cache.push(page); + record_body_col_page_cache_len(state.body_col_page_cache.len()); + if target_row < end { + return Ok(()); + } + } +} + +fn finalize_output_writer( + state: OutputWriterStorage, + per_output_static: &[PerOutputStatic], +) -> Result { + let OutputWriterStorage { + output_idx, + output_path, + writer, + mut service_names, + num_rows, + } = state; + + let _metadata = writer + .close() + .with_context(|| format!("closing writer for output {output_idx}"))?; + + let size_bytes = std::fs::metadata(&output_path) + .with_context(|| format!("stat output file: {}", output_path.display()))? + .len(); + + let static_meta = &per_output_static[output_idx]; + + // If `service` is a sort column for this schema, it took the + // sort-col write path and `service_names` (populated by the body- + // col `track_service` branch) never saw it. Fold in the names + // from the per-output sort batch so the `TAG_SERVICE` low- + // cardinality metadata stays accurate regardless of which path + // wrote the column. + if let Ok(service_col_idx) = static_meta.sort_optimised.schema().index_of("service") { + collect_service_names_from_page( + static_meta.sort_optimised.column(service_col_idx).as_ref(), + &mut service_names, + ) + .with_context(|| { + format!("collecting service names from sort col for output {output_idx}") + })?; + } + + let mut low_cardinality_tags: HashMap> = HashMap::new(); + if !service_names.is_empty() { + low_cardinality_tags.insert(TAG_SERVICE.to_string(), service_names); + } + + Ok(MergeOutputFile { + path: output_path, + num_rows, + num_row_groups: 1, + size_bytes, + row_keys_proto: static_meta.row_keys_proto.clone(), + zonemap_regexes: static_meta.zonemap_regexes.clone(), + metric_names: static_meta.metric_names.clone(), + time_range: static_meta.time_range, + low_cardinality_tags, + }) +} + +// ============================================================================ +// Tests +// ============================================================================ + +#[cfg(test)] +mod tests { + use std::path::PathBuf; + use std::sync::Arc; + + use arrow::array::{ + ArrayRef, BinaryArray, DictionaryArray, Float64Array, Int64Array, StringArray, UInt8Array, + UInt64Array, + }; + use arrow::datatypes::{DataType, Field, Int32Type, Schema as ArrowSchema}; + use bytes::Bytes; + use parquet::arrow::ArrowWriter; + use parquet::file::metadata::KeyValue; + use parquet::file::properties::WriterProperties; + use parquet::file::reader::{FileReader, SerializedFileReader}; + use tempfile::TempDir; + use tokio::io::AsyncRead; + + use super::*; + use crate::storage::page_decoder::StreamDecoder; + use crate::storage::streaming_reader::{RemoteByteSource, StreamingParquetReader}; + use crate::storage::{Compression, ParquetWriterConfig}; + + // -------- Fixtures -------- + + /// Build a sorted metrics RecordBatch with `num_rows` rows in + /// the **storage column order**: sort cols (metric_name, timestamp_secs) + /// → sorted_series → remaining body cols lexicographic + /// (metric_type, service, timeseries_id, value). All rows share + /// the single metric_name "cpu.usage". `sorted_series` is monotonic + /// from `start_series_idx`. `service` carries nulls every 5th row. + fn make_sorted_batch(num_rows: usize, start_series_idx: u64) -> RecordBatch { + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(vec![ + // sort cols (in sort schema order) + Field::new("metric_name", dict_type.clone(), false), + Field::new("timestamp_secs", DataType::UInt64, false), + // sorted_series marker + Field::new("sorted_series", DataType::Binary, false), + // body cols lexicographic + Field::new("metric_type", DataType::UInt8, false), + Field::new("service", dict_type, true), + Field::new("timeseries_id", DataType::Int64, false), + Field::new("value", DataType::Float64, false), + ])); + + let metric_keys: Vec = (0..num_rows as i32).map(|_| 0).collect(); + let metric_values = StringArray::from(vec!["cpu.usage", "memory.used"]); + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(metric_values), + ) + .expect("test dict array"), + ); + let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; num_rows])); + let timestamps: Vec = (0..num_rows as u64) + .map(|i| 1_700_000_000 + (num_rows as u64 - i)) + .collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let values: Vec = (0..num_rows).map(|i| i as f64).collect(); + let value: ArrayRef = Arc::new(Float64Array::from(values)); + let tsids: Vec = (0..num_rows as i64).map(|i| 1000 + i).collect(); + let timeseries_id: ArrayRef = Arc::new(Int64Array::from(tsids)); + let svc_keys: Vec> = (0..num_rows as i32) + .map(|i| if i % 5 == 0 { None } else { Some(i % 3) }) + .collect(); + let svc_values = StringArray::from(vec!["api", "db", "cache"]); + let service: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(svc_keys), + Arc::new(svc_values), + ) + .expect("test dict array"), + ); + let mut series_bytes: Vec> = Vec::with_capacity(num_rows); + for i in 0..num_rows as u64 { + let id = start_series_idx + i; + series_bytes.push(id.to_be_bytes().to_vec()); + } + let series_refs: Vec<&[u8]> = series_bytes.iter().map(|v| v.as_slice()).collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from(series_refs)); + + RecordBatch::try_new( + schema, + vec![ + metric_name, + timestamp_secs, + sorted_series, + metric_type, + service, + timeseries_id, + value, + ], + ) + .expect("test batch") + } + + /// Write a fixture parquet file with the standard `qh.*` KVs that the + /// streaming merge engine validates. + fn write_input_parquet(batches: &[RecordBatch], extra_kvs: &[(&str, &str)]) -> Bytes { + let schema = batches[0].schema(); + let cfg = ParquetWriterConfig { + compression: Compression::Snappy, + ..ParquetWriterConfig::default() + }; + let sort_fields = "metric_name|-timestamp_secs/V2"; + let sort_field_names = vec!["metric_name".to_string(), "timestamp_secs".to_string()]; + let mut kvs = vec![ + KeyValue::new( + PARQUET_META_SORT_FIELDS.to_string(), + sort_fields.to_string(), + ), + KeyValue::new( + PARQUET_META_WINDOW_START.to_string(), + "1700000000".to_string(), + ), + KeyValue::new(PARQUET_META_WINDOW_DURATION.to_string(), "60".to_string()), + KeyValue::new(PARQUET_META_NUM_MERGE_OPS.to_string(), "0".to_string()), + ]; + for (k, v) in extra_kvs { + kvs.push(KeyValue::new(k.to_string(), v.to_string())); + } + let sorting_cols = vec![ + parquet::file::metadata::SortingColumn { + column_idx: schema.index_of("metric_name").expect("test schema") as i32, + descending: false, + nulls_first: false, + }, + parquet::file::metadata::SortingColumn { + column_idx: schema.index_of("timestamp_secs").expect("test schema") as i32, + descending: true, + nulls_first: false, + }, + ]; + let props: WriterProperties = cfg.to_writer_properties_with_metadata( + &schema, + sorting_cols, + Some(kvs), + &sort_field_names, + ); + let mut buf: Vec = Vec::new(); + let mut writer = ArrowWriter::try_new(&mut buf, schema, Some(props)).expect("arrow writer"); + for b in batches { + writer.write(b).expect("test write"); + } + writer.close().expect("test close"); + Bytes::from(buf) + } + + // -------- In-memory byte source -------- + + struct InMemorySource { + bytes: Bytes, + } + + #[async_trait::async_trait] + impl RemoteByteSource for InMemorySource { + async fn file_size(&self, _path: &std::path::Path) -> std::io::Result { + Ok(self.bytes.len() as u64) + } + async fn get_slice( + &self, + _path: &std::path::Path, + range: std::ops::Range, + ) -> std::io::Result { + Ok(self.bytes.slice(range.start as usize..range.end as usize)) + } + async fn get_slice_stream( + &self, + _path: &std::path::Path, + range: std::ops::Range, + ) -> std::io::Result> { + let slice = self.bytes.slice(range.start as usize..range.end as usize); + Ok(Box::new(std::io::Cursor::new(slice.to_vec()))) + } + } + + async fn open_stream(bytes: Bytes) -> Box { + let source = Arc::new(InMemorySource { bytes }); + let reader = StreamingParquetReader::try_open(source, PathBuf::from("test.parquet")) + .await + .expect("open reader"); + Box::new(reader) + } + + /// Read an output parquet file back into a single concatenated RecordBatch. + fn read_output_to_record_batch(path: &Path) -> RecordBatch { + let bytes = std::fs::read(path).expect("read output"); + let builder = parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder::try_new( + Bytes::from(bytes), + ) + .expect("read output builder"); + let schema = builder.schema().clone(); + let reader = builder.build().expect("read output build"); + let batches: Vec = reader.collect::, _>>().expect("read output"); + if batches.is_empty() { + RecordBatch::new_empty(schema) + } else { + arrow::compute::concat_batches(&schema, &batches).expect("concat") + } + } + + fn merge_config(num_outputs: usize) -> MergeConfig { + MergeConfig { + num_outputs, + writer_config: ParquetWriterConfig { + compression: Compression::Snappy, + ..ParquetWriterConfig::default() + }, + } + } + + // -------- Tests -------- + + /// Two inputs → one output: row count and sort order preserved. + #[tokio::test] + async fn test_two_inputs_simple_merge() { + let batch_a = make_sorted_batch(50, 0); + let batch_b = make_sorted_batch(50, 50); + let bytes_a = write_input_parquet(std::slice::from_ref(&batch_a), &[]); + let bytes_b = write_input_parquet(std::slice::from_ref(&batch_b), &[]); + + let inputs: Vec> = + vec![open_stream(bytes_a).await, open_stream(bytes_b).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge"); + assert_eq!(outputs.len(), 1); + assert_eq!(outputs[0].num_rows, 100); + + let merged = read_output_to_record_batch(&outputs[0].path); + assert_eq!(merged.num_rows(), 100); + let ss_array = merged.column(merged.schema().index_of("sorted_series").expect("col")); + let ss = ss_array + .as_any() + .downcast_ref::() + .expect("binary"); + for i in 0..ss_array.len().saturating_sub(1) { + assert!( + ss.value(i) <= ss.value(i + 1), + "row {i}: sorted_series not ascending", + ); + } + } + + /// Single-metric_name input + num_outputs=1 → output is single row group. + #[tokio::test] + async fn test_output_is_single_row_group() { + let batch_a = make_sorted_batch(200, 0); + let bytes_a = write_input_parquet(std::slice::from_ref(&batch_a), &[]); + let inputs: Vec> = vec![open_stream(bytes_a).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge"); + assert_eq!(outputs.len(), 1); + + let bytes = std::fs::read(&outputs[0].path).expect("read"); + let reader = SerializedFileReader::new(Bytes::from(bytes)).expect("ser reader"); + assert_eq!( + reader.metadata().num_row_groups(), + 1, + "single-metric_name single-output merge must produce single row group", + ); + } + + /// N inputs → M outputs: total row count preserved (MC-1). + #[tokio::test] + async fn test_total_rows_preserved() { + let batch_a = make_sorted_batch(75, 0); + let batch_b = make_sorted_batch(50, 100); + let batch_c = make_sorted_batch(25, 200); + let bytes_a = write_input_parquet(std::slice::from_ref(&batch_a), &[]); + let bytes_b = write_input_parquet(std::slice::from_ref(&batch_b), &[]); + let bytes_c = write_input_parquet(std::slice::from_ref(&batch_c), &[]); + + let inputs: Vec> = vec![ + open_stream(bytes_a).await, + open_stream(bytes_b).await, + open_stream(bytes_c).await, + ]; + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(2)) + .await + .expect("merge"); + + let total: usize = outputs.iter().map(|o| o.num_rows).sum(); + assert_eq!(total, 150); + } + + /// Sort schema mismatch across inputs is rejected. + #[tokio::test] + async fn test_sort_schema_mismatch_rejected() { + let batch_a = make_sorted_batch(20, 0); + let bytes_a = write_input_parquet(std::slice::from_ref(&batch_a), &[]); + + let cfg = ParquetWriterConfig { + compression: Compression::Snappy, + ..ParquetWriterConfig::default() + }; + let kvs = vec![ + KeyValue::new( + PARQUET_META_SORT_FIELDS.to_string(), + "service|-timestamp_secs/V2".to_string(), + ), + KeyValue::new( + PARQUET_META_WINDOW_START.to_string(), + "1700000000".to_string(), + ), + KeyValue::new(PARQUET_META_WINDOW_DURATION.to_string(), "60".to_string()), + KeyValue::new(PARQUET_META_NUM_MERGE_OPS.to_string(), "0".to_string()), + ]; + let props: WriterProperties = cfg.to_writer_properties_with_metadata( + &batch_a.schema(), + Vec::new(), + Some(kvs), + &["service".to_string(), "timestamp_secs".to_string()], + ); + let mut buf: Vec = Vec::new(); + let mut writer = + ArrowWriter::try_new(&mut buf, batch_a.schema(), Some(props)).expect("arrow writer"); + writer.write(&batch_a).expect("write"); + writer.close().expect("close"); + let bytes_b = Bytes::from(buf); + + let inputs: Vec> = + vec![open_stream(bytes_a).await, open_stream(bytes_b).await]; + let tmp = TempDir::new().expect("tmpdir"); + let err = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect_err("must reject mismatched sort schema"); + let s = err.to_string(); + assert!( + s.contains("sort schema mismatch"), + "expected 'sort schema mismatch', got: {s}", + ); + } + + /// qh.* KV metadata is propagated to the output; num_merge_ops increments. + #[tokio::test] + async fn test_kv_metadata_propagated_to_output() { + let batch_a = make_sorted_batch(40, 0); + let bytes_a = write_input_parquet(std::slice::from_ref(&batch_a), &[]); + let inputs: Vec> = vec![open_stream(bytes_a).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge"); + let bytes = std::fs::read(&outputs[0].path).expect("read"); + let reader = SerializedFileReader::new(Bytes::from(bytes)).expect("ser reader"); + let kvs = reader + .metadata() + .file_metadata() + .key_value_metadata() + .cloned() + .unwrap_or_default(); + let find = |k: &str| -> Option { + kvs.iter() + .find(|kv| kv.key == k) + .and_then(|kv| kv.value.clone()) + }; + assert_eq!( + find(PARQUET_META_SORT_FIELDS).as_deref(), + Some("metric_name|-timestamp_secs/V2"), + ); + assert_eq!( + find(PARQUET_META_WINDOW_START).as_deref(), + Some("1700000000") + ); + assert_eq!(find(PARQUET_META_WINDOW_DURATION).as_deref(), Some("60")); + assert_eq!( + find(PARQUET_META_NUM_MERGE_OPS).as_deref(), + Some("1"), + "num_merge_ops must increment by 1 over input's max", + ); + } + + /// All-empty inputs produce no output. + #[tokio::test] + async fn test_all_empty_inputs_no_output() { + let empty = make_sorted_batch(0, 0); + let bytes = write_input_parquet(std::slice::from_ref(&empty), &[]); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge"); + assert!(outputs.is_empty()); + } + + /// The streaming engine's output can be drained back via the new + /// page-bounded decoder. End-to-end sanity check. + #[tokio::test] + async fn test_output_drainable_by_stream_decoder() { + let batch_a = make_sorted_batch(40, 0); + let batch_b = make_sorted_batch(40, 40); + let bytes_a = write_input_parquet(std::slice::from_ref(&batch_a), &[]); + let bytes_b = write_input_parquet(std::slice::from_ref(&batch_b), &[]); + + let inputs: Vec> = + vec![open_stream(bytes_a).await, open_stream(bytes_b).await]; + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge"); + + let bytes = std::fs::read(&outputs[0].path).expect("read"); + let mut output_stream = open_stream(Bytes::from(bytes)).await; + let mut decoder = StreamDecoder::new(&mut *output_stream); + let mut total_decoded = 0usize; + while let Some(page) = decoder.decode_next_page().await.expect("decode") { + // Count only sort col 0 (col_idx 0) pages to get a row count. + if page.col_idx == 0 { + total_decoded += page.array.len(); + } + } + assert_eq!(total_decoded, 80); + } + + /// Page-bounded contract sanity: with a row group large enough to + /// require many parquet pages per col, body col writes go through + /// the page-by-page assembler instead of materialising column + /// chunks. We can't directly observe peak memory from a test, but + /// we *can* assert that the merge completes correctly with input + /// data whose body cols span many pages, and that the output is + /// itself multi-page (no whole-column buffering happened on the + /// output side either). + #[tokio::test] + async fn test_body_col_streams_many_pages_per_column_chunk() { + // Force multiple pages per column chunk by setting a small + // data_page_row_count_limit. With 8000 rows and a 1000-row + // page limit, the output value col chunk must span ≥ 8 pages. + let batch = make_sorted_batch(8000, 0); + let bytes = write_input_parquet(std::slice::from_ref(&batch), &[]); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let writer_config = ParquetWriterConfig { + compression: Compression::Snappy, + data_page_row_count_limit: 1000, + ..ParquetWriterConfig::default() + }; + let config = MergeConfig { + num_outputs: 1, + writer_config, + }; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &config) + .await + .expect("merge"); + assert_eq!(outputs.len(), 1); + assert_eq!(outputs[0].num_rows, 8000); + + // Verify the output is itself multi-page-per-column (which is + // what page-bounded writing should produce, given the default + // data_page_size). Read via the page-bounded decoder and count + // pages for the value column. + let out_bytes = std::fs::read(&outputs[0].path).expect("read"); + let mut output_stream = open_stream(Bytes::from(out_bytes)).await; + // Find the "value" col index in the output's arrow schema BEFORE + // borrowing output_stream mutably for the decoder. + let arrow_schema = parquet::arrow::parquet_to_arrow_schema( + output_stream.metadata().file_metadata().schema_descr(), + None, + ) + .expect("arrow schema"); + let value_col_idx = arrow_schema.index_of("value").expect("value col"); + let mut decoder = StreamDecoder::new(&mut *output_stream); + + let mut value_pages = 0; + while let Some(page) = decoder.decode_next_page().await.expect("decode") { + if page.col_idx == value_col_idx { + value_pages += 1; + } + } + assert!( + value_pages >= 2, + "expected output 'value' col to span multiple pages (got {value_pages}); body col \ + writes should respect data_page_size", + ); + } + + /// Regression for Codex P1 on PR-6409: a zero-row-group input + /// must not panic the body-column path. Phase 0 explicitly accepts + /// empty inputs (returning a zero-row sort batch), so the body-col + /// loop has to defend against `row_group(0)` lookups on inputs + /// with `num_row_groups() == 0`. + #[tokio::test] + async fn test_zero_row_input_mixed_with_non_empty() { + let empty = make_sorted_batch(0, 0); + let populated = make_sorted_batch(50, 0); + let bytes_empty = write_input_parquet(std::slice::from_ref(&empty), &[]); + let bytes_populated = write_input_parquet(std::slice::from_ref(&populated), &[]); + + let inputs: Vec> = vec![ + open_stream(bytes_empty).await, + open_stream(bytes_populated).await, + ]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge with mixed empty + populated inputs"); + assert_eq!(outputs.len(), 1); + assert_eq!(outputs[0].num_rows, 50); + + let merged = read_output_to_record_batch(&outputs[0].path); + let value = merged + .column(merged.schema().index_of("value").expect("value col")) + .as_any() + .downcast_ref::() + .expect("Float64"); + // All 50 values from the populated input should round-trip in + // input row order (timestamps descend in input row order to + // match the sort key). + for i in 0..50 { + assert!( + (value.value(i) - i as f64).abs() < 1e-9, + "row {i}: expected {i}, got {}", + value.value(i), + ); + } + } + + /// Regression for Codex P2 on PR-6409: `derive_output_schema` + /// must drop sort fields that `optimize_output_batch` discarded as + /// all-null for a given output. Before the fix the check was + /// `sort_optimised.has(name) || full_union.has(name)`, where the + /// second disjunct is trivially true for every iterated field, so + /// all-null sort columns were never dropped. We exercise the + /// helper directly with a synthetic union + sort-optimised pair so + /// the regression doesn't depend on the merge plan creating an + /// all-null sort col (which requires a multi-input fixture with + /// divergent sort col presence). + #[test] + fn test_derive_output_schema_drops_all_null_sort_field() { + // Union covers sort fields {metric_name, env, timestamp_secs} + // plus body field {value}. `env` is a sort field declared in + // the sort union but `optimize_output_batch` dropped it from + // this output's `sort_optimised` (all-null for this output's + // rows). Body field `value` is NOT in the sort union — it + // must be preserved unconditionally. + let full_union = Arc::new(ArrowSchema::new(vec![ + Field::new("metric_name", DataType::Utf8, false), + Field::new("env", DataType::Utf8, true), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("value", DataType::Float64, false), + ])); + let sort_union = Arc::new(ArrowSchema::new(vec![ + Field::new("metric_name", DataType::Utf8, false), + Field::new("env", DataType::Utf8, true), + Field::new("timestamp_secs", DataType::UInt64, false), + ])); + // sort_optimised has dropped `env` (all-null for this output). + let sort_optimised_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("metric_name", DataType::Utf8, false), + Field::new("timestamp_secs", DataType::UInt64, false), + ])); + let sort_optimised = RecordBatch::try_new( + sort_optimised_schema, + vec![ + Arc::new(StringArray::from(vec!["cpu.usage"; 4])), + Arc::new(UInt64Array::from(vec![1u64, 2, 3, 4])), + ], + ) + .expect("test batch"); + + let derived = derive_output_schema(&full_union, &sort_union, &sort_optimised).expect("ok"); + let names: Vec<&str> = derived.fields().iter().map(|f| f.name().as_str()).collect(); + assert_eq!( + names, + vec!["metric_name", "timestamp_secs", "value"], + "all-null sort field 'env' must be dropped; body field 'value' preserved", + ); + + // Direct contrast: with the pre-fix logic (every field kept) + // the result would have included 'env'. Asserting the negative + // form makes the regression unambiguous. + assert!( + derived.index_of("env").is_err(), + "'env' must not appear in derived output schema", + ); + } + + /// Write a fixture parquet file where each body column is forced + /// to span multiple parquet data pages by pinning a small + /// `data_page_row_count_limit`. The merge engine must read those + /// pages back via a single persistent `StreamDecoder` per input — + /// reconstructing the decoder for each `fill_page_cache_to_row` + /// call (the pre-fix behaviour) would reset the per-column + /// `rows_decoded` counter, making `DecodedPage::row_start` reset + /// to zero on every page after the first. + fn write_input_parquet_with_small_pages( + batches: &[RecordBatch], + data_page_row_count_limit: usize, + ) -> Bytes { + let schema = batches[0].schema(); + // Lower `write_batch_size` and `data_page_size` so the arrow + // writer actually flushes pages at the row-count boundary. + // With the defaults (`write_batch_size = 64 KiB`, + // `data_page_size = 1 MiB`) the byte-size threshold doesn't + // trip for our small fixtures and the writer emits one page + // per column regardless of `data_page_row_count_limit`. + let cfg = ParquetWriterConfig { + compression: Compression::Snappy, + data_page_row_count_limit, + data_page_size: data_page_row_count_limit * 16, + write_batch_size: data_page_row_count_limit, + ..ParquetWriterConfig::default() + }; + let sort_fields = "metric_name|-timestamp_secs/V2"; + let sort_field_names = vec!["metric_name".to_string(), "timestamp_secs".to_string()]; + let kvs = vec![ + KeyValue::new( + PARQUET_META_SORT_FIELDS.to_string(), + sort_fields.to_string(), + ), + KeyValue::new( + PARQUET_META_WINDOW_START.to_string(), + "1700000000".to_string(), + ), + KeyValue::new(PARQUET_META_WINDOW_DURATION.to_string(), "60".to_string()), + KeyValue::new(PARQUET_META_NUM_MERGE_OPS.to_string(), "0".to_string()), + ]; + let sorting_cols = vec![ + parquet::file::metadata::SortingColumn { + column_idx: schema.index_of("metric_name").expect("test schema") as i32, + descending: false, + nulls_first: false, + }, + parquet::file::metadata::SortingColumn { + column_idx: schema.index_of("timestamp_secs").expect("test schema") as i32, + descending: true, + nulls_first: false, + }, + ]; + let props: WriterProperties = cfg.to_writer_properties_with_metadata( + &schema, + sorting_cols, + Some(kvs), + &sort_field_names, + ); + let mut buf: Vec = Vec::new(); + let mut writer = ArrowWriter::try_new(&mut buf, schema, Some(props)).expect("arrow writer"); + for b in batches { + writer.write(b).expect("test write"); + } + writer.close().expect("test close"); + Bytes::from(buf) + } + + /// Regression for Codex P1 on PR-6409: when a body column spans + /// multiple input pages, every page-fetch must come from the same + /// long-lived `StreamDecoder` so its per-column `rows_decoded` + /// counter keeps producing absolute row offsets. Before the fix, + /// each `fill_page_cache_to_row` call instantiated a fresh decoder + /// whose counter started at zero — the *second* decoded page + /// reported `row_start = 0` and the page cache's + /// `(input_row - cache_start)` indexing landed on the wrong rows + /// (or panicked on out-of-bounds). + #[tokio::test] + async fn test_body_col_multi_input_page_preserves_row_start() { + // The bug only surfaces when `assemble_one_output_page` is + // called more than once per output (so `fill_page_cache_to_row` + // is invoked repeatedly with a non-empty cache). That means we + // need more than `OUTPUT_PAGE_ROWS` (=1024) total input rows + // for a single output. 2500 rows × 50-row input pages = + // 50 body-col pages per column chunk; three output pages of + // 1024+1024+452 each trigger a separate decoder advance. + let total_rows = 2500; + let batch = make_sorted_batch(total_rows, 0); + let bytes = write_input_parquet_with_small_pages(std::slice::from_ref(&batch), 50); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge"); + assert_eq!(outputs.len(), 1); + assert_eq!(outputs[0].num_rows, total_rows); + + let merged = read_output_to_record_batch(&outputs[0].path); + let value_idx = merged.schema().index_of("value").expect("value col"); + let value = merged + .column(value_idx) + .as_any() + .downcast_ref::() + .expect("value col is Float64"); + + // `make_sorted_batch` fills `value` with `i as f64` (see the + // fixture). Timestamps descend in input row order, matching the + // sort key (timestamp_secs DESC), so the merge with a single + // input is the identity permutation. + for i in 0..total_rows { + let expected = i as f64; + let got = value.value(i); + assert!( + (expected - got).abs() < 1e-9, + "row {i}: expected value {expected}, got {got} — body col page cache reported \ + wrong row_start", + ); + } + } + + /// Regression for Codex P1 on PR-6409 (the multi-output half): when + /// a body column is consumed by more than one output, the per-input + /// page cache and decoder must outlive each output's assembler. The + /// stream cannot be rewound, so dropping a partially-consumed page + /// when output K ends would leave output K+1 unable to read rows + /// that physically live inside that same page. + #[tokio::test] + async fn test_body_col_cache_persists_across_outputs() { + // Two metric names so the engine splits the merge into two + // outputs at the metric_name boundary. Each input has 200 + // rows of cpu.usage then 200 rows of memory.used — small + // 50-row pages mean some pages span the boundary. + let schema = make_sorted_batch(0, 0).schema(); + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let metric_values = StringArray::from(vec!["cpu.usage", "memory.used"]); + let keys: Vec = (0..400).map(|i| if i < 200 { 0 } else { 1 }).collect(); + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(keys), + Arc::new(metric_values), + ) + .expect("dict"), + ); + + // Timestamps descend within each metric_name run so the file + // is sorted by (metric_name ASC, timestamp DESC) — matching + // the sort schema declared in write_input_parquet_with_small_pages. + let timestamps: Vec = (0..400) + .map(|i| { + let run_pos = if i < 200 { i } else { i - 200 }; + 1_700_000_000u64 + (199 - run_pos as u64) + }) + .collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + + let series: Vec> = (0..400u64).map(|i| i.to_be_bytes().to_vec()).collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from( + series.iter().map(|v| v.as_slice()).collect::>(), + )); + + let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![1u8; 400])); + let service_keys: Vec = (0..400i32).map(|_| 0).collect(); + let service_values = StringArray::from(vec!["svc-a"]); + let service: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(service_keys), + Arc::new(service_values), + ) + .expect("svc dict"), + ); + let timeseries_id: ArrayRef = Arc::new(Int64Array::from((0..400i64).collect::>())); + let value: ArrayRef = Arc::new(Float64Array::from( + (0..400).map(|i| i as f64 * 0.5).collect::>(), + )); + // Confirm the schema we hand-build still matches make_sorted_batch's: + let _ = dict_type; + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + metric_name, + timestamp_secs, + sorted_series, + metric_type, + service, + timeseries_id, + value, + ], + ) + .expect("test batch"); + + let bytes = write_input_parquet_with_small_pages(std::slice::from_ref(&batch), 50); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(2)) + .await + .expect("merge"); + assert_eq!(outputs.len(), 2, "two metric names → two outputs"); + + let total: usize = outputs.iter().map(|o| o.num_rows).sum(); + assert_eq!(total, 400); + + // Concatenate the two outputs' value columns in output order + // and verify every original value is present. The merge is + // sort-stable, so values within each output appear in input + // row order (timestamps descend within each metric run). + let mut seen_values: HashSet = HashSet::new(); + for out in &outputs { + let merged = read_output_to_record_batch(&out.path); + let v = merged + .column(merged.schema().index_of("value").expect("value col")) + .as_any() + .downcast_ref::() + .expect("Float64"); + for i in 0..v.len() { + // Encode as integer bits to dedupe; original values + // are i * 0.5 for i in 0..400, all distinct. + seen_values.insert(v.value(i).to_bits()); + } + } + assert_eq!( + seen_values.len(), + 400, + "every input value should round-trip through the merge across both outputs", + ); + } + + /// Multi-RG input is rejected (PR-6b.2 simplification). + #[tokio::test] + async fn test_multi_rg_input_rejected() { + // Force a 2-RG file by writing two batches with row_group_size = 1 + // small enough to trip RG rollover. + let batch_a = make_sorted_batch(50, 0); + let batch_b = make_sorted_batch(50, 50); + + let cfg = ParquetWriterConfig { + compression: Compression::Snappy, + row_group_size: 50, // force one RG per 50-row batch + ..ParquetWriterConfig::default() + }; + let kvs = vec![ + KeyValue::new( + PARQUET_META_SORT_FIELDS.to_string(), + "metric_name|-timestamp_secs/V2".to_string(), + ), + KeyValue::new( + PARQUET_META_WINDOW_START.to_string(), + "1700000000".to_string(), + ), + KeyValue::new(PARQUET_META_WINDOW_DURATION.to_string(), "60".to_string()), + KeyValue::new(PARQUET_META_NUM_MERGE_OPS.to_string(), "0".to_string()), + ]; + let props: WriterProperties = cfg.to_writer_properties_with_metadata( + &batch_a.schema(), + Vec::new(), + Some(kvs), + &["metric_name".to_string(), "timestamp_secs".to_string()], + ); + let mut buf: Vec = Vec::new(); + let mut writer = + ArrowWriter::try_new(&mut buf, batch_a.schema(), Some(props)).expect("arrow writer"); + writer.write(&batch_a).expect("write"); + writer.write(&batch_b).expect("write"); + writer.close().expect("close"); + let bytes = Bytes::from(buf); + + let inputs: Vec> = vec![open_stream(bytes).await]; + let tmp = TempDir::new().expect("tmpdir"); + let err = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect_err("multi-RG input must be rejected"); + let s = err.to_string(); + assert!( + s.contains("single-row-group"), + "expected 'single-row-group' error, got: {s}", + ); + } + + // ============================================================================ + // MC-2 round-trip: every parquet physical type the decoder supports must + // survive the streaming merge unchanged. + // ============================================================================ + + /// Build a batch covering every primitive type the page decoder + /// supports, plus byte arrays, dictionary encoding, and list types. + /// Each row's `sorted_series` key uniquely identifies the row so + /// callers can build a `(key → tuple)` map for output comparison. + fn make_typed_round_trip_batch(num_rows: usize, key_offset: u64) -> RecordBatch { + use arrow::array::{ + BooleanArray, Float32Array, Int8Array, Int16Array, Int32Array, LargeBinaryArray, + ListArray, UInt16Array, UInt32Array, + }; + use arrow::buffer::OffsetBuffer; + + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + // Body cols MUST be in lexicographic order — that's the + // storage convention the streaming engine assumes when + // iterating columns. Inputs that ship body cols in a + // different order trip "column ordering violated" mid-merge. + let schema = Arc::new(ArrowSchema::new(vec![ + // sort cols + Field::new("metric_name", dict_type.clone(), false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("sorted_series", DataType::Binary, false), + // body cols, all nullable, in lexicographic order. Null + // every 7th row to exercise the null-mask path. + Field::new("body_bool", DataType::Boolean, true), + Field::new("body_dict", dict_type, true), + Field::new("body_float32", DataType::Float32, true), + Field::new("body_float64", DataType::Float64, true), + Field::new("body_int16", DataType::Int16, true), + Field::new("body_int32", DataType::Int32, true), + Field::new("body_int64", DataType::Int64, true), + Field::new("body_int8", DataType::Int8, true), + Field::new("body_largebinary", DataType::LargeBinary, true), + // `List` covers production-shaped histogram bucket + // columns. Outer + inner both non-nullable to match the + // decoder's PR-6a.2 contract. + Field::new( + "body_list_f64", + DataType::List(Arc::new(Field::new("item", DataType::Float64, false))), + false, + ), + Field::new("body_string", DataType::Utf8, true), + Field::new("body_uint16", DataType::UInt16, true), + Field::new("body_uint32", DataType::UInt32, true), + Field::new("body_uint64", DataType::UInt64, true), + Field::new("body_uint8", DataType::UInt8, true), + ])); + + let is_null = |i: usize| i.is_multiple_of(7); + + let metric_keys: Vec = vec![0; num_rows]; + let metric_values = StringArray::from(vec!["cpu.usage"]); + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(metric_values), + ) + .expect("metric dict"), + ); + + // Timestamps DESC within the run so the input is pre-sorted on + // (metric_name ASC, timestamp DESC) per the sort schema. + let timestamps: Vec = (0..num_rows as u64) + .map(|i| 1_700_000_000 + (num_rows as u64 - i)) + .collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + + let key_bytes: Vec> = (0..num_rows as u64) + .map(|i| (key_offset + i).to_be_bytes().to_vec()) + .collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from( + key_bytes.iter().map(|v| v.as_slice()).collect::>(), + )); + + // Primitive value generators chosen to span each type's range + // including signed/unsigned boundaries. + let mk_opt = |i: usize, v: i64| if is_null(i) { None } else { Some(v) }; + let body_int8: ArrayRef = Arc::new(Int8Array::from( + (0..num_rows) + .map(|i| mk_opt(i, (i as i64 % 251) - 125).map(|v| v as i8)) + .collect::>(), + )); + let body_uint8: ArrayRef = Arc::new(UInt8Array::from( + (0..num_rows) + .map(|i| { + if is_null(i) { + None + } else { + Some((i % 255) as u8) + } + }) + .collect::>(), + )); + let body_int16: ArrayRef = Arc::new(Int16Array::from( + (0..num_rows) + .map(|i| mk_opt(i, (i as i64 % 30001) - 15000).map(|v| v as i16)) + .collect::>(), + )); + let body_uint16: ArrayRef = Arc::new(UInt16Array::from( + (0..num_rows) + .map(|i| { + if is_null(i) { + None + } else { + Some((i % 60000) as u16) + } + }) + .collect::>(), + )); + let body_int32: ArrayRef = Arc::new(Int32Array::from( + (0..num_rows) + .map(|i| { + mk_opt( + i, + i as i64 * 0x100_0000i64 - i64::from(i32::MIN.unsigned_abs() / 2), + ) + .map(|v| v as i32) + }) + .collect::>(), + )); + let body_uint32: ArrayRef = Arc::new(UInt32Array::from( + (0..num_rows) + .map(|i| { + if is_null(i) { + None + } else { + Some((i as u32).wrapping_mul(0xDEAD_BEEF)) + } + }) + .collect::>(), + )); + let body_int64: ArrayRef = Arc::new(Int64Array::from( + (0..num_rows) + .map(|i| { + if is_null(i) { + None + } else { + Some((i as i64).wrapping_mul(0x0123_4567_89AB_CDEF)) + } + }) + .collect::>(), + )); + let body_uint64: ArrayRef = Arc::new(UInt64Array::from( + (0..num_rows) + .map(|i| { + if is_null(i) { + None + } else { + Some((i as u64).wrapping_mul(0xFEDC_BA98_7654_3210)) + } + }) + .collect::>(), + )); + let body_float32: ArrayRef = Arc::new(Float32Array::from( + (0..num_rows) + .map(|i| { + if is_null(i) { + None + } else { + Some(i as f32 * 0.25 - 100.0) + } + }) + .collect::>(), + )); + let body_float64: ArrayRef = Arc::new(Float64Array::from( + (0..num_rows) + .map(|i| { + if is_null(i) { + None + } else { + Some(i as f64 * 0.5 - 1e6) + } + }) + .collect::>(), + )); + let body_bool: ArrayRef = Arc::new(BooleanArray::from( + (0..num_rows) + .map(|i| if is_null(i) { None } else { Some(i % 3 == 0) }) + .collect::>(), + )); + + let body_string_vals: Vec> = (0..num_rows) + .map(|i| { + if is_null(i) { + None + } else { + Some(format!("row-{i:08}-payload")) + } + }) + .collect(); + let body_string: ArrayRef = Arc::new(StringArray::from(body_string_vals)); + + let body_largebinary_vals: Vec>> = (0..num_rows) + .map(|i| { + if is_null(i) { + None + } else { + Some(vec![(i & 0xFF) as u8; 1 + (i % 5)]) + } + }) + .collect(); + let body_largebinary: ArrayRef = Arc::new(LargeBinaryArray::from_opt_vec( + body_largebinary_vals + .iter() + .map(|opt| opt.as_deref()) + .collect(), + )); + + // Dict body col cycles through a small set so the dict-encoding + // path is exercised end-to-end. + let dict_pool = ["api", "db", "cache", "auth", "billing"]; + let dict_keys: Vec> = (0..num_rows as i32) + .map(|i| { + if is_null(i as usize) { + None + } else { + Some(i % (dict_pool.len() as i32)) + } + }) + .collect(); + let body_dict: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(dict_keys), + Arc::new(StringArray::from(dict_pool.to_vec())), + ) + .expect("body_dict"), + ); + + // List: row i has a list of length (i % 4) with f64 + // values derived from i and j. Outer + inner non-null (the + // decoder's PR-6a.2 list path requires both). Empty lists are + // still exercised on rows where (i % 4) == 0. + let mut list_offsets: Vec = Vec::with_capacity(num_rows + 1); + let mut list_values: Vec = Vec::new(); + list_offsets.push(0); + for i in 0..num_rows { + for j in 0..(i % 4) { + list_values.push((i * 10 + j) as f64 + 0.125); + } + list_offsets.push(list_values.len() as i32); + } + let list_inner_field = Arc::new(Field::new("item", DataType::Float64, false)); + let list_inner: ArrayRef = Arc::new(Float64Array::from(list_values)); + let body_list_f64: ArrayRef = Arc::new(ListArray::new( + list_inner_field, + OffsetBuffer::new(arrow::buffer::ScalarBuffer::from(list_offsets)), + list_inner, + None, + )); + + RecordBatch::try_new( + schema, + vec![ + metric_name, + timestamp_secs, + sorted_series, + body_bool, + body_dict, + body_float32, + body_float64, + body_int16, + body_int32, + body_int64, + body_int8, + body_largebinary, + body_list_f64, + body_string, + body_uint16, + body_uint32, + body_uint64, + body_uint8, + ], + ) + .expect("typed round-trip batch") + } + + /// Render a single row's body-col cell to a comparable string. + /// Used by the round-trip test to compare logical values across + /// the merge. MC-2 (row contents preservation) is about logical + /// values, not internal storage layout: `Dictionary`, + /// `Utf8`, and `LargeUtf8` carrying the same string are the same + /// row content; similarly for `Binary` / `LargeBinary`. The + /// streaming engine normalizes string-flavoured types to `Utf8` + /// via `normalize_type` in the union schema, and parquet has only + /// one byte-array physical type so `LargeBinary` round-trips as + /// `Binary`. Both transformations are storage-encoding changes, + /// not value changes — the comparison must see them as equal. + fn render_cell(arr: &dyn arrow::array::Array, row: usize) -> String { + use arrow::array::AsArray; + use arrow::datatypes::Int32Type as DictKeyInt32; + + if !arr.is_valid(row) { + return "".to_string(); + } + match arr.data_type() { + DataType::Int8 => format!( + "i8:{}", + arr.as_primitive::().value(row) + ), + DataType::Int16 => format!( + "i16:{}", + arr.as_primitive::().value(row) + ), + DataType::Int32 => format!( + "i32:{}", + arr.as_primitive::().value(row) + ), + DataType::Int64 => format!( + "i64:{}", + arr.as_primitive::().value(row) + ), + DataType::UInt8 => format!( + "u8:{}", + arr.as_primitive::().value(row) + ), + DataType::UInt16 => format!( + "u16:{}", + arr.as_primitive::() + .value(row) + ), + DataType::UInt32 => format!( + "u32:{}", + arr.as_primitive::() + .value(row) + ), + DataType::UInt64 => format!( + "u64:{}", + arr.as_primitive::() + .value(row) + ), + DataType::Float32 => { + format!( + "f32:{:#x}", + arr.as_primitive::() + .value(row) + .to_bits() + ) + } + DataType::Float64 => { + format!( + "f64:{:#x}", + arr.as_primitive::() + .value(row) + .to_bits() + ) + } + DataType::Boolean => format!("bool:{}", arr.as_boolean().value(row)), + // String flavours collapse to one rendering — Dict, Utf8, LargeUtf8 are interchangeable by value. + DataType::Utf8 => format!("str:{}", arr.as_string::().value(row)), + DataType::LargeUtf8 => format!("str:{}", arr.as_string::().value(row)), + DataType::Dictionary(_, _) => { + let d = arr.as_dictionary::(); + let key = d.keys().value(row); + let values = d + .values() + .as_any() + .downcast_ref::() + .expect("dict values Utf8"); + format!("str:{}", values.value(key as usize)) + } + // Byte arrays collapse similarly — parquet has only one + // BYTE_ARRAY physical type. + DataType::Binary => format!("bin:{:?}", arr.as_binary::().value(row)), + DataType::LargeBinary => format!("bin:{:?}", arr.as_binary::().value(row)), + DataType::List(_) => { + let list = arr.as_list::(); + let inner = list.value(row); + let inner_f64 = inner.as_primitive::(); + let cells: Vec = (0..inner_f64.len()) + .map(|j| { + if inner_f64.is_valid(j) { + format!("{:#x}", inner_f64.value(j).to_bits()) + } else { + "null".to_string() + } + }) + .collect(); + format!("list_f64:[{}]", cells.join(",")) + } + DataType::LargeList(_) => { + let list = arr.as_list::(); + let inner = list.value(row); + let inner_f64 = inner.as_primitive::(); + let cells: Vec = (0..inner_f64.len()) + .map(|j| { + if inner_f64.is_valid(j) { + format!("{:#x}", inner_f64.value(j).to_bits()) + } else { + "null".to_string() + } + }) + .collect(); + format!("ll_f64:[{}]", cells.join(",")) + } + other => panic!("unexpected body-col data type in round-trip: {other:?}"), + } + } + + /// MC-2: row contents do not change during compaction. Build two + /// inputs that together cover every parquet physical type the + /// decoder supports, merge them via the streaming engine, then + /// build a `(sorted_series_key → rendered tuple)` map from both + /// inputs and from the output. The two maps must be byte-equal — + /// no row added, removed, or mutated. Catches silent type- + /// dispatch bugs (the class that the recent List + /// flattening regression was in). + #[tokio::test] + async fn test_mc2_all_types_round_trip_through_streaming_merge() { + let batch_a = make_typed_round_trip_batch(120, 0); + let batch_b = make_typed_round_trip_batch(120, 10_000); + let bytes_a = write_input_parquet(std::slice::from_ref(&batch_a), &[]); + let bytes_b = write_input_parquet(std::slice::from_ref(&batch_b), &[]); + + let inputs: Vec> = + vec![open_stream(bytes_a).await, open_stream(bytes_b).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge"); + assert_eq!(outputs.len(), 1); + assert_eq!(outputs[0].num_rows, 240); + + // Build the expected (sorted_series → rendered tuple) map from + // both inputs. Body cols are everything past `sorted_series`. + let mut expected: HashMap, String> = HashMap::new(); + for batch in [&batch_a, &batch_b] { + let series_idx = batch.schema().index_of("sorted_series").expect("series"); + let series_col = batch + .column(series_idx) + .as_any() + .downcast_ref::() + .expect("series Binary"); + let body_indices: Vec = + (series_idx + 1..batch.schema().fields().len()).collect(); + for row in 0..batch.num_rows() { + let mut tuple = String::new(); + for (n, &col_idx) in body_indices.iter().enumerate() { + if n > 0 { + tuple.push('|'); + } + tuple.push_str(&render_cell(batch.column(col_idx).as_ref(), row)); + } + let key = series_col.value(row).to_vec(); + let prior = expected.insert(key.clone(), tuple); + assert!( + prior.is_none(), + "input batches share a sorted_series key {key:?} — fixture bug", + ); + } + } + assert_eq!(expected.len(), 240); + + // Build the observed map from the merged output. Note arrow + // type coercions: Utf8 may come back as Dictionary because of + // per-output schema optimisation, and Int32-keyed Dict may + // come back with a different key width. Cast both sides to + // Utf8 / Float64 / etc. via the same `render_cell` helper so + // the comparison is type-insensitive. + let merged = read_output_to_record_batch(&outputs[0].path); + let merged_schema = merged.schema(); + let series_idx = merged_schema.index_of("sorted_series").expect("series"); + let series_col = merged + .column(series_idx) + .as_any() + .downcast_ref::() + .expect("merged series Binary"); + + // Map each body-col name in the inputs to its column index in + // the merged output (positions can shift if output schema + // dropped an all-null sort col). + let input_body_cols: Vec = batch_a + .schema() + .fields() + .iter() + .skip(batch_a.schema().index_of("sorted_series").unwrap() + 1) + .map(|f| f.name().clone()) + .collect(); + let merged_body_indices: Vec = input_body_cols + .iter() + .map(|name| { + merged_schema.index_of(name).unwrap_or_else(|_| { + panic!( + "merged output is missing body col '{name}' — MC-4 column union violated" + ) + }) + }) + .collect(); + + let mut observed: HashMap, String> = HashMap::with_capacity(merged.num_rows()); + for row in 0..merged.num_rows() { + let mut tuple = String::new(); + for (n, &col_idx) in merged_body_indices.iter().enumerate() { + if n > 0 { + tuple.push('|'); + } + tuple.push_str(&render_cell(merged.column(col_idx).as_ref(), row)); + } + let key = series_col.value(row).to_vec(); + let prior = observed.insert(key.clone(), tuple); + assert!( + prior.is_none(), + "merged output has duplicate sorted_series key {key:?} — MC-1 violated", + ); + } + + // MC-1: same set of keys. + assert_eq!( + observed.len(), + expected.len(), + "row count mismatch input vs output", + ); + for (key, want) in &expected { + let got = observed.get(key).unwrap_or_else(|| { + panic!( + "merged output is missing input key {:?} (first body cell expected: {})", + key, + want.split('|').next().unwrap_or("?") + ) + }); + assert_eq!( + got, want, + "body-col tuple mismatch for sorted_series {key:?}: got {got}, want {want}", + ); + } + } + + // ============================================================================ + // MS-7: page-cache bounded-memory contract. The streaming engine's + // raison d'être is that body-col memory stays bounded by ~constant + // (page size × small) regardless of how big the input column gets. + // Concretely, `body_col_page_cache` length per input must stay ≤ a + // small constant — never scale with row count. + // ============================================================================ + + /// Build a fixture that forces many input body-col pages with a + /// pinned `data_page_row_count_limit`, then merge it through the + /// streaming engine and read back the peak cache length. Used by + /// the MS-7 test below across multiple sizes. + async fn merge_and_observe_peak_page_cache(num_rows: usize, page_rows: usize) -> usize { + use std::sync::atomic::Ordering; + + PEAK_BODY_COL_PAGE_CACHE_LEN.store(0, Ordering::Relaxed); + + let batch = make_sorted_batch(num_rows, 0); + let bytes = write_input_parquet_with_small_pages(std::slice::from_ref(&batch), page_rows); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge"); + assert_eq!(outputs.len(), 1); + assert_eq!(outputs[0].num_rows, num_rows); + + PEAK_BODY_COL_PAGE_CACHE_LEN.load(Ordering::Relaxed) + } + + /// MS-7: peak `body_col_page_cache` length is bounded by the + /// ratio of output-page rows to input-page rows — NOT by input + /// column size. Each `assemble_one_output_page` call must read + /// enough input pages to cover `OUTPUT_PAGE_ROWS` rows, then + /// evict everything below the new cursor. So the peak per advance + /// loop is bounded by `ceil(OUTPUT_PAGE_ROWS / page_rows) + small + /// slack` regardless of how many output pages we produce. + /// + /// Without the per-output-page eviction, the peak would scale with + /// the total number of input pages — which scales with input size. + /// A regression that dropped the eviction loop would push the peak + /// past the ceiling for the 30 000-row fixture but not the 3 000- + /// row fixture, breaking both assertions below. + #[tokio::test] + async fn test_ms7_body_col_page_cache_bounded_regardless_of_input_size() { + const PAGE_ROWS: usize = 50; + // ceil(1024 / 50) = 21 in-flight pages needed for one output + // page. Allow 3 slack: decoder lookahead, transient between + // push and check, dict-page-as-data-page corner cases. + const MAX_RESIDENT_PAGES: usize = 24; + + let peak_small = merge_and_observe_peak_page_cache(300, PAGE_ROWS).await; + let peak_medium = merge_and_observe_peak_page_cache(3_000, PAGE_ROWS).await; + let peak_large = merge_and_observe_peak_page_cache(30_000, PAGE_ROWS).await; + + // 300-row fixture has only 6 input pages, so its peak can't + // exceed 6; verifying that the assembler doesn't somehow + // accumulate ghost entries past the input itself. + assert!( + peak_small <= 6 + 1, + "300-row fixture: peak cache len {peak_small} > 7", + ); + // Medium / large fixtures share the same OUTPUT_PAGE_ROWS / + // PAGE_ROWS ratio, so they must share the same peak ceiling. + assert!( + peak_medium <= MAX_RESIDENT_PAGES, + "3 000-row fixture: peak cache len {peak_medium} > {MAX_RESIDENT_PAGES}", + ); + assert!( + peak_large <= MAX_RESIDENT_PAGES, + "30 000-row fixture: peak cache len {peak_large} > {MAX_RESIDENT_PAGES} — body col \ + write is no longer page-bounded; likely buffering whole column chunks", + ); + + // The headline MS-7 claim: peak DOES NOT grow proportionally + // with input size. Going from 3 000 to 30 000 rows multiplies + // total input pages by 10, but peak resident cache should + // stay essentially flat. Allow a 2-page slack for transients. + let growth = peak_large.saturating_sub(peak_medium); + assert!( + growth <= 2, + "peak grows with input size: medium={peak_medium}, large={peak_large} — 10× more \ + input pages produced {growth} more resident pages, body-col path is not page-bounded", + ); + } + + /// MS-7 cross-input precondition: the body-col page cache bound + /// holds only if every input emits body cols lexicographically + /// (Husky order). A misaligned input would force + /// `fill_page_cache_to_row` to drain every body col preceding the + /// requested one in storage order — those pages stay cached under + /// their own col_idx until that col's turn comes up in the union + /// iteration, growing the cache to a full column chunk per + /// misaligned col. `assert_inputs_in_husky_body_col_order` rejects + /// such inputs at merge entry. This test constructs an input with + /// body cols `[value, metric_type]` (alphabetical would be + /// `[metric_type, value]`) and verifies the merge bails before + /// phase 3 runs. + #[tokio::test] + async fn test_assert_inputs_in_husky_body_col_order_rejects_misaligned_input() { + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("metric_name", dict_type.clone(), false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("sorted_series", DataType::Binary, false), + // Body cols out of order — 'value' precedes 'metric_type' + // in storage, but alphabetical would be metric_type then + // value. + Field::new("value", DataType::Float64, false), + Field::new("metric_type", DataType::UInt8, false), + ])); + + let metric_keys: Vec = vec![0, 0, 0]; + let metric_values = StringArray::from(vec!["cpu.usage"]); + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(metric_values), + ) + .expect("test dict"), + ); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(vec![ + 1_700_000_003u64, + 1_700_000_002, + 1_700_000_001, + ])); + let series: Vec> = (0u64..3).map(|i| i.to_be_bytes().to_vec()).collect(); + let series_refs: Vec<&[u8]> = series.iter().map(|v| v.as_slice()).collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from(series_refs)); + let value: ArrayRef = Arc::new(Float64Array::from(vec![0.0, 1.0, 2.0])); + let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8, 0, 0])); + + let batch = RecordBatch::try_new( + schema, + vec![ + metric_name, + timestamp_secs, + sorted_series, + value, + metric_type, + ], + ) + .expect("test batch"); + let bytes = write_input_parquet_with_small_pages(std::slice::from_ref(&batch), 100); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let err = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect_err("must reject misaligned body cols"); + let err_str = format!("{err:#}"); + assert!( + err_str.contains("not in Husky") && err_str.contains("'value'"), + "expected Husky-order error mentioning 'value', got: {err_str}", + ); + } + + // ============================================================================ + // Heterogeneous-output regressions (Codex P2 batch on PR-6409) + // ============================================================================ + + /// Regression for "Use the full union schema when driving column + /// writes": two outputs with the **same field count** that drop + /// *different* all-null sort fields. The previous + /// `build_parent_union_schema` heuristic picked the first such + /// schema and used it to drive the column loop — silently + /// skipping any column it dropped, even if another output still + /// needed it. The fix drives iteration from `full_union_schema` + /// directly. + /// + /// Setup: two inputs declaring an extended sort schema + /// `metric_name|tag_a|tag_b|-timestamp_secs/V2`. Input A has only + /// `tag_a` populated, input B has only `tag_b` populated. Merge + /// with `num_outputs=2` so each input's rows land in its own + /// output. After the per-output optimiser, output 0's schema + /// drops `tag_b` (all-null) and output 1's schema drops `tag_a` + /// — same field count, different dropped fields. Both outputs + /// must still write their kept tag column. + #[tokio::test] + async fn test_heterogeneous_dropped_fields_drive_from_full_union_schema() { + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + // Schema for input A: metric_name, tag_a (non-null), tag_b + // absent. Schema for input B: metric_name, tag_b (non-null), + // tag_a absent. The merge engine's union schema makes both + // tag fields nullable in the combined view. + let mk_schema = |with_a: bool, with_b: bool| -> SchemaRef { + let mut fields = vec![ + Field::new("metric_name", dict_type.clone(), false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("sorted_series", DataType::Binary, false), + ]; + // Body cols in lexicographic order. + if with_a { + fields.push(Field::new("tag_a", DataType::Utf8, false)); + } + if with_b { + fields.push(Field::new("tag_b", DataType::Utf8, false)); + } + fields.push(Field::new("value", DataType::Float64, false)); + Arc::new(ArrowSchema::new(fields)) + }; + + let make_batch = |metric: &str, + schema: SchemaRef, + tag_a_val: Option<&str>, + tag_b_val: Option<&str>, + row_count: usize, + base_series: u64| + -> RecordBatch { + let metric_keys: Vec = vec![0; row_count]; + let metric_values = StringArray::from(vec![metric]); + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(metric_values), + ) + .expect("dict"), + ); + let timestamps: Vec = (0..row_count as u64) + .map(|i| 1_700_000_000 + (row_count as u64 - i)) + .collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let series: Vec> = (0..row_count as u64) + .map(|i| (base_series + i).to_be_bytes().to_vec()) + .collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from( + series.iter().map(|v| v.as_slice()).collect::>(), + )); + let value: ArrayRef = Arc::new(Float64Array::from( + (0..row_count).map(|i| i as f64).collect::>(), + )); + let mut columns: Vec = vec![metric_name, timestamp_secs, sorted_series]; + if let Some(v) = tag_a_val { + columns.push(Arc::new(StringArray::from(vec![v; row_count])) as ArrayRef); + } + if let Some(v) = tag_b_val { + columns.push(Arc::new(StringArray::from(vec![v; row_count])) as ArrayRef); + } + columns.push(value); + RecordBatch::try_new(schema, columns).expect("batch") + }; + + // Input A has tag_a populated, sort key "aaa.metric" (sorts + // before "zzz.metric"). Input B has tag_b populated, sort key + // "zzz.metric". With num_outputs=2 the merge splits at the + // metric boundary so each input's rows land in its own output. + let schema_a = mk_schema(true, false); + let schema_b = mk_schema(false, true); + let batch_a = make_batch("aaa.metric", schema_a, Some("alpha"), None, 12, 0); + let batch_b = make_batch("zzz.metric", schema_b, None, Some("beta"), 12, 1000); + + let sort_fields_str = "metric_name|tag_a|tag_b|-timestamp_secs/V2"; + let make_input_bytes = |batch: &RecordBatch| -> Bytes { + let cfg = ParquetWriterConfig { + compression: Compression::Snappy, + ..ParquetWriterConfig::default() + }; + let kvs = vec![ + KeyValue::new( + PARQUET_META_SORT_FIELDS.to_string(), + sort_fields_str.to_string(), + ), + KeyValue::new( + PARQUET_META_WINDOW_START.to_string(), + "1700000000".to_string(), + ), + KeyValue::new(PARQUET_META_WINDOW_DURATION.to_string(), "60".to_string()), + KeyValue::new(PARQUET_META_NUM_MERGE_OPS.to_string(), "0".to_string()), + ]; + let props: WriterProperties = cfg.to_writer_properties_with_metadata( + &batch.schema(), + Vec::new(), + Some(kvs), + &[ + "metric_name".to_string(), + "tag_a".to_string(), + "tag_b".to_string(), + "timestamp_secs".to_string(), + ], + ); + let mut buf: Vec = Vec::new(); + let mut writer = + ArrowWriter::try_new(&mut buf, batch.schema(), Some(props)).expect("arrow writer"); + writer.write(batch).expect("write"); + writer.close().expect("close"); + Bytes::from(buf) + }; + let bytes_a = make_input_bytes(&batch_a); + let bytes_b = make_input_bytes(&batch_b); + + let inputs: Vec> = + vec![open_stream(bytes_a).await, open_stream(bytes_b).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(2)) + .await + .expect("merge with heterogeneous dropped fields"); + assert_eq!(outputs.len(), 2, "two metric names → two outputs"); + + // The output whose rows came from input A must carry tag_a + // values; the output whose rows came from input B must carry + // tag_b values. Before the fix, one of them was silently + // missing its kept tag column because the parent driver + // skipped it. + let mut saw_alpha = false; + let mut saw_beta = false; + for out in &outputs { + let merged = read_output_to_record_batch(&out.path); + let schema = merged.schema(); + if let Ok(tag_a_idx) = schema.index_of("tag_a") { + let col = merged.column(tag_a_idx); + for i in 0..col.len() { + if col.is_valid(i) { + let s = render_cell(col.as_ref(), i); + if s.ends_with("alpha") { + saw_alpha = true; + } + } + } + } + if let Ok(tag_b_idx) = schema.index_of("tag_b") { + let col = merged.column(tag_b_idx); + for i in 0..col.len() { + if col.is_valid(i) { + let s = render_cell(col.as_ref(), i); + if s.ends_with("beta") { + saw_beta = true; + } + } + } + } + } + assert!( + saw_alpha, + "expected to find tag_a='alpha' in some output — the full-union-schema-driven column \ + loop must visit tag_a even when another output dropped it", + ); + assert!( + saw_beta, + "expected to find tag_b='beta' in some output — the full-union-schema-driven column \ + loop must visit tag_b even when another output dropped it", + ); + } + + /// Regression for "Preserve service tags when service is a sort + /// column". If the sort schema places `service` in the sort key + /// (e.g., `metric_name|service|...`), the streaming engine writes + /// it via the sort-col path and the body-col `track_service` + /// branch never runs — so `MergeOutputFile.low_cardinality_tags` + /// historically came back empty even though every row in the + /// output has a service value. The fix folds in service names + /// from the per-output sort batch at finalize time. + #[tokio::test] + async fn test_service_as_sort_column_still_populates_low_cardinality_tags() { + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(vec![ + // Sort cols in sort schema order: metric_name, service, + // timestamp_secs (timestamp comes last per the sort + // validator's requirement). + Field::new("metric_name", dict_type.clone(), false), + Field::new("service", DataType::Utf8, false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("sorted_series", DataType::Binary, false), + Field::new("value", DataType::Float64, false), + ])); + + let row_count = 30usize; + let metric_keys: Vec = vec![0; row_count]; + let metric_values = StringArray::from(vec!["cpu.usage"]); + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(metric_values), + ) + .expect("dict"), + ); + // service grouped in ASC order so the input is genuinely + // sorted by (metric ASC, service ASC, timestamp DESC) and the + // engine's MC-3 sort verifier accepts it. + let mut services_sorted: Vec<&str> = Vec::with_capacity(row_count); + for s in ["api", "cache", "db"] { + for _ in 0..(row_count / 3) { + services_sorted.push(s); + } + } + let service: ArrayRef = Arc::new(StringArray::from(services_sorted)); + let timestamps: Vec = (0..row_count as u64) + .map(|i| 1_700_000_000 + (row_count as u64 - (i % 10))) + .collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let series_bytes: Vec> = (0..row_count as u64) + .map(|i| i.to_be_bytes().to_vec()) + .collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from( + series_bytes + .iter() + .map(|v| v.as_slice()) + .collect::>(), + )); + let value: ArrayRef = Arc::new(Float64Array::from( + (0..row_count).map(|i| i as f64).collect::>(), + )); + + let batch = RecordBatch::try_new( + schema.clone(), + vec![metric_name, service, timestamp_secs, sorted_series, value], + ) + .expect("batch"); + + let sort_fields_str = "metric_name|service|-timestamp_secs/V2"; + let cfg = ParquetWriterConfig { + compression: Compression::Snappy, + ..ParquetWriterConfig::default() + }; + let kvs = vec![ + KeyValue::new( + PARQUET_META_SORT_FIELDS.to_string(), + sort_fields_str.to_string(), + ), + KeyValue::new( + PARQUET_META_WINDOW_START.to_string(), + "1700000000".to_string(), + ), + KeyValue::new(PARQUET_META_WINDOW_DURATION.to_string(), "60".to_string()), + KeyValue::new(PARQUET_META_NUM_MERGE_OPS.to_string(), "0".to_string()), + ]; + let props: WriterProperties = cfg.to_writer_properties_with_metadata( + &schema, + Vec::new(), + Some(kvs), + &[ + "metric_name".to_string(), + "service".to_string(), + "timestamp_secs".to_string(), + ], + ); + let mut buf: Vec = Vec::new(); + let mut writer = + ArrowWriter::try_new(&mut buf, schema.clone(), Some(props)).expect("writer"); + writer.write(&batch).expect("write"); + writer.close().expect("close"); + + let inputs: Vec> = vec![open_stream(Bytes::from(buf)).await]; + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge"); + assert_eq!(outputs.len(), 1); + let svc_tags = outputs[0].low_cardinality_tags.get(TAG_SERVICE).expect( + "MergeOutputFile.low_cardinality_tags must contain TAG_SERVICE even when service is a \ + sort column", + ); + let mut got: Vec = svc_tags.iter().cloned().collect(); + got.sort(); + assert_eq!( + got, + vec!["api".to_string(), "cache".to_string(), "db".to_string()], + "service-name set must cover every distinct value in the sort col", + ); + } +} diff --git a/quickwit/quickwit-parquet-engine/src/merge/writer.rs b/quickwit/quickwit-parquet-engine/src/merge/writer.rs index 47f1dcb65f1..340ac93b0b4 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/writer.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/writer.rs @@ -185,7 +185,7 @@ pub fn write_merge_outputs( /// /// Takes the relevant row ranges from each input according to the merge runs, /// concatenates into a single batch, and applies the permutation via `take`. -fn apply_merge_permutation( +pub(super) fn apply_merge_permutation( inputs: &[RecordBatch], union_schema: &SchemaRef, runs: &[MergeRun], @@ -254,7 +254,7 @@ fn predict_num_row_groups(num_rows: usize, row_group_size: usize) -> usize { /// `qh.rg_partition_prefix_len` KV — caller computes this based on /// whether the file is going to be single-RG (preserve input prefix) /// or multi-RG (must be 0). -fn build_merge_kv_metadata( +pub(super) fn build_merge_kv_metadata( input_meta: &InputMetadata, row_keys_proto: &Option>, zonemap_regexes: &std::collections::HashMap, @@ -324,7 +324,10 @@ fn build_merge_kv_metadata( } /// Build `SortingColumn` entries for Parquet file metadata. -fn build_sorting_columns(batch: &RecordBatch, sort_fields_str: &str) -> Result> { +pub(super) fn build_sorting_columns( + batch: &RecordBatch, + sort_fields_str: &str, +) -> Result> { let sort_schema = parse_sort_fields(sort_fields_str)?; let schema = batch.schema(); @@ -347,7 +350,7 @@ fn build_sorting_columns(batch: &RecordBatch, sort_fields_str: &str) -> Result Result> { +pub(super) fn resolve_sort_field_names(sort_fields_str: &str) -> Result> { let sort_schema = parse_sort_fields(sort_fields_str)?; Ok(sort_schema .column @@ -361,7 +364,7 @@ fn resolve_sort_field_names(sort_fields_str: &str) -> Result> { /// /// Checks that sorted_series values are non-decreasing, and within equal /// sorted_series values, timestamp_secs respects the schema's sort direction. -fn verify_sort_order(batch: &RecordBatch, sort_fields_str: &str) { +pub(super) fn verify_sort_order(batch: &RecordBatch, sort_fields_str: &str) { if batch.num_rows() <= 1 { return; } diff --git a/quickwit/quickwit-parquet-engine/src/storage/page_decoder.rs b/quickwit/quickwit-parquet-engine/src/storage/page_decoder.rs index f6373d445f9..e8c16a93d9d 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/page_decoder.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/page_decoder.rs @@ -165,6 +165,17 @@ pub struct DecodedPage { /// cached dictionary page per (rg, col) for dictionary-encoded columns. /// Does not buffer the row group. /// +/// The decoder accepts the underlying stream either by mutable borrow +/// (via [`Self::new`]) or by ownership (via [`Self::from_owned`]). +/// Use the owned form when the decoder must persist across multiple +/// callers — its per-(rg, col) state (`rows_decoded`, cached pages, +/// `ColumnReader` instances) MUST live as long as the column is being +/// consumed; recreating a decoder mid-column discards the +/// `rows_decoded` counter (so subsequent pages report `row_start = 0`) +/// and the cached dictionary page (so subsequent data pages can't be +/// decoded). See the streaming merge engine for an example that holds +/// the decoder across phases 0 → 3. +/// /// # Single-consumer invariant /// /// [`Self::decode_next_page`] takes `&mut self`, so only one task ever @@ -184,12 +195,38 @@ pub struct DecodedPage { /// /// [`PageReader`]: parquet::column::page::PageReader pub struct StreamDecoder<'a> { - stream: &'a mut dyn ColumnPageStream, + stream: StreamSource<'a>, metadata: Arc, columns: HashMap<(usize, usize), ColumnState>, eof: bool, } +/// Holds the underlying [`ColumnPageStream`] either by mutable borrow +/// or by ownership. The borrowed form is the original short-lived +/// pattern used by unit tests; the owned form lets a [`StreamDecoder`] +/// outlive its construction scope (required to preserve per-column +/// state across multi-page / multi-consumer passes). +enum StreamSource<'a> { + Borrowed(&'a mut dyn ColumnPageStream), + Owned(Box), +} + +impl StreamSource<'_> { + fn as_mut(&mut self) -> &mut dyn ColumnPageStream { + match self { + StreamSource::Borrowed(s) => *s, + StreamSource::Owned(b) => b.as_mut(), + } + } + + fn metadata(&self) -> &Arc { + match self { + StreamSource::Borrowed(s) => s.metadata(), + StreamSource::Owned(b) => b.metadata(), + } + } +} + /// Per-(rg, col) state. Holds the [`ColumnReader`] that owns the /// page-decoder pipeline, plus a handle to the `PageQueue` we push /// converted pages into. The same `Arc>` queue lives both @@ -207,7 +244,7 @@ impl<'a> StreamDecoder<'a> { pub fn new(stream: &'a mut dyn ColumnPageStream) -> Self { let metadata = Arc::clone(stream.metadata()); Self { - stream, + stream: StreamSource::Borrowed(stream), metadata, columns: HashMap::new(), eof: false, @@ -244,7 +281,7 @@ impl<'a> StreamDecoder<'a> { return Ok(None); } - match self.stream.next_page().await? { + match self.stream.as_mut().next_page().await? { Some(page) => self.route_page_to_queue(page)?, None => { self.eof = true; @@ -322,7 +359,7 @@ impl<'a> StreamDecoder<'a> { }; if needs_lookahead && !self.eof { - match self.stream.next_page().await? { + match self.stream.as_mut().next_page().await? { Some(page) => self.route_page_to_queue(page)?, None => self.eof = true, } @@ -379,6 +416,24 @@ impl<'a> StreamDecoder<'a> { } } +impl StreamDecoder<'static> { + /// Build a decoder that owns its stream. Use this when the decoder + /// must outlive the scope that constructed the stream — e.g., when + /// the same decoder must be reused across multiple traversals of the + /// same column chunk, since reconstructing it would reset the + /// per-column `rows_decoded` counter and lose any cached dictionary + /// page. + pub fn from_owned(stream: Box) -> Self { + let metadata = Arc::clone(stream.metadata()); + Self { + stream: StreamSource::Owned(stream), + metadata, + columns: HashMap::new(), + eof: false, + } + } +} + // -------- Per-(rg, col) initialisation -------- fn init_column_state( From 1e627a865a2338c27c5cd501eeb6da6189b6c084 Mon Sep 17 00:00:00 2001 From: George Talbot Date: Mon, 18 May 2026 09:18:19 -0400 Subject: [PATCH 05/10] feat(streaming-merge): per-region engine + multi-output sorted_series splitting (#6424) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * feat: per-merge-region streaming engine — multi-RG inputs + outputs (PR-6c.2) Restructures PR-6b.2's flat phase 0 → phase 3 into a per-merge-region loop. Unlocks multi-RG metric-aligned input support and produces multi-RG output naturally — one output row group per merge region (typically one per metric_name when `rg_partition_prefix_len == 1`). Sort-prefix alignment (`prefix_len >= 1`) guarantees that any merge region has AT MOST one row group per input. That single invariant unlocks the restructure: 1. Pre-compute regions from RG metadata. For `prefix_len >= 1`, read each RG's metric_name min stat (must equal max — verifies metric-alignment). Group RGs across inputs by prefix_key. Sort regions by prefix_key. For `prefix_len == 0` (single-RG inputs only, validated earlier), one region covers everything. 2. Assign regions to output files by cumulative row count. Caller's `num_outputs` preserved as the upper bound. Each output file gets a contiguous slice of the region list, so output files have non-overlapping key ranges. 3. Per-region processing: for each region, advance contributing inputs' decoders through their RGs (drain sort cols of that RG, then stream body cols via the existing page-bounded BodyColOutputPageAssembler). Each region becomes one output RG in the current writer; when the assignment moves to a new output file, close the previous writer and open a new one. The streaming body-col mechanism from PR-6b.2 (arrow::compute:: interleave + handle.block_on driven decoder) is unchanged; it just runs over smaller row ranges (one region instead of one whole output). PR-6b.2's check that rejected any multi-RG input is replaced with: reject only `prefix_len == 0` AND multi-RG (those still need PR-5's LegacyMultiRGAdapter). Multi-RG metric-aligned inputs are now accepted natively. PR-6b.2 optimised the per-output schema based on per-output sort col data (drop all-null cols, re-dict-encode low-cardinality strings). With per-region streaming we don't know each region's content until we drain it, so PR-6c.2 declares the writer's schema as the full union schema and leaves output strings as Utf8. Per-output dict re-encoding can be reintroduced later by tracking cardinality during the streaming pass. - All 9 PR-6b.2 tests still pass (single-RG input regression — behaviour preserved). - New test_multi_rg_metric_aligned_input_produces_multi_rg_output: feeds a 2-RG metric-aligned input (prefix_len = 1, RG 0 = cpu.usage, RG 1 = memory.used); the streaming engine accepts it and produces a 2-RG output (one RG per metric_name region). - Renamed test_multi_rg_input_rejected → test_legacy_multi_rg_input_rejected to reflect the new rejection scope (only prefix_len == 0 multi-RG is rejected; metric-aligned is accepted). 10/10 streaming tests pass. Clippy, doc, machete, fmt all clean. 1. File-size cap with sort-key-boundary splits. 2. Per-output schema optimisation (track region body-col cardinality during the streaming pass). 3. Mid-region splits at sorted_series transitions for finer-grained M:N control when callers want more outputs than regions. Co-Authored-By: Claude Opus 4.7 (1M context) * fix(streaming): reject duplicate-prefix RGs + use escape encoding Two P1 bugs flagged by Codex on PR-6c.2 (#6410): 1. **Duplicate input row groups silently dropped.** When one input contained two RGs with the same composite prefix key, `process_region` overwrote `sort_col_batches[input_idx]` while `Region::total_rows` still counted both — losing rows and misaligning the body-col / sort-col mapping. Now enforce at-most-one-RG-per-input-per-prefix as a strong invariant at three sites: the merge read path (`extract_regions_from_metadata`), the streaming merge output finalize, and the indexing writer (`ParquetWriter::write_to_bytes` / `write_to_file_with_metadata`). The new `assert_unique_rg_prefix_keys` helper is shared. 2. **Byte-array prefix encoding broke lex order across lengths.** The 4-byte length prefix made `"b"` sort before `"aa"`, violating the declared ASC order. Switched to byte-stuffed escape encoding (`0x00` → `0x00 0x01`, terminator `0x00 0x00`), which preserves single-column lex order AND retains unambiguous concatenation for composite keys (the terminator is the smallest 2-byte sequence under escaping, so shorter values still sort before longer ones with the same prefix). Tests: - `test_byte_array_prefix_preserves_lex_order_across_lengths` — `"aa" < "b"`, empty < non-empty, shared-prefix shorter < longer, null-byte escaping preserves order. - `test_streaming_merge_rejects_duplicate_prefix_rgs_in_one_input` — end-to-end bail with clear error. - `test_write_to_bytes_rejects_duplicate_rg_prefix_when_claimed_aligned` + the `write_to_file` and single-RG positive counterparts. Co-Authored-By: Claude Opus 4.7 (1M context) * feat(streaming): split regions at sorted_series for prefix_len=0 multi-output When inputs declare rg_partition_prefix_len = 0 (legacy single-RG) and the caller asks for num_outputs > 1, the engine subdivides the single region at sorted_series transitions in the merge order so it can honor the output count. A single sorted_series run is never broken; if one run exceeds the remaining budget the whole run lands in one output anyway. The output inherits the input's rg_partition_prefix_len (=0) — the engine does not synthesize a prefix it can't unconditionally guarantee. Also handles the giant-single-metric case (prefix_len=0, one metric_name, num_outputs > 1): sorted_series transitions still split the merge order even though there are no metric_name transitions to drive a prefix synthesis. Implementation: - New `split_region_at_sorted_series` in region_grouping: walks the merge order and splits at sorted_series transitions when accumulated rows reach the target budget. - Main engine loop: when num_outputs > current_output_idx + 1 AND region's rows exceed the remaining budget, drain sort cols for the region, compute merge order, call split_region_at_sorted_series, process sub-regions. - Per-col page cache + cursor keyed by col_idx so the body-col path can read pages once and re-use them across sub-regions within the same top-level region. Resets between top-level regions (different RGs). Co-Authored-By: Claude Opus 4.7 (1M context) * docs(streaming): correct 'crash' → 'bail' in MS-2 doc comments The MS-2 validation path returns `Err` via `bail!()` (anyhow), not a panic / abort. Five doc-comment / inline-comment sites described the failure as "the engine would crash mid-merge" — overstated. Callers get a `Result::Err` propagated up the spawn_blocking task and the `streaming_merge_sorted_parquet_files` return. Sites updated: - `region_grouping.rs` module doc. - `validate_region_order_matches_physical_rg_order` doc. - streaming.rs MS-2 validation call-site comment. - Test docstrings for `test_streaming_merge_with_desc_prefix_col` and `test_ms2_region_order_disagrees_with_physical_rg_order_rejected`. No behaviour change. 477 lib tests pass; clippy + nightly fmt clean. Co-Authored-By: Claude Opus 4.7 (1M context) * docs(streaming): fix wrong adapter type name + explain rejection intent Two sites referenced a non-existent `LegacyMultiRGAdapter` — the actual type, introduced in PR-5 (#6408), is `LegacyInputAdapter` in `storage::legacy_adapter`. Fixed both references. Also expanded the rejection-block comment to make the *intent* of the guard explicit: it catches caller bugs (wiring a raw legacy multi-RG `StreamingParquetReader` straight into the streaming merge), not a degraded-input fallback. Production code routes legacy splits through `merge::execute_merge_operation` which wraps them in `LegacyInputAdapter` first. No behaviour change. Targeted test passes. Co-Authored-By: Claude Opus 4.7 (1M context) * fix(body_assembler): tighten output-iter termination + assert invariant adamtobey nit on PR #6424: `rows_emitted >= expected_rows` accepts `emitted > expected` as a normal termination condition, which would actually be a real accounting bug. The math rules `>` out by construction — `page_size = remaining.min(OUTPUT_PAGE_ROWS)` where `remaining = expected_rows - rows_emitted`, so each `rows_emitted += page_size` keeps `rows_emitted ≤ expected_rows`. Two changes: - Termination becomes `rows_emitted == expected_rows` so we don't silently accept an overshoot. - `debug_assert!(rows_emitted <= expected_rows, …)` at the top of `next()` documents the invariant and surfaces a regression loudly (panic in debug + tests) instead of silently terminating one iteration too late. No behaviour change in the happy path; bugs that would have produced `>` now fail tests instead of producing wrong output. 477 lib tests pass; clippy + nightly fmt clean. Co-Authored-By: Claude Opus 4.7 (1M context) * fix(streaming): recompute split budget across the output-rollover boundary Codex P1 finding on PR #6424: when a top-level region exactly fills the current output (so `remaining_in_current == 0`) and the next prefix-aligned region needs splitting, the split's first-sub-region budget was the stale zero remainder of the about-to-be-finalized output. `split_region_at_sorted_series` therefore cut after the first sorted_series run, producing a tiny leftover plus a large continuation that both inherited the parent region's prefix key. The sub-region loop then rolled over to a fresh output and wrote both pieces there, tripping the PA-3 duplicate-prefix-RG check in `finalize_output`. Fix: detect the rollover at decision time and compute `effective_first_target` / `effective_outputs_remaining` against the *next* output's empty budget. With the fix, the example above just chooses `needs_split = false` (region fits the fresh output's full target), processes the region whole, and rolls over cleanly. Regression test `test_region_exactly_fills_output_does_not_split_next_aligned_region` exercises the exact scenario Codex described: three 50-row RGs with distinct (metric, service) prefixes, `num_outputs = 3`, target = 50. Pre-fix, the merge bailed with PA-3 on output 1; post-fix, three clean outputs each with one unique prefix key. Verified by reading each output's parquet metadata back through `assert_unique_rg_prefix_keys`. 478 lib tests pass (477 prior + 1 new); clippy + nightly fmt clean. Co-Authored-By: Claude Opus 4.7 (1M context) * fix(streaming): reject null-mixed + all-null prefix RGs Codex P1 on PR #6424: `extract_aligned_prefix_value` decided prefix alignment purely from `min` / `max` statistics. Parquet records those over non-null cells only, with `null_count` reported separately, so two real failure modes slipped through: 1. **Mixed null + non-null.** A row group with `N` nulls plus a single non-null cell `"x"` reports `min == max == "x"` and the `min == max` check silently accepted it — but two distinct prefix keys (null and `"x"`) lived in that RG, breaking the at-most-one-prefix-value-per-RG invariant (PA-1). 2. **All-null RG.** Parquet records no `min` / `max` for an all- null chunk, so the legacy check bailed with the misleading "no min in stats" error. Logically the RG carries one prefix value (null) and is aligned — but supporting it cleanly requires a null marker in the composite-key encoding that agrees with SS-2's "nulls last" rule. `encode_byte_array_prefix(&[])` puts nulls *first*; coordinating that with SS-2 is a follow-up. Fix: read `null_count_opt()` from stats and `num_values()` from the column-chunk metadata. Bail explicitly in both cases — mixed with a PA-1 message naming the (nulls, non-null) split, all-null with a clear "not yet supported" pointer. Two regression tests in `streaming.rs::tests`: - `test_mixed_null_and_value_prefix_rg_rejected`: 1 RG, 3 cells `"cpu.usage"` + 1 null. Asserts PA-1 bail. - `test_all_null_prefix_rg_rejected`: 1 RG, 3 nulls. Asserts the "all-null … not yet supported" bail. 480 lib tests pass (+2 new); workspace clippy + nightly fmt clean. Co-Authored-By: Claude Opus 4.7 (1M context) * refactor(streaming): share storekey prefix encoding with sorted_series The per-RG composite prefix key now uses the same storekey-based encoding as `sorted_series` — same `(ordinal, value)` layout, same direction-inversion, same null-skip pattern — so a per-RG prefix key is a literal byte prefix of every `sorted_series` value emitted by rows in that RG. Why: the prior byte-stuffed escape encoding had no in-line way to represent an all-null prefix RG (an empty marker would lex-sort before any present-value key, conflicting with SS-2 nulls-last). With the shared encoding, an all-null column is skipped entirely and the next column's higher ordinal byte appears in its place, giving nulls-last ordering for free — the same trick already proven in `sorted_series::encode_row_key`. Per-column logic now goes through one helper: `crate::sorted_series::append_prefix_col_to_key(buf, ord, val, desc)` shared between `sorted_series` (per-row keys) and `merge::streaming::region_grouping` (per-RG keys). It writes `storekey(ord) || storekey(val)` and inverts only the value bytes for DESC columns. `sorted_series::encode_row_key` was refactored to call the helper; the open-coded inline encoding is gone. Trailing **prefix-length sentinel**: each per-RG key ends with a `u8(prefix_len)` ordinal byte. This handles the prefix_len=1 edge case where an all-null RG's empty body would otherwise lex-sort *before* any non-null RG — with the sentinel, the all-null key becomes `[prefix_len]` and non-null keys still start with `ord(0)` (< prefix_len), so non-null sorts first. The sentinel is also what `sorted_series` writes immediately after the prefix cols, so the literal-prefix property is preserved. Null handling in `extract_rg_composite_prefix_key`: - **All-null RG**: column skipped, RG groups into its own region (after non-null regions). - **Mixed null + non-null**: rejected as a PA-1 violation (rows in the same RG would encode to two distinct prefix keys; producer is supposed to start a new RG at the null/non-null transition). - **Otherwise**: standard `min == max` check, then the type-dispatched storekey encoding via the helper. Removed: - `extract_aligned_prefix_value` (replaced by `encode_prefix_col_value` which calls the helper). - `encode_byte_array_prefix` (byte-stuffed escape, no longer used). - `invert_for_descending` (the helper handles inversion per-column). - `test_invert_for_descending_reverses_lex_order` and `test_byte_array_prefix_preserves_lex_order_across_lengths` (byte-level tests of the removed encoding; semantic properties remain enforced by `storekey`'s own tests plus the higher-level prefix tests). Replaced `test_all_null_prefix_rg_rejected` with `test_all_null_prefix_rg_groups_into_separate_region_sorted_last`: builds two inputs (one with `metric_name = "cpu.usage"`, one with `metric_name = NULL`) and verifies the merged output has two RGs with the all-null region in RG 1 (sorted after the non-null region) — pinning the nulls-last ordering that the sentinel encoding produces. Updated `test_extract_rg_composite_prefix_key_two_byte_array_cols` for the new byte layout (`storekey(ord) || storekey(val)` per col plus the trailing sentinel byte). `PrefixColumn` gains an `ordinal: u8` field, populated from each column's position in `qh.sort_fields` so it matches the ordinal `sorted_series` would assign. 478 lib tests pass; workspace clippy + nightly fmt clean. Co-Authored-By: Claude Opus 4.7 (1M context) * style(streaming): re-fmt to latest nightly rustfmt CI's nightly rustfmt (1.9.0-nightly 2026-05-17) wrapped a handful of comment / bail!-message / where-clause / vec! literal lines slightly differently than my local nightly at commit time (1.9.0-nightly 2026-05-11). Re-formatting all three affected files catches the drift in this commit so CI Lints stops complaining; local nightly is now updated to match CI. No behaviour change. 478 lib tests still pass on the slice. Co-Authored-By: Claude Opus 4.7 (1M context) * fix(merge): preserve metastore rg_partition_prefix_len from writer's KV stamp The streaming merge engine produces sort-prefix-aligned multi-RG output and stamps `qh.rg_partition_prefix_len = input_meta.rg_partition_prefix_len` in the file's KV (verified by `assert_unique_rg_prefix_keys` before close). `merge_parquet_split_metadata` then ran after and unconditionally demoted to 0 whenever `output.num_row_groups > 1` — breaking CS-1 (metastore must mirror on-disk KV) for every multi-RG streaming-engine output. Aligned splits got tagged 0 in the metastore on every merge and leaked out of the prefix-aligned compaction bucket on the next pass. Carry the value the writer actually stamped via a new `MergeOutputFile.output_rg_partition_prefix_len` field, then propagate it as-is in metadata aggregation. Both engines populate the field: - Legacy `merge/writer.rs` reports its demoted value (row-count-driven RG boundaries can't honor prefix alignment, so it stamps 0 on multi-RG). - Streaming `merge/streaming/output.rs` reports the inputs' prefix unchanged (it splits at prefix transitions and the writer verifies). CS-1 holds by construction — same source of truth, no re-derivation. Tests: - `test_output_prefix_len_demoted_when_multi_rg` → renamed to `test_output_prefix_len_carries_writers_value_when_demoted`; now asserts that the metastore mirrors the writer's reported value. - New `test_output_prefix_len_preserved_on_multi_rg_streaming_engine` asserts that a multi-RG streaming output (writer reports prefix_len=2) keeps the prefix in the metastore — the regression case for F1. Co-Authored-By: Claude Opus 4.7 (1M context) --------- Co-authored-by: Claude Opus 4.7 (1M context) --- .../src/merge/metadata_aggregation.rs | 97 +- .../quickwit-parquet-engine/src/merge/mod.rs | 35 +- .../src/merge/streaming.rs | 3773 +++++++++-------- .../src/merge/streaming/body_assembler.rs | 494 +++ .../src/merge/streaming/output.rs | 365 ++ .../src/merge/streaming/region_grouping.rs | 802 ++++ .../src/merge/writer.rs | 1 + .../src/sorted_series/mod.rs | 62 +- .../src/storage/streaming_writer.rs | 10 + .../src/storage/writer.rs | 177 + 10 files changed, 4056 insertions(+), 1760 deletions(-) create mode 100644 quickwit/quickwit-parquet-engine/src/merge/streaming/body_assembler.rs create mode 100644 quickwit/quickwit-parquet-engine/src/merge/streaming/output.rs create mode 100644 quickwit/quickwit-parquet-engine/src/merge/streaming/region_grouping.rs diff --git a/quickwit/quickwit-parquet-engine/src/merge/metadata_aggregation.rs b/quickwit/quickwit-parquet-engine/src/merge/metadata_aggregation.rs index ddf254f0865..1da61905b88 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/metadata_aggregation.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/metadata_aggregation.rs @@ -119,22 +119,18 @@ pub fn merge_parquet_split_metadata( let split_id = ParquetSplitId::generate(first.kind); let parquet_file = format!("{split_id}.parquet"); - // `rg_partition_prefix_len` propagation rule: a single-row-group - // output vacuously satisfies any prefix claim (no boundary to - // misalign), so we keep the inputs' prefix. Multi-RG output with - // arbitrary row-count-driven boundaries (the only kind the current - // merge writer can produce) cannot honor a non-zero claim and must - // reset to 0. PR-6 (streaming column-major merge engine) will - // produce sort-prefix-aligned multi-RG output and propagate the - // prefix unconditionally. - // - // This must agree with the value the writer embeds in the file's - // `qh.rg_partition_prefix_len` KV — see `write_merge_outputs`. - let output_prefix_len = if output.num_row_groups <= 1 { - first.rg_partition_prefix_len - } else { - 0 - }; + // CS-1: the metastore-recorded `rg_partition_prefix_len` must equal + // the value the writer embedded in the file's + // `qh.rg_partition_prefix_len` KV. Each writer makes its own + // decision (the legacy `merge/writer.rs` demotes to 0 on multi-RG + // output because its boundaries are row-count-driven; the streaming + // writer propagates the inputs' prefix unchanged because it splits + // at prefix transitions and verifies via `assert_unique_rg_prefix_keys`) + // and reports it via `MergeOutputFile.output_rg_partition_prefix_len`. + // We propagate that one source of truth — re-deriving here from + // `num_row_groups` would silently diverge from the streaming + // engine's prefix-aligned multi-RG output. + let output_prefix_len = output.output_rg_partition_prefix_len; // Data-dependent fields come from the MergeOutputFile (extracted from // this output's actual rows during the merge write pass). @@ -212,11 +208,30 @@ mod tests { num_row_groups: usize, time_range: (u64, u64), metric_names: &[&str], + ) -> MergeOutputFile { + make_output_full_with_prefix( + num_rows, + size_bytes, + num_row_groups, + 0, + time_range, + metric_names, + ) + } + + fn make_output_full_with_prefix( + num_rows: usize, + size_bytes: u64, + num_row_groups: usize, + output_rg_partition_prefix_len: u32, + time_range: (u64, u64), + metric_names: &[&str], ) -> MergeOutputFile { MergeOutputFile { path: PathBuf::from("/tmp/merged.parquet"), num_rows, num_row_groups, + output_rg_partition_prefix_len, size_bytes, row_keys_proto: Some(vec![0x08, 0x01]), zonemap_regexes: HashMap::from([("metric_name".to_string(), "cpu\\..*".to_string())]), @@ -412,17 +427,21 @@ mod tests { } #[test] - fn test_output_prefix_len_demoted_when_multi_rg() { - // The current merge writer rolls over RGs at row count, not at - // sort-prefix transitions. When the output ends up with > 1 RG, - // the boundaries are at arbitrary places and the inputs' prefix - // claim cannot be honored — the output's prefix must be 0. + fn test_output_prefix_len_carries_writers_value_when_demoted() { + // CS-1: the metastore-recorded value must match the writer's + // KV stamp. Legacy `merge/writer.rs` demotes to 0 when its + // row-count-driven RG layout produces multi-RG output and + // reports that demoted value on the `MergeOutputFile`. The + // metastore aggregator must propagate it as-is (NOT re-derive + // from inputs) so the metastore agrees with the file's KV. let mut s0 = make_test_split("s0", (1000, 2000), 0); let mut s1 = make_test_split("s1", (1000, 2000), 0); s0.rg_partition_prefix_len = 3; s1.rg_partition_prefix_len = 3; - let output = make_output_full(200, 9000, 2, (1000, 2000), &["cpu.usage"]); + // num_row_groups = 2 + writer reports demoted prefix_len = 0 + // (the legacy writer's choice for a row-count-driven multi-RG). + let output = make_output_full_with_prefix(200, 9000, 2, 0, (1000, 2000), &["cpu.usage"]); let result = merge_parquet_split_metadata(&[s0, s1], &output).unwrap(); assert_eq!(result.rg_partition_prefix_len, 0); } @@ -430,20 +449,44 @@ mod tests { #[test] fn test_output_prefix_len_preserved_when_single_rg() { // A single-RG output vacuously satisfies any prefix alignment - // claim (one RG, no boundary to misalign). Propagate the inputs' - // prefix so the merge output stays in the same compaction bucket - // as the inputs, instead of leaking into the prefix=0 bucket on - // every merge. + // claim (one RG, no boundary to misalign). The writer reports + // the inputs' prefix; aggregator propagates it. let mut s0 = make_test_split("s0", (1000, 2000), 0); let mut s1 = make_test_split("s1", (1000, 2000), 0); s0.rg_partition_prefix_len = 3; s1.rg_partition_prefix_len = 3; - let output = make_output_full(200, 9000, 1, (1000, 2000), &["cpu.usage"]); + let output = make_output_full_with_prefix(200, 9000, 1, 3, (1000, 2000), &["cpu.usage"]); let result = merge_parquet_split_metadata(&[s0, s1], &output).unwrap(); assert_eq!(result.rg_partition_prefix_len, 3); } + #[test] + fn test_output_prefix_len_preserved_on_multi_rg_streaming_engine() { + // CS-1 regression for F1: the streaming engine produces + // sort-prefix-aligned multi-RG output and reports the inputs' + // prefix unchanged via `MergeOutputFile.output_rg_partition_prefix_len`. + // Before this fix, `merge_parquet_split_metadata` would + // unconditionally demote to 0 whenever `num_row_groups > 1`, + // breaking CS-1 (metastore disagreed with the file's KV) and + // leaking aligned outputs into the unaligned compaction bucket + // on every subsequent merge. + let mut s0 = make_test_split("s0", (1000, 2000), 0); + let mut s1 = make_test_split("s1", (1000, 2000), 0); + s0.rg_partition_prefix_len = 2; + s1.rg_partition_prefix_len = 2; + + // num_row_groups = 3 (multi-RG) AND writer reports prefix_len = 2 + // (the streaming engine's stamp because it verified alignment). + let output = make_output_full_with_prefix(300, 12000, 3, 2, (1000, 2000), &["cpu.usage"]); + let result = merge_parquet_split_metadata(&[s0, s1], &output).unwrap(); + assert_eq!( + result.rg_partition_prefix_len, 2, + "metastore must mirror the writer's KV (CS-1); multi-RG aligned output keeps its \ + prefix claim" + ); + } + #[test] fn test_fresh_split_id_generated() { let inputs = vec![ diff --git a/quickwit/quickwit-parquet-engine/src/merge/mod.rs b/quickwit/quickwit-parquet-engine/src/merge/mod.rs index fcc353992a9..7ab8d685f7a 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/mod.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/mod.rs @@ -65,19 +65,19 @@ pub struct MergeConfig { /// Metadata extracted from input files' Parquet KV metadata. /// All inputs must agree on sort_fields, window_start, window_duration, /// and rg_partition_prefix_len. -struct InputMetadata { +#[derive(Clone)] +pub(crate) struct InputMetadata { sort_fields: String, window_start_secs: Option, window_duration_secs: u32, num_merge_ops: u32, /// Number of leading sort columns whose transitions align with row /// group boundaries. All input files must agree on this value (it's - /// part of the compaction scope key). Splitting row groups at the - /// claimed prefix boundary is not implemented by the current merge - /// writer — it lands in PR-6 (streaming column-major merge engine). - /// Until then, the *output* file is written with prefix 0 regardless - /// of this value. - #[allow(dead_code)] // wired for PR-6 streaming engine; PR-1 only validates. + /// part of the compaction scope key). The streaming merge engine + /// (PR-6c.2) honours this on input AND produces prefix-aligned + /// output: when inputs have `prefix_len == 0`, the engine + /// synthesizes prefix-aligned regions from the merge order and + /// promotes the output's `rg_partition_prefix_len` accordingly. rg_partition_prefix_len: u32, } @@ -95,15 +95,22 @@ pub struct MergeOutputFile { /// Number of rows in this output file. pub num_rows: usize, - /// Number of row groups the writer produced for this file. Used by - /// `merge_parquet_split_metadata` to decide whether the input prefix - /// alignment claim (`rg_partition_prefix_len`) can be propagated to - /// the output: a single-RG file vacuously satisfies any claim, so - /// we keep the inputs' prefix; a multi-RG file with arbitrary - /// boundaries (the only kind the current writer can produce) must - /// reset the claim to 0. + /// Number of row groups the writer produced for this file. pub num_row_groups: usize, + /// `qh.rg_partition_prefix_len` value the writer embedded in this + /// file's KV metadata. The legacy `merge/writer.rs` writer demotes + /// to 0 when it produces multi-RG output (its RG boundaries are + /// row-count-driven, not prefix-aligned). The streaming writer + /// (`merge/streaming/output.rs`) propagates the inputs' prefix + /// unchanged because it splits at prefix transitions AND + /// `assert_unique_rg_prefix_keys` verifies the file. Carrying the + /// value here lets `merge_parquet_split_metadata` (CS-1: metastore + /// == KV) propagate it directly to `ParquetSplitMetadata` instead + /// of re-deriving — preventing the metastore from disagreeing with + /// the on-disk KV when both engines coexist. + pub output_rg_partition_prefix_len: u32, + /// File size in bytes. pub size_bytes: u64, diff --git a/quickwit/quickwit-parquet-engine/src/merge/streaming.rs b/quickwit/quickwit-parquet-engine/src/merge/streaming.rs index 19cf4c0405c..9e98736f04b 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/streaming.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/streaming.rs @@ -58,39 +58,43 @@ use std::collections::{HashMap, HashSet}; use std::ops::Range; -use std::path::{Path, PathBuf}; +use std::path::Path; use std::sync::Arc; -use anyhow::{Context, Result, anyhow, bail}; +use anyhow::{Context, Result, bail}; use arrow::array::{Array, ArrayRef, RecordBatch, new_null_array}; -use arrow::compute::interleave; -use arrow::datatypes::{DataType, Field, Schema as ArrowSchema, SchemaRef}; +use arrow::datatypes::{Field, Schema as ArrowSchema, SchemaRef}; use parquet::file::metadata::ParquetMetaData; use tokio::runtime::Handle; use tracing::info; -use ulid::Ulid; -use super::merge_order::{MergeRun, compute_merge_order, compute_output_boundaries}; -use super::schema::{align_inputs_to_union_schema, optimize_output_batch}; -use super::writer::{ - apply_merge_permutation, build_merge_kv_metadata, build_sorting_columns, - resolve_sort_field_names, verify_sort_order, -}; +use super::merge_order::{MergeRun, compute_merge_order}; +use super::schema::align_inputs_to_union_schema; +use super::writer::{apply_merge_permutation, verify_sort_order}; use super::{InputMetadata, MergeConfig, MergeOutputFile}; -use crate::row_keys; + +mod body_assembler; +mod output; +pub(crate) mod region_grouping; + +use body_assembler::{BodyColOutputPageAssembler, StreamingBodyColIter}; +use output::{ + OutputAccumulator, OutputWriterStorage, finalize_output, open_output_writer_for_streaming, +}; +use region_grouping::{ + Region, extract_regions_from_metadata, split_region_at_sorted_series, + validate_region_order_matches_physical_rg_order, +}; + use crate::sort_fields::{ equivalent_schemas_for_compaction, is_timestamp_column_name, parse_sort_fields, }; use crate::sorted_series::SORTED_SERIES_COLUMN; -use crate::split::TAG_SERVICE; use crate::storage::page_decoder::{DecodedPage, StreamDecoder}; -use crate::storage::split_writer::{extract_metric_names, extract_time_range}; -use crate::storage::streaming_writer::StreamingParquetWriter; use crate::storage::{ ColumnPageStream, PARQUET_META_NUM_MERGE_OPS, PARQUET_META_RG_PARTITION_PREFIX_LEN, PARQUET_META_SORT_FIELDS, PARQUET_META_WINDOW_DURATION, PARQUET_META_WINDOW_START, }; -use crate::zonemap::{self, ZonemapOptions}; /// Output page size in rows for body-col assembly. Each call to the /// sync iterator passed to [`write_next_column_arrays`] yields one @@ -101,7 +105,7 @@ use crate::zonemap::{self, ZonemapOptions}; /// fixed costs. /// /// [`write_next_column_arrays`]: crate::storage::streaming_writer::RowGroupBuilder::write_next_column_arrays -const OUTPUT_PAGE_ROWS: usize = 1024; +pub(crate) const OUTPUT_PAGE_ROWS: usize = 1024; /// Test-only peak observed length of any input's `body_col_page_cache` /// since the last reset. Used by the MS-7 page-bounded-memory test to @@ -113,7 +117,7 @@ pub(crate) static PEAK_BODY_COL_PAGE_CACHE_LEN: std::sync::atomic::AtomicUsize = std::sync::atomic::AtomicUsize::new(0); #[cfg(test)] -fn record_body_col_page_cache_len(len: usize) { +pub(crate) fn record_body_col_page_cache_len(len: usize) { use std::sync::atomic::Ordering; let mut prev = PEAK_BODY_COL_PAGE_CACHE_LEN.load(Ordering::Relaxed); while len > prev { @@ -130,7 +134,7 @@ fn record_body_col_page_cache_len(len: usize) { } #[cfg(not(test))] -fn record_body_col_page_cache_len(_len: usize) {} +pub(crate) fn record_body_col_page_cache_len(_len: usize) {} /// Streaming N-input → M-output column-major merge. /// @@ -151,22 +155,35 @@ pub async fn streaming_merge_sorted_parquet_files( bail!("num_outputs must be at least 1"); } - // Validate that all inputs are single-RG (or zero-RG, which means - // the file has no data). PR-6b.2 simplification — see module docs. - for (idx, stream) in inputs.iter().enumerate() { - let num_rgs = stream.metadata().num_row_groups(); - if num_rgs > 1 { - bail!( - "streaming merge requires single-row-group inputs in PR-6b.2 (input {idx} has \ - {num_rgs} row groups); multi-RG metric-aligned inputs land in a follow-up. \ - Legacy multi-RG (rg_partition_prefix_len=0) inputs must go through the PR-5 \ - adapter, which presents them as a single synthetic row group." - ); + let input_meta = extract_and_validate_input_metadata(&inputs)?; + + // Reject legacy multi-RG inputs (`rg_partition_prefix_len == 0` + // AND any input has >1 row group). These have no alignment claim, + // so RG boundaries are arbitrary row counts that may split a + // single sort-key value across two RGs. The streaming engine + // cannot determine merge regions without column-chunk-bounded + // buffering; such inputs must go through `LegacyInputAdapter` + // (from PR-5, see `storage::legacy_adapter`), which presents + // them as one synthetic single-RG stream. + // + // This guard catches caller bugs — production code always routes + // legacy splits through the adapter (see `merge::execute_merge_operation` + // in `merge/mod.rs`), so a raw legacy `StreamingParquetReader` + // arriving here is a wiring mistake, not a supported input shape. + // Bail with a clear pointer rather than wading further into the + // streaming pipeline with mis-aligned RGs. + if input_meta.rg_partition_prefix_len == 0 { + for (idx, stream) in inputs.iter().enumerate() { + let num_rgs = stream.metadata().num_row_groups(); + if num_rgs > 1 { + bail!( + "legacy multi-RG inputs (rg_partition_prefix_len=0) must go through the PR-5 \ + adapter — input {idx} has {num_rgs} row groups with no alignment claim" + ); + } } } - let input_meta = extract_and_validate_input_metadata(&inputs)?; - info!( num_inputs = inputs.len(), num_outputs = config.num_outputs, @@ -191,54 +208,203 @@ pub async fn streaming_merge_sorted_parquet_files( let mut decoders_state = build_input_decoders_state(&mut inputs)?; // Cross-input precondition for the body-col memory bound. See - // the function doc and the `body_col_page_cache` field doc for + // the function doc and the `body_col_page_caches` field doc for // the load-bearing argument. assert_inputs_in_husky_body_col_order(&decoders_state, &input_meta.sort_fields)?; - // Phase 0 - let sort_col_batches = - drain_sort_cols_all_inputs(&handle, &mut decoders_state, &input_meta.sort_fields)?; + // Pre-compute regions from RG metadata. With prefix_len >= 1 + // each region is one sort-prefix value across inputs (each + // contributing input has exactly one RG in that region). With + // prefix_len == 0 (validated single-RG above) there is one + // region covering all inputs — we'll subdivide it below by + // walking the merge order to find first-sort-col transitions. + let regions = extract_regions_from_metadata(&decoders_state, &input_meta)?; - if sort_col_batches.iter().all(|b| b.num_rows() == 0) { + if regions.is_empty() { info!("all inputs empty, producing no output"); return Ok(Vec::new()); } - // Phase 1: align inputs to a union sort-col schema so the merge-order - // comparator sees uniformly-typed `sorted_series` + `timestamp_secs`. - let (sort_union_schema, aligned_sort_batches) = - align_inputs_to_union_schema(&sort_col_batches, &input_meta.sort_fields)?; - let merge_order = compute_merge_order(&aligned_sort_batches, &input_meta.sort_fields)?; - - // Phase 2: split merge order into M outputs at sorted_series boundaries. - let boundaries = - compute_output_boundaries(&merge_order, &aligned_sort_batches, num_outputs)?; + // MS-2: validate that the BTreeMap-driven region order + // agrees with each input's physical RG order. The streaming + // engine reads inputs sequentially — it cannot rewind. If + // region K's contributing RG for input I is physically AFTER + // a later region L's contributing RG for the same input, the + // engine would bail mid-merge with "page from rg X while + // draining rg Y" (Err returned up the spawn_blocking task, + // not a panic). This typically means the input was sorted + // in the opposite direction from what the sort schema + // declares (e.g., metric_name written DESC on disk but the + // sort schema says ASC). Reject upfront with a clearer + // error that names the offending input and region. + validate_region_order_matches_physical_rg_order(®ions, decoders_state.len())?; + + let total_rows: usize = regions.iter().map(|r| r.total_rows()).sum(); + let target_per_output = (total_rows.div_ceil(num_outputs)).max(1); - let total_rows: usize = aligned_sort_batches.iter().map(|b| b.num_rows()).sum(); info!( total_rows, - num_outputs = boundaries.len(), - "streaming merge order computed" - ); - - // Pre-compute per-input row → (output_idx, output_position) destination map. - // Used by every column write to slice take/interleave indices per page. - let destinations = - build_input_row_destinations(&aligned_sort_batches, &merge_order, &boundaries); - - // Phase 3 - let outputs = write_streaming_outputs( - &handle, - &mut decoders_state, - &aligned_sort_batches, - &sort_union_schema, - &merge_order, - &boundaries, - &destinations, - &input_meta, - &writer_config, - &output_dir, - )?; + num_regions = regions.len(), + num_outputs, + "streaming merge regions computed" + ); + + // Build the union schema once across all inputs' arrow schemas. + let arrow_schemas: Vec = decoders_state + .iter() + .map(|s| Arc::clone(&s.arrow_schema)) + .collect(); + let union_schema = + build_full_union_schema_from_arrow_schemas(&arrow_schemas, &input_meta.sort_fields)?; + + // Region processing loop. For each top-level region we may + // need to subdivide it across multiple output files so we + // honor `num_outputs` even when the input layout doesn't + // give us enough region boundaries (e.g., one giant + // `metric_name` with prefix_len=0). Splitting happens at + // `sorted_series` transitions inside the region's merge + // order — never inside a single sorted_series run. + // + // Memory: between top-level regions we reset every input's + // per-col page cache + cursor, because pages from one RG + // would have row_start values that collide with the next + // RG's row-index space. Sub-regions of one top-level region + // share an RG, so the cache survives across sub-region + // boundaries — that's what lets the col write loop re-read + // an earlier col's pages for a later sub-region. + let mut outputs: Vec = Vec::new(); + let mut current_writer: Option = None; + let mut current_accumulator: Option = None; + let mut current_output_idx: usize = 0; + let mut current_output_rows: usize = 0; + + for (region_idx, region) in regions.iter().enumerate() { + if region_idx > 0 { + for state in decoders_state.iter_mut() { + state.reset_all_body_col_state(); + } + } + + // Decide whether we need to split this region across + // multiple outputs. Two conditions must both hold: the + // region's rows would push the current output past the + // target AND there is an unused output to advance to. + // When splitting kicks in we have to pre-drain this + // region's sort cols so we can compute the merge order + // and find sorted_series transitions; if no split is + // needed we let `process_region` drain internally to + // preserve the existing per-region memory bound. + // + // If the current output is already at-or-past target, + // the sub-region loop below will roll over to a fresh + // output BEFORE writing this region's first sub-region. + // The split decision must be made against that fresh + // output's full budget — not the current (zero) remainder + // — otherwise `split_region_at_sorted_series` would cut + // after the very first sorted_series run, and the small + // leftover plus the large continuation (both inheriting + // the region's prefix key) would land in the same new + // output, tripping the PA-3 duplicate-prefix-RG check in + // `finalize_output`. Same for `outputs_remaining`: after + // the rollover one fewer output is left to fill. + let outputs_remaining_raw = num_outputs - current_output_idx; + let remaining_in_current_raw = target_per_output.saturating_sub(current_output_rows); + let will_roll_over = current_writer.is_some() + && current_output_rows >= target_per_output + && current_output_idx + 1 < num_outputs; + let effective_first_target = if will_roll_over { + target_per_output + } else { + remaining_in_current_raw + }; + let effective_outputs_remaining = if will_roll_over { + outputs_remaining_raw.saturating_sub(1) + } else { + outputs_remaining_raw + }; + let needs_split = + effective_outputs_remaining > 1 && region.total_rows() > effective_first_target; + + let prefetched: Option> = if needs_split { + Some(drain_and_align_region( + &handle, + &mut decoders_state, + region, + &input_meta.sort_fields, + )?) + } else { + None + }; + + let sub_regions: Vec = if let Some(ref prefetched_batches) = prefetched { + let merge_order = compute_merge_order(prefetched_batches, &input_meta.sort_fields)?; + split_region_at_sorted_series( + region, + &merge_order, + prefetched_batches, + effective_first_target, + target_per_output, + effective_outputs_remaining, + )? + } else { + vec![region.clone()] + }; + + for sub_region in &sub_regions { + let sub_rows = sub_region.total_rows(); + + // Advance to the next output file BEFORE this sub- + // region if the current output already met the + // target and we still have outputs to fill. The + // very first sub-region is exempt so we don't open + // an empty writer. + if current_writer.is_some() + && current_output_rows >= target_per_output + && current_output_idx + 1 < num_outputs + { + if let (Some(w), Some(acc)) = + (current_writer.take(), current_accumulator.take()) + { + outputs.push(finalize_output(w, acc, &input_meta)?); + } + current_output_idx += 1; + current_output_rows = 0; + } + + if current_writer.is_none() { + let writer = open_output_writer_for_streaming( + current_output_idx, + &output_dir, + &union_schema, + &input_meta, + &writer_config, + )?; + current_writer = Some(writer); + current_accumulator = Some(OutputAccumulator::new(current_output_idx)); + } + + process_region( + &handle, + &mut decoders_state, + current_writer + .as_mut() + .expect("writer opened above for this sub-region"), + current_accumulator + .as_mut() + .expect("accumulator opened above for this sub-region"), + sub_region, + &union_schema, + &input_meta, + prefetched.as_deref(), + )?; + current_output_rows += sub_rows; + } + } + + // Close the last writer. + if let (Some(w), Some(acc)) = (current_writer.take(), current_accumulator.take()) { + outputs.push(finalize_output(w, acc, &input_meta)?); + } // MC-1: total row count preserved. let output_total: usize = outputs.iter().map(|o| o.num_rows).sum(); @@ -266,50 +432,106 @@ pub async fn streaming_merge_sorted_parquet_files( /// rows from more than one page). Reconstructing the decoder mid-pass /// would reset the per-column `rows_decoded` counter (so `row_start` /// becomes wrong) and discard cached dictionary / queued pages. -struct InputDecoderState { - decoder: StreamDecoder<'static>, - metadata: Arc, +pub(crate) struct InputDecoderState { + pub(crate) decoder: StreamDecoder<'static>, + pub(crate) metadata: Arc, /// Arrow schema of this input (from parquet → arrow conversion). - arrow_schema: SchemaRef, - /// Per-input page cache for the *currently active* body column. - /// Pages are appended as the decoder produces them and evicted from - /// the front once their last row is below `body_col_cursor`. The - /// cache must persist across all outputs that consume rows from - /// this column so that a page whose range straddles two outputs is - /// not re-decoded (the stream has already advanced past it). At - /// the start of each body column [`reset_body_col_state`] clears - /// this cache and zeroes the cursor. + pub(crate) arrow_schema: SchemaRef, + /// Per-input, per-parquet-col page cache. Pages emitted by the + /// decoder are stored under their actual `col_idx`, not the col + /// the caller was asking about — this lets `fill_page_cache_to_row` + /// pull pages of col B during a col A advance (storage order: + /// col A pages all stream before col B pages within an RG, so to + /// get any col B pages we may have to consume leftover col A + /// pages first). The synthesized-prefix path relies on this: + /// adjacent regions sharing one RG re-read col A from this cache + /// when region 2 starts, and the cache has already been filled in + /// by region 1's reads. + /// + /// Pages are evicted (per col) when the col's cursor advances + /// past their last row. /// /// **Memory bound (horizontal, not vertical).** This cache is /// per-input: total memory across inputs is /// `N_inputs × per_input_peak`. The per-input peak is bounded by /// `ceil(OUTPUT_PAGE_ROWS / input_page_rows) + small_slack` — - /// driven by [`fill_page_cache_to_row`], which fetches only enough - /// pages to cover the rows from *this* input that contribute to - /// the *current output page* (a 1024-row slice), and by the - /// eviction loop in [`assemble_one_output_page`], which drops any - /// page whose last row falls below the cursor after each output - /// page emits. The cache never holds an input column-chunk's worth - /// of pages — a regression that ever did would break the MS-7 - /// invariant asserted by + /// driven by `fill_page_cache_to_row` (in `body_assembler`), which + /// fetches only enough pages to cover the rows from *this* input + /// that contribute to the *current output page* (a 1024-row + /// slice), and by the eviction loop in `assemble_one_output_page`, + /// which drops any page whose last row falls below the cursor + /// after each output page emits. The cache never holds an input + /// column-chunk's worth of pages — a regression that ever did + /// would break the MS-7 invariant asserted by /// `test_ms7_body_col_page_cache_bounded_regardless_of_input_size`. - body_col_page_cache: Vec, - /// Next unconsumed input row for the active body column. Advances - /// monotonically across outputs because the merge plan assigns each - /// input's rows to outputs in input-row order. - body_col_cursor: usize, + pub(crate) body_col_page_caches: HashMap>, + /// Per-parquet-col cursor: the next unconsumed input row for that + /// col. Advances monotonically as outputs are written; survives + /// across regions of the same input so the body col assembler can + /// resume mid-RG for synthesized regions. + pub(crate) body_col_cursors: HashMap, } impl InputDecoderState { - /// Clear the per-input body-column cache. Called at the start of - /// each new body column so leftover pages from the previous column - /// (which have a different `col_idx`) don't poison the new column's - /// row-start arithmetic. The decoder itself is *not* reset — its - /// per-(rg, col) `rows_decoded` counters and queued pages must - /// survive so subsequent decode calls return correct row offsets. - fn reset_body_col_state(&mut self) { - self.body_col_page_cache.clear(); - self.body_col_cursor = 0; + /// Position the cursor for `col_idx` to `start_row` and drop any + /// cached pages strictly below it. Used at the start of each + /// (region, body col) write: for whole-RG regions this is a no-op + /// (cursor was already 0 and cache empty); for synthesized + /// regions that share an RG with earlier regions, this skips the + /// rows the previous region consumed without clearing the rest of + /// the cache. + pub(crate) fn set_body_col_cursor(&mut self, col_idx: usize, start_row: usize) { + self.body_col_cursors.insert(col_idx, start_row); + if let Some(pages) = self.body_col_page_caches.get_mut(&col_idx) { + while let Some(front) = pages.first() { + let front_end = front.row_start + front.array.len(); + if front_end <= start_row { + pages.remove(0); + } else { + break; + } + } + } + } + + /// Look up the cached pages for the given col, returning an empty + /// slice if none have been decoded yet. + pub(crate) fn body_col_cache(&self, col_idx: usize) -> &[DecodedPage] { + match self.body_col_page_caches.get(&col_idx) { + Some(v) => v.as_slice(), + None => &[], + } + } + + /// Mutable handle to the cache for `col_idx`, creating an empty + /// entry if none exists. + pub(crate) fn body_col_cache_mut(&mut self, col_idx: usize) -> &mut Vec { + self.body_col_page_caches.entry(col_idx).or_default() + } + + /// Current cursor for `col_idx`; defaults to 0 if untouched. + pub(crate) fn body_col_cursor(&self, col_idx: usize) -> usize { + self.body_col_cursors.get(&col_idx).copied().unwrap_or(0) + } + + /// Sum of cached pages across all cols. Used by the MS-7 peak- + /// length probe in tests. + pub(crate) fn body_col_caches_total_len(&self) -> usize { + self.body_col_page_caches.values().map(|v| v.len()).sum() + } + + /// Clear every per-col page cache and every per-col cursor. + /// Called between top-level regions: each region typically uses a + /// different input RG, and the per-col page `row_start` values + /// reported by the decoder are RG-local, so pages cached for RG K + /// would conflict with RG K+1's row-index space. Sub-regions of a + /// single top-level region share an RG and MUST NOT trigger this + /// reset — they rely on the cache surviving from one sub-region + /// to the next so an earlier-col read whose stream-tail spans + /// later sub-regions stays available. + pub(crate) fn reset_all_body_col_state(&mut self) { + self.body_col_page_caches.clear(); + self.body_col_cursors.clear(); } } @@ -330,8 +552,8 @@ fn build_input_decoders_state( decoder, metadata, arrow_schema: Arc::new(arrow_schema), - body_col_page_cache: Vec::new(), - body_col_cursor: 0, + body_col_page_caches: HashMap::new(), + body_col_cursors: HashMap::new(), }); } Ok(states) @@ -470,6 +692,59 @@ fn extract_and_validate_input_metadata( }) } +/// Drain a region's contributing inputs' sort cols and align them to +/// the union sort schema. The result has one entry per global input +/// index (zero-row placeholders for non-contributing inputs), which +/// is what `split_region_at_sorted_series` and `process_region` both +/// expect. +/// +/// Used by the main loop when a region needs to be sub-divided +/// across multiple output files — splitting needs the merge order, +/// which needs the drained sort cols. For regions that fit in a +/// single output we skip this and let `process_region` drain +/// internally so the per-region memory cost stays bounded by what +/// the writer actively consumes. +fn drain_and_align_region( + handle: &Handle, + decoders_state: &mut [InputDecoderState], + region: &Region, + sort_fields_str: &str, +) -> Result> { + let num_inputs = decoders_state.len(); + let mut sort_col_batches: Vec> = (0..num_inputs).map(|_| None).collect(); + for c in ®ion.contributing { + let batch = drain_sort_cols_one_input( + handle, + &mut decoders_state[c.input_idx], + sort_fields_str, + c.input_idx, + c.rg_idx, + )?; + if batch.num_columns() > 0 && batch.schema().index_of(SORTED_SERIES_COLUMN).is_err() { + bail!( + "input {} rg {} is missing the '{}' column required for merge", + c.input_idx, + c.rg_idx, + SORTED_SERIES_COLUMN, + ); + } + sort_col_batches[c.input_idx] = Some(batch); + } + + let mut sort_batch_vec: Vec = Vec::with_capacity(num_inputs); + for (idx, slot) in sort_col_batches.into_iter().enumerate() { + let batch = match slot { + Some(b) => b, + None => empty_sort_col_record_batch(&decoders_state[idx], sort_fields_str)?, + }; + sort_batch_vec.push(batch); + } + + let (_sort_union_schema, aligned_sort_batches) = + align_inputs_to_union_schema(&sort_batch_vec, sort_fields_str)?; + Ok(aligned_sort_batches) +} + // ============================================================================ // Phase 0: drain sort cols from each input // ============================================================================ @@ -484,9 +759,13 @@ fn drain_sort_cols_all_inputs( decoders_state: &mut [InputDecoderState], sort_fields_str: &str, ) -> Result> { + // Single-region path: drain RG 0 of each input. Used by the + // single-region streaming path (one region covering all inputs; + // applies when all inputs are single-RG OR `rg_partition_prefix_len + // == 0` with one synthetic adapter-presented RG per input). let mut batches = Vec::with_capacity(decoders_state.len()); for (idx, state) in decoders_state.iter_mut().enumerate() { - let batch = drain_sort_cols_one_input(handle, state, sort_fields_str, idx)?; + let batch = drain_sort_cols_one_input(handle, state, sort_fields_str, idx, 0)?; if batch.num_columns() > 0 && batch.schema().index_of(SORTED_SERIES_COLUMN).is_err() { bail!( "input {idx} is missing the '{}' column required for merge", @@ -498,14 +777,273 @@ fn drain_sort_cols_all_inputs( Ok(batches) } +/// Process one merge region: obtain the contributing inputs' sort col +/// batches (drain them fresh from the streaming decoder when +/// `prefetched_sort_batches` is `None`; otherwise slice the +/// pre-drained batches by each contribution's row range — used by the +/// synthesized prefix-region path), compute the region's merge order, +/// open a new output RG in the writer, write all cols (sort cols via +/// interleave, body cols via the page-bounded assembler), close the +/// RG, accumulate per-output static metadata. +/// +/// When `prefetched_sort_batches` is provided, each contribution's +/// `start_row` determines where the body col assembler starts inside +/// the contributing input's RG. The body col stream is shared with +/// adjacent synthesized regions of the same input, so its cache / +/// cursor must advance monotonically — see `reset_body_col_state`. +#[allow(clippy::too_many_arguments)] +fn process_region( + handle: &Handle, + decoders_state: &mut [InputDecoderState], + writer_state: &mut OutputWriterStorage, + accumulator: &mut OutputAccumulator, + region: &Region, + union_schema: &SchemaRef, + input_meta: &InputMetadata, + prefetched_sort_batches: Option<&[RecordBatch]>, +) -> Result<()> { + // 1. Obtain sort col batches for this region's contributing inputs. When prefetched batches are + // supplied (synthesized path), slice them by the contribution's row range so a single RG + // drained once can feed multiple adjacent regions. Otherwise drain a fresh whole-RG batch + // from the streaming decoder. The result is indexed BY GLOBAL INPUT INDEX, with zero-row + // placeholders for non-contributing inputs so the BodyColOutputPageAssembler sees a uniform + // input-count. + let num_inputs = decoders_state.len(); + let mut sort_col_batches: Vec> = (0..num_inputs).map(|_| None).collect(); + for c in ®ion.contributing { + let input_idx = c.input_idx; + let rg_idx = c.rg_idx; + let batch = match prefetched_sort_batches { + Some(prefetched) => prefetched[input_idx].slice(c.start_row, c.num_rows), + None => drain_sort_cols_one_input( + handle, + &mut decoders_state[input_idx], + &input_meta.sort_fields, + input_idx, + rg_idx, + )?, + }; + if batch.num_columns() > 0 && batch.schema().index_of(SORTED_SERIES_COLUMN).is_err() { + bail!( + "input {input_idx} rg {rg_idx} is missing the '{}' column required for merge", + SORTED_SERIES_COLUMN, + ); + } + sort_col_batches[input_idx] = Some(batch); + } + + // Materialise into a `Vec` per input. Non-contributing + // inputs get zero-row placeholders with the input's sort col schema + // so `compute_merge_order` and the body col assembler see uniform + // shapes. + let mut sort_batch_vec: Vec = Vec::with_capacity(num_inputs); + for (idx, slot) in sort_col_batches.into_iter().enumerate() { + let batch = match slot { + Some(b) => b, + None => empty_sort_col_record_batch(&decoders_state[idx], &input_meta.sort_fields)?, + }; + sort_batch_vec.push(batch); + } + + // Per-input cursor offsets to feed to the body col reset hook + // below. For whole-RG regions this is 0 everywhere; for + // synthesized regions it is `c.start_row` of the contributing + // input so the body col assembler walks rows starting at the + // region's first input row instead of restarting at 0. + let mut input_start_rows: Vec = vec![0; num_inputs]; + for c in ®ion.contributing { + input_start_rows[c.input_idx] = c.start_row; + } + + // 2. Align to union sort schema for the merge-order comparator. + let (sort_union_schema, aligned_sort_batches) = + align_inputs_to_union_schema(&sort_batch_vec, &input_meta.sort_fields)?; + + // 3. Compute merge order for this region. + let merge_order = compute_merge_order(&aligned_sort_batches, &input_meta.sort_fields)?; + let region_rows: usize = merge_order.iter().map(|r| r.row_count).sum(); + if region_rows == 0 { + return Ok(()); + } + + // 4. Apply the merge permutation to the sort col batches to get the region's sorted sort-col + // batch. This will be appended to the output accumulator; also used to compute take indices + // for the body col assembler. + let region_sort_batch = + apply_merge_permutation(&aligned_sort_batches, &sort_union_schema, &merge_order) + .context("applying merge permutation for region sort cols")?; + + // MC-3: verify the region's output is sorted. + verify_sort_order(®ion_sort_batch, &input_meta.sort_fields); + + // 5. Build per-region destinations: maps (input_idx, input_row) → (output_idx=0, + // position_in_region). The body col assembler walks this to find which (input, row) + // contributes each output position. + // + // The destinations array is indexed by row *within the sort + // batch* — which for whole-RG regions equals "row within the + // RG" and for synthesized regions equals "row within the + // region's slice". In both cases that index lines up with what + // the body col decoder's `row_start` reports for the current + // RG, modulo the per-input `start_row` offset added below. + let mut destinations: Vec>> = aligned_sort_batches + .iter() + .enumerate() + .map(|(idx, b)| { + // For the synthesized path the body col assembler walks + // absolute input rows; pad the destinations array so the + // index space matches what the page decoder reports. + vec![None; input_start_rows[idx] + b.num_rows()] + }) + .collect(); + let mut pos = 0usize; + for run in &merge_order { + for r in 0..run.row_count { + let absolute_row = input_start_rows[run.input_index] + run.start_row + r; + destinations[run.input_index][absolute_row] = Some((0, pos)); + pos += 1; + } + } + let region_destinations = InputRowDestinations { + per_input: destinations, + rows_per_output: vec![region_rows], + }; + + // 6. Open a new output RG and write all cols in union schema order. + let mut row_group = writer_state.writer.start_row_group().with_context(|| { + format!( + "opening row group for output {} region", + writer_state.output_idx, + ) + })?; + writer_state.num_row_groups += 1; + + for (col_idx, field) in union_schema.fields().iter().enumerate() { + let col_name = field.name(); + if sort_union_schema.index_of(col_name).is_ok() { + // Sort col: take from the already-built region_sort_batch. + let arrays = build_sort_col_pages_from_sorted_batch(®ion_sort_batch, col_name)?; + row_group + .write_next_column_arrays(arrays) + .with_context(|| { + format!( + "writing sort col '{col_name}' (col_idx {col_idx}) to output {}", + writer_state.output_idx, + ) + })?; + } else { + // Body col: stream via the page-bounded assembler. Resolve + // each input's parquet col_idx for this union-schema col + // first, then position the per-col cursor at the region's + // `start_row` for that input. For whole-RG regions + // `start_row == 0` (no-op for first region); for synthesized + // regions sharing an RG with earlier regions the cursor + // jumps past already-emitted rows so we don't re-emit them. + // The decoder itself is never reset — its per-(rg, col) + // `rows_decoded` counters and queued pages must survive so + // subsequent decode calls return correct row offsets. + let mut input_col_indices: Vec> = Vec::with_capacity(num_inputs); + for state in decoders_state.iter() { + input_col_indices.push(state.arrow_schema.index_of(col_name).ok()); + } + for (idx, state) in decoders_state.iter_mut().enumerate() { + if let Some(col_parquet_idx) = input_col_indices[idx] { + state.set_body_col_cursor(col_parquet_idx, input_start_rows[idx]); + } + } + + let track_service = col_name == "service"; + + let assembler = BodyColOutputPageAssembler::new( + handle, + decoders_state, + &input_col_indices, + ®ion_destinations, + 0, // out_idx is always 0 within a single-region call + col_name, + field.as_ref(), + ); + + // Feed pages one at a time into `write_next_column_arrays` + // via the streaming iterator: it surfaces assembly errors + // through `error_slot` so memory stays bounded by output- + // page size instead of column-chunk size. + let mut error_slot: Option = None; + let service_collector: Option<&mut HashSet> = if track_service { + Some(&mut accumulator.service_names) + } else { + None + }; + let stream_iter = StreamingBodyColIter { + inner: assembler.into_iter(), + error_slot: &mut error_slot, + service_collector, + }; + let write_result = row_group.write_next_column_arrays(stream_iter); + + if let Some(err) = error_slot { + return Err(err).with_context(|| { + format!( + "assembling body col '{col_name}' for output {} region", + writer_state.output_idx, + ) + }); + } + write_result.with_context(|| { + format!( + "writing body col '{col_name}' to output {} region", + writer_state.output_idx, + ) + })?; + } + } + + row_group.finish().with_context(|| { + format!( + "finishing region row group for output {}", + writer_state.output_idx + ) + })?; + + // 7. Accumulate this region's contribution to the output. + accumulator.append_sort_batch(region_sort_batch)?; + accumulator.num_rows += region_rows; + + Ok(()) +} + +/// Helper for sort col writes within a region: split the region's +/// already-sorted sort col into page-sized chunks for +/// `write_next_column_arrays`. +fn build_sort_col_pages_from_sorted_batch( + sorted_batch: &RecordBatch, + col_name: &str, +) -> Result> { + let col_idx = sorted_batch + .schema() + .index_of(col_name) + .with_context(|| format!("missing sort col '{col_name}' in region sorted batch"))?; + let col = sorted_batch.column(col_idx); + let total_rows = col.len(); + let mut pages = Vec::with_capacity(total_rows.div_ceil(OUTPUT_PAGE_ROWS)); + let mut start = 0; + while start < total_rows { + let len = (total_rows - start).min(OUTPUT_PAGE_ROWS); + pages.push(col.slice(start, len)); + start += len; + } + Ok(pages) +} + fn drain_sort_cols_one_input( handle: &Handle, state: &mut InputDecoderState, sort_fields_str: &str, input_idx: usize, + expected_rg_idx: usize, ) -> Result { - if state.metadata.num_row_groups() == 0 { - // Empty input — no rows to drain. Return a zero-row batch with the + if state.metadata.num_row_groups() == 0 || expected_rg_idx >= state.metadata.num_row_groups() { + // No rows to drain at this RG. Return a zero-row batch with the // sort cols' fields preserved so downstream merge order code sees a // uniform schema across inputs. return empty_sort_col_record_batch(state, sort_fields_str); @@ -541,20 +1079,20 @@ fn drain_sort_cols_one_input( } // Target row count per sort col (from row group's column chunk metadata). - let rg_meta = state.metadata.row_group(0); + let rg_meta = state.metadata.row_group(expected_rg_idx); let mut target_rows_per_col: HashMap = HashMap::new(); for &col_idx in sort_col_parquet_indices.keys() { target_rows_per_col.insert(col_idx, rg_meta.column(col_idx).num_values() as usize); } // Drain pages into per-col buffers until all sort cols are fully - // decoded. The streaming engine relies on a hard storage-ordering - // contract: within a row group, parquet emits column chunks in - // schema order (sort cols are declared first in our schema, body - // cols after), so all sort col pages appear before any body col - // page. Cross-file we don't require identical body-col ordering — + // decoded for this RG. The streaming engine relies on a hard + // storage-ordering contract: within each row group parquet emits + // column chunks in schema order (sort cols declared first, body + // cols after), so every sort-col page appears before any body-col + // page. Cross-file we do NOT require identical body-col ordering — // the body-col loop drives from the union schema and looks each - // column up by name. The contract we DO require cross-file is + // column up by name; the contract we do require cross-file is // "sort cols come first." A page from a body col arriving here // means a producer violated that contract; bail rather than guess. let mut per_col_pages: HashMap> = HashMap::new(); @@ -566,30 +1104,28 @@ fn drain_sort_cols_one_input( while sort_cols_finished < sort_col_target { let decoded = handle .block_on(state.decoder.decode_next_page()) - .with_context(|| format!("decoding sort col page (input {input_idx})"))?; + .with_context(|| { + format!("decoding sort col page (input {input_idx}, rg {expected_rg_idx})") + })?; let page = match decoded { Some(p) => p, None => bail!( - "stream ended before sort cols fully drained for input {input_idx}: \ - {sort_cols_finished}/{sort_col_target} cols complete", + "stream ended before sort cols fully drained for input {input_idx} rg \ + {expected_rg_idx}: {sort_cols_finished}/{sort_col_target} cols complete", ), }; if !sort_col_parquet_indices.contains_key(&page.col_idx) { bail!( "input {input_idx} returned a non-sort page (col {}) before all sort cols were \ - drained — sort-cols-first storage ordering violated", + drained for rg {expected_rg_idx} — this violates Husky storage ordering", page.col_idx, ); } - if page.rg_idx != 0 { - // PR-6b.2 (this PR) only supports single-RG inputs. The - // multi-RG path with prefix-aligned row groups is added - // in PR-6c.2 (#6424) along with `process_region` / - // composite-key encoding. + if page.rg_idx != expected_rg_idx { bail!( - "input {input_idx} returned a page from rg {} during sort col drain — only \ - single-RG inputs are supported in PR-6b.2", + "input {input_idx} returned a page from rg {} while draining sort cols of rg \ + {expected_rg_idx}", page.rg_idx, ); } @@ -760,11 +1296,11 @@ fn concat_arrays(arrays: &[ArrayRef]) -> Result { /// is not in any output (only possible for rows beyond the merge /// plan's coverage; shouldn't happen with our merge order). #[derive(Debug)] -struct InputRowDestinations { +pub(crate) struct InputRowDestinations { /// One Vec per input. Length = input's sort-col row count. - per_input: Vec>>, + pub(crate) per_input: Vec>>, /// Total rows per output index (cumulative writer "expected" rows). - rows_per_output: Vec, + pub(crate) rows_per_output: Vec, } fn build_input_row_destinations( @@ -798,206 +1334,11 @@ fn build_input_row_destinations( } // ============================================================================ -// Phase 3: streaming write with one writer per output +// Obsolete PR-6b.2 multi-output-parallel helpers (deleted in PR-6c.2's +// per-region restructure). The functions below are no longer used — +// per-region processing in `process_region` is the new path. // ============================================================================ -/// Per-output state owned across phase 3 (writer + bookkeeping). -/// The row group lives in a parallel Vec so its borrow into `writer` -/// is tracked by the compiler instead of through a `'static` -/// transmute. -struct OutputWriterStorage { - output_idx: usize, - output_path: PathBuf, - writer: StreamingParquetWriter, - /// Service-name set built during the body col write of "service" - /// (or empty if no service col). - service_names: HashSet, - /// Per-output total row count = sum of merge runs in this output's boundary. - num_rows: usize, -} - -#[allow(clippy::too_many_arguments)] -fn write_streaming_outputs( - handle: &Handle, - decoders_state: &mut [InputDecoderState], - aligned_sort_batches: &[RecordBatch], - sort_union_schema: &SchemaRef, - merge_order: &[MergeRun], - boundaries: &[Range], - destinations: &InputRowDestinations, - input_meta: &InputMetadata, - writer_config: &crate::storage::ParquetWriterConfig, - output_dir: &Path, -) -> Result> { - // 1. Build the union schema across full input arrow schemas (so the output covers every column - // that appears in any input). The sort union schema covers only sort cols. - let input_arrow_schemas: Vec = decoders_state - .iter() - .map(|s| Arc::clone(&s.arrow_schema)) - .collect(); - let (full_union_schema, _aligned_full_placeholder) = - build_full_union_schema_from_arrow_schemas(&input_arrow_schemas, &input_meta.sort_fields)?; - - // 2. Build per-output metadata (KV entries, row keys, zonemaps) up front from sort col data — - // these are what the schema + writer props depend on. - let per_output_static = boundaries - .iter() - .enumerate() - .map(|(out_idx, boundary)| { - build_per_output_static( - out_idx, - boundary, - aligned_sort_batches, - sort_union_schema, - merge_order, - input_meta, - ) - }) - .collect::>>()?; - - // 3. Decide per-output schema: optimise based on each output's sort col data (which determines - // metric_name cardinality, etc.). Body cols stay as declared by the union schema; we don't - // probe their cardinality here since we haven't read them yet. This is a slight regression - // vs. the non-streaming engine — it would dict-encode low-cardinality string body cols too. - // PR-6c.2 or later can revisit by gathering body-col cardinality during the streaming pass. - let per_output_schemas: Vec = per_output_static - .iter() - .map(|s| derive_output_schema(&full_union_schema, sort_union_schema, &s.sort_optimised)) - .collect::>>()?; - - // 4. Open M writers, one per output. Writers + bookkeeping live in `writer_states`; the row - // group borrows mutably from each writer and is held in a parallel `row_groups` Vec for the - // col loop. - let mut writer_states: Vec = Vec::with_capacity(boundaries.len()); - for (out_idx, (schema, static_meta)) in per_output_schemas - .iter() - .zip(per_output_static.iter()) - .enumerate() - { - if destinations.rows_per_output[out_idx] == 0 { - continue; - } - writer_states.push(open_output_writer( - out_idx, - output_dir, - Arc::clone(schema), - static_meta, - input_meta, - writer_config, - )?); - } - - // Snapshot the (output_idx, num_rows) for each storage entry BEFORE - // calling `start_row_group`, which borrows `writer_states` mutably - // for the rest of phase 3's col loop. - let writer_index_view = writer_states_index_view(&writer_states); - let num_storages = writer_states.len(); - - let mut row_groups: Vec> = - writer_states - .iter_mut() - .map(|s| { - s.writer - .start_row_group() - .with_context(|| format!("opening row group for output {}", s.output_idx)) - }) - .collect::>>()?; - - // Service names are collected into a separate Vec> - // parallel to `row_groups`; we can't write into `writer_states` here - // because it is already borrowed mutably by `row_groups`. We merge - // these back into `writer_states` after dropping the row groups. - let mut service_names_per_output: Vec> = - (0..num_storages).map(|_| HashSet::new()).collect(); - write_all_columns( - handle, - &mut row_groups, - &mut service_names_per_output, - &writer_index_view, - decoders_state, - aligned_sort_batches, - sort_union_schema, - &full_union_schema, - merge_order, - boundaries, - destinations, - &per_output_schemas, - )?; - - // 6. Finish all row groups (drops the borrows on writers). - for rg in row_groups { - rg.finish().context("finishing row group")?; - } - - // 7. Merge collected service names + close writers + build MergeOutputFiles. - let mut outputs = Vec::with_capacity(writer_states.len()); - for (mut state, services) in writer_states - .into_iter() - .zip(service_names_per_output.into_iter()) - { - state.service_names.extend(services); - outputs.push(finalize_output_writer(state, &per_output_static)?); - } - Ok(outputs) -} - -/// Static per-output state computed once from sort col data. Holds -/// the per-output sort-col-only batch (used for metadata extraction) -/// and the per-output schema-optimisation hints. -struct PerOutputStatic { - /// Sort-cols-only batch in output sort order — used by row_keys / - /// zonemap / metric_names / time_range extractors. - sort_optimised: RecordBatch, - row_keys_proto: Option>, - zonemap_regexes: HashMap, - metric_names: HashSet, - time_range: crate::split::TimeRange, - /// Number of rows that go into this output. - num_rows: usize, -} - -fn build_per_output_static( - out_idx: usize, - boundary: &Range, - aligned_sort_batches: &[RecordBatch], - sort_union_schema: &SchemaRef, - merge_order: &[MergeRun], - input_meta: &InputMetadata, -) -> Result { - let runs = &merge_order[boundary.clone()]; - let sort_batch = apply_merge_permutation(aligned_sort_batches, sort_union_schema, runs) - .with_context(|| format!("applying merge permutation for output {out_idx} sort cols"))?; - let num_rows = sort_batch.num_rows(); - - // MC-3 sort order on the sort-col-only batch (same check the - // non-streaming engine does, just restricted to columns we have). - verify_sort_order(&sort_batch, &input_meta.sort_fields); - let sort_optimised = optimize_output_batch(&sort_batch); - - let row_keys_proto = row_keys::extract_row_keys(&input_meta.sort_fields, &sort_optimised) - .with_context(|| format!("extracting row keys for output {out_idx}"))? - .map(|rk| row_keys::encode_row_keys_proto(&rk)); - - let zonemap_opts = ZonemapOptions::default(); - let zonemap_regexes = - zonemap::extract_zonemap_regexes(&input_meta.sort_fields, &sort_optimised, &zonemap_opts) - .with_context(|| format!("extracting zonemap regexes for output {out_idx}"))?; - - let metric_names = extract_metric_names(&sort_optimised) - .with_context(|| format!("extracting metric names for output {out_idx}"))?; - let time_range = extract_time_range(&sort_optimised) - .with_context(|| format!("extracting time range for output {out_idx}"))?; - - Ok(PerOutputStatic { - sort_optimised, - row_keys_proto, - zonemap_regexes, - metric_names, - time_range, - num_rows, - }) -} - /// Build the full union schema across all inputs' arrow schemas /// (NOT just sort cols). Reuses the same algorithm as /// [`align_inputs_to_union_schema`] but takes pre-extracted arrow @@ -1005,7 +1346,7 @@ fn build_per_output_static( fn build_full_union_schema_from_arrow_schemas( arrow_schemas: &[SchemaRef], sort_fields_str: &str, -) -> Result<(SchemaRef, ())> { +) -> Result { // Build zero-row batches with the right schemas; that lets us // reuse `align_inputs_to_union_schema`'s field-merge / storage- // ordering logic unchanged. @@ -1014,952 +1355,46 @@ fn build_full_union_schema_from_arrow_schemas( .map(|s| RecordBatch::new_empty(Arc::clone(s))) .collect(); let (schema, _) = align_inputs_to_union_schema(&empty_batches, sort_fields_str)?; - Ok((schema, ())) -} - -/// Compute the per-output schema. For PR-6b.2 we use the -/// (string-normalised) union schema as the output schema directly — -/// fields stay Utf8/LargeUtf8 rather than being re-dict-encoded. -/// Reason: streaming-decoded input arrays come out of the page -/// decoder as plain `StringArray`/`BinaryArray` (not Dictionary), and -/// dict re-encoding per output page would add a per-page CPU cost we -/// don't want to take in the page-bounded path. Re-introducing -/// dict-encoded output strings can be done later by tracking -/// cardinality during the streaming pass — call site is here. -/// -/// We do still want to drop columns that are all-null *for this -/// output* (e.g., a column only present in inputs that don't -/// contribute any rows to this output's range). The `sort_optimised` -/// batch has already discarded all-null sort fields; we mirror that -/// decision when building the per-output schema. Body cols are kept -/// unconditionally — tracking per-output body-col presence would -/// require pre-reading every body column for every output, which is -/// exactly the column-chunk-bounded buffering the streaming path -/// exists to avoid. -/// -/// `sort_union_schema` is the union of every input's sort columns -/// (before per-output optimisation). It's the only way to tell -/// whether a given union-schema field is a sort field or a body -/// field — `sort_optimised.schema()` alone can't disambiguate because -/// it has dropped some sort fields by design. Without this -/// distinction the function falls into the trap of using -/// `full_union_schema.index_of(field.name())`, which is trivially -/// true for every iterated field, and the all-null drop never -/// happens. -fn derive_output_schema( - full_union_schema: &SchemaRef, - sort_union_schema: &SchemaRef, - sort_optimised: &RecordBatch, -) -> Result { - let sort_optimised_schema = sort_optimised.schema(); - let mut fields: Vec> = Vec::with_capacity(full_union_schema.fields().len()); - for field in full_union_schema.fields() { - let is_sort_field = sort_union_schema.index_of(field.name()).is_ok(); - if is_sort_field { - // Sort field: keep only if the per-output optimiser kept - // it (i.e., not all-null for this output's rows). - if sort_optimised_schema.index_of(field.name()).is_ok() { - fields.push(Arc::clone(field)); - } - } else { - // Body field: always kept. - fields.push(Arc::clone(field)); - } - } - Ok(Arc::new(ArrowSchema::new(fields))) + Ok(schema) } -fn open_output_writer( - out_idx: usize, - output_dir: &Path, - schema: SchemaRef, - static_meta: &PerOutputStatic, - input_meta: &InputMetadata, - writer_config: &crate::storage::ParquetWriterConfig, -) -> Result { - let output_prefix_len = input_meta.rg_partition_prefix_len; - let kv_entries = build_merge_kv_metadata( - input_meta, - &static_meta.row_keys_proto, - &static_meta.zonemap_regexes, - output_prefix_len, - ); - let sorting_cols = build_sorting_columns(&static_meta.sort_optimised, &input_meta.sort_fields)?; - let sort_field_names = resolve_sort_field_names(&input_meta.sort_fields)?; - - let props = writer_config.to_writer_properties_with_metadata( - &schema, - sorting_cols, - Some(kv_entries), - &sort_field_names, - ); +// ============================================================================ +// Tests +// ============================================================================ - let output_filename = format!("merge_output_{}.parquet", Ulid::new()); - let output_path = output_dir.join(&output_filename); - let file = std::fs::File::create(&output_path) - .with_context(|| format!("creating output file: {}", output_path.display()))?; - let writer = StreamingParquetWriter::try_new(file, Arc::clone(&schema), props) - .with_context(|| format!("opening streaming writer for output {out_idx}"))?; - - Ok(OutputWriterStorage { - output_idx: out_idx, - output_path, - writer, - service_names: HashSet::new(), - num_rows: static_meta.num_rows, - }) -} +#[cfg(test)] +mod tests { + use std::path::PathBuf; + use std::sync::Arc; -/// Index view used inside the col loop to find the writer's -/// `output_idx` and `num_rows` without needing a mutable borrow on -/// `writer_states` (which is already mutably borrowed by `row_groups`). -fn writer_states_index_view(writer_states: &[OutputWriterStorage]) -> Vec<(usize, usize)> { - writer_states - .iter() - .map(|s| (s.output_idx, s.num_rows)) - .collect() -} + use arrow::array::{ + ArrayRef, BinaryArray, DictionaryArray, Float64Array, Int64Array, StringArray, UInt8Array, + UInt64Array, + }; + use arrow::datatypes::{DataType, Field, Int32Type, Schema as ArrowSchema}; + use base64::Engine; + use base64::engine::general_purpose::STANDARD as BASE64; + use bytes::Bytes; + use parquet::arrow::ArrowWriter; + use parquet::file::metadata::KeyValue; + use parquet::file::properties::WriterProperties; + use parquet::file::reader::{FileReader, SerializedFileReader}; + use tempfile::TempDir; + use tokio::io::AsyncRead; -#[allow(clippy::too_many_arguments)] -fn write_all_columns( - handle: &Handle, - row_groups: &mut [crate::storage::streaming_writer::RowGroupBuilder<'_, std::fs::File>], - service_names_per_output: &mut [HashSet], - writer_index_view: &[(usize, usize)], - decoders_state: &mut [InputDecoderState], - aligned_sort_batches: &[RecordBatch], - sort_union_schema: &SchemaRef, - full_union_schema: &SchemaRef, - merge_order: &[MergeRun], - boundaries: &[Range], - destinations: &InputRowDestinations, - per_output_schemas: &[SchemaRef], -) -> Result<()> { - // Iterate cols in the **full union schema** order. The union - // covers every column that appears in ANY output. For each col K - // and each output: - // - If output's schema includes col K: write col K's data (sort col → from buffer, body col → - // from decoder). - // - Else: skip — that output dropped col K as all-null for its row range; the next output's - // col K still gets written. - // - // It's tempting to drive from one per-output schema, since all - // per-output schemas share the same column ordering as a - // subsequence. But two outputs may drop *different* all-null - // fields and end up with the same field count — then picking - // either misses a field the other output still needs, and the - // writer for the latter output writes subsequent columns into - // the wrong slot. The full union schema is the only choice that - // covers every column every output may need, in the canonical - // storage order. - - // For each full-union-schema col K: - for parent_col_idx in 0..full_union_schema.fields().len() { - let parent_field = full_union_schema.field(parent_col_idx); - let parent_name = parent_field.name(); - - // Is this a sort col (in memory) or a body col (streamed)? - let is_sort_col = sort_union_schema.index_of(parent_name).is_ok(); - - if is_sort_col { - write_sort_col_for_all_outputs( - row_groups, - writer_index_view, - parent_name, - aligned_sort_batches, - sort_union_schema, - merge_order, - boundaries, - destinations, - per_output_schemas, - )?; - } else { - write_body_col_for_all_outputs( - handle, - row_groups, - service_names_per_output, - writer_index_view, - decoders_state, - parent_name, - destinations, - per_output_schemas, - )?; - } - } + use super::region_grouping::{ + assert_unique_rg_prefix_keys, extract_rg_composite_prefix_key, + find_prefix_parquet_col_indices, + }; + use super::*; + use crate::split::TAG_SERVICE; + use crate::storage::page_decoder::StreamDecoder; + use crate::storage::streaming_reader::{RemoteByteSource, StreamingParquetReader}; + use crate::storage::{ + Compression, PARQUET_META_ROW_KEYS, PARQUET_META_ZONEMAP_REGEXES, ParquetWriterConfig, + }; - Ok(()) -} - -#[allow(clippy::too_many_arguments)] -fn write_sort_col_for_all_outputs( - row_groups: &mut [crate::storage::streaming_writer::RowGroupBuilder<'_, std::fs::File>], - writer_index_view: &[(usize, usize)], - col_name: &str, - aligned_sort_batches: &[RecordBatch], - sort_union_schema: &SchemaRef, - merge_order: &[MergeRun], - boundaries: &[Range], - destinations: &InputRowDestinations, - per_output_schemas: &[SchemaRef], -) -> Result<()> { - let _ = sort_union_schema; - - let mut storage_idx = 0; - for (out_idx, boundary) in boundaries.iter().enumerate() { - if destinations.rows_per_output[out_idx] == 0 { - continue; - } - debug_assert_eq!(writer_index_view[storage_idx].0, out_idx); - - // Drop this col if the output's schema doesn't include it. - let out_schema = &per_output_schemas[out_idx]; - if out_schema.index_of(col_name).is_err() { - storage_idx += 1; - continue; - } - - let runs = &merge_order[boundary.clone()]; - let arrays = build_sort_col_pages_for_output(col_name, aligned_sort_batches, runs)?; - row_groups[storage_idx] - .write_next_column_arrays(arrays.into_iter()) - .with_context(|| format!("writing sort col '{col_name}' to output {out_idx}"))?; - storage_idx += 1; - } - Ok(()) -} - -/// Build per-output-page arrays for one sort col. The col is already -/// in memory across all inputs (`aligned_sort_batches`); for this -/// output we walk its merge runs and split the take result into -/// `OUTPUT_PAGE_ROWS`-sized chunks. -fn build_sort_col_pages_for_output( - col_name: &str, - aligned_sort_batches: &[RecordBatch], - runs: &[MergeRun], -) -> Result> { - // Collect references to each input's column array. - let mut input_arrays: Vec<&dyn Array> = Vec::with_capacity(aligned_sort_batches.len()); - for batch in aligned_sort_batches { - let idx = batch.schema().index_of(col_name).map_err(|_| { - anyhow!("input is missing sort col '{col_name}' that the union schema expected",) - })?; - input_arrays.push(batch.column(idx).as_ref()); - } - - let mut indices: Vec<(usize, usize)> = - Vec::with_capacity(runs.iter().map(|r| r.row_count).sum()); - for run in runs { - for r in 0..run.row_count { - indices.push((run.input_index, run.start_row + r)); - } - } - - // Split into OUTPUT_PAGE_ROWS-sized chunks; each chunk → one - // arrow::interleave call → one ArrayRef. - let mut pages = Vec::with_capacity(indices.len().div_ceil(OUTPUT_PAGE_ROWS)); - for chunk in indices.chunks(OUTPUT_PAGE_ROWS) { - let arr = interleave(&input_arrays, chunk) - .with_context(|| format!("interleaving sort col '{col_name}' pages"))?; - pages.push(arr); - } - Ok(pages) -} - -#[allow(clippy::too_many_arguments)] -fn write_body_col_for_all_outputs( - handle: &Handle, - row_groups: &mut [crate::storage::streaming_writer::RowGroupBuilder<'_, std::fs::File>], - service_names_per_output: &mut [HashSet], - writer_index_view: &[(usize, usize)], - decoders_state: &mut [InputDecoderState], - col_name: &str, - destinations: &InputRowDestinations, - per_output_schemas: &[SchemaRef], -) -> Result<()> { - // Find this col's per-input parquet leaf index (one per input). - // Inputs whose schema doesn't have this col OR which have zero - // row groups (legal — phase 0 explicitly accepts empty inputs and - // returns an empty sort batch for them) contribute null rows and - // don't advance their decoder for this col. Looking up - // `row_group(0)` on a zero-RG input would panic, so guard up - // front. - let mut input_col_indices: Vec> = Vec::with_capacity(decoders_state.len()); - for state in decoders_state.iter() { - if state.metadata.num_row_groups() == 0 { - input_col_indices.push(None); - continue; - } - match state.arrow_schema.index_of(col_name) { - Ok(idx) => input_col_indices.push(Some(idx)), - Err(_) => input_col_indices.push(None), - } - } - - // Reset each input's body-col cache + cursor at the start of this - // column. The persistent `StreamDecoder` retains its per-(rg, col) - // state for *every* column it has touched so the next page from - // this column has the correct `row_start`; only the cached pages - // (which belong to the previous column) need to be discarded. - for state in decoders_state.iter_mut() { - state.reset_body_col_state(); - } - - // Track service names while streaming the service col. - let track_service = col_name == "service"; - - // For each output sequentially: build output pages, feed to writer - // one page at a time. We must NOT collect the whole column into a - // Vec — that would defeat the page-bounded merge path and scale - // memory with column-chunk size on production splits. Instead we - // hand `write_next_column_arrays` a streaming iterator that - // captures the first error in a side cell so the writer stops as - // soon as assembly fails. - let mut storage_idx = 0; - for (out_idx, &row_count) in destinations.rows_per_output.iter().enumerate() { - if row_count == 0 { - continue; - } - debug_assert_eq!(writer_index_view[storage_idx].0, out_idx); - - let out_schema = &per_output_schemas[out_idx]; - if out_schema.index_of(col_name).is_err() { - storage_idx += 1; - continue; - } - let out_field_idx = out_schema.index_of(col_name)?; - let out_field = out_schema.field(out_field_idx); - - let assembler = BodyColOutputPageAssembler::new( - handle, - decoders_state, - &input_col_indices, - destinations, - out_idx, - col_name, - out_field, - ); - - let mut error_slot: Option = None; - let service_collector: Option<&mut HashSet> = if track_service { - Some(&mut service_names_per_output[storage_idx]) - } else { - None - }; - - let stream_iter = StreamingBodyColIter { - inner: assembler.into_iter(), - error_slot: &mut error_slot, - service_collector, - }; - - let write_result = row_groups[storage_idx].write_next_column_arrays(stream_iter); - - // Assembly errors are reported via `error_slot`; surface them - // first because a downstream write error is usually a - // consequence (the writer stops on `None` and reports a - // row-count mismatch otherwise). - if let Some(err) = error_slot { - return Err(err) - .with_context(|| format!("assembling body col '{col_name}' for output {out_idx}")); - } - write_result - .with_context(|| format!("writing body col '{col_name}' to output {out_idx}"))?; - storage_idx += 1; - } - - Ok(()) -} - -/// Adapts a `Result` page assembler into the -/// `Iterator` shape `write_next_column_arrays` expects. -/// The first assembly error is captured in `error_slot` and iteration -/// ends; the caller MUST check the slot after the writer returns. If -/// `service_collector` is `Some`, every yielded page is scanned for -/// service names and added to the set; collection failures also stop -/// the iterator and populate `error_slot`. -struct StreamingBodyColIter<'a, I> { - inner: I, - error_slot: &'a mut Option, - service_collector: Option<&'a mut HashSet>, -} - -impl Iterator for StreamingBodyColIter<'_, I> -where I: Iterator> -{ - type Item = ArrayRef; - - fn next(&mut self) -> Option { - if self.error_slot.is_some() { - return None; - } - match self.inner.next() { - Some(Ok(arr)) => { - if let Some(out) = self.service_collector.as_deref_mut() - && let Err(e) = collect_service_names_from_page(arr.as_ref(), out) - { - *self.error_slot = Some(e); - return None; - } - Some(arr) - } - Some(Err(e)) => { - *self.error_slot = Some(e); - None - } - None => None, - } - } -} - -/// Per-page service name collector. Used during the streaming write -/// of the "service" body col to populate per-output service_names. -fn collect_service_names_from_page(arr: &dyn Array, out: &mut HashSet) -> Result<()> { - use arrow::array::AsArray; - use arrow::datatypes::{Int8Type, Int16Type, Int32Type, Int64Type}; - - fn extend_from_strings(strings: &arrow::array::StringArray, out: &mut HashSet) { - for i in 0..strings.len() { - if strings.is_valid(i) { - out.insert(strings.value(i).to_string()); - } - } - } - - match arr.data_type() { - DataType::Utf8 => { - let strings = arr - .as_any() - .downcast_ref::() - .ok_or_else(|| anyhow!("expected StringArray for service col page"))?; - extend_from_strings(strings, out); - } - DataType::LargeUtf8 => { - let strings = arr - .as_any() - .downcast_ref::() - .ok_or_else(|| anyhow!("expected LargeStringArray for service col page"))?; - for i in 0..strings.len() { - if strings.is_valid(i) { - out.insert(strings.value(i).to_string()); - } - } - } - DataType::Dictionary(key_type, value_type) - if matches!(value_type.as_ref(), DataType::Utf8) => - { - // Extract the dictionary's values that are referenced by - // valid (non-null) keys. - match key_type.as_ref() { - DataType::Int8 => { - let dict = arr.as_dictionary::(); - if let Some(strings) = dict - .values() - .as_any() - .downcast_ref::() - { - for i in 0..dict.len() { - if dict.is_valid(i) { - let key = dict.keys().value(i) as usize; - if key < strings.len() && strings.is_valid(key) { - out.insert(strings.value(key).to_string()); - } - } - } - } - } - DataType::Int16 => { - let dict = arr.as_dictionary::(); - if let Some(strings) = dict - .values() - .as_any() - .downcast_ref::() - { - for i in 0..dict.len() { - if dict.is_valid(i) { - let key = dict.keys().value(i) as usize; - if key < strings.len() && strings.is_valid(key) { - out.insert(strings.value(key).to_string()); - } - } - } - } - } - DataType::Int32 => { - let dict = arr.as_dictionary::(); - if let Some(strings) = dict - .values() - .as_any() - .downcast_ref::() - { - for i in 0..dict.len() { - if dict.is_valid(i) { - let key = dict.keys().value(i) as usize; - if key < strings.len() && strings.is_valid(key) { - out.insert(strings.value(key).to_string()); - } - } - } - } - } - DataType::Int64 => { - let dict = arr.as_dictionary::(); - if let Some(strings) = dict - .values() - .as_any() - .downcast_ref::() - { - for i in 0..dict.len() { - if dict.is_valid(i) { - let key = dict.keys().value(i) as usize; - if key < strings.len() && strings.is_valid(key) { - out.insert(strings.value(key).to_string()); - } - } - } - } - } - _ => {} - } - } - _ => { - // Skip non-string types — service col is expected to be - // string-like; if it isn't, just don't collect names. - } - } - Ok(()) -} - -// ============================================================================ -// Body col output page assembler — the page-bounded streaming core -// ============================================================================ - -/// Assembles output pages for one (output_idx, body_col) by: -/// 1. Walking the destinations table forward through this output's row range, accumulating -/// `(input_idx, input_row)` index pairs. -/// 2. When the index buffer hits `OUTPUT_PAGE_ROWS`, advancing each contributing input's decoder -/// until its decoded pages cover the needed input rows, then calling -/// `arrow::compute::interleave`. -/// 3. Emitting one `ArrayRef` per iter step until the row range is exhausted; then `Ok(None)`. -/// -/// Memory per `next()` call: one in-progress output page (P rows) + -/// up to ~2 in-flight decoded pages per input (kept until all their -/// rows are consumed). Bounded by page sizes, not column-chunk sizes. -/// -/// **Cross-output state**: the per-input `body_col_page_cache` and -/// `body_col_cursor` live on [`InputDecoderState`], not the assembler. -/// They must persist across all outputs that consume rows from the -/// active body column — a page whose row range straddles two outputs -/// would otherwise be dropped when the first output's assembler ends, -/// even though the stream has already advanced past it and the next -/// output still needs rows from inside that page. -struct BodyColOutputPageAssembler<'a> { - handle: &'a Handle, - decoders_state: &'a mut [InputDecoderState], - input_col_indices: &'a [Option], - destinations: &'a InputRowDestinations, - out_idx: usize, - col_name: &'a str, - out_field: &'a Field, - /// Total rows written so far for this output's col. - rows_emitted: usize, - /// Total rows expected = destinations.rows_per_output[out_idx]. - expected_rows: usize, - /// EOF flag (returns None on subsequent calls once true). - done: bool, -} - -impl<'a> BodyColOutputPageAssembler<'a> { - #[allow(clippy::too_many_arguments)] - fn new( - handle: &'a Handle, - decoders_state: &'a mut [InputDecoderState], - input_col_indices: &'a [Option], - destinations: &'a InputRowDestinations, - out_idx: usize, - col_name: &'a str, - out_field: &'a Field, - ) -> Self { - Self { - handle, - decoders_state, - input_col_indices, - destinations, - out_idx, - col_name, - out_field, - rows_emitted: 0, - expected_rows: destinations.rows_per_output[out_idx], - done: false, - } - } - - fn into_iter(self) -> BodyColOutputPageIter<'a> { - BodyColOutputPageIter { inner: self } - } -} - -struct BodyColOutputPageIter<'a> { - inner: BodyColOutputPageAssembler<'a>, -} - -impl Iterator for BodyColOutputPageIter<'_> { - type Item = Result; - - fn next(&mut self) -> Option { - if self.inner.done || self.inner.rows_emitted >= self.inner.expected_rows { - self.inner.done = true; - return None; - } - match assemble_one_output_page(&mut self.inner) { - Ok(Some(arr)) => Some(Ok(arr)), - Ok(None) => { - self.inner.done = true; - None - } - Err(e) => { - self.inner.done = true; - Some(Err(e)) - } - } - } -} - -fn assemble_one_output_page(s: &mut BodyColOutputPageAssembler) -> Result> { - let remaining = s.expected_rows - s.rows_emitted; - if remaining == 0 { - return Ok(None); - } - let page_size = remaining.min(OUTPUT_PAGE_ROWS); - - // Walk this output's row positions and figure out which (input, input_row) - // contributes each one. We use the per-input destinations table: for - // input i, find the next input_row whose destination is (out_idx, *). - // Since `destinations.per_input[i]` is in input order and outputs are - // strictly increasing by sort key, the rows that go to this output are - // a contiguous slice in input i's row order. - // - // For each output position 0..page_size, we need (input_idx, input_row). - // Walk input cursors and pick the next row going to this output. - - // Collect (input_idx, input_row) indices for this output page. - let mut indices_per_input: Vec> = vec![Vec::new(); s.decoders_state.len()]; - let mut interleave_indices: Vec<(usize, usize)> = Vec::with_capacity(page_size); - let mut total_picked = 0usize; - - while total_picked < page_size { - // Look across all inputs for the next contribution to this output. - // Per the merge order, within each input the rows assigned to this - // output are a contiguous slice; once we've advanced cursor past - // them, no more rows from this input contribute. We collect ALL - // rows from one input up to a per-input limit determined by the - // merge order, but the simplest correct approach is to walk in - // merge-order globally. We don't have the merge order indexed by - // output here, so re-derive by scanning the destinations table. - // - // Better: pre-compute per-output, per-input row ranges. Each input - // contributes a contiguous half-open range `[lo_i..hi_i)` to this - // output (possibly empty). We could compute these ranges once and - // reuse. For now, lazy approach: scan forward from cursor on each - // input, picking the next row that maps to (out_idx, *). - // - // The ORDER in which we pick across inputs must match the merge - // plan's output position. We have output positions in destinations: - // `destinations.per_input[i][r] = Some((out_idx, pos))`. The merged - // output picks rows in order of increasing `pos`. - // - // For one output page, the positions we want are - // `s.rows_emitted..s.rows_emitted + page_size`. For each position - // p in that range, find (input_idx, input_row) such that - // destinations.per_input[input_idx][input_row] == Some((out_idx, p)). - let target_pos = s.rows_emitted + total_picked; - let mut found = false; - for (input_idx, dests) in s.destinations.per_input.iter().enumerate() { - let cursor = s.decoders_state[input_idx].body_col_cursor; - for (input_row, dest) in dests.iter().enumerate().skip(cursor) { - match dest { - Some((o, p)) if *o == s.out_idx => { - if *p == target_pos { - interleave_indices.push((input_idx, input_row)); - indices_per_input[input_idx].push(input_row); - // Don't advance the cursor past this row yet — - // we may need rows from input i in this page - // with positions ahead. We bump it after the - // whole page is collected. - found = true; - break; - } - } - _ => {} - } - if found { - break; - } - } - if found { - break; - } - } - if !found { - // Shouldn't happen — every output position should be reachable. - bail!( - "merge plan inconsistency: output {} position {target_pos} not found in any input", - s.out_idx, - ); - } - total_picked += 1; - } - - // Now ensure each input's decoder has decoded pages covering all - // `indices_per_input[i]` rows. Advance decoders as needed. - for (input_idx, input_rows) in indices_per_input.iter().enumerate() { - if input_rows.is_empty() { - continue; - } - let col_parquet_idx = match s.input_col_indices[input_idx] { - Some(c) => c, - None => { - // This input lacks this col entirely — null contributions. - // We'll handle null-filling in the interleave step below. - continue; - } - }; - let max_needed_row = *input_rows.iter().max().expect("non-empty"); - fill_page_cache_to_row( - s.handle, - &mut s.decoders_state[input_idx], - col_parquet_idx, - max_needed_row, - )?; - } - - // Build the per-(input, row) value array by: - // 1. Concatenating each input's cached pages into one ArrayRef (they cover a contiguous input - // row range from cache_start to cursor_max). - // 2. Computing local indices = input_row - cache_start. - // 3. Calling arrow::compute::interleave across N input arrays. - // - // For inputs without this col, we substitute a single null page of the - // out_field's type. - let mut input_array_refs: Vec = Vec::with_capacity(s.decoders_state.len()); - let mut input_cache_starts: Vec = Vec::with_capacity(s.decoders_state.len()); - - for input_idx in 0..s.decoders_state.len() { - match s.input_col_indices[input_idx] { - Some(_) => { - let pages = &s.decoders_state[input_idx].body_col_page_cache; - if pages.is_empty() { - // No pages decoded for this input (no rows from this input go to this output). - // Use a zero-row placeholder; we won't index into it. - input_array_refs.push(new_null_array(s.out_field.data_type(), 0)); - input_cache_starts.push(0); - } else { - let cache_start = pages[0].row_start; - let arrays: Vec<&dyn Array> = pages.iter().map(|p| p.array.as_ref()).collect(); - let concatenated = arrow::compute::concat(&arrays).with_context(|| { - format!( - "concatenating cached pages for input {input_idx} col '{}'", - s.col_name, - ) - })?; - input_array_refs.push(concatenated); - input_cache_starts.push(cache_start); - } - } - None => { - // Null-fill array of the right length. The max needed local - // index from this input is the largest index we'd reference; - // since we don't actually reference rows from this input (we'd - // need an alternate "null contribution" mechanism), we leave - // it as a 1-row null array and route indices to position 0. - let null_arr = new_null_array(s.out_field.data_type(), 1); - input_array_refs.push(null_arr); - input_cache_starts.push(0); - } - } - } - - let interleave_local: Vec<(usize, usize)> = interleave_indices - .iter() - .map(|&(i_idx, i_row)| match s.input_col_indices[i_idx] { - Some(_) => (i_idx, i_row - input_cache_starts[i_idx]), - None => (i_idx, 0), - }) - .collect(); - - let array_refs_ref: Vec<&dyn Array> = input_array_refs.iter().map(|a| a.as_ref()).collect(); - let assembled = interleave(&array_refs_ref, &interleave_local).with_context(|| { - format!( - "interleaving body col '{}' for output {}", - s.col_name, s.out_idx, - ) - })?; - - // Bump input cursors past rows we just consumed and drop pages - // whose rows are fully consumed. Both the cursor and the cache - // live on InputDecoderState so they persist across outputs that - // share this column. - for (input_idx, input_rows) in indices_per_input.iter().enumerate() { - if input_rows.is_empty() { - continue; - } - let max_row = *input_rows.iter().max().expect("non-empty"); - let state = &mut s.decoders_state[input_idx]; - state.body_col_cursor = max_row + 1; - - // Drop pages whose last row is < cursor. - if s.input_col_indices[input_idx].is_some() { - let pages = &mut state.body_col_page_cache; - while let Some(front) = pages.first() { - let front_end = front.row_start + front.array.len(); - if front_end <= state.body_col_cursor { - pages.remove(0); - } else { - break; - } - } - } - } - - s.rows_emitted += page_size; - Ok(Some(assembled)) -} - -/// Pull pages from the input's persistent decoder via `block_on` until -/// the cached pages for `col_parquet_idx` cover up through `target_row` -/// (inclusive). Stops as soon as the latest cached page ends past -/// `target_row`. The function's effect on the world is *adding pages -/// to the cache* — it does not skip data and does not consume any -/// rows on its own. -/// -/// The decoder MUST be the long-lived [`InputDecoderState::decoder`]: -/// it preserves the per-(rg, col) `rows_decoded` counter so successive -/// `DecodedPage::row_start` values are absolute input row indices, -/// not page-local zeros. Likewise, the cache lives on the state so -/// pages whose row range spans an output boundary survive into the -/// next output's assembler. -fn fill_page_cache_to_row( - handle: &Handle, - state: &mut InputDecoderState, - col_parquet_idx: usize, - target_row: usize, -) -> Result<()> { - // If cache already covers target_row, nothing to do. - if let Some(last) = state.body_col_page_cache.last() { - let last_end = last.row_start + last.array.len(); - if target_row < last_end { - return Ok(()); - } - } - - loop { - let decoded = handle - .block_on(state.decoder.decode_next_page()) - .context("decoding body col page")?; - let page = match decoded { - Some(p) => p, - None => bail!( - "stream EOF while advancing to row {target_row} for parquet col {col_parquet_idx}", - ), - }; - if page.col_idx != col_parquet_idx { - bail!( - "expected col {col_parquet_idx} page, got col {} — column ordering violated", - page.col_idx, - ); - } - let end = page.row_start + page.array.len(); - state.body_col_page_cache.push(page); - record_body_col_page_cache_len(state.body_col_page_cache.len()); - if target_row < end { - return Ok(()); - } - } -} - -fn finalize_output_writer( - state: OutputWriterStorage, - per_output_static: &[PerOutputStatic], -) -> Result { - let OutputWriterStorage { - output_idx, - output_path, - writer, - mut service_names, - num_rows, - } = state; - - let _metadata = writer - .close() - .with_context(|| format!("closing writer for output {output_idx}"))?; - - let size_bytes = std::fs::metadata(&output_path) - .with_context(|| format!("stat output file: {}", output_path.display()))? - .len(); - - let static_meta = &per_output_static[output_idx]; - - // If `service` is a sort column for this schema, it took the - // sort-col write path and `service_names` (populated by the body- - // col `track_service` branch) never saw it. Fold in the names - // from the per-output sort batch so the `TAG_SERVICE` low- - // cardinality metadata stays accurate regardless of which path - // wrote the column. - if let Ok(service_col_idx) = static_meta.sort_optimised.schema().index_of("service") { - collect_service_names_from_page( - static_meta.sort_optimised.column(service_col_idx).as_ref(), - &mut service_names, - ) - .with_context(|| { - format!("collecting service names from sort col for output {output_idx}") - })?; - } - - let mut low_cardinality_tags: HashMap> = HashMap::new(); - if !service_names.is_empty() { - low_cardinality_tags.insert(TAG_SERVICE.to_string(), service_names); - } - - Ok(MergeOutputFile { - path: output_path, - num_rows, - num_row_groups: 1, - size_bytes, - row_keys_proto: static_meta.row_keys_proto.clone(), - zonemap_regexes: static_meta.zonemap_regexes.clone(), - metric_names: static_meta.metric_names.clone(), - time_range: static_meta.time_range, - low_cardinality_tags, - }) -} - -// ============================================================================ -// Tests -// ============================================================================ - -#[cfg(test)] -mod tests { - use std::path::PathBuf; - use std::sync::Arc; - - use arrow::array::{ - ArrayRef, BinaryArray, DictionaryArray, Float64Array, Int64Array, StringArray, UInt8Array, - UInt64Array, - }; - use arrow::datatypes::{DataType, Field, Int32Type, Schema as ArrowSchema}; - use bytes::Bytes; - use parquet::arrow::ArrowWriter; - use parquet::file::metadata::KeyValue; - use parquet::file::properties::WriterProperties; - use parquet::file::reader::{FileReader, SerializedFileReader}; - use tempfile::TempDir; - use tokio::io::AsyncRead; - - use super::*; - use crate::storage::page_decoder::StreamDecoder; - use crate::storage::streaming_reader::{RemoteByteSource, StreamingParquetReader}; - use crate::storage::{Compression, ParquetWriterConfig}; - - // -------- Fixtures -------- + // -------- Fixtures -------- /// Build a sorted metrics RecordBatch with `num_rows` rows in /// the **storage column order**: sort cols (metric_name, timestamp_secs) @@ -2348,87 +1783,1381 @@ mod tests { .await .expect("merge"); - let bytes = std::fs::read(&outputs[0].path).expect("read"); - let mut output_stream = open_stream(Bytes::from(bytes)).await; - let mut decoder = StreamDecoder::new(&mut *output_stream); - let mut total_decoded = 0usize; - while let Some(page) = decoder.decode_next_page().await.expect("decode") { - // Count only sort col 0 (col_idx 0) pages to get a row count. - if page.col_idx == 0 { - total_decoded += page.array.len(); + let bytes = std::fs::read(&outputs[0].path).expect("read"); + let mut output_stream = open_stream(Bytes::from(bytes)).await; + let mut decoder = StreamDecoder::new(&mut *output_stream); + let mut total_decoded = 0usize; + while let Some(page) = decoder.decode_next_page().await.expect("decode") { + // Count only sort col 0 (col_idx 0) pages to get a row count. + if page.col_idx == 0 { + total_decoded += page.array.len(); + } + } + assert_eq!(total_decoded, 80); + } + + /// Page-bounded contract sanity: with a row group large enough to + /// require many parquet pages per col, body col writes go through + /// the page-by-page assembler instead of materialising column + /// chunks. We can't directly observe peak memory from a test, but + /// we *can* assert that the merge completes correctly with input + /// data whose body cols span many pages, and that the output is + /// itself multi-page (no whole-column buffering happened on the + /// output side either). + #[tokio::test] + async fn test_body_col_streams_many_pages_per_column_chunk() { + // Force multiple pages per column chunk by setting a small + // data_page_row_count_limit. With 8000 rows and a 1000-row + // page limit, the output value col chunk must span ≥ 8 pages. + let batch = make_sorted_batch(8000, 0); + let bytes = write_input_parquet(std::slice::from_ref(&batch), &[]); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let writer_config = ParquetWriterConfig { + compression: Compression::Snappy, + data_page_row_count_limit: 1000, + ..ParquetWriterConfig::default() + }; + let config = MergeConfig { + num_outputs: 1, + writer_config, + }; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &config) + .await + .expect("merge"); + assert_eq!(outputs.len(), 1); + assert_eq!(outputs[0].num_rows, 8000); + + // Verify the output is itself multi-page-per-column (which is + // what page-bounded writing should produce, given the default + // data_page_size). Read via the page-bounded decoder and count + // pages for the value column. + let out_bytes = std::fs::read(&outputs[0].path).expect("read"); + let mut output_stream = open_stream(Bytes::from(out_bytes)).await; + // Find the "value" col index in the output's arrow schema BEFORE + // borrowing output_stream mutably for the decoder. + let arrow_schema = parquet::arrow::parquet_to_arrow_schema( + output_stream.metadata().file_metadata().schema_descr(), + None, + ) + .expect("arrow schema"); + let value_col_idx = arrow_schema.index_of("value").expect("value col"); + let mut decoder = StreamDecoder::new(&mut *output_stream); + + let mut value_pages = 0; + while let Some(page) = decoder.decode_next_page().await.expect("decode") { + if page.col_idx == value_col_idx { + value_pages += 1; + } + } + assert!( + value_pages >= 2, + "expected output 'value' col to span multiple pages (got {value_pages}); body col \ + writes should respect data_page_size", + ); + } + + /// Multi-RG metric-aligned input (`prefix_len >= 1`) is accepted + /// and produces multi-RG output: one output RG per input metric_name + /// region. + #[tokio::test] + async fn test_multi_rg_metric_aligned_input_produces_multi_rg_output() { + // Build a fixture with 2 metric_names → 2 RGs each holding one + // metric_name. Use `prefix_len = 1` to declare metric_name + // alignment. + let bytes = make_two_metric_aligned_input(); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge multi-RG metric-aligned input"); + assert_eq!(outputs.len(), 1, "expected one output file"); + assert_eq!(outputs[0].num_rows, 60, "30 + 30 rows"); + + let out_bytes = std::fs::read(&outputs[0].path).expect("read"); + let reader = SerializedFileReader::new(Bytes::from(out_bytes)).expect("ser"); + assert_eq!( + reader.metadata().num_row_groups(), + 2, + "multi-RG metric-aligned input must produce multi-RG output (one RG per metric_name \ + region)", + ); + + // `MergeOutputFile.num_row_groups` must agree with the file + // on disk. Before the fix it was hard-coded to 1, so this + // assertion caught the regression on a multi-region output. + assert_eq!( + outputs[0].num_row_groups, 2, + "MergeOutputFile.num_row_groups should match physical row group count", + ); + } + + /// Regression for Codex P2 on PR-6410: a streaming merge output + /// whose multi-region content lands in a single file must report + /// `MergeOutputFile.num_row_groups` consistent with the parquet + /// footer. Two regions assigned to one output should yield + /// `num_row_groups = 2`. + #[tokio::test] + async fn test_streaming_output_num_row_groups_matches_footer() { + let bytes = make_two_metric_aligned_input(); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge"); + + let bytes_out = std::fs::read(&outputs[0].path).expect("read"); + let reader = SerializedFileReader::new(Bytes::from(bytes_out)).expect("ser"); + assert_eq!( + outputs[0].num_row_groups, + reader.metadata().num_row_groups(), + "MergeOutputFile.num_row_groups must match the physical RG count", + ); + } + + /// Regression for Codex P2 on PR-6410: `qh.row_keys` and + /// `qh.zonemap_regexes` must be written into the on-disk Parquet + /// footer for every streaming output. Per-output values come from + /// the rows that landed in that output specifically — merging + /// eliminates key overlap between outputs, so this metadata can't + /// be carried over from inputs. + #[tokio::test] + async fn test_streaming_output_kv_footer_contains_row_keys_and_zonemap() { + let bytes = make_two_metric_aligned_input(); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge"); + + let bytes_out = std::fs::read(&outputs[0].path).expect("read"); + let reader = SerializedFileReader::new(Bytes::from(bytes_out)).expect("ser"); + let kvs: Vec<(String, Option)> = reader + .metadata() + .file_metadata() + .key_value_metadata() + .map(|v| { + v.iter() + .map(|kv| (kv.key.clone(), kv.value.clone())) + .collect() + }) + .unwrap_or_default(); + let find = |k: &str| kvs.iter().find(|(key, _)| key == k).map(|(_, v)| v.clone()); + + // row_keys: streaming merge derives them from THIS output's + // sort cols, and the in-memory `MergeOutputFile` records the + // proto bytes. The base64-encoded KV in the footer must + // round-trip to the same bytes. + assert!( + outputs[0].row_keys_proto.is_some(), + "expected per-output row_keys_proto in MergeOutputFile", + ); + let row_keys_kv = find(PARQUET_META_ROW_KEYS).flatten().expect( + "qh.row_keys missing from streaming output footer — appended-after-write KV metadata \ + did not survive close", + ); + let decoded = BASE64.decode(row_keys_kv).expect("base64 decode"); + assert_eq!( + &decoded, + outputs[0].row_keys_proto.as_ref().unwrap(), + "footer row_keys bytes must equal MergeOutputFile.row_keys_proto", + ); + + // zonemap_regexes: footer carries a JSON-encoded map. + // metric_name alignment + multiple metric_names → non-empty. + assert!( + !outputs[0].zonemap_regexes.is_empty(), + "expected non-empty zonemap_regexes for multi-metric output", + ); + let zonemap_kv = find(PARQUET_META_ZONEMAP_REGEXES) + .flatten() + .expect("qh.zonemap_regexes missing from streaming output footer"); + let parsed: HashMap = + serde_json::from_str(&zonemap_kv).expect("zonemap JSON parse"); + assert_eq!( + parsed, outputs[0].zonemap_regexes, + "footer zonemap must equal MergeOutputFile.zonemap_regexes", + ); + } + + /// Composite-key extraction with two byte-array prefix columns: + /// `(metric_name, service)` ASC/ASC. Two RGs with the same + /// metric_name but different services must produce distinct + /// composite keys, with byte-lex order matching the natural + /// `(metric_name, service)` sort order. + #[test] + fn test_extract_rg_composite_prefix_key_two_byte_array_cols() { + let bytes = + make_prefix_len_two_input(&[("cpu.usage", "host-a"), ("cpu.usage", "host-b")], 10); + let reader = SerializedFileReader::new(bytes).expect("ser"); + let metadata = reader.metadata(); + assert_eq!(metadata.num_row_groups(), 2); + + let prefix_cols = find_prefix_parquet_col_indices( + metadata, + "metric_name|service|-timestamp_secs/V2", + 2, + 0, + ) + .expect("resolve"); + let key_rg0 = extract_rg_composite_prefix_key(metadata, 0, &prefix_cols, 0).expect("rg0"); + let key_rg1 = extract_rg_composite_prefix_key(metadata, 1, &prefix_cols, 0).expect("rg1"); + assert_ne!( + key_rg0, key_rg1, + "different services → different composite keys" + ); + // metric_name is equal across RGs; ordering must come from + // service ('host-a' < 'host-b' lex), so key_rg0 < key_rg1. + assert!( + key_rg0 < key_rg1, + "composite key for ('cpu.usage', 'host-a') must lex-sort before ('cpu.usage', \ + 'host-b')", + ); + + // Encoded representation under the storekey-based encoding + // (shared with `sorted_series` via `append_prefix_col_to_key`): + // each prefix column contributes `storekey(u8 ord) || + // storekey(str value)`, then the whole key ends with a + // `u8(prefix_len)` sentinel ordinal (so an all-null RG's + // empty body still sorts after any non-null key — see + // `extract_rg_composite_prefix_key` for the argument). + // + // [0x00] ord=0 (metric_name) + // b"cpu.usage" + 0x00 storekey("cpu.usage") — 10 bytes + // [0x01] ord=1 (service) + // b"host-a" + 0x00 storekey("host-a") — 7 bytes + // [0x02] sentinel u8(prefix_len) + // + // Total = 1 + 10 + 1 + 7 + 1 = 20 bytes. + assert_eq!(key_rg0[0], 0x00); + assert_eq!(&key_rg0[1..10], b"cpu.usage"); + assert_eq!(key_rg0[10], 0x00); + assert_eq!(key_rg0[11], 0x01); + assert_eq!(&key_rg0[12..18], b"host-a"); + assert_eq!(key_rg0[18], 0x00); + assert_eq!(key_rg0[19], 0x02); + assert_eq!(key_rg0.len(), 20); + } + + /// Regression for Codex P1 on PR-6410 (positive coverage of the + /// fix): `rg_partition_prefix_len = 2` groups RGs by the + /// composite (metric_name, service) value, producing one output + /// row group per (metric_name, service) pair. Two RGs that share + /// metric_name but differ in service must NOT be folded into one + /// region. + #[tokio::test] + async fn test_streaming_merge_with_prefix_len_two() { + let bytes = make_prefix_len_two_input( + &[ + ("cpu.usage", "host-a"), + ("cpu.usage", "host-b"), + ("memory.used", "host-a"), + ], + 20, + ); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge with prefix_len = 2"); + assert_eq!(outputs.len(), 1, "expected one output file"); + assert_eq!(outputs[0].num_rows, 60, "20 × 3 input RGs = 60 rows"); + + let bytes_out = std::fs::read(&outputs[0].path).expect("read"); + let reader = SerializedFileReader::new(Bytes::from(bytes_out)).expect("ser"); + // Three distinct (metric_name, service) tuples → three output + // row groups. With prefix_len truncated to 1 (the pre-fix + // bug) the two cpu.usage RGs would have folded into one + // region and only two output RGs would be written. + assert_eq!( + reader.metadata().num_row_groups(), + 3, + "three distinct (metric_name, service) pairs must produce three output RGs", + ); + assert_eq!(outputs[0].num_row_groups, 3); + } + + /// Regression for Codex finding #1 on PR-6410: when one input + /// file has two RGs sharing the same composite prefix key, the + /// streaming engine must reject up-front. Without the check, + /// `process_region` keys `sort_col_batches` by input_idx, so the + /// second RG silently overwrites the first while + /// `Region::total_rows` keeps counting both — rows would be + /// dropped and the body-col / sort-col mapping would be off by a + /// full RG. + /// + /// The fixture passes `("cpu.usage", "host-a")` twice, producing + /// an input with two RGs that have identical (metric_name, + /// service) statistics. The merge must bail with a clear error + /// pointing at the duplicate. + #[tokio::test] + async fn test_streaming_merge_rejects_duplicate_prefix_rgs_in_one_input() { + let bytes = make_prefix_len_two_input( + &[ + ("cpu.usage", "host-a"), + ("cpu.usage", "host-a"), // duplicate prefix key + ("memory.used", "host-a"), + ], + 20, + ); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let err = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect_err("must reject input with duplicate prefix RGs"); + let s = err.to_string(); + assert!( + s.contains("sharing a prefix key with an earlier RG"), + "expected duplicate-prefix error, got: {s}", + ); + assert!( + s.contains("input 0"), + "error should identify which input is bad, got: {s}", + ); + } + + /// Build a single-RG parquet file with a nullable `metric_name` + /// prefix column. `values` is one entry per row (`None` for a + /// null cell). `prefix_len` is stamped in the file metadata so + /// the streaming merge treats the file as prefix-aligned. + fn make_nullable_prefix_input_single_rg(values: &[Option<&str>], prefix_len: u32) -> Bytes { + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("metric_name", DataType::Utf8, true), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("sorted_series", DataType::Binary, false), + Field::new("value", DataType::Float64, false), + ])); + let n = values.len(); + let metric_name: ArrayRef = Arc::new(StringArray::from( + values + .iter() + .map(|v| v.map(str::to_string)) + .collect::>(), + )); + let timestamps: Vec = (0..n as u64).map(|i| 1_700_000_000 + i).collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let series_bytes: Vec> = (0..n as u64).map(|i| i.to_be_bytes().to_vec()).collect(); + let series_refs: Vec<&[u8]> = series_bytes.iter().map(|v| v.as_slice()).collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from(series_refs)); + let value: ArrayRef = Arc::new(Float64Array::from( + (0..n).map(|i| i as f64).collect::>(), + )); + let batch = RecordBatch::try_new( + schema.clone(), + vec![metric_name, timestamp_secs, sorted_series, value], + ) + .expect("test batch"); + + let cfg = ParquetWriterConfig { + compression: Compression::Snappy, + row_group_size: n.max(1), + ..ParquetWriterConfig::default() + }; + let kvs = vec![ + KeyValue::new( + PARQUET_META_SORT_FIELDS.to_string(), + "metric_name|-timestamp_secs/V2".to_string(), + ), + KeyValue::new( + PARQUET_META_WINDOW_START.to_string(), + "1700000000".to_string(), + ), + KeyValue::new(PARQUET_META_WINDOW_DURATION.to_string(), "60".to_string()), + KeyValue::new(PARQUET_META_NUM_MERGE_OPS.to_string(), "0".to_string()), + KeyValue::new( + PARQUET_META_RG_PARTITION_PREFIX_LEN.to_string(), + prefix_len.to_string(), + ), + ]; + let sorting_cols = vec![parquet::file::metadata::SortingColumn { + column_idx: 0, + descending: false, + nulls_first: false, + }]; + let props: WriterProperties = cfg.to_writer_properties_with_metadata( + &schema, + sorting_cols, + Some(kvs), + &["metric_name".to_string()], + ); + let mut buf: Vec = Vec::new(); + let mut writer = ArrowWriter::try_new(&mut buf, schema, Some(props)).expect("arrow writer"); + writer.write(&batch).expect("write"); + writer.close().expect("close"); + Bytes::from(buf) + } + + /// Regression for Codex P1 on PR #6424 (null prefix handling): an + /// RG with mixed null + non-null cells on a nullable prefix + /// column is NOT prefix-aligned (PA-1). The legacy `min == max` + /// check would silently accept it because Parquet stats hide + /// nulls from min/max, so a single non-null cell + N nulls + /// reports `min == max == non_null_value`. The fix reads + /// `null_count` from stats and bails when it's non-zero. + #[tokio::test] + async fn test_mixed_null_and_value_prefix_rg_rejected() { + // 1 RG, 4 rows: 3× "cpu.usage" + 1× null. Stats record + // `min == max == "cpu.usage"`, `null_count == 1`. + let bytes = make_nullable_prefix_input_single_rg( + &[ + Some("cpu.usage"), + Some("cpu.usage"), + None, + Some("cpu.usage"), + ], + 1, + ); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let err = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect_err("mixed null + non-null must be rejected"); + let s = format!("{err:#}"); + assert!( + s.contains("NOT prefix-aligned") && s.contains("nulls plus"), + "expected PA-1 mixed-null error, got: {s}", + ); + } + + /// Regression for Codex P1 on PR #6424 (null prefix handling): + /// an all-null prefix RG is logically aligned (its single prefix + /// "value" is null) and must successfully merge. The + /// composite-prefix encoding skips the column entirely for the + /// all-null RG; in a multi-input setup that mixes non-null and + /// all-null contributions on the same prefix column, BTreeMap + /// iteration order puts the all-null region AFTER any + /// non-null-prefix region (nulls-last), matching `sorted_series`'s + /// row-level null-skip convention. + #[tokio::test] + async fn test_all_null_prefix_rg_groups_into_separate_region_sorted_last() { + // Input A: one RG, metric_name = "cpu.usage" for all rows. + // Input B: one RG, metric_name = NULL for all rows. + // prefix_len = 1, merge into a single output. Each input + // contributes its own region; the all-null region should + // sort after the non-null region, so the merged output's + // RG 0 carries the non-null metric and RG 1 carries the + // all-null one. + let bytes_a = make_nullable_prefix_input_single_rg( + &[Some("cpu.usage"), Some("cpu.usage"), Some("cpu.usage")], + 1, + ); + let bytes_b = make_nullable_prefix_input_single_rg(&[None, None, None], 1); + let inputs: Vec> = + vec![open_stream(bytes_a).await, open_stream(bytes_b).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("all-null prefix RG must merge cleanly via column-skip encoding"); + assert_eq!(outputs.len(), 1); + assert_eq!(outputs[0].num_rows, 6, "3 non-null + 3 null = 6 rows total"); + + let bytes_out = std::fs::read(&outputs[0].path).expect("read"); + let reader = SerializedFileReader::new(Bytes::from(bytes_out)).expect("ser"); + assert_eq!( + reader.metadata().num_row_groups(), + 2, + "two regions (non-null + all-null) → two output RGs", + ); + + // Read the output back as Arrow and confirm RG 0 is the + // non-null region (rows have `metric_name = "cpu.usage"`) + // and RG 1 is the all-null region (rows have + // `metric_name = NULL`). This pins the nulls-last ordering + // produced by the composite-prefix's column-skip encoding. + use arrow::array::StringArray; + let combined = read_output_to_record_batch(&outputs[0].path); + let mn_idx = combined.schema().index_of("metric_name").expect("mn col"); + let arr = combined + .column(mn_idx) + .as_any() + .downcast_ref::(); + let arr = arr.expect("metric_name should decode as StringArray"); + // First 3 rows are the non-null region, last 3 are all-null. + for i in 0..3 { + assert!(arr.is_valid(i), "row {i} should be non-null"); + assert_eq!(arr.value(i), "cpu.usage"); + } + for i in 3..6 { + assert!( + arr.is_null(i), + "row {i} should be null (all-null region sorts last)" + ); + } + } + + /// Regression for Codex P1 on PR #6424: when a top-level region + /// exactly fills the current output (so `remaining_in_current == 0`) + /// AND the following prefix-aligned region needs splitting, the + /// split's first-sub-region budget must be computed against the + /// rolled-over fresh output's full target — NOT the stale zero + /// remainder. Before the fix, `split_region_at_sorted_series` cut + /// after the first sorted_series run, producing a tiny leftover + /// plus a large continuation that both shared the parent region's + /// prefix key and landed in the same new output, tripping the + /// PA-3 duplicate-prefix-RG check in `finalize_output`. + /// + /// Setup: three RGs of 50 rows each, distinct (metric, service) + /// prefixes, num_outputs = 3 → target_per_output = 50. Each region + /// should land cleanly in its own output with a single RG. + #[tokio::test] + async fn test_region_exactly_fills_output_does_not_split_next_aligned_region() { + let bytes = make_prefix_len_two_input( + &[ + ("cpu.usage", "host-a"), + ("cpu.usage", "host-b"), + ("memory.used", "host-a"), + ], + 50, + ); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(3)) + .await + .expect("merge must succeed; pre-fix this tripped PA-3 on output 1"); + + assert_eq!(outputs.len(), 3, "three regions → three outputs"); + for (i, out) in outputs.iter().enumerate() { + assert_eq!(out.num_rows, 50, "output {i} should have exactly 50 rows"); + assert_eq!( + out.num_row_groups, 1, + "output {i} should have a single RG (no spurious split)", + ); + } + + // PA-3 verification: each output's single RG carries a + // unique composite prefix key. + for (i, out) in outputs.iter().enumerate() { + let bytes_out = std::fs::read(&out.path).expect("read"); + let reader = SerializedFileReader::new(Bytes::from(bytes_out)).expect("ser"); + assert_unique_rg_prefix_keys( + reader.metadata(), + "metric_name|service|-timestamp_secs/V2", + 2, + &format!("output {i}"), + ) + .expect("each output RG must satisfy PA-1 + PA-3 on prefix"); + } + } + + /// Build a single-RG fixture with multiple metric_names sorted + /// by metric_name then timestamp. The file has + /// `rg_partition_prefix_len = 0` (legacy) so the streaming merge + /// synthesizes prefix-aligned regions during the merge. + /// + /// `metrics` = list of (metric_name, num_rows). Rows are emitted + /// in order so the resulting batch is already sorted by + /// metric_name; each row gets a unique sorted_series identifier + /// derived from its position so the k-way merge has a well- + /// defined order even when other tag dimensions are degenerate. + fn make_multi_metric_single_rg_input(metrics: &[(&str, usize)]) -> Bytes { + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("metric_name", dict_type.clone(), false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("sorted_series", DataType::Binary, false), + Field::new("metric_type", DataType::UInt8, false), + Field::new("service", dict_type, true), + Field::new("timeseries_id", DataType::Int64, false), + Field::new("value", DataType::Float64, false), + ])); + + let total_rows: usize = metrics.iter().map(|(_, n)| *n).sum(); + let mut metric_keys: Vec = Vec::with_capacity(total_rows); + let mut metric_values_vec: Vec<&str> = Vec::with_capacity(metrics.len()); + let mut timestamps: Vec = Vec::with_capacity(total_rows); + let mut series_bytes: Vec> = Vec::with_capacity(total_rows); + let mut tsids: Vec = Vec::with_capacity(total_rows); + let mut values: Vec = Vec::with_capacity(total_rows); + let mut row_idx: u64 = 0; + for (metric_idx, (name, num)) in metrics.iter().enumerate() { + metric_values_vec.push(name); + // -timestamp_secs/V2 in the sort schema means timestamps + // DESC within a metric. + for r in 0..*num { + metric_keys.push(metric_idx as i32); + timestamps.push(1_700_000_000 + ((*num - r) as u64)); + series_bytes.push(row_idx.to_be_bytes().to_vec()); + tsids.push(1000 + row_idx as i64); + values.push(row_idx as f64); + row_idx += 1; + } + } + let metric_names_arr = StringArray::from(metric_values_vec); + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(metric_names_arr), + ) + .expect("dict array"), + ); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let series_refs: Vec<&[u8]> = series_bytes.iter().map(|v| v.as_slice()).collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from(series_refs)); + let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; total_rows])); + let service: ArrayRef = { + let svc_keys: Vec> = (0..total_rows as i32).map(|i| Some(i % 3)).collect(); + let svc_values = StringArray::from(vec!["api", "db", "cache"]); + Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(svc_keys), + Arc::new(svc_values), + ) + .expect("svc dict"), + ) + }; + let timeseries_id: ArrayRef = Arc::new(Int64Array::from(tsids)); + let value: ArrayRef = Arc::new(Float64Array::from(values)); + + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + metric_name, + timestamp_secs, + sorted_series, + metric_type, + service, + timeseries_id, + value, + ], + ) + .expect("batch"); + + // Write as a single-RG legacy file (prefix_len absent → 0). + let cfg = ParquetWriterConfig { + compression: Compression::Snappy, + ..ParquetWriterConfig::default() + }; + let sort_fields = "metric_name|-timestamp_secs/V2"; + let kvs = vec![ + KeyValue::new( + PARQUET_META_SORT_FIELDS.to_string(), + sort_fields.to_string(), + ), + KeyValue::new( + PARQUET_META_WINDOW_START.to_string(), + "1700000000".to_string(), + ), + KeyValue::new(PARQUET_META_WINDOW_DURATION.to_string(), "60".to_string()), + KeyValue::new(PARQUET_META_NUM_MERGE_OPS.to_string(), "0".to_string()), + ]; + let sort_field_names = vec!["metric_name".to_string(), "timestamp_secs".to_string()]; + let props: WriterProperties = cfg.to_writer_properties_with_metadata( + &schema, + Vec::new(), + Some(kvs), + &sort_field_names, + ); + let mut buf: Vec = Vec::new(); + let mut writer = ArrowWriter::try_new(&mut buf, schema, Some(props)).expect("arrow writer"); + writer.write(&batch).expect("write"); + writer.close().expect("close"); + Bytes::from(buf) + } + + /// Regression for Codex P2 on PR-6410: prefix_len=0 inputs + + /// `num_outputs > 1` previously folded into a single oversized + /// output because the region-to-output assigner only split at + /// region boundaries and prefix_len=0 produces exactly one region. + /// The engine now subdivides the single region at sorted_series + /// transitions so it can honor `num_outputs`. The output inherits + /// the input's `rg_partition_prefix_len` (= 0 here) — the engine + /// does not declare a prefix it can't unconditionally guarantee. + #[tokio::test] + async fn test_prefix_len_zero_multi_output_splits_at_sorted_series() { + // 6 distinct metric_names × 50 rows = 300 rows total. + // num_outputs = 3 → target 100 rows/output. Splits land at + // sorted_series transitions (each row has a unique + // sorted_series in this fixture). + let metrics = [ + ("aaa.alpha", 50usize), + ("bbb.beta", 50), + ("ccc.gamma", 50), + ("ddd.delta", 50), + ("eee.epsilon", 50), + ("fff.zeta", 50), + ]; + let bytes = make_multi_metric_single_rg_input(&metrics); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(3)) + .await + .expect("merge"); + + assert_eq!( + outputs.len(), + 3, + "prefix_len=0 input + num_outputs=3 must produce 3 output files; got {} ({} rows \ + total)", + outputs.len(), + outputs.iter().map(|o| o.num_rows).sum::(), + ); + let total: usize = outputs.iter().map(|o| o.num_rows).sum(); + assert_eq!(total, 300, "rows preserved (MC-1)"); + + for out in &outputs { + let bytes_out = std::fs::read(&out.path).expect("read"); + let reader = SerializedFileReader::new(Bytes::from(bytes_out)).expect("ser"); + // Inherits input's prefix_len = 0 → KV absent. + let prefix_kv = reader + .metadata() + .file_metadata() + .key_value_metadata() + .and_then(|kvs| { + kvs.iter() + .find(|k| k.key == PARQUET_META_RG_PARTITION_PREFIX_LEN) + .and_then(|k| k.value.clone()) + }); + assert!( + prefix_kv.is_none(), + "prefix_len=0 input must produce prefix_len=0 output (KV absent); got \ + {prefix_kv:?}", + ); + // Each sub-region produces one output RG. + assert_eq!( + reader.metadata().num_row_groups(), + 1, + "expected one output RG per sub-region; got {}", + reader.metadata().num_row_groups(), + ); + } + } + + /// Giant single-metric input: prefix_len=0, only one metric_name, + /// so there are NO metric_name transitions in the merge order. + /// Splitting must still honor `num_outputs` by breaking at + /// sorted_series transitions inside the single metric. Confirms + /// the engine does not require prefix-value transitions to + /// honor the requested output count. + #[tokio::test] + async fn test_prefix_len_zero_giant_single_metric_splits_into_multiple_outputs() { + let metrics = [("cpu.usage", 200usize)]; + let bytes = make_multi_metric_single_rg_input(&metrics); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(2)) + .await + .expect("merge"); + + assert_eq!( + outputs.len(), + 2, + "one metric × num_outputs=2 must still split (at sorted_series boundaries); got {}", + outputs.len(), + ); + let total: usize = outputs.iter().map(|o| o.num_rows).sum(); + assert_eq!(total, 200, "rows preserved"); + // Each output is balanced near 100 rows. + for out in &outputs { + assert!( + out.num_rows > 0 && out.num_rows <= 200, + "output rows = {}", + out.num_rows + ); + } + } + + /// Prefix_len=0 + num_outputs=1: the whole region fits in one + /// output, no splitting needed. `process_region` drains + /// internally (no pre-drain) and produces a single output RG. + #[tokio::test] + async fn test_prefix_len_zero_single_output_is_single_rg() { + let metrics = [ + ("cpu.usage", 40usize), + ("memory.used", 40), + ("net.bytes", 40), + ]; + let bytes = make_multi_metric_single_rg_input(&metrics); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge"); + assert_eq!(outputs.len(), 1); + assert_eq!(outputs[0].num_rows, 120); + + let bytes_out = std::fs::read(&outputs[0].path).expect("read"); + let reader = SerializedFileReader::new(Bytes::from(bytes_out)).expect("ser"); + assert_eq!( + reader.metadata().num_row_groups(), + 1, + "single output, no split → single RG (engine does not synthesize prefix alignment \ + when inputs declare prefix_len=0)", + ); + let prefix_kv = reader + .metadata() + .file_metadata() + .key_value_metadata() + .and_then(|kvs| { + kvs.iter() + .find(|k| k.key == PARQUET_META_RG_PARTITION_PREFIX_LEN) + .and_then(|k| k.value.clone()) + }); + assert!( + prefix_kv.is_none(), + "output must inherit input's prefix_len = 0 (KV absent); got {prefix_kv:?}", + ); + } + + /// Build a multi-RG fixture where each RG has a single + /// (metric_name, service) tuple. `rg_partition_prefix_len = 2` + /// declares the alignment. + fn make_prefix_len_two_input(rgs: &[(&str, &str)], rows_per_rg: usize) -> Bytes { + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("metric_name", DataType::Utf8, false), + Field::new("service", DataType::Utf8, false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("sorted_series", DataType::Binary, false), + Field::new("value", DataType::Float64, false), + ])); + + let make_batch = |metric: &str, service: &str, start_series: u64| -> RecordBatch { + let metric_name: ArrayRef = Arc::new(StringArray::from(vec![metric; rows_per_rg])); + let svc: ArrayRef = Arc::new(StringArray::from(vec![service; rows_per_rg])); + let timestamps: Vec = (0..rows_per_rg as u64).map(|i| 1_700_000_000 + i).collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let series: Vec> = (0..rows_per_rg as u64) + .map(|i| (start_series + i).to_be_bytes().to_vec()) + .collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from( + series.iter().map(|v| v.as_slice()).collect::>(), + )); + let value: ArrayRef = Arc::new(Float64Array::from( + (0..rows_per_rg).map(|i| i as f64).collect::>(), + )); + RecordBatch::try_new( + schema.clone(), + vec![metric_name, svc, timestamp_secs, sorted_series, value], + ) + .expect("batch") + }; + + let cfg = ParquetWriterConfig { + compression: Compression::Snappy, + row_group_size: rows_per_rg, // one RG per batch + ..ParquetWriterConfig::default() + }; + let kvs = vec![ + KeyValue::new( + PARQUET_META_SORT_FIELDS.to_string(), + "metric_name|service|-timestamp_secs/V2".to_string(), + ), + KeyValue::new( + PARQUET_META_WINDOW_START.to_string(), + "1700000000".to_string(), + ), + KeyValue::new(PARQUET_META_WINDOW_DURATION.to_string(), "60".to_string()), + KeyValue::new(PARQUET_META_NUM_MERGE_OPS.to_string(), "0".to_string()), + KeyValue::new( + PARQUET_META_RG_PARTITION_PREFIX_LEN.to_string(), + "2".to_string(), + ), + ]; + let sorting_cols = vec![ + parquet::file::metadata::SortingColumn { + column_idx: 0, + descending: false, + nulls_first: false, + }, + parquet::file::metadata::SortingColumn { + column_idx: 1, + descending: false, + nulls_first: false, + }, + ]; + let props: WriterProperties = cfg.to_writer_properties_with_metadata( + &schema, + sorting_cols, + Some(kvs), + &["metric_name".to_string(), "service".to_string()], + ); + let mut buf: Vec = Vec::new(); + let mut writer = + ArrowWriter::try_new(&mut buf, schema.clone(), Some(props)).expect("arrow writer"); + for (i, (metric, service)) in rgs.iter().enumerate() { + let batch = make_batch(metric, service, (i as u64) * rows_per_rg as u64); + writer.write(&batch).expect("write"); + } + writer.close().expect("close"); + Bytes::from(buf) + } + + /// End-to-end regression for DESC prefix columns. Three RGs with + /// the same metric_name (ASC) and distinct `env` values; sort + /// schema declares env DESC. The input file must itself be + /// DESC-sorted on env (RGs in physical order staging → prod → + /// dev) because the streaming engine processes each input's RGs + /// in physical order; the BTreeMap-driven region order is the + /// thing the composite key + `invert_for_descending` controls, + /// and it must agree with the input's physical RG order for the + /// engine to drain sort cols in lockstep with the body cols. + /// + /// Regions must therefore come out in DESC order on env: + /// staging → prod → dev. Without the `invert_for_descending` + /// step the BTreeMap would emit dev → prod → staging, which + /// would disagree with the physical RG order and the engine + /// would bail with "page from rg 0 while draining sort cols of + /// rg 2". + #[tokio::test] + async fn test_streaming_merge_with_desc_prefix_col() { + let bytes = make_prefix_len_two_input_with_directions( + // (metric, env, marker_value). Each RG's body `value` + // column is filled with `marker_value` so we can identify + // which RG produced each output row group. Order is + // env-DESC physical: staging, prod, dev. + &[ + ("cpu.usage", "staging", 3.0), + ("cpu.usage", "prod", 2.0), + ("cpu.usage", "dev", 1.0), + ], + 20, + /* env_descending */ true, + ); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge with env DESC"); + assert_eq!(outputs.len(), 1); + + // Read the output and inspect the per-RG metric values: each + // RG should be filled with a single marker, and the RG order + // must be staging (3.0) → prod (2.0) → dev (1.0). + let bytes_out = std::fs::read(&outputs[0].path).expect("read"); + let reader = SerializedFileReader::new(Bytes::from(bytes_out)).expect("ser"); + let meta = reader.metadata(); + assert_eq!( + meta.num_row_groups(), + 3, + "three distinct env values → three RGs" + ); + + let merged = read_output_to_record_batch(&outputs[0].path); + let value = merged + .column(merged.schema().index_of("value").expect("value")) + .as_any() + .downcast_ref::() + .expect("Float64"); + // RG 0 spans rows 0..20, RG 1 spans 20..40, RG 2 spans 40..60. + // Within each RG the marker_value is constant. + let first_block = value.value(0); + let second_block = value.value(20); + let third_block = value.value(40); + assert!( + (first_block - 3.0).abs() < 1e-9, + "first output RG should be 'staging' (marker 3.0), got {first_block}", + ); + assert!( + (second_block - 2.0).abs() < 1e-9, + "second output RG should be 'prod' (marker 2.0), got {second_block}", + ); + assert!( + (third_block - 1.0).abs() < 1e-9, + "third output RG should be 'dev' (marker 1.0), got {third_block}", + ); + } + + /// Regression for the composite-key encoding when ASC and DESC + /// columns are interleaved. metric_name ASC + env DESC: composite + /// keys for ("cpu.usage", "dev") and ("cpu.usage", "prod") must + /// put 'prod' before 'dev' (because prod > dev in ASC lex, so + /// prod's DESC encoding sorts smaller). + #[test] + fn test_extract_rg_composite_prefix_key_mixed_directions() { + let bytes = make_prefix_len_two_input_with_directions( + &[("cpu.usage", "dev", 0.0), ("cpu.usage", "prod", 0.0)], + 5, + /* env_descending */ true, + ); + let reader = SerializedFileReader::new(Bytes::from(bytes.to_vec())).expect("ser"); + let metadata = reader.metadata(); + let prefix_cols = + find_prefix_parquet_col_indices(metadata, "metric_name|-env|-timestamp_secs/V2", 2, 0) + .expect("resolve"); + // Sanity: the second prefix column must be flagged DESC. + assert!( + prefix_cols[1].descending, + "env must be parsed as DESC from sort schema", + ); + + let key_dev = extract_rg_composite_prefix_key(metadata, 0, &prefix_cols, 0).expect("dev"); + let key_prod = extract_rg_composite_prefix_key(metadata, 1, &prefix_cols, 0).expect("prod"); + // metric_name is the same; env differs. DESC on env means + // 'prod' (larger lex) should encode to LESS-THAN 'dev', so + // the BTreeMap iterates prod first. + assert!( + key_prod < key_dev, + "with env DESC, composite key for 'prod' must lex-sort before 'dev'", + ); + } + + /// MS-2: a file whose physical RG order disagrees with the + /// composite-key encoding's derived order must be rejected + /// upfront at `extract_regions_from_metadata` time, not bail + /// later from inside `process_region`. Construct an input that declares + /// env DESC but physically writes RGs in ASC env order — the + /// BTreeMap region iteration will visit RG 2 (env DESC = "dev", + /// largest in DESC encoding ... wait, no — DESC means largest + /// first, so env "staging" should be first ...). + /// + /// Concretely: RGs written physically as `[dev, prod, staging]` + /// with sort direction declared DESC. DESC iteration order is + /// `[staging (RG 2), prod (RG 1), dev (RG 0)]`. The first region + /// the engine would try to drain is RG 2, but the input stream + /// reaches RG 0 first. MS-2 must reject this at + /// `extract_regions_from_metadata` time. + #[tokio::test] + async fn test_ms2_region_order_disagrees_with_physical_rg_order_rejected() { + let bytes = make_prefix_len_two_input_with_directions( + // Physical order ASC on env: dev, prod, staging. But the + // sort schema below declares env DESC. + &[ + ("cpu.usage", "dev", 1.0), + ("cpu.usage", "prod", 2.0), + ("cpu.usage", "staging", 3.0), + ], + 10, + /* env_descending */ true, + ); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let err = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect_err("region order vs physical RG order mismatch must be rejected"); + let s = err.to_string(); + assert!( + s.contains("disagrees with input") && s.contains("physical row order"), + "expected MS-2 rejection message, got: {s}", + ); + } + + /// Build a multi-RG fixture whose second sort col can be flagged + /// DESC. Sort schema written into KV metadata: either + /// `metric_name|env|-timestamp_secs/V2` or + /// `metric_name|-env|-timestamp_secs/V2`. + fn make_prefix_len_two_input_with_directions( + rgs: &[(&str, &str, f64)], + rows_per_rg: usize, + env_descending: bool, + ) -> Bytes { + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("metric_name", DataType::Utf8, false), + Field::new("env", DataType::Utf8, false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("sorted_series", DataType::Binary, false), + Field::new("value", DataType::Float64, false), + ])); + + let make_batch = |metric: &str, env: &str, marker: f64, start_series: u64| -> RecordBatch { + let metric_name: ArrayRef = Arc::new(StringArray::from(vec![metric; rows_per_rg])); + let env_arr: ArrayRef = Arc::new(StringArray::from(vec![env; rows_per_rg])); + let timestamps: Vec = (0..rows_per_rg as u64) + // Timestamps DESC within the RG to match the DESC sort. + .map(|i| 1_700_000_000 + (rows_per_rg as u64 - i)) + .collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let series: Vec> = (0..rows_per_rg as u64) + .map(|i| (start_series + i).to_be_bytes().to_vec()) + .collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from( + series.iter().map(|v| v.as_slice()).collect::>(), + )); + let value: ArrayRef = Arc::new(Float64Array::from(vec![marker; rows_per_rg])); + RecordBatch::try_new( + schema.clone(), + vec![metric_name, env_arr, timestamp_secs, sorted_series, value], + ) + .expect("batch") + }; + + let env_token = if env_descending { "-env" } else { "env" }; + let sort_fields = format!("metric_name|{env_token}|-timestamp_secs/V2"); + + let cfg = ParquetWriterConfig { + compression: Compression::Snappy, + row_group_size: rows_per_rg, + ..ParquetWriterConfig::default() + }; + let kvs = vec![ + KeyValue::new(PARQUET_META_SORT_FIELDS.to_string(), sort_fields), + KeyValue::new( + PARQUET_META_WINDOW_START.to_string(), + "1700000000".to_string(), + ), + KeyValue::new(PARQUET_META_WINDOW_DURATION.to_string(), "60".to_string()), + KeyValue::new(PARQUET_META_NUM_MERGE_OPS.to_string(), "0".to_string()), + KeyValue::new( + PARQUET_META_RG_PARTITION_PREFIX_LEN.to_string(), + "2".to_string(), + ), + ]; + let sorting_cols = vec![ + parquet::file::metadata::SortingColumn { + column_idx: 0, + descending: false, + nulls_first: false, + }, + parquet::file::metadata::SortingColumn { + column_idx: 1, + descending: env_descending, + nulls_first: false, + }, + ]; + let props: WriterProperties = cfg.to_writer_properties_with_metadata( + &schema, + sorting_cols, + Some(kvs), + &["metric_name".to_string(), "env".to_string()], + ); + let mut buf: Vec = Vec::new(); + let mut writer = + ArrowWriter::try_new(&mut buf, schema.clone(), Some(props)).expect("arrow writer"); + for (i, (metric, env, marker)) in rgs.iter().enumerate() { + let batch = make_batch(metric, env, *marker, (i as u64) * rows_per_rg as u64); + writer.write(&batch).expect("write"); + } + writer.close().expect("close"); + Bytes::from(buf) + } + + /// `extract_aligned_prefix_value` rejects an RG whose prefix + /// column has `min != max` — those RGs are not actually aligned + /// on the prefix value and grouping them into one region would be + /// silently wrong. + #[test] + fn test_composite_key_rejects_non_aligned_rg() { + // A single RG whose `metric_name` carries two distinct values: + // min ("cpu.usage") != max ("memory.used"). The composite-key + // extractor must refuse to mint a key for this RG. + let bytes = make_misaligned_metric_name_input(&[("cpu.usage", "memory.used")], 20); + let reader = SerializedFileReader::new(bytes).expect("ser"); + let metadata = reader.metadata(); + let prefix_cols = + find_prefix_parquet_col_indices(metadata, "metric_name|-timestamp_secs/V2", 1, 0) + .expect("resolve"); + let err = extract_rg_composite_prefix_key(metadata, 0, &prefix_cols, 0) + .expect_err("RG with min != max on prefix col must be rejected"); + let s = err.to_string(); + assert!( + s.contains("NOT prefix-aligned"), + "expected misalignment error, got: {s}", + ); + } + + /// Build a single-RG fixture whose `metric_name` column contains + /// two distinct values within the same RG. Used to exercise the + /// `min != max` rejection path in + /// `extract_aligned_prefix_value`. + fn make_misaligned_metric_name_input(names: &[(&str, &str)], rows_per_run: usize) -> Bytes { + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("metric_name", DataType::Utf8, false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("sorted_series", DataType::Binary, false), + Field::new("value", DataType::Float64, false), + ])); + + let total = rows_per_run * 2 * names.len(); + let mut metric_values: Vec<&str> = Vec::with_capacity(total); + for (a, b) in names { + for _ in 0..rows_per_run { + metric_values.push(*a); + } + for _ in 0..rows_per_run { + metric_values.push(*b); + } + } + let metric_name: ArrayRef = Arc::new(StringArray::from(metric_values)); + let timestamps: Vec = (0..total as u64).map(|i| 1_700_000_000 + i).collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let series: Vec> = (0..total as u64) + .map(|i| i.to_be_bytes().to_vec()) + .collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from( + series.iter().map(|v| v.as_slice()).collect::>(), + )); + let value: ArrayRef = Arc::new(Float64Array::from( + (0..total).map(|i| i as f64).collect::>(), + )); + + let batch = RecordBatch::try_new( + schema.clone(), + vec![metric_name, timestamp_secs, sorted_series, value], + ) + .expect("batch"); + + let cfg = ParquetWriterConfig { + compression: Compression::Snappy, + ..ParquetWriterConfig::default() + }; + let kvs = vec![ + KeyValue::new( + PARQUET_META_SORT_FIELDS.to_string(), + "metric_name|-timestamp_secs/V2".to_string(), + ), + KeyValue::new( + PARQUET_META_WINDOW_START.to_string(), + "1700000000".to_string(), + ), + KeyValue::new(PARQUET_META_WINDOW_DURATION.to_string(), "60".to_string()), + KeyValue::new(PARQUET_META_NUM_MERGE_OPS.to_string(), "0".to_string()), + KeyValue::new( + PARQUET_META_RG_PARTITION_PREFIX_LEN.to_string(), + "1".to_string(), + ), + ]; + let props: WriterProperties = cfg.to_writer_properties_with_metadata( + &schema, + Vec::new(), + Some(kvs), + &["metric_name".to_string(), "timestamp_secs".to_string()], + ); + let mut buf: Vec = Vec::new(); + let mut writer = + ArrowWriter::try_new(&mut buf, schema.clone(), Some(props)).expect("arrow writer"); + writer.write(&batch).expect("write"); + writer.close().expect("close"); + Bytes::from(buf) + } + + /// Build a parquet fixture with TWO row groups, each containing + /// rows of one distinct metric_name. RG 0 = "cpu.usage" × 30 rows, + /// RG 1 = "memory.used" × 30 rows. `rg_partition_prefix_len = 1` + /// declares metric_name alignment. + fn make_two_metric_aligned_input() -> Bytes { + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("metric_name", dict_type.clone(), false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("sorted_series", DataType::Binary, false), + Field::new("metric_type", DataType::UInt8, false), + Field::new("service", dict_type, true), + Field::new("timeseries_id", DataType::Int64, false), + Field::new("value", DataType::Float64, false), + ])); + + let make_batch = |metric_key: i32, start_series: u64, rows: usize| -> RecordBatch { + let metric_keys: Vec = vec![metric_key; rows]; + let metric_values = StringArray::from(vec!["cpu.usage", "memory.used"]); + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(metric_values), + ) + .expect("dict"), + ); + let timestamps: Vec = (0..rows as u64) + .map(|i| 1_700_000_000 + (rows as u64 - i)) + .collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let mut series_bytes: Vec> = Vec::with_capacity(rows); + for i in 0..rows as u64 { + series_bytes.push((start_series + i).to_be_bytes().to_vec()); } - } - assert_eq!(total_decoded, 80); - } + let series_refs: Vec<&[u8]> = series_bytes.iter().map(|v| v.as_slice()).collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from(series_refs)); + let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; rows])); + let svc_values = StringArray::from(vec!["api", "db", "cache"]); + let svc_keys: Vec> = (0..rows as i32) + .map(|i| if i % 5 == 0 { None } else { Some(i % 3) }) + .collect(); + let service: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(svc_keys), + Arc::new(svc_values), + ) + .expect("dict"), + ); + let tsids: Vec = (0..rows as i64).map(|i| 1000 + i).collect(); + let timeseries_id: ArrayRef = Arc::new(Int64Array::from(tsids)); + let values: Vec = (0..rows).map(|i| i as f64).collect(); + let value: ArrayRef = Arc::new(Float64Array::from(values)); + + RecordBatch::try_new( + schema.clone(), + vec![ + metric_name, + timestamp_secs, + sorted_series, + metric_type, + service, + timeseries_id, + value, + ], + ) + .expect("batch") + }; - /// Page-bounded contract sanity: with a row group large enough to - /// require many parquet pages per col, body col writes go through - /// the page-by-page assembler instead of materialising column - /// chunks. We can't directly observe peak memory from a test, but - /// we *can* assert that the merge completes correctly with input - /// data whose body cols span many pages, and that the output is - /// itself multi-page (no whole-column buffering happened on the - /// output side either). - #[tokio::test] - async fn test_body_col_streams_many_pages_per_column_chunk() { - // Force multiple pages per column chunk by setting a small - // data_page_row_count_limit. With 8000 rows and a 1000-row - // page limit, the output value col chunk must span ≥ 8 pages. - let batch = make_sorted_batch(8000, 0); - let bytes = write_input_parquet(std::slice::from_ref(&batch), &[]); - let inputs: Vec> = vec![open_stream(bytes).await]; + let batch_cpu = make_batch(0, 0, 30); + let batch_mem = make_batch(1, 100, 30); - let writer_config = ParquetWriterConfig { + let cfg = ParquetWriterConfig { compression: Compression::Snappy, - data_page_row_count_limit: 1000, + row_group_size: 30, // one RG per metric_name ..ParquetWriterConfig::default() }; - let config = MergeConfig { - num_outputs: 1, - writer_config, - }; - - let tmp = TempDir::new().expect("tmpdir"); - let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &config) - .await - .expect("merge"); - assert_eq!(outputs.len(), 1); - assert_eq!(outputs[0].num_rows, 8000); - - // Verify the output is itself multi-page-per-column (which is - // what page-bounded writing should produce, given the default - // data_page_size). Read via the page-bounded decoder and count - // pages for the value column. - let out_bytes = std::fs::read(&outputs[0].path).expect("read"); - let mut output_stream = open_stream(Bytes::from(out_bytes)).await; - // Find the "value" col index in the output's arrow schema BEFORE - // borrowing output_stream mutably for the decoder. - let arrow_schema = parquet::arrow::parquet_to_arrow_schema( - output_stream.metadata().file_metadata().schema_descr(), - None, - ) - .expect("arrow schema"); - let value_col_idx = arrow_schema.index_of("value").expect("value col"); - let mut decoder = StreamDecoder::new(&mut *output_stream); - - let mut value_pages = 0; - while let Some(page) = decoder.decode_next_page().await.expect("decode") { - if page.col_idx == value_col_idx { - value_pages += 1; - } - } - assert!( - value_pages >= 2, - "expected output 'value' col to span multiple pages (got {value_pages}); body col \ - writes should respect data_page_size", + let kvs = vec![ + KeyValue::new( + PARQUET_META_SORT_FIELDS.to_string(), + "metric_name|-timestamp_secs/V2".to_string(), + ), + KeyValue::new( + PARQUET_META_WINDOW_START.to_string(), + "1700000000".to_string(), + ), + KeyValue::new(PARQUET_META_WINDOW_DURATION.to_string(), "60".to_string()), + KeyValue::new(PARQUET_META_NUM_MERGE_OPS.to_string(), "0".to_string()), + // `prefix_len = 1` declares metric_name alignment. + KeyValue::new( + PARQUET_META_RG_PARTITION_PREFIX_LEN.to_string(), + "1".to_string(), + ), + ]; + let sorting_cols = vec![ + parquet::file::metadata::SortingColumn { + column_idx: 0, + descending: false, + nulls_first: false, + }, + parquet::file::metadata::SortingColumn { + column_idx: 1, + descending: true, + nulls_first: false, + }, + ]; + let props: WriterProperties = cfg.to_writer_properties_with_metadata( + &schema, + sorting_cols, + Some(kvs), + &["metric_name".to_string(), "timestamp_secs".to_string()], ); + let mut buf: Vec = Vec::new(); + let mut writer = ArrowWriter::try_new(&mut buf, schema, Some(props)).expect("arrow writer"); + writer.write(&batch_cpu).expect("write cpu"); + writer.write(&batch_mem).expect("write mem"); + writer.close().expect("close"); + Bytes::from(buf) } - /// Regression for Codex P1 on PR-6409: a zero-row-group input - /// must not panic the body-column path. Phase 0 explicitly accepts - /// empty inputs (returning a zero-row sort batch), so the body-col - /// loop has to defend against `row_group(0)` lookups on inputs - /// with `num_row_groups() == 0`. + /// Regression for Codex P1 on PR-6409 (the empty-input half): a + /// zero-row-group input mixed with a populated one must not panic + /// the body-column path. Phase 0 already accepts empty inputs; + /// PR-6c.2's per-region engine only iterates `region.contributing` + /// inputs for body cols, but verify directly so any future change + /// that broadens the iteration is caught. #[tokio::test] async fn test_zero_row_input_mixed_with_non_empty() { let empty = make_sorted_batch(0, 0); @@ -2454,9 +3183,6 @@ mod tests { .as_any() .downcast_ref::() .expect("Float64"); - // All 50 values from the populated input should round-trip in - // input row order (timestamps descend in input row order to - // match the sort key). for i in 0..50 { assert!( (value.value(i) - i as f64).abs() < 1e-9, @@ -2466,66 +3192,6 @@ mod tests { } } - /// Regression for Codex P2 on PR-6409: `derive_output_schema` - /// must drop sort fields that `optimize_output_batch` discarded as - /// all-null for a given output. Before the fix the check was - /// `sort_optimised.has(name) || full_union.has(name)`, where the - /// second disjunct is trivially true for every iterated field, so - /// all-null sort columns were never dropped. We exercise the - /// helper directly with a synthetic union + sort-optimised pair so - /// the regression doesn't depend on the merge plan creating an - /// all-null sort col (which requires a multi-input fixture with - /// divergent sort col presence). - #[test] - fn test_derive_output_schema_drops_all_null_sort_field() { - // Union covers sort fields {metric_name, env, timestamp_secs} - // plus body field {value}. `env` is a sort field declared in - // the sort union but `optimize_output_batch` dropped it from - // this output's `sort_optimised` (all-null for this output's - // rows). Body field `value` is NOT in the sort union — it - // must be preserved unconditionally. - let full_union = Arc::new(ArrowSchema::new(vec![ - Field::new("metric_name", DataType::Utf8, false), - Field::new("env", DataType::Utf8, true), - Field::new("timestamp_secs", DataType::UInt64, false), - Field::new("value", DataType::Float64, false), - ])); - let sort_union = Arc::new(ArrowSchema::new(vec![ - Field::new("metric_name", DataType::Utf8, false), - Field::new("env", DataType::Utf8, true), - Field::new("timestamp_secs", DataType::UInt64, false), - ])); - // sort_optimised has dropped `env` (all-null for this output). - let sort_optimised_schema = Arc::new(ArrowSchema::new(vec![ - Field::new("metric_name", DataType::Utf8, false), - Field::new("timestamp_secs", DataType::UInt64, false), - ])); - let sort_optimised = RecordBatch::try_new( - sort_optimised_schema, - vec![ - Arc::new(StringArray::from(vec!["cpu.usage"; 4])), - Arc::new(UInt64Array::from(vec![1u64, 2, 3, 4])), - ], - ) - .expect("test batch"); - - let derived = derive_output_schema(&full_union, &sort_union, &sort_optimised).expect("ok"); - let names: Vec<&str> = derived.fields().iter().map(|f| f.name().as_str()).collect(); - assert_eq!( - names, - vec!["metric_name", "timestamp_secs", "value"], - "all-null sort field 'env' must be dropped; body field 'value' preserved", - ); - - // Direct contrast: with the pre-fix logic (every field kept) - // the result would have included 'env'. Asserting the negative - // form makes the regression unambiguous. - assert!( - derived.index_of("env").is_err(), - "'env' must not appear in derived output schema", - ); - } - /// Write a fixture parquet file where each body column is forced /// to span multiple parquet data pages by pinning a small /// `data_page_row_count_limit`. The merge engine must read those @@ -2646,116 +3312,10 @@ mod tests { } } - /// Regression for Codex P1 on PR-6409 (the multi-output half): when - /// a body column is consumed by more than one output, the per-input - /// page cache and decoder must outlive each output's assembler. The - /// stream cannot be rewound, so dropping a partially-consumed page - /// when output K ends would leave output K+1 unable to read rows - /// that physically live inside that same page. - #[tokio::test] - async fn test_body_col_cache_persists_across_outputs() { - // Two metric names so the engine splits the merge into two - // outputs at the metric_name boundary. Each input has 200 - // rows of cpu.usage then 200 rows of memory.used — small - // 50-row pages mean some pages span the boundary. - let schema = make_sorted_batch(0, 0).schema(); - let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); - let metric_values = StringArray::from(vec!["cpu.usage", "memory.used"]); - let keys: Vec = (0..400).map(|i| if i < 200 { 0 } else { 1 }).collect(); - let metric_name: ArrayRef = Arc::new( - DictionaryArray::::try_new( - arrow::array::Int32Array::from(keys), - Arc::new(metric_values), - ) - .expect("dict"), - ); - - // Timestamps descend within each metric_name run so the file - // is sorted by (metric_name ASC, timestamp DESC) — matching - // the sort schema declared in write_input_parquet_with_small_pages. - let timestamps: Vec = (0..400) - .map(|i| { - let run_pos = if i < 200 { i } else { i - 200 }; - 1_700_000_000u64 + (199 - run_pos as u64) - }) - .collect(); - let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); - - let series: Vec> = (0..400u64).map(|i| i.to_be_bytes().to_vec()).collect(); - let sorted_series: ArrayRef = Arc::new(BinaryArray::from( - series.iter().map(|v| v.as_slice()).collect::>(), - )); - - let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![1u8; 400])); - let service_keys: Vec = (0..400i32).map(|_| 0).collect(); - let service_values = StringArray::from(vec!["svc-a"]); - let service: ArrayRef = Arc::new( - DictionaryArray::::try_new( - arrow::array::Int32Array::from(service_keys), - Arc::new(service_values), - ) - .expect("svc dict"), - ); - let timeseries_id: ArrayRef = Arc::new(Int64Array::from((0..400i64).collect::>())); - let value: ArrayRef = Arc::new(Float64Array::from( - (0..400).map(|i| i as f64 * 0.5).collect::>(), - )); - // Confirm the schema we hand-build still matches make_sorted_batch's: - let _ = dict_type; - let batch = RecordBatch::try_new( - Arc::clone(&schema), - vec![ - metric_name, - timestamp_secs, - sorted_series, - metric_type, - service, - timeseries_id, - value, - ], - ) - .expect("test batch"); - - let bytes = write_input_parquet_with_small_pages(std::slice::from_ref(&batch), 50); - let inputs: Vec> = vec![open_stream(bytes).await]; - - let tmp = TempDir::new().expect("tmpdir"); - let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(2)) - .await - .expect("merge"); - assert_eq!(outputs.len(), 2, "two metric names → two outputs"); - - let total: usize = outputs.iter().map(|o| o.num_rows).sum(); - assert_eq!(total, 400); - - // Concatenate the two outputs' value columns in output order - // and verify every original value is present. The merge is - // sort-stable, so values within each output appear in input - // row order (timestamps descend within each metric run). - let mut seen_values: HashSet = HashSet::new(); - for out in &outputs { - let merged = read_output_to_record_batch(&out.path); - let v = merged - .column(merged.schema().index_of("value").expect("value col")) - .as_any() - .downcast_ref::() - .expect("Float64"); - for i in 0..v.len() { - // Encode as integer bits to dedupe; original values - // are i * 0.5 for i in 0..400, all distinct. - seen_values.insert(v.value(i).to_bits()); - } - } - assert_eq!( - seen_values.len(), - 400, - "every input value should round-trip through the merge across both outputs", - ); - } - - /// Multi-RG input is rejected (PR-6b.2 simplification). + /// Legacy multi-RG input (prefix_len=0, num_RGs>1) is rejected — + /// these must route through `LegacyInputAdapter` (PR-5). #[tokio::test] - async fn test_multi_rg_input_rejected() { + async fn test_legacy_multi_rg_input_rejected() { // Force a 2-RG file by writing two batches with row_group_size = 1 // small enough to trip RG rollover. let batch_a = make_sorted_batch(50, 0); @@ -2796,11 +3356,11 @@ mod tests { let tmp = TempDir::new().expect("tmpdir"); let err = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) .await - .expect_err("multi-RG input must be rejected"); + .expect_err("legacy multi-RG input must be rejected"); let s = err.to_string(); assert!( - s.contains("single-row-group"), - "expected 'single-row-group' error, got: {s}", + s.contains("legacy multi-RG") || s.contains("PR-5 adapter"), + "expected legacy multi-RG rejection, got: {s}", ); } @@ -3498,189 +4058,6 @@ mod tests { // Heterogeneous-output regressions (Codex P2 batch on PR-6409) // ============================================================================ - /// Regression for "Use the full union schema when driving column - /// writes": two outputs with the **same field count** that drop - /// *different* all-null sort fields. The previous - /// `build_parent_union_schema` heuristic picked the first such - /// schema and used it to drive the column loop — silently - /// skipping any column it dropped, even if another output still - /// needed it. The fix drives iteration from `full_union_schema` - /// directly. - /// - /// Setup: two inputs declaring an extended sort schema - /// `metric_name|tag_a|tag_b|-timestamp_secs/V2`. Input A has only - /// `tag_a` populated, input B has only `tag_b` populated. Merge - /// with `num_outputs=2` so each input's rows land in its own - /// output. After the per-output optimiser, output 0's schema - /// drops `tag_b` (all-null) and output 1's schema drops `tag_a` - /// — same field count, different dropped fields. Both outputs - /// must still write their kept tag column. - #[tokio::test] - async fn test_heterogeneous_dropped_fields_drive_from_full_union_schema() { - let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); - // Schema for input A: metric_name, tag_a (non-null), tag_b - // absent. Schema for input B: metric_name, tag_b (non-null), - // tag_a absent. The merge engine's union schema makes both - // tag fields nullable in the combined view. - let mk_schema = |with_a: bool, with_b: bool| -> SchemaRef { - let mut fields = vec![ - Field::new("metric_name", dict_type.clone(), false), - Field::new("timestamp_secs", DataType::UInt64, false), - Field::new("sorted_series", DataType::Binary, false), - ]; - // Body cols in lexicographic order. - if with_a { - fields.push(Field::new("tag_a", DataType::Utf8, false)); - } - if with_b { - fields.push(Field::new("tag_b", DataType::Utf8, false)); - } - fields.push(Field::new("value", DataType::Float64, false)); - Arc::new(ArrowSchema::new(fields)) - }; - - let make_batch = |metric: &str, - schema: SchemaRef, - tag_a_val: Option<&str>, - tag_b_val: Option<&str>, - row_count: usize, - base_series: u64| - -> RecordBatch { - let metric_keys: Vec = vec![0; row_count]; - let metric_values = StringArray::from(vec![metric]); - let metric_name: ArrayRef = Arc::new( - DictionaryArray::::try_new( - arrow::array::Int32Array::from(metric_keys), - Arc::new(metric_values), - ) - .expect("dict"), - ); - let timestamps: Vec = (0..row_count as u64) - .map(|i| 1_700_000_000 + (row_count as u64 - i)) - .collect(); - let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); - let series: Vec> = (0..row_count as u64) - .map(|i| (base_series + i).to_be_bytes().to_vec()) - .collect(); - let sorted_series: ArrayRef = Arc::new(BinaryArray::from( - series.iter().map(|v| v.as_slice()).collect::>(), - )); - let value: ArrayRef = Arc::new(Float64Array::from( - (0..row_count).map(|i| i as f64).collect::>(), - )); - let mut columns: Vec = vec![metric_name, timestamp_secs, sorted_series]; - if let Some(v) = tag_a_val { - columns.push(Arc::new(StringArray::from(vec![v; row_count])) as ArrayRef); - } - if let Some(v) = tag_b_val { - columns.push(Arc::new(StringArray::from(vec![v; row_count])) as ArrayRef); - } - columns.push(value); - RecordBatch::try_new(schema, columns).expect("batch") - }; - - // Input A has tag_a populated, sort key "aaa.metric" (sorts - // before "zzz.metric"). Input B has tag_b populated, sort key - // "zzz.metric". With num_outputs=2 the merge splits at the - // metric boundary so each input's rows land in its own output. - let schema_a = mk_schema(true, false); - let schema_b = mk_schema(false, true); - let batch_a = make_batch("aaa.metric", schema_a, Some("alpha"), None, 12, 0); - let batch_b = make_batch("zzz.metric", schema_b, None, Some("beta"), 12, 1000); - - let sort_fields_str = "metric_name|tag_a|tag_b|-timestamp_secs/V2"; - let make_input_bytes = |batch: &RecordBatch| -> Bytes { - let cfg = ParquetWriterConfig { - compression: Compression::Snappy, - ..ParquetWriterConfig::default() - }; - let kvs = vec![ - KeyValue::new( - PARQUET_META_SORT_FIELDS.to_string(), - sort_fields_str.to_string(), - ), - KeyValue::new( - PARQUET_META_WINDOW_START.to_string(), - "1700000000".to_string(), - ), - KeyValue::new(PARQUET_META_WINDOW_DURATION.to_string(), "60".to_string()), - KeyValue::new(PARQUET_META_NUM_MERGE_OPS.to_string(), "0".to_string()), - ]; - let props: WriterProperties = cfg.to_writer_properties_with_metadata( - &batch.schema(), - Vec::new(), - Some(kvs), - &[ - "metric_name".to_string(), - "tag_a".to_string(), - "tag_b".to_string(), - "timestamp_secs".to_string(), - ], - ); - let mut buf: Vec = Vec::new(); - let mut writer = - ArrowWriter::try_new(&mut buf, batch.schema(), Some(props)).expect("arrow writer"); - writer.write(batch).expect("write"); - writer.close().expect("close"); - Bytes::from(buf) - }; - let bytes_a = make_input_bytes(&batch_a); - let bytes_b = make_input_bytes(&batch_b); - - let inputs: Vec> = - vec![open_stream(bytes_a).await, open_stream(bytes_b).await]; - - let tmp = TempDir::new().expect("tmpdir"); - let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(2)) - .await - .expect("merge with heterogeneous dropped fields"); - assert_eq!(outputs.len(), 2, "two metric names → two outputs"); - - // The output whose rows came from input A must carry tag_a - // values; the output whose rows came from input B must carry - // tag_b values. Before the fix, one of them was silently - // missing its kept tag column because the parent driver - // skipped it. - let mut saw_alpha = false; - let mut saw_beta = false; - for out in &outputs { - let merged = read_output_to_record_batch(&out.path); - let schema = merged.schema(); - if let Ok(tag_a_idx) = schema.index_of("tag_a") { - let col = merged.column(tag_a_idx); - for i in 0..col.len() { - if col.is_valid(i) { - let s = render_cell(col.as_ref(), i); - if s.ends_with("alpha") { - saw_alpha = true; - } - } - } - } - if let Ok(tag_b_idx) = schema.index_of("tag_b") { - let col = merged.column(tag_b_idx); - for i in 0..col.len() { - if col.is_valid(i) { - let s = render_cell(col.as_ref(), i); - if s.ends_with("beta") { - saw_beta = true; - } - } - } - } - } - assert!( - saw_alpha, - "expected to find tag_a='alpha' in some output — the full-union-schema-driven column \ - loop must visit tag_a even when another output dropped it", - ); - assert!( - saw_beta, - "expected to find tag_b='beta' in some output — the full-union-schema-driven column \ - loop must visit tag_b even when another output dropped it", - ); - } - /// Regression for "Preserve service tags when service is a sort /// column". If the sort schema places `service` in the sort key /// (e.g., `metric_name|service|...`), the streaming engine writes diff --git a/quickwit/quickwit-parquet-engine/src/merge/streaming/body_assembler.rs b/quickwit/quickwit-parquet-engine/src/merge/streaming/body_assembler.rs new file mode 100644 index 00000000000..5f6bb83ba0e --- /dev/null +++ b/quickwit/quickwit-parquet-engine/src/merge/streaming/body_assembler.rs @@ -0,0 +1,494 @@ +// 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. + +//! Page-bounded body column assembly — the streaming core. +//! +//! For each `(output_idx, body_col)` pair the engine builds a +//! [`BodyColOutputPageAssembler`] that yields one assembled output +//! page per `Iterator::next` call. Each call: +//! +//! 1. Walks the destinations table to pick `(input_idx, input_row)` pairs that map to the next +//! `OUTPUT_PAGE_ROWS` positions of this output. +//! 2. Advances each contributing input's decoder forward until its cached pages cover the needed +//! rows. The decoder + page cache live on [`InputDecoderState`] (not the assembler) so a page +//! whose row range straddles two outputs survives into the next output's assembler — the stream +//! cannot be rewound. +//! 3. Concatenates each input's cached pages and calls `arrow::compute::interleave` to assemble the +//! output page. +//! 4. Evicts pages whose last row falls below the cursor. +//! +//! Memory bound per `next()`: one in-progress output page plus a few +//! in-flight input pages per input — never a whole column chunk. See +//! the MS-7 test in the parent test module for the runtime assertion. +//! +//! [`StreamingBodyColIter`] wraps the `Result` page stream +//! into the `Iterator` shape +//! `write_next_column_arrays` expects, capturing the first assembly +//! error in a side cell. + +use std::collections::HashSet; + +use anyhow::{Context, Result, anyhow, bail}; +use arrow::array::{Array, ArrayRef, new_null_array}; +use arrow::compute::interleave; +use arrow::datatypes::{DataType, Field}; +use tokio::runtime::Handle; + +use super::{ + InputDecoderState, InputRowDestinations, OUTPUT_PAGE_ROWS, record_body_col_page_cache_len, +}; + +/// Adapts a `Result` page assembler into the +/// `Iterator` shape `write_next_column_arrays` expects. +/// The first assembly error is captured in `error_slot` and iteration +/// ends; the caller MUST check the slot after the writer returns. If +/// `service_collector` is `Some`, every yielded page is scanned for +/// service names and added to the set; collection failures also stop +/// the iterator and populate `error_slot`. +pub(crate) struct StreamingBodyColIter<'a, I> { + pub(crate) inner: I, + pub(crate) error_slot: &'a mut Option, + pub(crate) service_collector: Option<&'a mut HashSet>, +} + +impl Iterator for StreamingBodyColIter<'_, I> +where I: Iterator> +{ + type Item = ArrayRef; + + fn next(&mut self) -> Option { + if self.error_slot.is_some() { + return None; + } + match self.inner.next() { + Some(Ok(arr)) => { + if let Some(out) = self.service_collector.as_deref_mut() + && let Err(e) = collect_service_names_from_page(arr.as_ref(), out) + { + *self.error_slot = Some(e); + return None; + } + Some(arr) + } + Some(Err(e)) => { + *self.error_slot = Some(e); + None + } + None => None, + } + } +} + +/// Per-page service name collector. Used during the streaming write +/// of the "service" body col to populate per-output service_names. +pub(crate) fn collect_service_names_from_page( + arr: &dyn Array, + out: &mut HashSet, +) -> Result<()> { + use arrow::array::AsArray; + use arrow::datatypes::{Int8Type, Int16Type, Int32Type, Int64Type}; + + fn extend_from_strings(strings: &arrow::array::StringArray, out: &mut HashSet) { + for i in 0..strings.len() { + if strings.is_valid(i) { + out.insert(strings.value(i).to_string()); + } + } + } + + match arr.data_type() { + DataType::Utf8 => { + let strings = arr + .as_any() + .downcast_ref::() + .ok_or_else(|| anyhow!("expected StringArray for service col page"))?; + extend_from_strings(strings, out); + } + DataType::LargeUtf8 => { + let strings = arr + .as_any() + .downcast_ref::() + .ok_or_else(|| anyhow!("expected LargeStringArray for service col page"))?; + for i in 0..strings.len() { + if strings.is_valid(i) { + out.insert(strings.value(i).to_string()); + } + } + } + DataType::Dictionary(key_type, value_type) + if matches!(value_type.as_ref(), DataType::Utf8) => + { + // Extract the dictionary's values that are referenced by + // valid (non-null) keys. + match key_type.as_ref() { + DataType::Int8 => { + let dict = arr.as_dictionary::(); + if let Some(strings) = dict + .values() + .as_any() + .downcast_ref::() + { + for i in 0..dict.len() { + if dict.is_valid(i) { + let key = dict.keys().value(i) as usize; + if key < strings.len() && strings.is_valid(key) { + out.insert(strings.value(key).to_string()); + } + } + } + } + } + DataType::Int16 => { + let dict = arr.as_dictionary::(); + if let Some(strings) = dict + .values() + .as_any() + .downcast_ref::() + { + for i in 0..dict.len() { + if dict.is_valid(i) { + let key = dict.keys().value(i) as usize; + if key < strings.len() && strings.is_valid(key) { + out.insert(strings.value(key).to_string()); + } + } + } + } + } + DataType::Int32 => { + let dict = arr.as_dictionary::(); + if let Some(strings) = dict + .values() + .as_any() + .downcast_ref::() + { + for i in 0..dict.len() { + if dict.is_valid(i) { + let key = dict.keys().value(i) as usize; + if key < strings.len() && strings.is_valid(key) { + out.insert(strings.value(key).to_string()); + } + } + } + } + } + DataType::Int64 => { + let dict = arr.as_dictionary::(); + if let Some(strings) = dict + .values() + .as_any() + .downcast_ref::() + { + for i in 0..dict.len() { + if dict.is_valid(i) { + let key = dict.keys().value(i) as usize; + if key < strings.len() && strings.is_valid(key) { + out.insert(strings.value(key).to_string()); + } + } + } + } + } + _ => {} + } + } + _ => { + // Skip non-string types — service col is expected to be + // string-like; if it isn't, just don't collect names. + } + } + Ok(()) +} + +/// Assembles output pages for one (output_idx, body_col). See the +/// module docs for the full contract. +pub(crate) struct BodyColOutputPageAssembler<'a> { + handle: &'a Handle, + decoders_state: &'a mut [InputDecoderState], + input_col_indices: &'a [Option], + destinations: &'a InputRowDestinations, + out_idx: usize, + col_name: &'a str, + out_field: &'a Field, + /// Total rows written so far for this output's col. + rows_emitted: usize, + /// Total rows expected = destinations.rows_per_output[out_idx]. + expected_rows: usize, + /// EOF flag (returns None on subsequent calls once true). + done: bool, +} + +impl<'a> BodyColOutputPageAssembler<'a> { + #[allow(clippy::too_many_arguments)] + pub(crate) fn new( + handle: &'a Handle, + decoders_state: &'a mut [InputDecoderState], + input_col_indices: &'a [Option], + destinations: &'a InputRowDestinations, + out_idx: usize, + col_name: &'a str, + out_field: &'a Field, + ) -> Self { + Self { + handle, + decoders_state, + input_col_indices, + destinations, + out_idx, + col_name, + out_field, + rows_emitted: 0, + expected_rows: destinations.rows_per_output[out_idx], + done: false, + } + } + + pub(crate) fn into_iter(self) -> BodyColOutputPageIter<'a> { + BodyColOutputPageIter { inner: self } + } +} + +pub(crate) struct BodyColOutputPageIter<'a> { + inner: BodyColOutputPageAssembler<'a>, +} + +impl Iterator for BodyColOutputPageIter<'_> { + type Item = Result; + + fn next(&mut self) -> Option { + // `assemble_one_output_page` only advances `rows_emitted` by + // `page_size = remaining.min(OUTPUT_PAGE_ROWS)` where + // `remaining = expected_rows - rows_emitted`, so the increment + // never overshoots. `emitted > expected` would mean a real + // accounting bug — surface it loudly rather than silently + // terminating one iteration too late. + debug_assert!( + self.inner.rows_emitted <= self.inner.expected_rows, + "rows_emitted {} > expected_rows {} for output {}", + self.inner.rows_emitted, + self.inner.expected_rows, + self.inner.out_idx, + ); + if self.inner.done || self.inner.rows_emitted == self.inner.expected_rows { + self.inner.done = true; + return None; + } + match assemble_one_output_page(&mut self.inner) { + Ok(Some(arr)) => Some(Ok(arr)), + Ok(None) => { + self.inner.done = true; + None + } + Err(e) => { + self.inner.done = true; + Some(Err(e)) + } + } + } +} + +fn assemble_one_output_page(s: &mut BodyColOutputPageAssembler) -> Result> { + let remaining = s.expected_rows - s.rows_emitted; + if remaining == 0 { + return Ok(None); + } + let page_size = remaining.min(OUTPUT_PAGE_ROWS); + + // Collect (input_idx, input_row) indices for this output page by + // walking destinations.per_input from each input's cursor. For + // each target output position, find the (input, row) that maps to + // it. See module docs for why this is correct given the merge + // plan invariants. + let mut indices_per_input: Vec> = vec![Vec::new(); s.decoders_state.len()]; + let mut interleave_indices: Vec<(usize, usize)> = Vec::with_capacity(page_size); + let mut total_picked = 0usize; + + while total_picked < page_size { + let target_pos = s.rows_emitted + total_picked; + let mut found = false; + for (input_idx, dests) in s.destinations.per_input.iter().enumerate() { + let cursor = match s.input_col_indices[input_idx] { + Some(col_parquet_idx) => { + s.decoders_state[input_idx].body_col_cursor(col_parquet_idx) + } + None => 0, + }; + for (input_row, dest) in dests.iter().enumerate().skip(cursor) { + match dest { + Some((o, p)) if *o == s.out_idx && *p == target_pos => { + interleave_indices.push((input_idx, input_row)); + indices_per_input[input_idx].push(input_row); + found = true; + break; + } + _ => {} + } + if found { + break; + } + } + if found { + break; + } + } + if !found { + // Shouldn't happen — every output position should be reachable. + bail!( + "merge plan inconsistency: output {} position {target_pos} not found in any input", + s.out_idx, + ); + } + total_picked += 1; + } + + // Advance each contributing input's decoder until its cached + // pages cover the maximum input row we'll reference. + for (input_idx, input_rows) in indices_per_input.iter().enumerate() { + if input_rows.is_empty() { + continue; + } + let col_parquet_idx = match s.input_col_indices[input_idx] { + Some(c) => c, + None => continue, + }; + let max_needed_row = *input_rows.iter().max().expect("non-empty"); + fill_page_cache_to_row( + s.handle, + &mut s.decoders_state[input_idx], + col_parquet_idx, + max_needed_row, + )?; + } + + // Build the per-input value array. For inputs lacking this col, + // a single null-row placeholder routes interleave indices to position 0. + let mut input_array_refs: Vec = Vec::with_capacity(s.decoders_state.len()); + let mut input_cache_starts: Vec = Vec::with_capacity(s.decoders_state.len()); + + for input_idx in 0..s.decoders_state.len() { + match s.input_col_indices[input_idx] { + Some(col_parquet_idx) => { + let pages = s.decoders_state[input_idx].body_col_cache(col_parquet_idx); + if pages.is_empty() { + input_array_refs.push(new_null_array(s.out_field.data_type(), 0)); + input_cache_starts.push(0); + } else { + let cache_start = pages[0].row_start; + let arrays: Vec<&dyn Array> = pages.iter().map(|p| p.array.as_ref()).collect(); + let concatenated = arrow::compute::concat(&arrays).with_context(|| { + format!( + "concatenating cached pages for input {input_idx} col '{}'", + s.col_name, + ) + })?; + input_array_refs.push(concatenated); + input_cache_starts.push(cache_start); + } + } + None => { + let null_arr = new_null_array(s.out_field.data_type(), 1); + input_array_refs.push(null_arr); + input_cache_starts.push(0); + } + } + } + + let interleave_local: Vec<(usize, usize)> = interleave_indices + .iter() + .map(|&(i_idx, i_row)| match s.input_col_indices[i_idx] { + Some(_) => (i_idx, i_row - input_cache_starts[i_idx]), + None => (i_idx, 0), + }) + .collect(); + + let array_refs_ref: Vec<&dyn Array> = input_array_refs.iter().map(|a| a.as_ref()).collect(); + let assembled = interleave(&array_refs_ref, &interleave_local).with_context(|| { + format!( + "interleaving body col '{}' for output {}", + s.col_name, s.out_idx, + ) + })?; + + // Bump cursors past consumed rows and evict pages whose last row + // falls below the cursor. Both live on InputDecoderState (per + // col) so they persist across regions/outputs that share this + // column. + for (input_idx, input_rows) in indices_per_input.iter().enumerate() { + if input_rows.is_empty() { + continue; + } + let max_row = *input_rows.iter().max().expect("non-empty"); + let state = &mut s.decoders_state[input_idx]; + let Some(col_parquet_idx) = s.input_col_indices[input_idx] else { + continue; + }; + state.set_body_col_cursor(col_parquet_idx, max_row + 1); + } + + s.rows_emitted += page_size; + Ok(Some(assembled)) +} + +/// Pull pages from the input's persistent decoder via `block_on` until +/// the cached pages for `col_parquet_idx` cover up through `target_row` +/// (inclusive). Stops as soon as the latest cached page ends past +/// `target_row`. The function's effect on the world is *adding pages +/// to the cache* — it does not skip data and does not consume any +/// rows on its own. +/// +/// The decoder MUST be the long-lived [`InputDecoderState::decoder`]: +/// it preserves the per-(rg, col) `rows_decoded` counter so successive +/// `DecodedPage::row_start` values are absolute input row indices, +/// not page-local zeros. Likewise, the cache lives on the state so +/// pages whose row range spans an output boundary survive into the +/// next output's assembler. +fn fill_page_cache_to_row( + handle: &Handle, + state: &mut InputDecoderState, + col_parquet_idx: usize, + target_row: usize, +) -> Result<()> { + // Already covered by what's cached for this col? + if let Some(last) = state.body_col_cache(col_parquet_idx).last() { + let last_end = last.row_start + last.array.len(); + if target_row < last_end { + return Ok(()); + } + } + + // Drain pages from the stream until either the target is covered + // for `col_parquet_idx` or the stream runs out. Pages emitted for + // a different col_idx still get cached under their own col so a + // later request for that col can find them without re-fetching — + // critical for the synthesized-prefix path, which re-reads earlier + // cols across multiple regions after the stream has moved on. + loop { + let decoded = handle + .block_on(state.decoder.decode_next_page()) + .context("decoding body col page")?; + let page = match decoded { + Some(p) => p, + None => bail!( + "stream EOF while advancing to row {target_row} for parquet col {col_parquet_idx}", + ), + }; + let page_col = page.col_idx; + let end = page.row_start + page.array.len(); + state.body_col_cache_mut(page_col).push(page); + record_body_col_page_cache_len(state.body_col_caches_total_len()); + if page_col == col_parquet_idx && target_row < end { + return Ok(()); + } + } +} diff --git a/quickwit/quickwit-parquet-engine/src/merge/streaming/output.rs b/quickwit/quickwit-parquet-engine/src/merge/streaming/output.rs new file mode 100644 index 00000000000..071a5030dcd --- /dev/null +++ b/quickwit/quickwit-parquet-engine/src/merge/streaming/output.rs @@ -0,0 +1,365 @@ +// 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. + +//! Per-output state, writer setup, and finalization for the streaming +//! merge engine. +//! +//! Each output file in a streaming merge owns: +//! - An [`OutputWriterStorage`] holding the live parquet writer and a running count of row groups +//! written to it (used to verify MS-3). +//! - An [`OutputAccumulator`] that concatenates this output's sort-col contributions across regions +//! so per-output metadata (`qh.row_keys`, `qh.zonemap_regexes`, metric_names, time_range) can be +//! computed from the output's actual rows at finalize time. +//! +//! Per-output `qh.row_keys` and `qh.zonemap_regexes` are *appended* to +//! the parquet footer just before close — they cannot be set at writer +//! construction because they depend on row content that the streaming +//! pass only finishes accumulating in `finalize_output`. The static +//! KV entries (sort schema, window, num_merge_ops, prefix_len) are set +//! at construction since they're identical regardless of row content. + +use std::collections::{HashMap, HashSet}; +use std::path::{Path, PathBuf}; +use std::sync::Arc; + +use anyhow::{Context, Result}; +use arrow::array::RecordBatch; +use arrow::datatypes::{Schema as ArrowSchema, SchemaRef}; +use base64::Engine; +use base64::engine::general_purpose::STANDARD as BASE64; +use ulid::Ulid; + +use super::super::writer::{build_merge_kv_metadata, resolve_sort_field_names}; +use super::super::{InputMetadata, MergeOutputFile}; +use crate::row_keys; +use crate::sort_fields::{is_timestamp_column_name, parse_sort_fields}; +use crate::split::TAG_SERVICE; +use crate::storage::split_writer::{extract_metric_names, extract_time_range}; +use crate::storage::streaming_writer::StreamingParquetWriter; +use crate::storage::{ + PARQUET_META_ROW_KEYS, PARQUET_META_ROW_KEYS_JSON, PARQUET_META_ZONEMAP_REGEXES, +}; +use crate::zonemap::{self, ZonemapOptions}; + +/// Per-output-file mutable state owned across regions assigned to that file. +pub(crate) struct OutputWriterStorage { + pub(crate) output_idx: usize, + pub(crate) output_path: PathBuf, + pub(crate) writer: StreamingParquetWriter, + /// Number of parquet row groups written into this file so far. One + /// row group is opened per region in the per-region processor, so + /// this equals the count of regions assigned to this output once + /// processing is done. Reported on [`MergeOutputFile::num_row_groups`]. + pub(crate) num_row_groups: usize, +} + +/// Per-output-file accumulator. Each region's sort-col contribution is +/// merged into `accumulated_sort_batch`; per-output metadata +/// (row_keys, zonemap, metric_names, time_range) is computed once at +/// `finalize_output` time. Service names are collected during the +/// streaming write of the service body col within each region. +pub(crate) struct OutputAccumulator { + pub(crate) output_idx: usize, + /// Concatenated sort-col `RecordBatch` across all regions written + /// to this output. Memory bounded by total sort col bytes in the + /// output file (small — sort cols are narrow). + pub(crate) accumulated_sort_batch: Option, + /// Service names collected across regions' body-col writes for + /// this output file. + pub(crate) service_names: HashSet, + /// Cumulative row count = sum of regions' total_rows assigned here. + pub(crate) num_rows: usize, +} + +impl OutputAccumulator { + pub(crate) fn new(output_idx: usize) -> Self { + Self { + output_idx, + accumulated_sort_batch: None, + service_names: HashSet::new(), + num_rows: 0, + } + } + + pub(crate) fn append_sort_batch(&mut self, batch: RecordBatch) -> Result<()> { + match self.accumulated_sort_batch.take() { + None => { + self.accumulated_sort_batch = Some(batch); + } + Some(prev) => { + let schema = prev.schema(); + let combined = arrow::compute::concat_batches(&schema, [&prev, &batch]) + .context("appending region sort batch to output accumulator")?; + self.accumulated_sort_batch = Some(combined); + } + } + Ok(()) + } +} + +/// Open a streaming Parquet writer for one output file. Caller is +/// responsible for calling `start_row_group` per region and writing +/// columns. +pub(crate) fn open_output_writer_for_streaming( + output_idx: usize, + output_dir: &Path, + union_schema: &SchemaRef, + input_meta: &InputMetadata, + writer_config: &crate::storage::ParquetWriterConfig, +) -> Result { + let output_prefix_len = input_meta.rg_partition_prefix_len; + // `qh.row_keys` and `qh.zonemap_regexes` MUST be derived from the + // rows that end up in THIS output file, not from inputs — the + // merge eliminates key overlap between outputs, so an output's + // key metadata can be very different from any input's. We can't + // compute those values until every region has been written, so + // they are appended to the file's footer KV metadata in + // `finalize_output` via `StreamingParquetWriter::append_key_value_metadata` + // just before close. The KV entries set here cover the "static" + // keys (sort_fields, window, num_merge_ops, prefix_len) that are + // identical regardless of which rows the output contains. + let kv_entries = build_merge_kv_metadata(input_meta, &None, &HashMap::new(), output_prefix_len); + + let sort_field_names = resolve_sort_field_names(&input_meta.sort_fields)?; + let sorting_cols = build_sorting_columns_from_schema(union_schema, &input_meta.sort_fields)?; + + let props = writer_config.to_writer_properties_with_metadata( + union_schema, + sorting_cols, + Some(kv_entries), + &sort_field_names, + ); + + let output_filename = format!("merge_output_{}.parquet", Ulid::new()); + let output_path = output_dir.join(&output_filename); + let file = std::fs::File::create(&output_path) + .with_context(|| format!("creating output file: {}", output_path.display()))?; + let writer = StreamingParquetWriter::try_new(file, Arc::clone(union_schema), props) + .with_context(|| format!("opening streaming writer for output {output_idx}"))?; + + Ok(OutputWriterStorage { + output_idx, + output_path, + writer, + num_row_groups: 0, + }) +} + +/// Compute `SortingColumn` entries from the union schema (no +/// RecordBatch needed — we just need the col indices). +pub(crate) fn build_sorting_columns_from_schema( + schema: &SchemaRef, + sort_fields_str: &str, +) -> Result> { + let parsed = parse_sort_fields(sort_fields_str)?; + let mut cols = Vec::new(); + for sf in &parsed.column { + // Schema may use `timestamp_secs` for what the sort schema + // calls `timestamp`. Match the existing alias handling. + let resolved = + if is_timestamp_column_name(&sf.name) && schema.index_of("timestamp_secs").is_ok() { + "timestamp_secs" + } else { + sf.name.as_str() + }; + let Ok(col_idx) = schema.index_of(resolved) else { + continue; + }; + cols.push(parquet::file::metadata::SortingColumn { + column_idx: col_idx as i32, + descending: sf.sort_direction + == quickwit_proto::sortschema::SortColumnDirection::SortDirectionDescending as i32, + nulls_first: false, + }); + } + Ok(cols) +} + +/// Finalize one output file: close writer, gather size, compute +/// per-output static metadata from the accumulator's sort col data, +/// return the `MergeOutputFile`. +pub(crate) fn finalize_output( + writer_state: OutputWriterStorage, + accumulator: OutputAccumulator, + input_meta: &InputMetadata, +) -> Result { + let OutputWriterStorage { + output_idx, + output_path, + mut writer, + num_row_groups, + } = writer_state; + + // Compute per-output metadata from the rows that actually landed + // in THIS output file. Merging eliminates key overlap between + // outputs, so the row_keys / zonemap / metric_names / time_range + // each output advertises must come from its own accumulated sort + // batch — they cannot be carried over from any input file. + let sort_batch = accumulator + .accumulated_sort_batch + .unwrap_or_else(|| RecordBatch::new_empty(Arc::new(ArrowSchema::empty()))); + + let row_keys_proto = if sort_batch.num_rows() > 0 { + row_keys::extract_row_keys(&input_meta.sort_fields, &sort_batch) + .with_context(|| format!("extracting row keys for output {output_idx}"))? + .map(|rk| row_keys::encode_row_keys_proto(&rk)) + } else { + None + }; + + let zonemap_opts = ZonemapOptions::default(); + let zonemap_regexes = if sort_batch.num_rows() > 0 { + zonemap::extract_zonemap_regexes(&input_meta.sort_fields, &sort_batch, &zonemap_opts) + .with_context(|| format!("extracting zonemap regexes for output {output_idx}"))? + } else { + HashMap::new() + }; + + let metric_names = + if sort_batch.num_rows() > 0 && sort_batch.schema().index_of("metric_name").is_ok() { + extract_metric_names(&sort_batch) + .with_context(|| format!("extracting metric names for output {output_idx}"))? + } else { + HashSet::new() + }; + + let time_range = + if sort_batch.num_rows() > 0 && sort_batch.schema().index_of("timestamp_secs").is_ok() { + extract_time_range(&sort_batch) + .with_context(|| format!("extracting time range for output {output_idx}"))? + } else { + crate::split::TimeRange::new(0, 0) + }; + + // Write the per-output `qh.row_keys` / `qh.zonemap_regexes` into + // the file's KV metadata so downstream tools that read the + // Parquet footer directly see the same values that + // `MergeOutputFile` carries in memory. + append_per_output_kv_metadata(&mut writer, row_keys_proto.as_ref(), &zonemap_regexes); + + let footer_metadata = writer + .close() + .with_context(|| format!("closing writer for output {output_idx}"))?; + + // MS-3: the count we report on `MergeOutputFile.num_row_groups` + // must agree with what the on-disk parquet footer actually + // contains. `num_row_groups` is bumped per `start_row_group` call + // in the per-region processor; if we ever skip one or double- + // count, this catches the drift in debug builds before downstream + // metadata consumers see the inconsistency. + debug_assert_eq!( + footer_metadata.num_row_groups(), + num_row_groups, + "MergeOutputFile.num_row_groups ({num_row_groups}) disagrees with footer ({})", + footer_metadata.num_row_groups(), + ); + + // Strong invariant: the streaming engine pairs at most one input + // RG per region (`process_region` keys `sort_col_batches` by + // input_idx), and the read-path check in + // `extract_regions_from_metadata` enforces unique prefix keys per + // input. The output writer assigns one new RG per region in + // `process_region`, so the output should never violate the + // invariant either — but verifying here catches any future + // regression (e.g., a finalize-time rewrite that splits a region + // across RGs) before the file lands. + super::region_grouping::assert_unique_rg_prefix_keys( + &footer_metadata, + &input_meta.sort_fields, + input_meta.rg_partition_prefix_len, + &format!("merge output {}", output_path.display()), + )?; + + let size_bytes = std::fs::metadata(&output_path) + .with_context(|| format!("stat output file: {}", output_path.display()))? + .len(); + + // If `service` is a sort column for this schema, it took the + // sort-col write path in `process_region` and the body-col + // `track_service` branch never saw it. Fold in the names from the + // accumulated sort batch so `TAG_SERVICE` metadata stays accurate + // regardless of which path wrote the column. + let mut service_names = accumulator.service_names; + if sort_batch.num_rows() > 0 + && let Ok(service_col_idx) = sort_batch.schema().index_of("service") + { + super::body_assembler::collect_service_names_from_page( + sort_batch.column(service_col_idx).as_ref(), + &mut service_names, + ) + .with_context(|| { + format!("collecting service names from sort col for output {output_idx}") + })?; + } + + let mut low_cardinality_tags: HashMap> = HashMap::new(); + if !service_names.is_empty() { + low_cardinality_tags.insert(TAG_SERVICE.to_string(), service_names); + } + + Ok(MergeOutputFile { + path: output_path, + num_rows: accumulator.num_rows, + num_row_groups, + // The streaming engine stamps `input_meta.rg_partition_prefix_len` + // unconditionally in the file's KV (see + // `open_output_writer_for_streaming`) and verifies the claim with + // `assert_unique_rg_prefix_keys`. CS-1 requires the metastore- + // recorded value to match the on-disk KV, so propagate the same + // value here. + output_rg_partition_prefix_len: input_meta.rg_partition_prefix_len, + size_bytes, + row_keys_proto, + zonemap_regexes, + metric_names, + time_range, + low_cardinality_tags, + }) +} + +/// Append the per-output `qh.row_keys` / `qh.zonemap_regexes` KV +/// entries to the streaming writer just before close. Encoded the +/// same way as the non-streaming writer (`build_merge_kv_metadata`): +/// base64 for the proto, JSON for the zonemap map, plus the optional +/// human-readable `qh.row_keys_json`. +fn append_per_output_kv_metadata( + writer: &mut StreamingParquetWriter, + row_keys_proto: Option<&Vec>, + zonemap_regexes: &HashMap, +) { + if let Some(rk_bytes) = row_keys_proto { + writer.append_key_value_metadata(parquet::file::metadata::KeyValue::new( + PARQUET_META_ROW_KEYS.to_string(), + BASE64.encode(rk_bytes), + )); + if let Ok(rk) = + ::decode(rk_bytes.as_slice()) + && let Ok(json) = serde_json::to_string(&rk) + { + writer.append_key_value_metadata(parquet::file::metadata::KeyValue::new( + PARQUET_META_ROW_KEYS_JSON.to_string(), + json, + )); + } + } + + if !zonemap_regexes.is_empty() { + let json = serde_json::to_string(zonemap_regexes) + .expect("HashMap JSON serialization cannot fail"); + writer.append_key_value_metadata(parquet::file::metadata::KeyValue::new( + PARQUET_META_ZONEMAP_REGEXES.to_string(), + json, + )); + } +} diff --git a/quickwit/quickwit-parquet-engine/src/merge/streaming/region_grouping.rs b/quickwit/quickwit-parquet-engine/src/merge/streaming/region_grouping.rs new file mode 100644 index 00000000000..7660bb3119f --- /dev/null +++ b/quickwit/quickwit-parquet-engine/src/merge/streaming/region_grouping.rs @@ -0,0 +1,802 @@ +// 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. + +//! Region grouping for multi-RG streaming merge. +//! +//! A "region" is one merge unit: a contiguous slice of the merged +//! output where all contributing inputs share the same sort-prefix +//! value. For multi-RG metric-aligned inputs each region pairs with +//! at most one row group per input, which is the property that makes +//! per-region streaming work without column-chunk-bounded buffering. +//! +//! This module owns the composite prefix-key encoding (sort-order +//! preserving across multiple columns / per-type encoding rules) and +//! the BTreeMap-based region grouping. It also enforces MS-2: the +//! BTreeMap iteration order must agree with each input's physical +//! row-group order, otherwise the streaming engine would `bail!` +//! mid-merge when a page arrives from an unexpected row group. +//! +//! The encoding rules per parquet physical type: +//! - `ByteArray` / `FixedLenByteArray`: byte-stuffed escape encoding — each `0x00` byte in the +//! value becomes `0x00 0x01`, and a `0x00 0x00` terminator is appended. This preserves +//! lexicographic order both for single columns (`"aa"` < `"b"`) and across concatenated composite +//! keys (the `0x00 0x00` terminator is the smallest possible 2-byte sequence under escaping, so +//! shorter values sort before longer values when prefixes match). +//! - `Int32` / `Int64`: sign-flipped big-endian so byte order matches numeric order across the full +//! signed range. +//! - `Boolean`: single 0/1 byte. +//! - DESC columns: per-byte complement of the encoding above so smaller values' bytes sort +//! *larger*. +//! - `Float` / `Double` / `Int96`: rejected with a clear error. + +use std::collections::{BTreeMap, HashSet}; + +use anyhow::{Context, Result, anyhow, bail}; +use arrow::array::RecordBatch; +use parquet::file::metadata::ParquetMetaData; + +use super::super::InputMetadata; +use super::super::merge_order::MergeRun; +use super::InputDecoderState; +use crate::sort_fields::{is_timestamp_column_name, parse_sort_fields}; +use crate::sorted_series::append_prefix_col_to_key; + +/// One merge region: a contiguous slice of the merged output, where all +/// contributing inputs share the same sort-prefix value (e.g., one +/// `metric_name` when `rg_partition_prefix_len == 1`). +/// +/// A region pairs with **at most one row group per input** — the +/// property that makes per-region streaming work without +/// column-chunk-bounded buffering. The `start_row` field on each +/// contribution lets a single row group be sliced across multiple +/// adjacent regions, which is how the engine subdivides a region at +/// `sorted_series` transitions to honor `num_outputs` when one +/// region (e.g. a giant single metric with `prefix_len=0`) would +/// otherwise occupy a single output file. +#[derive(Debug, Clone)] +pub(crate) struct Region { + /// Sort-prefix value identifying this region (e.g., `metric_name` + /// bytes for `prefix_len == 1`). Used only for ordering and + /// diagnostics; the merge engine doesn't decode this value. + pub(crate) prefix_key: Vec, + /// Per contributing input: which slice of which row group belongs + /// to this region. Ordered by `input_idx`. + pub(crate) contributing: Vec, +} + +/// One input's contribution to a region: the input index, the row +/// group within that input, and the row range within that row group +/// that belongs to the region. +/// +/// For top-level regions from `extract_regions_from_metadata` each +/// contribution covers a whole RG: `start_row == 0` and +/// `num_rows == rg.num_rows()`. Sub-regions produced by +/// `split_region_at_sorted_series` reference the same +/// `(input_idx, rg_idx)` as their parent with disjoint contiguous row +/// ranges. +#[derive(Debug, Clone)] +pub(crate) struct RegionContribution { + pub(crate) input_idx: usize, + pub(crate) rg_idx: usize, + pub(crate) start_row: usize, + pub(crate) num_rows: usize, +} + +impl Region { + pub(crate) fn total_rows(&self) -> usize { + self.contributing.iter().map(|c| c.num_rows).sum() + } +} + +/// A prefix column's location in the parquet schema, plus the sort +/// direction and ordinal position declared for it. `name` is the +/// sort-schema name (used in error messages); `parquet_col_idx` is +/// the resolved index in the parquet schema's flat column list +/// (after applying the `timestamp` / `timestamp_secs` alias); +/// `ordinal` is the column's 0-based position in `qh.sort_fields`, +/// matching `sorted_series`'s ordinal assignment so the per-RG +/// prefix key composes as a literal prefix of every row's +/// `sorted_series` key. +#[derive(Debug, Clone)] +pub(crate) struct PrefixColumn { + pub(crate) name: String, + pub(crate) parquet_col_idx: usize, + pub(crate) descending: bool, + pub(crate) ordinal: u8, +} + +/// Resolve the first `prefix_len` sort columns to parquet leaf +/// indices. Honours the legacy `timestamp` → `timestamp_secs` alias. +/// Errors if the sort schema has fewer columns than `prefix_len` or +/// if any column is missing from the parquet schema. +pub(crate) fn find_prefix_parquet_col_indices( + metadata: &ParquetMetaData, + sort_fields_str: &str, + prefix_len: usize, + input_idx: usize, +) -> Result> { + let sort_field_schema = parse_sort_fields(sort_fields_str)?; + if sort_field_schema.column.len() < prefix_len { + bail!( + "sort schema has {} columns but rg_partition_prefix_len = {prefix_len}", + sort_field_schema.column.len(), + ); + } + let parquet_schema = metadata.file_metadata().schema_descr(); + let mut prefix_cols = Vec::with_capacity(prefix_len); + for (pos, sort_col) in sort_field_schema.column.iter().take(prefix_len).enumerate() { + // Apply the same `timestamp` / `timestamp_secs` alias the rest + // of the engine uses. + let resolved = if is_timestamp_column_name(&sort_col.name) + && parquet_has_column(parquet_schema, "timestamp_secs") + { + "timestamp_secs" + } else { + sort_col.name.as_str() + }; + let mut found = None; + for (col_idx, col) in parquet_schema.columns().iter().enumerate() { + if col.path().parts()[0] == resolved { + found = Some(col_idx); + break; + } + } + let parquet_col_idx = found.ok_or_else(|| { + anyhow!( + "input {input_idx} parquet schema is missing prefix sort column '{}' (position \ + {pos})", + sort_col.name, + ) + })?; + let descending = sort_col.sort_direction + == quickwit_proto::sortschema::SortColumnDirection::SortDirectionDescending as i32; + // Ordinal matches the column's position in `qh.sort_fields`. + // For prefix cols (always the first `prefix_len` entries of + // the sort schema) the ordinal equals the iteration index + // `pos`, which is also the ordinal `sorted_series` would + // assign — so the per-RG prefix key composes as a literal + // byte prefix of every sorted_series key. + prefix_cols.push(PrefixColumn { + name: sort_col.name.clone(), + parquet_col_idx, + descending, + ordinal: pos as u8, + }); + } + Ok(prefix_cols) +} + +fn parquet_has_column( + parquet_schema: &parquet::schema::types::SchemaDescriptor, + name: &str, +) -> bool { + parquet_schema + .columns() + .iter() + .any(|c| c.path().parts()[0] == name) +} + +/// Build the composite byte key identifying a row group's prefix +/// values for grouping into a region. Per prefix column, contributes +/// `storekey(ordinal) || storekey(value)` (with value bytes inverted +/// for DESC columns) using +/// [`crate::sorted_series::append_prefix_col_to_key`], the same +/// encoding `sorted_series` produces — so this per-RG key is a +/// literal byte prefix of every `sorted_series` key emitted by rows +/// in this RG. +/// +/// Null handling: +/// - **All-null RG on a prefix column**: the column is skipped entirely (the next column's higher +/// ordinal byte appears in its place), so the RG sorts after any RG carrying a non-null value for +/// this column. This mirrors the row-level convention in `sorted_series` and gives nulls-last +/// ordering for free. +/// - **Mixed null + non-null in one RG**: rows in the RG would encode to two distinct prefix keys +/// (the non-null value's key and the column-skipped key), breaking the +/// at-most-one-prefix-value-per-RG invariant (PA-1). Reject. +/// - **No nulls**: standard `min == max` check on stats, then encode that single value. +pub(crate) fn extract_rg_composite_prefix_key( + metadata: &ParquetMetaData, + rg_idx: usize, + prefix_cols: &[PrefixColumn], + input_idx: usize, +) -> Result> { + let rg_meta = metadata.row_group(rg_idx); + let mut key = Vec::new(); + for col in prefix_cols { + let chunk = rg_meta.column(col.parquet_col_idx); + let stats = chunk.statistics().ok_or_else(|| { + anyhow!( + "input {input_idx} rg {rg_idx} col '{}' has no statistics — cannot determine \ + prefix alignment without min/max", + col.name, + ) + })?; + + // Parquet's `num_values` is total cell count including nulls. + // `null_count_opt()` returns the explicitly-recorded null + // count (defaulting to 0 when absent, per parquet-rs guidance). + let num_values = chunk.num_values().max(0) as u64; + let null_count = stats.null_count_opt().unwrap_or(0); + + if num_values > 0 && null_count == num_values { + // All-null RG: skip the column entirely (don't write its + // ordinal byte or value). The trailing prefix-length + // sentinel below ensures the resulting all-null key + // still sorts after any non-null key. See the sentinel + // comment for the full argument. + continue; + } + if null_count > 0 { + // PA-1 violation: see function doc. Parquet's min/max + // hide nulls, so an RG with N nulls + 1 non-null cell + // reports `min == max == non_null_value` even though + // rows in that RG encode to two distinct prefix keys. + bail!( + "input {input_idx} rg {rg_idx} col '{}' is NOT prefix-aligned: contains \ + {null_count} nulls plus {} non-null values. PA-1 requires each row group to \ + carry a single prefix value; rows with null on this column encode to a different \ + prefix key (with the column skipped) than rows with the non-null value.", + col.name, + num_values - null_count, + ); + } + + encode_prefix_col_value(stats, col, rg_idx, input_idx, &mut key)?; + } + + // Trailing prefix-length sentinel: an additional `u8(prefix_len)` + // ordinal byte that does two things at once: + // + // 1. **Forces nulls-last ordering across RGs.** For prefix_len=1 an all-null RG produces an + // empty per-column body and would otherwise lex-sort *before* any non-null RG. With the + // sentinel, the all-null key becomes `[prefix_len]` and the non-null key becomes `[ord(0), + // storekey(value), ..., prefix_len]`. The non-null key starts with `ord(0) = 0x00`, smaller + // than `prefix_len >= 1`, so non-null RGs sort first — matching `sorted_series`'s row-level + // nulls-last convention via the same "the next ordinal byte appears in the skipped slot" + // mechanism. + // 2. **Preserves the "literal prefix of sorted_series" property.** The byte we append is + // exactly what `sorted_series` writes right after the prefix columns: the ordinal of the + // next sort-schema column (`u8(prefix_len)`). So the per-RG key remains a byte-for-byte + // prefix of every row's `sorted_series` value in that RG. + storekey::encode(&mut key, &(prefix_cols.len() as u8)) + .map_err(|e| anyhow!("storekey encode prefix-length sentinel: {}", e))?; + + Ok(key) +} + +/// Verify `min == max` on the column chunk's non-null stats and +/// append the single value to `key` via +/// [`crate::sorted_series::append_prefix_col_to_key`] (which handles +/// the ordinal prefix + descending-direction byte inversion). Caller +/// has already filtered out all-null and mixed-null cases. +/// +/// `Statistics::ByteArray` values are routed through the +/// `Encode for str` impl after a UTF-8 check — every realistic sort +/// prefix column (`metric_name`, `service`, tag names) is UTF-8 +/// text, and `sorted_series` itself only encodes strings, so the +/// "byte prefix of sorted_series" property only holds for UTF-8 +/// values. Non-UTF-8 byte-array prefix cols would never match a +/// `sorted_series` key in practice (sorted_series would not encode +/// them either) and so are rejected up front. +fn encode_prefix_col_value( + stats: &parquet::file::statistics::Statistics, + col: &PrefixColumn, + rg_idx: usize, + input_idx: usize, + key: &mut Vec, +) -> Result<()> { + use parquet::file::statistics::Statistics; + + fn require_eq( + min: Option, + max: Option, + col_name: &str, + rg_idx: usize, + input_idx: usize, + ) -> Result { + let min = min.ok_or_else(|| { + anyhow!( + "input {input_idx} rg {rg_idx} col '{col_name}' has no min in stats — cannot \ + determine prefix alignment" + ) + })?; + let max = max.ok_or_else(|| { + anyhow!( + "input {input_idx} rg {rg_idx} col '{col_name}' has no max in stats — cannot \ + determine prefix alignment" + ) + })?; + if min != max { + bail!( + "input {input_idx} rg {rg_idx} is NOT prefix-aligned on col '{col_name}': min \ + ({:?}) != max ({:?}). Multi-RG inputs declaring `rg_partition_prefix_len >= 1` \ + must carry one prefix-value per RG.", + min, + max, + ); + } + Ok(min) + } + + fn encode_byte_array_value( + min_bytes: Option<&[u8]>, + max_bytes: Option<&[u8]>, + col: &PrefixColumn, + rg_idx: usize, + input_idx: usize, + key: &mut Vec, + ) -> Result<()> { + let value = require_eq( + min_bytes.map(|b| b.to_vec()), + max_bytes.map(|b| b.to_vec()), + &col.name, + rg_idx, + input_idx, + )?; + let s = std::str::from_utf8(&value).map_err(|_| { + anyhow!( + "input {input_idx} rg {rg_idx} col '{}' has non-UTF-8 byte-array prefix value; \ + only UTF-8 string prefix columns are supported (matching sorted_series's `&str` \ + encoding)", + col.name, + ) + })?; + append_prefix_col_to_key(key, col.ordinal, s, col.descending) + } + + match stats { + Statistics::ByteArray(v) => { + encode_byte_array_value( + v.min_bytes_opt(), + v.max_bytes_opt(), + col, + rg_idx, + input_idx, + key, + )?; + } + Statistics::FixedLenByteArray(v) => { + encode_byte_array_value( + v.min_bytes_opt(), + v.max_bytes_opt(), + col, + rg_idx, + input_idx, + key, + )?; + } + Statistics::Int32(v) => { + let value = require_eq( + v.min_opt().copied(), + v.max_opt().copied(), + &col.name, + rg_idx, + input_idx, + )?; + append_prefix_col_to_key(key, col.ordinal, &value, col.descending)?; + } + Statistics::Int64(v) => { + let value = require_eq( + v.min_opt().copied(), + v.max_opt().copied(), + &col.name, + rg_idx, + input_idx, + )?; + append_prefix_col_to_key(key, col.ordinal, &value, col.descending)?; + } + Statistics::Boolean(v) => { + let value = require_eq( + v.min_opt().copied(), + v.max_opt().copied(), + &col.name, + rg_idx, + input_idx, + )?; + append_prefix_col_to_key(key, col.ordinal, &value, col.descending)?; + } + Statistics::Float(_) | Statistics::Double(_) => bail!( + "prefix col '{}' is floating-point; composite-key extraction does not yet support \ + IEEE-754 ordering. Open an issue if you hit this — the encoding needs a sign-aware \ + bit flip on negative values.", + col.name, + ), + Statistics::Int96(_) => bail!( + "prefix col '{}' is Int96 (deprecated timestamp type); use Int64-encoded \ + `timestamp_secs` instead.", + col.name, + ), + } + Ok(()) +} + +/// MS-2: verify that, for each input, the regions list visits its +/// row groups in physical (on-disk) order. The streaming engine +/// drains each input sequentially — once we've moved past RG K's +/// pages we cannot go back. The composite-key BTreeMap iteration +/// must agree with that physical ordering for every input. +/// +/// Disagreement usually means the input file's sort direction was +/// declared one way but the data was written the other — e.g., the +/// sort schema says `metric_name ASC` but the file has RG 0 with +/// metric `z` and RG 1 with metric `a`. Reject upfront with a clear +/// message rather than letting `process_region` bail mid-merge with +/// "page from rg X while draining rg Y" — same error class, but +/// caught at a point where we can report the offending input and +/// region instead of the lower-level page mismatch. +pub(crate) fn validate_region_order_matches_physical_rg_order( + regions: &[Region], + num_inputs: usize, +) -> Result<()> { + let mut last_position_per_input: Vec> = vec![None; num_inputs]; + for (region_idx, region) in regions.iter().enumerate() { + for c in ®ion.contributing { + let position = (c.rg_idx, c.start_row); + if let Some(prev) = last_position_per_input[c.input_idx] + && position < prev + { + bail!( + "region iteration disagrees with input {}'s physical row order: region \ + {region_idx} wants rg {} row {} but a previous region already passed \ + position rg {} row {}. The composite prefix key encoding does not match the \ + input's physical layout — check that the sort schema's direction matches how \ + the file is actually sorted on disk.", + c.input_idx, + c.rg_idx, + c.start_row, + prev.0, + prev.1, + ); + } + last_position_per_input[c.input_idx] = Some((c.rg_idx, c.start_row + c.num_rows)); + } + } + Ok(()) +} + +/// Build the region list across all inputs. +/// +/// - If `rg_partition_prefix_len == 0`: all inputs must be single-RG (caller's job to validate); +/// produces ONE region with each input's only RG. The region's `prefix_key` is empty (no +/// alignment claim). +/// - If `rg_partition_prefix_len >= 1`: reads each input's per-RG prefix col stats (must have `min +/// == max`), groups RGs across inputs by composite prefix key, sorts regions by that key. +/// +/// Returns regions in sort order (sort prefix ASC). +pub(crate) fn extract_regions_from_metadata( + decoders_state: &[InputDecoderState], + input_meta: &InputMetadata, +) -> Result> { + if input_meta.rg_partition_prefix_len == 0 { + // No alignment claim: single region covering each input's only RG. + // Multi-RG inputs with prefix_len == 0 are rejected earlier; here + // each input is single-RG (or zero-RG). + // + // The caller (`run_synthesized_prefix_path`) will split this + // region further once it has read the merge order and can see + // where the first sort col's value transitions. + let mut contributing = Vec::new(); + for (idx, state) in decoders_state.iter().enumerate() { + if state.metadata.num_row_groups() == 0 { + continue; + } + let rg_meta = state.metadata.row_group(0); + contributing.push(RegionContribution { + input_idx: idx, + rg_idx: 0, + start_row: 0, + num_rows: rg_meta.num_rows() as usize, + }); + } + if contributing.is_empty() { + return Ok(Vec::new()); + } + return Ok(vec![Region { + prefix_key: Vec::new(), + contributing, + }]); + } + + // Prefix_len >= 1: build regions by composite prefix key from + // per-RG stats. See `extract_rg_composite_prefix_key` for the + // per-type encoding. + // + // **Strong invariant** (enforced here on the merge read path, and + // mirrored on both write paths — see `assert_unique_rg_prefix_keys`): + // no single input may have two row groups sharing the same composite + // prefix key. The streaming engine pairs at most one RG per input + // per region (`process_region` keys `sort_col_batches` by + // `input_idx`), so a duplicate prefix would silently overwrite the + // first RG's sort batch while `Region::total_rows` still counts both + // — dropping rows and corrupting body-col / sort-col alignment. + // + // Cross-input duplicates are fine (and expected — that's the whole + // point of region merging). The constraint is **same input, same + // prefix key, multiple RGs**: producers must ensure prefix + // transitions align with RG boundaries. + let mut by_prefix: BTreeMap, Vec> = BTreeMap::new(); + let prefix_len = input_meta.rg_partition_prefix_len as usize; + + for (input_idx, state) in decoders_state.iter().enumerate() { + if state.metadata.num_row_groups() == 0 { + continue; + } + let prefix_cols = find_prefix_parquet_col_indices( + &state.metadata, + &input_meta.sort_fields, + prefix_len, + input_idx, + ) + .with_context(|| format!("resolving prefix cols for input {input_idx}"))?; + let mut seen_for_input: HashSet> = HashSet::new(); + for rg_idx in 0..state.metadata.num_row_groups() { + let prefix_key = + extract_rg_composite_prefix_key(&state.metadata, rg_idx, &prefix_cols, input_idx)?; + if !seen_for_input.insert(prefix_key.clone()) { + bail!( + "input {input_idx} has rg {rg_idx} sharing a prefix key with an earlier RG in \ + the same file. The streaming merge engine requires at-most-one-RG-per-input \ + per prefix value (rg_partition_prefix_len = {prefix_len}); the producer must \ + ensure prefix transitions align with RG boundaries. Either lower \ + rg_partition_prefix_len to include fewer columns, or rewrite the producer to \ + start a new RG at every prefix-value change." + ); + } + let num_rows = state.metadata.row_group(rg_idx).num_rows() as usize; + by_prefix + .entry(prefix_key) + .or_default() + .push(RegionContribution { + input_idx, + rg_idx, + start_row: 0, + num_rows, + }); + } + } + + Ok(by_prefix + .into_iter() + .map(|(prefix_key, contributing)| Region { + prefix_key, + contributing, + }) + .collect()) +} + +/// Post-write check: verify the parquet file at `metadata` has no two +/// row groups sharing the same composite prefix key, for the first +/// `prefix_len` sort columns. Returns `Ok(())` immediately if +/// `prefix_len == 0` (no alignment claim). +/// +/// This is the writer-side mirror of the read-side check in +/// `extract_regions_from_metadata` — both indexing and the compaction +/// merge output writer call this after sealing a parquet file so a +/// producer bug never lets a duplicate-prefix file land on disk. See +/// the doc-comment on `extract_regions_from_metadata` for why +/// at-most-one-RG-per-prefix is load-bearing for the streaming +/// engine. +/// +/// `context` is included in the error message — e.g., +/// `"indexing write at "` or `"merge output "`. +pub(crate) fn assert_unique_rg_prefix_keys( + metadata: &ParquetMetaData, + sort_fields_str: &str, + prefix_len: u32, + context: &str, +) -> Result<()> { + if prefix_len == 0 { + return Ok(()); + } + let num_rgs = metadata.num_row_groups(); + if num_rgs <= 1 { + // Single-RG (or zero-RG) files vacuously satisfy the invariant. + return Ok(()); + } + let prefix_cols = + find_prefix_parquet_col_indices(metadata, sort_fields_str, prefix_len as usize, 0) + .with_context(|| format!("resolving prefix cols for {context}"))?; + let mut seen: HashSet> = HashSet::with_capacity(num_rgs); + for rg_idx in 0..num_rgs { + let key = extract_rg_composite_prefix_key(metadata, rg_idx, &prefix_cols, 0) + .with_context(|| format!("extracting prefix key at {context} rg {rg_idx}"))?; + if !seen.insert(key) { + bail!( + "{context}: rg {rg_idx} shares a prefix key with an earlier row group. \ + rg_partition_prefix_len = {prefix_len} requires prefix transitions to align with \ + row group boundaries. Either lower the prefix length to include fewer columns, \ + or change the writer so each RG carries a unique value of the first {prefix_len} \ + sort columns." + ); + } + } + Ok(()) +} + +/// Subdivide a region into a sequence of sub-regions whose cumulative +/// row counts approach `target_per_output`, splitting only at +/// `sorted_series` transitions within the region's merge order. A +/// single `sorted_series` run is never broken — if one run exceeds +/// the remaining budget, the whole run goes to one output anyway. +/// +/// `first_target` is the budget for the FIRST sub-region (typically +/// the remaining capacity of the current output being filled by the +/// caller). Subsequent sub-regions target `target_per_output`. +/// `outputs_remaining` is the number of output files still available; +/// when it hits 1 we stop splitting and emit the rest as one sub- +/// region. +/// +/// The returned sub-regions: +/// - Cover the full input region in sort order. +/// - Each carries per-input row ranges (`start_row`/`num_rows`) inside the same `(input_idx, +/// rg_idx)` as the parent — sub-regions of one region all share their parent's RGs. +/// - Inherit the parent's `prefix_key`; the prefix value is constant across the parent and +/// therefore across every sub-region. +pub(crate) fn split_region_at_sorted_series( + region: &Region, + merge_order: &[MergeRun], + aligned_sort_batches: &[RecordBatch], + first_target: usize, + target_per_output: usize, + outputs_remaining: usize, +) -> Result> { + use arrow::array::BinaryArray; + + use crate::sorted_series::SORTED_SERIES_COLUMN; + + if merge_order.is_empty() { + return Ok(Vec::new()); + } + if outputs_remaining <= 1 { + return Ok(vec![region.clone()]); + } + + // Per-input sorted_series array. compute_merge_order already + // requires this column on every input, so a missing-column case + // here is a bug rather than a configuration error. + let mut ss_arrays: Vec> = Vec::with_capacity(aligned_sort_batches.len()); + for batch in aligned_sort_batches { + match batch.schema().index_of(SORTED_SERIES_COLUMN) { + Ok(idx) => { + let arr = batch + .column(idx) + .as_any() + .downcast_ref::() + .ok_or_else(|| anyhow!("`{SORTED_SERIES_COLUMN}` must be Binary-typed"))?; + ss_arrays.push(Some(arr)); + } + Err(_) => ss_arrays.push(None), + } + } + + let ss_at = |run_idx: usize| -> Option<&[u8]> { + let run = &merge_order[run_idx]; + ss_arrays[run.input_index].map(|a| a.value(run.start_row)) + }; + + // Walk runs, splitting before a run whose preceding sorted_series + // transition crosses the current target. We can only split at run + // boundaries (a run has constant sorted_series internally), so + // breaking inside a run is impossible — a giant single-series run + // simply lands in one output regardless of size. + let mut splits: Vec> = Vec::new(); + let mut current_start: usize = 0; + let mut accumulated: usize = 0; + let mut current_target = first_target; + let mut outputs_left = outputs_remaining; + + for (run_idx, run) in merge_order.iter().enumerate() { + if run_idx > 0 && outputs_left > 1 && accumulated >= current_target { + let prev_ss = ss_at(run_idx - 1); + let curr_ss = ss_at(run_idx); + let at_transition = match (prev_ss, curr_ss) { + (Some(a), Some(b)) => a != b, + _ => true, + }; + if at_transition { + splits.push(current_start..run_idx); + current_start = run_idx; + accumulated = 0; + outputs_left -= 1; + current_target = target_per_output; + } + } + accumulated += run.row_count; + } + splits.push(current_start..merge_order.len()); + + // Build each sub-region's contributing list from the runs in its + // range. Within a sub-region, each input's rows are contiguous + // (the merge engine consumes rows in increasing input-row order + // and the parent region's contributions are themselves + // contiguous), so a `(min_run.start_row, sum_row_count)` range + // captures the full slice. + let rg_for_input: std::collections::HashMap = region + .contributing + .iter() + .map(|c| (c.input_idx, c.rg_idx)) + .collect(); + let parent_start_row: std::collections::HashMap = region + .contributing + .iter() + .map(|c| (c.input_idx, c.start_row)) + .collect(); + + let mut sub_regions: Vec = Vec::with_capacity(splits.len()); + for range in splits { + let mut ranges: BTreeMap = BTreeMap::new(); + for run in &merge_order[range.clone()] { + let entry = ranges + .entry(run.input_index) + .or_insert((run.start_row, run.start_row)); + entry.0 = entry.0.min(run.start_row); + entry.1 = entry.1.max(run.start_row + run.row_count); + } + let contributing: Vec = ranges + .into_iter() + .map(|(input_idx, (start, end))| RegionContribution { + input_idx, + rg_idx: *rg_for_input.get(&input_idx).expect("rg_idx from parent"), + // The merge order's run.start_row is local to the + // aligned sort batch (which itself is the drained + // contribution); add the parent's start_row to get + // the absolute row inside the RG. + start_row: parent_start_row.get(&input_idx).copied().unwrap_or(0) + start, + num_rows: end - start, + }) + .collect(); + sub_regions.push(Region { + prefix_key: region.prefix_key.clone(), + contributing, + }); + } + + Ok(sub_regions) +} + +/// Assign each region to an output file index. +/// +/// Splits the region list across `num_outputs` files, balancing +/// cumulative row count. Each output file gets a contiguous slice of +/// the region list (preserving sort-prefix order so output files have +/// non-overlapping key ranges). Returns a `Vec` indexed by +/// `region_idx` with the target output file index. +/// +/// If `regions.len() < num_outputs`, fewer output files are produced +/// (matches the non-streaming engine's behaviour when there aren't +/// enough split points). +pub(crate) fn assign_regions_to_output_files(regions: &[Region], num_outputs: usize) -> Vec { + let total_rows: usize = regions.iter().map(|r| r.total_rows()).sum(); + let effective_num_outputs = num_outputs.min(regions.len()).max(1); + let target_rows_per_output = total_rows.div_ceil(effective_num_outputs).max(1); + + let mut assignments = Vec::with_capacity(regions.len()); + let mut current_output = 0; + let mut accumulated = 0; + for region in regions { + // If this region would push us past the target AND we have + // budget to start a new output AND the current output already + // has rows, advance to next output BEFORE assigning. + if accumulated > 0 + && accumulated + region.total_rows() > target_rows_per_output + && current_output + 1 < effective_num_outputs + { + current_output += 1; + accumulated = 0; + } + assignments.push(current_output); + accumulated += region.total_rows(); + } + assignments +} diff --git a/quickwit/quickwit-parquet-engine/src/merge/writer.rs b/quickwit/quickwit-parquet-engine/src/merge/writer.rs index 340ac93b0b4..6833820cf0c 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/writer.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/writer.rs @@ -169,6 +169,7 @@ pub fn write_merge_outputs( path: output_path, num_rows: sorted_batch.num_rows(), num_row_groups: written.num_row_groups, + output_rg_partition_prefix_len: output_prefix_len, size_bytes: written.size_bytes, row_keys_proto, zonemap_regexes, diff --git a/quickwit/quickwit-parquet-engine/src/sorted_series/mod.rs b/quickwit/quickwit-parquet-engine/src/sorted_series/mod.rs index e7fac1e8047..2e9db1203dd 100644 --- a/quickwit/quickwit-parquet-engine/src/sorted_series/mod.rs +++ b/quickwit/quickwit-parquet-engine/src/sorted_series/mod.rs @@ -234,29 +234,16 @@ fn encode_row_key( buf: &mut Vec, ) -> Result<()> { // Encode non-null sort schema columns: ordinal + string value. + // Null columns are skipped entirely so the next column's higher + // ordinal byte appears in their place — that's how nulls-last + // ordering is achieved without a sentinel marker. for kc in tag_columns { let col = batch.column(kc.batch_idx); if col.is_null(row_idx) { continue; } let value = extract_string_value(col.as_ref(), row_idx)?; - if kc.descending { - // Ordinal is written normally (ascending) so that null rows - // (which skip this column entirely) sort after non-null rows - // — matching the writer's nulls_first=false behavior. - // Only the value bytes are inverted to reverse the sort order. - storekey::encode(&mut *buf, &kc.ordinal) - .map_err(|e| anyhow!("storekey encode ordinal: {}", e))?; - let start = buf.len(); - storekey::encode(&mut *buf, value) - .map_err(|e| anyhow!("storekey encode value: {}", e))?; - invert_bytes(&mut buf[start..]); - } else { - storekey::encode(&mut *buf, &kc.ordinal) - .map_err(|e| anyhow!("storekey encode ordinal: {}", e))?; - storekey::encode(&mut *buf, value) - .map_err(|e| anyhow!("storekey encode value: {}", e))?; - } + append_prefix_col_to_key(buf, kc.ordinal, value, kc.descending)?; } // Append timeseries_id with its ordinal as the final discriminator. @@ -270,14 +257,47 @@ fn encode_row_key( row_idx ); let ts_id = extract_i64_value(col.as_ref(), row_idx); - storekey::encode(&mut *buf, &ts_id_column.ordinal) - .map_err(|e| anyhow!("storekey encode timeseries_id ordinal: {}", e))?; - storekey::encode(&mut *buf, &ts_id) - .map_err(|e| anyhow!("storekey encode timeseries_id: {}", e))?; + append_prefix_col_to_key(buf, ts_id_column.ordinal, &ts_id, false)?; Ok(()) } +/// Append `(ordinal, value)` to `buf` using the project's standard +/// storekey-based sort-prefix encoding. This is the single source of +/// truth for the on-the-wire format of one sort-schema column's +/// contribution to a composite key — used by both +/// [`compute_sorted_series_column`] (per-row sorted_series key) and +/// `merge::region_grouping::extract_rg_composite_prefix_key` +/// (per-RG prefix key for region grouping). Because both call sites +/// share this encoding, a per-RG prefix key is a *literal byte +/// prefix* of every sorted_series key produced by rows in that RG. +/// +/// Layout: `storekey(ordinal: u8) || storekey(value)`. For descending +/// columns the *value* bytes are inverted in place (NOT the ordinal +/// byte) so memcmp on the composite reverses the value's lex order +/// while ordinals stay in declared order. Caller skips this function +/// entirely for null columns — the next column's higher ordinal byte +/// then appears in this column's place, which gives nulls-last +/// ordering without a sentinel marker (matches the writer's +/// `nulls_first=false` convention). +pub(crate) fn append_prefix_col_to_key( + buf: &mut Vec, + ordinal: u8, + value: &T, + descending: bool, +) -> Result<()> +where + T: ?Sized + storekey::Encode, +{ + storekey::encode(&mut *buf, &ordinal).map_err(|e| anyhow!("storekey encode ordinal: {}", e))?; + let value_start = buf.len(); + storekey::encode(&mut *buf, value).map_err(|e| anyhow!("storekey encode value: {}", e))?; + if descending { + invert_bytes(&mut buf[value_start..]); + } + Ok(()) +} + /// Bitwise-NOT a byte slice in place, inverting the sort order for /// descending columns in the composite key. This is the standard /// ordered-code technique: if ascending bytes A < B, then !A > !B, diff --git a/quickwit/quickwit-parquet-engine/src/storage/streaming_writer.rs b/quickwit/quickwit-parquet-engine/src/storage/streaming_writer.rs index 550c03ce318..3b83ff1d332 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/streaming_writer.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/streaming_writer.rs @@ -153,6 +153,16 @@ impl StreamingParquetWriter { }) } + /// Append a `KeyValue` entry to the file's footer metadata. Useful + /// when the value can only be computed after all row groups have + /// been written (e.g., per-output `qh.row_keys` / `qh.zonemap_regexes` + /// in the streaming merge, which derive from the rows that + /// physically land in this output file). Must be called before + /// [`Self::close`] flushes the footer. + pub(crate) fn append_key_value_metadata(&mut self, kv: parquet::file::metadata::KeyValue) { + self.file_writer.append_key_value_metadata(kv); + } + /// Close the file and return its metadata. pub(crate) fn close(self) -> Result { Ok(self.file_writer.close()?) diff --git a/quickwit/quickwit-parquet-engine/src/storage/writer.rs b/quickwit/quickwit-parquet-engine/src/storage/writer.rs index 1eeb87992f4..d6315d27f42 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/writer.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/writer.rs @@ -533,6 +533,8 @@ impl ParquetWriter { writer.write(&prepared.sorted_batch)?; let bytes = writer.into_inner()?.into_inner(); + assert_indexing_prefix_alignment(&bytes, split_metadata, &self.sort_fields_string)?; + debug!(bytes_written = bytes.len(), "completed write to bytes"); Ok((bytes, (prepared.row_keys_proto, prepared.zonemap_regexes))) } @@ -555,6 +557,7 @@ impl ParquetWriter { writer.write(&prepared.sorted_batch)?; let bytes_written = writer.into_inner()?.metadata()?.len(); + assert_indexing_prefix_alignment_on_file(path, split_metadata, &self.sort_fields_string)?; debug!(bytes_written, "completed write to file"); Ok(( bytes_written, @@ -563,6 +566,81 @@ impl ParquetWriter { } } +/// Verify the just-written parquet bytes have no two row groups +/// sharing the same composite prefix key. No-op when split metadata +/// is absent or `rg_partition_prefix_len == 0` (no alignment claim). +/// +/// Mirrors the read-path check in +/// `merge::streaming::region_grouping::extract_regions_from_metadata` +/// and the merge-output check in `streaming::output::finalize_output` +/// — see `assert_unique_rg_prefix_keys` for the rationale. +fn assert_indexing_prefix_alignment( + bytes: &[u8], + split_metadata: Option<&ParquetSplitMetadata>, + fallback_sort_fields: &str, +) -> Result<(), ParquetWriteError> { + let Some(meta) = split_metadata else { + return Ok(()); + }; + if meta.rg_partition_prefix_len == 0 { + return Ok(()); + } + let sort_fields = if meta.sort_fields.is_empty() { + fallback_sort_fields + } else { + meta.sort_fields.as_str() + }; + let bytes_owned = bytes::Bytes::copy_from_slice(bytes); + let reader = parquet::file::reader::SerializedFileReader::new(bytes_owned).map_err(|e| { + ParquetWriteError::SchemaValidation(format!( + "post-write prefix alignment re-parse failed: {e}" + )) + })?; + use parquet::file::reader::FileReader; + crate::merge::streaming::region_grouping::assert_unique_rg_prefix_keys( + reader.metadata(), + sort_fields, + meta.rg_partition_prefix_len, + "indexing write_to_bytes", + ) + .map_err(|e| ParquetWriteError::SchemaValidation(e.to_string())) +} + +/// File-backed counterpart to `assert_indexing_prefix_alignment` — +/// re-opens the just-written file and verifies the same invariant. +fn assert_indexing_prefix_alignment_on_file( + path: &Path, + split_metadata: Option<&ParquetSplitMetadata>, + fallback_sort_fields: &str, +) -> Result<(), ParquetWriteError> { + let Some(meta) = split_metadata else { + return Ok(()); + }; + if meta.rg_partition_prefix_len == 0 { + return Ok(()); + } + let sort_fields = if meta.sort_fields.is_empty() { + fallback_sort_fields + } else { + meta.sort_fields.as_str() + }; + let file = File::open(path)?; + let reader = parquet::file::reader::SerializedFileReader::new(file).map_err(|e| { + ParquetWriteError::SchemaValidation(format!( + "post-write prefix alignment re-open failed for {}: {e}", + path.display() + )) + })?; + use parquet::file::reader::FileReader; + crate::merge::streaming::region_grouping::assert_unique_rg_prefix_keys( + reader.metadata(), + sort_fields, + meta.rg_partition_prefix_len, + &format!("indexing write_to_file {}", path.display()), + ) + .map_err(|e| ParquetWriteError::SchemaValidation(e.to_string())) +} + /// Parse a sort fields string and resolve column names to physical `ParquetField`s. /// /// Columns not present in the current schema (e.g., `timeseries_id`) are silently @@ -1499,4 +1577,103 @@ mod tests { std::fs::remove_file(&path).ok(); } + + /// Indexing-path strong invariant: when the writer is asked to + /// emit a file with `rg_partition_prefix_len > 0`, no two row + /// groups may share the same composite prefix key. Forcing a + /// small row_group_size on a batch where every row has the same + /// metric_name produces multi-RG output with all RGs sharing the + /// prefix value — and `write_to_bytes` must reject the result + /// rather than land a corrupt file. Mirrors the merge-read-side + /// check in `extract_regions_from_metadata`. + #[test] + fn test_write_to_bytes_rejects_duplicate_rg_prefix_when_claimed_aligned() { + use crate::split::{ParquetSplitId, TimeRange}; + + let config = ParquetWriterConfig::default().with_row_group_size(3); + let writer = ParquetWriter::new(config, &TableConfig::default()).unwrap(); + + // 9 rows of cpu.usage at 3 rows per RG → 3 RGs, all with the + // same metric_name. Claiming rg_partition_prefix_len = 1 makes + // every RG share the prefix key, which must fail. + let batch = create_test_batch_with_tags(9, &["service", "env"]); + let split_metadata = ParquetSplitMetadata::metrics_builder() + .split_id(ParquetSplitId::new("dup-prefix-rejection-test")) + .index_uid("test-index:00000000000000000000000000") + .time_range(TimeRange::new(100, 200)) + .sort_fields("metric_name|service|env|timeseries_id|timestamp_secs/V2") + .rg_partition_prefix_len(1) + .build(); + + let err = writer + .write_to_bytes(&batch, Some(&split_metadata)) + .expect_err("must reject duplicate prefix RGs at indexing write"); + let msg = err.to_string(); + assert!( + msg.contains("shares a prefix key with an earlier row group"), + "expected duplicate-prefix error, got: {msg}", + ); + assert!( + msg.contains("indexing"), + "error should mention the indexing context, got: {msg}", + ); + } + + /// File-backed counterpart: `write_to_file_with_metadata` must + /// fire the same check. The output file exists on disk by the + /// time we re-parse, but the writer surfaces the violation rather + /// than reporting success. + #[test] + fn test_write_to_file_rejects_duplicate_rg_prefix_when_claimed_aligned() { + use crate::split::{ParquetSplitId, TimeRange}; + + let config = ParquetWriterConfig::default().with_row_group_size(3); + let writer = ParquetWriter::new(config, &TableConfig::default()).unwrap(); + + let batch = create_test_batch_with_tags(9, &["service", "env"]); + let split_metadata = ParquetSplitMetadata::metrics_builder() + .split_id(ParquetSplitId::new("dup-prefix-rejection-file-test")) + .index_uid("test-index:00000000000000000000000000") + .time_range(TimeRange::new(100, 200)) + .sort_fields("metric_name|service|env|timeseries_id|timestamp_secs/V2") + .rg_partition_prefix_len(1) + .build(); + + let temp_dir = std::env::temp_dir(); + let path = temp_dir.join("test_dup_prefix_rejection.parquet"); + let err = writer + .write_to_file_with_metadata(&batch, &path, Some(&split_metadata)) + .expect_err("must reject duplicate prefix RGs at indexing write_to_file"); + let msg = err.to_string(); + assert!( + msg.contains("shares a prefix key with an earlier row group"), + "expected duplicate-prefix error, got: {msg}", + ); + + std::fs::remove_file(&path).ok(); + } + + /// Companion to the duplicate-rejection tests: a single-RG file + /// vacuously satisfies any prefix_len claim. The writer must + /// accept it without raising the duplicate-prefix error. + #[test] + fn test_write_to_bytes_accepts_single_rg_with_prefix_len_one() { + use crate::split::{ParquetSplitId, TimeRange}; + + let config = ParquetWriterConfig::default(); + let writer = ParquetWriter::new(config, &TableConfig::default()).unwrap(); + + let batch = create_test_batch_with_tags(20, &["service", "env"]); + let split_metadata = ParquetSplitMetadata::metrics_builder() + .split_id(ParquetSplitId::new("single-rg-test")) + .index_uid("test-index:00000000000000000000000000") + .time_range(TimeRange::new(100, 200)) + .sort_fields("metric_name|service|env|timeseries_id|timestamp_secs/V2") + .rg_partition_prefix_len(1) + .build(); + + writer + .write_to_bytes(&batch, Some(&split_metadata)) + .expect("single-RG must pass the prefix-alignment check"); + } } From 0c7de766d00fc9199a708b66876877f0805e7d5a Mon Sep 17 00:00:00 2001 From: George Talbot Date: Mon, 18 May 2026 09:37:27 -0400 Subject: [PATCH 06/10] feat(legacy-adapter): prefix-aware output with caller-supplied target_prefix_len (#6425) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * feat(legacy-adapter): synthesize prefix-aligned row groups The legacy adapter previously consolidated multi-RG legacy inputs into a single oversized row group and left `rg_partition_prefix_len` at the original's (typically `0`). The streaming merge engine then sent these single-RG/prefix=0 inputs through the new sub-region splitting path — correct, but it forfeits the prefix-aware fast path for outputs derived from legacy inputs and gives up the row-group pruning that prefix alignment enables. After consolidating, the adapter now slices the resulting record batch at first-sort-col transitions (typically `metric_name`) and emits one parquet row group per slice, stamping the re-encoded file with `qh.rg_partition_prefix_len = 1`. The merge engine then reads it through the prefix-aware fast path: one region per metric_name, the existing duplicate-prefix invariant on read validates uniqueness. Fallback: if the original file has no `qh.sort_fields` KV, the sort-fields string fails to parse, the first column can't be resolved in the arrow schema, or the consolidated batch is empty, the adapter reverts to a single-RG re-encode without claiming any prefix alignment. That input still works — the engine's prefix_len=0 sub-region splitting path picks it up. This keeps the adapter robust for files written by very early versions of the indexer that may pre-date the standard KV layout. Implementation: `reencode_prefix_aligned` replaces `reencode_as_single_row_group` and either dispatches to the new multi-RG writer or to the legacy single-RG writer based on whether the first sort col is resolvable. `RowConverter` handles the prefix-value equality check uniformly across dictionary, utf8, and primitive types. The KV injection helper replaces (rather than appends) any existing `qh.rg_partition_prefix_len` so re-runs and files mistakenly carrying a stale value still land at the freshly synthesized prefix. Tests: - `test_legacy_input_with_sort_fields_produces_prefix_aligned_multi_rg` — 3 metrics × 40 rows, multi-RG input → 3 prefix-aligned output RGs and `qh.rg_partition_prefix_len = 1` KV. - `test_legacy_input_single_metric_yields_one_rg_with_prefix_kv` — one metric → one RG, prefix KV still stamped (vacuously aligned). - `test_legacy_input_without_sort_fields_falls_back_to_single_rg` — fallback path preserved when sort-fields KV is missing. - All existing tests pass unchanged (they use empty KVs or unparseable sort-fields strings, both of which exercise the fallback path). Co-Authored-By: Claude Opus 4.7 (1M context) * feat(legacy-adapter): parameterize on target_prefix_len with composite-prefix support `LegacyInputAdapter::try_open` now takes `target_prefix_len: u32` chosen by the caller, matching the merge plan's consensus prefix length. The adapter slices the consolidated batch at every transition of the first N sort columns (composite key, via `RowConverter` over all N fields) and emits one output row group per slice, stamping the output with `qh.rg_partition_prefix_len = target_prefix_len`. With `target_prefix_len = 0` the adapter takes the original single-RG passthrough path with no prefix-alignment claim. A sort column that is named in `qh.sort_fields` but missing from the file's arrow schema is treated as implicitly null at every row per SS-3. A constantly-null column trivially satisfies alignment on that column (null == null) and contributes no transitions, so the split boundaries are driven by the columns that are present. This matches the merge engine's compaction-time treatment of missing columns and keeps a legacy file with an evolved schema usable as a prefix-aligned input. `PrefixUnresolvable` now fires only on cases where the file doesn't advertise enough sort *names* to honor the request: - `qh.sort_fields` absent or unparseable - `qh.sort_fields` declares fewer sort columns than `target_prefix_len` A column missing from the arrow schema no longer counts as unresolvable; the adapter materialises a `NullArray` of the batch's length in that slot and proceeds. Tests: - `test_target_prefix_len_zero_passes_through_as_single_rg` — explicit N=0 fallback, no prefix KV stamped. - `test_target_prefix_len_two_splits_by_metric_and_service` — composite prefix (`metric_name`, `service`) → 4 RGs, KV declares prefix_len=2. - `test_target_prefix_len_one_without_sort_fields_returns_unresolvable` — no `qh.sort_fields` KV → `PrefixUnresolvable`. - `test_target_prefix_len_exceeds_declared_sort_cols_returns_unresolvable` — sort schema declares 2 cols, caller asks 3 → `PrefixUnresolvable`. - `test_missing_prefix_col_treated_as_null_satisfies_alignment` — sort schema declares `metric_name|env|-timestamp_secs` but `env` is absent from the arrow schema → no error, only metric_name transitions split RGs, KV still stamps prefix_len=2. Co-Authored-By: Claude Opus 4.7 (1M context) * docs(legacy_adapter): note where reader-side SS-3 handling lands Codex P2 on PR #6425: the adapter records `None` for missing prefix columns and stamps `rg_partition_prefix_len = target_prefix_len` anyway. In isolation that produces a file with an advertised prefix the current reader (`find_prefix_parquet_col_indices` on the #6425 state) bails on. The reader-side fix — returning `Vec>` and synthesizing a constant `[0x00, 0x00]` byte for `None` slots — lands in PR #6426 (the hardening slice, F12 from the adversarial review). The only caller of this adapter is `execute_merge_operation`, introduced in PR #6423 which sits above #6426 in the stack, so no production caller can produce a missing- column prefix until the reader fix is in place. Adding the in-code pointer so a future reader bisecting the stack doesn't have to trace the relationship from scratch. Co-Authored-By: Claude Opus 4.7 (1M context) * fix(merge): consumer honors SS-3 (move F12 forward from #6426 to #6425) Previously the F12 fix — "consumer side honors SS-3 missing prefix columns" — lived in the hardening PR (#6426). At the #6425 isolation level, the legacy adapter records `None` for a prefix column absent from the parquet schema and stamps `rg_partition_prefix_len = target_prefix_len` on the output, but the reader's `find_prefix_parquet_col_indices` bails on any missing column. So #6425 + #6424 alone would produce a legacy-adapter file that the streaming-merge reader rejects mid-merge — i.e. a known- incoherent intermediate stack state. Move F12 into this PR so the adapter and reader agree at the same slice: - `find_prefix_parquet_col_indices` now returns `Result>>`. `Some(_)` when the column is present in the parquet schema; `None` per SS-3 when the column is named in `qh.sort_fields` but absent from the schema. - `extract_rg_composite_prefix_key` skips `None` slots entirely (no ordinal byte, no value bytes for that column). The trailing `u8(prefix_len)` sentinel introduced in the storekey refactor keeps the resulting key well-formed across present/absent columns. - Callers that index into `prefix_cols` updated to use `.as_ref().expect(…)` where they assume presence. Existing SS-3 test `test_missing_prefix_col_treated_as_null_satisfies_alignment` in `legacy_adapter.rs` gets an `assert_unique_rg_prefix_keys` call verifying the adapter's output is consumable by the reader — pins the "stack-coherent at #6425" property the F12 hop establishes. Also incidental nightly-fmt cleanups in `sorted_series::append_prefix_col_to_key` and the two-input fixture in `test_all_null_prefix_rg_groups_into_separate_region_sorted_last`. The hardening PR (#6426) will be re-cascaded to drop the now- duplicated F12 hunks (keeping its F8 adapter-rejects-unsorted + F2 verifier-strength changes intact). 485 lib tests pass on this slice; workspace clippy + nightly fmt clean. Co-Authored-By: Claude Opus 4.7 (1M context) * fix(legacy-adapter): strip stale rg_partition_prefix_len when target=0 Codex P2 on PR #6425: when the legacy adapter is called with `target_prefix_len == 0` it consolidates the input into a single RG, but the previous version preserved the input's footer KVs unchanged. If the input itself already carried a stale nonzero `qh.rg_partition_prefix_len` claim (e.g., a prefix-aware split being re-encoded through the legacy fallback path), the single-RG output would still advertise that claim. Downstream metadata extraction would take the prefix-aware path against an RG carrying multiple first-prefix values — failing the PA-1 min/max alignment check on read despite the caller explicitly asking for the legacy path. Strip `PARQUET_META_RG_PARTITION_PREFIX_LEN` from `original_kv` in the `target_prefix_len == 0` branch. Absence of the KV is the legacy convention for "no alignment claim", matching the existing `test_target_prefix_len_zero_passes_through_as_single_rg` test's `prefix_kv.is_none()` assertion. New regression test `test_target_prefix_len_zero_strips_stale_prefix_kv_from_input`: inputs a 2-RG file with `qh.rg_partition_prefix_len = "1"` AND opens through adapter with `target_prefix_len = 0`; asserts the re-encoded output has no prefix KV. Pre-fix this test caught the leak; post-fix the stale value is dropped. 487 lib tests pass on the slice; clippy + nightly fmt clean. Co-Authored-By: Claude Opus 4.7 (1M context) --------- Co-authored-by: Claude Opus 4.7 (1M context) --- .../src/merge/streaming.rs | 5 +- .../src/merge/streaming/region_grouping.rs | 88 +- .../src/storage/legacy_adapter.rs | 926 +++++++++++++++++- 3 files changed, 966 insertions(+), 53 deletions(-) diff --git a/quickwit/quickwit-parquet-engine/src/merge/streaming.rs b/quickwit/quickwit-parquet-engine/src/merge/streaming.rs index 9e98736f04b..3adbe007f2a 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/streaming.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/streaming.rs @@ -2789,7 +2789,10 @@ mod tests { .expect("resolve"); // Sanity: the second prefix column must be flagged DESC. assert!( - prefix_cols[1].descending, + prefix_cols[1] + .as_ref() + .expect("env present in this fixture") + .descending, "env must be parsed as DESC from sort schema", ); diff --git a/quickwit/quickwit-parquet-engine/src/merge/streaming/region_grouping.rs b/quickwit/quickwit-parquet-engine/src/merge/streaming/region_grouping.rs index 7660bb3119f..0e6eeef5979 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/streaming/region_grouping.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/streaming/region_grouping.rs @@ -118,14 +118,25 @@ pub(crate) struct PrefixColumn { /// Resolve the first `prefix_len` sort columns to parquet leaf /// indices. Honours the legacy `timestamp` → `timestamp_secs` alias. -/// Errors if the sort schema has fewer columns than `prefix_len` or -/// if any column is missing from the parquet schema. +/// +/// Returns one entry per requested prefix column. `Some(PrefixColumn)` +/// when the column is present in the parquet schema; `None` when the +/// column is named in `sort_fields_str` but absent from the parquet +/// schema. Per SS-3 the missing column is treated as constant null at +/// every row of the file — [`extract_rg_composite_prefix_key`] +/// synthesizes a fixed byte sequence in that slot so ordering is +/// driven entirely by the present columns. +/// +/// Errors only when the sort schema declares fewer columns than +/// requested — that means we don't have a *name* for one of the +/// prefix columns and can't claim alignment on something we can't +/// identify. pub(crate) fn find_prefix_parquet_col_indices( metadata: &ParquetMetaData, sort_fields_str: &str, prefix_len: usize, - input_idx: usize, -) -> Result> { + _input_idx: usize, +) -> Result>> { let sort_field_schema = parse_sort_fields(sort_fields_str)?; if sort_field_schema.column.len() < prefix_len { bail!( @@ -145,6 +156,8 @@ pub(crate) fn find_prefix_parquet_col_indices( } else { sort_col.name.as_str() }; + let descending = sort_col.sort_direction + == quickwit_proto::sortschema::SortColumnDirection::SortDirectionDescending as i32; let mut found = None; for (col_idx, col) in parquet_schema.columns().iter().enumerate() { if col.path().parts()[0] == resolved { @@ -152,27 +165,25 @@ pub(crate) fn find_prefix_parquet_col_indices( break; } } - let parquet_col_idx = found.ok_or_else(|| { - anyhow!( - "input {input_idx} parquet schema is missing prefix sort column '{}' (position \ - {pos})", - sort_col.name, - ) - })?; - let descending = sort_col.sort_direction - == quickwit_proto::sortschema::SortColumnDirection::SortDirectionDescending as i32; + // SS-3: missing column → `None`. The composite-key extractor + // skips this slot entirely (no ordinal byte, no value bytes); + // the trailing prefix-length sentinel in + // `extract_rg_composite_prefix_key` ensures the resulting key + // still sorts cleanly relative to RGs with present values + // (and matches sorted_series's row-level null-skip). + // // Ordinal matches the column's position in `qh.sort_fields`. // For prefix cols (always the first `prefix_len` entries of // the sort schema) the ordinal equals the iteration index // `pos`, which is also the ordinal `sorted_series` would // assign — so the per-RG prefix key composes as a literal // byte prefix of every sorted_series key. - prefix_cols.push(PrefixColumn { + prefix_cols.push(found.map(|parquet_col_idx| PrefixColumn { name: sort_col.name.clone(), parquet_col_idx, descending, ordinal: pos as u8, - }); + })); } Ok(prefix_cols) } @@ -197,10 +208,13 @@ fn parquet_has_column( /// in this RG. /// /// Null handling: -/// - **All-null RG on a prefix column**: the column is skipped entirely (the next column's higher -/// ordinal byte appears in its place), so the RG sorts after any RG carrying a non-null value for -/// this column. This mirrors the row-level convention in `sorted_series` and gives nulls-last -/// ordering for free. +/// - **Column absent from schema (`None` in `prefix_cols`)**: SS-3 case. Every row of the file has +/// a constant null in this slot, so the contribution to the composite is empty (column skipped). +/// The trailing prefix-length sentinel keeps the resulting key well-formed. +/// - **All-null RG on a present prefix column**: column skipped for this RG (the next column's +/// higher ordinal byte — or the trailing sentinel — appears in its place), so the RG sorts after +/// any RG carrying a non-null value for this column. Mirrors the row-level convention in +/// `sorted_series` and gives nulls-last ordering for free. /// - **Mixed null + non-null in one RG**: rows in the RG would encode to two distinct prefix keys /// (the non-null value's key and the column-skipped key), breaking the /// at-most-one-prefix-value-per-RG invariant (PA-1). Reject. @@ -208,12 +222,20 @@ fn parquet_has_column( pub(crate) fn extract_rg_composite_prefix_key( metadata: &ParquetMetaData, rg_idx: usize, - prefix_cols: &[PrefixColumn], + prefix_cols: &[Option], input_idx: usize, ) -> Result> { let rg_meta = metadata.row_group(rg_idx); let mut key = Vec::new(); - for col in prefix_cols { + for col_opt in prefix_cols { + let Some(col) = col_opt else { + // SS-3 implicit null: column absent from schema, so every + // row's value is null. Skip the slot entirely — the + // trailing prefix-length sentinel will keep this from + // colliding with present-value keys, and sorted_series + // applies the same "skip null cols" rule at the row level. + continue; + }; let chunk = rg_meta.column(col.parquet_col_idx); let stats = chunk.statistics().ok_or_else(|| { anyhow!( @@ -575,10 +597,22 @@ pub(crate) fn extract_regions_from_metadata( .collect()) } -/// Post-write check: verify the parquet file at `metadata` has no two -/// row groups sharing the same composite prefix key, for the first -/// `prefix_len` sort columns. Returns `Ok(())` immediately if -/// `prefix_len == 0` (no alignment claim). +/// Post-write check: verify every row group in `metadata` satisfies +/// the prefix-alignment claim declared by `prefix_len`. +/// +/// Enforces both halves of the prefix-alignment contract in one pass: +/// - **PA-1 (intra-RG constancy):** within each RG, each of the first `prefix_len` sort columns has +/// `min == max` (the column is constant across the RG). This is checked transitively by +/// [`extract_rg_composite_prefix_key`] — it returns an error when any prefix column's chunk stats +/// show `min != max`. +/// - **PA-3 (inter-RG uniqueness):** no two RGs share the same composite prefix value. The +/// streaming engine pairs at most one input RG per region per prefix value, so a duplicate would +/// silently drop rows or corrupt the body-col / sort-col mapping. +/// +/// Returns `Ok(())` immediately when `prefix_len == 0` (no claim to +/// verify) or `num_rgs == 0` (no RGs to check). Single-RG files are +/// NOT short-circuited — they still go through PA-1 because an +/// unsorted single-RG file CAN have `min != max` on a prefix column. /// /// This is the writer-side mirror of the read-side check in /// `extract_regions_from_metadata` — both indexing and the compaction @@ -600,8 +634,8 @@ pub(crate) fn assert_unique_rg_prefix_keys( return Ok(()); } let num_rgs = metadata.num_row_groups(); - if num_rgs <= 1 { - // Single-RG (or zero-RG) files vacuously satisfy the invariant. + if num_rgs == 0 { + // Zero-RG files vacuously satisfy both halves of the claim. return Ok(()); } let prefix_cols = diff --git a/quickwit/quickwit-parquet-engine/src/storage/legacy_adapter.rs b/quickwit/quickwit-parquet-engine/src/storage/legacy_adapter.rs index 6c7b85edc56..f33edca0dee 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/legacy_adapter.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/legacy_adapter.rs @@ -28,11 +28,22 @@ //! //! [`LegacyInputAdapter`] handles that case by buffering the whole //! file, decoding it through Arrow, concatenating into a single -//! [`RecordBatch`], and re-encoding it as a **single-row-group** -//! parquet stream that [`StreamingParquetReader`] can serve. The -//! original file is already sorted (legacy files were written sorted); -//! consolidating row groups preserves the order automatically — the -//! adapter does NOT re-sort. +//! [`RecordBatch`], and re-encoding it as a prefix-aligned multi-row- +//! group parquet stream that [`StreamingParquetReader`] can serve. +//! The adapter splits the consolidated batch at first-sort-col +//! transitions (typically `metric_name`) and declares +//! `qh.rg_partition_prefix_len = 1` on the re-encoded file so the +//! merge engine's prefix-aware fast path can consume it. The original +//! file is already sorted (legacy files were written sorted), so +//! consolidating then re-splitting preserves order automatically — +//! the adapter does NOT re-sort. +//! +//! When the original file lacks a `qh.sort_fields` KV or its first +//! sort column can't be resolved in the schema, the adapter falls +//! back to a single-row-group re-encode without claiming any prefix +//! alignment. That route is still valid as input to the merge engine +//! — it just goes through the engine's `prefix_len = 0` sub-region +//! splitting path instead of the fast prefix-aligned path. //! //! Costs: one full-file decode + one full-file re-encode per legacy //! input, per merge. This is acceptable because legacy files age out @@ -51,7 +62,8 @@ use std::ops::Range; use std::path::{Path, PathBuf}; use std::sync::Arc; -use arrow::array::RecordBatch; +use arrow::array::{ArrayRef, NullArray, RecordBatch}; +use arrow::row::{RowConverter, SortField}; use async_trait::async_trait; use bytes::Bytes; use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; @@ -66,7 +78,10 @@ use super::streaming_reader::{ ColumnPageStream, Page, ParquetReadError, RemoteByteSource, StreamingParquetReader, }; use super::streaming_writer::StreamingParquetWriter; -use super::writer::ParquetWriteError; +use super::writer::{ + PARQUET_META_RG_PARTITION_PREFIX_LEN, PARQUET_META_SORT_FIELDS, ParquetWriteError, +}; +use crate::sort_fields::{is_timestamp_column_name, parse_sort_fields}; /// Errors from the legacy input adapter. /// @@ -108,6 +123,24 @@ pub enum LegacyAdapterError { /// against pathological inputs. #[error("legacy input file is too large to buffer: {actual} bytes exceeds limit {limit}")] InputTooLarge { actual: u64, limit: u64 }, + + /// The caller asked for `target_prefix_len > 0` but the file does + /// not advertise enough sort information to honor the request: + /// `qh.sort_fields` is absent, or the sort-fields string declares + /// fewer columns than requested. Either case means the file lacks + /// a name for one of the first `target_prefix_len` sort columns, + /// so the adapter can't claim alignment on a column it can't + /// identify. (Prefix columns that are *named* in `qh.sort_fields` + /// but missing from the arrow schema are NOT an error — per SS-3 + /// the adapter treats them as implicitly null at every row, which + /// trivially satisfies alignment on that column.) The caller + /// should retry with a smaller `target_prefix_len` or pass `0` to + /// fall through to the single-row-group re-encode. + #[error( + "cannot honor target_prefix_len = {target}: {reason} (the legacy file does not advertise \ + enough sort information to safely synthesize prefix-aligned row groups)" + )] + PrefixUnresolvable { target: u32, reason: String }, } /// 4 GiB upper bound on the input file size we will buffer into RAM. @@ -128,8 +161,25 @@ pub struct LegacyInputAdapter { impl LegacyInputAdapter { /// Open the legacy file at `path` through `source`, re-encode it - /// as a single-row-group parquet stream, and prepare to serve its - /// pages. + /// into a prefix-aligned parquet stream advertising + /// `qh.rg_partition_prefix_len = target_prefix_len`, and prepare + /// to serve its pages. + /// + /// The caller picks `target_prefix_len` based on what the rest of + /// the merge plan expects. Typical sources: + /// - Match the consensus `rg_partition_prefix_len` of the non-legacy inputs in the same merge + /// (so all inputs end up at one value). + /// - Pass `0` when there is no non-legacy input, which produces a single-row-group re-encode + /// and no prefix-alignment claim — the merge engine's `prefix_len = 0` sub-region splitting + /// path handles it. + /// + /// When `target_prefix_len > 0`, the adapter slices the + /// consolidated batch at every transition of the first + /// `target_prefix_len` sort columns (composite key, via + /// [`RowConverter`]) and emits one output row group per slice. + /// Returns an error if the file does not have enough resolvable + /// sort columns to honor the request — the caller should either + /// retry with a smaller `target_prefix_len` or fall back to `0`. /// /// Issues exactly one buffered GET against `source` (covering the /// whole file). All subsequent reads are served from the in-memory @@ -137,6 +187,7 @@ impl LegacyInputAdapter { pub async fn try_open( source: Arc, path: PathBuf, + target_prefix_len: u32, ) -> Result { let file_size = source.file_size(&path).await?; if file_size > MAX_LEGACY_INPUT_BYTES { @@ -147,7 +198,7 @@ impl LegacyInputAdapter { } let buffered = source.get_slice(&path, 0..file_size).await?; - let reencoded_bytes = reencode_as_single_row_group(buffered)?; + let reencoded_bytes = reencode_prefix_aligned(buffered, target_prefix_len)?; let reencoded_source: Arc = Arc::new(InMemoryByteSource { bytes: Bytes::from(reencoded_bytes), }); @@ -176,10 +227,31 @@ impl ColumnPageStream for LegacyInputAdapter { } /// Decode `bytes` into a single concatenated [`RecordBatch`], then -/// re-encode it as a single-row-group parquet stream that preserves -/// the original file's `key_value_metadata` and `sorting_columns`. -fn reencode_as_single_row_group(bytes: Bytes) -> Result, LegacyAdapterError> { - let builder = ParquetRecordBatchReaderBuilder::try_new(bytes.clone()) +/// re-encode it according to `target_prefix_len`: +/// - `target_prefix_len == 0`: emit a single row group with no prefix alignment claim. The original +/// `qh.*` KV (which typically omits `qh.rg_partition_prefix_len`) is preserved verbatim. The +/// merge engine's `prefix_len = 0` sub-region splitting path consumes this without further +/// plumbing. +/// - `target_prefix_len > 0`: slice the consolidated batch at every transition of the first +/// `target_prefix_len` sort columns (composite key, via [`RowConverter`]) and emit one row group +/// per distinct composite value. Stamp the output's KV with `qh.rg_partition_prefix_len = +/// target_prefix_len` so the merge engine's prefix-aware fast path takes over. +/// +/// When `target_prefix_len > 0` and the requested alignment cannot be +/// honored — `qh.sort_fields` is absent, the sort-fields string +/// declares fewer columns than requested, or one of the first N +/// columns is missing from the arrow schema — returns +/// [`LegacyAdapterError::PrefixUnresolvable`]. The caller can retry +/// with a smaller `target_prefix_len` or fall back to `0`. +/// +/// The zero-rows-but-`target_prefix_len > 0` case is degenerate but +/// still stamps the KV: an empty file vacuously satisfies any prefix +/// alignment claim. +fn reencode_prefix_aligned( + bytes: Bytes, + target_prefix_len: u32, +) -> Result, LegacyAdapterError> { + let builder = ParquetRecordBatchReaderBuilder::try_new(bytes) .map_err(LegacyAdapterError::ParquetDecode)?; let arrow_schema = builder.schema().clone(); @@ -201,14 +273,212 @@ fn reencode_as_single_row_group(bytes: Bytes) -> Result, LegacyAdapterEr let consolidated_batch = arrow::compute::concat_batches(&arrow_schema, &decoded_batches) .map_err(LegacyAdapterError::ArrowDecode)?; + if target_prefix_len == 0 { + // Single-RG passthrough: re-encode the input as one row group + // with no prefix-alignment claim, so downstream readers take + // the legacy `prefix_len = 0` path. + // + // Preserving the original footer KVs unchanged would leak any + // pre-existing `qh.rg_partition_prefix_len = N` (nonzero) + // claim from the input into the output. The output is *one* + // row group whose rows can carry multiple prefix-column + // values, so advertising N > 0 would cause downstream to take + // the prefix-aware path and then fail PA-1 on the multi-value + // RG. Strip the prefix KV instead; absence of the key is the + // legacy convention for "no alignment claim". + let cleaned_kv = original_kv.map(|mut kvs| { + kvs.retain(|k| k.key != PARQUET_META_RG_PARTITION_PREFIX_LEN); + kvs + }); + let props = build_writer_properties( + &arrow_schema, + original_sorting_cols.unwrap_or_default(), + cleaned_kv, + consolidated_batch.num_rows(), + ); + return write_single_row_group(arrow_schema, props, consolidated_batch); + } + + let prefix_col_indices = + resolve_prefix_sort_cols(original_kv.as_ref(), &arrow_schema, target_prefix_len)?; + + let slices = if consolidated_batch.num_rows() == 0 { + Vec::new() + } else { + compute_prefix_value_slices(&consolidated_batch, &prefix_col_indices)? + }; + let kv_with_prefix = inject_prefix_len_kv(original_kv, target_prefix_len); let props = build_writer_properties( &arrow_schema, original_sorting_cols.unwrap_or_default(), - original_kv, + Some(kv_with_prefix), consolidated_batch.num_rows(), ); + write_multi_row_group(arrow_schema, props, consolidated_batch, &slices) +} + +/// Resolve the first `prefix_len` sort columns from `qh.sort_fields` +/// to arrow-schema indices. Honors the +/// `timestamp` / `timestamp_secs` alias the rest of the engine uses. +/// +/// Returns one entry per requested prefix column: `Some(idx)` if the +/// column is present in the schema, or `None` if the column is +/// declared in `qh.sort_fields` but absent from the arrow schema +/// (treated as implicitly null at every row per SS-3). Returns +/// [`LegacyAdapterError::PrefixUnresolvable`] only when the file +/// doesn't advertise enough sort-column *names* (missing/unparseable +/// `qh.sort_fields`, or declares fewer columns than requested) — +/// those are cases where we don't even know which column the prefix +/// alignment is supposed to be on. +fn resolve_prefix_sort_cols( + kv: Option<&Vec>, + arrow_schema: &arrow::datatypes::Schema, + prefix_len: u32, +) -> Result>, LegacyAdapterError> { + debug_assert!(prefix_len > 0); + let sort_fields_str = kv + .and_then(|kvs| kvs.iter().find(|k| k.key == PARQUET_META_SORT_FIELDS)) + .and_then(|kv| kv.value.as_deref()) + .ok_or_else(|| LegacyAdapterError::PrefixUnresolvable { + target: prefix_len, + reason: format!("{PARQUET_META_SORT_FIELDS} KV is absent"), + })?; + let parsed = + parse_sort_fields(sort_fields_str).map_err(|e| LegacyAdapterError::PrefixUnresolvable { + target: prefix_len, + reason: format!("{PARQUET_META_SORT_FIELDS} is unparseable: {e}"), + })?; + let prefix_len_usize = prefix_len as usize; + if parsed.column.len() < prefix_len_usize { + return Err(LegacyAdapterError::PrefixUnresolvable { + target: prefix_len, + reason: format!( + "{PARQUET_META_SORT_FIELDS} declares only {} sort columns", + parsed.column.len(), + ), + }); + } + let mut indices = Vec::with_capacity(prefix_len_usize); + for sf in parsed.column.iter().take(prefix_len_usize) { + let resolved = if is_timestamp_column_name(&sf.name) + && arrow_schema.index_of("timestamp_secs").is_ok() + { + "timestamp_secs" + } else { + sf.name.as_str() + }; + // Missing column → implicit null per SS-3. A column that is + // null at every row is constant, which trivially satisfies + // alignment on that column. The transition computation + // synthesizes a NullArray in its place. + // + // Adapter-vs-reader consistency note (Codex P2 on PR #6425): + // we record `None` here and the caller stamps + // `rg_partition_prefix_len = target_prefix_len` on the output. + // The matching reader-side handling — `find_prefix_parquet_col_indices` + // returning `Vec>` and synthesizing a + // constant `[0x00, 0x00]` byte sequence for `None` slots — + // lands in the next slice (PR #6426, the hardening PR; F12 + // from the adversarial review). The only caller of this + // adapter is `execute_merge_operation` introduced in + // PR #6423, which sits above #6426 in the stack, so there is + // no production caller that can produce a missing-column + // prefix until the reader fix is in place. + indices.push(arrow_schema.index_of(resolved).ok()); + } + Ok(indices) +} + +/// Walk the composite prefix value row-by-row over the columns at +/// `prefix_col_indices` and produce `(start, len)` slices, one per +/// distinct composite-value run. Uses a single [`RowConverter`] over +/// all prefix columns so dictionary / utf8 / primitive types are +/// handled uniformly and N-column equality is a single byte +/// comparison per row. +/// +/// An entry of `None` in `prefix_col_indices` represents a prefix +/// column that is named in `qh.sort_fields` but absent from the +/// file's arrow schema. Per SS-3 those rows are treated as having +/// null values, so this function materializes a [`NullArray`] of the +/// batch's length in that slot. A column that's null at every row is +/// constant and contributes no transitions to the composite key — +/// equivalent to skipping it, but kept explicit so the resulting +/// alignment claim matches the caller's requested `target_prefix_len`. +fn compute_prefix_value_slices( + batch: &RecordBatch, + prefix_col_indices: &[Option], +) -> Result, LegacyAdapterError> { + let n = batch.num_rows(); + let cols: Vec = prefix_col_indices + .iter() + .map(|idx_opt| match idx_opt { + Some(idx) => Arc::clone(batch.column(*idx)), + None => Arc::new(NullArray::new(n)) as ArrayRef, + }) + .collect(); + let sort_fields: Vec = cols + .iter() + .map(|c| SortField::new(c.data_type().clone())) + .collect(); + let converter = RowConverter::new(sort_fields).map_err(LegacyAdapterError::ArrowDecode)?; + let rows = converter + .convert_columns(&cols) + .map_err(LegacyAdapterError::ArrowDecode)?; + let n_rows = rows.num_rows(); + if n_rows == 0 { + return Ok(Vec::new()); + } + let mut slices = Vec::new(); + let mut start = 0; + for i in 1..n_rows { + if rows.row(i) != rows.row(i - 1) { + slices.push((start, i - start)); + start = i; + } + } + slices.push((start, n_rows - start)); + Ok(slices) +} - write_single_row_group(arrow_schema, props, consolidated_batch) +/// Inject (or replace) the `qh.rg_partition_prefix_len` KV entry on +/// the re-encoded file. Legacy files omit this key entirely; the +/// re-encoded output advertises the synthesized prefix alignment so +/// the merge engine's reader picks the fast path. +fn inject_prefix_len_kv(original: Option>, prefix_len: u32) -> Vec { + let mut kvs = original.unwrap_or_default(); + kvs.retain(|k| k.key != PARQUET_META_RG_PARTITION_PREFIX_LEN); + kvs.push(KeyValue::new( + PARQUET_META_RG_PARTITION_PREFIX_LEN.to_string(), + prefix_len.to_string(), + )); + kvs +} + +/// Write `batch` to a multi-row-group parquet stream: one RG per +/// `(start, len)` slice in `slices`. Slices are emitted in order, so +/// the sort order observed by readers matches the order of the +/// consolidated batch. +fn write_multi_row_group( + arrow_schema: arrow::datatypes::SchemaRef, + props: WriterProperties, + batch: RecordBatch, + slices: &[(usize, usize)], +) -> Result, LegacyAdapterError> { + let mut out: Vec = Vec::new(); + let mut writer = StreamingParquetWriter::try_new(&mut out, arrow_schema, props)?; + for &(start, len) in slices { + if len == 0 { + continue; + } + let mut row_group = writer.start_row_group()?; + for col_idx in 0..batch.num_columns() { + let slice = batch.column(col_idx).slice(start, len); + row_group.write_next_column(&slice)?; + } + row_group.finish()?; + } + writer.close()?; + Ok(out) } /// Read sorting columns from row group 0 of `metadata`, if present. @@ -536,6 +806,86 @@ mod tests { PathBuf::from("legacy_test.parquet") } + /// Build a multi-RG fixture whose rows are sorted by `metric_name` + /// (so consolidating them produces a batch with contiguous + /// metric_name runs, which is what the legacy adapter expects on + /// real legacy files). `metrics` is `(name, rows_per_metric)` in + /// the order they should appear; the writer rolls a new RG every + /// `rows_per_rg` so the multi-RG structure is exercised + /// independently of the metric_name partitioning. + fn write_sorted_multi_rg_legacy_file( + metrics: &[(&str, usize)], + sort_fields_value: &str, + rows_per_rg: usize, + ) -> Bytes { + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(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), + Field::new("timeseries_id", DataType::Int64, false), + Field::new("service", dict_type, true), + ])); + + let total: usize = metrics.iter().map(|(_, n)| *n).sum(); + let metric_names_vec: Vec<&str> = metrics.iter().map(|(name, _)| *name).collect(); + let mut metric_keys: Vec = Vec::with_capacity(total); + let mut tsids: Vec = Vec::with_capacity(total); + let mut timestamps: Vec = Vec::with_capacity(total); + let mut values: Vec = Vec::with_capacity(total); + let mut row_idx: u64 = 0; + for (metric_idx, (_, count)) in metrics.iter().enumerate() { + for _ in 0..*count { + metric_keys.push(metric_idx as i32); + tsids.push(1000 + row_idx as i64); + // -timestamp_secs/V2 in the sort schema means + // timestamps DESC within a metric run. + timestamps.push(1_700_000_000 + (*count as u64) - (row_idx % *count as u64)); + values.push(row_idx as f64); + row_idx += 1; + } + } + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(StringArray::from(metric_names_vec)), + ) + .expect("metric dict"), + ); + let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; total])); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let value: ArrayRef = Arc::new(Float64Array::from(values)); + let timeseries_id: ArrayRef = Arc::new(Int64Array::from(tsids)); + let svc_keys: Vec> = (0..total as i32).map(|i| Some(i % 3)).collect(); + let service: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(svc_keys), + Arc::new(StringArray::from(vec!["api", "db", "cache"])), + ) + .expect("svc dict"), + ); + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + metric_name, + metric_type, + timestamp_secs, + value, + timeseries_id, + service, + ], + ) + .expect("sorted fixture batch"); + + let kvs = vec![KeyValue::new( + PARQUET_META_SORT_FIELDS.to_string(), + sort_fields_value.to_string(), + )]; + let sorting_cols = default_sorting_cols(&schema); + write_multi_rg_file(&[batch], kvs, sorting_cols, rows_per_rg) + } + fn default_sorting_cols(arrow_schema: &ArrowSchema) -> Vec { vec![ SortingColumn { @@ -590,7 +940,7 @@ mod tests { 1, ); let source = CountingInMemorySource::new(bytes); - let adapter = LegacyInputAdapter::try_open(source.clone(), dummy_path()) + let adapter = LegacyInputAdapter::try_open(source.clone(), dummy_path(), 0) .await .expect("adapter open"); @@ -642,7 +992,7 @@ mod tests { assert_eq!(pre_total, 300); let source = CountingInMemorySource::new(bytes); - let mut adapter = LegacyInputAdapter::try_open(source, dummy_path()) + let mut adapter = LegacyInputAdapter::try_open(source, dummy_path(), 0) .await .expect("adapter open"); @@ -689,7 +1039,7 @@ mod tests { // file, and assert the consolidated row count + schema match // the adapter's metadata. let source = CountingInMemorySource::new(bytes); - let mut adapter = LegacyInputAdapter::try_open(source, dummy_path()) + let mut adapter = LegacyInputAdapter::try_open(source, dummy_path(), 0) .await .expect("adapter open"); @@ -730,7 +1080,7 @@ mod tests { 40, ); let source = CountingInMemorySource::new(bytes); - let adapter = LegacyInputAdapter::try_open(source, dummy_path()) + let adapter = LegacyInputAdapter::try_open(source, dummy_path(), 0) .await .expect("adapter open"); @@ -761,7 +1111,7 @@ mod tests { let sorting_cols = default_sorting_cols(&arrow_schema); let bytes = write_multi_rg_file(&[batch_a, batch_b], Vec::new(), sorting_cols.clone(), 30); let source = CountingInMemorySource::new(bytes); - let adapter = LegacyInputAdapter::try_open(source, dummy_path()) + let adapter = LegacyInputAdapter::try_open(source, dummy_path(), 0) .await .expect("adapter open"); @@ -791,7 +1141,7 @@ mod tests { let oracle = read_back_to_single_batch(bytes.clone()); let source = CountingInMemorySource::new(bytes); - let mut adapter = LegacyInputAdapter::try_open(source, dummy_path()) + let mut adapter = LegacyInputAdapter::try_open(source, dummy_path(), 0) .await .expect("adapter open"); // Drain to drive the streaming path. @@ -856,7 +1206,7 @@ mod tests { file_size: 4096, }); - match LegacyInputAdapter::try_open(source, dummy_path()).await { + match LegacyInputAdapter::try_open(source, dummy_path(), 0).await { Err(LegacyAdapterError::Io(err)) => { assert!( err.to_string().contains("simulated"), @@ -898,7 +1248,7 @@ mod tests { ); let oracle = read_back_to_single_batch(bytes.clone()); - let reencoded = reencode_as_single_row_group(bytes).expect("reencode helper"); + let reencoded = reencode_prefix_aligned(bytes, 0).expect("reencode helper"); let reencoded_batch = read_back_to_single_batch(Bytes::from(reencoded)); assert_eq!(reencoded_batch.num_rows(), oracle.num_rows()); @@ -929,7 +1279,7 @@ mod tests { 80, ); let source = CountingInMemorySource::new(bytes); - let adapter = LegacyInputAdapter::try_open(source, dummy_path()) + let adapter = LegacyInputAdapter::try_open(source, dummy_path(), 0) .await .expect("adapter open"); @@ -952,4 +1302,530 @@ mod tests { assert!(stream.next_page().await.expect("idempotent EOF").is_none()); } } + + /// Real legacy files carry `qh.sort_fields` and are written sorted + /// by the schema. The adapter must split the consolidated batch + /// into one RG per first-sort-col value and stamp the re-encoded + /// file with `qh.rg_partition_prefix_len = 1` so the merge engine + /// reads it through the prefix-aware fast path. The streaming + /// engine's duplicate-prefix invariant verifies on read that each + /// RG's metric_name is unique within the file; this test + /// indirectly exercises that contract. + #[tokio::test] + async fn test_legacy_input_with_sort_fields_produces_prefix_aligned_multi_rg() { + let metrics = [ + ("cpu.usage", 40usize), + ("memory.used", 40), + ("net.bytes", 40), + ]; + // Force multi-RG layout in the input (rows_per_rg=30, smaller + // than any metric run) so the fixture proves the adapter + // collapses arbitrary input RG boundaries into prefix-aligned + // output RG boundaries. + let bytes = + write_sorted_multi_rg_legacy_file(&metrics, "metric_name|-timestamp_secs/V2", 30); + let pre_builder = + ParquetRecordBatchReaderBuilder::try_new(bytes.clone()).expect("pre-builder"); + assert!( + pre_builder.metadata().num_row_groups() >= 2, + "fixture must produce multi-RG input; got {}", + pre_builder.metadata().num_row_groups(), + ); + + let source = CountingInMemorySource::new(bytes); + let adapter = LegacyInputAdapter::try_open(source, dummy_path(), 1) + .await + .expect("adapter open"); + + // Three distinct metric_names → three output RGs. + assert_eq!( + adapter.metadata().num_row_groups(), + 3, + "adapter must emit one RG per distinct first-sort-col value", + ); + let rg_rows: Vec = (0..adapter.metadata().num_row_groups()) + .map(|i| adapter.metadata().row_group(i).num_rows()) + .collect(); + assert_eq!(rg_rows, vec![40, 40, 40], "row counts per RG"); + + // KV must advertise prefix_len = 1. + let kv = adapter + .metadata() + .file_metadata() + .key_value_metadata() + .expect("kv metadata"); + let prefix_kv = kv + .iter() + .find(|k| k.key == PARQUET_META_RG_PARTITION_PREFIX_LEN) + .and_then(|k| k.value.clone()); + assert_eq!( + prefix_kv.as_deref(), + Some("1"), + "re-encoded file must declare rg_partition_prefix_len=1", + ); + } + + /// Single-metric legacy file: only one prefix value, so the + /// re-encoded file has exactly one RG (vacuously prefix-aligned). + /// The `qh.rg_partition_prefix_len = 1` KV is still set so the + /// reader's duplicate-prefix check has nothing to validate (one + /// RG can never violate the invariant) and the file looks + /// identical to a metric-aligned new-format file. + #[tokio::test] + async fn test_legacy_input_single_metric_yields_one_rg_with_prefix_kv() { + let metrics = [("cpu.usage", 90usize)]; + let bytes = + write_sorted_multi_rg_legacy_file(&metrics, "metric_name|-timestamp_secs/V2", 30); + let source = CountingInMemorySource::new(bytes); + let adapter = LegacyInputAdapter::try_open(source, dummy_path(), 1) + .await + .expect("adapter open"); + + assert_eq!(adapter.metadata().num_row_groups(), 1); + assert_eq!(adapter.metadata().row_group(0).num_rows(), 90); + + let prefix_kv = adapter + .metadata() + .file_metadata() + .key_value_metadata() + .and_then(|kvs| { + kvs.iter() + .find(|k| k.key == PARQUET_META_RG_PARTITION_PREFIX_LEN) + .and_then(|k| k.value.clone()) + }); + assert_eq!(prefix_kv.as_deref(), Some("1")); + } + + /// `target_prefix_len = 0`: the adapter consolidates into a + /// single row group and does NOT stamp + /// `qh.rg_partition_prefix_len`, regardless of what the original + /// file's KV says. This is the "all-legacy merge with no non- + /// legacy peers" path — the merge engine's `prefix_len = 0` + /// sub-region splitting consumes it directly. + #[tokio::test] + async fn test_target_prefix_len_zero_passes_through_as_single_rg() { + // Even with a parseable sort_fields KV, target = 0 must not + // alter the layout or stamp the prefix KV. + let metrics = [("cpu.usage", 50usize), ("memory.used", 50)]; + let bytes = + write_sorted_multi_rg_legacy_file(&metrics, "metric_name|-timestamp_secs/V2", 30); + + let source = CountingInMemorySource::new(bytes); + let adapter = LegacyInputAdapter::try_open(source, dummy_path(), 0) + .await + .expect("adapter open"); + + assert_eq!( + adapter.metadata().num_row_groups(), + 1, + "target_prefix_len = 0 must consolidate to a single RG", + ); + let prefix_kv = adapter + .metadata() + .file_metadata() + .key_value_metadata() + .and_then(|kvs| { + kvs.iter() + .find(|k| k.key == PARQUET_META_RG_PARTITION_PREFIX_LEN) + .and_then(|k| k.value.clone()) + }); + assert!( + prefix_kv.is_none(), + "target_prefix_len = 0 must not stamp a prefix_len value; got {prefix_kv:?}", + ); + } + + /// Regression for Codex P2 on PR #6425: when the *input* file + /// already carries a stale nonzero `qh.rg_partition_prefix_len` + /// (e.g., it was produced by a prefix-aware writer and is now + /// being re-encoded through the legacy fallback path), passing + /// `target_prefix_len = 0` must STRIP that KV from the re- + /// encoded output. Without the strip, the consolidated single-RG + /// file would still advertise the stale prefix claim and + /// downstream metadata extraction would take the prefix-aware + /// path against a single RG that carries multiple first-prefix + /// values — failing PA-1 min/max on read. + #[tokio::test] + async fn test_target_prefix_len_zero_strips_stale_prefix_kv_from_input() { + // Build a 2-RG input with sort_fields AND a stale prefix KV + // (1) — simulating a prefix-aware input being demoted to the + // legacy single-RG path. + let metrics = [("cpu.usage", 30usize), ("memory.used", 30)]; + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(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), + Field::new("timeseries_id", DataType::Int64, false), + Field::new("service", dict_type, true), + ])); + let total: usize = metrics.iter().map(|(_, n)| *n).sum(); + let mut metric_keys: Vec = Vec::with_capacity(total); + let mut tsids: Vec = Vec::with_capacity(total); + let mut timestamps: Vec = Vec::with_capacity(total); + let mut values: Vec = Vec::with_capacity(total); + let mut row_idx: u64 = 0; + for (metric_idx, (_, count)) in metrics.iter().enumerate() { + for _ in 0..*count { + metric_keys.push(metric_idx as i32); + tsids.push(1000 + row_idx as i64); + timestamps.push(1_700_000_000 + (*count as u64) - (row_idx % *count as u64)); + values.push(row_idx as f64); + row_idx += 1; + } + } + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(StringArray::from( + metrics.iter().map(|(n, _)| *n).collect::>(), + )), + ) + .expect("metric dict"), + ); + let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; total])); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let value: ArrayRef = Arc::new(Float64Array::from(values)); + let timeseries_id: ArrayRef = Arc::new(Int64Array::from(tsids)); + let svc_keys: Vec> = (0..total as i32).map(|i| Some(i % 3)).collect(); + let service: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(svc_keys), + Arc::new(StringArray::from(vec!["api", "db", "cache"])), + ) + .expect("svc dict"), + ); + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + metric_name, + metric_type, + timestamp_secs, + value, + timeseries_id, + service, + ], + ) + .expect("batch"); + + let kvs = vec![ + KeyValue::new( + PARQUET_META_SORT_FIELDS.to_string(), + "metric_name|-timestamp_secs/V2".to_string(), + ), + // The stale claim we want stripped. + KeyValue::new( + PARQUET_META_RG_PARTITION_PREFIX_LEN.to_string(), + "1".to_string(), + ), + ]; + let bytes = write_multi_rg_file(&[batch], kvs, default_sorting_cols(&schema), 30); + + let source = CountingInMemorySource::new(bytes); + let adapter = LegacyInputAdapter::try_open(source, dummy_path(), 0) + .await + .expect("adapter open with target_prefix_len = 0"); + + assert_eq!( + adapter.metadata().num_row_groups(), + 1, + "target_prefix_len = 0 must consolidate to a single RG", + ); + let prefix_kv = adapter + .metadata() + .file_metadata() + .key_value_metadata() + .and_then(|kvs| { + kvs.iter() + .find(|k| k.key == PARQUET_META_RG_PARTITION_PREFIX_LEN) + .and_then(|k| k.value.clone()) + }); + assert!( + prefix_kv.is_none(), + "stale `rg_partition_prefix_len = 1` from the input MUST be stripped when caller asks \ + for target_prefix_len = 0; got {prefix_kv:?}", + ); + } + + /// `target_prefix_len > 0` on a file with no `qh.sort_fields` KV + /// must surface a `PrefixUnresolvable` error rather than silently + /// fall through. The caller decides whether to retry at a smaller + /// `target_prefix_len` or with `0`. + #[tokio::test] + async fn test_target_prefix_len_one_without_sort_fields_returns_unresolvable() { + // No sort_fields KV → cannot resolve any prefix column. + let batch_a = make_metrics_batch(40); + let arrow_schema = batch_a.schema(); + let bytes = write_multi_rg_file( + &[batch_a], + Vec::new(), + default_sorting_cols(&arrow_schema), + 40, + ); + + let source = CountingInMemorySource::new(bytes); + let result = LegacyInputAdapter::try_open(source, dummy_path(), 1).await; + let Err(err) = result else { + panic!("missing sort_fields must surface as PrefixUnresolvable, got Ok(...)"); + }; + match err { + LegacyAdapterError::PrefixUnresolvable { target, reason } => { + assert_eq!(target, 1); + assert!( + reason.contains("sort_fields"), + "reason should mention sort_fields KV; got: {reason}", + ); + } + other => panic!("expected PrefixUnresolvable, got: {other}"), + } + } + + /// `target_prefix_len > declared sort cols` must also bail with + /// `PrefixUnresolvable`. Confirms the caller-driven negotiation + /// contract: the adapter never silently advertises a smaller + /// alignment than asked for. + #[tokio::test] + async fn test_target_prefix_len_exceeds_declared_sort_cols_returns_unresolvable() { + // Two-col sort schema → ask for prefix_len = 3 → bail. + let metrics = [("cpu.usage", 30usize), ("memory.used", 30)]; + let bytes = + write_sorted_multi_rg_legacy_file(&metrics, "metric_name|-timestamp_secs/V2", 30); + + let source = CountingInMemorySource::new(bytes); + let result = LegacyInputAdapter::try_open(source, dummy_path(), 3).await; + let Err(err) = result else { + panic!("prefix_len exceeding declared sort cols must surface as Unresolvable"); + }; + match err { + LegacyAdapterError::PrefixUnresolvable { target, reason } => { + assert_eq!(target, 3); + assert!( + reason.contains("declares only"), + "reason should mention sort col count; got: {reason}", + ); + } + other => panic!("expected PrefixUnresolvable, got: {other}"), + } + } + + /// Composite prefix (`target_prefix_len = 2`): each output RG + /// carries a unique `(metric_name, service)` tuple. Exercises the + /// N > 1 path of `compute_prefix_value_slices` and confirms the + /// stamped KV reflects the caller's request. + #[tokio::test] + async fn test_target_prefix_len_two_splits_by_metric_and_service() { + // 4 (metric, service) groups × 20 rows; sorted ascending by + // (metric_name, service). Multi-RG input layout (rows_per_rg=25) + // forces consolidation before re-splitting. + let groups = [ + ("cpu.usage", "api", 20usize), + ("cpu.usage", "db", 20), + ("memory.used", "api", 20), + ("memory.used", "cache", 20), + ]; + let bytes = write_sorted_composite_legacy_file( + &groups, + "metric_name|service|-timestamp_secs/V2", + 25, + ); + + let source = CountingInMemorySource::new(bytes); + let adapter = LegacyInputAdapter::try_open(source, dummy_path(), 2) + .await + .expect("adapter open with prefix_len=2"); + + // 4 distinct (metric_name, service) tuples → 4 output RGs. + assert_eq!( + adapter.metadata().num_row_groups(), + 4, + "composite prefix must split at (metric, service) transitions", + ); + let rg_rows: Vec = (0..adapter.metadata().num_row_groups()) + .map(|i| adapter.metadata().row_group(i).num_rows()) + .collect(); + assert_eq!(rg_rows, vec![20, 20, 20, 20]); + + let prefix_kv = adapter + .metadata() + .file_metadata() + .key_value_metadata() + .and_then(|kvs| { + kvs.iter() + .find(|k| k.key == PARQUET_META_RG_PARTITION_PREFIX_LEN) + .and_then(|k| k.value.clone()) + }); + assert_eq!( + prefix_kv.as_deref(), + Some("2"), + "stamped prefix_len must match caller's request", + ); + } + + /// SS-3: a sort column named in `qh.sort_fields` but missing from + /// the arrow schema is treated as implicitly null at every row. + /// Null at every row is constant, so the column trivially + /// satisfies any prefix-alignment claim involving it — the + /// adapter must NOT bail with `PrefixUnresolvable` in this case. + /// Transitions are driven purely by the columns that ARE present. + /// + /// Fixture: sort_fields declares `metric_name|env|-timestamp_secs` + /// but the schema doesn't contain `env`. With prefix_len = 2 the + /// adapter must succeed, split only at `metric_name` transitions + /// (the null `env` contributes no transitions), and stamp + /// `prefix_len = 2` on the output. + #[tokio::test] + async fn test_missing_prefix_col_treated_as_null_satisfies_alignment() { + let metrics = [("cpu.usage", 30usize), ("memory.used", 30)]; + // Sort schema declares 3 cols; the fixture schema has + // metric_name and timestamp_secs but NO `env` column. Per + // SS-3 the merge engine treats `env` as null at every row. + let bytes = + write_sorted_multi_rg_legacy_file(&metrics, "metric_name|env|-timestamp_secs/V2", 30); + + let source = CountingInMemorySource::new(bytes); + let adapter = LegacyInputAdapter::try_open(source, dummy_path(), 2) + .await + .expect("missing-col-as-null must satisfy alignment without erroring"); + + // Two metrics × constant null `env` → two output RGs (one per metric). + assert_eq!( + adapter.metadata().num_row_groups(), + 2, + "missing prefix col (treated as null) contributes no transitions; only metric_name \ + transitions split RGs", + ); + let rg_rows: Vec = (0..adapter.metadata().num_row_groups()) + .map(|i| adapter.metadata().row_group(i).num_rows()) + .collect(); + assert_eq!(rg_rows, vec![30, 30]); + + let prefix_kv = adapter + .metadata() + .file_metadata() + .key_value_metadata() + .and_then(|kvs| { + kvs.iter() + .find(|k| k.key == PARQUET_META_RG_PARTITION_PREFIX_LEN) + .and_then(|k| k.value.clone()) + }); + assert_eq!( + prefix_kv.as_deref(), + Some("2"), + "stamped prefix_len must match caller's request even when one col is implicitly null", + ); + + // SS-3 consumer-side verification: the file the adapter just + // produced must be consumable by the merge engine's + // `extract_rg_composite_prefix_key` reader. With `env` absent + // from the parquet schema, `find_prefix_parquet_col_indices` + // returns `None` in that slot and the composite-key extractor + // skips it. PA-1 + PA-3 still hold: each RG's metric_name + // min == max, and skipping the constant-null `env` slot makes + // the RG composite keys differ only by metric_name. + crate::merge::streaming::region_grouping::assert_unique_rg_prefix_keys( + adapter.metadata(), + "metric_name|env|-timestamp_secs/V2", + 2, + "test_missing_prefix_col_treated_as_null_satisfies_alignment adapter output", + ) + .expect("SS-3 null col must satisfy PA-1 + PA-3 (null is constant across all RGs)"); + } + + /// Composite-prefix fixture: rows grouped by `(metric, service)` + /// in the order supplied. Used by the prefix_len=2 test to verify + /// transitions on the second prefix column trigger RG splits. + fn write_sorted_composite_legacy_file( + groups: &[(&str, &str, usize)], + sort_fields_value: &str, + rows_per_rg: usize, + ) -> Bytes { + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(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), + Field::new("timeseries_id", DataType::Int64, false), + Field::new("service", dict_type, true), + ])); + + // Build per-group dictionary index tables for metric_name and + // service. Map each distinct value to its key. + let mut metric_names_vec: Vec<&str> = Vec::new(); + let mut service_values_vec: Vec<&str> = Vec::new(); + for (metric, service, _) in groups { + if !metric_names_vec.contains(metric) { + metric_names_vec.push(metric); + } + if !service_values_vec.contains(service) { + service_values_vec.push(service); + } + } + + let total: usize = groups.iter().map(|(_, _, n)| *n).sum(); + let mut metric_keys: Vec = Vec::with_capacity(total); + let mut svc_keys: Vec> = Vec::with_capacity(total); + let mut tsids: Vec = Vec::with_capacity(total); + let mut timestamps: Vec = Vec::with_capacity(total); + let mut values: Vec = Vec::with_capacity(total); + let mut row_idx: u64 = 0; + for (metric, service, count) in groups { + let metric_key = metric_names_vec + .iter() + .position(|m| m == metric) + .expect("metric known") as i32; + let svc_key = service_values_vec + .iter() + .position(|s| s == service) + .expect("service known") as i32; + for _ in 0..*count { + metric_keys.push(metric_key); + svc_keys.push(Some(svc_key)); + tsids.push(1000 + row_idx as i64); + timestamps.push(1_700_000_000 + (*count as u64) - (row_idx % *count as u64)); + values.push(row_idx as f64); + row_idx += 1; + } + } + + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(StringArray::from(metric_names_vec)), + ) + .expect("metric dict"), + ); + let service: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(svc_keys), + Arc::new(StringArray::from(service_values_vec)), + ) + .expect("svc dict"), + ); + let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; total])); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let value: ArrayRef = Arc::new(Float64Array::from(values)); + let timeseries_id: ArrayRef = Arc::new(Int64Array::from(tsids)); + + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + metric_name, + metric_type, + timestamp_secs, + value, + timeseries_id, + service, + ], + ) + .expect("composite fixture batch"); + + let kvs = vec![KeyValue::new( + PARQUET_META_SORT_FIELDS.to_string(), + sort_fields_value.to_string(), + )]; + let sorting_cols = default_sorting_cols(&schema); + write_multi_rg_file(&[batch], kvs, sorting_cols, rows_per_rg) + } } From 6fdced26bc0ce46010d2c9c3abe04602ceef2128 Mon Sep 17 00:00:00 2001 From: George Talbot Date: Mon, 18 May 2026 14:14:58 -0400 Subject: [PATCH 07/10] fix(merge): adapter rejects unsorted input; consumer honors SS-3; stronger test verifiers (#6426) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Three adversarial-review findings on the prefix/RG machinery, bundled because they touch the same producer/consumer contract: **F8: Legacy adapter rejects SS-1-violating input upfront.** The adapter walked rows in physical order and emitted one RG per prefix-value run. An unsorted legacy input (rows `[A,A,B,B,A,A]`) produced a 3-RG file where two RGs shared prefix `A`, violating PA-3. The streaming merge engine would later reject it mid-merge — but only after a quietly-bad file had been built. Now `compute_prefix_value_slices` tracks each slice's composite prefix-value bytes and bails with `LegacyAdapterError::InputNotSorted` on duplicates, surfacing the SS-1 violation before any file lands on disk. **F12: Consumer-side SS-3 (cross-layer divergence, discovered while wiring F2's chunk-level verifier into the SS-3 test).** The adapter implements SS-3 correctly (missing-from-schema → synthesized NullArray during slice computation, file stamps `prefix_len = N`). The streaming engine's reader did not: `find_prefix_parquet_col_indices` hard-required every named prefix column to be physically present, so a file the adapter produced from an SS-3 input was unreadable by the merge engine. Now `find_prefix_parquet_col_indices` returns `Vec>` and `extract_rg_composite_prefix_key` emits a constant null marker (`encode_byte_array_prefix(&[])`) for None slots. The column contributes no cross-RG ordering signal (constant everywhere) so region boundaries are driven entirely by the present columns. Both halves of SS-3 now agree end-to-end. Known limitation: cross-file SS-3 — where some inputs have a sort column and others don't — uses [0x00, 0x00] for the null contribution, which sorts BEFORE non-null per the encoded-empty-string convention. That weakly violates SS-2 (nulls sort last). Single-file SS-3 is correct because every RG in such a file contributes the same constant. If cross-file SS-3 becomes a production scenario, the encoding needs a leading-0xff sentinel instead. Not exercised today. **F2/F9/F11: Wire `assert_unique_rg_prefix_keys` into prefix-claiming tests.** Tests asserting `num_row_groups == N` + KV stamped to N would have passed even with an off-by-one in slice-boundary detection or column-content scrambling. The verifier reads chunk-level statistics directly: PA-1 (intra-RG `min == max`) + PA-3 (inter-RG uniqueness) on the composite key. Wired into six tests: - streaming engine: `test_streaming_merge_with_prefix_len_two`, `test_multi_rg_metric_aligned_input_produces_multi_rg_output`, `test_streaming_merge_with_desc_prefix_col` - legacy adapter: `test_target_prefix_len_two_splits_by_metric_and_service`, `test_legacy_input_with_sort_fields_produces_prefix_aligned_multi_rg`, `test_missing_prefix_col_treated_as_null_satisfies_alignment` (now passes thanks to F12). Also: `assert_unique_rg_prefix_keys` no longer short-circuits on single-RG files — they still go through PA-1 because an unsorted single-RG file CAN have `min != max` on a prefix column. Co-authored-by: Claude Opus 4.7 (1M context) --- .../src/merge/streaming.rs | 35 +++++ .../src/storage/legacy_adapter.rs | 135 +++++++++++++++++- 2 files changed, 167 insertions(+), 3 deletions(-) diff --git a/quickwit/quickwit-parquet-engine/src/merge/streaming.rs b/quickwit/quickwit-parquet-engine/src/merge/streaming.rs index 3adbe007f2a..431eb13a872 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/streaming.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/streaming.rs @@ -1893,6 +1893,17 @@ mod tests { outputs[0].num_row_groups, 2, "MergeOutputFile.num_row_groups should match physical row group count", ); + + // F2 chunk-level verification: confirm each output RG actually + // carries a single distinct metric_name (PA-1 + PA-3 read + // straight off the column-chunk statistics). + assert_unique_rg_prefix_keys( + reader.metadata(), + "metric_name|-timestamp_secs/V2", + 1, + "test_multi_rg_metric_aligned_input_produces_multi_rg_output output", + ) + .expect("streaming engine output must satisfy PA-1 + PA-3 on metric_name"); } /// Regression for Codex P2 on PR-6410: a streaming merge output @@ -2081,6 +2092,20 @@ mod tests { "three distinct (metric_name, service) pairs must produce three output RGs", ); assert_eq!(outputs[0].num_row_groups, 3); + + // F2 chunk-level verification: counting RGs and stamping a KV + // is not enough — the OUTPUT's row groups must actually be + // aligned on the composite (metric_name, service) prefix. + // `assert_unique_rg_prefix_keys` enforces PA-1 (intra-RG + // constancy) + PA-3 (inter-RG uniqueness) by reading the + // chunk-level statistics. + assert_unique_rg_prefix_keys( + reader.metadata(), + "metric_name|service|-timestamp_secs/V2", + 2, + "test_streaming_merge_with_prefix_len_two output", + ) + .expect("streaming engine output must satisfy PA-1 + PA-3 on the prefix columns"); } /// Regression for Codex finding #1 on PR-6410: when one input @@ -2768,6 +2793,16 @@ mod tests { (third_block - 1.0).abs() < 1e-9, "third output RG should be 'dev' (marker 1.0), got {third_block}", ); + + // F2 chunk-level verification: each output RG must be aligned + // on (metric_name, -env). PA-1 + PA-3 read from chunk stats. + assert_unique_rg_prefix_keys( + reader.metadata(), + "metric_name|-env|-timestamp_secs/V2", + 2, + "test_streaming_merge_with_desc_prefix_col output", + ) + .expect("DESC prefix output must satisfy PA-1 + PA-3"); } /// Regression for the composite-key encoding when ASC and DESC diff --git a/quickwit/quickwit-parquet-engine/src/storage/legacy_adapter.rs b/quickwit/quickwit-parquet-engine/src/storage/legacy_adapter.rs index f33edca0dee..91383f8997f 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/legacy_adapter.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/legacy_adapter.rs @@ -57,6 +57,7 @@ // deprecated items at module scope keeps that lookup direct. #![allow(deprecated)] +use std::collections::HashMap; use std::io; use std::ops::Range; use std::path::{Path, PathBuf}; @@ -141,6 +142,26 @@ pub enum LegacyAdapterError { enough sort information to safely synthesize prefix-aligned row groups)" )] PrefixUnresolvable { target: u32, reason: String }, + + /// The legacy file's rows are not sorted by its declared sort schema + /// (SS-1 violation): two row regions in the file carry the same + /// composite prefix value with other prefix values in between. The + /// adapter walks rows in physical order and emits one RG per + /// prefix-value run, so an unsorted input produces multiple RGs + /// sharing a prefix key — which violates PA-3 (per-input uniqueness). + /// Bail upfront instead of producing a file the downstream merge + /// engine will reject mid-merge. + #[error( + "legacy input is not sorted by its declared sort schema: rows at offset {first_offset} \ + and offset {second_offset} share composite prefix value (target_prefix_len = {target}). \ + The adapter relies on the file being sorted per SS-1; an unsorted file would synthesize \ + multiple row groups with the same prefix key (PA-3 violation)." + )] + InputNotSorted { + target: u32, + first_offset: usize, + second_offset: usize, + }, } /// 4 GiB upper bound on the input file size we will buffer into RAM. @@ -305,7 +326,7 @@ fn reencode_prefix_aligned( let slices = if consolidated_batch.num_rows() == 0 { Vec::new() } else { - compute_prefix_value_slices(&consolidated_batch, &prefix_col_indices)? + compute_prefix_value_slices(&consolidated_batch, &prefix_col_indices, target_prefix_len)? }; let kv_with_prefix = inject_prefix_len_kv(original_kv, target_prefix_len); let props = build_writer_properties( @@ -404,9 +425,19 @@ fn resolve_prefix_sort_cols( /// constant and contributes no transitions to the composite key — /// equivalent to skipping it, but kept explicit so the resulting /// alignment claim matches the caller's requested `target_prefix_len`. +/// +/// Detects SS-1 violations (unsorted input) up-front: each emitted +/// slice's composite prefix-value bytes must be unique. If two +/// non-adjacent slices carry the same prefix value (e.g., rows +/// `[A,A,B,B,A,A]`), the input is not sorted by its declared sort +/// schema, so we'd synthesize a file with two RGs sharing the prefix +/// — a PA-3 violation the downstream merge engine would reject +/// mid-merge. Bailing here with `InputNotSorted` keeps that bad file +/// from ever landing on disk. fn compute_prefix_value_slices( batch: &RecordBatch, prefix_col_indices: &[Option], + target_prefix_len: u32, ) -> Result, LegacyAdapterError> { let n = batch.num_rows(); let cols: Vec = prefix_col_indices @@ -428,15 +459,35 @@ fn compute_prefix_value_slices( if n_rows == 0 { return Ok(Vec::new()); } + // Track each emitted slice's starting prefix-value bytes; any + // repeat signals SS-1 violation on the input. + let mut seen: HashMap, usize> = HashMap::new(); let mut slices = Vec::new(); let mut start = 0; + let record_slice = |slices: &mut Vec<(usize, usize)>, + seen: &mut HashMap, usize>, + slice_start: usize, + slice_len: usize| + -> Result<(), LegacyAdapterError> { + let key = rows.row(slice_start).as_ref().to_vec(); + if let Some(&first_offset) = seen.get(&key) { + return Err(LegacyAdapterError::InputNotSorted { + target: target_prefix_len, + first_offset, + second_offset: slice_start, + }); + } + seen.insert(key, slice_start); + slices.push((slice_start, slice_len)); + Ok(()) + }; for i in 1..n_rows { if rows.row(i) != rows.row(i - 1) { - slices.push((start, i - start)); + record_slice(&mut slices, &mut seen, start, i - start)?; start = i; } } - slices.push((start, n_rows - start)); + record_slice(&mut slices, &mut seen, start, n_rows - start)?; Ok(slices) } @@ -1363,6 +1414,19 @@ mod tests { Some("1"), "re-encoded file must declare rg_partition_prefix_len=1", ); + + // F9 chunk-level verification: the count + KV checks above + // would still pass if `compute_prefix_value_slices` had an + // off-by-one in its boundary detection. PA-1 + PA-3 on chunk + // statistics nail down that each RG's metric_name column is + // actually constant and no two RGs share a value. + crate::merge::streaming::region_grouping::assert_unique_rg_prefix_keys( + adapter.metadata(), + "metric_name|-timestamp_secs/V2", + 1, + "test_legacy_input_with_sort_fields_produces_prefix_aligned_multi_rg adapter output", + ) + .expect("adapter output must satisfy PA-1 + PA-3 on metric_name"); } /// Single-metric legacy file: only one prefix value, so the @@ -1660,6 +1724,19 @@ mod tests { Some("2"), "stamped prefix_len must match caller's request", ); + + // F9 chunk-level verification: a `compute_prefix_value_slices` + // bug splitting on only the first prefix col (or off by one) + // would still yield 4 RGs of [20,20,20,20] but with the wrong + // CONTENTS. PA-1 + PA-3 on the composite (metric, service) + // composite key verifies content alignment directly. + crate::merge::streaming::region_grouping::assert_unique_rg_prefix_keys( + adapter.metadata(), + "metric_name|service|-timestamp_secs/V2", + 2, + "test_target_prefix_len_two_splits_by_metric_and_service adapter output", + ) + .expect("composite prefix output must satisfy PA-1 + PA-3"); } /// SS-3: a sort column named in `qh.sort_fields` but missing from @@ -1732,6 +1809,58 @@ mod tests { .expect("SS-3 null col must satisfy PA-1 + PA-3 (null is constant across all RGs)"); } + /// F8 regression: an unsorted legacy input (rows + /// `[A,A,B,B,A,A]` on `metric_name`) violates SS-1. Walking + /// row-by-row to find prefix transitions would emit three slices — + /// `A`, `B`, `A` — and synthesize a file with two RGs sharing the + /// prefix value `A`, violating PA-3. The downstream streaming + /// merge engine would catch this later, but only once the bad + /// file had been built and possibly archived. The adapter must + /// bail upfront with `InputNotSorted` so no PA-3-violating file + /// ever lands on disk. + #[tokio::test] + async fn test_unsorted_legacy_input_rejected_by_adapter() { + // metric_name in row order: cpu.usage, memory.used, cpu.usage. + // That's an SS-1 violation under sort schema `metric_name ASC`. + let bad_metrics = [ + ("cpu.usage", 20usize), + ("memory.used", 20), + ("cpu.usage", 20), + ]; + let bytes = + write_sorted_multi_rg_legacy_file(&bad_metrics, "metric_name|-timestamp_secs/V2", 20); + + let source = CountingInMemorySource::new(bytes); + let result = LegacyInputAdapter::try_open(source, dummy_path(), 1).await; + let Err(err) = result else { + panic!( + "unsorted legacy input must surface as InputNotSorted, got Ok(...) — the adapter \ + would have written a PA-3-violating file" + ); + }; + match err { + LegacyAdapterError::InputNotSorted { + target, + first_offset, + second_offset, + } => { + assert_eq!(target, 1); + // First `cpu.usage` run is at offset 0; second is at + // offset 40 (after the 20-row `cpu.usage` then 20-row + // `memory.used` runs). + assert_eq!( + first_offset, 0, + "first duplicate prefix offset should point at the first cpu.usage run", + ); + assert_eq!( + second_offset, 40, + "second duplicate prefix offset should point at the second cpu.usage run", + ); + } + other => panic!("expected InputNotSorted, got: {other}"), + } + } + /// Composite-prefix fixture: rows grouped by `(metric, service)` /// in the order supplied. Used by the prefix_len=2 test to verify /// transitions on the second prefix column trigger RG splits. From ead4848593f0c203ca1babc68b8c1ba335b45057 Mon Sep 17 00:00:00 2001 From: George Talbot Date: Mon, 18 May 2026 14:15:31 -0400 Subject: [PATCH 08/10] feat(merge): legacy promotion path + body-col schema evolution (#6423) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * fix(merge): adapter rejects unsorted input; consumer honors SS-3; stronger test verifiers Three adversarial-review findings on the prefix/RG machinery, bundled because they touch the same producer/consumer contract: **F8: Legacy adapter rejects SS-1-violating input upfront.** The adapter walked rows in physical order and emitted one RG per prefix-value run. An unsorted legacy input (rows `[A,A,B,B,A,A]`) produced a 3-RG file where two RGs shared prefix `A`, violating PA-3. The streaming merge engine would later reject it mid-merge — but only after a quietly-bad file had been built. Now `compute_prefix_value_slices` tracks each slice's composite prefix-value bytes and bails with `LegacyAdapterError::InputNotSorted` on duplicates, surfacing the SS-1 violation before any file lands on disk. **F12: Consumer-side SS-3 (cross-layer divergence, discovered while wiring F2's chunk-level verifier into the SS-3 test).** The adapter implements SS-3 correctly (missing-from-schema → synthesized NullArray during slice computation, file stamps `prefix_len = N`). The streaming engine's reader did not: `find_prefix_parquet_col_indices` hard-required every named prefix column to be physically present, so a file the adapter produced from an SS-3 input was unreadable by the merge engine. Now `find_prefix_parquet_col_indices` returns `Vec>` and `extract_rg_composite_prefix_key` emits a constant null marker (`encode_byte_array_prefix(&[])`) for None slots. The column contributes no cross-RG ordering signal (constant everywhere) so region boundaries are driven entirely by the present columns. Both halves of SS-3 now agree end-to-end. Known limitation: cross-file SS-3 — where some inputs have a sort column and others don't — uses [0x00, 0x00] for the null contribution, which sorts BEFORE non-null per the encoded-empty-string convention. That weakly violates SS-2 (nulls sort last). Single-file SS-3 is correct because every RG in such a file contributes the same constant. If cross-file SS-3 becomes a production scenario, the encoding needs a leading-0xff sentinel instead. Not exercised today. **F2/F9/F11: Wire `assert_unique_rg_prefix_keys` into prefix-claiming tests.** Tests asserting `num_row_groups == N` + KV stamped to N would have passed even with an off-by-one in slice-boundary detection or column-content scrambling. The verifier reads chunk-level statistics directly: PA-1 (intra-RG `min == max`) + PA-3 (inter-RG uniqueness) on the composite key. Wired into six tests: - streaming engine: `test_streaming_merge_with_prefix_len_two`, `test_multi_rg_metric_aligned_input_produces_multi_rg_output`, `test_streaming_merge_with_desc_prefix_col` - legacy adapter: `test_target_prefix_len_two_splits_by_metric_and_service`, `test_legacy_input_with_sort_fields_produces_prefix_aligned_multi_rg`, `test_missing_prefix_col_treated_as_null_satisfies_alignment` (now passes thanks to F12). Also: `assert_unique_rg_prefix_keys` no longer short-circuits on single-RG files — they still go through PA-1 because an unsorted single-RG file CAN have `min != max` on a prefix column. Co-Authored-By: Claude Opus 4.7 (1M context) * feat(merge): legacy-prefix promotion path + schema-evolution body cols Two adversarial-review follow-ups grouped because they share the streaming engine's input-routing and union-schema seams. ## (b) Legacy-prefix promotion A new operation type pairs a prefix_len=0 split with prefix_len>0 peers in one merge, so legacy splits can be folded into prefix- aligned buckets instead of aging out via retention. Adds: - `ParquetMergeOperation::promote_legacy(splits, target_prefix_len)`: relaxes MP-3 to allow mixed `rg_partition_prefix_len` as long as every input is `<= target`. Sort_fields + window equality unchanged. - `ParquetMergeOperation::target_prefix_len_override: Option` field records the promotion target; `None` is the default regular-merge form. - `merge_parquet_split_metadata(..., mixed_prefix_ok)`: skips the input-side prefix-len equality check in promotion mode. The output prefix_len still comes from the writer's KV stamp via `MergeOutputFile.output_rg_partition_prefix_len` (CS-1 holds by construction post-F1). - `merge::execute_merge_operation(op, sources, ...)`: new thin executor that opens each input as either `LegacyInputAdapter` (when `split.rg_partition_prefix_len < target`) or `StreamingParquetReader` (otherwise), then feeds them to the streaming engine. Becomes the seam PR-7 will wire from above. Tests: - `test_promote_legacy_pairs_legacy_with_aligned_peer`, `test_promote_legacy_rejects_higher_prefix_input`, `test_promote_legacy_still_enforces_sort_fields`, `test_promote_legacy_all_at_target_is_valid`. - `test_mixed_prefix_ok_skips_input_equality_check`. - `test_promote_legacy_executor_end_to_end`: legacy single-RG + aligned multi-RG → 3-RG output passing `assert_unique_rg_prefix_keys` with `prefix_len = 1`, plus metastore CS-1. - `test_executor_mismatched_sources_count_bails`. ## F6 + F13: Schema evolution for body columns The merger now supports MC-4 across heterogeneous body-col schemas: - F6: `normalize_type` collapses `Binary`/`LargeBinary` (and dict variants) to `Binary`, analogous to the existing string-flavour collapse. Two inputs whose body col differs only by byte-array flavour merge cleanly; before this they hit a "type conflict" at alignment time. - F13: `streaming_writer.rs::write_list_via_serialized_column_writer` (renamed from `..._non_nullable_...`) now handles nullable outer `List` / `LargeList`. MC-4 forces the union to be nullable when a List col is present in only some inputs; before this the writer rejected the merged output. Uses Dremel max_def_level = 2 (0 = outer null, 1 = empty list, 2 = element present) for nullable outer; non-nullable path unchanged. Test: `test_mc2_mixed_schemas_round_trip` builds two inputs A and B with the same sort schema but different body cols (Utf8 vs Dict, LargeBinary vs Binary, List in A only, Int32 A-only, Int64 B-only, common Float64). The merge produces the union schema; per-row rendering via `render_cell` matches across flavour boundaries; List cells from B render as nulls. Co-Authored-By: Claude Opus 4.7 (1M context) * style(indexing): re-fmt parquet_merge_executor to latest nightly rustfmt Same nightly-rustfmt drift as the storekey commit on #6424 (local nightly 2026-05-11 vs CI's 2026-05-17): the `mixed_prefix_ok` binding and the `merge_parquet_split_metadata` call now fit single-line under the newer width heuristics. No behaviour change. Co-Authored-By: Claude Opus 4.7 (1M context) * fix(merge-executor): route promotion merges through execute_merge_operation Codex P1 on PR #6423: the executor unconditionally called the in-memory `merge_sorted_parquet_files` path, which routes through `extract_and_validate_input_metadata` and bails on mixed `qh.rg_partition_prefix_len` before any output is produced. So a real promotion merge — `prefix_len = 0` plus `prefix_len = 1` with `target_prefix_len_override = Some(1)` — failed before reaching the downstream `mixed_prefix_ok` plumbing in `merge_parquet_split_metadata`. The escape hatch existed but was unreachable for actual promotion inputs. Fix: branch in the executor's handle on `target_prefix_len_override.is_some()`. Promotion merges go through the engine's streaming entry point `quickwit_parquet_engine::merge::execute_merge_operation`, which opens each below-target input via `LegacyInputAdapter` and each at-target input directly. The streaming merge then sees a homogeneous stream advertising `prefix_len = target` on every input. Regular (non-promotion) merges keep the in-memory path. `execute_merge_operation` expects `Vec>` parallel to `op.splits` — the engine deliberately doesn't depend on `quickwit-storage` (would invert layering and pull cloud SDKs into a pure parquet library). So this commit adds `LocalFileByteSource`: a tiny `RemoteByteSource` impl backed by `tokio::fs::File`, one instance per downloaded split, each bound to its scratch-directory path. The `path: &Path` argument on the trait surface is ignored — the downloader has already resolved each split to a concrete local file before the executor runs. Coverage: - Library-level: `quickwit-parquet-engine::merge::streaming::tests::test_promote_legacy_executor_end_to_end` already exercises `execute_merge_operation` with a `prefix_len = 0` + `prefix_len = 1` pair, verifying the output advertises `prefix_len = 1` and passes PA-1 + PA-3 on the composite key. That's now the same code path the in-tree executor takes. - Module doc on the executor rewritten to spell out which path runs when. Co-Authored-By: Claude Opus 4.7 (1M context) * docs(adr): track legacy promotion planner gap as GAP-011 The streaming Parquet merge stack landing in #6424–#6428 ships the full legacy-promotion *mechanism* (engine + adapter + executor wiring) but not the planner-level *trigger*. In production today, `MergePolicyState::record_split` buckets by `CompactionScope::from_split` which includes `rg_partition_prefix_len`, so legacy (prefix=0) and aligned (prefix>0) splits are separated before `ParquetMergePolicy::operations` runs. The policy only emits `ParquetMergeOperation::new`; a repo-wide search finds `promote_legacy` only in tests. Legacy splits therefore never migrate without an explicit trigger. Tracking this as GAP-011 so we pick it up at the right time. The gap doc walks three resolution options (merge buckets in the scope key, dedicated promotion pass, or hybrid prefer-multi-input-promotion) and the cost trade-offs between them, so the eventual implementation PR has a starting point. Raised by Codex review comment id 4311184497 on PR #6423. Co-Authored-By: Claude Opus 4.7 (1M context) * docs(adr): track download-vs-streaming merge executor gap as GAP-012 The Parquet streaming merge engine is built around `RemoteByteSource` and was designed to pull pages directly from object storage — two GETs per input, overlap fetch with merge, no scratch disk. The production actor pipeline doesn't take that path: a downloader actor materializes every input on local disk first, and the executor wraps the local files in a `LocalFileByteSource` to feed `execute_merge_operation` (or just calls the in-memory `merge_sorted_parquet_files` path). The streaming engine's central design benefit is unused. This isn't a correctness bug — both paths give the same result. It's a perf/architecture gap: every merge pays 2× I/O per input (network → scratch + scratch → merger), serializes phases (`max(input download time)` first-byte latency), and consumes scratch disk that scales with concurrent merges. Tracking as GAP-012 so we pick it up at the right time. The gap doc walks four options (stream-directly with download fallback, stream- by-default with circuit breaker, eliminate in-memory path only, stream-directly for promotion merges only) and the trade-offs between them — including the mid-merge retry surface, which is the main reason download-first is the current default. Surfaced during PR #6423 code walkthrough. Co-Authored-By: Claude Opus 4.7 (1M context) --------- Co-authored-by: Claude Opus 4.7 (1M context) --- .../gaps/011-no-legacy-promotion-planner.md | 136 +++++ ...12-merge-downloads-instead-of-streaming.md | 164 ++++++ docs/internals/adr/gaps/README.md | 2 + .../parquet_merge_executor.rs | 134 ++++- .../src/merge/metadata_aggregation.rs | 80 ++- .../quickwit-parquet-engine/src/merge/mod.rs | 81 ++- .../src/merge/policy/mod.rs | 150 +++++- .../src/merge/schema.rs | 29 +- .../src/merge/streaming.rs | 499 ++++++++++++++++++ .../src/storage/streaming_writer.rs | 197 ++++--- 10 files changed, 1344 insertions(+), 128 deletions(-) create mode 100644 docs/internals/adr/gaps/011-no-legacy-promotion-planner.md create mode 100644 docs/internals/adr/gaps/012-merge-downloads-instead-of-streaming.md diff --git a/docs/internals/adr/gaps/011-no-legacy-promotion-planner.md b/docs/internals/adr/gaps/011-no-legacy-promotion-planner.md new file mode 100644 index 00000000000..ec1e5642e34 --- /dev/null +++ b/docs/internals/adr/gaps/011-no-legacy-promotion-planner.md @@ -0,0 +1,136 @@ +# GAP-011: No Planner-Level Legacy Promotion + +**Status**: Open +**Discovered**: 2026-05-18 +**Context**: Codex review on PR #6423 (`feat(merge): legacy promotion path + body-col schema evolution`) flagged that the promotion path is wired end-to-end at the library + executor layer but has no production trigger at the planner / policy level. + +## Problem + +The streaming Parquet merge stack now contains a complete *legacy promotion* pipeline: + +- `ParquetMergeOperation::promote_legacy(splits, target_prefix_len)` constructs an operation with + `target_prefix_len_override = Some(target)`. +- `merge::execute_merge_operation` routes each input through `LegacyInputAdapter` when its + declared `rg_partition_prefix_len < target` and through `StreamingParquetReader` otherwise. The + streaming engine then sees a homogeneous stream advertising `prefix_len = target` on every + input. +- `ParquetMergeExecutor` (in `quickwit-indexing`) detects `target_prefix_len_override.is_some()` + and routes those merges through `execute_merge_operation` (with `LocalFileByteSource`) instead + of the in-memory `merge_sorted_parquet_files` path. +- `merge_parquet_split_metadata` accepts a `mixed_prefix_ok: bool` flag so the post-merge + aggregator skips the input-side equality check. + +What's missing: **nothing in the planner ever creates a `promote_legacy` operation in +production**. `MergePolicyState::record_split` buckets each split by +`CompactionScope::from_split`, and that scope key includes `rg_partition_prefix_len`. Legacy +splits (`prefix_len = 0`) and aligned splits (`prefix_len > 0`) therefore land in *different* +buckets before `ParquetMergePolicy::operations` ever runs. The production policy then iterates +each bucket independently and emits `ParquetMergeOperation::new` (regular merge). A repo-wide +search finds `promote_legacy` only in tests. + +In a mixed deployment (legacy + aligned splits coexisting), legacy splits therefore stay in +their `prefix_len = 0` bucket forever — never gaining the prefix alignment that downstream +locality compaction depends on. The promotion plumbing is reachable only from tests. + +## Evidence + +- `quickwit-parquet-engine/src/merge/policy/mod.rs`: `ParquetMergePolicy::operations` calls + `ParquetMergeOperation::new(...)` only. `promote_legacy` is constructed only by tests in the + same file. +- `MergePolicyState::record_split` keys its `BTreeMap` by `CompactionScope::from_split`. The + scope derivation includes `rg_partition_prefix_len`, so a legacy split and a prefix-aligned + split with otherwise identical sort fields / window / merge level are never compared by the + policy. +- The executor branch added in PR #6423 (`scratch.merge_operation.target_prefix_len_override + .is_some()`) routes promotion through `execute_merge_operation`. Library coverage at + `test_promote_legacy_executor_end_to_end` exercises a `prefix_len = 0` + `prefix_len = 1` pair + successfully. But that operation is only ever constructed inside the test. + +## State of the Art + +- **Iceberg**: Compaction policies inspect file-level metadata (partitioning, sort order) and + can rewrite files to align with the latest table partitioning even when individual files + pre-date the change. The compaction service treats schema-evolution-style rewrites as + first-class operations. +- **Husky**: Background re-organization passes that promote files into newer storage layouts. + Tracked separately from the size-tiered compaction policy so cost trade-offs can be tuned. + +In both cases, the design separates the *trigger* (decision to promote) from the *mechanism* +(how the promotion is performed). Quickwit currently has the mechanism but not the trigger. + +## Potential Solutions + +### Option A: Merge legacy + aligned buckets in `CompactionScope::from_split` + +Drop `rg_partition_prefix_len` from the scope key (or normalize it to a target value before +bucketing). The policy then sees legacy and aligned splits as candidates for the same +compaction operation and `ParquetMergePolicy::operations` decides whether to emit a regular +merge or a `promote_legacy` operation based on whether the bucket contains mixed prefix +lengths. + +Simplest change, but requires the policy to detect mixed-prefix buckets and choose between +`new` and `promote_legacy` per operation. + +### Option B: Dedicated promotion pass + +Run a separate pass before the regular compaction policy that scans for legacy splits and emits +`promote_legacy` operations for them. The regular policy then sees only aligned splits. + +Cleaner separation of concerns, but means legacy splits are migrated *before* any opportunity +to coalesce them with aligned neighbors in a single multi-input merge — possibly more work +overall. + +### Option C: Hybrid — bucket together, prefer single-pass promotion + +Keep scope bucketing as in option A. Inside the policy, when a bucket contains mixed prefix +lengths AND has enough splits to merit a multi-input merge, emit `promote_legacy`. When only +legacy splits exist (no aligned neighbor), emit `promote_legacy` with the same target — single- +input promotion is still valuable because it converts the file to the new format for future +locality compaction. + +Most flexible; gives the policy the freedom to amortize promotion cost when there are aligned +neighbors AND to still promote isolated legacy splits in the background. + +## Signal Impact + +Primarily affects **metrics** in the near term: the legacy split format pre-dates the +prefix-aligned RG layout, and only metrics has both formats in flight today. Traces and logs +on the Parquet path will eventually reach the same state if a layout change ever happens; the +same planner machinery would cover them. + +## Cost Considerations + +Promotion is strictly more expensive than a regular merge: the legacy adapter buffers the full +input file in memory and re-encodes it as a single-RG stream before the merge engine sees it. +For 50 MB metrics splits this is acceptable; for larger inputs the in-memory buffer is the +gating cost. + +The planner should account for this when scheduling — promotion is best amortized into a +multi-input merge rather than performed as a standalone file rewrite. Option C's "prefer +multi-input promotion, fall back to single-input" structure captures this. + +## Impact + +- **Severity**: Medium. Legacy splits accumulate cost (every query against them pays the + prefix-less scan cost) but correctness is preserved — the locality compaction stack still + works on aligned splits. +- **Frequency**: Persistent. Legacy splits never migrate without an explicit trigger. +- **Affected Areas**: `quickwit-parquet-engine/src/merge/policy/`, `quickwit-parquet-engine/src/merge/mod.rs` (`MergePolicyState::record_split` + `CompactionScope`). + +## Next Steps + +- [ ] Decide between options A / B / C based on operational priorities and benchmark data. +- [ ] Design the policy-level "should promote?" heuristic: how many legacy splits before + triggering, whether to wait for aligned neighbors, how to deprioritize promotion vs + regular compaction. +- [ ] Add metrics for `legacy_splits_pending_promotion` and `promotion_operations_emitted` so we + can observe the policy in production. +- [ ] Wire whichever option is chosen, with an integration test that exercises the full path + (legacy split → planner → executor → published prefix-aligned split). + +## References + +- PR #6423 (legacy promotion path + body-col schema evolution). +- Codex review comment id `4311184497` (raised the gap). +- `test_promote_legacy_executor_end_to_end` in `quickwit-parquet-engine::merge::streaming` — + library-level coverage of the mechanism. diff --git a/docs/internals/adr/gaps/012-merge-downloads-instead-of-streaming.md b/docs/internals/adr/gaps/012-merge-downloads-instead-of-streaming.md new file mode 100644 index 00000000000..871cdb84bb3 --- /dev/null +++ b/docs/internals/adr/gaps/012-merge-downloads-instead-of-streaming.md @@ -0,0 +1,164 @@ +# GAP-012: Parquet Merge Executor Downloads Inputs Instead of Streaming Them + +**Status**: Open +**Discovered**: 2026-05-18 +**Context**: Code review of the Parquet streaming merge stack (PRs #6407–#6428) — specifically the executor wiring on #6423 — surfaced the question of why the merge actor downloads every input to local disk before merging when the streaming engine is designed around `RemoteByteSource`. + +## Problem + +The Parquet streaming merge engine in `quickwit-parquet-engine` consumes inputs through a +minimal `RemoteByteSource` trait (`file_size`, `get_slice`, `get_slice_stream`). The trait was +deliberately defined so the engine can pull pages column-major directly from object storage — +two GETs per input (footer + body stream) and the merge progresses as bytes arrive, holding +only the page-bounded engine state in memory. + +The actor pipeline in `quickwit-indexing` doesn't use that design. The +`ParquetMergeSplitDownloader` actor pulls each input via `storage.copy_to_file(remote_path, +local_path)` into a scratch directory, then hands `Vec` to the +`ParquetMergeExecutor`. The executor then either: + +- Calls the in-memory `merge_sorted_parquet_files(input_paths, ...)` (regular merges), which + reads each file fully into Arrow RecordBatches before merging, OR +- Wraps each local path in a `LocalFileByteSource` and calls `execute_merge_operation` (added in + PR #6423 for promotion merges only). + +Either way, the streaming engine's central design benefit — overlapping the fetch with the +merge and skipping the scratch disk entirely — is unused in production. Every merge reads each +input twice: once over the network into scratch, once off scratch through the merger. + +## Evidence + +- `quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_split_downloader.rs`: per-split + loop calling `self.storage.copy_to_file(...)` to materialize every input on local disk before + forwarding `ParquetMergeScratch` to the executor. +- `quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_executor.rs`: receives + `downloaded_parquet_files: Vec` and chooses between the in-memory path or + `execute_merge_operation` with `LocalFileByteSource` wrappers — never a `RemoteByteSource` + that actually streams from object storage. +- `quickwit-parquet-engine/src/storage/streaming_reader.rs:62-67`: the `RemoteByteSource` trait + doc explicitly notes that callers in `quickwit-indexing` "provide a thin adapter that + delegates to `quickwit_storage::Storage`." The adapter exists in principle but isn't wired up + for the merge executor. + +## State of the Art + +- **ClickHouse `MergeTree`**: parts are accessed via the same storage abstraction whether the + merge runs locally or against tiered/object storage. There's no separate "download then + merge" actor pair — the merger reads parts where they live. +- **Iceberg compaction**: data files are read directly from object storage by the compaction + job. Local scratch is used only for the output file before commit. +- **Husky**: column-major streaming merge reads directly from blob storage. Designed around the + "two GETs per input" model the Quickwit streaming engine inherits. + +Across these systems, downloading inputs before merging is treated as a fallback for +operational reasons (unreliable network, kernel page-cache effects), not the default. + +## Trade-offs + +### Why download-first is the current default +- **Retry locality**: the downloader actor centralizes retry/backoff/timeout for one file at a + time. A mid-fetch S3 hiccup retries the download alone; the merger sees only successful + downloads. +- **Pure-compute executor**: once files are on disk the executor has no network dependency. + Mid-merge failures are restricted to disk I/O and compute errors. +- **Predictable disk budget**: scratch usage is bounded by `Σ input_sizes` per concurrent + merge. Easy to reason about; easy to cap. +- **Legacy in-memory path**: `merge_sorted_parquet_files` predates the streaming engine and + requires local file paths. The download-first pattern existed before there was a streaming + alternative. + +### What download-first costs +- **2× I/O per merge**: each input is transferred over the network into scratch AND read off + scratch into the merger. The kernel page cache mitigates the disk-read pass to some extent but + doesn't fully erase it. +- **Serialized phases**: the merge can't start until *all* inputs are downloaded. First-byte + latency on the merger is `max(input download time)` instead of `min(input first-byte time)`. +- **Scratch disk usage**: a typical compaction merging 8× 50 MB splits holds 400 MB of scratch + per merge, multiplied by the concurrent merge count. On lightweight indexer pods this caps + parallelism. +- **Underused design**: the streaming engine's single-body-GET model + page-bounded memory was + built specifically for the no-scratch-disk case. Wiring through `LocalFileByteSource` works + but bypasses the property the design was built around. + +### What streaming-directly would cost +- **Mid-merge retry surface**: a connection failure mid-body-GET kills the merge attempt + entirely. Single-body-GET is forward-only — no partial recovery. The retry surface becomes + "the merge failed after 30 % of work," not "the download failed, retry the file." +- **Per-merge S3 connection count**: an N-way merge holds N concurrent body streams plus N + footer connections. On dense merger nodes this multiplies. +- **Tail latency**: the merge progresses at the speed of the slowest input. With downloads, + parallel fetches average out; with streaming a slow input throttles the whole merge. + +## Potential Solutions + +### Option A: Streaming-directly when the input is reachable, download as fallback + +The executor receives a hint from the storage layer (or detects mid-merge failure rates) and +chooses per merge. Splits stored on reliable, low-latency backends go through `RemoteByteSource` +adapters that talk directly to `quickwit_storage::Storage`; on flaky or high-latency backends +the downloader actor still materializes files first. + +Largest design lift but matches what mature compaction systems do. + +### Option B: Stream-directly by default, fall back to download on persistent failures + +Default to streaming; a circuit-breaker on per-merge failure rate routes the next attempt +through download-first. Operationally simpler than Option A; tail latency is bounded by the +circuit's reaction time. + +### Option C: Keep download-first but eliminate the in-memory merge path + +Make every merge go through `execute_merge_operation` with `LocalFileByteSource`. This doesn't +recover the streaming engine's "no scratch disk" benefit but does remove the legacy in-memory +codepath, simplifying the executor to a single path. + +Smallest change, smallest gain. Worth doing regardless of A/B as a stepping stone. + +### Option D: Streaming-directly only for promotion merges + +Promotion already routes through `execute_merge_operation`; extend it to skip the download +phase entirely for those operations and let the regular path stay as-is. Gains: legacy-adapter- +backed promotion merges (the in-memory-buffering-heaviest case in the pipeline) avoid double +I/O. Costs: split executor logic into "promotion = stream" vs "regular = download." + +## Signal Impact + +All Parquet-backed signals. Metrics is the first product to ship, so the impact lands on +metrics first; traces and logs (when they migrate to Parquet storage) will pay the same cost +unless this is addressed by then. + +## Cost Considerations + +The streaming engine's body-col page cache is already designed for backpressure: pages stream +in column-major order as bytes arrive, and the engine processes them as quickly as it can. The +bottleneck for streaming-directly becomes the slowest input's transfer rate rather than the +total input size — usually a smaller number, but a longer tail. + +## Impact + +- **Severity**: Medium. Correctness is unaffected; the streaming engine works equivalently + whether the source is local or remote. The cost is bandwidth, disk, and wall-clock latency. +- **Frequency**: Every merge in production today pays the download cost. +- **Affected Areas**: `quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_split_downloader.rs`, + `quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_executor.rs`, + `quickwit-parquet-engine::merge::execute_merge_operation` callers. + +## Next Steps + +- [ ] Measure the current download-vs-merge phase breakdown on a representative production + merge load (wall-clock + bytes-read + disk-write). +- [ ] Build a `RemoteByteSource` adapter over `quickwit_storage::Storage` and prototype + streaming-directly for promotion merges (Option D) to validate the engine's behavior + against the existing storage backends. +- [ ] Decide between options A / B based on observed mid-merge failure rates under real S3 + conditions. +- [ ] Even if the default stays download-first, consider Option C as a simplification — the + in-memory merge path is dead weight once the streaming engine handles every case. + +## References + +- PR #6407–#6428 (Parquet streaming merge stack). +- [PR #6423 discussion](https://github.com/quickwit-oss/quickwit/pull/6423) — surfaced the + question while wiring promotion through `execute_merge_operation`. +- `quickwit-parquet-engine/src/storage/streaming_reader.rs` (`RemoteByteSource` trait). +- `quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_executor.rs::LocalFileByteSource`. diff --git a/docs/internals/adr/gaps/README.md b/docs/internals/adr/gaps/README.md index 2e69c27cdbd..e8ef3de02e3 100644 --- a/docs/internals/adr/gaps/README.md +++ b/docs/internals/adr/gaps/README.md @@ -115,3 +115,5 @@ Gap files use sequential numbering: `001-short-description.md` | [008](./008-no-high-query-rate-optimization.md) | No High Query Rate Optimization | Open | High | | [009](./009-no-leading-edge-prioritization.md) | No Leading Edge Prioritization | Open | High | | [010](./010-no-data-caching-or-query-affinity.md) | No Multi-Level Data Caching or Query Affinity Optimization | Open | High | +| [011](./011-no-legacy-promotion-planner.md) | No Planner-Level Legacy Promotion | Open | Medium | +| [012](./012-merge-downloads-instead-of-streaming.md) | Parquet Merge Executor Downloads Inputs Instead of Streaming Them | Open | Medium | diff --git a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_executor.rs b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_executor.rs index 8d2649973a2..d72884236fd 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_executor.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_executor.rs @@ -14,12 +14,30 @@ //! Parquet merge executor actor. //! -//! Calls the Phase 1 merge engine (`merge_sorted_parquet_files`) via -//! `run_cpu_intensive()`, builds output split metadata using -//! `merge_parquet_split_metadata()`, and sends the result to the uploader. +//! Receives a `ParquetMergeScratch` from the downloader. Two merge +//! paths, chosen on `target_prefix_len_override`: +//! +//! - **Regular merge** (`target_prefix_len_override == None`): the input splits already share +//! `qh.rg_partition_prefix_len` (MP-3). Run the in-memory `merge_sorted_parquet_files` engine +//! inside `run_cpu_intensive()`. +//! - **Promotion merge** (`target_prefix_len_override == Some(target)`): the inputs may carry +//! *different* prefix lengths (the whole point of promotion). The in-memory path's +//! `extract_and_validate_input_metadata` would bail on the mismatch before any output is +//! produced, so promotion runs through `execute_merge_operation` instead — that path opens each +//! input through `LegacyInputAdapter` for splits below the target prefix len and through +//! `StreamingParquetReader` for splits at the target, so the streaming engine sees a homogeneous +//! stream advertising `prefix_len = target` on every input. `mixed_prefix_ok` is then passed to +//! `merge_parquet_split_metadata` so the post-merge aggregator skips the input-side equality +//! check. + +use std::io; +use std::ops::Range; +use std::path::{Path, PathBuf}; +use std::sync::Arc; use anyhow::Context; use async_trait::async_trait; +use bytes::Bytes; use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler, Mailbox, QueueCapacity}; use quickwit_common::thread_pool::run_cpu_intensive; use quickwit_dst::check_invariant; @@ -28,9 +46,12 @@ use quickwit_dst::invariants::merge_policy::{ all_same_compaction_scope, all_same_merge_level, has_minimum_splits, }; use quickwit_parquet_engine::merge::metadata_aggregation::merge_parquet_split_metadata; -use quickwit_parquet_engine::merge::{MergeConfig, MergeOutputFile, merge_sorted_parquet_files}; -use quickwit_parquet_engine::storage::ParquetWriterConfig; +use quickwit_parquet_engine::merge::{ + MergeConfig, MergeOutputFile, execute_merge_operation, merge_sorted_parquet_files, +}; +use quickwit_parquet_engine::storage::{ParquetWriterConfig, RemoteByteSource}; use quickwit_proto::types::IndexUid; +use tokio::io::{AsyncRead, AsyncReadExt, AsyncSeekExt}; use tracing::{info, instrument, warn}; use super::ParquetUploader; @@ -38,6 +59,53 @@ use super::parquet_indexer::ParquetSplitBatch; use super::parquet_merge_messages::{ParquetMergeScratch, ParquetMergeTask}; use crate::models::PublishLock; +/// `RemoteByteSource` adapter over a single local file. Used by the +/// promotion-merge path to feed downloaded scratch-directory files +/// into `execute_merge_operation` (which composes them with +/// `LegacyInputAdapter` as needed). +/// +/// Each instance is bound to one absolute path at construction time +/// and ignores the `path` argument from the trait methods — the +/// trait surface assumes a remote backend keyed by path, but the +/// downloader has already resolved each split to a concrete local +/// file before the executor runs. +struct LocalFileByteSource { + path: PathBuf, +} + +impl LocalFileByteSource { + fn new(path: PathBuf) -> Self { + Self { path } + } +} + +#[async_trait] +impl RemoteByteSource for LocalFileByteSource { + async fn file_size(&self, _path: &Path) -> io::Result { + tokio::fs::metadata(&self.path).await.map(|m| m.len()) + } + + async fn get_slice(&self, _path: &Path, range: Range) -> io::Result { + let mut file = tokio::fs::File::open(&self.path).await?; + file.seek(io::SeekFrom::Start(range.start)).await?; + let len = (range.end - range.start) as usize; + let mut buf = vec![0u8; len]; + file.read_exact(&mut buf).await?; + Ok(Bytes::from(buf)) + } + + async fn get_slice_stream( + &self, + _path: &Path, + range: Range, + ) -> io::Result> { + let mut file = tokio::fs::File::open(&self.path).await?; + file.seek(io::SeekFrom::Start(range.start)).await?; + let len = range.end - range.start; + Ok(Box::new(file.take(len))) + } +} + /// Executes Parquet merge operations using the Phase 1 k-way merge engine. /// /// Receives `ParquetMergeScratch` from the downloader, runs the merge as a @@ -108,18 +176,46 @@ impl Handler for ParquetMergeExecutor { .context("failed to create merge output directory") .map_err(|e| ActorExitStatus::from(anyhow::anyhow!(e)))?; - // Run the CPU-intensive merge on the dedicated thread pool. - let input_paths = scratch.downloaded_parquet_files.clone(); - let output_dir_clone = output_dir.clone(); - let writer_config = self.writer_config.clone(); - let merge_result = run_cpu_intensive(move || { + // Route promotion merges (`target_prefix_len_override.is_some()`) + // through `execute_merge_operation`. That path opens each input + // via `LegacyInputAdapter` when the split's prefix_len is below + // the target, so the streaming merge engine sees a homogeneous + // stream with all inputs at `prefix_len = target`. Without this + // routing the in-memory `merge_sorted_parquet_files` would bail + // in `extract_and_validate_input_metadata` on mixed prefix + // lengths before any output is produced. + let merge_result: Result, _>, _> = if scratch + .merge_operation + .target_prefix_len_override + .is_some() + { + let sources: Vec> = scratch + .downloaded_parquet_files + .iter() + .map(|path| Arc::new(LocalFileByteSource::new(path.clone())) as _) + .collect(); let config = MergeConfig { num_outputs: 1, - writer_config, + writer_config: self.writer_config.clone(), }; - merge_sorted_parquet_files(&input_paths, &output_dir_clone, &config) - }) - .await; + Ok( + execute_merge_operation(&scratch.merge_operation, sources, &output_dir, &config) + .await, + ) + } else { + // Regular merge: in-memory path under `run_cpu_intensive`. + let input_paths = scratch.downloaded_parquet_files.clone(); + let output_dir_clone = output_dir.clone(); + let writer_config = self.writer_config.clone(); + run_cpu_intensive(move || { + let config = MergeConfig { + num_outputs: 1, + writer_config, + }; + merge_sorted_parquet_files(&input_paths, &output_dir_clone, &config) + }) + .await + }; let outputs: Vec = match merge_result { Ok(Ok(outputs)) => outputs, @@ -231,9 +327,17 @@ impl Handler for ParquetMergeExecutor { return Ok(()); } + // `mixed_prefix_ok` matches the operation's promotion mode: + // promote-legacy operations bundle inputs from different + // `rg_partition_prefix_len` buckets (the adapter normalizes + // them at read time), so the input-side equality check in + // `merge_parquet_split_metadata` would spuriously fail. Regular + // merges keep the strict check. + let mixed_prefix_ok = scratch.merge_operation.target_prefix_len_override.is_some(); + let mut merged_splits = Vec::with_capacity(outputs.len()); for output in &outputs { - let mut metadata = merge_parquet_split_metadata(input_splits, output) + let mut metadata = merge_parquet_split_metadata(input_splits, output, mixed_prefix_ok) .context("failed to build merge output metadata") .map_err(|e| ActorExitStatus::from(anyhow::anyhow!(e)))?; diff --git a/quickwit/quickwit-parquet-engine/src/merge/metadata_aggregation.rs b/quickwit/quickwit-parquet-engine/src/merge/metadata_aggregation.rs index 1da61905b88..0805f6ef35c 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/metadata_aggregation.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/metadata_aggregation.rs @@ -40,10 +40,17 @@ use crate::split::{ParquetSplitId, ParquetSplitMetadata}; /// # Preconditions /// /// All input splits must share the same kind, index_uid, partition_id, -/// sort_fields, and window. +/// sort_fields, and window. In the default case (`mixed_prefix_ok = false`) +/// they must also share `rg_partition_prefix_len`. In legacy-promotion +/// mode (`mixed_prefix_ok = true`) the prefix-len equality check is +/// skipped because inputs come from different prefix buckets — the +/// output's prefix_len is taken from the writer's KV stamp via +/// `output.output_rg_partition_prefix_len` (CS-1), so the input-side +/// equality is no longer load-bearing for the metastore record. pub fn merge_parquet_split_metadata( inputs: &[ParquetSplitMetadata], output: &MergeOutputFile, + mixed_prefix_ok: bool, ) -> Result { if inputs.is_empty() { bail!("merge_parquet_split_metadata requires at least one input split"); @@ -93,10 +100,11 @@ pub fn merge_parquet_split_metadata( first.window ); } - if input.rg_partition_prefix_len != first.rg_partition_prefix_len { + if !mixed_prefix_ok && input.rg_partition_prefix_len != first.rg_partition_prefix_len { bail!( "input {} has rg_partition_prefix_len {}, expected {} — splits with different \ - prefix lengths must not appear in the same merge", + prefix lengths must not appear in the same regular merge (legacy-promotion \ + operations bypass this check)", i, input.rg_partition_prefix_len, first.rg_partition_prefix_len @@ -248,7 +256,7 @@ mod tests { make_test_split("s1", (1200, 2000), 0), ]; let output = make_output(200, 9000); - let result = merge_parquet_split_metadata(&inputs, &output).unwrap(); + let result = merge_parquet_split_metadata(&inputs, &output, false).unwrap(); // Invariant fields come from inputs. assert_eq!(result.kind, ParquetSplitKind::Metrics); @@ -267,7 +275,7 @@ mod tests { make_test_split("s1", (1200, 2000), 0), ]; let output = make_output_with_metadata(200, 9000, (1000, 2000), &["cpu.usage", "mem.used"]); - let result = merge_parquet_split_metadata(&inputs, &output).unwrap(); + let result = merge_parquet_split_metadata(&inputs, &output, false).unwrap(); // Data-dependent fields come from the output, not inputs. assert_eq!(result.time_range.start_secs, 1000); @@ -302,7 +310,7 @@ mod tests { .or_default() .insert("api".to_string()); - let result = merge_parquet_split_metadata(&inputs, &output).unwrap(); + let result = merge_parquet_split_metadata(&inputs, &output, false).unwrap(); let service_values = result.low_cardinality_tags.get("service").unwrap(); assert_eq!(service_values.len(), 2); @@ -323,7 +331,7 @@ mod tests { .insert(format!("host-{i}")); } - let result = merge_parquet_split_metadata(&inputs, &output).unwrap(); + let result = merge_parquet_split_metadata(&inputs, &output, false).unwrap(); assert!(result.high_cardinality_tag_keys.contains("host")); assert!(!result.low_cardinality_tags.contains_key("host")); @@ -337,7 +345,7 @@ mod tests { make_test_split("s2", (1000, 2000), 2), ]; let output = make_output(300, 12000); - let result = merge_parquet_split_metadata(&inputs, &output).unwrap(); + let result = merge_parquet_split_metadata(&inputs, &output, false).unwrap(); assert_eq!(result.num_merge_ops, 3); // max(2,2,2) + 1 } @@ -345,7 +353,7 @@ mod tests { #[test] fn test_empty_inputs_error() { let output = make_output(0, 0); - let result = merge_parquet_split_metadata(&[], &output); + let result = merge_parquet_split_metadata(&[], &output, false); assert!(result.is_err()); assert!( result @@ -362,7 +370,7 @@ mod tests { s1.kind = ParquetSplitKind::Sketches; let output = make_output(200, 9000); - let result = merge_parquet_split_metadata(&[s0, s1], &output); + let result = merge_parquet_split_metadata(&[s0, s1], &output, false); assert!(result.is_err()); } @@ -373,7 +381,7 @@ mod tests { s1.index_uid = "other-index:00000000000000000000000002".to_string(); let output = make_output(200, 9000); - let result = merge_parquet_split_metadata(&[s0, s1], &output); + let result = merge_parquet_split_metadata(&[s0, s1], &output, false); assert!(result.is_err()); } @@ -384,7 +392,7 @@ mod tests { s1.partition_id = 99; let output = make_output(200, 9000); - let result = merge_parquet_split_metadata(&[s0, s1], &output); + let result = merge_parquet_split_metadata(&[s0, s1], &output, false); assert!(result.is_err()); } @@ -395,7 +403,7 @@ mod tests { s1.sort_fields = "different|schema/V2".to_string(); let output = make_output(200, 9000); - let result = merge_parquet_split_metadata(&[s0, s1], &output); + let result = merge_parquet_split_metadata(&[s0, s1], &output, false); assert!(result.is_err()); } @@ -406,7 +414,7 @@ mod tests { s1.window = Some(2000..5600); let output = make_output(200, 9000); - let result = merge_parquet_split_metadata(&[s0, s1], &output); + let result = merge_parquet_split_metadata(&[s0, s1], &output, false); assert!(result.is_err()); } @@ -417,7 +425,7 @@ mod tests { s1.rg_partition_prefix_len = 1; let output = make_output(200, 9000); - let result = merge_parquet_split_metadata(&[s0, s1], &output); + let result = merge_parquet_split_metadata(&[s0, s1], &output, false); let err = result.expect_err("merge must reject mismatched prefix lengths"); let msg = err.to_string(); assert!( @@ -442,7 +450,7 @@ mod tests { // num_row_groups = 2 + writer reports demoted prefix_len = 0 // (the legacy writer's choice for a row-count-driven multi-RG). let output = make_output_full_with_prefix(200, 9000, 2, 0, (1000, 2000), &["cpu.usage"]); - let result = merge_parquet_split_metadata(&[s0, s1], &output).unwrap(); + let result = merge_parquet_split_metadata(&[s0, s1], &output, false).unwrap(); assert_eq!(result.rg_partition_prefix_len, 0); } @@ -457,10 +465,42 @@ mod tests { s1.rg_partition_prefix_len = 3; let output = make_output_full_with_prefix(200, 9000, 1, 3, (1000, 2000), &["cpu.usage"]); - let result = merge_parquet_split_metadata(&[s0, s1], &output).unwrap(); + let result = merge_parquet_split_metadata(&[s0, s1], &output, false).unwrap(); assert_eq!(result.rg_partition_prefix_len, 3); } + #[test] + fn test_mixed_prefix_ok_skips_input_equality_check() { + // Promotion mode: inputs come from different prefix buckets + // (e.g. one prefix_len=0 legacy + one prefix_len=2 aligned). + // With `mixed_prefix_ok = true` the aggregator must accept this + // and take the output prefix from the writer's stamp. + let mut legacy = make_test_split("s0", (1000, 2000), 0); + legacy.rg_partition_prefix_len = 0; + let mut aligned = make_test_split("s1", (1000, 2000), 0); + aligned.rg_partition_prefix_len = 2; + + // Writer stamps prefix_len = 2 on the multi-RG output (streaming + // engine output that successfully promoted the legacy input). + let output = make_output_full_with_prefix(300, 12000, 3, 2, (1000, 2000), &["cpu.usage"]); + + let result = + merge_parquet_split_metadata(&[legacy.clone(), aligned.clone()], &output, true) + .expect("mixed-prefix inputs must be accepted in promotion mode"); + assert_eq!( + result.rg_partition_prefix_len, 2, + "output prefix matches the writer's stamp (CS-1)", + ); + + // Same inputs without the mixed_prefix_ok flag must still fail. + let strict = merge_parquet_split_metadata(&[legacy, aligned], &output, false); + let err = strict.expect_err("strict mode must reject mixed-prefix inputs"); + assert!( + err.to_string().contains("rg_partition_prefix_len"), + "error should mention the prefix-len mismatch, got: {err}", + ); + } + #[test] fn test_output_prefix_len_preserved_on_multi_rg_streaming_engine() { // CS-1 regression for F1: the streaming engine produces @@ -479,7 +519,7 @@ mod tests { // num_row_groups = 3 (multi-RG) AND writer reports prefix_len = 2 // (the streaming engine's stamp because it verified alignment). let output = make_output_full_with_prefix(300, 12000, 3, 2, (1000, 2000), &["cpu.usage"]); - let result = merge_parquet_split_metadata(&[s0, s1], &output).unwrap(); + let result = merge_parquet_split_metadata(&[s0, s1], &output, false).unwrap(); assert_eq!( result.rg_partition_prefix_len, 2, "metastore must mirror the writer's KV (CS-1); multi-RG aligned output keeps its \ @@ -494,7 +534,7 @@ mod tests { make_test_split("s1", (1000, 2000), 0), ]; let output = make_output(200, 9000); - let result = merge_parquet_split_metadata(&inputs, &output).unwrap(); + let result = merge_parquet_split_metadata(&inputs, &output, false).unwrap(); assert_ne!(result.split_id.as_str(), "s0"); assert_ne!(result.split_id.as_str(), "s1"); @@ -510,7 +550,7 @@ mod tests { output.row_keys_proto = None; output.zonemap_regexes = HashMap::new(); - let result = merge_parquet_split_metadata(&inputs, &output).unwrap(); + let result = merge_parquet_split_metadata(&inputs, &output, false).unwrap(); assert!(result.row_keys_proto.is_none()); assert!(result.zonemap_regexes.is_empty()); diff --git a/quickwit/quickwit-parquet-engine/src/merge/mod.rs b/quickwit/quickwit-parquet-engine/src/merge/mod.rs index 7ab8d685f7a..c32b130e170 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/mod.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/mod.rs @@ -31,6 +31,7 @@ mod writer; mod tests; use std::path::{Path, PathBuf}; +use std::sync::Arc; use anyhow::{Context, Result, bail}; use arrow::array::RecordBatch; @@ -41,8 +42,9 @@ pub use self::merge_order::MergeRun; use crate::sort_fields::{equivalent_schemas_for_compaction, parse_sort_fields}; use crate::sorted_series::SORTED_SERIES_COLUMN; use crate::storage::{ - PARQUET_META_NUM_MERGE_OPS, PARQUET_META_RG_PARTITION_PREFIX_LEN, PARQUET_META_SORT_FIELDS, - PARQUET_META_WINDOW_DURATION, PARQUET_META_WINDOW_START, ParquetWriterConfig, + ColumnPageStream, LegacyInputAdapter, PARQUET_META_NUM_MERGE_OPS, + PARQUET_META_RG_PARTITION_PREFIX_LEN, PARQUET_META_SORT_FIELDS, PARQUET_META_WINDOW_DURATION, + PARQUET_META_WINDOW_START, ParquetWriterConfig, RemoteByteSource, StreamingParquetReader, }; /// Configuration for a merge operation. @@ -483,3 +485,78 @@ fn extract_and_validate_input_metadata(paths: &[PathBuf]) -> Result>, + output_dir: &Path, + config: &MergeConfig, +) -> Result> { + if sources.len() != op.splits.len() { + bail!( + "execute_merge_operation: sources.len() ({}) != op.splits.len() ({})", + sources.len(), + op.splits.len(), + ); + } + + let mut streams: Vec> = Vec::with_capacity(op.splits.len()); + for (split, source) in op.splits.iter().zip(sources) { + let path = PathBuf::from(&split.parquet_file); + let stream: Box = match op.target_prefix_len_override { + Some(target) if split.rg_partition_prefix_len < target => { + // Promote this legacy input. The adapter re-encodes in + // memory and presents itself as a prefix_len = target + // single-RG stream to the merge engine. + let adapter = LegacyInputAdapter::try_open(source, path, target) + .await + .with_context(|| { + format!( + "opening legacy adapter for split {} with target_prefix_len = {target}", + split.split_id, + ) + })?; + Box::new(adapter) + } + _ => { + // Direct streaming reader: regular merge, or promotion + // where this input already satisfies the target. + let reader = StreamingParquetReader::try_open(source, path) + .await + .with_context(|| { + format!("opening streaming reader for split {}", split.split_id) + })?; + Box::new(reader) + } + }; + streams.push(stream); + } + + streaming::streaming_merge_sorted_parquet_files(streams, output_dir, config).await +} diff --git a/quickwit/quickwit-parquet-engine/src/merge/policy/mod.rs b/quickwit/quickwit-parquet-engine/src/merge/policy/mod.rs index f373f71bd0c..3a0a72d7da8 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/policy/mod.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/policy/mod.rs @@ -40,13 +40,25 @@ use crate::split::{ParquetSplitId, ParquetSplitKind, ParquetSplitMetadata}; pub struct ParquetMergeOperation { /// New split ID for the merged output. pub merge_split_id: ParquetSplitId, - /// The input splits being merged (all from the same compaction scope and - /// `num_merge_ops` level). + /// The input splits being merged. All share the same `num_merge_ops` + /// level and the same windowed compaction scope (sort_fields + + /// window). In the default form they also share `rg_partition_prefix_len`; + /// in promotion mode (`target_prefix_len_override` is `Some`) they + /// may differ in prefix length up to the target — see + /// [`Self::promote_legacy`]. pub splits: Vec, + /// When `Some(N)`, this operation is a legacy-promotion merge: + /// inputs may carry `rg_partition_prefix_len < N` (they will be + /// opened through [`crate::storage::LegacyInputAdapter`] with + /// `target_prefix_len = N`) and the output will declare + /// `rg_partition_prefix_len = N`. When `None`, the operation is a + /// regular merge and all inputs must agree on prefix_len (default + /// behaviour). + pub target_prefix_len_override: Option, } impl ParquetMergeOperation { - /// Create a new merge operation consuming the given splits. + /// Create a regular merge operation consuming the given splits. /// /// Generates a fresh split ID for the merged output. The `kind` is inferred /// from the first split (all splits in a merge share the same kind). @@ -56,7 +68,66 @@ impl ParquetMergeOperation { /// - **MP-1**: all splits share the same `num_merge_ops` level /// - **MP-2**: at least 2 input splits /// - **MP-3**: all splits share the same compaction scope (sort_fields + window) + /// + /// For legacy-promotion operations (inputs at different + /// `rg_partition_prefix_len`), use [`Self::promote_legacy`] instead. pub fn new(splits: Vec) -> Self { + Self::check_mp1_mp2_mp3(&splits); + let kind = splits + .first() + .map(|s| s.kind) + .unwrap_or(ParquetSplitKind::Metrics); + Self { + merge_split_id: ParquetSplitId::generate(kind), + splits, + target_prefix_len_override: None, + } + } + + /// Create a legacy-promotion merge operation. + /// + /// Inputs may have heterogeneous `rg_partition_prefix_len` as long + /// as every input's value is `<= target_prefix_len`. The executor + /// opens any input with `prefix_len < target_prefix_len` through + /// [`crate::storage::LegacyInputAdapter`] with `target` set to the + /// override; inputs already at the target are opened directly via + /// the streaming reader. + /// + /// All other MP-3 dimensions (sort_fields, window) still must + /// agree — only the prefix-len equality is relaxed. + /// + /// # Panics (debug builds) / metrics (all builds) + /// + /// - **MP-1**: all splits share the same `num_merge_ops` level + /// - **MP-2**: at least 2 input splits + /// - **MP-3 (relaxed)**: all splits share sort_fields + window + /// - All inputs' `rg_partition_prefix_len <= target_prefix_len`. Inputs above the target are a + /// planner bug — they shouldn't be demoted, only promoted. + pub fn promote_legacy(splits: Vec, target_prefix_len: u32) -> Self { + Self::check_mp1_mp2_mp3(&splits); + // Every input must be promotable: prefix_len <= target. + // Demoting (input > target) is not the adapter's contract. + for (i, split) in splits.iter().enumerate() { + assert!( + split.rg_partition_prefix_len <= target_prefix_len, + "promote_legacy: input {i} has rg_partition_prefix_len = {} > target_prefix_len = \ + {target_prefix_len}; the adapter cannot demote a higher prefix to a lower one. \ + Pick a target >= max(inputs' prefix_len) or exclude this input.", + split.rg_partition_prefix_len, + ); + } + let kind = splits + .first() + .map(|s| s.kind) + .unwrap_or(ParquetSplitKind::Metrics); + Self { + merge_split_id: ParquetSplitId::generate(kind), + splits, + target_prefix_len_override: Some(target_prefix_len), + } + } + + fn check_mp1_mp2_mp3(splits: &[ParquetSplitMetadata]) { use quickwit_dst::check_invariant; use quickwit_dst::invariants::{InvariantId, merge_policy}; @@ -90,15 +161,6 @@ impl ParquetMergeOperation { InvariantId::MP3, merge_policy::all_same_compaction_scope(&sort_fields_vec, &windows) ); - - let kind = splits - .first() - .map(|s| s.kind) - .unwrap_or(ParquetSplitKind::Metrics); - Self { - merge_split_id: ParquetSplitId::generate(kind), - splits, - } } /// Returns the input splits as a slice. @@ -251,5 +313,69 @@ mod tests { ]; let op = ParquetMergeOperation::new(splits); assert_eq!(op.splits.len(), 2); + assert!( + op.target_prefix_len_override.is_none(), + "regular merges don't set the override", + ); + } + + /// Legacy-promotion happy path: a prefix_len=0 split + a prefix_len=2 + /// split with target=2. Both inputs share the windowed scope; the + /// operation records `target_prefix_len_override = Some(2)`. + #[test] + fn test_promote_legacy_pairs_legacy_with_aligned_peer() { + let mut legacy = make_split("legacy", 0, "metric_name|service|ts/V2", Some((0, 3600))); + legacy.rg_partition_prefix_len = 0; + + let mut aligned = make_split("aligned", 0, "metric_name|service|ts/V2", Some((0, 3600))); + aligned.rg_partition_prefix_len = 2; + + let op = ParquetMergeOperation::promote_legacy(vec![legacy, aligned], 2); + assert_eq!(op.splits.len(), 2); + assert_eq!(op.target_prefix_len_override, Some(2)); + } + + /// Promotion requires all inputs to have `prefix_len <= target`. + /// Passing an input whose prefix_len exceeds the target is a planner + /// bug — the adapter cannot DEMOTE alignment, only promote. + #[test] + #[should_panic(expected = "cannot demote a higher prefix")] + fn test_promote_legacy_rejects_higher_prefix_input() { + let mut legacy = make_split("legacy", 0, "a|b|ts/V2", Some((0, 3600))); + legacy.rg_partition_prefix_len = 0; + + let mut too_high = make_split("too_high", 0, "a|b|ts/V2", Some((0, 3600))); + too_high.rg_partition_prefix_len = 3; + + // target = 2, but too_high.rg_partition_prefix_len = 3. + ParquetMergeOperation::promote_legacy(vec![legacy, too_high], 2); + } + + /// Promotion still requires MP-3 on the non-prefix scope + /// dimensions: sort_fields + window. Mixed sort_fields must still + /// panic. + #[test] + #[should_panic(expected = "MP-3 violated")] + fn test_promote_legacy_still_enforces_sort_fields() { + let mut a = make_split("a", 0, "metric_name|ts/V2", Some((0, 3600))); + a.rg_partition_prefix_len = 0; + let mut b = make_split("b", 0, "different|schema/V2", Some((0, 3600))); + b.rg_partition_prefix_len = 1; + ParquetMergeOperation::promote_legacy(vec![a, b], 1); + } + + /// All inputs at the target prefix_len (no actual legacy promotion + /// happening) — the constructor still accepts it. The executor + /// will just open every input directly without the adapter. + /// Useful when a planner produces a uniform op that happens to be + /// at the same target. + #[test] + fn test_promote_legacy_all_at_target_is_valid() { + let mut a = make_split("a", 0, "a|ts/V2", Some((0, 3600))); + a.rg_partition_prefix_len = 1; + let mut b = make_split("b", 0, "a|ts/V2", Some((0, 3600))); + b.rg_partition_prefix_len = 1; + let op = ParquetMergeOperation::promote_legacy(vec![a, b], 1); + assert_eq!(op.target_prefix_len_override, Some(1)); } } diff --git a/quickwit/quickwit-parquet-engine/src/merge/schema.rs b/quickwit/quickwit-parquet-engine/src/merge/schema.rs index 68ebf851e59..085a673b463 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/schema.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/schema.rs @@ -300,14 +300,23 @@ fn align_batch_to_schema(batch: &RecordBatch, target_schema: &SchemaRef) -> Resu /// Normalize an Arrow data type for the internal union schema. /// /// All string-like types (Utf8, LargeUtf8, Dictionary(*, Utf8/LargeUtf8)) -/// are normalized to Utf8. This ensures `take` works uniformly across -/// concatenated inputs regardless of their original encoding. +/// are normalized to Utf8. All byte-array-like types (Binary, +/// LargeBinary, Dictionary(*, Binary/LargeBinary)) are normalized to +/// Binary. Parquet stores both string flavours under the same `BYTE_ARRAY` +/// physical type and both binary flavours likewise, so two inputs whose +/// schemas differ only by string/binary flavour represent the same +/// logical data; the union must accept them as one column. /// -/// Non-string types are returned as-is. +/// This ensures `take` works uniformly across concatenated inputs +/// regardless of their original encoding. Non-string/non-binary types +/// are returned as-is. fn normalize_type(dt: &DataType) -> DataType { if is_string_type(dt) { return DataType::Utf8; } + if is_byte_array_type(dt) { + return DataType::Binary; + } dt.clone() } @@ -319,3 +328,17 @@ fn is_string_type(dt: &DataType) -> bool { _ => false, } } + +/// Returns true if the data type represents raw byte arrays. +/// +/// Parquet has a single `BYTE_ARRAY` physical type; Binary and +/// LargeBinary (and dict-encoded variants) all map to it on the +/// wire. Schema evolution that changes one to the other across +/// ingester versions must merge cleanly. +fn is_byte_array_type(dt: &DataType) -> bool { + match dt { + DataType::Binary | DataType::LargeBinary => true, + DataType::Dictionary(_, value_type) => is_byte_array_type(value_type), + _ => false, + } +} diff --git a/quickwit/quickwit-parquet-engine/src/merge/streaming.rs b/quickwit/quickwit-parquet-engine/src/merge/streaming.rs index 431eb13a872..de4bcf33159 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/streaming.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/streaming.rs @@ -3934,6 +3934,337 @@ mod tests { } } + /// Build two batches with the SAME sort schema but **different body- + /// col schemas**, exercising the merger's schema-evolution paths: + /// + /// - `body_string`: `Utf8` in A, `Dict` in B (string-flavour variation — + /// `normalize_type` collapses to Utf8). + /// - `body_bytes`: `LargeBinary` in A, `Binary` in B (byte-array-flavour variation — F6 + /// normalizer extension collapses to Binary). + /// - `body_list`: `List` present in A, ABSENT from B (MC-4 column union — B's rows + /// appear as nulls in the merged output; outer becomes nullable in the union, exercising the + /// F13 nullable-outer list write path). + /// - `body_a_only`: `Int32` in A only (MC-4 column union — B's rows null). + /// - `body_b_only`: `Int64` in B only (MC-4 column union — A's rows null). + /// - `body_value`: `Float64` in both (common-typed control column). + /// + /// Each row is keyed by a unique `sorted_series` value, and the cell + /// values are derived from the key + column name so that comparison + /// is byte-stable. + fn make_mixed_schema_input_a(num_rows: usize, key_offset: u64) -> RecordBatch { + use arrow::array::{Int32Array, LargeBinaryArray, ListArray}; + use arrow::buffer::OffsetBuffer; + + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(vec![ + // sort cols + Field::new("metric_name", dict_type, false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("sorted_series", DataType::Binary, false), + // body cols in lex order. body_a_only is A-exclusive. + // body_list is non-nullable here (A always has lists) but + // becomes nullable in the union because B lacks the col. + Field::new("body_a_only", DataType::Int32, true), + Field::new("body_bytes", DataType::LargeBinary, true), + Field::new( + "body_list", + DataType::List(Arc::new(Field::new("item", DataType::Float64, false))), + false, + ), + Field::new("body_string", DataType::Utf8, true), + Field::new("body_value", DataType::Float64, true), + ])); + + let metric_keys: Vec = vec![0; num_rows]; + let metric_values = StringArray::from(vec!["cpu.usage"]); + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(metric_values), + ) + .expect("dict"), + ); + let timestamps: Vec = (0..num_rows as u64) + .map(|i| 1_700_000_000 + (num_rows as u64 - i)) + .collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let key_bytes: Vec> = (0..num_rows as u64) + .map(|i| (key_offset + i).to_be_bytes().to_vec()) + .collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from( + key_bytes.iter().map(|v| v.as_slice()).collect::>(), + )); + + let body_a_only: ArrayRef = Arc::new(Int32Array::from( + (0..num_rows).map(|i| i as i32 * 7 - 5).collect::>(), + )); + let body_bytes_vals: Vec> = (0..num_rows) + .map(|i| format!("a-bytes-{i:04}").into_bytes()) + .collect(); + let body_bytes: ArrayRef = Arc::new(LargeBinaryArray::from( + body_bytes_vals + .iter() + .map(|v| Some(v.as_slice())) + .collect::>(), + )); + let body_string_vals: Vec = + (0..num_rows).map(|i| format!("a-str-{i:04}")).collect(); + let body_string: ArrayRef = Arc::new(StringArray::from(body_string_vals)); + let body_value: ArrayRef = Arc::new(Float64Array::from( + (0..num_rows).map(|i| i as f64 + 0.5).collect::>(), + )); + + // List body col: row i has a list of length (i % 3) + 1. + let mut list_offsets: Vec = Vec::with_capacity(num_rows + 1); + let mut list_values: Vec = Vec::new(); + list_offsets.push(0); + for i in 0..num_rows { + for j in 0..((i % 3) + 1) { + list_values.push((i * 10 + j) as f64 + 0.25); + } + list_offsets.push(list_values.len() as i32); + } + let list_inner: ArrayRef = Arc::new(Float64Array::from(list_values)); + let body_list: ArrayRef = Arc::new(ListArray::new( + Arc::new(Field::new("item", DataType::Float64, false)), + OffsetBuffer::new(arrow::buffer::ScalarBuffer::from(list_offsets)), + list_inner, + None, + )); + + RecordBatch::try_new( + schema, + vec![ + metric_name, + timestamp_secs, + sorted_series, + body_a_only, + body_bytes, + body_list, + body_string, + body_value, + ], + ) + .expect("input A batch") + } + + fn make_mixed_schema_input_b(num_rows: usize, key_offset: u64) -> RecordBatch { + use arrow::array::{BinaryArray as BinArr, Int64Array as I64}; + + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(vec![ + // Same sort cols. + Field::new("metric_name", dict_type.clone(), false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("sorted_series", DataType::Binary, false), + // Body cols in lex order. body_string is Dict (flavor + // change from A's Utf8). body_bytes is Binary (flavor + // change from A's LargeBinary). No body_list. Adds + // body_b_only. + Field::new("body_b_only", DataType::Int64, true), + Field::new("body_bytes", DataType::Binary, true), + Field::new("body_string", dict_type.clone(), true), + Field::new("body_value", DataType::Float64, true), + ])); + + let metric_keys: Vec = vec![0; num_rows]; + let metric_values = StringArray::from(vec!["cpu.usage"]); + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(metric_values), + ) + .expect("dict"), + ); + let timestamps: Vec = (0..num_rows as u64) + .map(|i| 1_700_000_000 + (num_rows as u64 - i)) + .collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let key_bytes: Vec> = (0..num_rows as u64) + .map(|i| (key_offset + i).to_be_bytes().to_vec()) + .collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from( + key_bytes.iter().map(|v| v.as_slice()).collect::>(), + )); + + let body_b_only: ArrayRef = Arc::new(I64::from( + (0..num_rows) + .map(|i| (i as i64) * 1_000_003 + 17) + .collect::>(), + )); + let body_bytes_vals: Vec> = (0..num_rows) + .map(|i| format!("b-bytes-{i:04}").into_bytes()) + .collect(); + let body_bytes: ArrayRef = Arc::new(BinArr::from( + body_bytes_vals + .iter() + .map(|v| Some(v.as_slice())) + .collect::>(), + )); + + // Dict-encoded body_string. + let body_string_pool: Vec = + (0..num_rows).map(|i| format!("b-str-{i:04}")).collect(); + let body_string_keys: Vec = (0..num_rows as i32).collect(); + let body_string: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(body_string_keys), + Arc::new(StringArray::from(body_string_pool)), + ) + .expect("dict"), + ); + + let body_value: ArrayRef = Arc::new(Float64Array::from( + (0..num_rows) + .map(|i| (i as f64) * -1.25 + 7.0) + .collect::>(), + )); + + RecordBatch::try_new( + schema, + vec![ + metric_name, + timestamp_secs, + sorted_series, + body_b_only, + body_bytes, + body_string, + body_value, + ], + ) + .expect("input B batch") + } + + /// F6 — MC-2/MC-4 across heterogeneous body schemas. Inputs A and B + /// share the sort schema but differ in: + /// - body_string flavour (Utf8 vs Dict) + /// - body_bytes flavour (LargeBinary vs Binary) — relies on the F6 `normalize_type` extension + /// - body_list present in A, missing in B + /// - body_a_only / body_b_only present in only one input + /// + /// After merging, the output must contain the union of body cols + /// (MC-4) with B's rows null in body_list / body_a_only and A's + /// rows null in body_b_only. Shared-col values must survive (MC-2) + /// despite the flavour differences. + #[tokio::test] + async fn test_mc2_mixed_schemas_round_trip() { + let batch_a = make_mixed_schema_input_a(40, 0); + let batch_b = make_mixed_schema_input_b(40, 10_000); + let bytes_a = write_input_parquet(std::slice::from_ref(&batch_a), &[]); + let bytes_b = write_input_parquet(std::slice::from_ref(&batch_b), &[]); + + let inputs: Vec> = + vec![open_stream(bytes_a).await, open_stream(bytes_b).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) + .await + .expect("merge with mixed schemas"); + assert_eq!(outputs.len(), 1); + assert_eq!( + outputs[0].num_rows, 80, + "rows preserved across schema-evolved inputs (MC-1)", + ); + + let merged = read_output_to_record_batch(&outputs[0].path); + let merged_schema = merged.schema(); + + // MC-4: union of body-col names must appear in the output. The + // streaming engine drops all-null sort cols, but body cols + // present in any input must survive (even if half the rows + // are null). + for col_name in [ + "body_a_only", + "body_b_only", + "body_bytes", + "body_list", + "body_string", + "body_value", + ] { + merged_schema.index_of(col_name).unwrap_or_else(|_| { + panic!("merged output is missing body col '{col_name}' — MC-4 violated") + }); + } + + // MC-2: build (sorted_series → tuple) maps for each input and + // the output, then verify every input row's tuple matches the + // output row's tuple. `render_cell` normalizes the same + // string/byte flavour rendering so the type variations don't + // generate spurious diffs. + let series_idx_a = batch_a.schema().index_of("sorted_series").unwrap(); + let series_idx_b = batch_b.schema().index_of("sorted_series").unwrap(); + let body_col_names_union: Vec<&str> = vec![ + "body_a_only", + "body_b_only", + "body_bytes", + "body_list", + "body_string", + "body_value", + ]; + + let render_row = |batch: &RecordBatch, row: usize| -> String { + let mut tuple = String::new(); + for (n, &name) in body_col_names_union.iter().enumerate() { + if n > 0 { + tuple.push('|'); + } + match batch.schema().index_of(name) { + Ok(col_idx) => { + tuple.push_str(&render_cell(batch.column(col_idx).as_ref(), row)) + } + // Col absent from this input → null when seen + // in the merged output. + Err(_) => tuple.push_str(""), + } + } + tuple + }; + + let mut expected: HashMap, String> = HashMap::new(); + let series_col_a = batch_a + .column(series_idx_a) + .as_any() + .downcast_ref::() + .unwrap(); + for row in 0..batch_a.num_rows() { + let key = series_col_a.value(row).to_vec(); + expected.insert(key, render_row(&batch_a, row)); + } + let series_col_b = batch_b + .column(series_idx_b) + .as_any() + .downcast_ref::() + .unwrap(); + for row in 0..batch_b.num_rows() { + let key = series_col_b.value(row).to_vec(); + expected.insert(key, render_row(&batch_b, row)); + } + assert_eq!(expected.len(), 80); + + let series_idx_out = merged_schema.index_of("sorted_series").unwrap(); + let series_col_out = merged + .column(series_idx_out) + .as_any() + .downcast_ref::() + .unwrap(); + let mut observed: HashMap, String> = HashMap::with_capacity(merged.num_rows()); + for row in 0..merged.num_rows() { + let key = series_col_out.value(row).to_vec(); + observed.insert(key, render_row(&merged, row)); + } + assert_eq!(observed.len(), 80, "MC-1 — no rows lost"); + + for (key, want) in &expected { + let got = observed + .get(key) + .unwrap_or_else(|| panic!("missing key {key:?} in merged output")); + assert_eq!( + got, want, + "MC-2 mismatch for sorted_series {key:?}: got {got}, want {want}", + ); + } + } + // ============================================================================ // MS-7: page-cache bounded-memory contract. The streaming engine's // raison d'être is that body-col memory stays bounded by ~constant @@ -4212,4 +4543,172 @@ mod tests { "service-name set must cover every distinct value in the sort col", ); } + + // ============================================================================ + // (b) Legacy-promotion executor: an end-to-end test through + // `ParquetMergeOperation::promote_legacy` → + // `execute_merge_operation` → streaming engine output. + // ============================================================================ + + /// Build a minimal `ParquetSplitMetadata` for use by the promotion + /// executor. Only the routing-relevant fields are real — the rest + /// are placeholders sized to match across inputs so MP-3 holds. + fn make_promotion_split_meta( + split_id: &str, + rg_partition_prefix_len: u32, + ) -> crate::split::ParquetSplitMetadata { + crate::split::ParquetSplitMetadata::metrics_builder() + .split_id(crate::split::ParquetSplitId::new(split_id)) + .index_uid("test-index:00000000000000000000000001") + .partition_id(0) + .time_range(crate::split::TimeRange::new(1_700_000_000, 1_700_000_060)) + .sort_fields("metric_name|-timestamp_secs/V2") + .window_start_secs(1_700_000_000) + .window_duration_secs(60) + .num_merge_ops(0) + .rg_partition_prefix_len(rg_partition_prefix_len) + .num_rows(0) + .size_bytes(0) + .build() + } + + /// End-to-end legacy promotion: a prefix_len=0 legacy single-RG + /// input + a prefix_len=1 aligned multi-RG input, merged via + /// `execute_merge_operation` with `target_prefix_len = 1`. The + /// legacy input is routed through `LegacyInputAdapter`, the + /// aligned one goes direct. The streaming engine sees uniform + /// prefix_len=1 inputs and produces aligned multi-RG output that + /// passes `assert_unique_rg_prefix_keys`. + #[tokio::test] + async fn test_promote_legacy_executor_end_to_end() { + use crate::merge::execute_merge_operation; + use crate::merge::policy::ParquetMergeOperation; + use crate::storage::RemoteByteSource; + + // Legacy input: 3 metrics × 20 rows, prefix_len = 0. + let legacy_bytes = make_multi_metric_single_rg_input(&[ + ("cpu.usage", 20), + ("memory.used", 20), + ("net.bytes", 20), + ]); + let legacy_meta = make_promotion_split_meta("legacy_001", 0); + + // Aligned input: 2 metrics × 30 rows in 2 RGs, prefix_len = 1. + // `make_two_metric_aligned_input` uses metric names "cpu.usage" + // and "memory.used", overlapping with the legacy input — the + // merge engine must interleave by metric_name across inputs. + let aligned_bytes = make_two_metric_aligned_input(); + let aligned_meta = make_promotion_split_meta("aligned_001", 1); + + let op = ParquetMergeOperation::promote_legacy(vec![legacy_meta, aligned_meta], 1); + + let sources: Vec> = vec![ + Arc::new(InMemorySource { + bytes: legacy_bytes, + }), + Arc::new(InMemorySource { + bytes: aligned_bytes, + }), + ]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = execute_merge_operation(&op, sources, tmp.path(), &merge_config(1)) + .await + .expect("promote-legacy merge"); + + assert_eq!(outputs.len(), 1, "single output file"); + assert_eq!( + outputs[0].num_rows, + 60 + 60, + "legacy 3×20 + aligned 2×30 rows = 120 total", + ); + + // The streaming engine sees the legacy input as prefix_len=1 + // (adapter rewrote the KV), and the aligned input is already + // prefix_len=1. So the output should advertise prefix_len=1. + assert_eq!( + outputs[0].output_rg_partition_prefix_len, 1, + "executor must produce an output that carries the promoted prefix_len", + ); + + // Read the on-disk KV: must match. (CS-1 inside the file.) + let bytes_out = std::fs::read(&outputs[0].path).expect("read"); + let reader = SerializedFileReader::new(Bytes::from(bytes_out)).expect("ser"); + let prefix_kv = reader + .metadata() + .file_metadata() + .key_value_metadata() + .and_then(|kvs| { + kvs.iter() + .find(|k| k.key == PARQUET_META_RG_PARTITION_PREFIX_LEN) + .and_then(|k| k.value.clone()) + }); + assert_eq!( + prefix_kv.as_deref(), + Some("1"), + "on-disk KV must declare prefix_len = 1 (target)", + ); + + // Output should have one RG per distinct metric_name. The + // union of legacy + aligned metric names is {cpu.usage, + // memory.used, net.bytes} → 3 RGs. + assert_eq!( + reader.metadata().num_row_groups(), + 3, + "output should have one RG per distinct metric_name across inputs", + ); + + // PA-1 + PA-3: each output RG carries a unique, constant + // metric_name. This is the strong chunk-stats check that + // motivated F2. + assert_unique_rg_prefix_keys( + reader.metadata(), + "metric_name|-timestamp_secs/V2", + 1, + "test_promote_legacy_executor_end_to_end output", + ) + .expect("promoted output must satisfy PA-1 + PA-3"); + + // Metastore record (CS-1): builds successfully with + // mixed_prefix_ok = true, and `rg_partition_prefix_len` + // matches the on-disk KV. + let metastore_meta = crate::merge::metadata_aggregation::merge_parquet_split_metadata( + op.splits_as_slice(), + &outputs[0], + /* mixed_prefix_ok */ true, + ) + .expect("metastore aggregation accepts mixed-prefix inputs in promotion mode"); + assert_eq!( + metastore_meta.rg_partition_prefix_len, 1, + "metastore prefix_len must match the on-disk KV (CS-1)", + ); + } + + /// Negative: if the executor is given more sources than splits, or + /// fewer, it bails up-front rather than producing a partial merge. + #[tokio::test] + async fn test_executor_mismatched_sources_count_bails() { + use crate::merge::execute_merge_operation; + use crate::merge::policy::ParquetMergeOperation; + use crate::storage::RemoteByteSource; + + let legacy_meta = make_promotion_split_meta("a", 0); + let aligned_meta = make_promotion_split_meta("b", 1); + let op = ParquetMergeOperation::promote_legacy(vec![legacy_meta, aligned_meta], 1); + + // One source for two splits. + let sources: Vec> = vec![Arc::new(InMemorySource { + bytes: make_multi_metric_single_rg_input(&[("cpu.usage", 10)]), + })]; + + let tmp = TempDir::new().expect("tmpdir"); + let err = execute_merge_operation(&op, sources, tmp.path(), &merge_config(1)) + .await + .expect_err("mismatched source/split count must fail"); + let msg = err.to_string(); + assert!( + msg.contains("sources.len()") && msg.contains("op.splits.len()"), + "error should explain the mismatch, got: {msg}", + ); + } } diff --git a/quickwit/quickwit-parquet-engine/src/storage/streaming_writer.rs b/quickwit/quickwit-parquet-engine/src/storage/streaming_writer.rs index 3b83ff1d332..df903cacee2 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/streaming_writer.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/streaming_writer.rs @@ -580,14 +580,15 @@ fn write_array_via_serialized_column_writer( &materialized_ref, ); } - // `List` / `LargeList` with non-nullable outer + inner. - // The DDSketch `keys` (`List`) and `counts` - // (`List`) columns are this shape. We compute Dremel - // def/rep levels from each input array and write them through - // the same `SerializedColumnWriter::write_batch` call the flat + // `List` / `LargeList` with non-nullable inner. The outer + // may be either nullable (schema-evolution case where the col + // is present in only some inputs) or non-nullable (e.g. + // DDSketch `keys` / `counts`). We compute Dremel def/rep + // levels from each input array and write them through the + // same `SerializedColumnWriter::write_batch` call the flat // path uses, so memory stays bounded by one in-flight page. DataType::List(_) | DataType::LargeList(_) => { - write_non_nullable_list_via_serialized_column_writer(col_writer, field, array)?; + write_list_via_serialized_column_writer(col_writer, field, array)?; } // Multi-leaf nested (Struct, Map) and other unsupported types. // Single-leaf multi-child Structs are rejected at @@ -607,37 +608,24 @@ fn write_array_via_serialized_column_writer( Ok(()) } -/// Page-bounded write for `List` / `LargeList` where the outer -/// field is non-nullable and the inner field is non-nullable. Computes -/// Dremel def/rep levels (max_def = 1, max_rep = 1) and dispatches the -/// flat inner values through the same typed `write_batch` call the flat -/// arms use. Pages flush as the writer's -/// `data_page_size_limit` / `data_page_row_count_limit` thresholds are -/// reached — same memory-bound contract as the flat path. -fn write_non_nullable_list_via_serialized_column_writer( - col_writer: &mut parquet::file::writer::SerializedColumnWriter<'_>, - field: &arrow::datatypes::Field, - array: &arrow::array::ArrayRef, -) -> Result<(), ParquetWriteError> { - use arrow::array::{Array, LargeListArray, ListArray}; - use arrow::datatypes::DataType; - - if field.is_nullable() { - return Err(ParquetWriteError::SchemaValidation(format!( - "field '{}' is a nullable List; only non-nullable List is supported on the streaming \ - write path", - field.name(), - ))); - } - - // Resolve inner field + values + per-row offsets uniformly across - // List and LargeList. Offsets coerce to i64 so a single - // function body handles both representations. - let (inner_field, inner_values, offsets): ( - &arrow::datatypes::Field, - &arrow::array::ArrayRef, +/// Resolve a `ListArray` or `LargeListArray` into a unified +/// `(inner_field, inner_values, offsets)` triple. Offsets always coerce +/// to `i64` so the caller doesn't need to branch on `List` vs +/// `LargeList`. +fn resolve_list_components<'a>( + field: &'a arrow::datatypes::Field, + array: &'a arrow::array::ArrayRef, +) -> Result< + ( + &'a arrow::datatypes::Field, + &'a arrow::array::ArrayRef, Vec, - ) = match field.data_type() { + ), + ParquetWriteError, +> { + use arrow::array::{LargeListArray, ListArray}; + use arrow::datatypes::DataType; + match field.data_type() { DataType::List(inner_field_ref) => { let arr = array.as_any().downcast_ref::().ok_or_else(|| { ParquetWriteError::SchemaValidation(format!( @@ -646,7 +634,7 @@ fn write_non_nullable_list_via_serialized_column_writer( )) })?; let offsets: Vec = arr.value_offsets().iter().map(|&o| o as i64).collect(); - (inner_field_ref.as_ref(), arr.values(), offsets) + Ok((inner_field_ref.as_ref(), arr.values(), offsets)) } DataType::LargeList(inner_field_ref) => { let arr = array @@ -659,14 +647,42 @@ fn write_non_nullable_list_via_serialized_column_writer( )) })?; let offsets: Vec = arr.value_offsets().to_vec(); - (inner_field_ref.as_ref(), arr.values(), offsets) - } - other => { - return Err(ParquetWriteError::SchemaValidation(format!( - "internal: write_non_nullable_list called with non-list type {other:?}", - ))); + Ok((inner_field_ref.as_ref(), arr.values(), offsets)) } - }; + other => Err(ParquetWriteError::SchemaValidation(format!( + "internal: resolve_list_components called with non-list type {other:?}", + ))), + } +} + +/// Page-bounded write for `List` / `LargeList` with non-nullable +/// inner element. Handles both nullable and non-nullable outer fields: +/// +/// - **Non-nullable outer** (e.g. DDSketch `keys` / `counts`): max_def = 1, max_rep = 1. +/// - Empty list at row → def=0, rep=0, no value. +/// - N-element list at row → N×(def=1, rep=[0,1,1,…]) plus N values. +/// - **Nullable outer** (schema-evolution case where the col is missing from some inputs): max_def +/// = 2, max_rep = 1. +/// - Outer is null at row → def=0, rep=0, no value. +/// - Empty list at row → def=1, rep=0, no value. +/// - N-element list at row → N×(def=2, rep=[0,1,1,…]) plus N values. +/// +/// Pages flush as the writer's `data_page_size_limit` / +/// `data_page_row_count_limit` thresholds are reached — same +/// memory-bound contract as the flat path. +fn write_list_via_serialized_column_writer( + col_writer: &mut parquet::file::writer::SerializedColumnWriter<'_>, + field: &arrow::datatypes::Field, + array: &arrow::array::ArrayRef, +) -> Result<(), ParquetWriteError> { + use arrow::array::Array; + + let outer_nullable = field.is_nullable(); + + // Resolve inner field + values + per-row offsets uniformly across + // List and LargeList. Offsets coerce to i64 so a single + // function body handles both representations. + let (inner_field, inner_values, offsets) = resolve_list_components(field, array)?; if inner_field.is_nullable() { return Err(ParquetWriteError::SchemaValidation(format!( @@ -676,49 +692,53 @@ fn write_non_nullable_list_via_serialized_column_writer( ))); } - // Walk per-row to build Dremel levels. - // - // Path: required outer group → repeated `list` → required `element`. - // - max_rep_level = 1 (only `list` is repeated). - // - max_def_level = 1 (the repeated `list` group can occur 0 times, which is how parquet - // encodes an empty list; 1 marks "element present"). - // - // Per row: - // - empty list: emit one slot with def = 0, rep = 0, no value - // - list of N elements: emit N slots, def = 1 each, rep = 0 for the first and rep = 1 for the - // rest, plus N values. + let empty_list_def: i16 = if outer_nullable { 1 } else { 0 }; + let element_present_def: i16 = if outer_nullable { 2 } else { 1 }; + let num_rows = array.len(); - let total_present: usize = (0..num_rows) - .map(|row| (offsets[row + 1] - offsets[row]).max(0) as usize) - .sum(); - // Each row contributes either 1 level (empty) or list_len levels. - let total_levels = (0..num_rows) - .map(|row| { - let len = (offsets[row + 1] - offsets[row]).max(0) as usize; - if len == 0 { 1 } else { len } - }) - .sum::(); - let mut def_levels: Vec = Vec::with_capacity(total_levels); - let mut rep_levels: Vec = Vec::with_capacity(total_levels); + // A null outer at row R contributes one def=0 level and no value. + // The inner-values gather (in `write_list_inner_values`) skips + // null rows entirely via this mask. + let null_rows: Option> = if outer_nullable { + Some((0..num_rows).map(|row| array.is_null(row)).collect()) + } else { + None + }; + + let mut def_levels: Vec = Vec::new(); + let mut rep_levels: Vec = Vec::new(); + let mut total_present: usize = 0; for row in 0..num_rows { - let start = offsets[row]; - let end = offsets[row + 1]; - let len = (end - start).max(0) as usize; - if len == 0 { + let is_null = match null_rows.as_ref() { + Some(n) => n[row], + None => false, + }; + if is_null { + // Null outer: one def=0 level, no value. def_levels.push(0); rep_levels.push(0); + continue; + } + let len = (offsets[row + 1] - offsets[row]).max(0) as usize; + if len == 0 { + def_levels.push(empty_list_def); + rep_levels.push(0); } else { for i in 0..len { - def_levels.push(1); + def_levels.push(element_present_def); rep_levels.push(if i == 0 { 0 } else { 1 }); } + total_present += len; } } - // Dispatch the inner primitive through the appropriate typed - // writer. Indexing iterates only the present (non-empty-list) rows - // — start..end ranges, walked once for the whole array — so we - // emit exactly `total_present` values. + // The inner-values dispatcher walks `start..end` per row and + // gathers into a contiguous vec. To do that against the original + // inner_values (which is shared across all rows including the + // null ones), we hand it the ORIGINAL per-row offsets plus a + // `null_rows` mask. Non-null rows contribute their full range; + // null rows are skipped. The dispatcher emits exactly + // `total_present` values. write_list_inner_values( col_writer, field, @@ -728,6 +748,7 @@ fn write_non_nullable_list_via_serialized_column_writer( total_present, &def_levels, &rep_levels, + null_rows.as_deref(), ) } @@ -746,6 +767,11 @@ fn write_list_inner_values( total_present: usize, def_levels: &[i16], rep_levels: &[i16], + // Present for nullable-outer lists: `null_rows[row]` is true when + // row is null on the outer (no inner values to emit). Indexed by + // the same row range as `offsets`. None means "no nullable outer" + // (all rows are present), so the per-row check is skipped. + null_rows: Option<&[bool]>, ) -> Result<(), ParquetWriteError> { use arrow::array::{ Float32Array, Float64Array, Int8Array, Int16Array, Int32Array, Int64Array, UInt8Array, @@ -756,9 +782,19 @@ fn write_list_inner_values( // Walk the per-row [start, end) ranges once and gather the // present-only values into a contiguous Vec for `write_batch`. + // Null outer rows are skipped — their inner range is not emitted. + let is_null_row = |row: usize| -> bool { + match null_rows { + Some(n) => n[row], + None => false, + } + }; let collect_i32 = |extract: &dyn Fn(usize) -> i32| -> Vec { let mut out = Vec::with_capacity(total_present); for row in 0..(offsets.len() - 1) { + if is_null_row(row) { + continue; + } let start = offsets[row].max(0) as usize; let end = offsets[row + 1].max(0) as usize; for i in start..end { @@ -770,6 +806,9 @@ fn write_list_inner_values( let collect_i64 = |extract: &dyn Fn(usize) -> i64| -> Vec { let mut out = Vec::with_capacity(total_present); for row in 0..(offsets.len() - 1) { + if is_null_row(row) { + continue; + } let start = offsets[row].max(0) as usize; let end = offsets[row + 1].max(0) as usize; for i in start..end { @@ -861,6 +900,9 @@ fn write_list_inner_values( .unwrap(); let mut values = Vec::with_capacity(total_present); for row in 0..(offsets.len() - 1) { + if is_null_row(row) { + continue; + } let start = offsets[row].max(0) as usize; let end = offsets[row + 1].max(0) as usize; for i in start..end { @@ -880,6 +922,9 @@ fn write_list_inner_values( .unwrap(); let mut values = Vec::with_capacity(total_present); for row in 0..(offsets.len() - 1) { + if is_null_row(row) { + continue; + } let start = offsets[row].max(0) as usize; let end = offsets[row + 1].max(0) as usize; for i in start..end { From 24ec1af4e243a5c2492e0be334400c2a6b7fa29b Mon Sep 17 00:00:00 2001 From: George Talbot Date: Tue, 19 May 2026 04:41:41 -0400 Subject: [PATCH 09/10] feat(merge): adversarial-review test coverage (F4/F5/F7) + F14 sub-region engine fix (#6428) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * feat(merge): close F4/F5/F7/F14 from the adversarial review Three test additions plus one engine fix surfaced by the F4 tests. The existing MS-7 test proved the per-input page-cache bound for one input and one region. F4 extends the coverage: - `test_ms7_per_input_bound_across_num_inputs` sweeps `num_inputs ∈ {1, 3, 8}` × `rows_per_input ∈ {3 000, 30 000}` and asserts the per-input peak stays bounded. Cross-axis growth check: going from 1 input to 8 must not push the peak up. - `test_ms7_per_input_bound_across_sub_regions_does_not_scale_with_rows` runs the prefix_len=0 multi-output sub-region path at 3 000 vs 30 000 rows and asserts peak doesn't scale with input row count. **This test surfaced F14 (below) — without the engine fix, the sub-region path's peak grew ~9× when rows grew 10×.** Tests serialize via `ms7_serial_lock` because `PEAK_BODY_COL_PAGE_CACHE_LEN` is process-global; concurrent tests would pollute each other's readings. Parquet streams emit pages in column-major order (all of col 0, then all of col 1, ...). The old sub-region-outer / col-inner ordering meant that while processing sub-region 0's col K, the stream emitted cols 0..K-1's remaining pages first to reach col K — those skipped pages got cached under their own col_idx for later sub-regions to consume, and the cache scaled with input row count. Fix: new `process_split_region_col_outer` function for the `needs_split` path. Cols iterate in the outer loop, sub-regions in the inner. Each parquet col chunk is fully consumed from the stream across all sub-regions before col K+1 starts. Cache for col K is empty before col K+1's pages arrive. Mechanics: pre-determine writer assignments for the region's sub-regions (a top-level region's sub-regions may span multiple output writers; consecutive sub-regions on the same writer get coalesced into one combined Region so each writer holds one RG concurrently — RGs on the same writer are sequential, so coalescing keeps the parquet writer's single-active-RG constraint intact). Single-region path stays on the existing `process_region`. `prop_merge_prefix_aligned_streaming` sweeps `(num_inputs ∈ 1..=3, per-input RG specs, num_outputs ∈ 1..=3)` with prefix_len=1 and asserts MC-1 (rows preserved), MC-3 (sorted_series monotone within each output), MS-3 (num_row_groups matches footer), PA-1+PA-3 (`assert_unique_rg_prefix_keys`), and CS-1 (metastore prefix_len == KV) on every generated case. 32 cases capped to keep runtime under a second. Fixture: `make_prefix_len_one_input` writes one RG per `(metric_name, rows)` entry by calling `writer.flush()` between batches. `sorted_series` encodes `metric_base + row_offset_within_metric`, mirroring production's storekey property that different metric_names produce non-overlapping `sorted_series` byte ranges. Plus a focused unit test `test_f5_single_input_two_metrics_minimal` that pins one specific case for fast iteration. `test_f7_production_shape_multi_input_multi_rg_multi_output`: 5 inputs × ~4 prefix-aligned RGs each × 4 outputs × prefix_len=1. Asserts the full invariant bundle (MC-1, MS-3, PA-1+PA-3, MS-5 cross-output sorted_series monotonicity, CS-1) — the corner the adversarial review flagged as "untested production case". MS-5 is "across adjacent outputs, sorted_series is monotone non-decreasing." A single metric CAN span outputs (the engine splits at sorted_series transitions inside an overflowing region), so the cross-output invariant is sorted_series monotonicity, not "each metric in one output." - `cargo test -p quickwit-parquet-engine --lib` — 498 unit tests pass. - `cargo clippy -p quickwit-parquet-engine --tests --all-features` with `-Dwarnings`. - `cargo doc --no-deps -p quickwit-parquet-engine` warning-free. - `cargo fmt --all -- --check` (nightly via PATH override). Co-Authored-By: Claude Opus 4.7 (1M context) * style(streaming): drop stray blank line before tests section header Newer nightly rustfmt (2026-05-17) flags the extra blank line that crept into the test module between the F4 fixture helper and the "Heterogeneous-output regressions" section header. Single-line gap is what nightly fmt wants. No behaviour change. Co-Authored-By: Claude Opus 4.7 (1M context) * fix(streaming): roll over chunk-assignment before first chunk after split Codex P1 on PR #6428: the previous "Recompute split budget after rolling over" fix (commit 56e773f9f, #6424) handled the split *decision* but not the split *assignment*. When the previous region fills the current output exactly and the next region enters the `needs_split` path, the chunk-assignment loop in `process_split_region_col_outer`'s setup initializes from the stale `current_output_idx` / `current_output_rows`. Its inner `needs_new_writer` check guards on `!chunk_assignments.is_empty()`, so the first iteration cannot roll over: the first sub-region is appended to the already-full output and only the second one advances. Output K ends up at 2× target while subsequent outputs are short or empty. Fix: initialize `active_output_idx` / `active_rows` from the `will_roll_over` case before the loop. The inner `needs_new_writer` check then works for both the first and subsequent iterations (on the first iteration `active_rows = 0 < target` so it correctly doesn't re-roll). The `can_reuse_current` check in the writer- materialization loop already handles "first chunk's output_idx doesn't match current_writer" by finalizing the current output (which is correct: it's full, close it) and opening a fresh writer at the next index. Regression test `test_split_chunk_assignment_rolls_over_before_first_chunk`: prefix_len=1, two metrics of 200 + 400 rows = 600 total, `num_outputs = 3` → `target_per_output = 200`. Region A fills output 0 exactly; region B needs splitting. Pre-fix the merge produced 2 outputs of 400 + 200 (output 0 overfilled, output 2 empty); post-fix it produces 3 outputs of ~200 rows each. 502 lib tests pass (+1); workspace clippy + nightly fmt clean. Co-Authored-By: Claude Opus 4.7 (1M context) --------- Co-authored-by: Claude Opus 4.7 (1M context) --- .../src/merge/streaming.rs | 1448 ++++++++++++++++- 1 file changed, 1432 insertions(+), 16 deletions(-) diff --git a/quickwit/quickwit-parquet-engine/src/merge/streaming.rs b/quickwit/quickwit-parquet-engine/src/merge/streaming.rs index de4bcf33159..492bc03f166 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/streaming.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/streaming.rs @@ -350,14 +350,200 @@ pub async fn streaming_merge_sorted_parquet_files( vec![region.clone()] }; - for sub_region in &sub_regions { - let sub_rows = sub_region.total_rows(); - - // Advance to the next output file BEFORE this sub- - // region if the current output already met the - // target and we still have outputs to fill. The - // very first sub-region is exempt so we don't open - // an empty writer. + if needs_split { + // Multi-output sub-region path: hoist the col loop OUT + // of the sub-region loop via `process_split_region_col_outer` + // so each parquet col chunk is fully consumed across + // all sub-regions before the next col starts (F14). + // + // Group consecutive sub-regions that share an output + // writer into one "writer chunk" — each chunk writes + // one RG containing all its sub-regions' rows. The + // group boundaries are where the running row count + // crosses `target_per_output` and there's still + // budget for another output. + let prefetched_batches = prefetched + .as_deref() + .expect("needs_split path always drains prefetched sort batches"); + + // Build writer chunks: pre-decide writer transitions + // for this region's sub-regions and coalesce + // consecutive sub-regions assigned to the same + // writer into one Region. Each writer-chunk needs a + // distinct OutputWriterStorage (one RG per chunk; + // RGs must be sequential per writer so two + // sub-regions on the same writer would need + // sequential RGs — but we coalesce them into one + // RG to keep the col-outer loop simple). + let mut chunk_assignments: Vec<(usize, Vec<&Region>)> = Vec::new(); + // Pre-rollover initialization. If `will_roll_over` is set + // (the previous region already filled `current_output_idx` + // and we still have outputs to fill), the first chunk must + // land in the NEXT output — not append to the already-full + // one. The inner `needs_new_writer` check below guards on + // `!chunk_assignments.is_empty()`, so it only fires from + // the second iteration on; the first chunk's destination + // has to be decided here. + // + // Companion fix to the `effective_first_target` / + // `effective_outputs_remaining` rollover handling above: + // the split *decision* uses the rolled-over output's + // budget, and the chunk *assignment* must too. Otherwise + // the first sub-region would be glued onto the already- + // full output, overfilling it by up to `target_per_output` + // rows and shrinking the final output count. + let (mut active_output_idx, mut active_rows) = if will_roll_over { + (current_output_idx + 1, 0) + } else { + (current_output_idx, current_output_rows) + }; + for sub_region in &sub_regions { + let sub_rows = sub_region.total_rows(); + let needs_new_writer = !chunk_assignments.is_empty() + && active_rows >= target_per_output + && active_output_idx + 1 < num_outputs; + if needs_new_writer { + active_output_idx += 1; + active_rows = 0; + } + let same_writer_as_last = match chunk_assignments.last() { + Some((o, _)) => *o == active_output_idx, + None => false, + }; + if same_writer_as_last { + chunk_assignments + .last_mut() + .expect("non-empty checked above") + .1 + .push(sub_region); + } else { + chunk_assignments.push((active_output_idx, vec![sub_region])); + } + active_rows += sub_rows; + } + + // Open writers for any new output indices in this + // region's chunk assignments. The first chunk reuses + // the existing current_writer if it matches. + // Materialize concrete writer/accumulator storage in + // a Vec so we can take &mut to each. + struct ChunkStorage { + writer: OutputWriterStorage, + accumulator: OutputAccumulator, + region: Region, + } + let mut chunk_storage: Vec = + Vec::with_capacity(chunk_assignments.len()); + for (idx_in_region, (output_idx, sub_regions_for_chunk)) in + chunk_assignments.iter().enumerate() + { + // First chunk: try to reuse current_writer / current_accumulator + // if they match the output_idx; otherwise finalize them and open new. + let can_reuse_current = idx_in_region == 0 + && match current_writer.as_ref() { + Some(w) => w.output_idx == *output_idx, + None => false, + }; + let (writer, accumulator) = if can_reuse_current { + ( + current_writer.take().expect("matched above"), + current_accumulator.take().expect("matched above"), + ) + } else { + // Need to advance: finalize current if it's + // not for this output, then open a fresh + // writer at output_idx. + if let (Some(w), Some(acc)) = + (current_writer.take(), current_accumulator.take()) + { + outputs.push(finalize_output(w, acc, &input_meta)?); + } + let writer = open_output_writer_for_streaming( + *output_idx, + &output_dir, + &union_schema, + &input_meta, + &writer_config, + )?; + (writer, OutputAccumulator::new(*output_idx)) + }; + + // Coalesce sub-regions into one Region by summing + // contributions per (input_idx, rg_idx). + let mut by_input: std::collections::BTreeMap<(usize, usize), (usize, usize)> = + std::collections::BTreeMap::new(); + for sr in sub_regions_for_chunk { + for c in &sr.contributing { + by_input + .entry((c.input_idx, c.rg_idx)) + .and_modify(|(min_start, total)| { + *min_start = (*min_start).min(c.start_row); + *total += c.num_rows; + }) + .or_insert((c.start_row, c.num_rows)); + } + } + let contributing: Vec = by_input + .into_iter() + .map(|((input_idx, rg_idx), (start_row, num_rows))| { + region_grouping::RegionContribution { + input_idx, + rg_idx, + start_row, + num_rows, + } + }) + .collect(); + let combined = Region { + prefix_key: sub_regions_for_chunk[0].prefix_key.clone(), + contributing, + }; + chunk_storage.push(ChunkStorage { + writer, + accumulator, + region: combined, + }); + } + + // Build &mut chunks for the col-outer processor. + let chunks: Vec> = chunk_storage + .iter_mut() + .map(|s| SplitRegionChunk { + writer_state: &mut s.writer, + accumulator: &mut s.accumulator, + region: s.region.clone(), + }) + .collect(); + process_split_region_col_outer( + &handle, + &mut decoders_state, + chunks, + &union_schema, + &input_meta, + prefetched_batches, + )?; + + // Hand chunk storage back to main-loop state: the + // last chunk becomes the new current writer; all + // earlier chunks finalize now. + let last_chunk_storage = chunk_storage + .pop() + .expect("chunk_assignments is non-empty so storage is non-empty"); + for finished in chunk_storage { + outputs.push(finalize_output( + finished.writer, + finished.accumulator, + &input_meta, + )?); + } + current_writer = Some(last_chunk_storage.writer); + current_accumulator = Some(last_chunk_storage.accumulator); + current_output_idx = active_output_idx; + current_output_rows = active_rows; + } else { + // No split needed: ensure a writer is open at the + // current output index, then process the single + // region via the existing per-region path. if current_writer.is_some() && current_output_rows >= target_per_output && current_output_idx + 1 < num_outputs @@ -388,16 +574,16 @@ pub async fn streaming_merge_sorted_parquet_files( &mut decoders_state, current_writer .as_mut() - .expect("writer opened above for this sub-region"), + .expect("writer opened above for this region"), current_accumulator .as_mut() - .expect("accumulator opened above for this sub-region"), - sub_region, + .expect("accumulator opened above for this region"), + region, &union_schema, &input_meta, prefetched.as_deref(), )?; - current_output_rows += sub_rows; + current_output_rows += region.total_rows(); } } @@ -1012,6 +1198,286 @@ fn process_region( Ok(()) } +/// One writer-chunk handed to [`process_split_region_col_outer`]. +/// +/// `region` is the combined region for this chunk: one or more +/// adjacent sub-regions of a single top-level region that share an +/// output writer. Coalescing adjacent sub-regions into one Region +/// lets each chunk's RG cover all the rows that this writer-chunk +/// owns in one shot, so we open exactly one RG per chunk and the +/// col-outer body-col loop can interleave writes across N chunks +/// holding N concurrent RGs (one per writer). +pub(crate) struct SplitRegionChunk<'a> { + pub(crate) writer_state: &'a mut OutputWriterStorage, + pub(crate) accumulator: &'a mut OutputAccumulator, + pub(crate) region: Region, +} + +/// Per-chunk pre-computed state built up in phase 1 of +/// [`process_split_region_col_outer`] before any body cols are +/// written. Persisted across the col-outer loop so each iteration of +/// the inner loop has the destinations / cursors / row counts it +/// needs without re-computing. +struct ChunkPrep { + sort_union_schema: SchemaRef, + region_sort_batch: RecordBatch, + region_destinations: InputRowDestinations, + region_rows: usize, + input_start_rows: Vec, +} + +/// Col-outer processor for the prefix_len=0 multi-output sub-region +/// case. Same per-region work as [`process_region`] (merge order, +/// permutation, sort cols, body cols), but with the col loop hoisted +/// OUT of the sub-region loop so each parquet col chunk is fully +/// consumed from the page stream before the next col starts. +/// +/// Why: the parquet stream emits pages in column-major order. The +/// previous sub-region-outer / col-inner ordering meant that while +/// processing sub-region 0's col K, the stream emitted cols 0..K-1's +/// remaining pages first (they had to be skipped past to reach col +/// K). Those skipped pages were cached under their own col_idx for +/// later sub-regions to consume — and the cache scaled with input +/// row count (more rows = more leftover pages per col chunk). +/// Col-outer fully drains col K's chunk across all sub-regions +/// before reading any col K+1 pages, so cache[col K] empties before +/// col K+1's pages start arriving. Peak total cache returns to +/// O(K × num_body_cols × num_chunks) where K is the per-col output- +/// page bound — bounded independent of row count. See the MS-7 +/// sub-region test for the regression guard. +/// +/// Each chunk holds one RG concurrently; chunks correspond to +/// distinct output writers, so the parquet writers' single-active-RG +/// constraint is respected. Sub-regions belonging to the same writer +/// are pre-coalesced into one Region by the caller. +fn process_split_region_col_outer( + handle: &Handle, + decoders_state: &mut [InputDecoderState], + chunks: Vec>, + union_schema: &SchemaRef, + input_meta: &InputMetadata, + prefetched_sort_batches: &[RecordBatch], +) -> Result<()> { + if chunks.is_empty() { + return Ok(()); + } + let num_inputs = decoders_state.len(); + + // Phase 1: per-chunk prep — sort batches, merge order, permutation, + // destinations. No I/O against the body-col stream yet. + let mut preps: Vec = Vec::with_capacity(chunks.len()); + for chunk in &chunks { + let mut sort_col_batches: Vec> = + (0..num_inputs).map(|_| None).collect(); + for c in &chunk.region.contributing { + // Slice the prefetched (already-drained for the whole top- + // level region) sort batch by this chunk's row range for + // each contributing input. + sort_col_batches[c.input_idx] = + Some(prefetched_sort_batches[c.input_idx].slice(c.start_row, c.num_rows)); + } + let mut sort_batch_vec: Vec = Vec::with_capacity(num_inputs); + for (idx, slot) in sort_col_batches.into_iter().enumerate() { + let batch = match slot { + Some(b) => b, + None => empty_sort_col_record_batch(&decoders_state[idx], &input_meta.sort_fields)?, + }; + sort_batch_vec.push(batch); + } + + let mut input_start_rows: Vec = vec![0; num_inputs]; + for c in &chunk.region.contributing { + input_start_rows[c.input_idx] = c.start_row; + } + + let (sort_union_schema, aligned_sort_batches) = + align_inputs_to_union_schema(&sort_batch_vec, &input_meta.sort_fields)?; + let merge_order = compute_merge_order(&aligned_sort_batches, &input_meta.sort_fields)?; + let region_rows: usize = merge_order.iter().map(|r| r.row_count).sum(); + let region_sort_batch = + apply_merge_permutation(&aligned_sort_batches, &sort_union_schema, &merge_order) + .context("applying merge permutation for split-region chunk")?; + verify_sort_order(®ion_sort_batch, &input_meta.sort_fields); + + let mut destinations: Vec>> = aligned_sort_batches + .iter() + .enumerate() + .map(|(idx, b)| vec![None; input_start_rows[idx] + b.num_rows()]) + .collect(); + let mut pos = 0usize; + for run in &merge_order { + for r in 0..run.row_count { + let absolute_row = input_start_rows[run.input_index] + run.start_row + r; + destinations[run.input_index][absolute_row] = Some((0, pos)); + pos += 1; + } + } + let region_destinations = InputRowDestinations { + per_input: destinations, + rows_per_output: vec![region_rows], + }; + + preps.push(ChunkPrep { + sort_union_schema, + region_sort_batch, + region_destinations, + region_rows, + input_start_rows, + }); + } + + // Phase 2: open one RG per chunk and bundle the per-chunk + // resources (rg + accumulator + num_row_groups ref + prep) into a + // single Vec. Consuming `chunks` in one pass avoids the + // borrow-checker conflict of holding rg-borrows of chunks while + // re-borrowing chunks for the body col loop. + struct ActiveChunk<'a> { + rg: crate::storage::streaming_writer::RowGroupBuilder<'a, std::fs::File>, + accumulator: &'a mut OutputAccumulator, + num_row_groups: &'a mut usize, + output_idx: usize, + prep: ChunkPrep, + } + + let mut actives: Vec> = Vec::with_capacity(chunks.len()); + for (chunk, prep) in chunks.into_iter().zip(preps) { + let SplitRegionChunk { + writer_state, + accumulator, + region: _, + } = chunk; + let output_idx = writer_state.output_idx; + // Split writer_state into the writer field (consumed by rg) + // and the num_row_groups field (held separately): disjoint + // field borrows through a &mut OutputWriterStorage are + // allowed. + let OutputWriterStorage { + writer, + num_row_groups, + .. + } = writer_state; + let rg = writer.start_row_group().with_context(|| { + format!("opening row group for split-region chunk output {output_idx}") + })?; + actives.push(ActiveChunk { + rg, + accumulator, + num_row_groups, + output_idx, + prep, + }); + } + + // Phase 3: write all columns in union-schema order. For each col, + // every chunk writes that col to its RG before any chunk moves + // on to the next col. Sort cols write from each chunk's + // already-sorted region_sort_batch (no stream involvement). + // Body cols stream through the per-input page cache; col-outer + // ordering ensures col K's chunk pages are fully consumed before + // col K+1's pages enter the stream. + let sort_field_names: Vec = actives[0] + .prep + .sort_union_schema + .fields() + .iter() + .map(|f| f.name().clone()) + .collect(); + for (col_idx, field) in union_schema.fields().iter().enumerate() { + let col_name = field.name(); + let is_sort_col = sort_field_names.iter().any(|n| n == col_name); + + if is_sort_col { + for (chunk_idx, active) in actives.iter_mut().enumerate() { + let arrays = build_sort_col_pages_from_sorted_batch( + &active.prep.region_sort_batch, + col_name, + )?; + active + .rg + .write_next_column_arrays(arrays) + .with_context(|| { + format!( + "writing sort col '{col_name}' (col_idx {col_idx}) to split-region \ + chunk {chunk_idx}", + ) + })?; + } + } else { + let mut input_col_indices: Vec> = Vec::with_capacity(num_inputs); + for state in decoders_state.iter() { + input_col_indices.push(state.arrow_schema.index_of(col_name).ok()); + } + let track_service = col_name == "service"; + + for (chunk_idx, active) in actives.iter_mut().enumerate() { + for (idx, state) in decoders_state.iter_mut().enumerate() { + if let Some(col_parquet_idx) = input_col_indices[idx] { + state.set_body_col_cursor( + col_parquet_idx, + active.prep.input_start_rows[idx], + ); + } + } + + let assembler = BodyColOutputPageAssembler::new( + handle, + decoders_state, + &input_col_indices, + &active.prep.region_destinations, + 0, + col_name, + field.as_ref(), + ); + + let mut error_slot: Option = None; + let service_collector: Option<&mut HashSet> = if track_service { + Some(&mut active.accumulator.service_names) + } else { + None + }; + let stream_iter = StreamingBodyColIter { + inner: assembler.into_iter(), + error_slot: &mut error_slot, + service_collector, + }; + let write_result = active.rg.write_next_column_arrays(stream_iter); + + if let Some(err) = error_slot { + return Err(err).with_context(|| { + format!( + "assembling body col '{col_name}' for split-region chunk {chunk_idx}", + ) + }); + } + write_result.with_context(|| { + format!("writing body col '{col_name}' to split-region chunk {chunk_idx}",) + })?; + } + } + } + + // Phase 4: finalize each chunk's RG and accumulate per-output + // metadata. RG finish must precede num_row_groups bump (MS-3: + // they have to agree). + for active in actives.into_iter() { + let ActiveChunk { + rg, + accumulator, + num_row_groups, + output_idx, + prep, + } = active; + rg.finish().with_context(|| { + format!("finishing row group for split-region chunk output {output_idx}",) + })?; + *num_row_groups += 1; + accumulator.append_sort_batch(prep.region_sort_batch)?; + accumulator.num_rows += prep.region_rows; + } + + Ok(()) +} + /// Helper for sort col writes within a region: split the region's /// already-sorted sort col into page-sized chunks for /// `write_next_column_arrays`. @@ -2374,6 +2840,65 @@ mod tests { } } + /// Regression for Codex P1 on PR #6428: companion to + /// `test_region_exactly_fills_output_does_not_split_next_aligned_region`. + /// The earlier fix handled the split *decision* (use the rolled- + /// over output's full budget) but missed the split *assignment* + /// loop in `process_split_region_col_outer`'s setup: it + /// initialized `active_output_idx` / `active_rows` from the stale + /// `current_output_idx` / `current_output_rows`. The inner + /// `needs_new_writer` check guards on `!chunk_assignments.is_empty()`, + /// so the first chunk never rolled over — it was appended to the + /// already-full output, doubling its rows and shrinking the total + /// output count. + /// + /// Setup: prefix_len = 1, two metrics with very different sizes + /// (200 rows + 400 rows = 600 total). `num_outputs = 3` → + /// `target_per_output = 200`. Region A (metric_name = "alpha") + /// fills output 0 exactly. Region B (metric_name = "beta") needs + /// splitting: 400 rows > the rolled-over output's 200 budget. + /// Pre-fix the merge produced 2 outputs of 400 + 200 (output 0 + /// overfilled, output 2 empty). Post-fix the merge produces 3 + /// outputs of 200 + 200 + 200. + #[tokio::test] + async fn test_split_chunk_assignment_rolls_over_before_first_chunk() { + let bytes = make_prefix_len_one_input(&[("aaa.alpha", 200), ("bbb.beta", 400)]); + let inputs: Vec> = vec![open_stream(bytes).await]; + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(3)) + .await + .expect("merge must succeed"); + + // Pre-fix: outputs.len() == 2 (output 0 = 400 rows, output 1 = + // 200 rows). Post-fix: 3 outputs of ~200 rows each. + assert_eq!( + outputs.len(), + 3, + "expected 3 outputs (one per target), got {} — chunk assignment didn't roll over \ + before the first chunk", + outputs.len(), + ); + let total_rows: usize = outputs.iter().map(|o| o.num_rows).sum(); + assert_eq!(total_rows, 600, "row conservation"); + for (i, out) in outputs.iter().enumerate() { + // All-non-empty: pre-fix output 2 was empty (the loop + // assigned both region-B sub-chunks within outputs 0+1). + assert!( + out.num_rows > 0, + "output {i} should be non-empty post-fix; got num_rows = {}", + out.num_rows, + ); + // No output exceeds the target by more than one full + // sub-region. Pre-fix output 0 was 400 rows (2× target). + assert!( + out.num_rows <= 250, + "output {i} should not exceed target by more than one sub-region; got {}", + out.num_rows, + ); + } + } + /// Build a single-RG fixture with multiple metric_names sorted /// by metric_name then timestamp. The file has /// `rg_partition_prefix_len = 0` (legacy) so the streaming merge @@ -4273,25 +4798,84 @@ mod tests { // small constant — never scale with row count. // ============================================================================ + /// `PEAK_BODY_COL_PAGE_CACHE_LEN` is a process-global atomic, so + /// concurrent MS-7 tests would pollute each other's readings. + /// Every MS-7 test must acquire this lock for the duration of its + /// merge + read sequence. The static guarantees there's only one + /// MS-7 merge in flight at a time across the whole test binary. + /// + /// Held across `.await` points in the MS-7 tests — that's why + /// each test allows `clippy::await_holding_lock`. In production + /// code we'd use an async-aware mutex, but this is test-only + /// process-wide serialization for a global atomic that has no + /// async-safe alternative; `tokio::sync::Mutex` is also banned + /// by GAP-002 (cancel-correctness). The lock is `std::sync::Mutex` + /// and the executor is `tokio::test`'s single-threaded runtime, + /// so holding the guard across await won't deadlock another + /// thread. + fn ms7_serial_lock() -> std::sync::MutexGuard<'static, ()> { + static LOCK: std::sync::Mutex<()> = std::sync::Mutex::new(()); + // Poisoning is fine — a previous test panicking shouldn't + // prevent the next one from acquiring; just unwrap the inner. + LOCK.lock().unwrap_or_else(|poisoned| poisoned.into_inner()) + } + /// Build a fixture that forces many input body-col pages with a /// pinned `data_page_row_count_limit`, then merge it through the /// streaming engine and read back the peak cache length. Used by /// the MS-7 test below across multiple sizes. + /// + /// Caller must hold [`ms7_serial_lock`] across this call. async fn merge_and_observe_peak_page_cache(num_rows: usize, page_rows: usize) -> usize { + merge_n_inputs_and_observe_peak_page_cache(1, num_rows, page_rows).await + } + + /// Run the same merge through `num_inputs` identical small-pages + /// fixtures and return the peak per-input cache length observed. + /// + /// `PEAK_BODY_COL_PAGE_CACHE_LEN` is set per-input by the body + /// assembler — it tracks `body_col_caches_total_len` (sum across + /// body cols for a single `InputDecoderState`). The atomic stores + /// `max(per-input)`, NOT the sum, so the bound should hold flat + /// regardless of `num_inputs`. Each input's body-col cache is + /// independent; adding more inputs adds parallel caches but + /// doesn't push any one input's cache size up. The MS-7 + /// per-input invariant must hold in the multi-input case the + /// same way it does in single-input. + /// + /// The fixture writes `num_inputs` distinct files (each with a + /// unique sort-key offset so the merge engine genuinely + /// interleaves them — using identical files would let the engine + /// take shortcuts). Each input has `num_rows_per_input` rows on + /// the same single metric_name, so all rows fall in one region. + async fn merge_n_inputs_and_observe_peak_page_cache( + num_inputs: usize, + num_rows_per_input: usize, + page_rows: usize, + ) -> usize { use std::sync::atomic::Ordering; PEAK_BODY_COL_PAGE_CACHE_LEN.store(0, Ordering::Relaxed); - let batch = make_sorted_batch(num_rows, 0); - let bytes = write_input_parquet_with_small_pages(std::slice::from_ref(&batch), page_rows); - let inputs: Vec> = vec![open_stream(bytes).await]; + let mut inputs: Vec> = Vec::with_capacity(num_inputs); + for i in 0..num_inputs { + // Stagger sort-key offsets so each input has a disjoint + // sorted_series range. Without this the merge engine + // would compare row-equal keys and the interleaving + // pattern wouldn't exercise per-input page caches. + let key_offset = i as u64 * (num_rows_per_input as u64); + let batch = make_sorted_batch(num_rows_per_input, key_offset); + let bytes = + write_input_parquet_with_small_pages(std::slice::from_ref(&batch), page_rows); + inputs.push(open_stream(bytes).await); + } let tmp = TempDir::new().expect("tmpdir"); let outputs = streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(1)) .await .expect("merge"); assert_eq!(outputs.len(), 1); - assert_eq!(outputs[0].num_rows, num_rows); + assert_eq!(outputs[0].num_rows, num_rows_per_input * num_inputs); PEAK_BODY_COL_PAGE_CACHE_LEN.load(Ordering::Relaxed) } @@ -4309,8 +4893,10 @@ mod tests { /// A regression that dropped the eviction loop would push the peak /// past the ceiling for the 30 000-row fixture but not the 3 000- /// row fixture, breaking both assertions below. + #[allow(clippy::await_holding_lock, reason = "see ms7_serial_lock docs")] #[tokio::test] async fn test_ms7_body_col_page_cache_bounded_regardless_of_input_size() { + let _guard = ms7_serial_lock(); const PAGE_ROWS: usize = 50; // ceil(1024 / 50) = 21 in-flight pages needed for one output // page. Allow 3 slack: decoder lookahead, transient between @@ -4423,6 +5009,227 @@ mod tests { ); } + /// MS-7 multi-input dimension: the per-input page-cache bound must + /// hold across `num_inputs ∈ {1, 3, 8}` independent inputs being + /// merged together. Each input has its own `InputDecoderState` and + /// therefore its own per-col page cache; the atomic + /// `PEAK_BODY_COL_PAGE_CACHE_LEN` stores + /// `max(body_col_caches_total_len)` over all inputs, so the bound + /// being checked is per-input, not summed across inputs. + /// + /// The motivation: production merges combine many splits at once. + /// A regression that, say, shared a buffer across input states (so + /// pages from N inputs piled into the same cache) would push the + /// observed peak to roughly N × the single-input peak. The current + /// design keeps caches independent — this test pins that invariant. + /// + /// Sweep over `num_inputs ∈ {1, 3, 8}` AND + /// `rows_per_input ∈ {3_000, 30_000}` so a "peak grows with + /// num_inputs" regression and a "peak grows with rows" regression + /// would each surface separately. + #[allow(clippy::await_holding_lock, reason = "see ms7_serial_lock docs")] + #[tokio::test] + async fn test_ms7_per_input_bound_across_num_inputs() { + let _guard = ms7_serial_lock(); + const PAGE_ROWS: usize = 50; + // Same constant as the single-input MS-7 test: + // ceil(OUTPUT_PAGE_ROWS / PAGE_ROWS) + slack. + const MAX_RESIDENT_PAGES_PER_INPUT: usize = 24; + + for &num_inputs in &[1usize, 3, 8] { + for &rows_per_input in &[3_000usize, 30_000] { + let peak = merge_n_inputs_and_observe_peak_page_cache( + num_inputs, + rows_per_input, + PAGE_ROWS, + ) + .await; + assert!( + peak <= MAX_RESIDENT_PAGES_PER_INPUT, + "num_inputs={num_inputs}, rows_per_input={rows_per_input}: per-input peak \ + cache len {peak} > {MAX_RESIDENT_PAGES_PER_INPUT} — body-col caches are no \ + longer per-input-independent", + ); + } + } + + // Cross-axis growth: 1→8 inputs at the same row count must + // not push peak up. The atomic tracks max-per-input, so a + // value increase between (1 input, 30 000 rows) and (8 inputs, + // 30 000 rows) would indicate caches bleeding across inputs. + let peak_1in = merge_n_inputs_and_observe_peak_page_cache(1, 30_000, PAGE_ROWS).await; + let peak_8in = merge_n_inputs_and_observe_peak_page_cache(8, 30_000, PAGE_ROWS).await; + let growth = peak_8in.saturating_sub(peak_1in); + assert!( + growth <= 2, + "peak grows with num_inputs: 1in={peak_1in}, 8in={peak_8in} — caches likely shared \ + across inputs", + ); + } + + /// MS-7 sub-region dimension: when `prefix_len = 0` + + /// `num_outputs > 1` triggers `split_region_at_sorted_series`, + /// the per-input page cache survives across sub-region boundaries + /// within one top-level region (the engine reads col K once and + /// re-uses it across sub-regions). The headline MS-7 claim must + /// still hold: peak DOES NOT scale with input row count. + /// + /// This test guards the F14 fix: `process_split_region_col_outer` + /// inverts the col/sub-region loop nesting so each parquet col + /// chunk is fully consumed from the page stream before the next + /// col starts. Before that fix, peak scaled ~linearly with row + /// count (3 000 rows → 140 cached pages; 30 000 rows → 1 200 + /// pages, ~9× growth for 10× rows). With the fix, the per-input + /// peak stays bounded by `K × num_body_cols × num_chunks` where + /// K is the per-col output-page bound — independent of row + /// count. + /// + /// Fixture: single multi-metric single-RG input (prefix_len = 0) + /// with 6 distinct metric_names. `num_outputs = 3` forces the + /// engine into the drain-and-align-then-split path. Two row-count + /// regimes (small + large): peak at the large size must be within + /// a small additive slack of the small size's peak. + #[allow(clippy::await_holding_lock, reason = "see ms7_serial_lock docs")] + #[tokio::test] + async fn test_ms7_per_input_bound_across_sub_regions_does_not_scale_with_rows() { + let _guard = ms7_serial_lock(); + use std::sync::atomic::Ordering; + + const PAGE_ROWS: usize = 50; + // Slack covers: page-decoder lookahead, transient between + // push and check across body cols, the inter-col handoff + // where col K's cache hasn't drained before col K+1's first + // page arrives. + const SCALING_SLACK_PAGES: usize = 4; + + async fn run(rows_per_metric: usize) -> usize { + PEAK_BODY_COL_PAGE_CACHE_LEN.store(0, Ordering::Relaxed); + let metrics: Vec<(&str, usize)> = vec![ + ("aaa.alpha", rows_per_metric), + ("bbb.beta", rows_per_metric), + ("ccc.gamma", rows_per_metric), + ("ddd.delta", rows_per_metric), + ("eee.epsilon", rows_per_metric), + ("fff.zeta", rows_per_metric), + ]; + let bytes = make_multi_metric_single_rg_input_with_small_pages(&metrics, PAGE_ROWS); + let inputs: Vec> = vec![open_stream(bytes).await]; + let tmp = TempDir::new().expect("tmpdir"); + let outputs = + streaming_merge_sorted_parquet_files(inputs, tmp.path(), &merge_config(3)) + .await + .expect("merge"); + assert_eq!(outputs.len(), 3, "sub-region split must honor num_outputs"); + let total: usize = outputs.iter().map(|o| o.num_rows).sum(); + assert_eq!(total, rows_per_metric * 6); + PEAK_BODY_COL_PAGE_CACHE_LEN.load(Ordering::Relaxed) + } + + // Small: 6 × 500 = 3 000 rows = 60 input pages per col chunk. + let peak_small = run(500).await; + // Large: 6 × 5 000 = 30 000 rows = 600 input pages per col chunk. + let peak_large = run(5_000).await; + + // The MS-7 invariant: peak does not grow proportionally with + // input row count. 10× more rows must produce roughly the + // same peak — any growth beyond `SCALING_SLACK_PAGES` means + // the cache is no longer page-bounded in the sub-region + // path. + let growth = peak_large.saturating_sub(peak_small); + assert!( + growth <= SCALING_SLACK_PAGES, + "sub-region path violates MS-7 scaling: small (3 000 rows)={peak_small}, large (30 \ + 000 rows)={peak_large} — 10× more rows produced {growth} more resident pages, \ + body-col path is not page-bounded across sub-regions", + ); + } + + /// Like `make_multi_metric_single_rg_input` but writes the file + /// with a pinned `data_page_row_count_limit` so the body-col path + /// actually spans many parquet pages. Otherwise the default + /// `data_page_size` byte threshold dominates and small fixtures + /// emit one page per col chunk — no opportunity to observe the + /// page-cache bound. + fn make_multi_metric_single_rg_input_with_small_pages( + metrics: &[(&str, usize)], + page_rows: usize, + ) -> Bytes { + // Build the same arrow batch shape as + // `make_multi_metric_single_rg_input`, then route through the + // small-pages writer instead of the default-config writer. + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("metric_name", dict_type.clone(), false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("sorted_series", DataType::Binary, false), + Field::new("metric_type", DataType::UInt8, false), + Field::new("service", dict_type, true), + Field::new("timeseries_id", DataType::Int64, false), + Field::new("value", DataType::Float64, false), + ])); + + let total_rows: usize = metrics.iter().map(|(_, n)| *n).sum(); + let mut metric_keys: Vec = Vec::with_capacity(total_rows); + let mut metric_values_vec: Vec<&str> = Vec::with_capacity(metrics.len()); + let mut timestamps: Vec = Vec::with_capacity(total_rows); + let mut series_bytes: Vec> = Vec::with_capacity(total_rows); + let mut tsids: Vec = Vec::with_capacity(total_rows); + let mut values: Vec = Vec::with_capacity(total_rows); + let mut row_idx: u64 = 0; + for (metric_idx, (name, num)) in metrics.iter().enumerate() { + metric_values_vec.push(name); + for r in 0..*num { + metric_keys.push(metric_idx as i32); + timestamps.push(1_700_000_000 + ((*num - r) as u64)); + series_bytes.push(row_idx.to_be_bytes().to_vec()); + tsids.push(1000 + row_idx as i64); + values.push(row_idx as f64); + row_idx += 1; + } + } + let metric_names_arr = StringArray::from(metric_values_vec); + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(metric_names_arr), + ) + .expect("dict"), + ); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let series_refs: Vec<&[u8]> = series_bytes.iter().map(|v| v.as_slice()).collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from(series_refs)); + let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; total_rows])); + let service: ArrayRef = { + let svc_keys: Vec> = (0..total_rows as i32).map(|i| Some(i % 3)).collect(); + let svc_values = StringArray::from(vec!["api", "db", "cache"]); + Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(svc_keys), + Arc::new(svc_values), + ) + .expect("svc dict"), + ) + }; + let timeseries_id: ArrayRef = Arc::new(Int64Array::from(tsids)); + let value: ArrayRef = Arc::new(Float64Array::from(values)); + + let batch = RecordBatch::try_new( + schema, + vec![ + metric_name, + timestamp_secs, + sorted_series, + metric_type, + service, + timeseries_id, + value, + ], + ) + .expect("batch"); + + write_input_parquet_with_small_pages(&[batch], page_rows) + } + // ============================================================================ // Heterogeneous-output regressions (Codex P2 batch on PR-6409) // ============================================================================ @@ -4711,4 +5518,613 @@ mod tests { "error should explain the mismatch, got: {msg}", ); } + + // ============================================================================ + // F5: prefix-aware proptest (varies the new prefix machinery to surface + // regressions the hand-picked fixtures miss). + // ============================================================================ + + /// F7 — production-shape integration test for the streaming + /// engine. The "compact many already-aligned splits" scenario: + /// + /// - 5 inputs, each prefix_len=1 multi-RG (one RG per metric_name). + /// - 15 distinct metric_names total across all inputs, with overlap (every metric appears in + /// 2-3 of the 5 inputs). + /// - `num_outputs = 4` so the engine has to split by region group while keeping every output + /// prefix-aligned. + /// + /// Asserts the full "production-shape" invariant bundle: + /// - **MC-1**: total rows preserved. + /// - **MS-3**: every `MergeOutputFile.num_row_groups` matches its footer. + /// - **PA-1 + PA-3**: every output passes `assert_unique_rg_prefix_keys` with prefix_len = 1. + /// - **MS-5**: across adjacent output files, sorted_series is monotone non-decreasing — output + /// K's last row's sorted_series ≤ output K+1's first row's sorted_series. A single metric CAN + /// span output boundaries (the engine splits at sorted_series transitions inside an + /// overflowing metric region), so the cross-output invariant is sorted_series monotonicity, + /// not "each metric in one output". + /// - **CS-1**: each output's `MergeOutputFile.output_rg_partition_prefix_len` matches the + /// on-disk KV. + /// + /// This is the corner the original adversarial review flagged as + /// "untested production case": multi-input × multi-RG × + /// multi-output × prefix_len > 0. Every piece is tested in + /// isolation by hand-picked fixtures elsewhere; this test + /// exercises them together. + #[tokio::test] + async fn test_f7_production_shape_multi_input_multi_rg_multi_output() { + // Six metric pool. Each input picks 4-5 from this pool so + // there's overlap (the merge has to combine same-metric RGs + // across inputs). + let pool = [ + "aaa.alpha", + "bbb.beta", + "ccc.gamma", + "ddd.delta", + "eee.epsilon", + "fff.zeta", + ]; + // Each input is a sorted prefix of the pool with a specific + // row count per metric. Different starts/lengths so the + // inputs aren't identical. + let inputs_spec: Vec> = vec![ + // input 0: aaa, bbb, ccc, ddd + vec![(pool[0], 40), (pool[1], 30), (pool[2], 50), (pool[3], 20)], + // input 1: aaa, bbb, eee, fff + vec![(pool[0], 25), (pool[1], 35), (pool[4], 45), (pool[5], 30)], + // input 2: bbb, ccc, ddd, eee + vec![(pool[1], 20), (pool[2], 40), (pool[3], 30), (pool[4], 25)], + // input 3: aaa, ccc, fff + vec![(pool[0], 35), (pool[2], 30), (pool[5], 40)], + // input 4: ddd, eee, fff + vec![(pool[3], 25), (pool[4], 35), (pool[5], 20)], + ]; + let total_input_rows: usize = inputs_spec + .iter() + .flat_map(|i| i.iter().map(|(_, r)| *r)) + .sum(); + + let mut streams: Vec> = Vec::with_capacity(inputs_spec.len()); + for spec in &inputs_spec { + let bytes = make_prefix_len_one_input(spec); + streams.push(open_stream(bytes).await); + } + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(streams, tmp.path(), &merge_config(4)) + .await + .expect("production-shape merge"); + + // MC-1: rows preserved. + let total_output_rows: usize = outputs.iter().map(|o| o.num_rows).sum(); + assert_eq!( + total_output_rows, total_input_rows, + "MC-1: input total {total_input_rows} != output total {total_output_rows}", + ); + + // Per-output checks. Capture each output's first and last + // sorted_series bytes for the MS-5 cross-output monotone check. + let mut last_ss_per_output: Vec> = Vec::with_capacity(outputs.len()); + let mut first_ss_per_output: Vec> = Vec::with_capacity(outputs.len()); + + for (out_idx, output) in outputs.iter().enumerate() { + let bytes_out = std::fs::read(&output.path).expect("read"); + let reader = SerializedFileReader::new(Bytes::from(bytes_out)).expect("ser"); + let metadata = reader.metadata(); + + // MS-3: reported num_row_groups matches footer. + assert_eq!( + output.num_row_groups, + metadata.num_row_groups(), + "MS-3 violated for output {out_idx}: reported {} vs footer {}", + output.num_row_groups, + metadata.num_row_groups(), + ); + + // PA-1 + PA-3 chunk-level: each RG has constant + unique metric_name. + assert_unique_rg_prefix_keys( + metadata, + "metric_name|-timestamp_secs/V2", + 1, + "test_f7_production_shape output", + ) + .unwrap_or_else(|e| panic!("PA-1/PA-3 violated for output {out_idx}: {e}")); + + // CS-1: MergeOutputFile reports the same prefix_len as the on-disk KV. + let kv_prefix_len: u32 = metadata + .file_metadata() + .key_value_metadata() + .and_then(|kvs| { + kvs.iter() + .find(|k| k.key == PARQUET_META_RG_PARTITION_PREFIX_LEN) + .and_then(|k| k.value.as_deref()) + }) + .map(|s| s.parse().unwrap_or(0)) + .unwrap_or(0); + assert_eq!( + output.output_rg_partition_prefix_len, kv_prefix_len, + "CS-1 violated for output {out_idx}: MergeOutputFile reports {} vs KV {}", + output.output_rg_partition_prefix_len, kv_prefix_len, + ); + + // MC-3 within-file: sorted_series monotone across the + // whole output. Capture first/last sorted_series for the + // cross-output MS-5 check below. + let merged = read_output_to_record_batch(&output.path); + let ss_idx = merged.schema().index_of("sorted_series").expect("series"); + let ss = merged + .column(ss_idx) + .as_any() + .downcast_ref::() + .expect("Binary"); + for i in 0..merged.num_rows().saturating_sub(1) { + assert!( + ss.value(i) <= ss.value(i + 1), + "MC-3 within output {out_idx}: sorted_series decreased at row {i}", + ); + } + first_ss_per_output.push(ss.value(0).to_vec()); + last_ss_per_output.push(ss.value(merged.num_rows() - 1).to_vec()); + } + + // MS-5: across adjacent output files, sorted_series is + // monotone non-decreasing. Output K's last row's sorted_series + // must be ≤ output K+1's first row's sorted_series. A single + // metric CAN span outputs (engine splits at sorted_series + // transitions inside an overflowing region), so this is the + // cross-output ordering invariant — not "each metric in one + // output". + for i in 0..outputs.len().saturating_sub(1) { + assert!( + last_ss_per_output[i].as_slice() <= first_ss_per_output[i + 1].as_slice(), + "MS-5 violated: output {i}'s last sorted_series {:?} > output {}'s first \ + sorted_series {:?}", + last_ss_per_output[i], + i + 1, + first_ss_per_output[i + 1], + ); + } + } + + /// Focused unit test on the minimal proptest failure case — used + /// as a fast iteration channel for debugging. + #[tokio::test] + async fn test_f5_single_input_two_metrics_minimal() { + let bytes = make_prefix_len_one_input(&[("aaa.alpha", 10), ("bbb.beta", 11)]); + let streams: Vec> = vec![open_stream(bytes).await]; + let tmp = TempDir::new().expect("tmpdir"); + let outputs = streaming_merge_sorted_parquet_files(streams, tmp.path(), &merge_config(1)) + .await + .expect("merge"); + assert_eq!(outputs.len(), 1); + assert_eq!(outputs[0].num_rows, 21); + + let merged = read_output_to_record_batch(&outputs[0].path); + let ss_idx = merged.schema().index_of("sorted_series").expect("series"); + let ts_idx = merged + .schema() + .index_of("timestamp_secs") + .expect("timestamp"); + let metric_idx = merged.schema().index_of("metric_name").expect("metric"); + let ss = merged + .column(ss_idx) + .as_any() + .downcast_ref::() + .expect("Binary"); + let ts = merged + .column(ts_idx) + .as_any() + .downcast_ref::() + .expect("UInt64"); + let metric = merged.column(metric_idx); + + for i in 0..merged.num_rows() { + let metric_str = match metric.data_type() { + DataType::Utf8 => metric + .as_any() + .downcast_ref::() + .map(|a| a.value(i).to_string()), + DataType::Dictionary(_, _) => { + use arrow::array::AsArray; + let dict = metric.as_dictionary::(); + let key = dict.keys().value(i); + dict.values() + .as_any() + .downcast_ref::() + .map(|a| a.value(key as usize).to_string()) + } + _ => None, + }; + eprintln!( + "row {i}: metric={metric_str:?} ss={:?} ts={}", + ss.value(i), + ts.value(i), + ); + } + + for i in 0..merged.num_rows().saturating_sub(1) { + assert!( + ss.value(i) <= ss.value(i + 1), + "sorted_series decreased at row {i}: {:?} > {:?}", + ss.value(i), + ss.value(i + 1), + ); + } + } + + /// Per-metric `sorted_series` base. In production the + /// `sorted_series` column is the order-preserving storekey + /// encoding of `(sort cols before timeseries_id, timeseries_id)` + /// — a concatenation, not a hash. Byte-level ordering matches + /// the logical (sort cols ASC, timeseries_id ASC) ordering, so + /// rows from different metric_names produce sorted_series + /// values from non-overlapping byte ranges: every "aaa.alpha" + /// row's sorted_series sorts before every "bbb.beta" row's. + /// The proptest fixture mirrors that property by giving each + /// metric a numeric base so its (base + row_offset) range + /// doesn't overlap the next metric's range. + fn proptest_metric_sorted_series_base(metric: &str) -> u64 { + match metric { + "aaa.alpha" => 0, + "bbb.beta" => 1_000_000, + "ccc.gamma" => 2_000_000, + "ddd.delta" => 3_000_000, + "eee.epsilon" => 4_000_000, + "fff.zeta" => 5_000_000, + other => panic!("unknown metric in proptest pool: {other}"), + } + } + + /// Build a multi-RG fixture with prefix_len=1: one RG per + /// (metric_name, rows_per_rg) entry. Caller guarantees the + /// metric names are sorted ascending and unique within this file + /// (the streaming engine enforces both — duplicate prefix keys + /// are rejected as PA-3 violations, mis-sorted physical RG order + /// is rejected as MS-2). `sorted_series` is computed as + /// `metric_base + row_offset_within_metric`. Because the storekey + /// encoding of production `sorted_series` puts metric_name bytes + /// before tsid bytes, different metric_names yield non-overlapping + /// sorted_series ranges; this fixture preserves that property via + /// the per-metric base table. The same `(metric_name, row_offset)` + /// across two inputs gets the SAME sorted_series — that's the + /// realistic case where the same series appears in multiple + /// splits. Used by the F5 proptest. + fn make_prefix_len_one_input(rgs: &[(&str, usize)]) -> Bytes { + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("metric_name", dict_type.clone(), false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("sorted_series", DataType::Binary, false), + Field::new("metric_type", DataType::UInt8, false), + Field::new("service", dict_type, true), + Field::new("timeseries_id", DataType::Int64, false), + Field::new("value", DataType::Float64, false), + ])); + + let make_batch = |metric: &str, start_series: u64, rows: usize| -> RecordBatch { + let metric_keys: Vec = vec![0; rows]; + let metric_values = StringArray::from(vec![metric]); + let metric_name: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(metric_keys), + Arc::new(metric_values), + ) + .expect("dict"), + ); + let timestamps: Vec = (0..rows as u64) + .map(|i| 1_700_000_000 + (rows as u64 - i)) + .collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let series_bytes: Vec> = (0..rows as u64) + .map(|i| (start_series + i).to_be_bytes().to_vec()) + .collect(); + let sorted_series: ArrayRef = Arc::new(BinaryArray::from( + series_bytes + .iter() + .map(|v| v.as_slice()) + .collect::>(), + )); + let metric_type: ArrayRef = Arc::new(UInt8Array::from(vec![0u8; rows])); + let svc_keys: Vec> = (0..rows as i32).map(|i| Some(i % 3)).collect(); + let svc_values = StringArray::from(vec!["api", "db", "cache"]); + let service: ArrayRef = Arc::new( + DictionaryArray::::try_new( + arrow::array::Int32Array::from(svc_keys), + Arc::new(svc_values), + ) + .expect("svc dict"), + ); + let tsids: Vec = (0..rows as i64).map(|i| 1000 + i).collect(); + let timeseries_id: ArrayRef = Arc::new(Int64Array::from(tsids)); + let values: Vec = (0..rows).map(|i| i as f64).collect(); + let value: ArrayRef = Arc::new(Float64Array::from(values)); + + RecordBatch::try_new( + schema.clone(), + vec![ + metric_name, + timestamp_secs, + sorted_series, + metric_type, + service, + timeseries_id, + value, + ], + ) + .expect("prefix-len-1 batch") + }; + + // Make every batch land in its own RG by setting row_group_size + // larger than any single batch and calling flush() after each + // write — ArrowWriter rolls over the open RG on flush. + let max_rows = rgs.iter().map(|(_, n)| *n).max().unwrap_or(1); + let cfg = ParquetWriterConfig { + compression: Compression::Snappy, + row_group_size: max_rows.saturating_add(1), + ..ParquetWriterConfig::default() + }; + let kvs = vec![ + KeyValue::new( + PARQUET_META_SORT_FIELDS.to_string(), + "metric_name|-timestamp_secs/V2".to_string(), + ), + KeyValue::new( + PARQUET_META_WINDOW_START.to_string(), + "1700000000".to_string(), + ), + KeyValue::new(PARQUET_META_WINDOW_DURATION.to_string(), "60".to_string()), + KeyValue::new(PARQUET_META_NUM_MERGE_OPS.to_string(), "0".to_string()), + KeyValue::new( + PARQUET_META_RG_PARTITION_PREFIX_LEN.to_string(), + "1".to_string(), + ), + ]; + let sorting_cols = vec![ + parquet::file::metadata::SortingColumn { + column_idx: 0, + descending: false, + nulls_first: false, + }, + parquet::file::metadata::SortingColumn { + column_idx: 1, + descending: true, + nulls_first: false, + }, + ]; + let props: WriterProperties = cfg.to_writer_properties_with_metadata( + &schema, + sorting_cols, + Some(kvs), + &["metric_name".to_string(), "timestamp_secs".to_string()], + ); + let mut buf: Vec = Vec::new(); + let mut writer = + ArrowWriter::try_new(&mut buf, schema.clone(), Some(props)).expect("arrow writer"); + for (metric, rows) in rgs { + let start_series = proptest_metric_sorted_series_base(metric); + let batch = make_batch(metric, start_series, *rows); + writer.write(&batch).expect("write"); + // Force a fresh RG for the next batch so each (metric, + // rows) entry maps to exactly one RG. + writer.flush().expect("flush rg"); + } + writer.close().expect("close"); + Bytes::from(buf) + } + + /// F5 — prefix-aware proptest over the streaming engine. + /// + /// Sweeps `(num_inputs, per-input RG specs, num_outputs)` with + /// `prefix_len = 1` and asserts the merger's load-bearing + /// invariants on every generated case: + /// + /// - **MC-1**: total row count preserved across inputs → outputs. + /// - **MC-3**: each output is sorted on `(sorted_series ASC, timestamp DESC)`. + /// - **MS-3**: every output's `MergeOutputFile.num_row_groups` matches the on-disk footer. + /// - **PA-1 + PA-3**: every output's row groups pass `assert_unique_rg_prefix_keys` + /// (intra-RG constancy + inter-RG uniqueness). + /// - **CS-1**: the metastore-recorded `rg_partition_prefix_len` matches the on-disk KV via + /// the writer-stamped `MergeOutputFile.output_rg_partition_prefix_len`. + /// + /// Strategy: + /// - `num_inputs ∈ 1..=3`. + /// - Each input: 1..=4 RGs, each `(metric_name, rows)` where metric names come from a + /// small pool, sorted-and-deduped within each input (so prefix_len=1's MS-2 + PA-3 + /// hold). + /// - `num_outputs ∈ 1..=3`. + /// + /// Cases capped at 32 so the proptest completes well under 30s + /// (each case opens streams + runs an async merge). + use proptest::test_runner::TestCaseError; + proptest::proptest! { + #![proptest_config(proptest::prelude::ProptestConfig { + cases: 32, + ..proptest::prelude::ProptestConfig::default() + })] + + #[test] + fn prop_merge_prefix_aligned_streaming( + per_input_specs in proptest::collection::vec( + prefix_one_input_strategy(), + 1usize..=3, + ), + num_outputs in 1usize..=3, + ) { + let rt = tokio::runtime::Builder::new_current_thread() + .enable_all() + .build() + .expect("tokio rt"); + let outcome: std::result::Result<(), TestCaseError> = + rt.block_on(async move { + run_prefix_aligned_case(per_input_specs, num_outputs).await + }); + outcome?; + } + } + + /// Per-input strategy: 1..=4 RGs, each `(metric_name, rows)`, then + /// dedup-by-metric and sort-by-metric so the resulting list + /// satisfies SS-1 (sorted) and PA-3 (unique prefixes within the + /// input). + fn prefix_one_input_strategy() -> impl proptest::strategy::Strategy> + { + use proptest::prelude::*; + let metric_pool: &[&'static str] = &[ + "aaa.alpha", + "bbb.beta", + "ccc.gamma", + "ddd.delta", + "eee.epsilon", + "fff.zeta", + ]; + let metric_strat = + proptest::sample::select(metric_pool.to_vec()).prop_map(|s| s.to_string()); + let row_count_strat = 10usize..=80; + let entry_strat = (metric_strat, row_count_strat); + proptest::collection::vec(entry_strat, 1usize..=4).prop_map(|mut rgs| { + rgs.sort_by(|a, b| a.0.cmp(&b.0)); + rgs.dedup_by(|a, b| a.0 == b.0); + rgs + }) + } + + async fn run_prefix_aligned_case( + per_input_specs: Vec>, + num_outputs: usize, + ) -> std::result::Result<(), proptest::test_runner::TestCaseError> { + use proptest::prelude::*; + + // Build each input's bytes via the prefix_len=1 fixture. + // Sorted_series numbering starts at 0 per input — production + // inputs can legitimately share sorted_series values across + // splits (sorted_series is a (sort cols + timeseries_id) + // hash, and the same series appears in multiple splits when + // the time window spans more than one ingest interval). The + // merger must handle this correctly; the MC-3 check below + // exercises that invariant. + let mut input_bytes: Vec = Vec::with_capacity(per_input_specs.len()); + let mut total_input_rows: usize = 0; + for spec in &per_input_specs { + let rgs_borrowed: Vec<(&str, usize)> = + spec.iter().map(|(m, n)| (m.as_str(), *n)).collect(); + total_input_rows += rgs_borrowed.iter().map(|(_, n)| *n).sum::(); + input_bytes.push(make_prefix_len_one_input(&rgs_borrowed)); + } + + let mut streams: Vec> = Vec::with_capacity(input_bytes.len()); + for bytes in input_bytes { + streams.push(open_stream(bytes).await); + } + + let tmp = TempDir::new().expect("tmpdir"); + let outputs = + streaming_merge_sorted_parquet_files(streams, tmp.path(), &merge_config(num_outputs)) + .await + .map_err(|e| { + proptest::test_runner::TestCaseError::fail(format!( + "streaming merge failed: {e}" + )) + })?; + + // MC-1: total rows preserved. + let output_total: usize = outputs.iter().map(|o| o.num_rows).sum(); + prop_assert_eq!( + output_total, + total_input_rows, + "MC-1 violated: input total = {}, output total = {}", + total_input_rows, + output_total + ); + + for (out_idx, output) in outputs.iter().enumerate() { + let bytes_out = std::fs::read(&output.path) + .map_err(|e| proptest::test_runner::TestCaseError::fail(e.to_string()))?; + let reader = SerializedFileReader::new(Bytes::from(bytes_out)) + .map_err(|e| proptest::test_runner::TestCaseError::fail(e.to_string()))?; + let metadata = reader.metadata(); + + // MS-3: MergeOutputFile.num_row_groups == footer's RG count. + prop_assert_eq!( + output.num_row_groups, + metadata.num_row_groups(), + "MS-3 violated for output {}: MergeOutputFile.num_row_groups = {}, footer = {}", + out_idx, + output.num_row_groups, + metadata.num_row_groups() + ); + + // PA-1 + PA-3 chunk-level: every RG has constant metric_name AND unique across RGs. + assert_unique_rg_prefix_keys( + metadata, + "metric_name|-timestamp_secs/V2", + 1, + "prop_merge_prefix_aligned_streaming output", + ) + .map_err(|e| { + proptest::test_runner::TestCaseError::fail(format!( + "PA-1/PA-3 violated for output {out_idx}: {e}" + )) + })?; + + // CS-1: metastore-side prefix_len (output.output_rg_partition_prefix_len) + // matches on-disk KV. + let kv_prefix_len: u32 = metadata + .file_metadata() + .key_value_metadata() + .and_then(|kvs| { + kvs.iter() + .find(|k| k.key == PARQUET_META_RG_PARTITION_PREFIX_LEN) + .and_then(|k| k.value.as_deref()) + }) + .map(|s| s.parse().unwrap_or(0)) + .unwrap_or(0); + prop_assert_eq!( + output.output_rg_partition_prefix_len, + kv_prefix_len, + "CS-1 violated for output {}: MergeOutputFile reports {}, KV reports {}", + out_idx, + output.output_rg_partition_prefix_len, + kv_prefix_len + ); + + // MC-3: row-by-row sort check on (sorted_series ASC, timestamp DESC within same + // series). + let merged = read_output_to_record_batch(&output.path); + let ss_idx = merged.schema().index_of("sorted_series").expect("series"); + let ts_idx = merged + .schema() + .index_of("timestamp_secs") + .expect("timestamp"); + let ss = merged + .column(ss_idx) + .as_any() + .downcast_ref::() + .expect("Binary"); + let ts = merged + .column(ts_idx) + .as_any() + .downcast_ref::() + .expect("UInt64"); + for i in 0..merged.num_rows().saturating_sub(1) { + let a = ss.value(i); + let b = ss.value(i + 1); + prop_assert!( + a <= b, + "MC-3 violated in output {}: sorted_series decreased at row {}", + out_idx, + i + ); + if a == b { + prop_assert!( + ts.value(i) >= ts.value(i + 1), + "MC-3 violated in output {}: timestamp not descending within series at \ + row {}", + out_idx, + i + ); + } + } + } + + Ok(()) + } } From 43b034511c33c76dd921359c46abe565e606d731 Mon Sep 17 00:00:00 2001 From: George Talbot Date: Tue, 19 May 2026 08:55:44 -0400 Subject: [PATCH 10/10] feat(merge-executor): YAML flag to route regular merges through streaming engine (default off, no-op rollout) (#6441) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * test(merge): engine-parity tests + share MS-7 serial lock Adds `merge::tests::parity` with two tests that run the same realistic input fixture through both `merge_sorted_parquet_files` (in-memory engine) and `execute_merge_operation` (streaming engine over the same `LocalFileByteSource` the executor uses in production), then asserts row-by-row equivalence on every visible column. These gate the upcoming YAML flag that flips regular merges to the streaming engine: parity must hold before the default is flipped in production. The streaming engine writes a process-global atomic (`PEAK_BODY_COL_PAGE_CACHE_LEN`) that the MS-7 tests reset-then-read. Any test that runs a streaming merge must serialise against MS-7 or inflate its readings. Move `ms7_serial_lock` from the streaming-tests submodule to module scope (still `#[cfg(test)] pub(crate)`) so the new parity tests acquire the same lock. Co-Authored-By: Claude Opus 4.7 (1M context) * feat(merge-executor): YAML flag to route regular merges through the streaming engine Wires the streaming Parquet merge engine into the regular (non-promotion) merge path behind a node-level YAML flag, `indexer.parquet_merge_use_streaming_engine`, defaulted to false. When true, `ParquetMergeExecutor::handle` runs every merge through `execute_merge_operation` (the column-major, page-bounded streaming engine) instead of the in-memory `merge_sorted_parquet_files`. Promotion merges (`target_prefix_len_override.is_some()`) continue to take the streaming path unconditionally — the in-memory engine can't handle mixed `rg_partition_prefix_len` inputs. The in-memory engine stays in place as the runtime fallback. If the streaming engine hits a bug in production, an operator can flip the flag back to `false` via YAML without redeploying. Once the streaming path has soaked, the fallback branch and `merge_sorted_parquet_files` itself can be removed. The flag is plumbed `IndexerConfig` → `IndexingService` → `ParquetMergePipelineParams` → `ParquetMergeExecutor::new`, and exercised end-to-end by the engine parity tests in the previous commit. Co-Authored-By: Claude Opus 4.7 (1M context) * test(config): exercise parquet_merge_use_streaming_engine in YAML/JSON/TOML fixtures Adds the new flag to the YAML, JSON, and TOML node-config test fixtures and bumps the expected `IndexerConfig` in `node_config_parse_*` to `parquet_merge_use_streaming_engine: true`. Catches parse / serde regressions on the field — e.g., a rename or a default-fn typo would fail the test instead of silently parsing as `false`. Co-Authored-By: Claude Opus 4.7 (1M context) * test(merge-pipeline): end-to-end streaming-engine flag verification Adds `test_merge_pipeline_end_to_end_with_streaming_engine_flag`, an integration test that runs the full actor chain (planner → downloader → executor → uploader → publisher) with `ParquetMergePipelineParams::use_streaming_engine = true`. Asserts: 1. Publish fired with the right replaced_split_ids (merge ran end-to-end through the executor). 2. `PEAK_BODY_COL_PAGE_CACHE_LEN > 0` after the merge. The streaming engine increments this on every body-col page assembly; the in-memory engine never touches it. Non-zero is direct evidence the streaming path executed — not a silent fallback to in-memory. 3. The merge output row count and metric names are correct. To make assertion (2) work cross-crate, exposes `PEAK_BODY_COL_PAGE_CACHE_LEN` as `pub` under `#[cfg(any(test, feature = "testsuite"))]`. The visibility widening is test-only — production builds never see the symbol. This is the closest analog to the sesh-mode "production-path" rule that is feasible today: the metrics pipeline's OTLP gRPC ingest path is not yet wired into `quickwit-serve`, so the closest end-to-end test is the actor-chain integration test that this PR adds. Co-Authored-By: Claude Opus 4.7 (1M context) * docs(adr): record dual Parquet merge engines as deviation #001 Captures the intentional, time-bounded divergence from ADR-003 §4 introduced by the streaming-engine wire-in: two engines coexist in production behind `IndexerConfig::parquet_merge_use_streaming_engine`, with the in-memory engine retained as the runtime fallback. Documents: - The ADR-003 §4 quote the deviation diverges from (page-granular streaming, bounded memory). - The current dual-engine implementation and routing logic. - Why this exists (production safety, staged rollout, parity is strong-but-not-total). - Explicit exit criteria: default flipped to `true`, ≥ 2-week production soak with no merge-correctness incidents, no rollback. When met, a follow-up PR deletes the in-memory branch and engine, the flag, and the parity tests. This is the first deviation recorded under the EVOLUTION.md framework. Indexes the doc in `deviations/README.md`. Co-Authored-By: Claude Opus 4.7 (1M context) * test(merge-pipeline): share full correctness contract between both engine tests Extracts the steps-5-through-8 assertions (replaced_split_ids, staged metadata, Parquet file content, Parquet KV headers) into `assert_cpu_mem_merge_outputs_correct` and calls it from both `test_merge_pipeline_end_to_end` (in-memory engine) and `test_merge_pipeline_end_to_end_with_streaming_engine_flag` (streaming engine). The streaming-engine test had been doing only a small subset of the checks — row count and metric names. It now runs the full contract: time_range, num_merge_ops, sort_fields, row_keys_proto, zonemap_regexes, low_cardinality_tags, all 100 timestamps, sorted_series monotonicity, cpu/mem sort-order semantics, and every `qh.*` Parquet KV header. By construction both engines must produce a file that satisfies the same contract — the helper is the executable parity between engines at the pipeline-integration level, complementing the column-level parity tests in `quickwit-parquet-engine::merge::tests::parity`. Co-Authored-By: Claude Opus 4.7 (1M context) * test(merge-pipeline,merge-engine): multi-metric + multi-RG + m:n disjointness Expands test coverage along three axes the existing helpers didn't hit: 1. **Multi-input, multi-metric pipeline tests** (new file `parquet_merge_pipeline_multi_metric_test.rs`). Three inputs, each carrying three metrics with overlapping per-metric timeseries IDs and overlapping-but-distinct timestamps — the merge must row-by-row interleave across all three inputs. Output writer uses `row_group_size = 50` so the 180-row merge output breaks into four row groups, exercising the writer's multi-RG path in both engines. Both engine variants (in-memory + streaming) covered. Streaming-engine test asserts `PEAK_BODY_COL_PAGE_CACHE_LEN > 0` to confirm the flag routed through the streaming path. 2. **Engine-level multi-output contract** in `merge::tests::parity::assert_engine_parity`. Beyond the existing engine-vs-engine column equivalence, every parity test now also verifies on the in-memory engine's outputs (equivalent to the streaming engine's): sum of per-output row counts equals total input rows, each output internally monotonic on `sorted_series`, and across outputs the partition is disjoint (no two outputs share any `sorted_series` value). This is the m:n non-overlap contract. 3. **Multi-metric overlapping-input m:n** test `parity_multi_metric_overlapping_inputs_multi_output` exercises the strengthened contract with three inputs × three metrics where per-metric keyspaces overlap across inputs. n = 3 outputs target. Honest scope note in the new pipeline test module's doc: the actor pipeline today hardcodes `num_outputs = 1` in `ParquetMergeExecutor`, so n > 1 is not reachable end-to-end through the actor system. The new engine-level test covers the n > 1 correctness contract for now; when the executor is taught to accept `num_outputs > 1` from the merge policy, the pipeline tests can grow an n > 1 variant. Co-Authored-By: Claude Opus 4.7 (1M context) * feat(merge-executor): compute num_outputs from target_split_size_bytes Replaces the hardcoded `MergeConfig { num_outputs: 1, ... }` in `ParquetMergeExecutor::handle` with a per-merge computation: num_outputs = max(1, ceil(total_input_bytes / target_split_size_bytes)) So a merge that ingests more than one target's worth of data spreads across multiple output files; merges that fit in one target keep producing a single output (preserving today's behavior for the common case). The engine clamps the request to the number of `sorted_series` boundaries actually available, so the value is an upper bound, not an exact count. Plumbing: `IndexerConfig` already carries `target_split_size_bytes` in `ParquetMergePolicyConfig`. Pass that through `ParquetMergePipelineParams.target_split_size_bytes` → `ParquetMergeExecutor::new`. Default for tests: `256 * 1024 * 1024` (matches the production default). Latent multi-output bug fixed at the same time: with n>1, the executor used to assign the planner-supplied `merge_split_id` to **every** output split, which would have collided on the rename to `{split_id}.parquet`. First output keeps the planner ID for observability continuity; subsequent outputs use the fresh IDs generated by `merge_parquet_split_metadata`. Also exposes `quickwit_parquet_engine::merge::streaming::ms7_serial_lock` as `pub` under the `testsuite` feature so cross-crate streaming tests (in `quickwit-indexing`) can serialise against the same global lock the in-crate MS-7 tests use. The streaming engine writes to a process-global atomic on every merge — without shared locking, the existing pipeline streaming-engine test races `store(0)` against other tests' merges. Adds the appropriate `#[allow(clippy::await_holding_lock)]` to the in-crate `test_merge_pipeline_end_to_end_with_streaming_engine_flag` to match. Co-Authored-By: Claude Opus 4.7 (1M context) * test(merge-pipeline): bonus — prefix_len=1 multi-RG inputs + m:n outputs Adds the bonus scenario: three multi-metric inputs each written with `rg_partition_prefix_len = 1` and one row group per distinct metric_name (via `row_group_size = ROWS_PER_METRIC_PER_INPUT` so the writer flushes at every metric boundary after sorting). Merged with a small `ParquetMergePipelineParams::target_split_size_bytes = 500` that forces the executor's `num_outputs` calculation to ask the engine for multiple outputs — exercising the m:n merge path now reachable through the actor pipeline (PR's earlier commit removed the `num_outputs = 1` hardcode). Both engines covered: - `test_prefix_aligned_multi_metric_three_input_multi_output_in_memory_engine` - `test_prefix_aligned_multi_metric_three_input_multi_output_streaming_engine` The streaming-engine variant also asserts `PEAK_BODY_COL_PAGE_CACHE_LEN > 0` (under `ms7_serial_lock`) so a silent fallback to the in-memory path would fail. The shared assertion helper `assert_three_input_three_metric_multi_output_correct` checks the m:n contract end-to-end at the pipeline level: - All three input splits replaced. - ≥ 2 output splits staged (proves splitting happened). - Sum of per-output row counts = total input rows. - Each output internally monotonic on `sorted_series`. - Across outputs, the `sorted_series` partition is disjoint — no two outputs share any key, which is the "non-overlapping output" contract the engine promises. - Union of metric_names / services across outputs = full input set. - Every output has `num_merge_ops = 1`, `row_keys_proto`, and a `metric_name` zonemap regex. To pin the test to exactly one merge (not a cascade of merges over the now-multiple staged outputs), `make_pipeline_params` now takes `max_merge_ops` and the bonus tests set it to `1`: outputs land at `num_merge_ops = 1`, equal to the policy ceiling, and the planner refuses to merge them again. The existing n=1 tests stay at 5 (headroom — they produce a single output that can't trigger another merge anyway, since `merge_factor = 3`). Updates the module doc to drop the now-stale scope note about m:n not being reachable through the pipeline. Co-Authored-By: Claude Opus 4.7 (1M context) * style: nightly rustfmt + drop useless borrows in assert! Reformats doc comments / format strings under nightly rustfmt (`wrap_comments`, `format_strings`), and removes two redundant `&` in `assert!` arguments flagged by clippy. Co-Authored-By: Claude Opus 4.7 (1M context) --------- Co-authored-by: Claude Opus 4.7 (1M context) --- .../001-dual-parquet-merge-engines.md | 178 +++ docs/internals/adr/deviations/README.md | 3 +- .../resources/tests/node_config/quickwit.json | 3 +- .../resources/tests/node_config/quickwit.toml | 1 + .../resources/tests/node_config/quickwit.yaml | 1 + .../quickwit-config/src/node_config/mod.rs | 16 + .../src/node_config/serialize.rs | 1 + .../src/actors/indexing_service.rs | 10 + .../src/actors/parquet_pipeline/mod.rs | 4 + .../parquet_merge_executor.rs | 134 ++- .../parquet_merge_pipeline.rs | 24 +- .../parquet_merge_pipeline_crash_test.rs | 4 + ...arquet_merge_pipeline_multi_metric_test.rs | 1061 +++++++++++++++++ .../parquet_merge_pipeline_sketch_test.rs | 2 + .../parquet_merge_pipeline_test.rs | 211 +++- ...t_merge_pipeline_trace_conformance_test.rs | 4 + .../src/merge/streaming.rs | 75 +- .../src/merge/tests.rs | 472 ++++++++ 18 files changed, 2134 insertions(+), 70 deletions(-) create mode 100644 docs/internals/adr/deviations/001-dual-parquet-merge-engines.md create mode 100644 quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_multi_metric_test.rs diff --git a/docs/internals/adr/deviations/001-dual-parquet-merge-engines.md b/docs/internals/adr/deviations/001-dual-parquet-merge-engines.md new file mode 100644 index 00000000000..b8cd0d3d1d2 --- /dev/null +++ b/docs/internals/adr/deviations/001-dual-parquet-merge-engines.md @@ -0,0 +1,178 @@ +# Deviation 001: Dual Parquet merge engines during streaming-engine rollout + +## Summary + +Two Parquet merge engines coexist in production behind a runtime YAML flag. +The streaming engine (`execute_merge_operation`) matches the intent of +[ADR-003 §4](../003-time-windowed-sorted-compaction.md) (page-granular +streaming, bounded memory). The in-memory engine +(`merge_sorted_parquet_files`) is retained as the runtime fallback so an +operator can flip back via configuration if the streaming engine hits a +production bug. The dual-engine state is intentional and time-bounded — +it ends when the streaming engine has soaked at the new default in +production. + +## Related ADR + +- **ADR**: [ADR-003 Time-Windowed Sorted Compaction](../003-time-windowed-sorted-compaction.md) +- **Section**: §4 Sorted Merge, Phase 2 (column streaming) + +## ADR States + +> Phase 2: Stream columns through the merge. +> +> Once the global sort order is determined, each column is read from the +> input splits and written to the output in sorted order. Columns are +> processed one at a time (or in small groups) for memory efficiency. +> +> For large columns, it may be advantageous to operate at **page +> granularity** rather than loading an entire column from each input: +> read individual Parquet pages from inputs as needed and write +> individual pages to the output. This bounds memory usage for columns +> with large values (e.g., high-cardinality string tags, large attribute +> maps) and avoids materializing an entire column across all inputs +> simultaneously. + +ADR-003 §4 describes the merge as a streaming operation that bounds +memory by reading and writing pages incrementally. The in-memory +`merge_sorted_parquet_files` engine pre-materializes whole columns from +all inputs simultaneously — directly contrary to the ADR's stated +memory model. + +## Current Implementation + +Both engines live in `quickwit-parquet-engine/src/merge/`: + +- **Streaming engine** (`execute_merge_operation`, in `merge/mod.rs`, + backed by `merge/streaming.rs`). Column-major, page-bounded body cache, + reads inputs through `RemoteByteSource`. This is the + ADR-003-compliant implementation. It is the unconditional path for + promotion merges (the in-memory path can't handle mixed + `rg_partition_prefix_len`) and the opt-in path for regular merges. +- **In-memory engine** (`merge_sorted_parquet_files`, in `merge/mod.rs`). + Buffers all inputs through arrow-rs into memory, runs the merge under + `run_cpu_intensive`. This is the original bootstrap implementation + retained as the runtime fallback. + +`ParquetMergeExecutor::handle` routes between them: + +```rust +let is_promotion = scratch.merge_operation.target_prefix_len_override.is_some(); +if is_promotion || self.use_streaming_engine { + execute_merge_operation(&op, sources, &output_dir, &config).await +} else { + run_cpu_intensive(move || { + merge_sorted_parquet_files(&input_paths, &output_dir_clone, &config) + }).await +} +``` + +The `use_streaming_engine` boolean is sourced from the node-level +`IndexerConfig::parquet_merge_use_streaming_engine` YAML field, default +`false`. + +Row-content equivalence between the two engines is enforced by parity +tests in `quickwit-parquet-engine/src/merge/tests.rs::parity`. These +must keep passing as long as both engines coexist. + +## Signal Impact + +Applies to **metrics** (the only signal currently using the Parquet +pipeline). Will apply to **traces** and **logs** when those signals +adopt Parquet splits. The deviation does not affect Tantivy-backed +pipelines. + +## Impact + +| Aspect | ADR Target | Current Reality | +|--------|------------|-----------------| +| Engines in production | One streaming engine | Two (streaming + in-memory) | +| Memory model | Page-bounded; ~constant per column | In-memory engine: O(total input column size) per merge | +| Configuration surface | None — engine choice is internal | One YAML flag (`parquet_merge_use_streaming_engine`) | +| Code surface to maintain | One engine | Two engines + parity tests + routing branch | +| Operator rollback | Not applicable — only one path | Flip flag to `false`, no redeploy needed | + +## Why This Exists + +The streaming engine is new code. ADR-003 describes the target memory +model but does not guarantee bug-free first-deployment behavior. Three +forces produced the dual-engine state: + +1. **Production safety.** The in-memory engine has been the live merge + path during the metrics pipeline's bring-up. Replacing it wholesale + on a single PR, without an in-place fallback, would mean any bug in + the streaming engine requires a redeploy to recover. With the flag, + recovery is `config edit + restart`. +2. **Staged rollout.** Production confidence is built by enabling the + streaming engine on a soak fleet, observing for some time, then + flipping the default. The dual-engine state is the necessary + infrastructure for that rollout. +3. **Parity verifiable, not certain.** The parity tests in + `merge::tests::parity` cover representative synthetic fixtures. + Production data has shapes those fixtures don't cover. The fallback + exists because parity is a strong-but-not-total guarantee. + +## Priority Assessment + +- **PoC / MVP**: acceptable — dual-engine is in fact the deliberate + state during MVP rollout. +- **Production (current)**: acceptable — flag defaults to `false`, + rollout has not begun. The streaming engine is exercised only by + promotion merges (whose execution will start once GAP-011 is closed). +- **Production (post-soak)**: not acceptable. Once the streaming engine + has soaked at default-`true` in production, the in-memory engine + becomes dead code that complicates the merge-executor and obscures + the ADR-003 memory contract. Resolve before merging additional + significant work into `parquet_merge_executor.rs`. + +## Exit Criteria + +The deviation resolves when **all** of the following hold: + +1. `IndexerConfig::default_parquet_merge_use_streaming_engine` defaults + to `true` in `quickwit-config`. +2. At least one production fleet has run with the flag set to `true` for + a soak window of ≥ 2 weeks with no merge-correctness incidents (no + data loss, no schema mismatch, no merge-output-rows-≠-input-rows + alerts). +3. No deviation-resolving rollback has been issued during the soak. + +When those are met, the follow-up PR: + +- Deletes `merge_sorted_parquet_files` from `quickwit-parquet-engine`. +- Deletes the in-memory branch in `ParquetMergeExecutor::handle`. +- Deletes the `use_streaming_engine` field on `ParquetMergeExecutor` and + `ParquetMergePipelineParams`. +- Deletes `IndexerConfig::parquet_merge_use_streaming_engine`. +- Deletes `merge::tests::parity` (both engines no longer exist to + compare). +- Closes this deviation. + +## Work Required to Match ADR + +| Change | Difficulty | Description | +|--------|------------|-------------| +| Flip default to `true` | Trivial | One-line change in `IndexerConfig::default_parquet_merge_use_streaming_engine`. Lands after soak. | +| Production soak | Operational | Run with `true` on at least one fleet for ≥ 2 weeks, monitor merge-correctness signals. | +| Delete in-memory engine | Moderate | Remove `merge_sorted_parquet_files`, the fallback branch, the flag, and the parity tests. Mechanically straightforward but touches several call sites. | + +## Recommendation + +**Accept for now.** The dual-engine state is the deliberate output of a +flag-with-fallback rollout pattern (see commit history of #6441 and +related PRs). Resolution is a known follow-up, not technical debt that +needs to be paid down ahead of schedule. + +Track the exit criteria in this doc. When all three conditions hold, +open the deletion PR and close this deviation. + +## References + +- [ADR-003 Time-Windowed Sorted Compaction](../003-time-windowed-sorted-compaction.md) §4 +- [GAP-011 No legacy promotion planner](../gaps/011-no-legacy-promotion-planner.md) +- [GAP-012 Merge downloads instead of streaming](../gaps/012-merge-downloads-instead-of-streaming.md) +- PR #6441 (wire-in of the YAML flag) + +## Date + +2026-05-18 diff --git a/docs/internals/adr/deviations/README.md b/docs/internals/adr/deviations/README.md index 000c2e5e0e5..2ddb1dce09f 100644 --- a/docs/internals/adr/deviations/README.md +++ b/docs/internals/adr/deviations/README.md @@ -94,8 +94,7 @@ Deviation files use sequential numbering: `001-short-description.md` | Deviation | Title | Related ADR | Priority | |-----------|-------|-------------|----------| - -*No deviations recorded yet.* +| [001](./001-dual-parquet-merge-engines.md) | Dual Parquet merge engines during streaming-engine rollout | [ADR-003](../003-time-windowed-sorted-compaction.md) | Accept until post-soak | ## Lifecycle diff --git a/quickwit/quickwit-config/resources/tests/node_config/quickwit.json b/quickwit/quickwit-config/resources/tests/node_config/quickwit.json index 7269b37ae22..6909594396b 100644 --- a/quickwit/quickwit-config/resources/tests/node_config/quickwit.json +++ b/quickwit/quickwit-config/resources/tests/node_config/quickwit.json @@ -54,7 +54,8 @@ "split_store_max_num_splits": 10000, "max_concurrent_split_uploads": 8, "max_merge_write_throughput": "100mb", - "merge_concurrency": 2 + "merge_concurrency": 2, + "parquet_merge_use_streaming_engine": true }, "ingest_api": { "replication_factor": 2 diff --git a/quickwit/quickwit-config/resources/tests/node_config/quickwit.toml b/quickwit/quickwit-config/resources/tests/node_config/quickwit.toml index ea715dcffe0..80d3ec28867 100644 --- a/quickwit/quickwit-config/resources/tests/node_config/quickwit.toml +++ b/quickwit/quickwit-config/resources/tests/node_config/quickwit.toml @@ -45,6 +45,7 @@ split_store_max_num_splits = 10_000 max_concurrent_split_uploads = 8 max_merge_write_throughput = "100mb" merge_concurrency = 2 +parquet_merge_use_streaming_engine = true [ingest_api] replication_factor = 2 diff --git a/quickwit/quickwit-config/resources/tests/node_config/quickwit.yaml b/quickwit/quickwit-config/resources/tests/node_config/quickwit.yaml index face0852972..2c33cc367c0 100644 --- a/quickwit/quickwit-config/resources/tests/node_config/quickwit.yaml +++ b/quickwit/quickwit-config/resources/tests/node_config/quickwit.yaml @@ -49,6 +49,7 @@ indexer: max_concurrent_split_uploads: 8 max_merge_write_throughput: 100mb merge_concurrency: 2 + parquet_merge_use_streaming_engine: true ingest_api: replication_factor: 2 diff --git a/quickwit/quickwit-config/src/node_config/mod.rs b/quickwit/quickwit-config/src/node_config/mod.rs index cf67768966d..fff19960e57 100644 --- a/quickwit/quickwit-config/src/node_config/mod.rs +++ b/quickwit/quickwit-config/src/node_config/mod.rs @@ -163,6 +163,16 @@ pub struct IndexerConfig { pub enable_cooperative_indexing: bool, #[serde(default = "IndexerConfig::default_cpu_capacity")] pub cpu_capacity: CpuCapacity, + /// If true, run Parquet merges through the streaming column-major engine + /// (`execute_merge_operation`). If false (default), use the in-memory + /// `merge_sorted_parquet_files` engine. The legacy in-memory engine is + /// kept as the runtime fallback so production can flip back to it + /// without redeploying if the streaming engine hits a bug. Promotion + /// merges (those with `target_prefix_len_override`) always go through + /// the streaming engine regardless of this flag — the in-memory path + /// can't handle mixed prefix lengths. + #[serde(default = "IndexerConfig::default_parquet_merge_use_streaming_engine")] + pub parquet_merge_use_streaming_engine: bool, } impl IndexerConfig { @@ -201,6 +211,10 @@ impl IndexerConfig { CpuCapacity::one_cpu_thread() * (quickwit_common::num_cpus() as u32) } + fn default_parquet_merge_use_streaming_engine() -> bool { + false + } + #[cfg(any(test, feature = "testsuite"))] pub fn for_test() -> anyhow::Result { use quickwit_proto::indexing::PIPELINE_FULL_CAPACITY; @@ -213,6 +227,7 @@ impl IndexerConfig { cpu_capacity: PIPELINE_FULL_CAPACITY * 4u32, max_merge_write_throughput: None, merge_concurrency: NonZeroUsize::new(3).unwrap(), + parquet_merge_use_streaming_engine: Self::default_parquet_merge_use_streaming_engine(), }; Ok(indexer_config) } @@ -229,6 +244,7 @@ impl Default for IndexerConfig { cpu_capacity: Self::default_cpu_capacity(), merge_concurrency: Self::default_merge_concurrency(), max_merge_write_throughput: None, + parquet_merge_use_streaming_engine: Self::default_parquet_merge_use_streaming_engine(), } } } diff --git a/quickwit/quickwit-config/src/node_config/serialize.rs b/quickwit/quickwit-config/src/node_config/serialize.rs index ae19a92c45f..c7740e7146e 100644 --- a/quickwit/quickwit-config/src/node_config/serialize.rs +++ b/quickwit/quickwit-config/src/node_config/serialize.rs @@ -657,6 +657,7 @@ mod tests { cpu_capacity: IndexerConfig::default_cpu_capacity(), enable_cooperative_indexing: false, max_merge_write_throughput: Some(ByteSize::mb(100)), + parquet_merge_use_streaming_engine: true, } ); assert_eq!( diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index 3f0bd91c301..4453ffbfe00 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -113,6 +113,12 @@ pub struct IndexingService { counters: IndexingServiceCounters, local_split_store: Arc, pub(crate) max_concurrent_split_uploads: usize, + /// Cached from `IndexerConfig`. Selects whether new Parquet merge + /// pipelines route regular merges through the streaming engine or + /// the in-memory fallback. Promotion merges always use the + /// streaming engine regardless of this flag. + #[cfg(feature = "metrics")] + pub(crate) parquet_merge_use_streaming_engine: bool, merge_pipeline_handles: HashMap, #[cfg(feature = "metrics")] parquet_merge_pipeline_handles: HashMap, @@ -178,6 +184,8 @@ impl IndexingService { indexing_pipelines: Default::default(), counters: Default::default(), max_concurrent_split_uploads: indexer_config.max_concurrent_split_uploads, + #[cfg(feature = "metrics")] + parquet_merge_use_streaming_engine: indexer_config.parquet_merge_use_streaming_engine, merge_pipeline_handles: HashMap::new(), #[cfg(feature = "metrics")] parquet_merge_pipeline_handles: HashMap::new(), @@ -723,6 +731,8 @@ impl IndexingService { max_concurrent_split_uploads: self.max_concurrent_split_uploads, event_broker: self.event_broker.clone(), writer_config, + use_streaming_engine: self.parquet_merge_use_streaming_engine, + target_split_size_bytes: cfg.target_split_size_bytes, }; let pipeline = super::parquet_pipeline::ParquetMergePipeline::new( diff --git a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/mod.rs b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/mod.rs index 65ebe3a5333..fc598889d81 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/mod.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/mod.rs @@ -61,6 +61,10 @@ mod parquet_merge_pipeline_trace_conformance_test; #[allow(clippy::disallowed_methods)] mod parquet_merge_pipeline_sketch_test; +#[cfg(test)] +#[allow(clippy::disallowed_methods)] +mod parquet_merge_pipeline_multi_metric_test; + pub use parquet_doc_processor::{ ParquetDocProcessor, ParquetDocProcessorCounters, ParquetDocProcessorError, is_arrow_ipc, }; diff --git a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_executor.rs b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_executor.rs index d72884236fd..ffe27b09a0e 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_executor.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_executor.rs @@ -14,21 +14,27 @@ //! Parquet merge executor actor. //! -//! Receives a `ParquetMergeScratch` from the downloader. Two merge -//! paths, chosen on `target_prefix_len_override`: +//! Receives a `ParquetMergeScratch` from the downloader. Two engines are available: //! -//! - **Regular merge** (`target_prefix_len_override == None`): the input splits already share -//! `qh.rg_partition_prefix_len` (MP-3). Run the in-memory `merge_sorted_parquet_files` engine -//! inside `run_cpu_intensive()`. -//! - **Promotion merge** (`target_prefix_len_override == Some(target)`): the inputs may carry -//! *different* prefix lengths (the whole point of promotion). The in-memory path's -//! `extract_and_validate_input_metadata` would bail on the mismatch before any output is -//! produced, so promotion runs through `execute_merge_operation` instead — that path opens each -//! input through `LegacyInputAdapter` for splits below the target prefix len and through -//! `StreamingParquetReader` for splits at the target, so the streaming engine sees a homogeneous -//! stream advertising `prefix_len = target` on every input. `mixed_prefix_ok` is then passed to -//! `merge_parquet_split_metadata` so the post-merge aggregator skips the input-side equality -//! check. +//! - **Streaming engine** (`execute_merge_operation`): column-major, page-bounded body cache. Used +//! unconditionally for promotion merges (the in-memory path can't handle mixed prefix lengths). +//! Optionally used for regular merges when the node-level +//! `IndexerConfig::parquet_merge_use_streaming_engine` flag is true. +//! - **In-memory engine** (`merge_sorted_parquet_files`): buffers all inputs in memory and runs +//! inside `run_cpu_intensive`. Kept as the runtime fallback so production can flip back via YAML +//! config if the streaming engine hits a bug. To be removed once the streaming path has soaked. +//! +//! Routing in `handle()`: +//! +//! - `target_prefix_len_override.is_some()` → streaming engine. Promotion is the whole point of +//! `target_prefix_len_override`, and the in-memory path's `extract_and_validate_input_metadata` +//! would bail on the mixed `rg_partition_prefix_len` before any output is produced. +//! - Else `use_streaming_engine == true` → streaming engine (the new default once soaked). +//! - Else → in-memory engine (the runtime fallback). +//! +//! `mixed_prefix_ok` is passed to `merge_parquet_split_metadata` only for promotion merges so +//! the post-merge aggregator's strict input-side equality check stays on for ordinary +//! same-prefix merges. use std::io; use std::ops::Range; @@ -117,18 +123,58 @@ impl RemoteByteSource for LocalFileByteSource { pub struct ParquetMergeExecutor { uploader_mailbox: Mailbox, writer_config: ParquetWriterConfig, + /// When true, regular merges run through the streaming engine. When + /// false, they run through the in-memory `merge_sorted_parquet_files` + /// fallback. Promotion merges always use the streaming engine + /// regardless of this flag. + use_streaming_engine: bool, + /// Target output split size, sourced from the merge policy. Drives + /// `num_outputs`: the executor asks the merge engine for + /// `ceil(total_input_bytes / target_split_size_bytes)` outputs so a + /// merge that ingests more than one target's worth of data spreads + /// across multiple output files. The engine clamps the request to + /// the number of `sorted_series` boundaries actually available, so + /// the result is an upper bound rather than an exact count. + /// Operations whose inputs already fit in one target naturally get + /// `num_outputs = 1`. + target_split_size_bytes: u64, } impl ParquetMergeExecutor { pub fn new( uploader_mailbox: Mailbox, writer_config: ParquetWriterConfig, + use_streaming_engine: bool, + target_split_size_bytes: u64, ) -> Self { Self { uploader_mailbox, writer_config, + use_streaming_engine, + target_split_size_bytes, } } + + /// Compute the requested `num_outputs` for a merge. Returns at + /// least 1 (the merge always produces at least one output unless + /// every input is empty). Guards against the + /// `target_split_size_bytes = 0` misconfiguration — falling back to + /// 1 rather than dividing by zero. + fn compute_num_outputs(&self, total_input_bytes: u64) -> usize { + if self.target_split_size_bytes == 0 { + return 1; + } + // Ceiling division without overflow: `(a + b - 1) / b` would + // overflow at u64::MAX; use the explicit add-one-after-divide + // form when `a` is non-zero. + let quot = total_input_bytes / self.target_split_size_bytes; + let extra = if total_input_bytes.is_multiple_of(self.target_split_size_bytes) { + 0 + } else { + 1 + }; + ((quot + extra) as usize).max(1) + } } #[async_trait] @@ -176,18 +222,29 @@ impl Handler for ParquetMergeExecutor { .context("failed to create merge output directory") .map_err(|e| ActorExitStatus::from(anyhow::anyhow!(e)))?; - // Route promotion merges (`target_prefix_len_override.is_some()`) - // through `execute_merge_operation`. That path opens each input - // via `LegacyInputAdapter` when the split's prefix_len is below - // the target, so the streaming merge engine sees a homogeneous - // stream with all inputs at `prefix_len = target`. Without this - // routing the in-memory `merge_sorted_parquet_files` would bail - // in `extract_and_validate_input_metadata` on mixed prefix - // lengths before any output is produced. - let merge_result: Result, _>, _> = if scratch - .merge_operation - .target_prefix_len_override - .is_some() + // Promotion merges (`target_prefix_len_override.is_some()`) must + // use the streaming engine; the in-memory path's + // `extract_and_validate_input_metadata` would bail on mixed + // `rg_partition_prefix_len` before producing any output. Regular + // merges follow the operator-controlled `use_streaming_engine` + // flag: true means the streaming engine, false means the + // in-memory fallback. Keeping the in-memory branch lets + // production flip back at runtime if the streaming engine hits a + // bug; once the streaming path has soaked, the in-memory branch + // and `merge_sorted_parquet_files` itself can be removed. + let total_input_bytes = scratch.merge_operation.total_size_bytes(); + let num_outputs = self.compute_num_outputs(total_input_bytes); + info!( + merge_split_id = %merge_split_id, + total_input_bytes, + target_split_size_bytes = self.target_split_size_bytes, + num_outputs, + "computed num_outputs from total input bytes / target split size" + ); + + let is_promotion = scratch.merge_operation.target_prefix_len_override.is_some(); + let merge_result: Result, _>, _> = if is_promotion + || self.use_streaming_engine { let sources: Vec> = scratch .downloaded_parquet_files @@ -195,7 +252,7 @@ impl Handler for ParquetMergeExecutor { .map(|path| Arc::new(LocalFileByteSource::new(path.clone())) as _) .collect(); let config = MergeConfig { - num_outputs: 1, + num_outputs, writer_config: self.writer_config.clone(), }; Ok( @@ -203,13 +260,15 @@ impl Handler for ParquetMergeExecutor { .await, ) } else { - // Regular merge: in-memory path under `run_cpu_intensive`. + // Fallback: in-memory engine under `run_cpu_intensive`. + // Kept as the runtime rollback target while the streaming + // engine soaks in production. let input_paths = scratch.downloaded_parquet_files.clone(); let output_dir_clone = output_dir.clone(); let writer_config = self.writer_config.clone(); run_cpu_intensive(move || { let config = MergeConfig { - num_outputs: 1, + num_outputs, writer_config, }; merge_sorted_parquet_files(&input_paths, &output_dir_clone, &config) @@ -336,16 +395,21 @@ impl Handler for ParquetMergeExecutor { let mixed_prefix_ok = scratch.merge_operation.target_prefix_len_override.is_some(); let mut merged_splits = Vec::with_capacity(outputs.len()); - for output in &outputs { + // First output keeps the planner-assigned `merge_split_id` so + // existing observability paths (logs, metrics, traces keyed on + // this ID) continue to see the planned ID at execute time. For + // n>1 the subsequent outputs get fresh IDs generated by + // `merge_parquet_split_metadata`. Assigning the same ID to + // multiple outputs would collide on the rename below and + // overwrite earlier files. + for (output_idx, output) in outputs.iter().enumerate() { let mut metadata = merge_parquet_split_metadata(input_splits, output, mixed_prefix_ok) .context("failed to build merge output metadata") .map_err(|e| ActorExitStatus::from(anyhow::anyhow!(e)))?; - // Use the split ID that was assigned when the merge operation was - // planned, rather than the one generated inside - // merge_parquet_split_metadata(). This keeps the ID consistent - // across scheduling, tracing, and the final published split. - metadata.split_id = scratch.merge_operation.merge_split_id.clone(); + if output_idx == 0 { + metadata.split_id = scratch.merge_operation.merge_split_id.clone(); + } metadata.parquet_file = metadata.split_id.to_string() + ".parquet"; // Rename the output file to match the split ID. diff --git a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline.rs b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline.rs index 7794e3f14ed..e028dad0f3a 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline.rs @@ -309,8 +309,12 @@ impl ParquetMergePipeline { .spawn(merge_uploader); // 4. Merge executor - let merge_executor = - ParquetMergeExecutor::new(merge_uploader_mailbox, self.params.writer_config.clone()); + let merge_executor = ParquetMergeExecutor::new( + merge_uploader_mailbox, + self.params.writer_config.clone(), + self.params.use_streaming_engine, + self.params.target_split_size_bytes, + ); let (merge_executor_mailbox, merge_executor_handle) = ctx .spawn_actor() .set_kill_switch(self.kill_switch.clone()) @@ -601,6 +605,20 @@ pub struct ParquetMergePipelineParams { /// Should match the ingest pipeline's writer config so merged files have /// consistent compression. pub writer_config: quickwit_parquet_engine::storage::ParquetWriterConfig, + /// When true, regular merges run through the streaming engine + /// (`execute_merge_operation`); when false, they run through the + /// in-memory `merge_sorted_parquet_files` fallback. Promotion + /// merges always use the streaming engine. Sourced from + /// `IndexerConfig::parquet_merge_use_streaming_engine`. + pub use_streaming_engine: bool, + /// Target output split size in bytes, sourced from the merge + /// policy. The executor uses this to compute `num_outputs` from + /// total input size, so a merge that ingests more than one + /// target's worth of data is allowed to spread across multiple + /// output files. Smaller values increase output split count + /// (subject to the number of `sorted_series` boundaries actually + /// available in the input). + pub target_split_size_bytes: u64, } #[cfg(test)] @@ -646,6 +664,8 @@ mod tests { max_concurrent_split_uploads: 4, event_broker: EventBroker::default(), writer_config: quickwit_parquet_engine::storage::ParquetWriterConfig::default(), + use_streaming_engine: false, + target_split_size_bytes: 256 * 1024 * 1024, } } diff --git a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_crash_test.rs b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_crash_test.rs index 050c922e8d6..b5f39ec4733 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_crash_test.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_crash_test.rs @@ -203,6 +203,8 @@ async fn test_merge_pipeline_crash_and_restart() { max_concurrent_split_uploads: 4, event_broker: EventBroker::default(), writer_config: ParquetWriterConfig::default(), + use_streaming_engine: false, + target_split_size_bytes: 256 * 1024 * 1024, }; let pipeline = ParquetMergePipeline::new(params, Some(initial_splits), universe.spawn_ctx()); @@ -326,6 +328,8 @@ async fn test_merge_pipeline_multi_round() { max_concurrent_split_uploads: 4, event_broker: EventBroker::default(), writer_config: ParquetWriterConfig::default(), + use_streaming_engine: false, + target_split_size_bytes: 256 * 1024 * 1024, }; let pipeline = ParquetMergePipeline::new(params, Some(initial_splits), universe.spawn_ctx()); diff --git a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_multi_metric_test.rs b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_multi_metric_test.rs new file mode 100644 index 00000000000..676eb578965 --- /dev/null +++ b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_multi_metric_test.rs @@ -0,0 +1,1061 @@ +// 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. + +//! End-to-end pipeline tests exercising the merge engines on multi-input, +//! multi-metric, multi-row-group fixtures, in both n=1 and m:n (n > 1) +//! output configurations. Complements `parquet_merge_pipeline_test.rs` +//! (which covers the simpler two-input, one-metric-per-input case) with +//! the harder scenarios: +//! +//! - **Three inputs**, each carrying **three metrics** (`aaa.alpha`, `bbb.beta`, `ccc.gamma`). +//! Across inputs, the metrics overlap and the per-metric timeseries IDs collide (each row's +//! `timeseries_id` is derived from the metric name, so input-x, input-y, input-z all share the +//! same set of IDs per metric). Timestamps within each (metric, timeseries) overlap across inputs +//! but are unique — the merge must interleave rows from all three inputs heavily, not concatenate +//! them. +//! - **Multi-row-group output** via `ParquetWriterConfig::row_group_size = 50` on the n=1 tests, so +//! the 180-row merge output breaks into 4 row groups. Exercises the writer's multi-RG path in +//! both engines. +//! - **Multi-row-group inputs with `rg_partition_prefix_len = 1`** in the bonus tests +//! (`write_prefix_aligned_input`): the writer flushes one row group per distinct `metric_name`, +//! so each input file carries three row groups in alignment with the sort prefix. The streaming +//! engine reads these through its prefix-aware fast path. +//! - **m:n merges** in the bonus tests: a small +//! `ParquetMergePipelineParams::target_split_size_bytes` forces the executor to ask the engine +//! for `num_outputs > 1`. The bonus assertions cover the multi-output contract — +//! sum-equals-total, internal monotonicity, inter-output `sorted_series` disjointness, and +//! union-equals-full-set on metrics/services. +//! +//! Both `ParquetMergePipelineParams::use_streaming_engine = false` (the +//! in-memory engine) and `= true` (the streaming engine) are exercised +//! across all four scenarios (n=1 × {prefix_len=0}, n>1 × {prefix_len=1}). +//! Streaming-engine variants additionally assert +//! `PEAK_BODY_COL_PAGE_CACHE_LEN > 0` to confirm the flag routed through +//! the streaming path and not the in-memory fallback. + +use std::collections::HashSet; +use std::path::Path; +use std::sync::Arc; +use std::sync::atomic::Ordering; +use std::time::Duration; + +use arrow::record_batch::RecordBatch; +use quickwit_actors::Universe; +use quickwit_common::pubsub::EventBroker; +use quickwit_common::temp_dir::TempDirectory; +use quickwit_common::test_utils::wait_until_predicate; +use quickwit_metastore::StageParquetSplitsRequestExt; +use quickwit_parquet_engine::merge::policy::{ + ConstWriteAmplificationParquetMergePolicy, ParquetMergePolicyConfig, +}; +use quickwit_parquet_engine::sorted_series::SORTED_SERIES_COLUMN; +use quickwit_parquet_engine::split::{ParquetSplitId, ParquetSplitMetadata, TimeRange}; +use quickwit_parquet_engine::storage::{ParquetWriter, ParquetWriterConfig}; +use quickwit_parquet_engine::table_config::TableConfig; +use quickwit_proto::metastore::{ + EmptyResponse, MetastoreServiceClient, MockMetastoreService, StageMetricsSplitsRequest, +}; +use quickwit_storage::{RamStorage, Storage}; + +use super::parquet_merge_pipeline::{ParquetMergePipeline, ParquetMergePipelineParams}; +use super::parquet_merge_pipeline_test::{ + create_custom_test_batch, extract_binary_column, extract_string_column, extract_u64_column, + read_parquet_from_bytes, write_test_parquet_file, +}; + +// --------------------------------------------------------------------------- +// Multi-metric fixture +// --------------------------------------------------------------------------- + +/// Per-metric specification for one input file: `(metric_name, ts_start, num_rows)`. +/// All rows for a given metric share the same `service` / `host` tag values +/// across the file. +type MetricSpec<'a> = (&'a str, u64, usize); + +/// Concatenate per-metric `RecordBatch`es into a single batch covering several +/// metrics. The writer downstream sorts the rows before writing, so the +/// caller does not need to sort. +fn create_multi_metric_batch( + metrics: &[MetricSpec<'_>], + service_val: &str, + host_val: &str, +) -> RecordBatch { + let batches: Vec = metrics + .iter() + .map(|(metric, ts_start, num_rows)| { + create_custom_test_batch(metric, *ts_start, *num_rows, service_val, host_val) + }) + .collect(); + let schema = batches[0].schema(); + arrow::compute::concat_batches(&schema, &batches).expect("concat multi-metric batches") +} + +/// Build a `ParquetSplitMetadata` advertising multiple metric names and a +/// caller-supplied row count + time range. `prefix_len` controls the +/// `rg_partition_prefix_len` stamped into the input file — 0 for legacy +/// inputs, 1 for prefix-aligned inputs (each row group covers exactly one +/// `metric_name`). +fn make_multi_metric_split_metadata( + split_id: &str, + num_rows: u64, + size_bytes: u64, + ts_start: u64, + ts_end: u64, + metric_names: &[&str], + prefix_len: u32, +) -> ParquetSplitMetadata { + let table_config = TableConfig::default(); + let mut builder = ParquetSplitMetadata::metrics_builder() + .split_id(ParquetSplitId::new(split_id)) + .index_uid("test-merge-index-multi:00000000000000000000000001") + .partition_id(0) + .time_range(TimeRange::new(ts_start, ts_end)) + .num_rows(num_rows) + .size_bytes(size_bytes) + .sort_fields(table_config.effective_sort_fields()) + .window_start_secs(0) + .window_duration_secs(900) + .rg_partition_prefix_len(prefix_len); + for metric in metric_names { + builder = builder.add_metric_name(*metric); + } + builder.build() +} + +/// Write a multi-metric input file with `rg_partition_prefix_len = 1` and +/// one row group per distinct metric. Picks `row_group_size = per-metric row +/// count` so the writer naturally flushes at every metric boundary after +/// sorting — each row group ends up containing exactly one distinct +/// `metric_name`, satisfying the prefix-alignment invariant the writer +/// validates on close. +/// +/// Returns the file size in bytes (the caller stamps this back into the +/// `ParquetSplitMetadata.size_bytes` field for the planner / executor). +fn write_prefix_aligned_input( + dir: &Path, + filename: &str, + batch: &RecordBatch, + split_metadata: &ParquetSplitMetadata, + rows_per_metric: usize, +) -> u64 { + let table_config = TableConfig::default(); + let writer_config = ParquetWriterConfig::default().with_row_group_size(rows_per_metric); + let writer = ParquetWriter::new(writer_config, &table_config) + .expect("test ParquetWriter (prefix-aligned)"); + let path = dir.join(filename); + let (file_size, _write_metadata) = writer + .write_to_file_with_metadata(batch, &path, Some(split_metadata)) + .expect("write_to_file_with_metadata for prefix-aligned input"); + file_size +} + +/// Three canonical metric names sorted alphabetically. Picked so the sort +/// order is unambiguous (every byte-comparison resolves on the first +/// distinguishing character of the metric name). +const METRIC_AAA: &str = "aaa.alpha"; +const METRIC_BBB: &str = "bbb.beta"; +const METRIC_CCC: &str = "ccc.gamma"; + +/// Per-metric row count in each input. With three inputs and three metrics, +/// the total input/output row count is `3 * 3 * ROWS_PER_METRIC = 180`. +const ROWS_PER_METRIC_PER_INPUT: usize = 20; + +/// Total rows across all three inputs. Held constant across all tests in this +/// module so per-test expectations stay self-consistent. +const TOTAL_INPUT_ROWS: u64 = 3 * 3 * ROWS_PER_METRIC_PER_INPUT as u64; + +/// Output writer's row-group size. Picked small enough that the n=1 merge +/// (180 rows) produces 4 row groups and each n=3 output (~60 rows) produces +/// 2. Exercises the multi-row-group write path in both engines without +/// triggering prefix-alignment validation (we do not set +/// `rg_partition_prefix_len` on inputs). +const TEST_OUTPUT_ROW_GROUP_SIZE: usize = 50; + +/// Three inputs that share metric names and per-metric `timeseries_id` +/// ranges (every row's `timeseries_id` is derived from the metric name +/// hash + per-input row index, so cross-input collisions for the same +/// `(metric_name, row_index_within_metric)` are intentional). Timestamps +/// are picked so each (metric, timeseries) appears in all three inputs at +/// three distinct overlapping timestamps — the merge must interleave row +/// streams from all three inputs, not concatenate them. +/// +/// Returns `(file_paths, splits_metadata)` for direct use by the pipeline +/// (one entry per input, paths uploaded to the supplied storage). +async fn build_three_multi_metric_inputs( + temp_dir: &Path, + ram_storage: &Arc, +) -> (Vec, Vec) { + // input-x: every metric at ts 100..120 with service=web, host=h1. + let batch_x = create_multi_metric_batch( + &[ + (METRIC_AAA, 100, ROWS_PER_METRIC_PER_INPUT), + (METRIC_BBB, 100, ROWS_PER_METRIC_PER_INPUT), + (METRIC_CCC, 100, ROWS_PER_METRIC_PER_INPUT), + ], + "web", + "h1", + ); + + // input-y: every metric at ts 110..130 with service=api, host=h2. + let batch_y = create_multi_metric_batch( + &[ + (METRIC_AAA, 110, ROWS_PER_METRIC_PER_INPUT), + (METRIC_BBB, 110, ROWS_PER_METRIC_PER_INPUT), + (METRIC_CCC, 110, ROWS_PER_METRIC_PER_INPUT), + ], + "api", + "h2", + ); + + // input-z: every metric at ts 120..140 with service=db, host=h3. + let batch_z = create_multi_metric_batch( + &[ + (METRIC_AAA, 120, ROWS_PER_METRIC_PER_INPUT), + (METRIC_BBB, 120, ROWS_PER_METRIC_PER_INPUT), + (METRIC_CCC, 120, ROWS_PER_METRIC_PER_INPUT), + ], + "db", + "h3", + ); + + let mut paths = Vec::new(); + let mut splits = Vec::new(); + for (split_id, batch, ts_start, ts_end, service, host) in [ + ("split-x", &batch_x, 100, 120, "web", "h1"), + ("split-y", &batch_y, 110, 130, "api", "h2"), + ("split-z", &batch_z, 120, 140, "db", "h3"), + ] { + let _ = (service, host); + let filename = format!("{split_id}.parquet"); + let num_rows = (3 * ROWS_PER_METRIC_PER_INPUT) as u64; + let meta = make_multi_metric_split_metadata( + split_id, + num_rows, + 0, // size_bytes filled in below + ts_start, + ts_end, + &[METRIC_AAA, METRIC_BBB, METRIC_CCC], + 0, // prefix_len = 0: legacy default, no per-RG alignment claim + ); + let size_bytes = write_test_parquet_file(temp_dir, &filename, batch, &meta); + // Re-build metadata now that size_bytes is known. Mirrors what the + // simpler test does — keep both fields self-consistent. + let meta = { + let mut m = make_multi_metric_split_metadata( + split_id, + num_rows, + size_bytes, + ts_start, + ts_end, + &[METRIC_AAA, METRIC_BBB, METRIC_CCC], + 0, + ); + m.parquet_file = filename.clone(); + m + }; + let bytes_on_disk = std::fs::read(temp_dir.join(&filename)).unwrap(); + ram_storage + .put(Path::new(&filename), Box::new(bytes_on_disk)) + .await + .unwrap(); + paths.push(temp_dir.join(&filename)); + splits.push(meta); + } + (paths, splits) +} + +/// Same shape and content as `build_three_multi_metric_inputs`, but each +/// input is written with `rg_partition_prefix_len = 1` and one row group +/// per distinct metric. With sort schema `metric_name | ...` and +/// `row_group_size = ROWS_PER_METRIC_PER_INPUT`, the writer naturally +/// flushes a row group every `ROWS_PER_METRIC_PER_INPUT` rows after +/// sorting — those flush boundaries align with metric_name transitions, +/// so each row group contains rows for exactly one distinct +/// `metric_name`. The writer's prefix-alignment self-check passes, and +/// the streaming engine reads the inputs as prefix_len=1 multi-row-group +/// files. +async fn build_three_prefix_aligned_multi_metric_inputs( + temp_dir: &Path, + ram_storage: &Arc, +) -> (Vec, Vec) { + let batch_x = create_multi_metric_batch( + &[ + (METRIC_AAA, 100, ROWS_PER_METRIC_PER_INPUT), + (METRIC_BBB, 100, ROWS_PER_METRIC_PER_INPUT), + (METRIC_CCC, 100, ROWS_PER_METRIC_PER_INPUT), + ], + "web", + "h1", + ); + let batch_y = create_multi_metric_batch( + &[ + (METRIC_AAA, 110, ROWS_PER_METRIC_PER_INPUT), + (METRIC_BBB, 110, ROWS_PER_METRIC_PER_INPUT), + (METRIC_CCC, 110, ROWS_PER_METRIC_PER_INPUT), + ], + "api", + "h2", + ); + let batch_z = create_multi_metric_batch( + &[ + (METRIC_AAA, 120, ROWS_PER_METRIC_PER_INPUT), + (METRIC_BBB, 120, ROWS_PER_METRIC_PER_INPUT), + (METRIC_CCC, 120, ROWS_PER_METRIC_PER_INPUT), + ], + "db", + "h3", + ); + + let mut paths = Vec::new(); + let mut splits = Vec::new(); + for (split_id, batch, ts_start, ts_end) in [ + ("split-px", &batch_x, 100, 120), + ("split-py", &batch_y, 110, 130), + ("split-pz", &batch_z, 120, 140), + ] { + let filename = format!("{split_id}.parquet"); + let num_rows = (3 * ROWS_PER_METRIC_PER_INPUT) as u64; + let meta = make_multi_metric_split_metadata( + split_id, + num_rows, + 0, + ts_start, + ts_end, + &[METRIC_AAA, METRIC_BBB, METRIC_CCC], + 1, // prefix_len = 1: one row group per metric_name. + ); + let size_bytes = write_prefix_aligned_input( + temp_dir, + &filename, + batch, + &meta, + ROWS_PER_METRIC_PER_INPUT, + ); + let meta = { + let mut m = make_multi_metric_split_metadata( + split_id, + num_rows, + size_bytes, + ts_start, + ts_end, + &[METRIC_AAA, METRIC_BBB, METRIC_CCC], + 1, + ); + m.parquet_file = filename.clone(); + m + }; + let bytes_on_disk = std::fs::read(temp_dir.join(&filename)).unwrap(); + ram_storage + .put(Path::new(&filename), Box::new(bytes_on_disk)) + .await + .unwrap(); + paths.push(temp_dir.join(&filename)); + splits.push(meta); + } + (paths, splits) +} + +// --------------------------------------------------------------------------- +// Mock metastore plumbing (captures staged + published state) +// --------------------------------------------------------------------------- + +/// Handles returned by `mount_capturing_metastore`: caller waits on +/// `publish_called`, then reads `staged_metadata` / `replaced_ids`. +struct MetastoreCapture { + metastore: MetastoreServiceClient, + staged_metadata: Arc>>, + replaced_ids: Arc>>, + publish_called: Arc, +} + +fn mount_capturing_metastore() -> MetastoreCapture { + let mut mock_metastore = MockMetastoreService::new(); + + let staged_metadata: Arc>> = + Arc::new(std::sync::Mutex::new(Vec::new())); + let staged_metadata_clone = staged_metadata.clone(); + mock_metastore.expect_stage_metrics_splits().returning( + move |request: StageMetricsSplitsRequest| { + let splits = request + .deserialize_splits_metadata() + .expect("failed to deserialize staged metadata"); + staged_metadata_clone.lock().unwrap().extend(splits); + Ok(EmptyResponse {}) + }, + ); + + let publish_called = Arc::new(std::sync::atomic::AtomicBool::new(false)); + let publish_called_clone = publish_called.clone(); + let replaced_ids = Arc::new(std::sync::Mutex::new(Vec::::new())); + let replaced_ids_clone = replaced_ids.clone(); + mock_metastore + .expect_publish_metrics_splits() + .returning(move |request| { + replaced_ids_clone + .lock() + .unwrap() + .extend(request.replaced_split_ids.clone()); + publish_called_clone.store(true, Ordering::SeqCst); + Ok(EmptyResponse {}) + }); + + MetastoreCapture { + metastore: MetastoreServiceClient::from_mock(mock_metastore), + staged_metadata, + replaced_ids, + publish_called, + } +} + +// --------------------------------------------------------------------------- +// Pipeline-params builder +// --------------------------------------------------------------------------- + +fn make_pipeline_params( + universe: &Universe, + metastore: MetastoreServiceClient, + ram_storage: Arc, + use_streaming_engine: bool, + target_split_size_bytes: u64, + max_merge_ops: u32, +) -> ParquetMergePipelineParams { + // merge_factor = max_merge_factor = 3 lets the planner pick up all + // three inputs in a single operation. `target_split_size_bytes` on + // the policy controls when an additional merge-up is scheduled; + // `target_split_size_bytes` on the pipeline params controls how the + // executor splits the merge output. They are the same value in + // production but kept independent in tests so the bonus test can + // ask for a small output target without disturbing the planner. + // + // `max_merge_ops` bounds the cascade depth. n=1 tests use 5 (plenty + // of headroom — they produce one output that doesn't re-trigger + // the planner anyway). m:n tests use 1 to keep the test fixed at + // one merge: outputs from the first merge land at + // `num_merge_ops = 1` and the planner refuses to merge them again + // because they've hit the policy's ceiling. + let merge_policy = Arc::new(ConstWriteAmplificationParquetMergePolicy::new( + ParquetMergePolicyConfig { + merge_factor: 3, + max_merge_factor: 3, + max_merge_ops, + target_split_size_bytes: 256 * 1024 * 1024, + maturation_period: Duration::from_secs(3600), + max_finalize_merge_operations: 3, + }, + )); + + let writer_config = + ParquetWriterConfig::default().with_row_group_size(TEST_OUTPUT_ROW_GROUP_SIZE); + + ParquetMergePipelineParams { + index_uid: quickwit_proto::types::IndexUid::for_test("test-merge-index-multi", 0), + indexing_directory: TempDirectory::for_test(), + metastore, + storage: ram_storage, + merge_policy, + merge_scheduler_service: universe.get_or_spawn_one(), + max_concurrent_split_uploads: 4, + event_broker: EventBroker::default(), + writer_config, + use_streaming_engine, + target_split_size_bytes, + } +} + +// --------------------------------------------------------------------------- +// Shared assertions: rich n=1 case +// --------------------------------------------------------------------------- + +/// Asserts the post-merge state for the canonical three-input, +/// three-metric, single-output fixture used by this module. Both engines +/// must produce a merged split that passes every check below. +async fn assert_three_input_three_metric_single_output_correct( + staged_metadata: &Arc>>, + replaced_ids: &Arc>>, + ram_storage: &Arc, +) { + // Publisher should replace exactly the three input splits. + let mut replaced_sorted: Vec = replaced_ids.lock().unwrap().clone(); + replaced_sorted.sort(); + assert_eq!( + replaced_sorted, + vec![ + "split-x".to_string(), + "split-y".to_string(), + "split-z".to_string(), + ], + "publish must replace all three input splits" + ); + + let staged = staged_metadata.lock().unwrap().clone(); + assert_eq!( + staged.len(), + 1, + "exactly one merged split should be staged for n=1 merge" + ); + let merged_meta = &staged[0]; + + // MC-1: every input row is in the output. + assert_eq!( + merged_meta.num_rows, TOTAL_INPUT_ROWS, + "merged split must contain all {TOTAL_INPUT_ROWS} input rows" + ); + + // Time range is the union of inputs: ts_min = 100 (input-x's first row), + // ts_max = 139 (input-z's last row), end_secs is max + 1. + assert_eq!( + merged_meta.time_range.start_secs, 100, + "time_range.start should be the min timestamp across all inputs" + ); + assert_eq!( + merged_meta.time_range.end_secs, 140, + "time_range.end should be max timestamp + 1" + ); + + // All three metric names must survive. + let expected_metrics: HashSet = [METRIC_AAA, METRIC_BBB, METRIC_CCC] + .iter() + .map(|s| s.to_string()) + .collect(); + assert_eq!( + merged_meta.metric_names, expected_metrics, + "merged split must contain all three metric names from the input set" + ); + + // First merge over level-0 inputs. + assert_eq!( + merged_meta.num_merge_ops, 1, + "first merge must set num_merge_ops to 1" + ); + + let table_config = TableConfig::default(); + assert_eq!( + merged_meta.sort_fields, + table_config.effective_sort_fields(), + "sort_fields must be preserved through merge" + ); + + // Row keys + zonemaps must be populated. + assert!( + merged_meta + .row_keys_proto + .as_ref() + .is_some_and(|b| !b.is_empty()), + "row_keys_proto must be present and non-empty" + ); + assert!( + merged_meta.zonemap_regexes.contains_key("metric_name"), + "zonemap_regexes must include metric_name; got keys: {:?}", + merged_meta.zonemap_regexes.keys().collect::>() + ); + + // Services across all three inputs must surface in low_cardinality_tags. + let service_tags = &merged_meta.low_cardinality_tags["service"]; + let expected_services: HashSet = + ["web", "api", "db"].iter().map(|s| s.to_string()).collect(); + assert_eq!( + *service_tags, expected_services, + "service tag values must include all three inputs" + ); + + // Read the merged file back and verify content. + let merged_bytes = ram_storage + .get_all(Path::new(&merged_meta.parquet_file)) + .await + .expect("merged parquet file must exist in storage"); + let merged_batch = read_parquet_from_bytes(&merged_bytes); + + assert_eq!( + merged_batch.num_rows() as u64, + TOTAL_INPUT_ROWS, + "merged Parquet file row count must match expected total" + ); + + // sorted_series is monotonically non-decreasing — the fundamental + // post-merge invariant. + let sorted_series = extract_binary_column(&merged_batch, SORTED_SERIES_COLUMN); + for i in 1..sorted_series.len() { + assert!( + sorted_series[i] >= sorted_series[i - 1], + "sorted_series must be non-decreasing at row {i}: {:?} < {:?}", + sorted_series[i], + sorted_series[i - 1], + ); + } + + // Sort-order semantics: with `metric_name` as the leading sort column + // (ascending), every `aaa.alpha` row precedes every `bbb.beta` row, + // and every `bbb.beta` row precedes every `ccc.gamma` row. + let metric_name_vec = extract_string_column(&merged_batch, "metric_name"); + let last_aaa = metric_name_vec + .iter() + .rposition(|n| n == METRIC_AAA) + .expect("aaa rows must be present"); + let first_bbb = metric_name_vec + .iter() + .position(|n| n == METRIC_BBB) + .expect("bbb rows must be present"); + let last_bbb = metric_name_vec + .iter() + .rposition(|n| n == METRIC_BBB) + .expect("bbb rows must be present"); + let first_ccc = metric_name_vec + .iter() + .position(|n| n == METRIC_CCC) + .expect("ccc rows must be present"); + assert!( + last_aaa < first_bbb, + "all aaa rows must precede all bbb rows ({last_aaa} >= {first_bbb})", + ); + assert!( + last_bbb < first_ccc, + "all bbb rows must precede all ccc rows ({last_bbb} >= {first_ccc})", + ); + + // Per-metric row count = 3 inputs * ROWS_PER_METRIC_PER_INPUT each. + let expected_per_metric = (3 * ROWS_PER_METRIC_PER_INPUT) as u64; + for metric in [METRIC_AAA, METRIC_BBB, METRIC_CCC] { + let count = metric_name_vec.iter().filter(|n| *n == metric).count() as u64; + assert_eq!( + count, expected_per_metric, + "metric {metric} must have exactly {expected_per_metric} rows; got {count}", + ); + } + + // All timestamps from the union [100, 140) must appear, and the metadata + // time_range must match the actual extrema. + let timestamps_in_file = extract_u64_column(&merged_batch, "timestamp_secs"); + assert_eq!( + timestamps_in_file.len() as u64, + TOTAL_INPUT_ROWS, + "timestamp column must have one entry per row" + ); + let ts_min = *timestamps_in_file.iter().min().unwrap(); + let ts_max = *timestamps_in_file.iter().max().unwrap(); + assert_eq!(ts_min, 100, "min timestamp must be 100"); + assert_eq!(ts_max, 139, "max timestamp must be 139"); + assert_eq!( + merged_meta.time_range.start_secs, ts_min, + "metadata time_range.start must match actual min timestamp" + ); + assert_eq!( + merged_meta.time_range.end_secs, + ts_max + 1, + "metadata time_range.end must be max timestamp + 1" + ); + + // Service / host tag values from every input must appear in the file. + let services_in_file: HashSet = extract_string_column(&merged_batch, "service") + .into_iter() + .collect(); + assert_eq!( + services_in_file, expected_services, + "service column must contain all three input values" + ); + let hosts_in_file: HashSet = extract_string_column(&merged_batch, "host") + .into_iter() + .collect(); + let expected_hosts: HashSet = + ["h1", "h2", "h3"].iter().map(|s| s.to_string()).collect(); + assert_eq!( + hosts_in_file, expected_hosts, + "host column must contain all three input values" + ); +} + +// --------------------------------------------------------------------------- +// Shared assertions: m:n case (n > 1) +// --------------------------------------------------------------------------- + +/// Asserts the post-merge state for the canonical three-input fixture when +/// the merge produced **more than one output**. Both engines must satisfy +/// the m:n contract: +/// +/// - Replacement covers all three inputs. +/// - The pipeline staged at least two output splits (proves splitting happened). +/// - The sum of per-output row counts equals the total input row count. +/// - Each output is internally sorted on `sorted_series`. +/// - Across outputs, the `sorted_series` partition is **disjoint** (no two outputs share any +/// `sorted_series` value — the merge engine splits at series boundaries, never inside). +/// - The union of metric_names / services across outputs covers the full input set. +/// - Every output declares `num_merge_ops = 1` (first merge over level-0 inputs) and has +/// `row_keys_proto` + `metric_name` zonemap regex populated. +async fn assert_three_input_three_metric_multi_output_correct( + staged_metadata: &Arc>>, + replaced_ids: &Arc>>, + ram_storage: &Arc, + expected_input_split_ids: &[&str], +) { + let mut replaced_sorted: Vec = replaced_ids.lock().unwrap().clone(); + replaced_sorted.sort(); + let mut expected_sorted: Vec = expected_input_split_ids + .iter() + .map(|s| s.to_string()) + .collect(); + expected_sorted.sort(); + assert_eq!( + replaced_sorted, expected_sorted, + "publish must replace all three input splits", + ); + + let staged = staged_metadata.lock().unwrap().clone(); + assert!( + staged.len() >= 2, + "m:n merge must produce at least two outputs; got {}", + staged.len() + ); + + let total_output_rows: u64 = staged.iter().map(|s| s.num_rows).sum(); + assert_eq!( + total_output_rows, TOTAL_INPUT_ROWS, + "sum of output row counts must equal total input rows", + ); + + // Each output internally sorted on sorted_series; collect ranges for + // the disjointness check across outputs. + let mut output_series_ranges: Vec<(Vec, Vec, String)> = + Vec::with_capacity(staged.len()); + for meta in &staged { + let bytes = ram_storage + .get_all(Path::new(&meta.parquet_file)) + .await + .expect("output parquet file must exist in storage"); + let batch = read_parquet_from_bytes(&bytes); + assert_eq!( + batch.num_rows() as u64, + meta.num_rows, + "output {} row count {} disagrees with metadata num_rows {}", + meta.parquet_file, + batch.num_rows(), + meta.num_rows, + ); + let series = extract_binary_column(&batch, SORTED_SERIES_COLUMN); + assert!( + !series.is_empty(), + "every output must have at least one row (empty outputs should be dropped by the \ + engine)" + ); + for i in 1..series.len() { + assert!( + series[i] >= series[i - 1], + "output {} sorted_series not monotone at row {i}", + meta.parquet_file, + ); + } + output_series_ranges.push(( + series.first().unwrap().clone(), + series.last().unwrap().clone(), + meta.parquet_file.clone(), + )); + } + + // Sort outputs by min_series for pairwise disjointness comparison. + output_series_ranges.sort_by(|a, b| a.0.cmp(&b.0)); + for window in output_series_ranges.windows(2) { + let (_, left_max, left_file) = &window[0]; + let (right_min, _, right_file) = &window[1]; + assert!( + left_max < right_min, + "outputs {} and {} overlap on sorted_series: left max = {:?}, right min = {:?}", + left_file, + right_file, + left_max, + right_min, + ); + } + + let union_metrics: HashSet = staged + .iter() + .flat_map(|s| s.metric_names.iter().cloned()) + .collect(); + let expected_metrics: HashSet = [METRIC_AAA, METRIC_BBB, METRIC_CCC] + .iter() + .map(|s| s.to_string()) + .collect(); + assert_eq!( + union_metrics, expected_metrics, + "union of output metric_names must equal the full input set", + ); + + let union_services: HashSet = staged + .iter() + .flat_map(|s| { + s.low_cardinality_tags + .get("service") + .into_iter() + .flat_map(|set| set.iter().cloned()) + }) + .collect(); + let expected_services: HashSet = + ["web", "api", "db"].iter().map(|s| s.to_string()).collect(); + assert_eq!( + union_services, expected_services, + "union of output services must equal the full input set", + ); + + for meta in &staged { + assert_eq!( + meta.num_merge_ops, 1, + "output {} num_merge_ops must be 1 for the first merge", + meta.parquet_file, + ); + assert!( + meta.row_keys_proto.as_ref().is_some_and(|b| !b.is_empty()), + "output {} missing row_keys_proto", + meta.parquet_file, + ); + assert!( + meta.zonemap_regexes.contains_key("metric_name"), + "output {} missing metric_name zonemap regex", + meta.parquet_file, + ); + } +} + +// --------------------------------------------------------------------------- +// Tests +// --------------------------------------------------------------------------- + +/// Run a merge pipeline over the canonical three-input multi-metric fixture +/// and apply the supplied assertion. Shared between the two engine variants +/// below. `target_split_size_bytes` drives the executor's `num_outputs` +/// calculation; pass `u64::MAX` (or anything bigger than the total input +/// size) for the n=1 case, or a small value to force m:n. +async fn run_three_input_multi_metric_merge( + use_streaming_engine: bool, + target_split_size_bytes: u64, + assertions: F, +) where + F: for<'a> FnOnce( + Arc>>, + Arc>>, + Arc, + ) -> Fut, + Fut: std::future::Future, +{ + quickwit_common::setup_logging_for_tests(); + + let universe = Universe::with_accelerated_time(); + let temp_dir = tempfile::tempdir().unwrap(); + let ram_storage: Arc = Arc::new(RamStorage::default()); + + let (_paths, splits) = build_three_multi_metric_inputs(temp_dir.path(), &ram_storage).await; + + let capture = mount_capturing_metastore(); + let params = make_pipeline_params( + &universe, + capture.metastore.clone(), + ram_storage.clone(), + use_streaming_engine, + target_split_size_bytes, + 5, // max_merge_ops: n=1 tests don't cascade, give plenty of headroom + ); + + let pipeline = ParquetMergePipeline::new(params, Some(splits), universe.spawn_ctx()); + let (_pipeline_mailbox, _pipeline_handle) = universe.spawn_builder().spawn(pipeline); + + wait_until_predicate( + || { + let publish_called = capture.publish_called.clone(); + async move { publish_called.load(Ordering::SeqCst) } + }, + Duration::from_secs(60), + Duration::from_millis(100), + ) + .await + .expect("timed out waiting for merge publish"); + + assertions(capture.staged_metadata, capture.replaced_ids, ram_storage).await; + + universe.assert_quit().await; +} + +/// In-memory engine, n=1 output, three multi-metric inputs with overlapping +/// timestamps and timeseries IDs. Verifies the merge correctly interleaves +/// rows across all three inputs and produces a single sorted output with +/// the full input rowset. +#[tokio::test] +async fn test_multi_metric_three_input_single_output_in_memory_engine() { + run_three_input_multi_metric_merge( + false, // use_streaming_engine + // Larger than total input size — forces num_outputs = 1. + 256 * 1024 * 1024, + |staged, replaced, storage| async move { + assert_three_input_three_metric_single_output_correct(&staged, &replaced, &storage) + .await; + }, + ) + .await; +} + +/// Streaming engine, n=1 output. Same fixture as the in-memory variant; +/// must produce a row-content-equivalent output. Additionally asserts +/// `PEAK_BODY_COL_PAGE_CACHE_LEN > 0` to confirm the streaming engine +/// actually ran (the in-memory engine never writes to that atomic). +#[allow( + clippy::await_holding_lock, + reason = "see ms7_serial_lock rationale: std::sync::Mutex on a single-threaded tokio runtime" +)] +#[tokio::test] +async fn test_multi_metric_three_input_single_output_streaming_engine() { + use quickwit_parquet_engine::merge::streaming::{ + PEAK_BODY_COL_PAGE_CACHE_LEN, ms7_serial_lock, + }; + + let _ms7_guard = ms7_serial_lock(); + PEAK_BODY_COL_PAGE_CACHE_LEN.store(0, Ordering::Relaxed); + + run_three_input_multi_metric_merge( + true, // use_streaming_engine + 256 * 1024 * 1024, + |staged, replaced, storage| async move { + assert!( + PEAK_BODY_COL_PAGE_CACHE_LEN.load(Ordering::Relaxed) > 0, + "streaming engine did not write to PEAK_BODY_COL_PAGE_CACHE_LEN — routing may \ + have silently fallen back to the in-memory engine", + ); + assert_three_input_three_metric_single_output_correct(&staged, &replaced, &storage) + .await; + }, + ) + .await; +} + +/// Run a merge pipeline over the canonical three-input fixture with +/// **prefix-aligned multi-row-group inputs** (`rg_partition_prefix_len = 1`, +/// one row group per metric_name). Drives the m:n bonus tests. +async fn run_three_input_prefix_aligned_merge( + use_streaming_engine: bool, + target_split_size_bytes: u64, + assertions: F, +) where + F: for<'a> FnOnce( + Arc>>, + Arc>>, + Arc, + ) -> Fut, + Fut: std::future::Future, +{ + quickwit_common::setup_logging_for_tests(); + + let universe = Universe::with_accelerated_time(); + let temp_dir = tempfile::tempdir().unwrap(); + let ram_storage: Arc = Arc::new(RamStorage::default()); + + let (_paths, splits) = + build_three_prefix_aligned_multi_metric_inputs(temp_dir.path(), &ram_storage).await; + + let capture = mount_capturing_metastore(); + let params = make_pipeline_params( + &universe, + capture.metastore.clone(), + ram_storage.clone(), + use_streaming_engine, + target_split_size_bytes, + // max_merge_ops = 1: outputs from the first (and only) merge land + // at num_merge_ops = 1 and the planner refuses to merge them again, + // pinning this test to exactly one merge regardless of how many + // outputs the engine chose to produce. + 1, + ); + + let pipeline = ParquetMergePipeline::new(params, Some(splits), universe.spawn_ctx()); + let (_pipeline_mailbox, _pipeline_handle) = universe.spawn_builder().spawn(pipeline); + + wait_until_predicate( + || { + let publish_called = capture.publish_called.clone(); + async move { publish_called.load(Ordering::SeqCst) } + }, + Duration::from_secs(60), + Duration::from_millis(100), + ) + .await + .expect("timed out waiting for merge publish"); + + assertions(capture.staged_metadata, capture.replaced_ids, ram_storage).await; + + universe.assert_quit().await; +} + +/// **Bonus** test, in-memory engine: three multi-metric inputs each with +/// `rg_partition_prefix_len = 1` (one row group per metric_name), merged +/// with a small `target_split_size_bytes` that forces the executor to ask +/// the engine for `num_outputs > 1`. Exercises the previously-impossible +/// pipeline-level m:n merge path now that the executor's hardcoded +/// `num_outputs = 1` is gone. Verifies the multi-output contract: +/// sum-equals-total, internal monotonicity, inter-output disjointness on +/// `sorted_series`, and union-equals-full-set on metrics/services. +#[tokio::test] +async fn test_prefix_aligned_multi_metric_three_input_multi_output_in_memory_engine() { + run_three_input_prefix_aligned_merge( + false, // use_streaming_engine + // Smaller than per-input size — guarantees num_outputs ≥ 2. The + // engine clamps to available sorted_series boundaries (~60 in + // this fixture: 3 metrics × 20 timeseries each), well above 2. + 500, + |staged, replaced, storage| async move { + assert_three_input_three_metric_multi_output_correct( + &staged, + &replaced, + &storage, + &["split-px", "split-py", "split-pz"], + ) + .await; + }, + ) + .await; +} + +/// **Bonus** test, streaming engine: same fixture and contract as the +/// in-memory variant. Additionally asserts +/// `PEAK_BODY_COL_PAGE_CACHE_LEN > 0` to confirm the streaming engine +/// actually ran. With prefix-aligned inputs the streaming engine reads +/// each input's row groups (one per metric_name) through the prefix-aware +/// `StreamingParquetReader` path — distinct from the legacy +/// `LegacyInputAdapter` route, since these inputs do not require +/// promotion (`target_prefix_len_override` is `None` for regular merges). +#[allow( + clippy::await_holding_lock, + reason = "see ms7_serial_lock rationale: std::sync::Mutex on a single-threaded tokio runtime" +)] +#[tokio::test] +async fn test_prefix_aligned_multi_metric_three_input_multi_output_streaming_engine() { + use quickwit_parquet_engine::merge::streaming::{ + PEAK_BODY_COL_PAGE_CACHE_LEN, ms7_serial_lock, + }; + + let _ms7_guard = ms7_serial_lock(); + PEAK_BODY_COL_PAGE_CACHE_LEN.store(0, Ordering::Relaxed); + + run_three_input_prefix_aligned_merge( + true, // use_streaming_engine + 500, + |staged, replaced, storage| async move { + assert!( + PEAK_BODY_COL_PAGE_CACHE_LEN.load(Ordering::Relaxed) > 0, + "streaming engine did not write to PEAK_BODY_COL_PAGE_CACHE_LEN — routing may \ + have silently fallen back to the in-memory engine", + ); + assert_three_input_three_metric_multi_output_correct( + &staged, + &replaced, + &storage, + &["split-px", "split-py", "split-pz"], + ) + .await; + }, + ) + .await; +} diff --git a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_sketch_test.rs b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_sketch_test.rs index 0ac1f5e69c2..f54f5ecd39e 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_sketch_test.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_sketch_test.rs @@ -172,6 +172,8 @@ async fn test_merge_pipeline_end_to_end_sketches() { max_concurrent_split_uploads: 4, event_broker: EventBroker::default(), writer_config: ParquetWriterConfig::default(), + use_streaming_engine: false, + target_split_size_bytes: 256 * 1024 * 1024, }; let initial_splits = vec![meta_a, meta_b]; diff --git a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_test.rs b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_test.rs index 547060668b0..000ee5cb355 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_test.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_test.rs @@ -405,6 +405,8 @@ async fn test_merge_pipeline_end_to_end() { max_concurrent_split_uploads: 4, event_broker: EventBroker::default(), writer_config: ParquetWriterConfig::default(), + use_streaming_engine: false, + target_split_size_bytes: 256 * 1024 * 1024, }; let initial_splits = vec![meta_a, meta_b]; @@ -424,6 +426,43 @@ async fn test_merge_pipeline_end_to_end() { .await .expect("timed out waiting for merge publish"); + assert_cpu_mem_merge_outputs_correct(&staged_metadata, &replaced_ids, &ram_storage).await; + + universe.assert_quit().await; +} + +/// Asserts the post-merge state captured from the metastore mock and the +/// Parquet file in storage for the canonical two-input fixture used by +/// the end-to-end tests: +/// +/// - `split-a`: 50 rows of `cpu.usage`, ts 100..150, service `web`, host `host-1` +/// - `split-b`: 50 rows of `mem.usage`, ts 200..250, service `api`, host `host-2` +/// +/// Both engines (in-memory and streaming) must produce a merged split that +/// passes every check below — that is the **contract** of a regular merge, +/// independent of which engine ran. Driven by both +/// `test_merge_pipeline_end_to_end` (in-memory engine via the flag default) +/// and `test_merge_pipeline_end_to_end_with_streaming_engine_flag` +/// (streaming engine via the flag). +/// +/// Verifies, in order: +/// - **Step 5**: `replaced_split_ids = [split-a, split-b]`. +/// - **Step 6**: staged `ParquetSplitMetadata` — `num_rows = 100`, `time_range = [100, 250)`, +/// `metric_names = {cpu.usage, mem.usage}`, `num_merge_ops = 1`, `sort_fields` matches the table +/// config, `row_keys_proto` present + non-empty, `zonemap_regexes` contains `metric_name`, +/// `low_cardinality_tags["service"] = {web, api}`. +/// - **Step 7**: merged Parquet file content — row count, all 100 timestamps match expected and +/// metadata `time_range`, both service / host values survive, `sorted_series` is monotonically +/// non-decreasing, and `cpu.usage` rows precede `mem.usage` rows (the global sort order +/// semantics). +/// - **Step 8**: Parquet KV metadata — `qh.sort_fields`, `qh.num_merge_ops`, `qh.row_keys` +/// (non-empty), `qh.zonemap_regexes` parses as JSON and contains `metric_name`, and +/// cross-validates with the staged metadata. +async fn assert_cpu_mem_merge_outputs_correct( + staged_metadata: &Arc>>, + replaced_ids: &Arc>>, + ram_storage: &Arc, +) { // --- Step 5: Verify replaced_split_ids --- let mut replaced_sorted: Vec = replaced_ids.lock().unwrap().clone(); @@ -598,8 +637,8 @@ async fn test_merge_pipeline_end_to_end() { sorted_series[i] >= sorted_series[i - 1], "sorted_series must be monotonically non-decreasing at row {}: {:?} < {:?}", i, - &sorted_series[i], - &sorted_series[i - 1] + sorted_series[i], + sorted_series[i - 1] ); } @@ -663,6 +702,174 @@ async fn test_merge_pipeline_end_to_end() { merged_meta.zonemap_regexes, zonemaps_parsed, "metadata zonemap_regexes must match Parquet qh.zonemap_regexes" ); +} + +/// End-to-end production-path test for the YAML-flag-gated streaming +/// engine wire-in. Runs the full actor chain — planner → downloader → +/// executor → uploader → publisher — with +/// `ParquetMergePipelineParams::use_streaming_engine = true` and asserts +/// the merge completed correctly. +/// +/// The executor's branch is `if is_promotion || use_streaming_engine +/// { streaming } else { in-memory }`. With promotion off and +/// `use_streaming_engine = true`, the streaming engine is the only +/// reachable path — there is no silent fallback. The test confirms +/// this by: +/// +/// 1. Asserting the merge published with the right `replaced_split_ids` (the merge actually ran +/// end-to-end through the executor). +/// 2. Reading `PEAK_BODY_COL_PAGE_CACHE_LEN` and asserting it is non-zero (the streaming engine +/// writes to this atomic on every body-col page assembly; if the in-memory engine had run +/// instead the counter would stay at zero). +/// 3. Asserting row count and metric names on the output match the inputs (the streaming engine +/// produces correct results, not just "something"). +#[allow( + clippy::await_holding_lock, + reason = "the lock is `std::sync::Mutex` and the `#[tokio::test]` runtime is single-threaded, \ + so holding the guard across await won't deadlock another thread — see \ + `ms7_serial_lock` rationale" +)] +#[tokio::test] +async fn test_merge_pipeline_end_to_end_with_streaming_engine_flag() { + use std::sync::atomic::Ordering; + + use quickwit_parquet_engine::merge::streaming::{ + PEAK_BODY_COL_PAGE_CACHE_LEN, ms7_serial_lock, + }; + + // Serialise against every other test in this binary that runs a + // streaming merge: they all touch the same process-global atomic, + // and a concurrent `store(0)` would race our load. The lock is + // re-exported from `quickwit_parquet_engine` under the `testsuite` + // feature for exactly this cross-crate case. + let _ms7_guard = ms7_serial_lock(); + PEAK_BODY_COL_PAGE_CACHE_LEN.store(0, Ordering::Relaxed); + + quickwit_common::setup_logging_for_tests(); + + let universe = Universe::with_accelerated_time(); + let temp_dir = tempfile::tempdir().unwrap(); + let ram_storage: Arc = Arc::new(RamStorage::default()); + + let batch_a = create_custom_test_batch("cpu.usage", 100, 50, "web", "host-1"); + let meta_a = make_test_split_metadata("split-a", 50, 0, 100, "cpu.usage"); + let size_a = write_test_parquet_file(temp_dir.path(), "split-a.parquet", &batch_a, &meta_a); + let meta_a = { + let mut m = meta_a; + m.size_bytes = size_a; + m.parquet_file = "split-a.parquet".to_string(); + m + }; + let batch_b = create_custom_test_batch("mem.usage", 200, 50, "api", "host-2"); + let meta_b = make_test_split_metadata("split-b", 50, 0, 200, "mem.usage"); + let size_b = write_test_parquet_file(temp_dir.path(), "split-b.parquet", &batch_b, &meta_b); + let meta_b = { + let mut m = meta_b; + m.size_bytes = size_b; + m.parquet_file = "split-b.parquet".to_string(); + m + }; + + let content_a = std::fs::read(temp_dir.path().join("split-a.parquet")).unwrap(); + ram_storage + .put(Path::new("split-a.parquet"), Box::new(content_a)) + .await + .unwrap(); + let content_b = std::fs::read(temp_dir.path().join("split-b.parquet")).unwrap(); + ram_storage + .put(Path::new("split-b.parquet"), Box::new(content_b)) + .await + .unwrap(); + + let mut mock_metastore = MockMetastoreService::new(); + let staged_metadata: Arc>> = + Arc::new(std::sync::Mutex::new(Vec::new())); + let staged_metadata_clone = staged_metadata.clone(); + mock_metastore.expect_stage_metrics_splits().returning( + move |request: StageMetricsSplitsRequest| { + let splits = request + .deserialize_splits_metadata() + .expect("failed to deserialize staged metadata"); + staged_metadata_clone.lock().unwrap().extend(splits); + Ok(EmptyResponse {}) + }, + ); + let publish_called = Arc::new(std::sync::atomic::AtomicBool::new(false)); + let publish_called_clone = publish_called.clone(); + let replaced_ids = Arc::new(std::sync::Mutex::new(Vec::::new())); + let replaced_ids_clone = replaced_ids.clone(); + mock_metastore + .expect_publish_metrics_splits() + .returning(move |request| { + replaced_ids_clone + .lock() + .unwrap() + .extend(request.replaced_split_ids.clone()); + publish_called_clone.store(true, Ordering::SeqCst); + Ok(EmptyResponse {}) + }); + let metastore = MetastoreServiceClient::from_mock(mock_metastore); + + let merge_policy = Arc::new(ConstWriteAmplificationParquetMergePolicy::new( + ParquetMergePolicyConfig { + merge_factor: 2, + max_merge_factor: 2, + max_merge_ops: 5, + target_split_size_bytes: 256 * 1024 * 1024, + maturation_period: Duration::from_secs(3600), + max_finalize_merge_operations: 3, + }, + )); + + let params = ParquetMergePipelineParams { + index_uid: quickwit_proto::types::IndexUid::for_test("test-merge-index-streaming", 0), + indexing_directory: TempDirectory::for_test(), + metastore, + storage: ram_storage.clone(), + merge_policy, + merge_scheduler_service: universe.get_or_spawn_one(), + max_concurrent_split_uploads: 4, + event_broker: EventBroker::default(), + writer_config: ParquetWriterConfig::default(), + // This is the bit under test: regular merges must route through + // `execute_merge_operation`, not the in-memory fallback. + use_streaming_engine: true, + target_split_size_bytes: 256 * 1024 * 1024, + }; + + let initial_splits = vec![meta_a, meta_b]; + let pipeline = ParquetMergePipeline::new(params, Some(initial_splits), universe.spawn_ctx()); + let (_pipeline_mailbox, _pipeline_handle) = universe.spawn_builder().spawn(pipeline); + + wait_until_predicate( + || { + let publish_called = publish_called.clone(); + async move { publish_called.load(Ordering::SeqCst) } + }, + Duration::from_secs(30), + Duration::from_millis(100), + ) + .await + .expect("timed out waiting for streaming-engine merge publish"); + + // The streaming engine writes to `PEAK_BODY_COL_PAGE_CACHE_LEN` on + // every body-col page assembly; the in-memory engine never touches + // it. A non-zero post-merge value is direct evidence the streaming + // engine ran the body-col path — distinguishes this test from a + // silent fallback to the in-memory engine. + let peak = PEAK_BODY_COL_PAGE_CACHE_LEN.load(Ordering::Relaxed); + assert!( + peak > 0, + "streaming engine did not write to PEAK_BODY_COL_PAGE_CACHE_LEN — routing may have \ + silently fallen back to the in-memory engine", + ); + + // Same correctness contract as the in-memory variant: every check on + // the merged metadata, the Parquet file content, and the Parquet KV + // headers must hold regardless of which engine ran. This shared + // helper is the executable parity between engines at the + // pipeline-integration level. + assert_cpu_mem_merge_outputs_correct(&staged_metadata, &replaced_ids, &ram_storage).await; universe.assert_quit().await; } diff --git a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_trace_conformance_test.rs b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_trace_conformance_test.rs index 20ab86cfc02..e5a25a9de7a 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_trace_conformance_test.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_pipeline/parquet_merge_pipeline_trace_conformance_test.rs @@ -628,6 +628,8 @@ async fn test_trace_conformance_normal_path() { max_concurrent_split_uploads: 4, event_broker: EventBroker::default(), writer_config: ParquetWriterConfig::default(), + use_streaming_engine: false, + target_split_size_bytes: 256 * 1024 * 1024, }; let pipeline = @@ -747,6 +749,8 @@ async fn test_trace_conformance_crash_mid_cascade() { max_concurrent_split_uploads: 4, event_broker: EventBroker::default(), writer_config: ParquetWriterConfig::default(), + use_streaming_engine: false, + target_split_size_bytes: 256 * 1024 * 1024, }; let pipeline = diff --git a/quickwit/quickwit-parquet-engine/src/merge/streaming.rs b/quickwit/quickwit-parquet-engine/src/merge/streaming.rs index 492bc03f166..2ce432725b3 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/streaming.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/streaming.rs @@ -107,16 +107,18 @@ use crate::storage::{ /// [`write_next_column_arrays`]: crate::storage::streaming_writer::RowGroupBuilder::write_next_column_arrays pub(crate) const OUTPUT_PAGE_ROWS: usize = 1024; -/// Test-only peak observed length of any input's `body_col_page_cache` -/// since the last reset. Used by the MS-7 page-bounded-memory test to -/// assert that the cache stays bounded by a small constant regardless -/// of input column size. Set unconditionally inside the merge so the -/// invariant is observable in any test build; reset on each merge entry. -#[cfg(test)] -pub(crate) static PEAK_BODY_COL_PAGE_CACHE_LEN: std::sync::atomic::AtomicUsize = +/// Peak observed length of any input's `body_col_page_cache` since the +/// last reset, set unconditionally inside the merge so the invariant is +/// observable in any test build. Used by the MS-7 page-bounded-memory +/// test to assert that the cache stays bounded by a small constant +/// regardless of input column size, and by cross-crate integration tests +/// that need to confirm the streaming engine ran (any non-zero value +/// proves a streaming merge executed). +#[cfg(any(test, feature = "testsuite"))] +pub static PEAK_BODY_COL_PAGE_CACHE_LEN: std::sync::atomic::AtomicUsize = std::sync::atomic::AtomicUsize::new(0); -#[cfg(test)] +#[cfg(any(test, feature = "testsuite"))] pub(crate) fn record_body_col_page_cache_len(len: usize) { use std::sync::atomic::Ordering; let mut prev = PEAK_BODY_COL_PAGE_CACHE_LEN.load(Ordering::Relaxed); @@ -133,7 +135,7 @@ pub(crate) fn record_body_col_page_cache_len(len: usize) { } } -#[cfg(not(test))] +#[cfg(not(any(test, feature = "testsuite")))] pub(crate) fn record_body_col_page_cache_len(_len: usize) {} /// Streaming N-input → M-output column-major merge. @@ -1824,6 +1826,37 @@ fn build_full_union_schema_from_arrow_schemas( Ok(schema) } +// ============================================================================ +// Test support +// ============================================================================ + +/// Process-wide serial lock for tests that run streaming merges. Every +/// streaming merge writes to the global `PEAK_BODY_COL_PAGE_CACHE_LEN` +/// atomic; MS-7 tests reset-then-read it and would race any +/// concurrent merge in the same test binary. Both the MS-7 tests in +/// this module and any other test that invokes +/// `streaming_merge_sorted_parquet_files` (engine-crate parity tests +/// in `merge::tests::parity`, indexing-crate pipeline tests that +/// reset the atomic for the same reason) must acquire this lock for +/// the duration of the merge. +/// +/// Exposed `pub` under the `testsuite` feature so cross-crate tests +/// can share the same lock — the streaming engine's atomic is +/// process-global and the lock has to be too. +/// +/// Held across `.await` points in MS-7 tests — that's why each +/// MS-7 test allows `clippy::await_holding_lock`. The lock is +/// `std::sync::Mutex` and the `#[tokio::test]` runtime is +/// single-threaded, so holding the guard across await won't deadlock +/// another thread. `tokio::sync::Mutex` is forbidden by GAP-002. +#[cfg(any(test, feature = "testsuite"))] +pub fn ms7_serial_lock() -> std::sync::MutexGuard<'static, ()> { + static LOCK: std::sync::Mutex<()> = std::sync::Mutex::new(()); + // Poisoning is fine — a previous test panicking shouldn't prevent + // the next one from acquiring; just unwrap the inner. + LOCK.lock().unwrap_or_else(|poisoned| poisoned.into_inner()) +} + // ============================================================================ // Tests // ============================================================================ @@ -4800,25 +4833,11 @@ mod tests { /// `PEAK_BODY_COL_PAGE_CACHE_LEN` is a process-global atomic, so /// concurrent MS-7 tests would pollute each other's readings. - /// Every MS-7 test must acquire this lock for the duration of its - /// merge + read sequence. The static guarantees there's only one - /// MS-7 merge in flight at a time across the whole test binary. - /// - /// Held across `.await` points in the MS-7 tests — that's why - /// each test allows `clippy::await_holding_lock`. In production - /// code we'd use an async-aware mutex, but this is test-only - /// process-wide serialization for a global atomic that has no - /// async-safe alternative; `tokio::sync::Mutex` is also banned - /// by GAP-002 (cancel-correctness). The lock is `std::sync::Mutex` - /// and the executor is `tokio::test`'s single-threaded runtime, - /// so holding the guard across await won't deadlock another - /// thread. - fn ms7_serial_lock() -> std::sync::MutexGuard<'static, ()> { - static LOCK: std::sync::Mutex<()> = std::sync::Mutex::new(()); - // Poisoning is fine — a previous test panicking shouldn't - // prevent the next one from acquiring; just unwrap the inner. - LOCK.lock().unwrap_or_else(|poisoned| poisoned.into_inner()) - } + /// Re-exported from the module-level helper so existing MS-7 tests + /// in this submodule keep their unqualified name; other test + /// modules call `super::ms7_serial_lock` (or the full path) + /// directly. + use super::ms7_serial_lock; /// Build a fixture that forces many input body-col pages with a /// pinned `data_page_row_count_limit`, then merge it through the diff --git a/quickwit/quickwit-parquet-engine/src/merge/tests.rs b/quickwit/quickwit-parquet-engine/src/merge/tests.rs index 419057db1de..0ef9b45288e 100644 --- a/quickwit/quickwit-parquet-engine/src/merge/tests.rs +++ b/quickwit/quickwit-parquet-engine/src/merge/tests.rs @@ -1679,6 +1679,478 @@ fn test_merge_descending_pre_timestamp_column() { ); } +// ---- Engine parity: streaming vs in-memory ---- +// +// Verifies that `execute_merge_operation` (the streaming column-major engine) +// and `merge_sorted_parquet_files` (the in-memory engine) produce the same +// row content for the same inputs. Gates the executor's +// `parquet_merge_use_streaming_engine` YAML flag: once parity holds on a +// realistic fixture, production can flip the flag with confidence the +// streaming engine is a drop-in replacement. The in-memory engine stays in +// the executor as the runtime fallback until the streaming engine has +// soaked, at which point both this test and the fallback branch can be +// removed. + +mod parity { + use std::io; + use std::ops::Range; + use std::path::Path; + + use async_trait::async_trait; + use bytes::Bytes; + use tokio::io::{AsyncRead, AsyncReadExt, AsyncSeekExt}; + + use super::*; + use crate::merge::execute_merge_operation; + use crate::merge::policy::ParquetMergeOperation; + use crate::storage::RemoteByteSource; + + /// Mirrors the executor's `LocalFileByteSource` so parity tests exercise + /// the same code path production will use: a local Parquet file wrapped + /// in `RemoteByteSource` and handed to `execute_merge_operation`. + /// Ignores the `_path` argument from the trait surface — the source is + /// already bound to one concrete file at construction time. + struct LocalFileByteSource { + path: PathBuf, + } + + #[async_trait] + impl RemoteByteSource for LocalFileByteSource { + async fn file_size(&self, _path: &Path) -> io::Result { + tokio::fs::metadata(&self.path).await.map(|m| m.len()) + } + + async fn get_slice(&self, _path: &Path, range: Range) -> io::Result { + let mut file = tokio::fs::File::open(&self.path).await?; + file.seek(io::SeekFrom::Start(range.start)).await?; + let len = (range.end - range.start) as usize; + let mut buf = vec![0u8; len]; + file.read_exact(&mut buf).await?; + Ok(Bytes::from(buf)) + } + + async fn get_slice_stream( + &self, + _path: &Path, + range: Range, + ) -> io::Result> { + let mut file = tokio::fs::File::open(&self.path).await?; + file.seek(io::SeekFrom::Start(range.start)).await?; + let len = range.end - range.start; + Ok(Box::new(file.take(len))) + } + } + + /// Build a `ParquetMergeOperation` for a regular (non-promotion) merge + /// over the given file paths. Each split's `parquet_file` is set to the + /// basename — the streaming engine passes it to `RemoteByteSource`, + /// which `LocalFileByteSource` ignores. + fn make_regular_merge_op(input_paths: &[PathBuf]) -> ParquetMergeOperation { + let splits: Vec = input_paths + .iter() + .map(|path| { + let name = path + .file_name() + .and_then(|n| n.to_str()) + .expect("input path must have a UTF-8 filename"); + ParquetSplitMetadata::metrics_builder() + .split_id(ParquetSplitId::generate(ParquetSplitKind::Metrics)) + .index_uid("test-index:0") + .sort_fields(TEST_SORT_FIELDS) + .window_duration_secs(900) + .window_start_secs(0) + .time_range(TimeRange::new(0, 1)) + .rg_partition_prefix_len(0) + .parquet_file(name) + .build() + }) + .collect(); + ParquetMergeOperation::new(splits) + } + + /// Run both engines on the same inputs and assert the materialised row + /// content matches column-by-column on every output, in order. Per-output + /// physical metadata (page boundaries, row group sizing) is allowed to + /// differ — we only assert what the merge contract guarantees to readers. + fn assert_engine_parity(input_paths: &[PathBuf], num_outputs: usize) { + // The streaming engine increments the process-global + // `PEAK_BODY_COL_PAGE_CACHE_LEN` atomic that MS-7 tests + // reset-then-read. Serialise against MS-7 by acquiring the + // same lock for the duration of the streaming-engine run. + let _ms7_guard = crate::merge::streaming::ms7_serial_lock(); + + let parent_dir = input_paths + .first() + .and_then(|p| p.parent()) + .expect("at least one input expected"); + let in_memory_out_dir = parent_dir.join("output_in_memory"); + let streaming_out_dir = parent_dir.join("output_streaming"); + std::fs::create_dir_all(&in_memory_out_dir).unwrap(); + std::fs::create_dir_all(&streaming_out_dir).unwrap(); + + let config = MergeConfig { + num_outputs, + writer_config: ParquetWriterConfig::default(), + }; + + // In-memory engine. + let in_memory_outputs = + merge_sorted_parquet_files(input_paths, &in_memory_out_dir, &config).unwrap(); + + // Streaming engine through the same surface the executor uses. + let op = make_regular_merge_op(input_paths); + let sources: Vec> = input_paths + .iter() + .map(|path| { + Arc::new(LocalFileByteSource { path: path.clone() }) as Arc + }) + .collect(); + let runtime = tokio::runtime::Builder::new_current_thread() + .enable_all() + .build() + .unwrap(); + let streaming_outputs = runtime + .block_on(execute_merge_operation( + &op, + sources, + &streaming_out_dir, + &config, + )) + .unwrap(); + + assert_eq!( + streaming_outputs.len(), + in_memory_outputs.len(), + "engines disagree on output count" + ); + for (i, (s, m)) in streaming_outputs + .iter() + .zip(in_memory_outputs.iter()) + .enumerate() + { + assert_eq!( + s.num_rows, m.num_rows, + "output #{i}: engines disagree on row count ({} vs {})", + s.num_rows, m.num_rows + ); + let s_batch = read_parquet_file(&s.path); + let m_batch = read_parquet_file(&m.path); + for col_name in ["metric_name", "timestamp_secs", "value", "timeseries_id"] { + if s_batch.schema().index_of(col_name).is_err() { + continue; + } + let m_idx = m_batch.schema().index_of(col_name).unwrap_or_else(|_| { + panic!("output #{i}: streaming has column {col_name} but in-memory does not") + }); + let s_idx = s_batch.schema().index_of(col_name).unwrap(); + // Compare column values through the existing string/numeric + // extractors; raw Arrow array equality would reject benign + // representational differences like the dict-key permutation. + match s_batch.column(s_idx).data_type() { + DataType::Dictionary(_, _) | DataType::Utf8 | DataType::LargeUtf8 => { + let s_vals = extract_string_column(&s_batch, col_name); + let m_vals = extract_string_column(&m_batch, col_name); + assert_eq!( + s_vals, m_vals, + "output #{i}: column {col_name} (string) differs between engines" + ); + } + DataType::UInt64 => { + let s_vals = extract_u64_column(&s_batch, col_name); + let m_vals = extract_u64_column(&m_batch, col_name); + assert_eq!( + s_vals, m_vals, + "output #{i}: column {col_name} (u64) differs between engines" + ); + } + DataType::Float64 => { + let s_vals = extract_f64_column(&s_batch, col_name); + let m_vals = extract_f64_column(&m_batch, col_name); + assert_eq!( + s_vals, m_vals, + "output #{i}: column {col_name} (f64) differs between engines" + ); + } + DataType::Int64 => { + let s_idx = s_batch.schema().index_of(col_name).unwrap(); + let s_col = s_batch + .column(s_idx) + .as_any() + .downcast_ref::() + .unwrap(); + let m_col = m_batch + .column(m_idx) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!( + s_col.values(), + m_col.values(), + "output #{i}: column {col_name} (i64) differs between engines" + ); + } + other => { + panic!( + "output #{i}: column {col_name} has unexpected type {other:?} for \ + parity comparison" + ); + } + } + } + } + + // Beyond engine-vs-engine equivalence, both outputs must satisfy + // the m:n merge contract: every row preserved exactly once across + // outputs, each output internally sorted on `sorted_series`, and + // no two outputs sharing any `sorted_series` value (the engine + // promises a non-overlapping partition over the input keyspace). + // The two engines produce equivalent output, so checking either + // is sufficient; the in-memory side is the historical baseline. + let total_input_rows: usize = input_paths + .iter() + .map(|path| { + let bytes = std::fs::read(path).expect("read input parquet for row-count"); + let builder = + parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder::try_new( + Bytes::from(bytes), + ) + .expect("input parquet builder"); + builder.metadata().file_metadata().num_rows() as usize + }) + .sum(); + assert_multi_output_invariants(&in_memory_outputs, total_input_rows); + } + + /// Verify the m:n merge contract on a single engine's outputs: + /// + /// 1. Sum of per-output row counts equals the total input row count (no duplication, no loss). + /// 2. Within each output, the `sorted_series` column is monotonically non-decreasing. + /// 3. Across outputs, after sorting by min `sorted_series`, every output's max sorted_series is + /// strictly less than the next output's min — the partition is disjoint on the keyspace. + /// + /// Holds for any merge with `num_outputs >= 1`. Trivial for n=1 + /// (only invariant 2 is non-trivial there). + fn assert_multi_output_invariants( + outputs: &[crate::merge::MergeOutputFile], + total_input_rows: usize, + ) { + let total_output_rows: u64 = outputs.iter().map(|o| o.num_rows as u64).sum(); + assert_eq!( + total_output_rows, total_input_rows as u64, + "sum of output row counts ({total_output_rows}) must equal total input rows \ + ({total_input_rows}) — MC-1" + ); + + let mut series_ranges: Vec<(Vec, Vec, &Path)> = Vec::with_capacity(outputs.len()); + for output in outputs { + let batch = read_parquet_file(&output.path); + let series = extract_binary_column(&batch, SORTED_SERIES_COLUMN); + assert!( + !series.is_empty(), + "output {} has zero rows (engine should have dropped it)", + output.path.display(), + ); + for i in 1..series.len() { + assert!( + series[i] >= series[i - 1], + "output {}: sorted_series not monotone at row {i}", + output.path.display(), + ); + } + series_ranges.push(( + series.first().unwrap().clone(), + series.last().unwrap().clone(), + output.path.as_path(), + )); + } + + // Sort by min_series so we can walk adjacent pairs to check + // disjointness. The engine's staged order isn't a documented + // contract — sort here for the comparison. + series_ranges.sort_by(|a, b| a.0.cmp(&b.0)); + for window in series_ranges.windows(2) { + let (_, left_max, left_path) = &window[0]; + let (right_min, _, right_path) = &window[1]; + assert!( + left_max < right_min, + "outputs {} and {} overlap on sorted_series: left max = {:?}, right min = {:?}", + left_path.display(), + right_path.display(), + left_max, + right_min, + ); + } + } + + /// Multi-input, single-output merge. The streaming and in-memory engines + /// must produce the same number of rows and the same row-by-row content + /// on every visible column. + #[test] + fn parity_multi_input_single_output() { + let dir = TempDir::new().unwrap(); + + // Three inputs with overlapping metric names so the merge interleaves + // across all three streams (the streaming engine's hot path under the + // page-bounded body cache). + let input1 = write_test_split( + dir.path(), + "p1.parquet", + &["cpu", "cpu", "mem"], + &[300, 100, 200], + &[1.0, 2.0, 3.0], + &[42, 42, 99], + ); + let input2 = write_test_split( + dir.path(), + "p2.parquet", + &["cpu", "mem", "mem"], + &[250, 250, 100], + &[4.0, 5.0, 6.0], + &[42, 99, 99], + ); + let input3 = write_test_split( + dir.path(), + "p3.parquet", + &["cpu", "mem", "net"], + &[200, 150, 100], + &[7.0, 8.0, 9.0], + &[42, 99, 7], + ); + + assert_engine_parity(&[input1, input2, input3], 1); + } + + /// Multi-input, multi-output merge. With `num_outputs > 1` the streaming + /// engine splits at `sorted_series` boundaries; this test guards that the + /// split policy and per-output row content match the in-memory engine, + /// and that the engine's multi-output contract holds (sum-equals-total, + /// internal monotonicity, inter-output disjointness — see + /// `assert_multi_output_invariants`). + #[test] + fn parity_multi_input_multi_output() { + let dir = TempDir::new().unwrap(); + + let input1 = write_test_split( + dir.path(), + "m1.parquet", + &["alpha", "beta", "gamma"], + &[100, 100, 100], + &[1.0, 2.0, 3.0], + &[1, 2, 3], + ); + let input2 = write_test_split( + dir.path(), + "m2.parquet", + &["alpha", "beta", "gamma"], + &[200, 200, 200], + &[4.0, 5.0, 6.0], + &[1, 2, 3], + ); + + assert_engine_parity(&[input1, input2], 3); + } + + /// Stress variant: 3 inputs × 3 metrics with the per-metric keyspaces + /// **overlapping across inputs** (each metric appears in every input + /// with the same `timeseries_id` range and overlapping but distinct + /// timestamps). The merge must heavily interleave rows from all three + /// inputs, not concatenate them. Asserts engine-vs-engine parity plus + /// the multi-output contract (disjointness across the three outputs). + #[test] + fn parity_multi_metric_overlapping_inputs_multi_output() { + let dir = TempDir::new().unwrap(); + + // Each input has 3 metrics × 10 rows. For metric=alpha, ts_id_base + // depends only on the metric name (see write_test_split), so all + // three inputs share the same set of timeseries_ids per metric. + // Timestamps are chosen so each (metric, timeseries) appears in + // all three inputs at three overlapping-but-distinct times — the + // merge must interleave row-by-row. + let input_x = write_test_split( + dir.path(), + "x.parquet", + // Per-input layout: alpha x10, beta x10, gamma x10 = 30 rows. + &[ + "alpha", "alpha", "alpha", "alpha", "alpha", "alpha", "alpha", "alpha", "alpha", + "alpha", "beta", "beta", "beta", "beta", "beta", "beta", "beta", "beta", "beta", + "beta", "gamma", "gamma", "gamma", "gamma", "gamma", "gamma", "gamma", "gamma", + "gamma", "gamma", + ], + // input-x: each metric at ts 100..109. + &[ + 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, // alpha + 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, // beta + 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, // gamma + ], + &[ + 1.0, 1.1, 1.2, 1.3, 1.4, 1.5, 1.6, 1.7, 1.8, 1.9, 2.0, 2.1, 2.2, 2.3, 2.4, 2.5, + 2.6, 2.7, 2.8, 2.9, 3.0, 3.1, 3.2, 3.3, 3.4, 3.5, 3.6, 3.7, 3.8, 3.9, + ], + // Each (metric, row-position) within the input has a unique + // ts_id, but cross-input collisions on the same (metric, pos) + // ARE intentional — that's what makes the merge interleave. + &[ + 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, // alpha: ts_ids 1-10 + 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, // beta: 11-20 + 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, // gamma: 21-30 + ], + ); + let input_y = write_test_split( + dir.path(), + "y.parquet", + &[ + "alpha", "alpha", "alpha", "alpha", "alpha", "alpha", "alpha", "alpha", "alpha", + "alpha", "beta", "beta", "beta", "beta", "beta", "beta", "beta", "beta", "beta", + "beta", "gamma", "gamma", "gamma", "gamma", "gamma", "gamma", "gamma", "gamma", + "gamma", "gamma", + ], + // input-y: each metric at ts 110..119. + &[ + 110, 111, 112, 113, 114, 115, 116, 117, 118, 119, 110, 111, 112, 113, 114, 115, + 116, 117, 118, 119, 110, 111, 112, 113, 114, 115, 116, 117, 118, 119, + ], + &[ + 4.0, 4.1, 4.2, 4.3, 4.4, 4.5, 4.6, 4.7, 4.8, 4.9, 5.0, 5.1, 5.2, 5.3, 5.4, 5.5, + 5.6, 5.7, 5.8, 5.9, 6.0, 6.1, 6.2, 6.3, 6.4, 6.5, 6.6, 6.7, 6.8, 6.9, + ], + // Same ts_id ranges as input-x — collisions intentional. + &[ + 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, + 24, 25, 26, 27, 28, 29, 30, + ], + ); + let input_z = write_test_split( + dir.path(), + "z.parquet", + &[ + "alpha", "alpha", "alpha", "alpha", "alpha", "alpha", "alpha", "alpha", "alpha", + "alpha", "beta", "beta", "beta", "beta", "beta", "beta", "beta", "beta", "beta", + "beta", "gamma", "gamma", "gamma", "gamma", "gamma", "gamma", "gamma", "gamma", + "gamma", "gamma", + ], + // input-z: each metric at ts 105..114 (interleaved with x and y). + &[ + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 105, 106, 107, 108, 109, 110, + 111, 112, 113, 114, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + ], + &[ + 7.0, 7.1, 7.2, 7.3, 7.4, 7.5, 7.6, 7.7, 7.8, 7.9, 8.0, 8.1, 8.2, 8.3, 8.4, 8.5, + 8.6, 8.7, 8.8, 8.9, 9.0, 9.1, 9.2, 9.3, 9.4, 9.5, 9.6, 9.7, 9.8, 9.9, + ], + &[ + 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, + 24, 25, 26, 27, 28, 29, 30, + ], + ); + + // Three outputs targeted. The engine splits at sorted_series + // boundaries; with three distinct metric_name values and a + // single timeseries_id per (metric, row-position), there are + // enough natural boundaries to produce three outputs. + assert_engine_parity(&[input_x, input_y, input_z], 3); + } +} + // ---- Proptest DST: property-based invariant verification ---- mod proptests {