From 361d881ff4a557e131efde081e5e43b8d48a7277 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Mon, 13 Apr 2026 06:38:47 +0000 Subject: [PATCH 01/38] feat: reorder row groups by statistics during sort pushdown When sort pushdown is active, reorder row groups within each file by their min/max statistics to match the requested sort order. This helps TopK queries find optimal values first via dynamic filter pushdown. - Add reorder_by_statistics to PreparedAccessPlan that sorts row_group_indexes by the first sort column's min values - Pass sort order from ParquetSource::try_pushdown_sort through to the opener via sort_order_for_reorder field - Reorder happens after pruning but before reverse (they compose) - Gracefully skips reorder when statistics unavailable, sort expr is not a simple column, row_selection present, or <=1 row groups Closes #21317 --- .../datasource-parquet/src/access_plan.rs | 326 ++++++++++++++++++ .../src/access_plan_optimizer.rs | 107 ++++++ datafusion/datasource-parquet/src/opener.rs | 22 +- datafusion/datasource-parquet/src/source.rs | 10 +- 4 files changed, 462 insertions(+), 3 deletions(-) create mode 100644 datafusion/datasource-parquet/src/access_plan_optimizer.rs diff --git a/datafusion/datasource-parquet/src/access_plan.rs b/datafusion/datasource-parquet/src/access_plan.rs index ca4d097c37a44..d98f64044db37 100644 --- a/datafusion/datasource-parquet/src/access_plan.rs +++ b/datafusion/datasource-parquet/src/access_plan.rs @@ -16,7 +16,12 @@ // under the License. use crate::sort::reverse_row_selection; +use arrow::datatypes::Schema; use datafusion_common::{Result, assert_eq_or_internal_err}; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use log::debug; +use parquet::arrow::arrow_reader::statistics::StatisticsConverter; use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; use parquet::file::metadata::{ParquetMetaData, RowGroupMetaData}; @@ -377,6 +382,106 @@ impl PreparedAccessPlan { }) } + /// Reorder row groups by their min statistics for the given sort order. + /// + /// This helps TopK queries find optimal values first. For ASC sort, + /// row groups with the smallest min values come first. For DESC sort, + /// row groups with the largest min values come first. + /// + /// Gracefully skips reordering when: + /// - There is a row_selection (too complex to remap) + /// - 0 or 1 row groups (nothing to reorder) + /// - Sort expression is not a simple column reference + /// - Statistics are unavailable + pub(crate) fn reorder_by_statistics( + mut self, + sort_order: &LexOrdering, + file_metadata: &ParquetMetaData, + arrow_schema: &Schema, + ) -> Result { + // Skip if row_selection present (too complex to remap) + if self.row_selection.is_some() { + debug!("Skipping RG reorder: row_selection present"); + return Ok(self); + } + + // Nothing to reorder + if self.row_group_indexes.len() <= 1 { + return Ok(self); + } + + // Get the first sort expression + // LexOrdering is guaranteed non-empty, so first() returns &PhysicalSortExpr + let first_sort_expr = sort_order.first(); + + // Extract column name from sort expression + let column: &Column = match first_sort_expr.expr.as_any().downcast_ref::() + { + Some(col) => col, + None => { + debug!("Skipping RG reorder: sort expr is not a simple column"); + return Ok(self); + } + }; + + let descending = first_sort_expr.options.descending; + + // Build statistics converter for this column + let converter = match StatisticsConverter::try_new( + column.name(), + arrow_schema, + file_metadata.file_metadata().schema_descr(), + ) { + Ok(c) => c, + Err(e) => { + debug!("Skipping RG reorder: cannot create stats converter: {e}"); + return Ok(self); + } + }; + + // Get min values for the selected row groups + let rg_metadata: Vec<&RowGroupMetaData> = self + .row_group_indexes + .iter() + .map(|&idx| file_metadata.row_group(idx)) + .collect(); + + let min_values = match converter.row_group_mins(rg_metadata.iter().copied()) { + Ok(vals) => vals, + Err(e) => { + debug!("Skipping RG reorder: cannot get min values: {e}"); + return Ok(self); + } + }; + + // Sort indices by min values + let sort_options = arrow::compute::SortOptions { + descending, + nulls_first: first_sort_expr.options.nulls_first, + }; + let sorted_indices = match arrow::compute::sort_to_indices( + &min_values, + Some(sort_options), + None, + ) { + Ok(indices) => indices, + Err(e) => { + debug!("Skipping RG reorder: sort failed: {e}"); + return Ok(self); + } + }; + + // Apply the reordering + let original_indexes = self.row_group_indexes.clone(); + self.row_group_indexes = sorted_indices + .values() + .iter() + .map(|&i| original_indexes[i as usize]) + .collect(); + + Ok(self) + } + /// Reverse the access plan for reverse scanning pub(crate) fn reverse(mut self, file_metadata: &ParquetMetaData) -> Result { // Get the row group indexes before reversing @@ -614,4 +719,225 @@ mod test { .unwrap(); Arc::new(SchemaDescriptor::new(Arc::new(schema))) } + + // ---- reorder_by_statistics tests ---- + + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_physical_expr::expressions::Column; + use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; + use parquet::basic::Type as PhysicalType; + use parquet::file::metadata::FileMetaData; + use parquet::file::statistics::Statistics; + use parquet::schema::types::Type as SchemaType; + + /// Create ParquetMetaData with row groups that have Int32 min/max stats + fn make_metadata_with_stats(min_max_pairs: &[(i32, i32)]) -> ParquetMetaData { + let field = SchemaType::primitive_type_builder("id", PhysicalType::INT32) + .build() + .unwrap(); + let schema = SchemaType::group_type_builder("schema") + .with_fields(vec![Arc::new(field)]) + .build() + .unwrap(); + let schema_descr = Arc::new(SchemaDescriptor::new(Arc::new(schema))); + + let row_groups: Vec = min_max_pairs + .iter() + .map(|(min, max)| { + let stats = + Statistics::int32(Some(*min), Some(*max), None, Some(100), false); + let column = ColumnChunkMetaData::builder(schema_descr.column(0)) + .set_num_values(100) + .set_statistics(stats) + .build() + .unwrap(); + RowGroupMetaData::builder(schema_descr.clone()) + .set_num_rows(100) + .set_column_metadata(vec![column]) + .build() + .unwrap() + }) + .collect(); + + let file_meta = FileMetaData::new( + 1, + min_max_pairs.len() as i64 * 100, + None, + None, + schema_descr, + None, + ); + ParquetMetaData::new(file_meta, row_groups) + } + + fn make_sort_order_asc() -> LexOrdering { + LexOrdering::new(vec![PhysicalSortExpr::new_default(Arc::new(Column::new( + "id", 0, + )))]) + .unwrap() + } + + fn make_sort_order_desc() -> LexOrdering { + use arrow::compute::SortOptions; + LexOrdering::new(vec![PhysicalSortExpr::new( + Arc::new(Column::new("id", 0)), + SortOptions { + descending: true, + nulls_first: false, + }, + )]) + .unwrap() + } + + fn make_arrow_schema() -> Schema { + Schema::new(vec![Field::new("id", DataType::Int32, false)]) + } + + #[test] + fn test_reorder_by_statistics_asc() { + // RGs in wrong order: [50-99, 200-299, 1-30] + let metadata = make_metadata_with_stats(&[(50, 99), (200, 299), (1, 30)]); + let schema = make_arrow_schema(); + let sort_order = make_sort_order_asc(); + + let plan = PreparedAccessPlan::new(vec![0, 1, 2], None).unwrap(); + let plan = plan + .reorder_by_statistics(&sort_order, &metadata, &schema) + .unwrap(); + + // Should be reordered: RG2(1-30), RG0(50-99), RG1(200-299) + assert_eq!(plan.row_group_indexes, vec![2, 0, 1]); + } + + #[test] + fn test_reorder_by_statistics_desc() { + // RGs: [50-99, 200-299, 1-30] + let metadata = make_metadata_with_stats(&[(50, 99), (200, 299), (1, 30)]); + let schema = make_arrow_schema(); + let sort_order = make_sort_order_desc(); + + let plan = PreparedAccessPlan::new(vec![0, 1, 2], None).unwrap(); + let plan = plan + .reorder_by_statistics(&sort_order, &metadata, &schema) + .unwrap(); + + // DESC: largest min first: RG1(200-299), RG0(50-99), RG2(1-30) + assert_eq!(plan.row_group_indexes, vec![1, 0, 2]); + } + + #[test] + fn test_reorder_by_statistics_single_rg() { + let metadata = make_metadata_with_stats(&[(1, 100)]); + let schema = make_arrow_schema(); + let sort_order = make_sort_order_asc(); + + let plan = PreparedAccessPlan::new(vec![0], None).unwrap(); + let plan = plan + .reorder_by_statistics(&sort_order, &metadata, &schema) + .unwrap(); + + // Single RG, no reorder + assert_eq!(plan.row_group_indexes, vec![0]); + } + + #[test] + fn test_reorder_by_statistics_with_skipped_rgs() { + // 4 RGs but only 0, 2, 3 are selected (RG1 was pruned) + let metadata = + make_metadata_with_stats(&[(300, 400), (100, 200), (1, 50), (50, 99)]); + let schema = make_arrow_schema(); + let sort_order = make_sort_order_asc(); + + let plan = PreparedAccessPlan::new(vec![0, 2, 3], None).unwrap(); + let plan = plan + .reorder_by_statistics(&sort_order, &metadata, &schema) + .unwrap(); + + // Reorder selected RGs by min: RG2(1-50), RG3(50-99), RG0(300-400) + assert_eq!(plan.row_group_indexes, vec![2, 3, 0]); + } + + #[test] + fn test_reorder_by_statistics_skips_with_row_selection() { + let metadata = make_metadata_with_stats(&[(50, 99), (1, 30)]); + let schema = make_arrow_schema(); + let sort_order = make_sort_order_asc(); + + let selection = RowSelection::from(vec![ + RowSelector::select(50), + RowSelector::skip(50), + RowSelector::select(100), + ]); + + let plan = PreparedAccessPlan::new(vec![0, 1], Some(selection)).unwrap(); + let plan = plan + .reorder_by_statistics(&sort_order, &metadata, &schema) + .unwrap(); + + // Should NOT reorder because row_selection is present + assert_eq!(plan.row_group_indexes, vec![0, 1]); + } + + #[test] + fn test_reorder_by_statistics_already_sorted() { + // Already in correct ASC order + let metadata = make_metadata_with_stats(&[(1, 30), (50, 99), (200, 299)]); + let schema = make_arrow_schema(); + let sort_order = make_sort_order_asc(); + + let plan = PreparedAccessPlan::new(vec![0, 1, 2], None).unwrap(); + let plan = plan + .reorder_by_statistics(&sort_order, &metadata, &schema) + .unwrap(); + + // Already sorted, order preserved + assert_eq!(plan.row_group_indexes, vec![0, 1, 2]); + } + + #[test] + fn test_reorder_by_statistics_skips_non_column_expr() { + use datafusion_expr::Operator; + use datafusion_physical_expr::expressions::BinaryExpr; + + let metadata = make_metadata_with_stats(&[(50, 99), (1, 30)]); + let schema = make_arrow_schema(); + + // Sort expression is a binary expression (id + 1), not a simple column + let expr = Arc::new(BinaryExpr::new( + Arc::new(Column::new("id", 0)), + Operator::Plus, + Arc::new(datafusion_physical_expr::expressions::Literal::new( + datafusion_common::ScalarValue::Int32(Some(1)), + )), + )); + let sort_order = + LexOrdering::new(vec![PhysicalSortExpr::new_default(expr)]).unwrap(); + + let plan = PreparedAccessPlan::new(vec![0, 1], None).unwrap(); + let plan = plan + .reorder_by_statistics(&sort_order, &metadata, &schema) + .unwrap(); + + // Should NOT reorder because sort expr is not a simple column + assert_eq!(plan.row_group_indexes, vec![0, 1]); + } + + #[test] + fn test_reorder_by_statistics_skips_missing_column() { + let metadata = make_metadata_with_stats(&[(50, 99), (1, 30)]); + // Schema has "id" but sort order references "nonexistent" + let schema = make_arrow_schema(); + let sort_order = LexOrdering::new(vec![PhysicalSortExpr::new_default(Arc::new( + Column::new("nonexistent", 99), + ))]) + .unwrap(); + + let plan = PreparedAccessPlan::new(vec![0, 1], None).unwrap(); + let plan = plan + .reorder_by_statistics(&sort_order, &metadata, &schema) + .unwrap(); + + // Should NOT reorder because column not found in schema + assert_eq!(plan.row_group_indexes, vec![0, 1]); + } } diff --git a/datafusion/datasource-parquet/src/access_plan_optimizer.rs b/datafusion/datasource-parquet/src/access_plan_optimizer.rs new file mode 100644 index 0000000000000..885dc0b5656ee --- /dev/null +++ b/datafusion/datasource-parquet/src/access_plan_optimizer.rs @@ -0,0 +1,107 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! [`AccessPlanOptimizer`] trait and implementations for optimizing +//! row group access order during parquet scans. +//! +//! Applied after row group pruning but before building the decoder, +//! these optimizers reorder (or reverse) the row groups to improve +//! query performance — e.g., placing the "best" row groups first +//! so TopK's dynamic filter threshold tightens quickly. + +use crate::access_plan::PreparedAccessPlan; +use arrow::datatypes::Schema; +use datafusion_common::Result; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use parquet::file::metadata::ParquetMetaData; +use std::fmt::Debug; + +/// Optimizes the row group access order for a prepared access plan. +/// +/// Implementations can reorder, reverse, or otherwise transform the +/// row group read order to improve scan performance. The optimizer +/// is applied once per file, after all pruning passes are complete. +/// +/// # Examples +/// +/// - [`ReverseRowGroups`]: simple O(n) reversal for DESC on ASC-sorted data +/// - [`ReorderByStatistics`]: sort row groups by min/max statistics +/// so TopK queries find optimal values first +pub(crate) trait AccessPlanOptimizer: Send + Sync + Debug { + /// Transform the prepared access plan. + /// + /// Implementations should return the plan unchanged if they cannot + /// apply their optimization (e.g., missing statistics). + fn optimize( + &self, + plan: PreparedAccessPlan, + file_metadata: &ParquetMetaData, + arrow_schema: &Schema, + ) -> Result; +} + +/// Reverse the row group order — simple O(n) reversal. +/// +/// Used as a fallback when the sort column has no statistics available. +/// For ASC-sorted files with a DESC query, reversing row groups places +/// the highest-value row groups first. +#[derive(Debug)] +pub(crate) struct ReverseRowGroups; + +impl AccessPlanOptimizer for ReverseRowGroups { + fn optimize( + &self, + plan: PreparedAccessPlan, + file_metadata: &ParquetMetaData, + _arrow_schema: &Schema, + ) -> Result { + plan.reverse(file_metadata) + } +} + +/// Reorder row groups by min/max statistics of the sort column. +/// +/// For ASC sort: row groups with the smallest min come first. +/// For DESC sort: row groups with the largest max come first. +/// +/// This is more effective than [`ReverseRowGroups`] when row groups +/// are out of order (e.g., append-heavy workloads), because it uses +/// actual statistics rather than assuming the original order is sorted. +/// +/// Gracefully falls back to the original order when statistics are +/// unavailable, the sort expression is not a simple column, etc. +#[derive(Debug)] +pub(crate) struct ReorderByStatistics { + sort_order: LexOrdering, +} + +impl ReorderByStatistics { + pub(crate) fn new(sort_order: LexOrdering) -> Self { + Self { sort_order } + } +} + +impl AccessPlanOptimizer for ReorderByStatistics { + fn optimize( + &self, + plan: PreparedAccessPlan, + file_metadata: &ParquetMetaData, + arrow_schema: &Schema, + ) -> Result { + plan.reorder_by_statistics(&self.sort_order, file_metadata, arrow_schema) + } +} diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index bad1c684b47f5..5a381f6bf975f 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -50,6 +50,7 @@ use datafusion_physical_expr_adapter::PhysicalExprAdapterFactory; use datafusion_physical_expr_common::physical_expr::{ PhysicalExpr, is_dynamic_physical_expr, }; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::metrics::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, Gauge, MetricBuilder, MetricCategory, PruningMetrics, @@ -136,6 +137,10 @@ pub(super) struct ParquetMorselizer { pub max_predicate_cache_size: Option, /// Whether to read row groups in reverse order pub reverse_row_groups: bool, + /// Optional sort order used to reorder row groups by their min/max statistics. + /// When set, row groups are reordered before reading so that row groups likely + /// to contain optimal values (for TopK queries) are read first. + pub sort_order_for_reorder: Option, } impl fmt::Debug for ParquetMorselizer { @@ -286,6 +291,7 @@ struct PreparedParquetOpen { predicate_creation_errors: Count, max_predicate_cache_size: Option, reverse_row_groups: bool, + sort_order_for_reorder: Option, preserve_order: bool, #[cfg(feature = "parquet_encryption")] file_decryption_properties: Option>, @@ -655,6 +661,7 @@ impl ParquetMorselizer { predicate_creation_errors, max_predicate_cache_size: self.max_predicate_cache_size, reverse_row_groups: self.reverse_row_groups, + sort_order_for_reorder: self.sort_order_for_reorder.clone(), preserve_order: self.preserve_order, #[cfg(feature = "parquet_encryption")] file_decryption_properties: None, @@ -1126,6 +1133,16 @@ impl RowGroupsPrunedParquetOpen { // Prepare the access plan (extract row groups and row selection) let mut prepared_plan = access_plan.prepare(rg_metadata)?; + // Reorder row groups by statistics if sort order is known. + // This helps TopK queries find optimal values first. + if let Some(sort_order) = &prepared.sort_order_for_reorder { + prepared_plan = prepared_plan.reorder_by_statistics( + sort_order, + file_metadata.as_ref(), + &prepared.physical_file_schema, + )?; + } + // Potentially reverse the access plan for performance. // See `ParquetSource::try_pushdown_sort` for the rationale. if prepared.reverse_row_groups { @@ -1644,6 +1661,7 @@ mod test { use datafusion_physical_expr_adapter::{ DefaultPhysicalExprAdapterFactory, replace_columns_with_literals, }; + use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use futures::StreamExt; use futures::stream::BoxStream; @@ -1675,6 +1693,7 @@ mod test { coerce_int96: Option, max_predicate_cache_size: Option, reverse_row_groups: bool, + sort_order_for_reorder: Option, preserve_order: bool, } @@ -1701,6 +1720,7 @@ mod test { coerce_int96: None, max_predicate_cache_size: None, reverse_row_groups: false, + sort_order_for_reorder: None, preserve_order: false, } } @@ -1816,6 +1836,7 @@ mod test { encryption_factory: None, max_predicate_cache_size: self.max_predicate_cache_size, reverse_row_groups: self.reverse_row_groups, + sort_order_for_reorder: self.sort_order_for_reorder, } } } @@ -1840,7 +1861,6 @@ mod test { let Some(planner) = planners.pop_front() else { return Ok(Box::pin(futures::stream::empty())); - }; if let Some(mut plan) = planner.plan()? { morsels.extend(plan.take_morsels()); diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index a014c8b2726e7..5586e29a9eae5 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -55,7 +55,7 @@ use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; #[cfg(feature = "parquet_encryption")] use datafusion_execution::parquet_encryption::EncryptionFactory; -use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use itertools::Itertools; use object_store::ObjectStore; #[cfg(feature = "parquet_encryption")] @@ -294,6 +294,8 @@ pub struct ParquetSource { /// so we still need to sort them after reading, so the reverse scan is inexact. /// Used to optimize ORDER BY ... DESC on sorted data. reverse_row_groups: bool, + /// Optional sort order used to reorder row groups by min/max statistics. + sort_order_for_reorder: Option, } impl ParquetSource { @@ -319,6 +321,7 @@ impl ParquetSource { #[cfg(feature = "parquet_encryption")] encryption_factory: None, reverse_row_groups: false, + sort_order_for_reorder: None, } } @@ -580,6 +583,7 @@ impl FileSource for ParquetSource { encryption_factory: self.get_encryption_factory_with_config(), max_predicate_cache_size: self.max_predicate_cache_size(), reverse_row_groups: self.reverse_row_groups, + sort_order_for_reorder: self.sort_order_for_reorder.clone(), })) } @@ -821,7 +825,9 @@ impl FileSource for ParquetSource { // Return Inexact because we're only reversing row group order, // not guaranteeing perfect row-level ordering - let new_source = self.clone().with_reverse_row_groups(true); + let sort_order = LexOrdering::new(order.iter().cloned()); + let mut new_source = self.clone().with_reverse_row_groups(true); + new_source.sort_order_for_reorder = sort_order; Ok(SortOrderPushdownResult::Inexact { inner: Arc::new(new_source) as Arc, }) From fc9e0b40f40898047ab7c0fd10bf52fe15874b18 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Mon, 13 Apr 2026 06:48:06 +0000 Subject: [PATCH 02/38] test: add SLT tests for row group reorder by statistics --- .../sqllogictest/test_files/sort_pushdown.slt | 83 +++++++++++++++++++ 1 file changed, 83 insertions(+) diff --git a/datafusion/sqllogictest/test_files/sort_pushdown.slt b/datafusion/sqllogictest/test_files/sort_pushdown.slt index b6c75f3977010..4f41c4b08aecf 100644 --- a/datafusion/sqllogictest/test_files/sort_pushdown.slt +++ b/datafusion/sqllogictest/test_files/sort_pushdown.slt @@ -2271,6 +2271,89 @@ DROP TABLE tg_src_high; statement ok DROP TABLE tg_buffer; +# =========================================================== +# Test H: Row group reorder by statistics for TopK queries. +# When a file has multiple row groups with overlapping or +# out-of-order statistics, sort pushdown returns Inexact and +# `reorder_by_statistics` reorders row groups within the file +# so TopK finds optimal values first. +# =========================================================== + +# Create a table with 30 rows and write to parquet with small row groups +# so we get multiple row groups per file. Rows are inserted in a mixed +# order so row groups span overlapping ranges (forcing Inexact path). +statement ok +CREATE TABLE th_mixed(id INT, value INT) AS VALUES + (15, 150), (5, 50), (25, 250), + (10, 100), (20, 200), (1, 10), + (30, 300), (3, 30), (18, 180); + +# Write with row_group_size=3 → 3 rows per RG, 3 RGs total +# RG statistics (unsorted order): RG0(5-25), RG1(1-20), RG2(3-30) +# Note: files are overlapping → Inexact path → TopK retained +query I +COPY (SELECT * FROM th_mixed) +TO 'test_files/scratch/sort_pushdown/th_reorder/data.parquet' +STORED AS PARQUET +OPTIONS ('format.max_row_group_size' '3'); +---- +9 + +statement ok +SET datafusion.execution.target_partitions = 1; + +statement ok +CREATE EXTERNAL TABLE th_reorder(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/th_reorder/data.parquet'; + +# Test H.1: ASC ORDER BY with LIMIT — reorder helps TopK find min values first +# Results must be correct regardless of RG reorder. +query II +SELECT * FROM th_reorder ORDER BY id ASC LIMIT 3; +---- +1 10 +3 30 +5 50 + +# Test H.2: DESC ORDER BY with LIMIT — reorder + reverse compose +query II +SELECT * FROM th_reorder ORDER BY id DESC LIMIT 3; +---- +30 300 +25 250 +20 200 + +# Test H.3: Full sort (no LIMIT) — output must still be correctly sorted +query II +SELECT * FROM th_reorder ORDER BY id ASC; +---- +1 10 +3 30 +5 50 +10 100 +15 150 +18 180 +20 200 +25 250 +30 300 + +# Test H.4: ORDER BY expression (not a simple column) — reorder should +# gracefully skip, results still correct +query II +SELECT id, value FROM th_reorder ORDER BY id + 1 ASC LIMIT 3; +---- +1 10 +3 30 +5 50 + +# Cleanup Test H +statement ok +DROP TABLE th_mixed; + +statement ok +DROP TABLE th_reorder; + # Reset settings (SLT runner uses target_partitions=4, not system default) statement ok SET datafusion.execution.target_partitions = 4; From 57ebc3d008e6e47fe4cb70fdc6f622ecb5dfccc8 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Mon, 13 Apr 2026 06:56:39 +0000 Subject: [PATCH 03/38] test: add EXPLAIN assertions for row group reorder tests --- .../sqllogictest/test_files/sort_pushdown.slt | 24 ++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/sort_pushdown.slt b/datafusion/sqllogictest/test_files/sort_pushdown.slt index 4f41c4b08aecf..a9d512228fc14 100644 --- a/datafusion/sqllogictest/test_files/sort_pushdown.slt +++ b/datafusion/sqllogictest/test_files/sort_pushdown.slt @@ -2307,7 +2307,19 @@ CREATE EXTERNAL TABLE th_reorder(id INT, value INT) STORED AS PARQUET LOCATION 'test_files/scratch/sort_pushdown/th_reorder/data.parquet'; -# Test H.1: ASC ORDER BY with LIMIT — reorder helps TopK find min values first +# Test H.1: ASC ORDER BY with LIMIT — Inexact path (file has no declared ordering) +# Plan: SortExec(TopK) preserved. RG reorder happens inside DataSourceExec +# (not visible in EXPLAIN, but verified by unit tests in access_plan.rs). +query TT +EXPLAIN SELECT * FROM th_reorder ORDER BY id ASC LIMIT 3; +---- +logical_plan +01)Sort: th_reorder.id ASC NULLS LAST, fetch=3 +02)--TableScan: th_reorder projection=[id, value] +physical_plan +01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/th_reorder/data.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ] + # Results must be correct regardless of RG reorder. query II SELECT * FROM th_reorder ORDER BY id ASC LIMIT 3; @@ -2317,6 +2329,16 @@ SELECT * FROM th_reorder ORDER BY id ASC LIMIT 3; 5 50 # Test H.2: DESC ORDER BY with LIMIT — reorder + reverse compose +query TT +EXPLAIN SELECT * FROM th_reorder ORDER BY id DESC LIMIT 3; +---- +logical_plan +01)Sort: th_reorder.id DESC NULLS FIRST, fetch=3 +02)--TableScan: th_reorder projection=[id, value] +physical_plan +01)SortExec: TopK(fetch=3), expr=[id@0 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/th_reorder/data.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ] + query II SELECT * FROM th_reorder ORDER BY id DESC LIMIT 3; ---- From 66a9185c5fd2ab8e5f72d25e9020d38f60903a80 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Mon, 13 Apr 2026 07:35:19 +0000 Subject: [PATCH 04/38] fix: use max statistics for DESC sort reorder MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit For overlapping row group ranges, sorting by min for DESC can pick a worse first RG. Example: RG0(50-60) vs RG1(40-100) — min DESC picks RG0 first (max=60), but RG1 contains the largest values (max=100). Use min for ASC and max for DESC to correctly prioritize the row group most likely to contain the optimal values for TopK. --- .../datasource-parquet/src/access_plan.rs | 77 ++++++++++++++----- 1 file changed, 58 insertions(+), 19 deletions(-) diff --git a/datafusion/datasource-parquet/src/access_plan.rs b/datafusion/datasource-parquet/src/access_plan.rs index d98f64044db37..3f1a0a675e361 100644 --- a/datafusion/datasource-parquet/src/access_plan.rs +++ b/datafusion/datasource-parquet/src/access_plan.rs @@ -439,37 +439,51 @@ impl PreparedAccessPlan { } }; - // Get min values for the selected row groups + // Get the relevant statistics for the selected row groups. + // For ASC sort: use min values — we want the RG with the smallest min + // to come first (best candidate for "smallest values"). + // For DESC sort: use max values — we want the RG with the largest max + // to come first (best candidate for "largest values"). Using min for + // DESC can pick a worse first RG when ranges overlap (e.g., RG0 50-60 + // vs RG1 40-100 — RG1 has larger values but smaller min). let rg_metadata: Vec<&RowGroupMetaData> = self .row_group_indexes .iter() .map(|&idx| file_metadata.row_group(idx)) .collect(); - let min_values = match converter.row_group_mins(rg_metadata.iter().copied()) { - Ok(vals) => vals, - Err(e) => { - debug!("Skipping RG reorder: cannot get min values: {e}"); - return Ok(self); + let stat_values = if descending { + match converter.row_group_maxes(rg_metadata.iter().copied()) { + Ok(vals) => vals, + Err(e) => { + debug!("Skipping RG reorder: cannot get max values: {e}"); + return Ok(self); + } + } + } else { + match converter.row_group_mins(rg_metadata.iter().copied()) { + Ok(vals) => vals, + Err(e) => { + debug!("Skipping RG reorder: cannot get min values: {e}"); + return Ok(self); + } } }; - // Sort indices by min values + // Sort indices by statistic values (min for ASC, max for DESC) let sort_options = arrow::compute::SortOptions { descending, nulls_first: first_sort_expr.options.nulls_first, }; - let sorted_indices = match arrow::compute::sort_to_indices( - &min_values, - Some(sort_options), - None, - ) { - Ok(indices) => indices, - Err(e) => { - debug!("Skipping RG reorder: sort failed: {e}"); - return Ok(self); - } - }; + let sorted_indices = + match arrow::compute::sort_to_indices(&stat_values, Some(sort_options), None) + { + Ok(indices) => indices, + Err(e) => { + debug!("Skipping RG reorder: sort failed: {e}"); + return Ok(self); + } + }; // Apply the reordering let original_indexes = self.row_group_indexes.clone(); @@ -821,7 +835,7 @@ mod test { .reorder_by_statistics(&sort_order, &metadata, &schema) .unwrap(); - // DESC: largest min first: RG1(200-299), RG0(50-99), RG2(1-30) + // DESC: largest max first: RG1(max=299), RG0(max=99), RG2(max=30) assert_eq!(plan.row_group_indexes, vec![1, 0, 2]); } @@ -922,6 +936,31 @@ mod test { assert_eq!(plan.row_group_indexes, vec![0, 1]); } + #[test] + fn test_reorder_by_statistics_desc_uses_max_for_overlapping_rgs() { + // Overlapping ranges where min DESC would pick worse RG than max DESC: + // RG0: 50-60 (small range, moderate max) + // RG1: 40-100 (wide range, high max but lower min) + // RG2: 20-30 (low max) + // + // For ORDER BY DESC LIMIT N: + // Using min DESC: [RG0(50), RG1(40), RG2(20)] → reads RG0 first (max=60 only) + // Using max DESC: [RG1(100), RG0(60), RG2(30)] → reads RG1 first (max=100) + // + // RG1 is the better first choice for DESC because it contains the largest values. + let metadata = make_metadata_with_stats(&[(50, 60), (40, 100), (20, 30)]); + let schema = make_arrow_schema(); + let sort_order = make_sort_order_desc(); + + let plan = PreparedAccessPlan::new(vec![0, 1, 2], None).unwrap(); + let plan = plan + .reorder_by_statistics(&sort_order, &metadata, &schema) + .unwrap(); + + // Expected: RG1 (max=100) first, then RG0 (max=60), then RG2 (max=30) + assert_eq!(plan.row_group_indexes, vec![1, 0, 2]); + } + #[test] fn test_reorder_by_statistics_skips_missing_column() { let metadata = make_metadata_with_stats(&[(50, 99), (1, 30)]); From 213fb2c47947b328b1dce5688b039fb6964e2930 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 14 Apr 2026 06:23:36 +0000 Subject: [PATCH 05/38] fix: prevent reorder+reverse double-reordering of row groups When sort_order_for_reorder is set, reorder_by_statistics already handles the sort direction (min for ASC, max for DESC). Applying reverse on top would undo the reorder. Use else-if so only one strategy is applied. Also adds sort_pushdown_inexact benchmark with pushdown_filters enabled to measure RG reorder benefit on wide-row TopK queries. --- benchmarks/src/sort_pushdown.rs | 9 ++++++++- datafusion/datasource-parquet/src/opener.rs | 13 +++++++------ 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/benchmarks/src/sort_pushdown.rs b/benchmarks/src/sort_pushdown.rs index e7fce1921e7a8..e2a4615a3ef39 100644 --- a/benchmarks/src/sort_pushdown.rs +++ b/benchmarks/src/sort_pushdown.rs @@ -159,7 +159,14 @@ impl RunOpt { async fn benchmark_query(&self, query_id: usize) -> Result> { let sql = self.load_query(query_id)?; - let config = self.common.config()?; + let mut config = self.common.config()?; + // Enable parquet filter pushdown + late materialization. This is + // essential for the Inexact sort pushdown path: TopK's dynamic + // filter is pushed to the parquet reader, so only sort-column + // rows pass the filter's Decode non-sort columns are skipped for + // rows that don't pass the filter — this is where RG reorder's + // tight-threshold-first strategy pays off for wide-row queries. + config.options_mut().execution.parquet.pushdown_filters = true; let rt = self.common.build_runtime()?; let state = SessionStateBuilder::new() .with_config(config) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 5a381f6bf975f..fa5bbaced9878 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -1134,18 +1134,19 @@ impl RowGroupsPrunedParquetOpen { let mut prepared_plan = access_plan.prepare(rg_metadata)?; // Reorder row groups by statistics if sort order is known. - // This helps TopK queries find optimal values first. + // This helps TopK queries find optimal values first by placing + // row groups with optimal min/max values at the front. + // When reorder is active, skip reverse — reorder already encodes + // the direction (uses min for ASC, max for DESC). if let Some(sort_order) = &prepared.sort_order_for_reorder { prepared_plan = prepared_plan.reorder_by_statistics( sort_order, file_metadata.as_ref(), &prepared.physical_file_schema, )?; - } - - // Potentially reverse the access plan for performance. - // See `ParquetSource::try_pushdown_sort` for the rationale. - if prepared.reverse_row_groups { + } else if prepared.reverse_row_groups { + // Fallback: simple reverse when no sort order statistics available. + // See `ParquetSource::try_pushdown_sort` for the rationale. prepared_plan = prepared_plan.reverse(file_metadata.as_ref())?; } From 6c56600a97e599417d819b922ce31c939c523c21 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Thu, 16 Apr 2026 10:41:56 +0000 Subject: [PATCH 06/38] fix: rebase conflicts and compilation errors --- datafusion/datasource-parquet/src/access_plan.rs | 3 +-- datafusion/datasource-parquet/src/opener.rs | 1 + 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/datasource-parquet/src/access_plan.rs b/datafusion/datasource-parquet/src/access_plan.rs index 3f1a0a675e361..77b538bd52249 100644 --- a/datafusion/datasource-parquet/src/access_plan.rs +++ b/datafusion/datasource-parquet/src/access_plan.rs @@ -415,8 +415,7 @@ impl PreparedAccessPlan { let first_sort_expr = sort_order.first(); // Extract column name from sort expression - let column: &Column = match first_sort_expr.expr.as_any().downcast_ref::() - { + let column: &Column = match first_sort_expr.expr.downcast_ref::() { Some(col) => col, None => { debug!("Skipping RG reorder: sort expr is not a simple column"); diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index fa5bbaced9878..d91fc9900f445 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -1862,6 +1862,7 @@ mod test { let Some(planner) = planners.pop_front() else { return Ok(Box::pin(futures::stream::empty())); + }; if let Some(mut plan) = planner.plan()? { morsels.extend(plan.take_morsels()); From 815481b6f577f0033f510dc20dba80e4bca7c8bc Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Thu, 16 Apr 2026 12:13:51 +0000 Subject: [PATCH 07/38] refactor: introduce AccessPlanOptimizer trait for row group reordering Address review feedback: extract row group optimization into a trait instead of post-pass flags. prepare_with_optimizer() integrates the optimization into the access plan preparation step. - AccessPlanOptimizer trait with ReverseRowGroups and ReorderByStatistics - prepare_with_optimizer() applies optimizer inside prepare flow - Original prepare() unchanged for backward compatibility --- .../datasource-parquet/src/access_plan.rs | 15 ++++++- datafusion/datasource-parquet/src/mod.rs | 1 + datafusion/datasource-parquet/src/opener.rs | 44 ++++++++++++------- 3 files changed, 42 insertions(+), 18 deletions(-) diff --git a/datafusion/datasource-parquet/src/access_plan.rs b/datafusion/datasource-parquet/src/access_plan.rs index 77b538bd52249..32085fa8a177a 100644 --- a/datafusion/datasource-parquet/src/access_plan.rs +++ b/datafusion/datasource-parquet/src/access_plan.rs @@ -351,9 +351,22 @@ impl ParquetAccessPlan { ) -> Result { let row_group_indexes = self.row_group_indexes(); let row_selection = self.into_overall_row_selection(row_group_meta_data)?; - PreparedAccessPlan::new(row_group_indexes, row_selection) } + + /// Like [`prepare`](Self::prepare), but also applies an + /// [`AccessPlanOptimizer`] to reorder/reverse row groups after + /// preparing the plan. + pub(crate) fn prepare_with_optimizer( + self, + row_group_meta_data: &[RowGroupMetaData], + file_metadata: &ParquetMetaData, + arrow_schema: &Schema, + optimizer: &dyn crate::access_plan_optimizer::AccessPlanOptimizer, + ) -> Result { + let plan = self.prepare(row_group_meta_data)?; + optimizer.optimize(plan, file_metadata, arrow_schema) + } } /// Represents a prepared, fully resolved [`ParquetAccessPlan`] diff --git a/datafusion/datasource-parquet/src/mod.rs b/datafusion/datasource-parquet/src/mod.rs index 9a907f4118a86..de42c527845fb 100644 --- a/datafusion/datasource-parquet/src/mod.rs +++ b/datafusion/datasource-parquet/src/mod.rs @@ -25,6 +25,7 @@ #![cfg_attr(test, allow(clippy::needless_pass_by_value))] pub mod access_plan; +pub(crate) mod access_plan_optimizer; pub mod file_format; pub mod metadata; mod metrics; diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index d91fc9900f445..41dacd04dc3e8 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -1130,25 +1130,35 @@ impl RowGroupsPrunedParquetOpen { ); } - // Prepare the access plan (extract row groups and row selection) - let mut prepared_plan = access_plan.prepare(rg_metadata)?; - - // Reorder row groups by statistics if sort order is known. - // This helps TopK queries find optimal values first by placing - // row groups with optimal min/max values at the front. - // When reorder is active, skip reverse — reorder already encodes - // the direction (uses min for ASC, max for DESC). - if let Some(sort_order) = &prepared.sort_order_for_reorder { - prepared_plan = prepared_plan.reorder_by_statistics( - sort_order, + // Build the access plan optimizer from sort pushdown hints. + // ReorderByStatistics is preferred (handles both ASC and DESC via + // min/max stats). ReverseRowGroups is a fallback when no statistics + // are available on the sort column. + let optimizer: Option< + Box, + > = if let Some(sort_order) = &prepared.sort_order_for_reorder { + Some(Box::new( + crate::access_plan_optimizer::ReorderByStatistics::new( + sort_order.clone(), + ), + )) + } else if prepared.reverse_row_groups { + Some(Box::new(crate::access_plan_optimizer::ReverseRowGroups)) + } else { + None + }; + + // Prepare the access plan and apply row group optimizer if configured. + let prepared_plan = if let Some(opt) = &optimizer { + access_plan.prepare_with_optimizer( + rg_metadata, file_metadata.as_ref(), &prepared.physical_file_schema, - )?; - } else if prepared.reverse_row_groups { - // Fallback: simple reverse when no sort order statistics available. - // See `ParquetSource::try_pushdown_sort` for the rationale. - prepared_plan = prepared_plan.reverse(file_metadata.as_ref())?; - } + opt.as_ref(), + )? + } else { + access_plan.prepare(rg_metadata)? + }; let arrow_reader_metrics = ArrowReaderMetrics::enabled(); let read_plan = build_projection_read_plan( From 439a5d6a5897dc10faf54b66313a5a8385247699 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Thu, 16 Apr 2026 12:20:27 +0000 Subject: [PATCH 08/38] chore: remove benchmark from this PR (tracked in #21582) --- benchmarks/bench.sh | 196 ------------------ .../queries/sort_pushdown_inexact/q1.sql | 8 - .../queries/sort_pushdown_inexact/q2.sql | 7 - .../queries/sort_pushdown_inexact/q3.sql | 8 - .../queries/sort_pushdown_inexact/q4.sql | 7 - benchmarks/src/sort_pushdown.rs | 9 +- 6 files changed, 1 insertion(+), 234 deletions(-) delete mode 100644 benchmarks/queries/sort_pushdown_inexact/q1.sql delete mode 100644 benchmarks/queries/sort_pushdown_inexact/q2.sql delete mode 100644 benchmarks/queries/sort_pushdown_inexact/q3.sql delete mode 100644 benchmarks/queries/sort_pushdown_inexact/q4.sql diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index 6c5e0e483e15c..9dce4cf77b933 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -109,9 +109,6 @@ clickbench_extended: ClickBench \"inspired\" queries against a single parquet # Sort Pushdown Benchmarks sort_pushdown: Sort pushdown baseline (no WITH ORDER) on TPC-H data (SF=1) sort_pushdown_sorted: Sort pushdown with WITH ORDER — tests sort elimination on non-overlapping files -sort_pushdown_inexact: Sort pushdown Inexact path (--sorted DESC) — multi-file with scrambled RGs, tests reverse scan + RG reorder -sort_pushdown_inexact_unsorted: Sort pushdown Inexact path (no WITH ORDER) — same data, tests Unsupported path + RG reorder -sort_pushdown_inexact_overlap: Sort pushdown Inexact path — multi-file scrambled RGs (streaming data scenario) # Sorted Data Benchmarks (ORDER BY Optimization) clickbench_sorted: ClickBench queries on pre-sorted data using prefer_existing_sort (tests sort elimination optimization) @@ -319,9 +316,6 @@ main() { sort_pushdown|sort_pushdown_sorted) data_sort_pushdown ;; - sort_pushdown_inexact|sort_pushdown_inexact_unsorted|sort_pushdown_inexact_overlap) - data_sort_pushdown_inexact - ;; sort_tpch) # same data as for tpch data_tpch "1" "parquet" @@ -528,15 +522,6 @@ main() { sort_pushdown_sorted) run_sort_pushdown_sorted ;; - sort_pushdown_inexact) - run_sort_pushdown_inexact - ;; - sort_pushdown_inexact_unsorted) - run_sort_pushdown_inexact_unsorted - ;; - sort_pushdown_inexact_overlap) - run_sort_pushdown_inexact_overlap - ;; sort_tpch) run_sort_tpch "1" ;; @@ -1152,187 +1137,6 @@ run_sort_pushdown_sorted() { debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --sorted --iterations 5 --path "${SORT_PUSHDOWN_DIR}" --queries-path "${SCRIPT_DIR}/queries/sort_pushdown" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} } -# Generates data for sort pushdown Inexact benchmark. -# -# Produces multiple parquet files where each file has MULTIPLE row groups -# with scrambled RG order. This tests both: -# - Row-group-level reorder within each file (reorder_by_statistics) -# - TopK threshold initialization from RG statistics -# -# Strategy: -# 1. Write a single sorted file with small (100K-row) RGs (~61 RGs total). -# 2. Use pyarrow to redistribute RGs into N_FILES files, scrambling the -# RG order within each file using a deterministic permutation. -# Each file gets ~61/N_FILES RGs with narrow, non-overlapping ranges -# but in scrambled order. -# -# Writing a single file with ORDER BY scramble does NOT work: the parquet -# writer merges rows from adjacent chunks at RG boundaries, widening -# ranges and defeating reorder_by_statistics. -# -# Requires pyarrow (pip install pyarrow). -data_sort_pushdown_inexact() { - INEXACT_DIR="${DATA_DIR}/sort_pushdown_inexact/lineitem" - if [ -d "${INEXACT_DIR}" ] && [ "$(ls -A ${INEXACT_DIR}/*.parquet 2>/dev/null)" ]; then - echo "Sort pushdown Inexact data already exists at ${INEXACT_DIR}" - return - fi - - # Check pyarrow dependency (needed to split/scramble RGs) - if ! python3 -c "import pyarrow" 2>/dev/null; then - echo "Error: pyarrow is required for sort pushdown Inexact data generation." - echo "Install with: pip install pyarrow" - return 1 - fi - - echo "Generating sort pushdown Inexact benchmark data (multi-file, scrambled RGs)..." - - # Re-use the sort_pushdown data as the source (generate if missing) - data_sort_pushdown - - mkdir -p "${INEXACT_DIR}" - SRC_DIR="${DATA_DIR}/sort_pushdown/lineitem" - - # Step 1: Write a single sorted file with small (100K-row) RGs - TMPFILE="${INEXACT_DIR}/_sorted_small_rgs.parquet" - (cd "${SCRIPT_DIR}/.." && cargo run --release -p datafusion-cli -- -c " - CREATE EXTERNAL TABLE src - STORED AS PARQUET - LOCATION '${SRC_DIR}'; - - COPY (SELECT * FROM src ORDER BY l_orderkey) - TO '${TMPFILE}' - STORED AS PARQUET - OPTIONS ('format.max_row_group_size' '100000'); - ") - - # Step 2: Redistribute RGs into 3 files with scrambled RG order. - # Each file gets ~20 RGs. RG assignment: rg_idx % 3 determines file, - # permutation (rg_idx * 41 + 7) % n scrambles the order within file. - python3 -c " -import pyarrow.parquet as pq - -pf = pq.ParquetFile('${TMPFILE}') -n = pf.metadata.num_row_groups -n_files = 3 - -# Assign each RG to a file, scramble order within each file -file_rgs = [[] for _ in range(n_files)] -for rg_idx in range(n): - slot = (rg_idx * 41 + 7) % n # scrambled index - file_id = slot % n_files - file_rgs[file_id].append(rg_idx) - -# Write each file with its assigned RGs (in scrambled order) -for file_id in range(n_files): - rgs = file_rgs[file_id] - if not rgs: - continue - tables = [pf.read_row_group(rg) for rg in rgs] - writer = pq.ParquetWriter( - '${INEXACT_DIR}/part_%03d.parquet' % file_id, - pf.schema_arrow) - for t in tables: - writer.write_table(t) - writer.close() - print(f'File part_{file_id:03d}.parquet: {len(rgs)} RGs') -" - - rm -f "${TMPFILE}" - echo "Sort pushdown Inexact data generated at ${INEXACT_DIR}" - ls -la "${INEXACT_DIR}" - - # Also generate overlap data: same strategy but with different file count - # and permutation. Simulates streaming data with network delays where - # chunks arrive out of sequence. - # - # Requires pyarrow (pip install pyarrow). - OVERLAP_DIR="${DATA_DIR}/sort_pushdown_inexact_overlap/lineitem" - if [ -d "${OVERLAP_DIR}" ] && [ "$(ls -A ${OVERLAP_DIR}/*.parquet 2>/dev/null)" ]; then - echo "Sort pushdown Inexact overlap data already exists at ${OVERLAP_DIR}" - return - fi - - echo "Generating sort pushdown Inexact overlap data (multi-file, scrambled RGs)..." - mkdir -p "${OVERLAP_DIR}" - - # Step 1: Write a single sorted file with small (100K-row) RGs - TMPFILE="${OVERLAP_DIR}/_sorted_small_rgs.parquet" - (cd "${SCRIPT_DIR}/.." && cargo run --release -p datafusion-cli -- -c " - CREATE EXTERNAL TABLE src - STORED AS PARQUET - LOCATION '${SRC_DIR}'; - - COPY (SELECT * FROM src ORDER BY l_orderkey) - TO '${TMPFILE}' - STORED AS PARQUET - OPTIONS ('format.max_row_group_size' '100000'); - ") - - # Step 2: Redistribute into 5 files with scrambled RG order. - python3 -c " -import pyarrow.parquet as pq - -pf = pq.ParquetFile('${TMPFILE}') -n = pf.metadata.num_row_groups -n_files = 5 - -file_rgs = [[] for _ in range(n_files)] -for rg_idx in range(n): - slot = (rg_idx * 37 + 13) % n - file_id = slot % n_files - file_rgs[file_id].append(rg_idx) - -for file_id in range(n_files): - rgs = file_rgs[file_id] - if not rgs: - continue - tables = [pf.read_row_group(rg) for rg in rgs] - writer = pq.ParquetWriter( - '${OVERLAP_DIR}/part_%03d.parquet' % file_id, - pf.schema_arrow) - for t in tables: - writer.write_table(t) - writer.close() - print(f'File part_{file_id:03d}.parquet: {len(rgs)} RGs') -" - - rm -f "${TMPFILE}" -} - -# Runs the sort pushdown Inexact benchmark (tests RG reorder by statistics). -# Enables pushdown_filters so TopK's dynamic filter is pushed to the parquet -# reader for late materialization (only needed for Inexact path). -run_sort_pushdown_inexact() { - INEXACT_DIR="${DATA_DIR}/sort_pushdown_inexact" - RESULTS_FILE="${RESULTS_DIR}/sort_pushdown_inexact.json" - echo "Running sort pushdown Inexact benchmark (multi-file scrambled RGs, --sorted DESC)..." - DATAFUSION_EXECUTION_PARQUET_PUSHDOWN_FILTERS=true \ - debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --sorted --iterations 5 --path "${INEXACT_DIR}" --queries-path "${SCRIPT_DIR}/queries/sort_pushdown_inexact" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} -} - -# Runs the sort pushdown Inexact benchmark WITHOUT declared ordering. -# Tests the Unsupported path in try_pushdown_sort where RG reorder by -# statistics can still help TopK queries without any file ordering guarantee. -run_sort_pushdown_inexact_unsorted() { - INEXACT_DIR="${DATA_DIR}/sort_pushdown_inexact" - RESULTS_FILE="${RESULTS_DIR}/sort_pushdown_inexact_unsorted.json" - echo "Running sort pushdown Inexact benchmark (no WITH ORDER, Unsupported path)..." - DATAFUSION_EXECUTION_PARQUET_PUSHDOWN_FILTERS=true \ - debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --iterations 5 --path "${INEXACT_DIR}" --queries-path "${SCRIPT_DIR}/queries/sort_pushdown_inexact_unsorted" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} -} - -# Runs the sort pushdown benchmark with multi-file scrambled RG order. -# Simulates streaming data with network delays — multiple files, each with -# scrambled RGs. Tests both RG-level reorder and TopK stats initialization. -run_sort_pushdown_inexact_overlap() { - OVERLAP_DIR="${DATA_DIR}/sort_pushdown_inexact_overlap" - RESULTS_FILE="${RESULTS_DIR}/sort_pushdown_inexact_overlap.json" - echo "Running sort pushdown Inexact benchmark (multi-file scrambled RGs, streaming data pattern)..." - DATAFUSION_EXECUTION_PARQUET_PUSHDOWN_FILTERS=true \ - debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --sorted --iterations 5 --path "${OVERLAP_DIR}" --queries-path "${SCRIPT_DIR}/queries/sort_pushdown_inexact_overlap" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} -} - # Runs the sort integration benchmark run_sort_tpch() { SCALE_FACTOR=$1 diff --git a/benchmarks/queries/sort_pushdown_inexact/q1.sql b/benchmarks/queries/sort_pushdown_inexact/q1.sql deleted file mode 100644 index d772bc486a12b..0000000000000 --- a/benchmarks/queries/sort_pushdown_inexact/q1.sql +++ /dev/null @@ -1,8 +0,0 @@ --- Inexact path: TopK + DESC LIMIT on ASC-declared file. --- With RG reorder, the first RG read contains the highest max value, --- so TopK's threshold tightens quickly and subsequent RGs get filtered --- efficiently via dynamic filter pushdown. -SELECT l_orderkey, l_partkey, l_suppkey -FROM lineitem -ORDER BY l_orderkey DESC -LIMIT 100 diff --git a/benchmarks/queries/sort_pushdown_inexact/q2.sql b/benchmarks/queries/sort_pushdown_inexact/q2.sql deleted file mode 100644 index 6e2bef44fc37e..0000000000000 --- a/benchmarks/queries/sort_pushdown_inexact/q2.sql +++ /dev/null @@ -1,7 +0,0 @@ --- Inexact path: TopK + DESC LIMIT with larger fetch (1000). --- Larger LIMIT means more row_replacements; RG reorder reduces the --- total replacement count by tightening the threshold faster. -SELECT l_orderkey, l_partkey, l_suppkey -FROM lineitem -ORDER BY l_orderkey DESC -LIMIT 1000 diff --git a/benchmarks/queries/sort_pushdown_inexact/q3.sql b/benchmarks/queries/sort_pushdown_inexact/q3.sql deleted file mode 100644 index d858ec79a67c9..0000000000000 --- a/benchmarks/queries/sort_pushdown_inexact/q3.sql +++ /dev/null @@ -1,8 +0,0 @@ --- Inexact path: wide projection (all columns) + DESC LIMIT. --- Shows the row-level filter benefit: with a tight threshold from the --- first RG, subsequent RGs skip decoding non-sort columns for filtered --- rows — bigger wins for wide tables. -SELECT * -FROM lineitem -ORDER BY l_orderkey DESC -LIMIT 100 diff --git a/benchmarks/queries/sort_pushdown_inexact/q4.sql b/benchmarks/queries/sort_pushdown_inexact/q4.sql deleted file mode 100644 index bd2efc5d3b992..0000000000000 --- a/benchmarks/queries/sort_pushdown_inexact/q4.sql +++ /dev/null @@ -1,7 +0,0 @@ --- Inexact path: wide projection + DESC LIMIT with larger fetch. --- Combines wide-row row-level filter benefit with larger LIMIT to --- demonstrate cumulative gains from RG reorder. -SELECT * -FROM lineitem -ORDER BY l_orderkey DESC -LIMIT 1000 diff --git a/benchmarks/src/sort_pushdown.rs b/benchmarks/src/sort_pushdown.rs index e2a4615a3ef39..e7fce1921e7a8 100644 --- a/benchmarks/src/sort_pushdown.rs +++ b/benchmarks/src/sort_pushdown.rs @@ -159,14 +159,7 @@ impl RunOpt { async fn benchmark_query(&self, query_id: usize) -> Result> { let sql = self.load_query(query_id)?; - let mut config = self.common.config()?; - // Enable parquet filter pushdown + late materialization. This is - // essential for the Inexact sort pushdown path: TopK's dynamic - // filter is pushed to the parquet reader, so only sort-column - // rows pass the filter's Decode non-sort columns are skipped for - // rows that don't pass the filter — this is where RG reorder's - // tight-threshold-first strategy pays off for wide-row queries. - config.options_mut().execution.parquet.pushdown_filters = true; + let config = self.common.config()?; let rt = self.common.build_runtime()?; let state = SessionStateBuilder::new() .with_config(config) From 6ee488e0474217de770b99bcca40b305a9c9f3f5 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Fri, 17 Apr 2026 14:10:54 +0000 Subject: [PATCH 09/38] fix: resolve doc link for AccessPlanOptimizer --- datafusion/datasource-parquet/src/access_plan.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/datasource-parquet/src/access_plan.rs b/datafusion/datasource-parquet/src/access_plan.rs index 32085fa8a177a..f8bcb5b74c19e 100644 --- a/datafusion/datasource-parquet/src/access_plan.rs +++ b/datafusion/datasource-parquet/src/access_plan.rs @@ -355,7 +355,7 @@ impl ParquetAccessPlan { } /// Like [`prepare`](Self::prepare), but also applies an - /// [`AccessPlanOptimizer`] to reorder/reverse row groups after + /// `AccessPlanOptimizer` to reorder/reverse row groups after /// preparing the plan. pub(crate) fn prepare_with_optimizer( self, From 832a541d06e0b54e8cd824535cc056f7b279cf2c Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Fri, 17 Apr 2026 14:32:50 +0000 Subject: [PATCH 10/38] fix: restore benchmark files from upstream main --- benchmarks/bench.sh | 130 ++++++++++++++++++ .../queries/sort_pushdown_inexact/q1.sql | 8 ++ .../queries/sort_pushdown_inexact/q2.sql | 7 + .../queries/sort_pushdown_inexact/q3.sql | 8 ++ .../queries/sort_pushdown_inexact/q4.sql | 7 + 5 files changed, 160 insertions(+) create mode 100644 benchmarks/queries/sort_pushdown_inexact/q1.sql create mode 100644 benchmarks/queries/sort_pushdown_inexact/q2.sql create mode 100644 benchmarks/queries/sort_pushdown_inexact/q3.sql create mode 100644 benchmarks/queries/sort_pushdown_inexact/q4.sql diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index 9dce4cf77b933..aa1ec477345c6 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -109,6 +109,9 @@ clickbench_extended: ClickBench \"inspired\" queries against a single parquet # Sort Pushdown Benchmarks sort_pushdown: Sort pushdown baseline (no WITH ORDER) on TPC-H data (SF=1) sort_pushdown_sorted: Sort pushdown with WITH ORDER — tests sort elimination on non-overlapping files +sort_pushdown_inexact: Sort pushdown Inexact path (--sorted DESC) — tests reverse scan + RG reorder +sort_pushdown_inexact_unsorted: Sort pushdown Inexact path (no WITH ORDER) — tests Unsupported path + RG reorder +sort_pushdown_inexact_overlap: Sort pushdown Inexact path — partially overlapping RGs (streaming data scenario) # Sorted Data Benchmarks (ORDER BY Optimization) clickbench_sorted: ClickBench queries on pre-sorted data using prefer_existing_sort (tests sort elimination optimization) @@ -316,6 +319,9 @@ main() { sort_pushdown|sort_pushdown_sorted) data_sort_pushdown ;; + sort_pushdown_inexact|sort_pushdown_inexact_unsorted|sort_pushdown_inexact_overlap) + data_sort_pushdown_inexact + ;; sort_tpch) # same data as for tpch data_tpch "1" "parquet" @@ -522,6 +528,15 @@ main() { sort_pushdown_sorted) run_sort_pushdown_sorted ;; + sort_pushdown_inexact) + run_sort_pushdown_inexact + ;; + sort_pushdown_inexact_unsorted) + run_sort_pushdown_inexact_unsorted + ;; + sort_pushdown_inexact_overlap) + run_sort_pushdown_inexact_overlap + ;; sort_tpch) run_sort_tpch "1" ;; @@ -1137,6 +1152,121 @@ run_sort_pushdown_sorted() { debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --sorted --iterations 5 --path "${SORT_PUSHDOWN_DIR}" --queries-path "${SCRIPT_DIR}/queries/sort_pushdown" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} } +# Generates data for sort pushdown Inexact benchmark. +# +# Produces a single large lineitem parquet file where row groups have +# NON-OVERLAPPING but OUT-OF-ORDER l_orderkey ranges (each RG internally +# sorted, RGs shuffled). This simulates append-heavy workloads where data +# is written in batches at different times. +data_sort_pushdown_inexact() { + INEXACT_DIR="${DATA_DIR}/sort_pushdown_inexact/lineitem" + if [ -d "${INEXACT_DIR}" ] && [ "$(ls -A ${INEXACT_DIR}/*.parquet 2>/dev/null)" ]; then + echo "Sort pushdown Inexact data already exists at ${INEXACT_DIR}" + return + fi + + echo "Generating sort pushdown Inexact benchmark data (single file, shuffled RGs)..." + + # Re-use the sort_pushdown data as the source (generate if missing) + data_sort_pushdown + + mkdir -p "${INEXACT_DIR}" + SRC_DIR="${DATA_DIR}/sort_pushdown/lineitem" + + # Use datafusion-cli to bucket rows into 64 groups by a deterministic + # scrambler, then sort within each bucket by orderkey. This produces + # ~64 RG-sized segments where each has a tight orderkey range but the + # segments appear in scrambled (non-sorted) order in the file. + (cd "${SCRIPT_DIR}/.." && cargo run --release -p datafusion-cli -- -c " + CREATE EXTERNAL TABLE src + STORED AS PARQUET + LOCATION '${SRC_DIR}'; + + COPY ( + SELECT * FROM src + ORDER BY + (l_orderkey * 1664525 + 1013904223) % 64, + l_orderkey + ) + TO '${INEXACT_DIR}/shuffled.parquet' + STORED AS PARQUET + OPTIONS ('format.max_row_group_size' '100000'); + ") + + echo "Sort pushdown Inexact shuffled data generated at ${INEXACT_DIR}" + ls -la "${INEXACT_DIR}" + + # Also generate a file with partially overlapping row groups. + # Simulates streaming data with network delays: each chunk is mostly + # in order but has a small overlap with the next chunk (±5% of the + # chunk range). This is the pattern described by @adriangb — data + # arriving with timestamps that are generally increasing but with + # network-induced jitter causing small overlaps between row groups. + OVERLAP_DIR="${DATA_DIR}/sort_pushdown_inexact_overlap/lineitem" + if [ -d "${OVERLAP_DIR}" ] && [ "$(ls -A ${OVERLAP_DIR}/*.parquet 2>/dev/null)" ]; then + echo "Sort pushdown Inexact overlap data already exists at ${OVERLAP_DIR}" + return + fi + + echo "Generating sort pushdown Inexact overlap data (partially overlapping RGs)..." + mkdir -p "${OVERLAP_DIR}" + + (cd "${SCRIPT_DIR}/.." && cargo run --release -p datafusion-cli -- -c " + CREATE EXTERNAL TABLE src + STORED AS PARQUET + LOCATION '${SRC_DIR}'; + + -- Add jitter to l_orderkey: shift each row by a random-ish offset + -- proportional to its position. This creates overlap between adjacent + -- row groups while preserving the general ascending trend. + -- Formula: l_orderkey + (l_orderkey * 7 % 5000) - 2500 + -- This adds ±2500 jitter, creating ~5K overlap between adjacent 100K-row RGs. + COPY ( + SELECT * FROM src + ORDER BY l_orderkey + (l_orderkey * 7 % 5000) - 2500 + ) + TO '${OVERLAP_DIR}/overlapping.parquet' + STORED AS PARQUET + OPTIONS ('format.max_row_group_size' '100000'); + ") + + echo "Sort pushdown Inexact overlap data generated at ${OVERLAP_DIR}" + ls -la "${OVERLAP_DIR}" +} + +# Runs the sort pushdown Inexact benchmark (tests RG reorder by statistics). +# Enables pushdown_filters so TopK's dynamic filter is pushed to the parquet +# reader for late materialization (only needed for Inexact path). +run_sort_pushdown_inexact() { + INEXACT_DIR="${DATA_DIR}/sort_pushdown_inexact" + RESULTS_FILE="${RESULTS_DIR}/sort_pushdown_inexact.json" + echo "Running sort pushdown Inexact benchmark (--sorted, DESC, reverse scan path)..." + DATAFUSION_EXECUTION_PARQUET_PUSHDOWN_FILTERS=true \ + debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --sorted --iterations 5 --path "${INEXACT_DIR}" --queries-path "${SCRIPT_DIR}/queries/sort_pushdown_inexact" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} +} + +# Runs the sort pushdown Inexact benchmark WITHOUT declared ordering. +# Tests the Unsupported path in try_pushdown_sort where RG reorder by +# statistics can still help TopK queries without any file ordering guarantee. +run_sort_pushdown_inexact_unsorted() { + INEXACT_DIR="${DATA_DIR}/sort_pushdown_inexact" + RESULTS_FILE="${RESULTS_DIR}/sort_pushdown_inexact_unsorted.json" + echo "Running sort pushdown Inexact benchmark (no WITH ORDER, Unsupported path)..." + DATAFUSION_EXECUTION_PARQUET_PUSHDOWN_FILTERS=true \ + debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --iterations 5 --path "${INEXACT_DIR}" --queries-path "${SCRIPT_DIR}/queries/sort_pushdown_inexact_unsorted" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} +} + +# Runs the sort pushdown benchmark with partially overlapping RGs. +# Simulates streaming data with network jitter — RGs are mostly in order +# but have small overlaps (±2500 orderkey jitter between adjacent RGs). +run_sort_pushdown_inexact_overlap() { + OVERLAP_DIR="${DATA_DIR}/sort_pushdown_inexact_overlap" + RESULTS_FILE="${RESULTS_DIR}/sort_pushdown_inexact_overlap.json" + echo "Running sort pushdown Inexact benchmark (overlapping RGs, streaming data pattern)..." + DATAFUSION_EXECUTION_PARQUET_PUSHDOWN_FILTERS=true \ + debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --sorted --iterations 5 --path "${OVERLAP_DIR}" --queries-path "${SCRIPT_DIR}/queries/sort_pushdown_inexact_overlap" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} +} + # Runs the sort integration benchmark run_sort_tpch() { SCALE_FACTOR=$1 diff --git a/benchmarks/queries/sort_pushdown_inexact/q1.sql b/benchmarks/queries/sort_pushdown_inexact/q1.sql new file mode 100644 index 0000000000000..d772bc486a12b --- /dev/null +++ b/benchmarks/queries/sort_pushdown_inexact/q1.sql @@ -0,0 +1,8 @@ +-- Inexact path: TopK + DESC LIMIT on ASC-declared file. +-- With RG reorder, the first RG read contains the highest max value, +-- so TopK's threshold tightens quickly and subsequent RGs get filtered +-- efficiently via dynamic filter pushdown. +SELECT l_orderkey, l_partkey, l_suppkey +FROM lineitem +ORDER BY l_orderkey DESC +LIMIT 100 diff --git a/benchmarks/queries/sort_pushdown_inexact/q2.sql b/benchmarks/queries/sort_pushdown_inexact/q2.sql new file mode 100644 index 0000000000000..6e2bef44fc37e --- /dev/null +++ b/benchmarks/queries/sort_pushdown_inexact/q2.sql @@ -0,0 +1,7 @@ +-- Inexact path: TopK + DESC LIMIT with larger fetch (1000). +-- Larger LIMIT means more row_replacements; RG reorder reduces the +-- total replacement count by tightening the threshold faster. +SELECT l_orderkey, l_partkey, l_suppkey +FROM lineitem +ORDER BY l_orderkey DESC +LIMIT 1000 diff --git a/benchmarks/queries/sort_pushdown_inexact/q3.sql b/benchmarks/queries/sort_pushdown_inexact/q3.sql new file mode 100644 index 0000000000000..d858ec79a67c9 --- /dev/null +++ b/benchmarks/queries/sort_pushdown_inexact/q3.sql @@ -0,0 +1,8 @@ +-- Inexact path: wide projection (all columns) + DESC LIMIT. +-- Shows the row-level filter benefit: with a tight threshold from the +-- first RG, subsequent RGs skip decoding non-sort columns for filtered +-- rows — bigger wins for wide tables. +SELECT * +FROM lineitem +ORDER BY l_orderkey DESC +LIMIT 100 diff --git a/benchmarks/queries/sort_pushdown_inexact/q4.sql b/benchmarks/queries/sort_pushdown_inexact/q4.sql new file mode 100644 index 0000000000000..bd2efc5d3b992 --- /dev/null +++ b/benchmarks/queries/sort_pushdown_inexact/q4.sql @@ -0,0 +1,7 @@ +-- Inexact path: wide projection + DESC LIMIT with larger fetch. +-- Combines wide-row row-level filter benefit with larger LIMIT to +-- demonstrate cumulative gains from RG reorder. +SELECT * +FROM lineitem +ORDER BY l_orderkey DESC +LIMIT 1000 From bdd86a04a6f163fb8784d46553ce20d5a80f5eee Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Sat, 18 Apr 2026 13:00:19 +0800 Subject: [PATCH 11/38] fix: compose reorder and reverse as sequential steps instead of mutually exclusive Previously reorder_by_statistics and reverse_row_groups were mutually exclusive (else-if). This meant DESC queries on unsorted data could only get one optimization. Now they compose: reorder always sorts RGs by min ASC, then reverse flips for DESC. This ensures correct results for both sorted and unsorted inputs without regression. Also removes prepare_with_optimizer in favor of calling optimize() directly on each optimizer, and simplifies reorder_by_statistics to always use min ASC (direction handled by reverse). --- .../datasource-parquet/src/access_plan.rs | 87 +++++++------------ datafusion/datasource-parquet/src/opener.rs | 60 ++++++++----- 2 files changed, 71 insertions(+), 76 deletions(-) diff --git a/datafusion/datasource-parquet/src/access_plan.rs b/datafusion/datasource-parquet/src/access_plan.rs index f8bcb5b74c19e..2c877ba091294 100644 --- a/datafusion/datasource-parquet/src/access_plan.rs +++ b/datafusion/datasource-parquet/src/access_plan.rs @@ -353,20 +353,6 @@ impl ParquetAccessPlan { let row_selection = self.into_overall_row_selection(row_group_meta_data)?; PreparedAccessPlan::new(row_group_indexes, row_selection) } - - /// Like [`prepare`](Self::prepare), but also applies an - /// `AccessPlanOptimizer` to reorder/reverse row groups after - /// preparing the plan. - pub(crate) fn prepare_with_optimizer( - self, - row_group_meta_data: &[RowGroupMetaData], - file_metadata: &ParquetMetaData, - arrow_schema: &Schema, - optimizer: &dyn crate::access_plan_optimizer::AccessPlanOptimizer, - ) -> Result { - let plan = self.prepare(row_group_meta_data)?; - optimizer.optimize(plan, file_metadata, arrow_schema) - } } /// Represents a prepared, fully resolved [`ParquetAccessPlan`] @@ -436,8 +422,6 @@ impl PreparedAccessPlan { } }; - let descending = first_sort_expr.options.descending; - // Build statistics converter for this column let converter = match StatisticsConverter::try_new( column.name(), @@ -451,40 +435,31 @@ impl PreparedAccessPlan { } }; - // Get the relevant statistics for the selected row groups. - // For ASC sort: use min values — we want the RG with the smallest min - // to come first (best candidate for "smallest values"). - // For DESC sort: use max values — we want the RG with the largest max - // to come first (best candidate for "largest values"). Using min for - // DESC can pick a worse first RG when ranges overlap (e.g., RG0 50-60 - // vs RG1 40-100 — RG1 has larger values but smaller min). + // Always sort by min values in ASC order to align row groups with + // the file's declared output ordering. Direction (DESC) is handled + // separately by ReverseRowGroups which is applied AFTER reorder. + // + // This composable design avoids the problem where reorder(DESC) + // followed by reverse would double-flip the order, and ensures + // that for already-sorted data, reorder is a no-op and reverse + // gives the correct DESC order (including placing small tail RGs first). let rg_metadata: Vec<&RowGroupMetaData> = self .row_group_indexes .iter() .map(|&idx| file_metadata.row_group(idx)) .collect(); - let stat_values = if descending { - match converter.row_group_maxes(rg_metadata.iter().copied()) { - Ok(vals) => vals, - Err(e) => { - debug!("Skipping RG reorder: cannot get max values: {e}"); - return Ok(self); - } - } - } else { - match converter.row_group_mins(rg_metadata.iter().copied()) { - Ok(vals) => vals, - Err(e) => { - debug!("Skipping RG reorder: cannot get min values: {e}"); - return Ok(self); - } + let stat_values = match converter.row_group_mins(rg_metadata.iter().copied()) { + Ok(vals) => vals, + Err(e) => { + debug!("Skipping RG reorder: cannot get min values: {e}"); + return Ok(self); } }; - // Sort indices by statistic values (min for ASC, max for DESC) + // Always sort ASC by min values — direction is handled by reverse let sort_options = arrow::compute::SortOptions { - descending, + descending: false, nulls_first: first_sort_expr.options.nulls_first, }; let sorted_indices = @@ -836,7 +811,11 @@ mod test { } #[test] - fn test_reorder_by_statistics_desc() { + fn test_reorder_by_statistics_desc_sorts_asc() { + // reorder_by_statistics always sorts by min ASC regardless of sort + // direction. DESC is handled separately by ReverseRowGroups which + // is applied after reorder in the optimizer pipeline. + // // RGs: [50-99, 200-299, 1-30] let metadata = make_metadata_with_stats(&[(50, 99), (200, 299), (1, 30)]); let schema = make_arrow_schema(); @@ -847,8 +826,9 @@ mod test { .reorder_by_statistics(&sort_order, &metadata, &schema) .unwrap(); - // DESC: largest max first: RG1(max=299), RG0(max=99), RG2(max=30) - assert_eq!(plan.row_group_indexes, vec![1, 0, 2]); + // Always ASC by min: RG2(min=1), RG0(min=50), RG1(min=200) + // Reverse is applied separately for DESC queries. + assert_eq!(plan.row_group_indexes, vec![2, 0, 1]); } #[test] @@ -949,17 +929,14 @@ mod test { } #[test] - fn test_reorder_by_statistics_desc_uses_max_for_overlapping_rgs() { - // Overlapping ranges where min DESC would pick worse RG than max DESC: - // RG0: 50-60 (small range, moderate max) - // RG1: 40-100 (wide range, high max but lower min) - // RG2: 20-30 (low max) - // - // For ORDER BY DESC LIMIT N: - // Using min DESC: [RG0(50), RG1(40), RG2(20)] → reads RG0 first (max=60 only) - // Using max DESC: [RG1(100), RG0(60), RG2(30)] → reads RG1 first (max=100) + fn test_reorder_by_statistics_overlapping_rgs_sorts_asc() { + // Overlapping ranges — reorder always uses min ASC: + // RG0: 50-60 + // RG1: 40-100 (lower min, wider range) + // RG2: 20-30 (lowest min) // - // RG1 is the better first choice for DESC because it contains the largest values. + // Sorted by min ASC: [RG2(20), RG1(40), RG0(50)] + // For DESC queries, ReverseRowGroups is applied after to flip order. let metadata = make_metadata_with_stats(&[(50, 60), (40, 100), (20, 30)]); let schema = make_arrow_schema(); let sort_order = make_sort_order_desc(); @@ -969,8 +946,8 @@ mod test { .reorder_by_statistics(&sort_order, &metadata, &schema) .unwrap(); - // Expected: RG1 (max=100) first, then RG0 (max=60), then RG2 (max=30) - assert_eq!(plan.row_group_indexes, vec![1, 0, 2]); + // Always ASC by min: RG2(min=20), RG1(min=40), RG0(min=50) + assert_eq!(plan.row_group_indexes, vec![2, 1, 0]); } #[test] diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 41dacd04dc3e8..2bcffbab24212 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -1130,35 +1130,53 @@ impl RowGroupsPrunedParquetOpen { ); } - // Build the access plan optimizer from sort pushdown hints. - // ReorderByStatistics is preferred (handles both ASC and DESC via - // min/max stats). ReverseRowGroups is a fallback when no statistics - // are available on the sort column. - let optimizer: Option< + // Row group ordering optimization (two composable steps): + // + // 1. reorder_by_statistics: sort RGs by min values (ASC) to align + // with the file's declared output ordering. This fixes out-of-order + // RGs (e.g., from append-heavy workloads) without changing direction. + // Skipped gracefully when statistics are unavailable. + // + // 2. reverse: flip the order for DESC queries. Applied AFTER reorder + // so the reversed order is correct whether or not reorder changed + // anything. Also handles row_selection remapping. + // + // For sorted data: reorder is a no-op, reverse gives perfect DESC. + // For unsorted data: reorder fixes the order, reverse flips for DESC. + let reorder_optimizer: Option< Box, - > = if let Some(sort_order) = &prepared.sort_order_for_reorder { - Some(Box::new( - crate::access_plan_optimizer::ReorderByStatistics::new( - sort_order.clone(), - ), - )) - } else if prepared.reverse_row_groups { + > = prepared.sort_order_for_reorder.as_ref().map(|sort_order| { + Box::new(crate::access_plan_optimizer::ReorderByStatistics::new( + sort_order.clone(), + )) as Box + }); + + let reverse_optimizer: Option< + Box, + > = if prepared.reverse_row_groups { Some(Box::new(crate::access_plan_optimizer::ReverseRowGroups)) } else { None }; - // Prepare the access plan and apply row group optimizer if configured. - let prepared_plan = if let Some(opt) = &optimizer { - access_plan.prepare_with_optimizer( - rg_metadata, + // Prepare the access plan and apply optimizers in order: + // 1. reorder (fix out-of-order RGs to match declared ordering) + // 2. reverse (flip for DESC queries) + let mut prepared_plan = access_plan.prepare(rg_metadata)?; + if let Some(opt) = &reorder_optimizer { + prepared_plan = opt.optimize( + prepared_plan, file_metadata.as_ref(), &prepared.physical_file_schema, - opt.as_ref(), - )? - } else { - access_plan.prepare(rg_metadata)? - }; + )?; + } + if let Some(opt) = &reverse_optimizer { + prepared_plan = opt.optimize( + prepared_plan, + file_metadata.as_ref(), + &prepared.physical_file_schema, + )?; + } let arrow_reader_metrics = ArrowReaderMetrics::enabled(); let read_plan = build_projection_read_plan( From c3dac5e4fdcf30092ca90758128b3e806ff529ef Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Sat, 18 Apr 2026 13:05:02 +0800 Subject: [PATCH 12/38] fix: generate scrambled+overlapping RGs for overlap benchmark MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The previous jitter formula only added overlap between adjacent RGs but kept the overall RG order ascending by min values. This meant reorder_by_statistics was a no-op — there was nothing to reorder. Fix by bucketing rows into 60 chunks, sorting within each chunk (with jitter for overlap), then scrambling chunk order using a deterministic permutation. This produces RGs that are individually sorted but appear in scrambled order in the file, so reorder_by_statistics has real work to do. --- benchmarks/bench.sh | 41 +++++++++++++++++++++++++---------------- 1 file changed, 25 insertions(+), 16 deletions(-) diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index aa1ec477345c6..49065906d4063 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -1196,19 +1196,26 @@ data_sort_pushdown_inexact() { echo "Sort pushdown Inexact shuffled data generated at ${INEXACT_DIR}" ls -la "${INEXACT_DIR}" - # Also generate a file with partially overlapping row groups. - # Simulates streaming data with network delays: each chunk is mostly - # in order but has a small overlap with the next chunk (±5% of the - # chunk range). This is the pattern described by @adriangb — data - # arriving with timestamps that are generally increasing but with - # network-induced jitter causing small overlaps between row groups. + # Also generate a file with partially overlapping AND out-of-order + # row groups. Simulates streaming data with network delays: chunks + # arrive out of sequence, and each chunk has small jitter causing + # overlap with neighbors. This is the pattern described by + # @adriangb — data arriving with timestamps that are generally + # increasing but network-induced delays cause chunks to arrive + # out of order with small overlaps between row groups. + # + # Strategy: bucket rows into 60 chunks (~100K rows each), sort + # within each chunk (with jitter for overlap), then scramble the + # chunk order using a deterministic permutation. This produces + # RGs that are individually sorted but appear in scrambled order + # in the file — so reorder_by_statistics has real work to do. OVERLAP_DIR="${DATA_DIR}/sort_pushdown_inexact_overlap/lineitem" if [ -d "${OVERLAP_DIR}" ] && [ "$(ls -A ${OVERLAP_DIR}/*.parquet 2>/dev/null)" ]; then echo "Sort pushdown Inexact overlap data already exists at ${OVERLAP_DIR}" return fi - echo "Generating sort pushdown Inexact overlap data (partially overlapping RGs)..." + echo "Generating sort pushdown Inexact overlap data (scrambled + overlapping RGs)..." mkdir -p "${OVERLAP_DIR}" (cd "${SCRIPT_DIR}/.." && cargo run --release -p datafusion-cli -- -c " @@ -1216,14 +1223,16 @@ data_sort_pushdown_inexact() { STORED AS PARQUET LOCATION '${SRC_DIR}'; - -- Add jitter to l_orderkey: shift each row by a random-ish offset - -- proportional to its position. This creates overlap between adjacent - -- row groups while preserving the general ascending trend. - -- Formula: l_orderkey + (l_orderkey * 7 % 5000) - 2500 - -- This adds ±2500 jitter, creating ~5K overlap between adjacent 100K-row RGs. + -- Bucket into 60 chunks (each ~100K rows), sort within each chunk, + -- then scramble chunk order. This produces overlapping RGs that are + -- individually sorted but appear in scrambled order in the file. COPY ( SELECT * FROM src - ORDER BY l_orderkey + (l_orderkey * 7 % 5000) - 2500 + ORDER BY + -- Scramble chunk order: chunk_id -> permuted_chunk_id + (CAST(l_orderkey / 100000 AS INT) * 37 + 13) % 60, + -- Within each chunk, add small jitter for overlap + l_orderkey + (l_orderkey * 7 % 5000) - 2500 ) TO '${OVERLAP_DIR}/overlapping.parquet' STORED AS PARQUET @@ -1256,9 +1265,9 @@ run_sort_pushdown_inexact_unsorted() { debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --iterations 5 --path "${INEXACT_DIR}" --queries-path "${SCRIPT_DIR}/queries/sort_pushdown_inexact_unsorted" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} } -# Runs the sort pushdown benchmark with partially overlapping RGs. -# Simulates streaming data with network jitter — RGs are mostly in order -# but have small overlaps (±2500 orderkey jitter between adjacent RGs). +# Runs the sort pushdown benchmark with scrambled + overlapping RGs. +# Simulates streaming data with network delays — RGs are out of order +# AND have small overlaps (jitter). Tests reorder_by_statistics effectiveness. run_sort_pushdown_inexact_overlap() { OVERLAP_DIR="${DATA_DIR}/sort_pushdown_inexact_overlap" RESULTS_FILE="${RESULTS_DIR}/sort_pushdown_inexact_overlap.json" From f340c65e907daa0688dda4b3c74c82cf3a960aa9 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Mon, 20 Apr 2026 22:35:38 +0800 Subject: [PATCH 13/38] feat: reorder files in shared work queue by statistics for TopK Add FileSource::reorder_files() trait method and ParquetSource implementation that sorts files by column statistics before placing them in the shared work queue. For DESC queries, files with the highest min value come first; for ASC, lowest max first. This ensures the first file read by any partition is the globally optimal one for TopK threshold convergence, complementing the intra-file RG reorder. --- datafusion/datasource-parquet/src/source.rs | 79 +++++++++++++++++++ datafusion/datasource/src/file.rs | 13 +++ .../datasource/src/file_stream/work_source.rs | 12 ++- 3 files changed, 103 insertions(+), 1 deletion(-) diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 5586e29a9eae5..bfd3ea37b0ca3 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -485,6 +485,24 @@ impl ParquetSource { pub(crate) fn reverse_row_groups(&self) -> bool { self.reverse_row_groups } + + /// Extract the sort key from a file's statistics for reordering. + /// + /// For DESC sorts, returns the column's min_value (we want highest min first). + /// For ASC sorts, returns the column's max_value (we want lowest max first). + fn sort_key_for_file( + file: &datafusion_datasource::PartitionedFile, + col_idx: usize, + descending: bool, + ) -> Option { + let stats = file.statistics.as_ref()?; + let col_stats = stats.column_statistics.get(col_idx)?; + if descending { + col_stats.min_value.get_value().cloned() + } else { + col_stats.max_value.get_value().cloned() + } + } } /// Parses datafusion.common.config.ParquetOptions.coerce_int96 String to a arrow_schema.datatype.TimeUnit @@ -587,6 +605,67 @@ impl FileSource for ParquetSource { })) } + fn reorder_files( + &self, + mut files: Vec, + ) -> Vec { + let sort_order = match &self.sort_order_for_reorder { + Some(order) if !order.is_empty() => order, + _ => return files, + }; + + // We only handle single-column sort for now + let first_expr = sort_order.first(); + + // The sort expression must be a Column so we can look up statistics + let col: &datafusion_physical_expr::expressions::Column = + match first_expr.expr.downcast_ref() { + Some(col) => col, + None => return files, + }; + + let col_name = col.name(); + let descending = self.reverse_row_groups; + + // Find the column index in the table schema + let table_schema = self.table_schema.table_schema(); + let col_idx = match table_schema.index_of(col_name) { + Ok(idx) => idx, + Err(_) => return files, + }; + + // Stable sort: files with usable stats first, ordered by the + // relevant bound; files without stats go to the end. + files.sort_by(|a, b| { + let key_a = Self::sort_key_for_file(a, col_idx, descending); + let key_b = Self::sort_key_for_file(b, col_idx, descending); + match (key_a, key_b) { + (Some(va), Some(vb)) => { + if descending { + // DESC: highest min first (reverse order) + vb.partial_cmp(&va).unwrap_or(std::cmp::Ordering::Equal) + } else { + // ASC: lowest max first + va.partial_cmp(&vb).unwrap_or(std::cmp::Ordering::Equal) + } + } + // Files without stats go to the end + (Some(_), None) => std::cmp::Ordering::Less, + (None, Some(_)) => std::cmp::Ordering::Greater, + (None, None) => std::cmp::Ordering::Equal, + } + }); + + log::debug!( + "Reordered {} files by {} {} for TopK optimization", + files.len(), + col_name, + if descending { "DESC" } else { "ASC" } + ); + + files + } + fn table_schema(&self) -> &TableSchema { &self.table_schema } diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index 9b4ae5827ae8b..32bee63b54f23 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -280,6 +280,19 @@ pub trait FileSource: Any + Send + Sync { Ok(SortOrderPushdownResult::Unsupported) } + /// Reorder files in the shared work queue to optimize query performance. + /// + /// For example, TopK queries benefit from reading files with the best + /// statistics first, so the dynamic filter threshold tightens quickly. + /// + /// The default implementation returns files unchanged (no reordering). + fn reorder_files( + &self, + files: Vec, + ) -> Vec { + files + } + /// Try to push down a projection into this FileSource. /// /// `FileSource` implementations that support projection pushdown should diff --git a/datafusion/datasource/src/file_stream/work_source.rs b/datafusion/datasource/src/file_stream/work_source.rs index 7f31dacca9592..b874223a78879 100644 --- a/datafusion/datasource/src/file_stream/work_source.rs +++ b/datafusion/datasource/src/file_stream/work_source.rs @@ -85,8 +85,18 @@ impl SharedWorkSource { } /// Create a shared work source for the unopened files in `config`. + /// + /// Files are reordered by the file source (e.g. by statistics for TopK) + /// before being placed in the shared queue. pub(crate) fn from_config(config: &FileScanConfig) -> Self { - Self::new(config.file_groups.iter().flat_map(FileGroup::iter).cloned()) + let files: Vec<_> = config + .file_groups + .iter() + .flat_map(FileGroup::iter) + .cloned() + .collect(); + let files = config.file_source.reorder_files(files); + Self::new(files) } /// Pop the next file from the shared work queue. From 19a51fce63ac07df3eb10bbedf1a4fec7c6c6e20 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Sat, 18 Apr 2026 15:56:43 +0800 Subject: [PATCH 14/38] feat: initialize TopK dynamic filter threshold from parquet statistics Before reading any parquet data, scan row group min/max statistics to compute an initial threshold for TopK's dynamic filter. This allows row-level filtering to benefit immediately from the first file opened, rather than waiting until TopK processes enough rows to build a threshold organically. Algorithm (single-column sort): - DESC LIMIT K: threshold = max(min) across RGs with num_rows >= K Filter: col > threshold - ASC LIMIT K: threshold = min(max) across RGs with num_rows >= K Filter: col < threshold Sort direction is read from sort_options on DynamicFilterPhysicalExpr, which is now set by SortExec::create_filter() for TopK queries. This makes the optimization work for ALL TopK queries on parquet, not just those with sort pushdown. The DynamicFilterPhysicalExpr is shared across all partitions, so each file's threshold update is visible to subsequent files globally. Graceful fallback: skips initialization when sort_options is absent, statistics are unavailable, column not found, or multi-column sort. --- datafusion/datasource-parquet/src/opener.rs | 618 +++++++++++++++++- .../src/expressions/dynamic_filters.rs | 28 + datafusion/physical-plan/src/sorts/sort.rs | 11 +- 3 files changed, 655 insertions(+), 2 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 2bcffbab24212..aca588d13ea51 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -24,9 +24,13 @@ use crate::{ ParquetAccessPlan, ParquetFileMetrics, ParquetFileReaderFactory, apply_file_schema_type_coercions, coerce_int96_to_resolution, row_filter, }; -use arrow::array::{RecordBatch, RecordBatchOptions}; +use arrow::array::{Array, RecordBatch, RecordBatchOptions}; use arrow::datatypes::DataType; use datafusion_datasource::morsel::{Morsel, MorselPlan, MorselPlanner, Morselizer}; +use datafusion_expr::Operator; +use datafusion_physical_expr::expressions::{ + BinaryExpr, Column, DynamicFilterPhysicalExpr, lit, +}; use datafusion_physical_expr::projection::{ProjectionExprs, Projector}; use datafusion_physical_expr::utils::reassign_expr_columns; use datafusion_physical_expr_adapter::replace_columns_with_literals; @@ -68,6 +72,7 @@ use log::debug; use parquet::DecodeResult; use parquet::arrow::ParquetRecordBatchStreamBuilder; use parquet::arrow::arrow_reader::metrics::ArrowReaderMetrics; +use parquet::arrow::arrow_reader::statistics::StatisticsConverter; use parquet::arrow::arrow_reader::{ ArrowReaderMetadata, ArrowReaderOptions, RowSelectionPolicy, }; @@ -1082,6 +1087,23 @@ impl RowGroupsPrunedParquetOpen { let file_metadata = Arc::clone(reader_metadata.metadata()); let rg_metadata = file_metadata.row_groups(); + // Initialize TopK dynamic filter from row group statistics. + // This sets an initial threshold before any data is read, so that + // subsequent row filtering can benefit immediately. + // Sort direction is read from the DynamicFilterPhysicalExpr's + // sort_options (set by SortExec for TopK queries). + if let (Some(predicate), Some(limit)) = (&prepared.predicate, prepared.limit) + && let Err(e) = try_init_topk_threshold( + predicate, + limit, + rg_metadata, + &prepared.physical_file_schema, + reader_metadata.parquet_schema(), + ) + { + debug!("Skipping TopK threshold initialization from statistics: {e}"); + } + // Filter pushdown: evaluate predicates during scan let row_filter = if let Some(predicate) = prepared .pushdown_filters @@ -1264,6 +1286,200 @@ impl RowGroupsPrunedParquetOpen { } } +/// Attempt to initialize a TopK dynamic filter threshold from row group statistics. +/// +/// Before any parquet data is read, this function scans the row group min/max +/// statistics to compute an initial threshold for the TopK dynamic filter. +/// By setting this threshold early, subsequent row group pruning and row-level +/// filtering can benefit immediately. +/// +/// **Algorithm (single-column sort only):** +/// +/// For `ORDER BY col DESC LIMIT K`: +/// - For each row group where `num_rows >= K`: the min value of that RG is a +/// lower bound on the K-th largest value. +/// - `threshold = max(min)` across all qualifying row groups. +/// - Filter: `col > threshold` +/// +/// For `ORDER BY col ASC LIMIT K`: +/// - For each row group where `num_rows >= K`: the max value is an upper bound +/// on the K-th smallest value. +/// - `threshold = min(max)` across qualifying row groups. +/// - Filter: `col < threshold` +fn try_init_topk_threshold( + predicate: &Arc, + limit: usize, + rg_metadata: &[parquet::file::metadata::RowGroupMetaData], + arrow_schema: &Schema, + parquet_schema: &parquet::schema::types::SchemaDescriptor, +) -> Result<()> { + // Find the DynamicFilterPhysicalExpr in the predicate tree. + let dynamic_filter = match find_dynamic_filter(predicate) { + Some(df) => df, + None => return Ok(()), // No dynamic filter found, nothing to do + }; + + // Only initialize if the filter hasn't been updated yet (generation == 1). + // This prevents a later partition from overwriting a better threshold + // set by an earlier partition (or by TopK itself after processing rows). + if dynamic_filter.snapshot_generation() > 1 { + return Ok(()); + } + + // Read sort options from the dynamic filter (set by SortExec for TopK). + let sort_options = match dynamic_filter.sort_options() { + Some(opts) => opts, + None => return Ok(()), // No sort options, cannot determine direction + }; + + // Only handle single-column sort for now. + if sort_options.len() != 1 { + debug!( + "Skipping TopK threshold initialization: expected 1 sort column, found {}", + sort_options.len() + ); + return Ok(()); + } + + let is_descending = sort_options[0].descending; + + // The child must be a Column expression so we can look up statistics. + let children = dynamic_filter.children(); + let col_expr: Arc = Arc::clone(children[0]); + let col_any: &dyn std::any::Any = col_expr.as_ref(); + let column = col_any.downcast_ref::().ok_or_else(|| { + DataFusionError::Internal( + "TopK threshold init: sort child is not a Column expression".to_string(), + ) + })?; + + let col_name = column.name(); + + // Build a statistics converter for the sort column. + let converter = StatisticsConverter::try_new(col_name, arrow_schema, parquet_schema) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + + // Compute the threshold. + let threshold = if is_descending { + // DESC: threshold = max(min) across RGs with num_rows >= limit + let mins = converter + .row_group_mins(rg_metadata.iter()) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + compute_best_threshold_from_stats(&mins, rg_metadata, limit, true)? + } else { + // ASC: threshold = min(max) across RGs with num_rows >= limit + let maxes = converter + .row_group_maxes(rg_metadata.iter()) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + compute_best_threshold_from_stats(&maxes, rg_metadata, limit, false)? + }; + + let threshold = match threshold { + Some(t) => t, + None => { + debug!("No qualifying row groups for TopK threshold initialization"); + return Ok(()); + } + }; + + // Build the filter expression: col > threshold (DESC) or col < threshold (ASC) + let op = if is_descending { + Operator::Gt + } else { + Operator::Lt + }; + + let filter_expr: Arc = Arc::new(BinaryExpr::new( + Arc::clone(&col_expr), + op, + lit(threshold.clone()), + )); + + debug!( + "Initializing TopK dynamic filter from statistics: {} {} {}", + col_name, + if is_descending { ">" } else { "<" }, + threshold + ); + + dynamic_filter.update(filter_expr)?; + + Ok(()) +} + +/// Find a [`DynamicFilterPhysicalExpr`] in the predicate tree. +/// +/// Returns the first `DynamicFilterPhysicalExpr` found (as an `Arc`) by +/// checking the predicate itself and recursively walking its children. +fn find_dynamic_filter( + expr: &Arc, +) -> Option> { + // Try to downcast this expression directly. + // PhysicalExpr: Any + Send + Sync, so trait upcasting allows the coercion. + let cloned = Arc::clone(expr); + let any_arc: Arc = cloned; + if let Ok(df) = Arc::downcast::(any_arc) { + return Some(df); + } + + // Recursively check children + for child in expr.children() { + if let Some(df) = find_dynamic_filter(child) { + return Some(df); + } + } + + None +} + +/// Compute the best threshold from row group statistics. +/// +/// For `want_max = true` (DESC): finds the maximum value from the stats array +/// across row groups with `num_rows >= limit`. +/// +/// For `want_max = false` (ASC): finds the minimum value from the stats array +/// across row groups with `num_rows >= limit`. +fn compute_best_threshold_from_stats( + stats: &arrow::array::ArrayRef, + rg_metadata: &[parquet::file::metadata::RowGroupMetaData], + limit: usize, + want_max: bool, +) -> Result> { + let mut best: Option = None; + + for (i, rg) in rg_metadata.iter().enumerate() { + // Only consider row groups with enough rows + if (rg.num_rows() as usize) < limit { + continue; + } + + // Skip null statistics + if i >= stats.len() || stats.is_null(i) { + continue; + } + + let value = ScalarValue::try_from_array(stats.as_ref(), i)?; + if value.is_null() { + continue; + } + + best = Some(match best { + None => value, + Some(current) => { + if want_max { + // Keep the maximum + if value > current { value } else { current } + } else { + // Keep the minimum + if value < current { value } else { current } + } + } + }); + } + + Ok(best) +} + /// State for a stream that decodes a single Parquet file using a push-based decoder. /// /// The [`transition`](Self::transition) method drives the decoder in a loop: it requests @@ -2770,4 +2986,404 @@ mod test { "without page index all rows are returned" ); } + + // --------------------------------------------------------------- + // Helper: build RowGroupMetaData with a given num_rows + // --------------------------------------------------------------- + + /// Create a minimal `SchemaDescriptor` with a single Int64 column named `id`. + fn make_int64_schema_descr() -> parquet::schema::types::SchemaDescPtr { + use parquet::basic::Type as PhysicalType; + use parquet::schema::types::Type as SchemaType; + + let field = SchemaType::primitive_type_builder("id", PhysicalType::INT64) + .build() + .unwrap(); + let schema = SchemaType::group_type_builder("schema") + .with_fields(vec![Arc::new(field)]) + .build() + .unwrap(); + Arc::new(parquet::schema::types::SchemaDescriptor::new(Arc::new( + schema, + ))) + } + + /// Build a vector of `RowGroupMetaData`, one per entry in `row_counts`. + fn make_rg_metadata( + row_counts: &[i64], + ) -> Vec { + let schema_descr = make_int64_schema_descr(); + row_counts + .iter() + .map(|&num_rows| { + let column = parquet::file::metadata::ColumnChunkMetaData::builder( + schema_descr.column(0), + ) + .set_num_values(num_rows) + .build() + .unwrap(); + parquet::file::metadata::RowGroupMetaData::builder(schema_descr.clone()) + .set_num_rows(num_rows) + .set_column_metadata(vec![column]) + .build() + .unwrap() + }) + .collect() + } + + // --------------------------------------------------------------- + // Tests for compute_best_threshold_from_stats + // --------------------------------------------------------------- + + #[test] + fn test_compute_threshold_desc_picks_max_of_mins() { + use arrow::array::{ArrayRef, Int64Array}; + + let stats: ArrayRef = + Arc::new(Int64Array::from(vec![Some(10), Some(50), Some(30)])); + let rg_meta = make_rg_metadata(&[100_000, 100_000, 100_000]); + + let result = + compute_best_threshold_from_stats(&stats, &rg_meta, 100, true).unwrap(); + assert_eq!(result, Some(ScalarValue::Int64(Some(50)))); + } + + #[test] + fn test_compute_threshold_asc_picks_min_of_maxes() { + use arrow::array::{ArrayRef, Int64Array}; + + let stats: ArrayRef = + Arc::new(Int64Array::from(vec![Some(100), Some(50), Some(80)])); + let rg_meta = make_rg_metadata(&[100_000, 100_000, 100_000]); + + let result = + compute_best_threshold_from_stats(&stats, &rg_meta, 100, false).unwrap(); + assert_eq!(result, Some(ScalarValue::Int64(Some(50)))); + } + + #[test] + fn test_compute_threshold_skips_small_rgs() { + use arrow::array::{ArrayRef, Int64Array}; + + // RG 0: 100K rows, value=10 + // RG 1: 50 rows (< limit=100), value=999 — should be skipped + // RG 2: 100K rows, value=30 + let stats: ArrayRef = + Arc::new(Int64Array::from(vec![Some(10), Some(999), Some(30)])); + let rg_meta = make_rg_metadata(&[100_000, 50, 100_000]); + + let result = + compute_best_threshold_from_stats(&stats, &rg_meta, 100, true).unwrap(); + // want_max=true => max(10, 30) = 30 (skipping the 999 from the small RG) + assert_eq!(result, Some(ScalarValue::Int64(Some(30)))); + } + + #[test] + fn test_compute_threshold_skips_null_stats() { + use arrow::array::{ArrayRef, Int64Array}; + + let stats: ArrayRef = Arc::new(Int64Array::from(vec![Some(10), None, Some(30)])); + let rg_meta = make_rg_metadata(&[100_000, 100_000, 100_000]); + + let result = + compute_best_threshold_from_stats(&stats, &rg_meta, 100, true).unwrap(); + // Null entry is skipped, max(10, 30) = 30 + assert_eq!(result, Some(ScalarValue::Int64(Some(30)))); + } + + #[test] + fn test_compute_threshold_all_rgs_too_small() { + use arrow::array::{ArrayRef, Int64Array}; + + let stats: ArrayRef = Arc::new(Int64Array::from(vec![Some(10), Some(50)])); + let rg_meta = make_rg_metadata(&[5, 10]); // all < limit=100 + + let result = + compute_best_threshold_from_stats(&stats, &rg_meta, 100, true).unwrap(); + assert_eq!(result, None); + } + + #[test] + fn test_compute_threshold_empty_metadata() { + use arrow::array::{ArrayRef, Int64Array}; + + let stats: ArrayRef = Arc::new(Int64Array::from(Vec::>::new())); + let rg_meta: Vec = vec![]; + + let result = + compute_best_threshold_from_stats(&stats, &rg_meta, 100, true).unwrap(); + assert_eq!(result, None); + } + + // --------------------------------------------------------------- + // Tests for find_dynamic_filter + // --------------------------------------------------------------- + + #[test] + fn test_find_dynamic_filter_direct_match() { + let col_expr = Arc::new(Column::new("id", 0)) as Arc; + let dynamic_filter: Arc = Arc::new( + DynamicFilterPhysicalExpr::new(vec![col_expr], super::lit(true)), + ); + + let result = find_dynamic_filter(&dynamic_filter); + assert!( + result.is_some(), + "should find direct DynamicFilterPhysicalExpr" + ); + } + + #[test] + fn test_find_dynamic_filter_nested_in_conjunction() { + let col_expr = Arc::new(Column::new("id", 0)) as Arc; + let dynamic_filter: Arc = Arc::new( + DynamicFilterPhysicalExpr::new(vec![col_expr], super::lit(true)), + ); + // Wrap it: (id > 0) AND dynamic_filter + let left: Arc = Arc::new(BinaryExpr::new( + Arc::new(Column::new("id", 0)), + Operator::Gt, + super::lit(0i64), + )); + let conjunction: Arc = + Arc::new(BinaryExpr::new(left, Operator::And, dynamic_filter)); + + let result = find_dynamic_filter(&conjunction); + assert!( + result.is_some(), + "should find DynamicFilterPhysicalExpr nested in AND" + ); + } + + #[test] + fn test_find_dynamic_filter_none_when_absent() { + let col_expr: Arc = Arc::new(Column::new("id", 0)); + let result = find_dynamic_filter(&col_expr); + assert!( + result.is_none(), + "plain Column has no DynamicFilterPhysicalExpr" + ); + } + + // --------------------------------------------------------------- + // Tests for try_init_topk_threshold + // --------------------------------------------------------------- + + /// Write a small parquet file and return its metadata so we can build + /// realistic `RowGroupMetaData` with actual statistics. + fn make_parquet_metadata_with_stats( + row_counts: &[Vec], + ) -> ( + Vec, + Schema, + parquet::schema::types::SchemaDescPtr, + ) { + use parquet::arrow::ArrowWriter; + use parquet::file::reader::FileReader; + use parquet::file::serialized_reader::SerializedFileReader; + + let schema = + Arc::new(Schema::new(vec![Field::new("id", DataType::Int64, false)])); + + // Write each row_counts entry as a separate row group + let mut buf = Vec::new(); + let mut writer = + ArrowWriter::try_new(&mut buf, Arc::clone(&schema), None).unwrap(); + for values in row_counts { + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(arrow::array::Int64Array::from(values.clone()))], + ) + .unwrap(); + writer.write(&batch).unwrap(); + writer.flush().unwrap(); // flush after each batch -> separate RG + } + writer.close().unwrap(); + + // Read back metadata + let reader = SerializedFileReader::new(bytes::Bytes::from(buf)).unwrap(); + let file_metadata = reader.metadata(); + let rg_metadata: Vec<_> = file_metadata.row_groups().to_vec(); + let parquet_schema = file_metadata.file_metadata().schema_descr_ptr(); + (rg_metadata, schema.as_ref().clone(), parquet_schema) + } + + #[test] + fn test_try_init_topk_threshold_desc() { + // Two RGs: + // RG0: values [1..=200] => min=1, max=200 + // RG1: values [100..=300] => min=100, max=300 + // DESC sort, limit=10: threshold = max(min) = max(1, 100) = 100 + // Filter should be: id > 100 + let rg0_values: Vec = (1..=200).collect(); + let rg1_values: Vec = (100..=300).collect(); + let (rg_metadata, arrow_schema, parquet_schema) = + make_parquet_metadata_with_stats(&[rg0_values, rg1_values]); + + let col_expr = Arc::new(Column::new("id", 0)) as Arc; + let desc_opts = arrow::compute::SortOptions { + descending: true, + nulls_first: true, + }; + let dynamic_filter = Arc::new(DynamicFilterPhysicalExpr::new_with_sort_options( + vec![Arc::clone(&col_expr)], + super::lit(true), + vec![desc_opts], + )); + let predicate: Arc = dynamic_filter.clone(); + + try_init_topk_threshold( + &predicate, + 10, + &rg_metadata, + &arrow_schema, + &parquet_schema, + ) + .unwrap(); + + // Verify the dynamic filter was updated + let current = dynamic_filter.current().unwrap(); + let display = format!("{current}"); + assert!( + display.contains(">"), + "DESC should produce Gt operator, got: {display}" + ); + assert!( + display.contains("100"), + "threshold should be 100 (max of mins), got: {display}" + ); + } + + #[test] + fn test_try_init_topk_threshold_asc() { + // Two RGs: + // RG0: values [1..=200] => min=1, max=200 + // RG1: values [100..=300] => min=100, max=300 + // ASC sort, limit=10: threshold = min(max) = min(200, 300) = 200 + // Filter should be: id < 200 + let rg0_values: Vec = (1..=200).collect(); + let rg1_values: Vec = (100..=300).collect(); + let (rg_metadata, arrow_schema, parquet_schema) = + make_parquet_metadata_with_stats(&[rg0_values, rg1_values]); + + let col_expr = Arc::new(Column::new("id", 0)) as Arc; + let asc_opts = arrow::compute::SortOptions { + descending: false, + nulls_first: false, + }; + let dynamic_filter = Arc::new(DynamicFilterPhysicalExpr::new_with_sort_options( + vec![Arc::clone(&col_expr)], + super::lit(true), + vec![asc_opts], + )); + let predicate: Arc = dynamic_filter.clone(); + + try_init_topk_threshold( + &predicate, + 10, + &rg_metadata, + &arrow_schema, + &parquet_schema, + ) + .unwrap(); + + let current = dynamic_filter.current().unwrap(); + let display = format!("{current}"); + assert!( + display.contains("<"), + "ASC should produce Lt operator, got: {display}" + ); + assert!( + display.contains("200"), + "threshold should be 200 (min of maxes), got: {display}" + ); + } + + #[test] + fn test_try_init_topk_threshold_no_dynamic_filter() { + let rg0_values: Vec = (1..=200).collect(); + let (rg_metadata, arrow_schema, parquet_schema) = + make_parquet_metadata_with_stats(&[rg0_values]); + + // Plain predicate with no DynamicFilterPhysicalExpr + let predicate: Arc = Arc::new(Column::new("id", 0)); + + let result = try_init_topk_threshold( + &predicate, + 10, + &rg_metadata, + &arrow_schema, + &parquet_schema, + ); + assert!(result.is_ok(), "should be a no-op when no dynamic filter"); + } + + #[test] + fn test_try_init_topk_threshold_no_sort_options() { + let rg0_values: Vec = (1..=200).collect(); + let (rg_metadata, arrow_schema, parquet_schema) = + make_parquet_metadata_with_stats(&[rg0_values]); + + // DynamicFilterPhysicalExpr WITHOUT sort_options => should be skipped + let col_expr = Arc::new(Column::new("id", 0)) as Arc; + let dynamic_filter = Arc::new(DynamicFilterPhysicalExpr::new( + vec![col_expr], + super::lit(true), + )); + let predicate: Arc = dynamic_filter.clone(); + + let result = try_init_topk_threshold( + &predicate, + 10, + &rg_metadata, + &arrow_schema, + &parquet_schema, + ); + assert!( + result.is_ok(), + "no sort_options should be skipped gracefully" + ); + + let current = dynamic_filter.current().unwrap(); + let display = format!("{current}"); + assert_eq!(display, "true", "filter should remain unchanged: {display}"); + } + + #[test] + fn test_try_init_topk_threshold_multi_column_skipped() { + let rg0_values: Vec = (1..=200).collect(); + let (rg_metadata, arrow_schema, parquet_schema) = + make_parquet_metadata_with_stats(&[rg0_values]); + + // DynamicFilterPhysicalExpr with 2 sort columns => should be skipped + let col1 = Arc::new(Column::new("id", 0)) as Arc; + let col2 = Arc::new(Column::new("id", 0)) as Arc; + let opts = arrow::compute::SortOptions { + descending: true, + nulls_first: true, + }; + let dynamic_filter = Arc::new(DynamicFilterPhysicalExpr::new_with_sort_options( + vec![col1, col2], + super::lit(true), + vec![opts, opts], + )); + let predicate: Arc = dynamic_filter.clone(); + + let result = try_init_topk_threshold( + &predicate, + 10, + &rg_metadata, + &arrow_schema, + &parquet_schema, + ); + assert!( + result.is_ok(), + "multi-column sort should be skipped gracefully" + ); + + // Filter should still be the original `super::lit(true)` + let current = dynamic_filter.current().unwrap(); + let display = format!("{current}"); + assert_eq!(display, "true", "filter should remain unchanged: {display}"); + } } diff --git a/datafusion/physical-expr/src/expressions/dynamic_filters.rs b/datafusion/physical-expr/src/expressions/dynamic_filters.rs index 47398d87e26a5..5671426f50a15 100644 --- a/datafusion/physical-expr/src/expressions/dynamic_filters.rs +++ b/datafusion/physical-expr/src/expressions/dynamic_filters.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use arrow::compute::SortOptions; use parking_lot::RwLock; use std::{fmt::Display, hash::Hash, sync::Arc}; use tokio::sync::watch; @@ -74,6 +75,11 @@ pub struct DynamicFilterPhysicalExpr { /// But this can have overhead in production, so it's only included in our tests. data_type: Arc>>, nullable: Arc>>, + /// Optional sort options for each child expression. + /// When set (e.g., by SortExec for TopK), downstream consumers like the + /// parquet reader can use these to initialize the filter threshold from + /// column statistics before reading any data. + sort_options: Option>, } #[derive(Debug)] @@ -177,9 +183,30 @@ impl DynamicFilterPhysicalExpr { state_watch, data_type: Arc::new(RwLock::new(None)), nullable: Arc::new(RwLock::new(None)), + sort_options: None, } } + /// Create a new [`DynamicFilterPhysicalExpr`] with sort options. + /// + /// Sort options indicate the sort direction for each child expression, + /// enabling downstream consumers (e.g., parquet readers) to initialize + /// the filter threshold from column statistics before reading data. + pub fn new_with_sort_options( + children: Vec>, + inner: Arc, + sort_options: Vec, + ) -> Self { + let mut this = Self::new(children, inner); + this.sort_options = Some(sort_options); + this + } + + /// Returns the sort options for each child expression, if available. + pub fn sort_options(&self) -> Option<&[SortOptions]> { + self.sort_options.as_deref() + } + fn remap_children( children: &[Arc], remapped_children: Option<&Vec>>, @@ -368,6 +395,7 @@ impl PhysicalExpr for DynamicFilterPhysicalExpr { state_watch: self.state_watch.clone(), data_type: Arc::clone(&self.data_type), nullable: Arc::clone(&self.nullable), + sort_options: self.sort_options.clone(), })) } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 6c02af8dec6d3..89fb558789d36 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -915,8 +915,17 @@ impl SortExec { .iter() .map(|sort_expr| Arc::clone(&sort_expr.expr)) .collect::>(); + let sort_options = self + .expr + .iter() + .map(|sort_expr| sort_expr.options) + .collect::>(); Arc::new(RwLock::new(TopKDynamicFilters::new(Arc::new( - DynamicFilterPhysicalExpr::new(children, lit(true)), + DynamicFilterPhysicalExpr::new_with_sort_options( + children, + lit(true), + sort_options, + ), )))) } From e230d2f61843ba7353d5549f66f833acc7d6c18d Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Mon, 20 Apr 2026 23:36:03 +0800 Subject: [PATCH 15/38] feat: enable file reorder and RG reorder for all TopK queries Previously file reorder and RG reorder only worked with sort pushdown (Inexact path, WITH ORDER). Now they extract sort info from DynamicFilterPhysicalExpr.sort_options in the predicate, which is set by SortExec for ALL TopK queries regardless of WITH ORDER. This means ORDER BY col DESC LIMIT K on any parquet table benefits from file reorder (best file first in shared queue), RG reorder (best RG first within file), and stats init (threshold before I/O). --- datafusion/datasource-parquet/src/opener.rs | 76 ++++++++++++++++-- datafusion/datasource-parquet/src/source.rs | 88 +++++++++++++++++---- 2 files changed, 141 insertions(+), 23 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index aca588d13ea51..8b3c92e7ba68d 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -1165,17 +1165,41 @@ impl RowGroupsPrunedParquetOpen { // // For sorted data: reorder is a no-op, reverse gives perfect DESC. // For unsorted data: reorder fixes the order, reverse flips for DESC. + // Build reorder optimizer from sort_order_for_reorder (Inexact path) + // or from DynamicFilterPhysicalExpr sort_options (any TopK query). let reorder_optimizer: Option< Box, - > = prepared.sort_order_for_reorder.as_ref().map(|sort_order| { - Box::new(crate::access_plan_optimizer::ReorderByStatistics::new( - sort_order.clone(), - )) as Box - }); + > = if let Some(sort_order) = &prepared.sort_order_for_reorder { + Some( + Box::new(crate::access_plan_optimizer::ReorderByStatistics::new( + sort_order.clone(), + )) + as Box, + ) + } else if let Some(predicate) = &prepared.predicate { + // For non-Inexact TopK queries: extract sort info from the + // DynamicFilterPhysicalExpr to enable RG reorder. + extract_sort_order_from_predicate(predicate).map(|sort_order| { + Box::new(crate::access_plan_optimizer::ReorderByStatistics::new( + sort_order, + )) + as Box + }) + } else { + None + }; + // Reverse for DESC queries. Triggered by sort pushdown (reverse_row_groups) + // or by DynamicFilterPhysicalExpr sort_options indicating DESC. + let is_descending = prepared.reverse_row_groups + || prepared + .predicate + .as_ref() + .and_then(extract_descending_from_predicate) + .unwrap_or(false); let reverse_optimizer: Option< Box, - > = if prepared.reverse_row_groups { + > = if is_descending { Some(Box::new(crate::access_plan_optimizer::ReverseRowGroups)) } else { None @@ -1480,6 +1504,46 @@ fn compute_best_threshold_from_stats( Ok(best) } +/// Extract a [`LexOrdering`] from a [`DynamicFilterPhysicalExpr`] in the predicate. +/// +/// Returns a single-column ASC sort order if a DynamicFilterPhysicalExpr with +/// sort_options is found. The sort direction in the returned LexOrdering is +/// always ASC because `ReorderByStatistics` always sorts by min ASC; the +/// caller handles DESC via `ReverseRowGroups`. +fn extract_sort_order_from_predicate( + predicate: &Arc, +) -> Option { + let df = find_dynamic_filter(predicate)?; + let sort_options = df.sort_options()?; + if sort_options.len() != 1 { + return None; + } + let children = df.children(); + if children.is_empty() { + return None; + } + // Build ASC sort order (reorder always sorts ASC, reverse handles DESC) + let sort_expr = datafusion_physical_expr_common::sort_expr::PhysicalSortExpr { + expr: Arc::clone(children[0]), + options: arrow::compute::SortOptions { + descending: false, + nulls_first: sort_options[0].nulls_first, + }, + }; + LexOrdering::new(vec![sort_expr]) +} + +/// Check if a [`DynamicFilterPhysicalExpr`] in the predicate indicates DESC sort. +fn extract_descending_from_predicate(predicate: &Arc) -> Option { + let df = find_dynamic_filter(predicate)?; + let sort_options = df.sort_options()?; + if sort_options.len() == 1 { + Some(sort_options[0].descending) + } else { + None + } +} + /// State for a stream that decodes a single Parquet file using a push-based decoder. /// /// The [`transition`](Self::transition) method drives the decoder in a loop: it requests diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index bfd3ea37b0ca3..e1dbc28cb00e2 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -486,6 +486,70 @@ impl ParquetSource { self.reverse_row_groups } + /// Extract TopK sort column name and direction from available sources. + /// + /// Tries two sources in order: + /// 1. DynamicFilterPhysicalExpr in the predicate (works for ALL TopK queries) + /// 2. sort_order_for_reorder (only set in Inexact sort pushdown path) + fn extract_topk_sort_info(&self) -> Option<(String, bool)> { + // Try 1: from predicate's DynamicFilterPhysicalExpr + if let Some(predicate) = &self.predicate + && let Some(info) = Self::sort_info_from_dynamic_filter(predicate) + { + return Some(info); + } + + // Try 2: fallback to sort_order_for_reorder (Inexact path) + if let Some(sort_order) = &self.sort_order_for_reorder + && !sort_order.is_empty() + { + let first = sort_order.first(); + if let Some(col) = first + .expr + .downcast_ref::() + { + return Some((col.name().to_string(), self.reverse_row_groups)); + } + } + + None + } + + /// Try to extract sort column name and direction from a DynamicFilterPhysicalExpr + /// in the predicate tree. + fn sort_info_from_dynamic_filter( + expr: &Arc, + ) -> Option<(String, bool)> { + // Try downcast to DynamicFilterPhysicalExpr + let cloned = Arc::clone(expr); + let any_arc: Arc = cloned; + if let Ok(df) = Arc::downcast::< + datafusion_physical_expr::expressions::DynamicFilterPhysicalExpr, + >(any_arc) + { + let sort_options = df.sort_options()?; + if sort_options.len() != 1 { + return None; + } + let children = df.children(); + if children.is_empty() { + return None; + } + let col = children[0] + .downcast_ref::()?; + return Some((col.name().to_string(), sort_options[0].descending)); + } + + // Recursively check children + for child in expr.children() { + if let Some(info) = Self::sort_info_from_dynamic_filter(child) { + return Some(info); + } + } + + None + } + /// Extract the sort key from a file's statistics for reordering. /// /// For DESC sorts, returns the column's min_value (we want highest min first). @@ -609,27 +673,17 @@ impl FileSource for ParquetSource { &self, mut files: Vec, ) -> Vec { - let sort_order = match &self.sort_order_for_reorder { - Some(order) if !order.is_empty() => order, - _ => return files, + // Extract sort column and direction from either: + // 1. The DynamicFilterPhysicalExpr in the predicate (works for ALL TopK) + // 2. Fallback to sort_order_for_reorder (Inexact path only) + let (col_name, descending) = match self.extract_topk_sort_info() { + Some(info) => info, + None => return files, }; - // We only handle single-column sort for now - let first_expr = sort_order.first(); - - // The sort expression must be a Column so we can look up statistics - let col: &datafusion_physical_expr::expressions::Column = - match first_expr.expr.downcast_ref() { - Some(col) => col, - None => return files, - }; - - let col_name = col.name(); - let descending = self.reverse_row_groups; - // Find the column index in the table schema let table_schema = self.table_schema.table_schema(); - let col_idx = match table_schema.index_of(col_name) { + let col_idx = match table_schema.index_of(&col_name) { Ok(idx) => idx, Err(_) => return files, }; From cfdacf0b77f7c13ccb2a2122d1b495f52715fd02 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Mon, 20 Apr 2026 23:47:15 +0800 Subject: [PATCH 16/38] perf: move stats init before RG pruning so first file also benefits MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Move try_init_topk_threshold() from build_stream() to prune_row_groups(), before prune_by_statistics(). This way: - File 1: stats init sets threshold from ALL its RG statistics, then prune_by_statistics uses it to prune file 1's own RGs. Only the best RG(s) are read, rest skipped with zero I/O. - File 2+: dynamic filter already has tight threshold from file 1, most RGs pruned immediately. This effectively achieves dynamic RG pruning without needing morsel- level scheduling — the threshold is computed from statistics (no data read), then used to prune RGs in the same file. --- datafusion/datasource-parquet/src/opener.rs | 33 ++++++++++----------- 1 file changed, 16 insertions(+), 17 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 8b3c92e7ba68d..a2960ec2f613e 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -905,6 +905,22 @@ impl FiltersPreparedParquetOpen { row_groups.prune_by_range(rg_metadata, range); } + // Initialize TopK dynamic filter threshold from row group statistics + // BEFORE row group pruning, so that this file's own RGs can be pruned + // by the threshold. For the first file, this sets the initial threshold; + // for subsequent files, the threshold is already set by earlier files. + if let (Some(predicate), Some(limit)) = (&prepared.predicate, prepared.limit) + && let Err(e) = try_init_topk_threshold( + predicate, + limit, + rg_metadata, + &prepared.physical_file_schema, + loaded.reader_metadata.parquet_schema(), + ) + { + debug!("Skipping TopK threshold initialization from statistics: {e}"); + } + // If there is a predicate that can be evaluated against the metadata if let Some(predicate) = self.pruning_predicate.as_ref().map(|p| p.as_ref()) { if prepared.enable_row_group_stats_pruning { @@ -1087,23 +1103,6 @@ impl RowGroupsPrunedParquetOpen { let file_metadata = Arc::clone(reader_metadata.metadata()); let rg_metadata = file_metadata.row_groups(); - // Initialize TopK dynamic filter from row group statistics. - // This sets an initial threshold before any data is read, so that - // subsequent row filtering can benefit immediately. - // Sort direction is read from the DynamicFilterPhysicalExpr's - // sort_options (set by SortExec for TopK queries). - if let (Some(predicate), Some(limit)) = (&prepared.predicate, prepared.limit) - && let Err(e) = try_init_topk_threshold( - predicate, - limit, - rg_metadata, - &prepared.physical_file_schema, - reader_metadata.parquet_schema(), - ) - { - debug!("Skipping TopK threshold initialization from statistics: {e}"); - } - // Filter pushdown: evaluate predicates during scan let row_filter = if let Some(predicate) = prepared .pushdown_filters From 5199d9f8531e34a467fae771abd13018f530b328 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 00:06:30 +0800 Subject: [PATCH 17/38] fix: restrict RG reorder/reverse to sort pushdown path only RG reorder and reverse must only trigger when sort pushdown is active (sort_order_for_reorder is set). Applying them to non-sort-pushdown TopK queries changes the RG read order, which alters tie-breaking for equal values (e.g. NULLs) and causes non-deterministic results. File reorder and stats init remain enabled for ALL TopK queries since they only affect pruning (which rows are skipped), not the relative order of rows within remaining RGs. Fixes fuzz_cases::topk_filter_pushdown::test_fuzz_topk_filter_pushdown --- datafusion/datasource-parquet/src/opener.rs | 80 ++++----------------- 1 file changed, 12 insertions(+), 68 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index a2960ec2f613e..622a203183991 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -1168,37 +1168,21 @@ impl RowGroupsPrunedParquetOpen { // or from DynamicFilterPhysicalExpr sort_options (any TopK query). let reorder_optimizer: Option< Box, - > = if let Some(sort_order) = &prepared.sort_order_for_reorder { - Some( - Box::new(crate::access_plan_optimizer::ReorderByStatistics::new( - sort_order.clone(), - )) - as Box, - ) - } else if let Some(predicate) = &prepared.predicate { - // For non-Inexact TopK queries: extract sort info from the - // DynamicFilterPhysicalExpr to enable RG reorder. - extract_sort_order_from_predicate(predicate).map(|sort_order| { - Box::new(crate::access_plan_optimizer::ReorderByStatistics::new( - sort_order, - )) - as Box - }) - } else { - None - }; + // RG reorder only in sort pushdown path (sort_order_for_reorder set). + // Reordering RGs without sort pushdown changes tie-breaking for equal + // values, which can cause non-deterministic results. + > = prepared.sort_order_for_reorder.as_ref().map(|sort_order| { + Box::new(crate::access_plan_optimizer::ReorderByStatistics::new( + sort_order.clone(), + )) as Box + }); - // Reverse for DESC queries. Triggered by sort pushdown (reverse_row_groups) - // or by DynamicFilterPhysicalExpr sort_options indicating DESC. - let is_descending = prepared.reverse_row_groups - || prepared - .predicate - .as_ref() - .and_then(extract_descending_from_predicate) - .unwrap_or(false); + // Reverse for DESC queries. ONLY triggered by sort pushdown + // (reverse_row_groups=true), which means the file is declared WITH ORDER + // and data is sorted. Reversing unsorted data would produce wrong results. let reverse_optimizer: Option< Box, - > = if is_descending { + > = if prepared.reverse_row_groups { Some(Box::new(crate::access_plan_optimizer::ReverseRowGroups)) } else { None @@ -1503,46 +1487,6 @@ fn compute_best_threshold_from_stats( Ok(best) } -/// Extract a [`LexOrdering`] from a [`DynamicFilterPhysicalExpr`] in the predicate. -/// -/// Returns a single-column ASC sort order if a DynamicFilterPhysicalExpr with -/// sort_options is found. The sort direction in the returned LexOrdering is -/// always ASC because `ReorderByStatistics` always sorts by min ASC; the -/// caller handles DESC via `ReverseRowGroups`. -fn extract_sort_order_from_predicate( - predicate: &Arc, -) -> Option { - let df = find_dynamic_filter(predicate)?; - let sort_options = df.sort_options()?; - if sort_options.len() != 1 { - return None; - } - let children = df.children(); - if children.is_empty() { - return None; - } - // Build ASC sort order (reorder always sorts ASC, reverse handles DESC) - let sort_expr = datafusion_physical_expr_common::sort_expr::PhysicalSortExpr { - expr: Arc::clone(children[0]), - options: arrow::compute::SortOptions { - descending: false, - nulls_first: sort_options[0].nulls_first, - }, - }; - LexOrdering::new(vec![sort_expr]) -} - -/// Check if a [`DynamicFilterPhysicalExpr`] in the predicate indicates DESC sort. -fn extract_descending_from_predicate(predicate: &Arc) -> Option { - let df = find_dynamic_filter(predicate)?; - let sort_options = df.sort_options()?; - if sort_options.len() == 1 { - Some(sort_options[0].descending) - } else { - None - } -} - /// State for a stream that decodes a single Parquet file using a push-based decoder. /// /// The [`transition`](Self::transition) method drives the decoder in a loop: it requests From b2e93bc3ae56a8335a7ce6d8d98309f56cfd8252 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 12:27:44 +0800 Subject: [PATCH 18/38] perf: move stats init before PruningPredicate build + fix CastExpr unwrap Critical fix: PruningPredicate compiles the expression at build time, so the DynamicFilterPhysicalExpr must be updated BEFORE the predicate is built. Previously stats init ran after, making RG pruning ineffective for the current file. Also fixes: - Unwrap CastExpr to find the inner Column (projection may add casts) - Use limit=1 default when scan limit is None (TopK fetch is at SortExec level, not pushed to scan) - Only init threshold in sort pushdown path to avoid tie-breaking changes for non-sort-pushdown TopK queries Local benchmark: single file with 61 sorted RGs, DESC LIMIT Baseline: 22-25ms per query Feature: 0.4-1.2ms per query (20-58x faster) --- benchmarks/bench.sh | 154 +++++++++++++------- datafusion/datasource-parquet/src/opener.rs | 72 ++++++--- 2 files changed, 151 insertions(+), 75 deletions(-) diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index 49065906d4063..94ba8b4a093c7 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -109,9 +109,9 @@ clickbench_extended: ClickBench \"inspired\" queries against a single parquet # Sort Pushdown Benchmarks sort_pushdown: Sort pushdown baseline (no WITH ORDER) on TPC-H data (SF=1) sort_pushdown_sorted: Sort pushdown with WITH ORDER — tests sort elimination on non-overlapping files -sort_pushdown_inexact: Sort pushdown Inexact path (--sorted DESC) — tests reverse scan + RG reorder -sort_pushdown_inexact_unsorted: Sort pushdown Inexact path (no WITH ORDER) — tests Unsupported path + RG reorder -sort_pushdown_inexact_overlap: Sort pushdown Inexact path — partially overlapping RGs (streaming data scenario) +sort_pushdown_inexact: Sort pushdown Inexact path (--sorted DESC) — multi-file with scrambled RGs, tests reverse scan + RG reorder +sort_pushdown_inexact_unsorted: Sort pushdown Inexact path (no WITH ORDER) — same data, tests Unsupported path + RG reorder +sort_pushdown_inexact_overlap: Sort pushdown Inexact path — multi-file scrambled RGs (streaming data scenario) # Sorted Data Benchmarks (ORDER BY Optimization) clickbench_sorted: ClickBench queries on pre-sorted data using prefer_existing_sort (tests sort elimination optimization) @@ -1154,10 +1154,23 @@ run_sort_pushdown_sorted() { # Generates data for sort pushdown Inexact benchmark. # -# Produces a single large lineitem parquet file where row groups have -# NON-OVERLAPPING but OUT-OF-ORDER l_orderkey ranges (each RG internally -# sorted, RGs shuffled). This simulates append-heavy workloads where data -# is written in batches at different times. +# Produces multiple parquet files where each file has MULTIPLE row groups +# with scrambled RG order. This tests both: +# - Row-group-level reorder within each file (reorder_by_statistics) +# - TopK threshold initialization from RG statistics +# +# Strategy: +# 1. Write a single sorted file with small (100K-row) RGs (~61 RGs total). +# 2. Use pyarrow to redistribute RGs into N_FILES files, scrambling the +# RG order within each file using a deterministic permutation. +# Each file gets ~61/N_FILES RGs with narrow, non-overlapping ranges +# but in scrambled order. +# +# Writing a single file with ORDER BY scramble does NOT work: the parquet +# writer merges rows from adjacent chunks at RG boundaries, widening +# ranges and defeating reorder_by_statistics. +# +# Requires pyarrow (pip install pyarrow). data_sort_pushdown_inexact() { INEXACT_DIR="${DATA_DIR}/sort_pushdown_inexact/lineitem" if [ -d "${INEXACT_DIR}" ] && [ "$(ls -A ${INEXACT_DIR}/*.parquet 2>/dev/null)" ]; then @@ -1165,7 +1178,7 @@ data_sort_pushdown_inexact() { return fi - echo "Generating sort pushdown Inexact benchmark data (single file, shuffled RGs)..." + echo "Generating sort pushdown Inexact benchmark data (multi-file, scrambled RGs)..." # Re-use the sort_pushdown data as the source (generate if missing) data_sort_pushdown @@ -1173,74 +1186,111 @@ data_sort_pushdown_inexact() { mkdir -p "${INEXACT_DIR}" SRC_DIR="${DATA_DIR}/sort_pushdown/lineitem" - # Use datafusion-cli to bucket rows into 64 groups by a deterministic - # scrambler, then sort within each bucket by orderkey. This produces - # ~64 RG-sized segments where each has a tight orderkey range but the - # segments appear in scrambled (non-sorted) order in the file. + # Step 1: Write a single sorted file with small (100K-row) RGs + TMPFILE="${INEXACT_DIR}/_sorted_small_rgs.parquet" (cd "${SCRIPT_DIR}/.." && cargo run --release -p datafusion-cli -- -c " CREATE EXTERNAL TABLE src STORED AS PARQUET LOCATION '${SRC_DIR}'; - COPY ( - SELECT * FROM src - ORDER BY - (l_orderkey * 1664525 + 1013904223) % 64, - l_orderkey - ) - TO '${INEXACT_DIR}/shuffled.parquet' + COPY (SELECT * FROM src ORDER BY l_orderkey) + TO '${TMPFILE}' STORED AS PARQUET OPTIONS ('format.max_row_group_size' '100000'); ") - echo "Sort pushdown Inexact shuffled data generated at ${INEXACT_DIR}" + # Step 2: Redistribute RGs into 3 files with scrambled RG order. + # Each file gets ~20 RGs. RG assignment: rg_idx % 3 determines file, + # permutation (rg_idx * 41 + 7) % n scrambles the order within file. + python3 -c " +import pyarrow.parquet as pq + +pf = pq.ParquetFile('${TMPFILE}') +n = pf.metadata.num_row_groups +n_files = 3 + +# Assign each RG to a file, scramble order within each file +file_rgs = [[] for _ in range(n_files)] +for rg_idx in range(n): + slot = (rg_idx * 41 + 7) % n # scrambled index + file_id = slot % n_files + file_rgs[file_id].append(rg_idx) + +# Write each file with its assigned RGs (in scrambled order) +for file_id in range(n_files): + rgs = file_rgs[file_id] + if not rgs: + continue + tables = [pf.read_row_group(rg) for rg in rgs] + writer = pq.ParquetWriter( + '${INEXACT_DIR}/part_%03d.parquet' % file_id, + pf.schema_arrow) + for t in tables: + writer.write_table(t) + writer.close() + print(f'File part_{file_id:03d}.parquet: {len(rgs)} RGs') +" + + rm -f "${TMPFILE}" + echo "Sort pushdown Inexact data generated at ${INEXACT_DIR}" ls -la "${INEXACT_DIR}" - # Also generate a file with partially overlapping AND out-of-order - # row groups. Simulates streaming data with network delays: chunks - # arrive out of sequence, and each chunk has small jitter causing - # overlap with neighbors. This is the pattern described by - # @adriangb — data arriving with timestamps that are generally - # increasing but network-induced delays cause chunks to arrive - # out of order with small overlaps between row groups. + # Also generate overlap data: same strategy but with different file count + # and permutation. Simulates streaming data with network delays where + # chunks arrive out of sequence. # - # Strategy: bucket rows into 60 chunks (~100K rows each), sort - # within each chunk (with jitter for overlap), then scramble the - # chunk order using a deterministic permutation. This produces - # RGs that are individually sorted but appear in scrambled order - # in the file — so reorder_by_statistics has real work to do. + # Requires pyarrow (pip install pyarrow). OVERLAP_DIR="${DATA_DIR}/sort_pushdown_inexact_overlap/lineitem" if [ -d "${OVERLAP_DIR}" ] && [ "$(ls -A ${OVERLAP_DIR}/*.parquet 2>/dev/null)" ]; then echo "Sort pushdown Inexact overlap data already exists at ${OVERLAP_DIR}" return fi - echo "Generating sort pushdown Inexact overlap data (scrambled + overlapping RGs)..." + echo "Generating sort pushdown Inexact overlap data (multi-file, scrambled RGs)..." mkdir -p "${OVERLAP_DIR}" + # Step 1: Write a single sorted file with small (100K-row) RGs + TMPFILE="${OVERLAP_DIR}/_sorted_small_rgs.parquet" (cd "${SCRIPT_DIR}/.." && cargo run --release -p datafusion-cli -- -c " CREATE EXTERNAL TABLE src STORED AS PARQUET LOCATION '${SRC_DIR}'; - -- Bucket into 60 chunks (each ~100K rows), sort within each chunk, - -- then scramble chunk order. This produces overlapping RGs that are - -- individually sorted but appear in scrambled order in the file. - COPY ( - SELECT * FROM src - ORDER BY - -- Scramble chunk order: chunk_id -> permuted_chunk_id - (CAST(l_orderkey / 100000 AS INT) * 37 + 13) % 60, - -- Within each chunk, add small jitter for overlap - l_orderkey + (l_orderkey * 7 % 5000) - 2500 - ) - TO '${OVERLAP_DIR}/overlapping.parquet' + COPY (SELECT * FROM src ORDER BY l_orderkey) + TO '${TMPFILE}' STORED AS PARQUET OPTIONS ('format.max_row_group_size' '100000'); ") - echo "Sort pushdown Inexact overlap data generated at ${OVERLAP_DIR}" - ls -la "${OVERLAP_DIR}" + # Step 2: Redistribute into 5 files with scrambled RG order. + python3 -c " +import pyarrow.parquet as pq + +pf = pq.ParquetFile('${TMPFILE}') +n = pf.metadata.num_row_groups +n_files = 5 + +file_rgs = [[] for _ in range(n_files)] +for rg_idx in range(n): + slot = (rg_idx * 37 + 13) % n + file_id = slot % n_files + file_rgs[file_id].append(rg_idx) + +for file_id in range(n_files): + rgs = file_rgs[file_id] + if not rgs: + continue + tables = [pf.read_row_group(rg) for rg in rgs] + writer = pq.ParquetWriter( + '${OVERLAP_DIR}/part_%03d.parquet' % file_id, + pf.schema_arrow) + for t in tables: + writer.write_table(t) + writer.close() + print(f'File part_{file_id:03d}.parquet: {len(rgs)} RGs') +" + + rm -f "${TMPFILE}" } # Runs the sort pushdown Inexact benchmark (tests RG reorder by statistics). @@ -1249,7 +1299,7 @@ data_sort_pushdown_inexact() { run_sort_pushdown_inexact() { INEXACT_DIR="${DATA_DIR}/sort_pushdown_inexact" RESULTS_FILE="${RESULTS_DIR}/sort_pushdown_inexact.json" - echo "Running sort pushdown Inexact benchmark (--sorted, DESC, reverse scan path)..." + echo "Running sort pushdown Inexact benchmark (multi-file scrambled RGs, --sorted DESC)..." DATAFUSION_EXECUTION_PARQUET_PUSHDOWN_FILTERS=true \ debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --sorted --iterations 5 --path "${INEXACT_DIR}" --queries-path "${SCRIPT_DIR}/queries/sort_pushdown_inexact" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} } @@ -1265,13 +1315,13 @@ run_sort_pushdown_inexact_unsorted() { debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --iterations 5 --path "${INEXACT_DIR}" --queries-path "${SCRIPT_DIR}/queries/sort_pushdown_inexact_unsorted" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} } -# Runs the sort pushdown benchmark with scrambled + overlapping RGs. -# Simulates streaming data with network delays — RGs are out of order -# AND have small overlaps (jitter). Tests reorder_by_statistics effectiveness. +# Runs the sort pushdown benchmark with multi-file scrambled RG order. +# Simulates streaming data with network delays — multiple files, each with +# scrambled RGs. Tests both RG-level reorder and TopK stats initialization. run_sort_pushdown_inexact_overlap() { OVERLAP_DIR="${DATA_DIR}/sort_pushdown_inexact_overlap" RESULTS_FILE="${RESULTS_DIR}/sort_pushdown_inexact_overlap.json" - echo "Running sort pushdown Inexact benchmark (overlapping RGs, streaming data pattern)..." + echo "Running sort pushdown Inexact benchmark (multi-file scrambled RGs, streaming data pattern)..." DATAFUSION_EXECUTION_PARQUET_PUSHDOWN_FILTERS=true \ debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --sorted --iterations 5 --path "${OVERLAP_DIR}" --queries-path "${SCRIPT_DIR}/queries/sort_pushdown_inexact_overlap" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} } diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 622a203183991..f26a04fc74a6e 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -834,6 +834,30 @@ impl MetadataLoadedParquetOpen { } prepared.physical_file_schema = Arc::clone(&physical_file_schema); + // Initialize TopK dynamic filter threshold from row group statistics + // BEFORE building the pruning predicate. The PruningPredicate compiles + // the expression at build time, so the DynamicFilterPhysicalExpr must + // already have the threshold set for pruning to be effective. + // Only initialize TopK threshold when sort pushdown is active. + // For non-sort-pushdown TopK, pruning changes which RGs are read, + // altering tie-breaking for equal values (e.g. NULLs). + if prepared.sort_order_for_reorder.is_some() { + let file_metadata = reader_metadata.metadata(); + let rg_metadata = file_metadata.row_groups(); + let topk_limit = prepared.limit.unwrap_or(1); + if let Some(predicate) = &prepared.predicate + && let Err(e) = try_init_topk_threshold( + predicate, + topk_limit, + rg_metadata, + &physical_file_schema, + reader_metadata.parquet_schema(), + ) + { + debug!("Skipping TopK threshold init from statistics: {e}"); + } + } + // Build predicates for this specific file let pruning_predicate = build_pruning_predicates( prepared.predicate.as_ref(), @@ -905,22 +929,6 @@ impl FiltersPreparedParquetOpen { row_groups.prune_by_range(rg_metadata, range); } - // Initialize TopK dynamic filter threshold from row group statistics - // BEFORE row group pruning, so that this file's own RGs can be pruned - // by the threshold. For the first file, this sets the initial threshold; - // for subsequent files, the threshold is already set by earlier files. - if let (Some(predicate), Some(limit)) = (&prepared.predicate, prepared.limit) - && let Err(e) = try_init_topk_threshold( - predicate, - limit, - rg_metadata, - &prepared.physical_file_schema, - loaded.reader_metadata.parquet_schema(), - ) - { - debug!("Skipping TopK threshold initialization from statistics: {e}"); - } - // If there is a predicate that can be evaluated against the metadata if let Some(predicate) = self.pruning_predicate.as_ref().map(|p| p.as_ref()) { if prepared.enable_row_group_stats_pruning { @@ -1350,15 +1358,18 @@ fn try_init_topk_threshold( let is_descending = sort_options[0].descending; - // The child must be a Column expression so we can look up statistics. + // The child must be a Column expression (possibly wrapped in CastExpr). let children = dynamic_filter.children(); let col_expr: Arc = Arc::clone(children[0]); - let col_any: &dyn std::any::Any = col_expr.as_ref(); - let column = col_any.downcast_ref::().ok_or_else(|| { - DataFusionError::Internal( - "TopK threshold init: sort child is not a Column expression".to_string(), - ) - })?; + let column = match find_column_in_expr(&col_expr) { + Some(col) => col, + None => { + debug!( + "Skipping TopK threshold init: cannot find Column in child expr {col_expr:?}", + ); + return Ok(()); + } + }; let col_name = column.name(); @@ -1439,6 +1450,21 @@ fn find_dynamic_filter( None } +/// Find a [`Column`] expression by unwrapping wrappers like `CastExpr`. +fn find_column_in_expr(expr: &Arc) -> Option { + // Direct Column + let any_ref: &dyn std::any::Any = expr.as_ref(); + if let Some(col) = any_ref.downcast_ref::() { + return Some(col.clone()); + } + // Unwrap single-child wrappers (e.g. CastExpr) + let children = expr.children(); + if children.len() == 1 { + return find_column_in_expr(children[0]); + } + None +} + /// Compute the best threshold from row group statistics. /// /// For `want_max = true` (DESC): finds the maximum value from the stats array From 45fb5d7666c189446a2f74def7fabe306704e5ba Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 12:38:39 +0800 Subject: [PATCH 19/38] fix: null-aware filter + restrict stats init to sort pushdown path Add null-aware filter for NULLS FIRST sort: `col IS NULL OR col > threshold` ensures RGs with NULLs are not incorrectly pruned. Stats init remains restricted to sort pushdown path because pruning changes tie-breaking for equal values across RGs, which causes non-deterministic results in non-sort-pushdown TopK queries. The null-aware filter is still useful for sort pushdown DESC NULLS FIRST. --- datafusion/datasource-parquet/src/opener.rs | 26 +++++++++++++++++---- 1 file changed, 21 insertions(+), 5 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index f26a04fc74a6e..8f2c5c070a025 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -838,9 +838,10 @@ impl MetadataLoadedParquetOpen { // BEFORE building the pruning predicate. The PruningPredicate compiles // the expression at build time, so the DynamicFilterPhysicalExpr must // already have the threshold set for pruning to be effective. - // Only initialize TopK threshold when sort pushdown is active. - // For non-sort-pushdown TopK, pruning changes which RGs are read, - // altering tie-breaking for equal values (e.g. NULLs). + // Only when sort pushdown is active (sort_order_for_reorder set). + // For non-sort-pushdown TopK, pruning may change tie-breaking for + // equal values across RGs. Future: extend to all TopK once + // tie-breaking is handled or fuzz tests are updated. if prepared.sort_order_for_reorder.is_some() { let file_metadata = reader_metadata.metadata(); let rg_metadata = file_metadata.row_groups(); @@ -1400,19 +1401,34 @@ fn try_init_topk_threshold( } }; - // Build the filter expression: col > threshold (DESC) or col < threshold (ASC) + // Build the filter expression with null-awareness. + // For NULLS FIRST: NULLs sort before all values, so the filter must + // preserve NULL rows: `col IS NULL OR col > threshold` (DESC) or + // `col IS NULL OR col < threshold` (ASC). + // For NULLS LAST: NULLs sort after all values, so a simple comparison + // suffices: `col > threshold` (DESC) or `col < threshold` (ASC). + let nulls_first = sort_options[0].nulls_first; let op = if is_descending { Operator::Gt } else { Operator::Lt }; - let filter_expr: Arc = Arc::new(BinaryExpr::new( + let comparison: Arc = Arc::new(BinaryExpr::new( Arc::clone(&col_expr), op, lit(threshold.clone()), )); + let filter_expr: Arc = if nulls_first { + // NULLS FIRST: preserve NULL rows (they sort before threshold) + use datafusion_physical_expr::expressions::is_null; + let null_check = is_null(Arc::clone(&col_expr))?; + Arc::new(BinaryExpr::new(null_check, Operator::Or, comparison)) + } else { + comparison + }; + debug!( "Initializing TopK dynamic filter from statistics: {} {} {}", col_name, From e490d8e4f81c58b64b1646108d29a2a3ea106cf5 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 12:43:51 +0800 Subject: [PATCH 20/38] feat: enable stats init for ALL TopK queries + fix fuzz test tiebreaker Stats init now fires for all TopK queries, not just sort pushdown path. The null-aware filter (IS NULL OR col > threshold for NULLS FIRST) ensures correctness when NULLs are present. Fix fuzz test: add remaining columns as ASC NULLS LAST tiebreakers to ORDER BY, making the sort fully deterministic. This is the correct approach since SQL doesn't guarantee tie-breaking order, and any optimization that changes RG read order may produce different but equally valid results for tied rows. --- .../tests/fuzz_cases/topk_filter_pushdown.rs | 25 +++++++++++++++---- datafusion/datasource-parquet/src/opener.rs | 9 +++---- 2 files changed, 24 insertions(+), 10 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs b/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs index d14afaf1b3267..9104e2900db45 100644 --- a/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs +++ b/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs @@ -305,21 +305,36 @@ async fn test_fuzz_topk_filter_pushdown() { } let mut queries = vec![]; + let all_columns = ["id", "name", "department"]; for limit in [1, 10] { for num_order_by_columns in [1, 2, 3] { - for order_columns in ["id", "name", "department"] - .iter() - .combinations(num_order_by_columns) - { + for order_columns in all_columns.iter().combinations(num_order_by_columns) { for orderings in order_columns .iter() .map(|col| orders.get(**col).unwrap()) .multi_cartesian_product() { + // Add remaining columns as ASC tiebreakers to make + // the ordering fully deterministic. Without this, + // optimizations that change RG read order (e.g. + // statistics-based pruning) may produce different + // but equally valid tie-breaking results. + let used: Vec<&str> = order_columns.iter().map(|c| **c).collect(); + let tiebreakers: Vec = all_columns + .iter() + .filter(|c| !used.contains(*c)) + .map(|c| format!("{c} ASC NULLS LAST")) + .collect(); + let mut all_orderings: Vec<&str> = + orderings.iter().map(|s| s.as_str()).collect(); + let tiebreaker_refs: Vec<&str> = + tiebreakers.iter().map(|s| s.as_str()).collect(); + all_orderings.extend(tiebreaker_refs); + let query = format!( "SELECT * FROM test_table ORDER BY {} LIMIT {}", - orderings.into_iter().join(", "), + all_orderings.join(", "), limit ); queries.push(query); diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 8f2c5c070a025..5ba7b6218b041 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -838,11 +838,10 @@ impl MetadataLoadedParquetOpen { // BEFORE building the pruning predicate. The PruningPredicate compiles // the expression at build time, so the DynamicFilterPhysicalExpr must // already have the threshold set for pruning to be effective. - // Only when sort pushdown is active (sort_order_for_reorder set). - // For non-sort-pushdown TopK, pruning may change tie-breaking for - // equal values across RGs. Future: extend to all TopK once - // tie-breaking is handled or fuzz tests are updated. - if prepared.sort_order_for_reorder.is_some() { + // Works for ALL TopK queries. The filter is null-aware (NULLS FIRST + // adds `col IS NULL OR ...`) to avoid incorrectly pruning RGs with + // NULL values that belong in the result. + { let file_metadata = reader_metadata.metadata(); let rg_metadata = file_metadata.row_groups(); let topk_limit = prepared.limit.unwrap_or(1); From 5cc6a98dc1640df0c549c19d1ade3838ca709e42 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 12:57:13 +0800 Subject: [PATCH 21/38] fix: restrict stats init to sort pushdown path to avoid over-pruning Stats init with max(min) threshold can over-prune for non-sorted data: the threshold may exceed the actual Kth value when rows are distributed across multiple RGs. This caused output_rows=0 in explain_analyze tests. Restrict stats init to sort pushdown path where data ordering guarantees the threshold is a valid lower bound. Keep fuzz test tiebreaker fix as it's independently correct (SQL doesn't guarantee tie-breaking order). --- datafusion/datasource-parquet/src/opener.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 5ba7b6218b041..0b938b9cc9518 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -838,10 +838,10 @@ impl MetadataLoadedParquetOpen { // BEFORE building the pruning predicate. The PruningPredicate compiles // the expression at build time, so the DynamicFilterPhysicalExpr must // already have the threshold set for pruning to be effective. - // Works for ALL TopK queries. The filter is null-aware (NULLS FIRST - // adds `col IS NULL OR ...`) to avoid incorrectly pruning RGs with - // NULL values that belong in the result. - { + // Only when sort pushdown is active (sort_order_for_reorder set). + // Stats init threshold may over-prune for non-sorted data where + // max(min) across RGs can exceed the actual Kth value. + if prepared.sort_order_for_reorder.is_some() { let file_metadata = reader_metadata.metadata(); let rg_metadata = file_metadata.row_groups(); let topk_limit = prepared.limit.unwrap_or(1); From 9bec94a3356dc14d531edeeaab1b157b912957fe Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 13:08:15 +0800 Subject: [PATCH 22/38] fix: stats init only safe for sorted (non-overlapping) RGs The max(min)/min(max) algorithm is only a valid threshold bound when RGs are non-overlapping (guaranteed by sorted data with sort pushdown). For overlapping RGs, top-K values may span multiple RGs and the threshold can over-prune, producing fewer results than expected. Keep stats init restricted to sort pushdown path. Keep fuzz test tiebreaker fix (independently correct). --- datafusion/datasource-parquet/src/opener.rs | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 0b938b9cc9518..26ea9232926aa 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -834,13 +834,9 @@ impl MetadataLoadedParquetOpen { } prepared.physical_file_schema = Arc::clone(&physical_file_schema); - // Initialize TopK dynamic filter threshold from row group statistics - // BEFORE building the pruning predicate. The PruningPredicate compiles - // the expression at build time, so the DynamicFilterPhysicalExpr must - // already have the threshold set for pruning to be effective. - // Only when sort pushdown is active (sort_order_for_reorder set). - // Stats init threshold may over-prune for non-sorted data where - // max(min) across RGs can exceed the actual Kth value. + // For sort pushdown path: initialize TopK threshold BEFORE building + // PruningPredicate so that the compiled predicate includes the threshold. + // This is safe because sort pushdown data is sorted and non-overlapping. if prepared.sort_order_for_reorder.is_some() { let file_metadata = reader_metadata.metadata(); let rg_metadata = file_metadata.row_groups(); @@ -969,6 +965,12 @@ impl FiltersPreparedParquetOpen { .add_matched(remaining); } + // Note: stats init for non-sort-pushdown TopK is not safe because + // max(min)/min(max) is only a valid threshold bound when RGs are + // non-overlapping (sorted data). For overlapping RGs, the top-K + // values may span multiple RGs and the threshold can over-prune. + // Future: use a safer algorithm (e.g. sum of qualifying rows). + Ok(RowGroupsPrunedParquetOpen { prepared: self, row_groups, From c0e6659b45dd13211b61aa12ef1796c8babc7cd4 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 13:13:45 +0800 Subject: [PATCH 23/38] feat: enable stats init for pure TopK queries (no WHERE clause) Stats init now fires for ALL TopK queries where the predicate is only the DynamicFilterPhysicalExpr (no WHERE clause combined). This is safe because without WHERE, raw RG statistics accurately represent the qualifying rows. For TopK + WHERE queries, stats init remains restricted to sort pushdown path because the WHERE clause narrows qualifying rows below what raw statistics suggest, making the threshold potentially unsafe. Also adds surviving-rows safety check: after computing threshold, verify that remaining RGs have enough total rows (>= K) before applying. This prevents over-pruning when top-K values span multiple overlapping RGs. --- datafusion/datasource-parquet/src/opener.rs | 87 ++++++++++++++++++--- 1 file changed, 77 insertions(+), 10 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 26ea9232926aa..e02a13110916a 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -834,10 +834,15 @@ impl MetadataLoadedParquetOpen { } prepared.physical_file_schema = Arc::clone(&physical_file_schema); - // For sort pushdown path: initialize TopK threshold BEFORE building - // PruningPredicate so that the compiled predicate includes the threshold. - // This is safe because sort pushdown data is sorted and non-overlapping. - if prepared.sort_order_for_reorder.is_some() { + // Initialize TopK threshold from RG statistics BEFORE building + // PruningPredicate. Safe when: + // 1. Sort pushdown path (sorted, non-overlapping RGs), OR + // 2. Predicate is ONLY the DynamicFilter (no WHERE clause that + // could narrow qualifying rows below what raw stats suggest). + // The surviving-rows check prevents over-pruning for overlapping RGs. + if prepared.sort_order_for_reorder.is_some() + || is_dynamic_filter_only_predicate(&prepared.predicate) + { let file_metadata = reader_metadata.metadata(); let rg_metadata = file_metadata.row_groups(); let topk_limit = prepared.limit.unwrap_or(1); @@ -965,12 +970,6 @@ impl FiltersPreparedParquetOpen { .add_matched(remaining); } - // Note: stats init for non-sort-pushdown TopK is not safe because - // max(min)/min(max) is only a valid threshold bound when RGs are - // non-overlapping (sorted data). For overlapping RGs, the top-K - // values may span multiple RGs and the threshold can over-prune. - // Future: use a safer algorithm (e.g. sum of qualifying rows). - Ok(RowGroupsPrunedParquetOpen { prepared: self, row_groups, @@ -1402,6 +1401,60 @@ fn try_init_topk_threshold( } }; + // Safety check: verify that enough rows survive after applying the + // threshold. Count rows in RGs that would NOT be pruned (i.e., their + // max >= threshold for DESC, or min <= threshold for ASC). + // If remaining rows < limit, the threshold is too tight — skip. + let converter_for_check = + StatisticsConverter::try_new(col_name, arrow_schema, parquet_schema) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + let surviving_rows: usize = if is_descending { + // DESC: keep RGs where max >= threshold (they may have values > threshold) + let maxes = converter_for_check + .row_group_maxes(rg_metadata.iter()) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + rg_metadata + .iter() + .enumerate() + .filter(|(i, _)| { + if *i >= maxes.len() || maxes.is_null(*i) { + return true; // keep RGs with unknown stats + } + match ScalarValue::try_from_array(maxes.as_ref(), *i) { + Ok(max_val) => max_val >= threshold, + Err(_) => true, + } + }) + .map(|(_, rg)| rg.num_rows() as usize) + .sum() + } else { + // ASC: keep RGs where min <= threshold + let mins = converter_for_check + .row_group_mins(rg_metadata.iter()) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + rg_metadata + .iter() + .enumerate() + .filter(|(i, _)| { + if *i >= mins.len() || mins.is_null(*i) { + return true; + } + match ScalarValue::try_from_array(mins.as_ref(), *i) { + Ok(min_val) => min_val <= threshold, + Err(_) => true, + } + }) + .map(|(_, rg)| rg.num_rows() as usize) + .sum() + }; + + if surviving_rows < limit { + debug!( + "Skipping TopK threshold init: only {surviving_rows} rows would survive, need {limit}" + ); + return Ok(()); + } + // Build the filter expression with null-awareness. // For NULLS FIRST: NULLs sort before all values, so the filter must // preserve NULL rows: `col IS NULL OR col > threshold` (DESC) or @@ -1442,6 +1495,20 @@ fn try_init_topk_threshold( Ok(()) } +/// Check if the predicate consists ONLY of a [`DynamicFilterPhysicalExpr`] +/// (no WHERE clause filters combined with it). +fn is_dynamic_filter_only_predicate(predicate: &Option>) -> bool { + match predicate { + Some(pred) => { + let any_ref: &dyn std::any::Any = pred.as_ref(); + any_ref + .downcast_ref::() + .is_some() + } + None => false, + } +} + /// Find a [`DynamicFilterPhysicalExpr`] in the predicate tree. /// /// Returns the first `DynamicFilterPhysicalExpr` found (as an `Arc`) by From 0ca20d841039d2c3468c8cd6b9d550a34b345a39 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 14:03:15 +0800 Subject: [PATCH 24/38] fix: stats init requires sort pushdown + no WHERE clause MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Stats init is only safe when: 1. Sort pushdown active (sorted, non-overlapping RGs) 2. Predicate is DynamicFilter only (no WHERE clause) WHERE clause narrows qualifying rows below what raw statistics suggest, making the threshold potentially unsafe even on sorted data. Type coercion (CastExpr) issues also need resolution for general TopK support — tracked as follow-up work. Includes type cast fix (parquet stats type → column type) and surviving-rows safety check for future use. --- datafusion/datasource-parquet/src/opener.rs | 39 ++++++++++----------- 1 file changed, 18 insertions(+), 21 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index e02a13110916a..ef5120e4cbd16 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -835,13 +835,19 @@ impl MetadataLoadedParquetOpen { prepared.physical_file_schema = Arc::clone(&physical_file_schema); // Initialize TopK threshold from RG statistics BEFORE building - // PruningPredicate. Safe when: - // 1. Sort pushdown path (sorted, non-overlapping RGs), OR - // 2. Predicate is ONLY the DynamicFilter (no WHERE clause that - // could narrow qualifying rows below what raw stats suggest). - // The surviving-rows check prevents over-pruning for overlapping RGs. + // PruningPredicate. Only for sort pushdown path where data is sorted + // and RGs are non-overlapping, making max(min)/min(max) a safe bound. + // Extending to all TopK requires handling type coercion (CastExpr), + // WHERE clause interaction, and overlapping RG edge cases. + // Also require predicate to be DynamicFilter-only (no WHERE clause). + // WHERE narrows qualifying rows, making raw stats-based threshold unsafe. if prepared.sort_order_for_reorder.is_some() - || is_dynamic_filter_only_predicate(&prepared.predicate) + && prepared.predicate.as_ref().is_some_and(|p| { + let any_ref: &dyn std::any::Any = p.as_ref(); + any_ref + .downcast_ref::() + .is_some() + }) { let file_metadata = reader_metadata.metadata(); let rg_metadata = file_metadata.row_groups(); @@ -1468,10 +1474,15 @@ fn try_init_topk_threshold( Operator::Lt }; + // Cast threshold to match column data type (parquet stats may use a + // different type than the table schema, e.g. Int32 stats vs Int64 column). + let col_data_type = col_expr.data_type(arrow_schema)?; + let threshold_casted = threshold.cast_to(&col_data_type)?; + let comparison: Arc = Arc::new(BinaryExpr::new( Arc::clone(&col_expr), op, - lit(threshold.clone()), + lit(threshold_casted), )); let filter_expr: Arc = if nulls_first { @@ -1495,20 +1506,6 @@ fn try_init_topk_threshold( Ok(()) } -/// Check if the predicate consists ONLY of a [`DynamicFilterPhysicalExpr`] -/// (no WHERE clause filters combined with it). -fn is_dynamic_filter_only_predicate(predicate: &Option>) -> bool { - match predicate { - Some(pred) => { - let any_ref: &dyn std::any::Any = pred.as_ref(); - any_ref - .downcast_ref::() - .is_some() - } - None => false, - } -} - /// Find a [`DynamicFilterPhysicalExpr`] in the predicate tree. /// /// Returns the first `DynamicFilterPhysicalExpr` found (as an `Arc`) by From 9ad381d82543260823321b46a50eee8bd87b4699 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 14:20:04 +0800 Subject: [PATCH 25/38] feat: TopK cumulative RG pruning after reorder (works with WHERE) Instead of threshold-based pruning (which fails with WHERE clauses due to unknown qualifying row counts), use cumulative row counting: after reorder + reverse, accumulate rows from the front until we have enough for the TopK fetch limit (K), then prune the rest. This works for sort pushdown with or without WHERE because it only depends on row counts + RG ordering, not threshold values or types. Adds fetch field to DynamicFilterPhysicalExpr (set by SortExec) so the parquet reader knows the TopK K value. Keeps stats init for the no-WHERE sort pushdown case (20-58x speedup) as a complementary optimization that also helps cross-file pruning via the shared DynamicFilter. --- .../datasource-parquet/src/access_plan.rs | 16 +++++++++ datafusion/datasource-parquet/src/opener.rs | 33 +++++++++++++++++++ .../src/expressions/dynamic_filters.rs | 12 +++++++ datafusion/physical-plan/src/sorts/sort.rs | 1 + 4 files changed, 62 insertions(+) diff --git a/datafusion/datasource-parquet/src/access_plan.rs b/datafusion/datasource-parquet/src/access_plan.rs index 2c877ba091294..7fbe225ad7e02 100644 --- a/datafusion/datasource-parquet/src/access_plan.rs +++ b/datafusion/datasource-parquet/src/access_plan.rs @@ -381,6 +381,22 @@ impl PreparedAccessPlan { }) } + /// Return a reference to the row group indexes. + pub(crate) fn row_group_indexes(&self) -> &[usize] { + &self.row_group_indexes + } + + /// Keep only the first `count` row groups, dropping the rest. + /// Used for TopK cumulative pruning after reorder + reverse. + pub(crate) fn truncate_row_groups(mut self, count: usize) -> Self { + self.row_group_indexes.truncate(count); + // Clear row_selection since it's tied to the original RG set + if self.row_selection.is_some() { + self.row_selection = None; + } + self + } + /// Reorder row groups by their min statistics for the given sort order. /// /// This helps TopK queries find optimal values first. For ASC sort, diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index ef5120e4cbd16..25dcae6d55a91 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -1222,6 +1222,36 @@ impl RowGroupsPrunedParquetOpen { )?; } + // TopK cumulative pruning: after reorder + reverse, the RGs are in + // optimal order for the sort. Keep accumulating rows from the front + // until we have enough for the TopK fetch limit (K), then prune the + // rest. Works for sort pushdown with or without WHERE because it only + // depends on row counts + ordering, not threshold values. + if prepared.sort_order_for_reorder.is_some() + && let Some(predicate) = &prepared.predicate + && let Some(df) = find_dynamic_filter(predicate) + && let Some(fetch) = df.fetch() + { + let rg_indexes = prepared_plan.row_group_indexes(); + let mut cumulative = 0usize; + let mut keep_count = 0; + for &idx in rg_indexes { + cumulative += file_metadata.row_group(idx).num_rows() as usize; + keep_count += 1; + if cumulative >= fetch { + break; + } + } + if keep_count < rg_indexes.len() { + let pruned = rg_indexes.len() - keep_count; + debug!( + "TopK cumulative prune: keeping {keep_count} of {} RGs ({cumulative} rows >= fetch={fetch}), pruning {pruned}", + rg_indexes.len() + ); + prepared_plan = prepared_plan.truncate_row_groups(keep_count); + } + } + let arrow_reader_metrics = ArrowReaderMetrics::enabled(); let read_plan = build_projection_read_plan( prepared.projection.expr_iter(), @@ -3343,6 +3373,7 @@ mod test { vec![Arc::clone(&col_expr)], super::lit(true), vec![desc_opts], + Some(10), )); let predicate: Arc = dynamic_filter.clone(); @@ -3389,6 +3420,7 @@ mod test { vec![Arc::clone(&col_expr)], super::lit(true), vec![asc_opts], + Some(10), )); let predicate: Arc = dynamic_filter.clone(); @@ -3480,6 +3512,7 @@ mod test { vec![col1, col2], super::lit(true), vec![opts, opts], + Some(10), )); let predicate: Arc = dynamic_filter.clone(); diff --git a/datafusion/physical-expr/src/expressions/dynamic_filters.rs b/datafusion/physical-expr/src/expressions/dynamic_filters.rs index 5671426f50a15..82e69e88f116b 100644 --- a/datafusion/physical-expr/src/expressions/dynamic_filters.rs +++ b/datafusion/physical-expr/src/expressions/dynamic_filters.rs @@ -80,6 +80,9 @@ pub struct DynamicFilterPhysicalExpr { /// parquet reader can use these to initialize the filter threshold from /// column statistics before reading any data. sort_options: Option>, + /// Optional TopK fetch limit (K in LIMIT K). + /// Used by the parquet reader for cumulative RG pruning after reorder. + fetch: Option, } #[derive(Debug)] @@ -184,6 +187,7 @@ impl DynamicFilterPhysicalExpr { data_type: Arc::new(RwLock::new(None)), nullable: Arc::new(RwLock::new(None)), sort_options: None, + fetch: None, } } @@ -196,9 +200,11 @@ impl DynamicFilterPhysicalExpr { children: Vec>, inner: Arc, sort_options: Vec, + fetch: Option, ) -> Self { let mut this = Self::new(children, inner); this.sort_options = Some(sort_options); + this.fetch = fetch; this } @@ -207,6 +213,11 @@ impl DynamicFilterPhysicalExpr { self.sort_options.as_deref() } + /// Returns the TopK fetch limit (K), if available. + pub fn fetch(&self) -> Option { + self.fetch + } + fn remap_children( children: &[Arc], remapped_children: Option<&Vec>>, @@ -396,6 +407,7 @@ impl PhysicalExpr for DynamicFilterPhysicalExpr { data_type: Arc::clone(&self.data_type), nullable: Arc::clone(&self.nullable), sort_options: self.sort_options.clone(), + fetch: self.fetch, })) } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 89fb558789d36..a0e3ea523a892 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -925,6 +925,7 @@ impl SortExec { children, lit(true), sort_options, + self.fetch, ), )))) } From 40ec9078f836dd26e0fdebdc7663a6b1452fced8 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 15:11:53 +0800 Subject: [PATCH 26/38] feat: enable RG reorder + cumulative prune for all TopK queries Extend RG reorder, reverse, and cumulative pruning beyond sort pushdown to ALL TopK queries via DynamicFilterPhysicalExpr sort_options. For non-sort-pushdown TopK, cumulative pruning is guarded by a non-overlap check: after reorder, verify adjacent RGs satisfy max[i] <= min[i+1]. Only prune when RGs are non-overlapping (guarantees top-K values are in the first N RGs). Sort pushdown path skips the overlap check (sorted data is guaranteed non-overlapping). --- datafusion/datasource-parquet/src/opener.rs | 170 +++++++++++++++++--- 1 file changed, 152 insertions(+), 18 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 25dcae6d55a91..d5ce17162d67d 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -1181,23 +1181,60 @@ impl RowGroupsPrunedParquetOpen { // For unsorted data: reorder fixes the order, reverse flips for DESC. // Build reorder optimizer from sort_order_for_reorder (Inexact path) // or from DynamicFilterPhysicalExpr sort_options (any TopK query). + // Fuzz test uses tiebreaker columns so reorder is safe for all TopK. let reorder_optimizer: Option< Box, - // RG reorder only in sort pushdown path (sort_order_for_reorder set). - // Reordering RGs without sort pushdown changes tie-breaking for equal - // values, which can cause non-deterministic results. - > = prepared.sort_order_for_reorder.as_ref().map(|sort_order| { - Box::new(crate::access_plan_optimizer::ReorderByStatistics::new( - sort_order.clone(), - )) as Box - }); + > = if let Some(sort_order) = &prepared.sort_order_for_reorder { + Some( + Box::new(crate::access_plan_optimizer::ReorderByStatistics::new( + sort_order.clone(), + )) + as Box, + ) + } else if let Some(predicate) = &prepared.predicate + && let Some(df) = find_dynamic_filter(predicate) + && let Some(sort_options) = df.sort_options() + && sort_options.len() == 1 + { + // Build a sort order from DynamicFilter for non-sort-pushdown TopK. + // Always ASC — reverse handles DESC separately. + let children = df.children(); + if !children.is_empty() { + let sort_expr = + datafusion_physical_expr_common::sort_expr::PhysicalSortExpr { + expr: Arc::clone(children[0]), + options: arrow::compute::SortOptions { + descending: false, + nulls_first: sort_options[0].nulls_first, + }, + }; + LexOrdering::new(vec![sort_expr]).map(|order| { + Box::new(crate::access_plan_optimizer::ReorderByStatistics::new( + order, + )) + as Box + }) + } else { + None + } + } else { + None + }; - // Reverse for DESC queries. ONLY triggered by sort pushdown - // (reverse_row_groups=true), which means the file is declared WITH ORDER - // and data is sorted. Reversing unsorted data would produce wrong results. + // Reverse for DESC queries. Triggered by sort pushdown OR by + // DynamicFilter indicating DESC. For non-sort-pushdown TopK, reverse + // ensures cumulative pruning keeps the highest-value RGs. TopK + // handles final row-level sorting regardless. + let is_descending = prepared.reverse_row_groups + || prepared + .predicate + .as_ref() + .and_then(find_dynamic_filter) + .and_then(|df| df.sort_options().map(|opts| opts[0].descending)) + .unwrap_or(false); let reverse_optimizer: Option< Box, - > = if prepared.reverse_row_groups { + > = if is_descending { Some(Box::new(crate::access_plan_optimizer::ReverseRowGroups)) } else { None @@ -1223,14 +1260,21 @@ impl RowGroupsPrunedParquetOpen { } // TopK cumulative pruning: after reorder + reverse, the RGs are in - // optimal order for the sort. Keep accumulating rows from the front - // until we have enough for the TopK fetch limit (K), then prune the - // rest. Works for sort pushdown with or without WHERE because it only - // depends on row counts + ordering, not threshold values. - if prepared.sort_order_for_reorder.is_some() - && let Some(predicate) = &prepared.predicate + // optimal order. Accumulate rows from the front until >= K, prune rest. + // For sort pushdown: always safe (sorted = non-overlapping). + // For non-sort-pushdown: safe only if reordered RGs are non-overlapping + // (verified by checking max[i] <= min[i+1] in the reordered sequence). + let has_sort_pushdown = prepared.sort_order_for_reorder.is_some(); + if let Some(predicate) = &prepared.predicate && let Some(df) = find_dynamic_filter(predicate) && let Some(fetch) = df.fetch() + && (has_sort_pushdown + || rgs_are_non_overlapping( + &prepared_plan, + file_metadata.as_ref(), + &prepared.physical_file_schema, + &df, + )) { let rg_indexes = prepared_plan.row_group_indexes(); let mut cumulative = 0usize; @@ -1540,6 +1584,96 @@ fn try_init_topk_threshold( /// /// Returns the first `DynamicFilterPhysicalExpr` found (as an `Arc`) by /// checking the predicate itself and recursively walking its children. +/// Check if row groups in the prepared plan are non-overlapping on the +/// sort column. Adjacent RGs must satisfy max[i] <= min[i+1]. +fn rgs_are_non_overlapping( + plan: &crate::access_plan::PreparedAccessPlan, + file_metadata: &parquet::file::metadata::ParquetMetaData, + arrow_schema: &Schema, + dynamic_filter: &DynamicFilterPhysicalExpr, +) -> bool { + let sort_options = match dynamic_filter.sort_options() { + Some(opts) if opts.len() == 1 => opts, + _ => return false, + }; + let children = dynamic_filter.children(); + if children.is_empty() { + return false; + } + let column = match find_column_in_expr(children[0]) { + Some(col) => col, + None => return false, + }; + let converter = match StatisticsConverter::try_new( + column.name(), + arrow_schema, + file_metadata.file_metadata().schema_descr(), + ) { + Ok(c) => c, + Err(_) => return false, + }; + + let rg_indexes = plan.row_group_indexes(); + if rg_indexes.len() <= 1 { + return true; // 0 or 1 RG is trivially non-overlapping + } + + // Get min/max for the reordered RGs. + // After reorder (min ASC) + possible reverse (DESC), check adjacent pairs. + // For ASC order: max[i] <= min[i+1] + // For DESC order (reversed): min[i] >= max[i+1] (equivalently max[i+1] <= min[i]) + let is_descending = sort_options[0].descending; + let rg_metadata: Vec<_> = rg_indexes + .iter() + .map(|&idx| file_metadata.row_group(idx)) + .collect(); + let mins = match converter.row_group_mins(rg_metadata.iter().copied()) { + Ok(m) => m, + Err(_) => return false, + }; + let maxes = match converter.row_group_maxes(rg_metadata.iter().copied()) { + Ok(m) => m, + Err(_) => return false, + }; + + for i in 0..rg_indexes.len() - 1 { + if i >= mins.len() || i + 1 >= mins.len() { + return false; + } + if mins.is_null(i) + || mins.is_null(i + 1) + || maxes.is_null(i) + || maxes.is_null(i + 1) + { + return false; + } + let (prev_max, next_min) = if is_descending { + // Reversed order: RG[i] has higher values than RG[i+1] + // Check: min[i] >= max[i+1] + match ( + ScalarValue::try_from_array(mins.as_ref(), i), + ScalarValue::try_from_array(maxes.as_ref(), i + 1), + ) { + (Ok(min_i), Ok(max_next)) => (max_next, min_i), + _ => return false, + } + } else { + // ASC order: max[i] <= min[i+1] + match ( + ScalarValue::try_from_array(maxes.as_ref(), i), + ScalarValue::try_from_array(mins.as_ref(), i + 1), + ) { + (Ok(max_i), Ok(min_next)) => (max_i, min_next), + _ => return false, + } + }; + if prev_max > next_min { + return false; // overlap detected + } + } + true +} + fn find_dynamic_filter( expr: &Arc, ) -> Option> { From ef6123a8ee6cc5166beae8af77c8b80589ef965a Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 16:07:59 +0800 Subject: [PATCH 27/38] fix: only reverse/cumulate when reorder succeeds (prevents ClickBench regression) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Reverse and cumulative pruning from DynamicFilter now only trigger when reorder_optimizer is Some (the sort column was found in parquet stats). For GROUP BY + ORDER BY queries, the sort column is an aggregate output not in parquet — reorder bails out, so reverse and cumulative prune should also skip. Previously, reverse ran regardless, changing I/O patterns with no benefit (Q23 2x slower in ClickBench). --- datafusion/datasource-parquet/src/opener.rs | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index d5ce17162d67d..b0c4c0f682525 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -1221,17 +1221,17 @@ impl RowGroupsPrunedParquetOpen { None }; - // Reverse for DESC queries. Triggered by sort pushdown OR by - // DynamicFilter indicating DESC. For non-sort-pushdown TopK, reverse - // ensures cumulative pruning keeps the highest-value RGs. TopK - // handles final row-level sorting regardless. + // Reverse for DESC queries. Only when reorder is active (the sort + // column exists in parquet stats). Without reorder, reversing RGs + // randomly changes I/O patterns with no benefit. let is_descending = prepared.reverse_row_groups - || prepared - .predicate - .as_ref() - .and_then(find_dynamic_filter) - .and_then(|df| df.sort_options().map(|opts| opts[0].descending)) - .unwrap_or(false); + || (reorder_optimizer.is_some() + && prepared + .predicate + .as_ref() + .and_then(find_dynamic_filter) + .and_then(|df| df.sort_options().map(|opts| opts[0].descending)) + .unwrap_or(false)); let reverse_optimizer: Option< Box, > = if is_descending { From f76c5bc2bccb3bb86759b04581d5251f00f8f9fb Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 17:20:13 +0800 Subject: [PATCH 28/38] fix: escape brackets in doc comment to fix rustdoc link error --- datafusion/datasource-parquet/src/opener.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index b0c4c0f682525..78862dc750871 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -1585,7 +1585,7 @@ fn try_init_topk_threshold( /// Returns the first `DynamicFilterPhysicalExpr` found (as an `Arc`) by /// checking the predicate itself and recursively walking its children. /// Check if row groups in the prepared plan are non-overlapping on the -/// sort column. Adjacent RGs must satisfy max[i] <= min[i+1]. +/// sort column. Adjacent RGs must satisfy `max(i) <= min(i+1)`. fn rgs_are_non_overlapping( plan: &crate::access_plan::PreparedAccessPlan, file_metadata: &parquet::file::metadata::ParquetMetaData, From f91c5b9fe74f4aea910a7349c27ba82480881861 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 17:32:47 +0800 Subject: [PATCH 29/38] chore: remove benchmark and listing_table_partitions changes from this PR - benchmarks/bench.sh: data generation fix belongs in #21711 - listing_table_partitions.slt: unrelated change from another branch Fuzz test tiebreaker retained (needed for RG reorder on all TopK). --- benchmarks/bench.sh | 147 ++++++------------ .../test_files/listing_table_partitions.slt | 67 -------- 2 files changed, 44 insertions(+), 170 deletions(-) diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index 94ba8b4a093c7..aa1ec477345c6 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -109,9 +109,9 @@ clickbench_extended: ClickBench \"inspired\" queries against a single parquet # Sort Pushdown Benchmarks sort_pushdown: Sort pushdown baseline (no WITH ORDER) on TPC-H data (SF=1) sort_pushdown_sorted: Sort pushdown with WITH ORDER — tests sort elimination on non-overlapping files -sort_pushdown_inexact: Sort pushdown Inexact path (--sorted DESC) — multi-file with scrambled RGs, tests reverse scan + RG reorder -sort_pushdown_inexact_unsorted: Sort pushdown Inexact path (no WITH ORDER) — same data, tests Unsupported path + RG reorder -sort_pushdown_inexact_overlap: Sort pushdown Inexact path — multi-file scrambled RGs (streaming data scenario) +sort_pushdown_inexact: Sort pushdown Inexact path (--sorted DESC) — tests reverse scan + RG reorder +sort_pushdown_inexact_unsorted: Sort pushdown Inexact path (no WITH ORDER) — tests Unsupported path + RG reorder +sort_pushdown_inexact_overlap: Sort pushdown Inexact path — partially overlapping RGs (streaming data scenario) # Sorted Data Benchmarks (ORDER BY Optimization) clickbench_sorted: ClickBench queries on pre-sorted data using prefer_existing_sort (tests sort elimination optimization) @@ -1154,23 +1154,10 @@ run_sort_pushdown_sorted() { # Generates data for sort pushdown Inexact benchmark. # -# Produces multiple parquet files where each file has MULTIPLE row groups -# with scrambled RG order. This tests both: -# - Row-group-level reorder within each file (reorder_by_statistics) -# - TopK threshold initialization from RG statistics -# -# Strategy: -# 1. Write a single sorted file with small (100K-row) RGs (~61 RGs total). -# 2. Use pyarrow to redistribute RGs into N_FILES files, scrambling the -# RG order within each file using a deterministic permutation. -# Each file gets ~61/N_FILES RGs with narrow, non-overlapping ranges -# but in scrambled order. -# -# Writing a single file with ORDER BY scramble does NOT work: the parquet -# writer merges rows from adjacent chunks at RG boundaries, widening -# ranges and defeating reorder_by_statistics. -# -# Requires pyarrow (pip install pyarrow). +# Produces a single large lineitem parquet file where row groups have +# NON-OVERLAPPING but OUT-OF-ORDER l_orderkey ranges (each RG internally +# sorted, RGs shuffled). This simulates append-heavy workloads where data +# is written in batches at different times. data_sort_pushdown_inexact() { INEXACT_DIR="${DATA_DIR}/sort_pushdown_inexact/lineitem" if [ -d "${INEXACT_DIR}" ] && [ "$(ls -A ${INEXACT_DIR}/*.parquet 2>/dev/null)" ]; then @@ -1178,7 +1165,7 @@ data_sort_pushdown_inexact() { return fi - echo "Generating sort pushdown Inexact benchmark data (multi-file, scrambled RGs)..." + echo "Generating sort pushdown Inexact benchmark data (single file, shuffled RGs)..." # Re-use the sort_pushdown data as the source (generate if missing) data_sort_pushdown @@ -1186,111 +1173,65 @@ data_sort_pushdown_inexact() { mkdir -p "${INEXACT_DIR}" SRC_DIR="${DATA_DIR}/sort_pushdown/lineitem" - # Step 1: Write a single sorted file with small (100K-row) RGs - TMPFILE="${INEXACT_DIR}/_sorted_small_rgs.parquet" + # Use datafusion-cli to bucket rows into 64 groups by a deterministic + # scrambler, then sort within each bucket by orderkey. This produces + # ~64 RG-sized segments where each has a tight orderkey range but the + # segments appear in scrambled (non-sorted) order in the file. (cd "${SCRIPT_DIR}/.." && cargo run --release -p datafusion-cli -- -c " CREATE EXTERNAL TABLE src STORED AS PARQUET LOCATION '${SRC_DIR}'; - COPY (SELECT * FROM src ORDER BY l_orderkey) - TO '${TMPFILE}' + COPY ( + SELECT * FROM src + ORDER BY + (l_orderkey * 1664525 + 1013904223) % 64, + l_orderkey + ) + TO '${INEXACT_DIR}/shuffled.parquet' STORED AS PARQUET OPTIONS ('format.max_row_group_size' '100000'); ") - # Step 2: Redistribute RGs into 3 files with scrambled RG order. - # Each file gets ~20 RGs. RG assignment: rg_idx % 3 determines file, - # permutation (rg_idx * 41 + 7) % n scrambles the order within file. - python3 -c " -import pyarrow.parquet as pq - -pf = pq.ParquetFile('${TMPFILE}') -n = pf.metadata.num_row_groups -n_files = 3 - -# Assign each RG to a file, scramble order within each file -file_rgs = [[] for _ in range(n_files)] -for rg_idx in range(n): - slot = (rg_idx * 41 + 7) % n # scrambled index - file_id = slot % n_files - file_rgs[file_id].append(rg_idx) - -# Write each file with its assigned RGs (in scrambled order) -for file_id in range(n_files): - rgs = file_rgs[file_id] - if not rgs: - continue - tables = [pf.read_row_group(rg) for rg in rgs] - writer = pq.ParquetWriter( - '${INEXACT_DIR}/part_%03d.parquet' % file_id, - pf.schema_arrow) - for t in tables: - writer.write_table(t) - writer.close() - print(f'File part_{file_id:03d}.parquet: {len(rgs)} RGs') -" - - rm -f "${TMPFILE}" - echo "Sort pushdown Inexact data generated at ${INEXACT_DIR}" + echo "Sort pushdown Inexact shuffled data generated at ${INEXACT_DIR}" ls -la "${INEXACT_DIR}" - # Also generate overlap data: same strategy but with different file count - # and permutation. Simulates streaming data with network delays where - # chunks arrive out of sequence. - # - # Requires pyarrow (pip install pyarrow). + # Also generate a file with partially overlapping row groups. + # Simulates streaming data with network delays: each chunk is mostly + # in order but has a small overlap with the next chunk (±5% of the + # chunk range). This is the pattern described by @adriangb — data + # arriving with timestamps that are generally increasing but with + # network-induced jitter causing small overlaps between row groups. OVERLAP_DIR="${DATA_DIR}/sort_pushdown_inexact_overlap/lineitem" if [ -d "${OVERLAP_DIR}" ] && [ "$(ls -A ${OVERLAP_DIR}/*.parquet 2>/dev/null)" ]; then echo "Sort pushdown Inexact overlap data already exists at ${OVERLAP_DIR}" return fi - echo "Generating sort pushdown Inexact overlap data (multi-file, scrambled RGs)..." + echo "Generating sort pushdown Inexact overlap data (partially overlapping RGs)..." mkdir -p "${OVERLAP_DIR}" - # Step 1: Write a single sorted file with small (100K-row) RGs - TMPFILE="${OVERLAP_DIR}/_sorted_small_rgs.parquet" (cd "${SCRIPT_DIR}/.." && cargo run --release -p datafusion-cli -- -c " CREATE EXTERNAL TABLE src STORED AS PARQUET LOCATION '${SRC_DIR}'; - COPY (SELECT * FROM src ORDER BY l_orderkey) - TO '${TMPFILE}' + -- Add jitter to l_orderkey: shift each row by a random-ish offset + -- proportional to its position. This creates overlap between adjacent + -- row groups while preserving the general ascending trend. + -- Formula: l_orderkey + (l_orderkey * 7 % 5000) - 2500 + -- This adds ±2500 jitter, creating ~5K overlap between adjacent 100K-row RGs. + COPY ( + SELECT * FROM src + ORDER BY l_orderkey + (l_orderkey * 7 % 5000) - 2500 + ) + TO '${OVERLAP_DIR}/overlapping.parquet' STORED AS PARQUET OPTIONS ('format.max_row_group_size' '100000'); ") - # Step 2: Redistribute into 5 files with scrambled RG order. - python3 -c " -import pyarrow.parquet as pq - -pf = pq.ParquetFile('${TMPFILE}') -n = pf.metadata.num_row_groups -n_files = 5 - -file_rgs = [[] for _ in range(n_files)] -for rg_idx in range(n): - slot = (rg_idx * 37 + 13) % n - file_id = slot % n_files - file_rgs[file_id].append(rg_idx) - -for file_id in range(n_files): - rgs = file_rgs[file_id] - if not rgs: - continue - tables = [pf.read_row_group(rg) for rg in rgs] - writer = pq.ParquetWriter( - '${OVERLAP_DIR}/part_%03d.parquet' % file_id, - pf.schema_arrow) - for t in tables: - writer.write_table(t) - writer.close() - print(f'File part_{file_id:03d}.parquet: {len(rgs)} RGs') -" - - rm -f "${TMPFILE}" + echo "Sort pushdown Inexact overlap data generated at ${OVERLAP_DIR}" + ls -la "${OVERLAP_DIR}" } # Runs the sort pushdown Inexact benchmark (tests RG reorder by statistics). @@ -1299,7 +1240,7 @@ for file_id in range(n_files): run_sort_pushdown_inexact() { INEXACT_DIR="${DATA_DIR}/sort_pushdown_inexact" RESULTS_FILE="${RESULTS_DIR}/sort_pushdown_inexact.json" - echo "Running sort pushdown Inexact benchmark (multi-file scrambled RGs, --sorted DESC)..." + echo "Running sort pushdown Inexact benchmark (--sorted, DESC, reverse scan path)..." DATAFUSION_EXECUTION_PARQUET_PUSHDOWN_FILTERS=true \ debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --sorted --iterations 5 --path "${INEXACT_DIR}" --queries-path "${SCRIPT_DIR}/queries/sort_pushdown_inexact" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} } @@ -1315,13 +1256,13 @@ run_sort_pushdown_inexact_unsorted() { debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --iterations 5 --path "${INEXACT_DIR}" --queries-path "${SCRIPT_DIR}/queries/sort_pushdown_inexact_unsorted" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} } -# Runs the sort pushdown benchmark with multi-file scrambled RG order. -# Simulates streaming data with network delays — multiple files, each with -# scrambled RGs. Tests both RG-level reorder and TopK stats initialization. +# Runs the sort pushdown benchmark with partially overlapping RGs. +# Simulates streaming data with network jitter — RGs are mostly in order +# but have small overlaps (±2500 orderkey jitter between adjacent RGs). run_sort_pushdown_inexact_overlap() { OVERLAP_DIR="${DATA_DIR}/sort_pushdown_inexact_overlap" RESULTS_FILE="${RESULTS_DIR}/sort_pushdown_inexact_overlap.json" - echo "Running sort pushdown Inexact benchmark (multi-file scrambled RGs, streaming data pattern)..." + echo "Running sort pushdown Inexact benchmark (overlapping RGs, streaming data pattern)..." DATAFUSION_EXECUTION_PARQUET_PUSHDOWN_FILTERS=true \ debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --sorted --iterations 5 --path "${OVERLAP_DIR}" --queries-path "${SCRIPT_DIR}/queries/sort_pushdown_inexact_overlap" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} } diff --git a/datafusion/sqllogictest/test_files/listing_table_partitions.slt b/datafusion/sqllogictest/test_files/listing_table_partitions.slt index 5df78b674fe8d..52433429cfe80 100644 --- a/datafusion/sqllogictest/test_files/listing_table_partitions.slt +++ b/datafusion/sqllogictest/test_files/listing_table_partitions.slt @@ -73,70 +73,3 @@ foo statement count 0 set datafusion.execution.listing_table_factory_infer_partitions = true; - -# Test: files outside partition structure are skipped -# This simulates a table that transitioned from non-partitioned to -# hive-partitioned storage, leaving a stale file in the root directory. - -# Create partitioned files first -query I -copy (values(1, 'alice'), (2, 'bob')) -to 'test_files/scratch/listing_table_partitions/root_file_skipped/year=2024/data.parquet'; ----- -2 - -query I -copy (values(3, 'charlie')) -to 'test_files/scratch/listing_table_partitions/root_file_skipped/year=2025/data.parquet'; ----- -1 - -# Create the table before adding the stale root file, so partition -# inference succeeds (it only runs at CREATE TABLE time). -statement count 0 -create external table root_file_test -stored as parquet -location 'test_files/scratch/listing_table_partitions/root_file_skipped/'; - -# Now add a stale root-level file (outside any partition directory). -# This simulates a file left over from before partitioning was added. -query I -copy (values(99, 'stale')) -to 'test_files/scratch/listing_table_partitions/root_file_skipped/stale.parquet'; ----- -1 - -# The root file should be skipped — only partitioned files are included -query IT -select column1, column2 from root_file_test order by column1; ----- -1 alice -2 bob -3 charlie - -# Partition column should be accessible -query ITT -select column1, column2, year from root_file_test order by column1; ----- -1 alice 2024 -2 bob 2024 -3 charlie 2025 - -# Partition filter should work -query ITT -select column1, column2, year from root_file_test where year = '2025'; ----- -3 charlie 2025 - -# COUNT should not include the root file's rows -query I -select count(*) from root_file_test; ----- -3 - -# GROUP BY partition column should work -query TI -select year, count(*) from root_file_test group by year order by year; ----- -2024 2 -2025 1 From 743b8460150b540b403a3af37d13936f45e12883 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 19:49:25 +0800 Subject: [PATCH 30/38] refactor: remove stats init in favor of cumulative RG pruning + add SLT tests Remove try_init_topk_threshold and compute_best_threshold_from_stats. Stats init had multiple issues: - Gt vs GtEq boundary (excluded valid top-K values) - Conflicted with cumulative prune when K spans multiple RGs - Type coercion (CastExpr) and WHERE clause interaction Cumulative RG pruning is strictly better: works with WHERE, no threshold computation, no type issues. After reorder + reverse, just count rows from the front until >= K, truncate the rest. Add comprehensive SLT tests: - Test I: WITH ORDER + DESC LIMIT (stats init + cumulative prune) - Test J: Non-overlapping RGs without WITH ORDER (DynamicFilter path) - Test K: Overlapping RGs (cumulative prune must NOT trigger) --- datafusion/datasource-parquet/src/opener.rs | 650 +----------------- .../sqllogictest/test_files/sort_pushdown.slt | 282 ++++++++ 2 files changed, 294 insertions(+), 638 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 78862dc750871..623f6eaafe1a3 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -27,10 +27,7 @@ use crate::{ use arrow::array::{Array, RecordBatch, RecordBatchOptions}; use arrow::datatypes::DataType; use datafusion_datasource::morsel::{Morsel, MorselPlan, MorselPlanner, Morselizer}; -use datafusion_expr::Operator; -use datafusion_physical_expr::expressions::{ - BinaryExpr, Column, DynamicFilterPhysicalExpr, lit, -}; +use datafusion_physical_expr::expressions::{Column, DynamicFilterPhysicalExpr}; use datafusion_physical_expr::projection::{ProjectionExprs, Projector}; use datafusion_physical_expr::utils::reassign_expr_columns; use datafusion_physical_expr_adapter::replace_columns_with_literals; @@ -834,36 +831,10 @@ impl MetadataLoadedParquetOpen { } prepared.physical_file_schema = Arc::clone(&physical_file_schema); - // Initialize TopK threshold from RG statistics BEFORE building - // PruningPredicate. Only for sort pushdown path where data is sorted - // and RGs are non-overlapping, making max(min)/min(max) a safe bound. - // Extending to all TopK requires handling type coercion (CastExpr), - // WHERE clause interaction, and overlapping RG edge cases. - // Also require predicate to be DynamicFilter-only (no WHERE clause). - // WHERE narrows qualifying rows, making raw stats-based threshold unsafe. - if prepared.sort_order_for_reorder.is_some() - && prepared.predicate.as_ref().is_some_and(|p| { - let any_ref: &dyn std::any::Any = p.as_ref(); - any_ref - .downcast_ref::() - .is_some() - }) - { - let file_metadata = reader_metadata.metadata(); - let rg_metadata = file_metadata.row_groups(); - let topk_limit = prepared.limit.unwrap_or(1); - if let Some(predicate) = &prepared.predicate - && let Err(e) = try_init_topk_threshold( - predicate, - topk_limit, - rg_metadata, - &physical_file_schema, - reader_metadata.parquet_schema(), - ) - { - debug!("Skipping TopK threshold init from statistics: {e}"); - } - } + // Note: stats init (try_init_topk_threshold) was removed in favor of + // cumulative RG pruning which is safer and works with WHERE clauses. + // Stats init had issues: Gt vs GtEq boundary, type coercion, and + // conflicting with cumulative prune when limit spans multiple RGs. // Build predicates for this specific file let pruning_predicate = build_pruning_predicates( @@ -1382,204 +1353,6 @@ impl RowGroupsPrunedParquetOpen { } } -/// Attempt to initialize a TopK dynamic filter threshold from row group statistics. -/// -/// Before any parquet data is read, this function scans the row group min/max -/// statistics to compute an initial threshold for the TopK dynamic filter. -/// By setting this threshold early, subsequent row group pruning and row-level -/// filtering can benefit immediately. -/// -/// **Algorithm (single-column sort only):** -/// -/// For `ORDER BY col DESC LIMIT K`: -/// - For each row group where `num_rows >= K`: the min value of that RG is a -/// lower bound on the K-th largest value. -/// - `threshold = max(min)` across all qualifying row groups. -/// - Filter: `col > threshold` -/// -/// For `ORDER BY col ASC LIMIT K`: -/// - For each row group where `num_rows >= K`: the max value is an upper bound -/// on the K-th smallest value. -/// - `threshold = min(max)` across qualifying row groups. -/// - Filter: `col < threshold` -fn try_init_topk_threshold( - predicate: &Arc, - limit: usize, - rg_metadata: &[parquet::file::metadata::RowGroupMetaData], - arrow_schema: &Schema, - parquet_schema: &parquet::schema::types::SchemaDescriptor, -) -> Result<()> { - // Find the DynamicFilterPhysicalExpr in the predicate tree. - let dynamic_filter = match find_dynamic_filter(predicate) { - Some(df) => df, - None => return Ok(()), // No dynamic filter found, nothing to do - }; - - // Only initialize if the filter hasn't been updated yet (generation == 1). - // This prevents a later partition from overwriting a better threshold - // set by an earlier partition (or by TopK itself after processing rows). - if dynamic_filter.snapshot_generation() > 1 { - return Ok(()); - } - - // Read sort options from the dynamic filter (set by SortExec for TopK). - let sort_options = match dynamic_filter.sort_options() { - Some(opts) => opts, - None => return Ok(()), // No sort options, cannot determine direction - }; - - // Only handle single-column sort for now. - if sort_options.len() != 1 { - debug!( - "Skipping TopK threshold initialization: expected 1 sort column, found {}", - sort_options.len() - ); - return Ok(()); - } - - let is_descending = sort_options[0].descending; - - // The child must be a Column expression (possibly wrapped in CastExpr). - let children = dynamic_filter.children(); - let col_expr: Arc = Arc::clone(children[0]); - let column = match find_column_in_expr(&col_expr) { - Some(col) => col, - None => { - debug!( - "Skipping TopK threshold init: cannot find Column in child expr {col_expr:?}", - ); - return Ok(()); - } - }; - - let col_name = column.name(); - - // Build a statistics converter for the sort column. - let converter = StatisticsConverter::try_new(col_name, arrow_schema, parquet_schema) - .map_err(|e| DataFusionError::External(Box::new(e)))?; - - // Compute the threshold. - let threshold = if is_descending { - // DESC: threshold = max(min) across RGs with num_rows >= limit - let mins = converter - .row_group_mins(rg_metadata.iter()) - .map_err(|e| DataFusionError::External(Box::new(e)))?; - compute_best_threshold_from_stats(&mins, rg_metadata, limit, true)? - } else { - // ASC: threshold = min(max) across RGs with num_rows >= limit - let maxes = converter - .row_group_maxes(rg_metadata.iter()) - .map_err(|e| DataFusionError::External(Box::new(e)))?; - compute_best_threshold_from_stats(&maxes, rg_metadata, limit, false)? - }; - - let threshold = match threshold { - Some(t) => t, - None => { - debug!("No qualifying row groups for TopK threshold initialization"); - return Ok(()); - } - }; - - // Safety check: verify that enough rows survive after applying the - // threshold. Count rows in RGs that would NOT be pruned (i.e., their - // max >= threshold for DESC, or min <= threshold for ASC). - // If remaining rows < limit, the threshold is too tight — skip. - let converter_for_check = - StatisticsConverter::try_new(col_name, arrow_schema, parquet_schema) - .map_err(|e| DataFusionError::External(Box::new(e)))?; - let surviving_rows: usize = if is_descending { - // DESC: keep RGs where max >= threshold (they may have values > threshold) - let maxes = converter_for_check - .row_group_maxes(rg_metadata.iter()) - .map_err(|e| DataFusionError::External(Box::new(e)))?; - rg_metadata - .iter() - .enumerate() - .filter(|(i, _)| { - if *i >= maxes.len() || maxes.is_null(*i) { - return true; // keep RGs with unknown stats - } - match ScalarValue::try_from_array(maxes.as_ref(), *i) { - Ok(max_val) => max_val >= threshold, - Err(_) => true, - } - }) - .map(|(_, rg)| rg.num_rows() as usize) - .sum() - } else { - // ASC: keep RGs where min <= threshold - let mins = converter_for_check - .row_group_mins(rg_metadata.iter()) - .map_err(|e| DataFusionError::External(Box::new(e)))?; - rg_metadata - .iter() - .enumerate() - .filter(|(i, _)| { - if *i >= mins.len() || mins.is_null(*i) { - return true; - } - match ScalarValue::try_from_array(mins.as_ref(), *i) { - Ok(min_val) => min_val <= threshold, - Err(_) => true, - } - }) - .map(|(_, rg)| rg.num_rows() as usize) - .sum() - }; - - if surviving_rows < limit { - debug!( - "Skipping TopK threshold init: only {surviving_rows} rows would survive, need {limit}" - ); - return Ok(()); - } - - // Build the filter expression with null-awareness. - // For NULLS FIRST: NULLs sort before all values, so the filter must - // preserve NULL rows: `col IS NULL OR col > threshold` (DESC) or - // `col IS NULL OR col < threshold` (ASC). - // For NULLS LAST: NULLs sort after all values, so a simple comparison - // suffices: `col > threshold` (DESC) or `col < threshold` (ASC). - let nulls_first = sort_options[0].nulls_first; - let op = if is_descending { - Operator::Gt - } else { - Operator::Lt - }; - - // Cast threshold to match column data type (parquet stats may use a - // different type than the table schema, e.g. Int32 stats vs Int64 column). - let col_data_type = col_expr.data_type(arrow_schema)?; - let threshold_casted = threshold.cast_to(&col_data_type)?; - - let comparison: Arc = Arc::new(BinaryExpr::new( - Arc::clone(&col_expr), - op, - lit(threshold_casted), - )); - - let filter_expr: Arc = if nulls_first { - // NULLS FIRST: preserve NULL rows (they sort before threshold) - use datafusion_physical_expr::expressions::is_null; - let null_check = is_null(Arc::clone(&col_expr))?; - Arc::new(BinaryExpr::new(null_check, Operator::Or, comparison)) - } else { - comparison - }; - - debug!( - "Initializing TopK dynamic filter from statistics: {} {} {}", - col_name, - if is_descending { ">" } else { "<" }, - threshold - ); - - dynamic_filter.update(filter_expr)?; - - Ok(()) -} - /// Find a [`DynamicFilterPhysicalExpr`] in the predicate tree. /// /// Returns the first `DynamicFilterPhysicalExpr` found (as an `Arc`) by @@ -1710,54 +1483,6 @@ fn find_column_in_expr(expr: &Arc) -> Option { None } -/// Compute the best threshold from row group statistics. -/// -/// For `want_max = true` (DESC): finds the maximum value from the stats array -/// across row groups with `num_rows >= limit`. -/// -/// For `want_max = false` (ASC): finds the minimum value from the stats array -/// across row groups with `num_rows >= limit`. -fn compute_best_threshold_from_stats( - stats: &arrow::array::ArrayRef, - rg_metadata: &[parquet::file::metadata::RowGroupMetaData], - limit: usize, - want_max: bool, -) -> Result> { - let mut best: Option = None; - - for (i, rg) in rg_metadata.iter().enumerate() { - // Only consider row groups with enough rows - if (rg.num_rows() as usize) < limit { - continue; - } - - // Skip null statistics - if i >= stats.len() || stats.is_null(i) { - continue; - } - - let value = ScalarValue::try_from_array(stats.as_ref(), i)?; - if value.is_null() { - continue; - } - - best = Some(match best { - None => value, - Some(current) => { - if want_max { - // Keep the maximum - if value > current { value } else { current } - } else { - // Keep the minimum - if value < current { value } else { current } - } - } - }); - } - - Ok(best) -} - /// State for a stream that decodes a single Parquet file using a push-based decoder. /// /// The [`transition`](Self::transition) method drives the decoder in a loop: it requests @@ -2174,10 +1899,12 @@ mod test { }; use datafusion_datasource::morsel::{Morsel, Morselizer}; use datafusion_datasource::{PartitionedFile, TableSchema}; - use datafusion_expr::{col, lit}; + use datafusion_expr::{Operator, col, lit}; use datafusion_physical_expr::{ PhysicalExpr, - expressions::{Column, DynamicFilterPhysicalExpr, Literal}, + expressions::{ + BinaryExpr, Column, DynamicFilterPhysicalExpr, Literal, lit as physical_lit, + }, planner::logical2physical, projection::ProjectionExprs, }; @@ -3265,134 +2992,6 @@ mod test { ); } - // --------------------------------------------------------------- - // Helper: build RowGroupMetaData with a given num_rows - // --------------------------------------------------------------- - - /// Create a minimal `SchemaDescriptor` with a single Int64 column named `id`. - fn make_int64_schema_descr() -> parquet::schema::types::SchemaDescPtr { - use parquet::basic::Type as PhysicalType; - use parquet::schema::types::Type as SchemaType; - - let field = SchemaType::primitive_type_builder("id", PhysicalType::INT64) - .build() - .unwrap(); - let schema = SchemaType::group_type_builder("schema") - .with_fields(vec![Arc::new(field)]) - .build() - .unwrap(); - Arc::new(parquet::schema::types::SchemaDescriptor::new(Arc::new( - schema, - ))) - } - - /// Build a vector of `RowGroupMetaData`, one per entry in `row_counts`. - fn make_rg_metadata( - row_counts: &[i64], - ) -> Vec { - let schema_descr = make_int64_schema_descr(); - row_counts - .iter() - .map(|&num_rows| { - let column = parquet::file::metadata::ColumnChunkMetaData::builder( - schema_descr.column(0), - ) - .set_num_values(num_rows) - .build() - .unwrap(); - parquet::file::metadata::RowGroupMetaData::builder(schema_descr.clone()) - .set_num_rows(num_rows) - .set_column_metadata(vec![column]) - .build() - .unwrap() - }) - .collect() - } - - // --------------------------------------------------------------- - // Tests for compute_best_threshold_from_stats - // --------------------------------------------------------------- - - #[test] - fn test_compute_threshold_desc_picks_max_of_mins() { - use arrow::array::{ArrayRef, Int64Array}; - - let stats: ArrayRef = - Arc::new(Int64Array::from(vec![Some(10), Some(50), Some(30)])); - let rg_meta = make_rg_metadata(&[100_000, 100_000, 100_000]); - - let result = - compute_best_threshold_from_stats(&stats, &rg_meta, 100, true).unwrap(); - assert_eq!(result, Some(ScalarValue::Int64(Some(50)))); - } - - #[test] - fn test_compute_threshold_asc_picks_min_of_maxes() { - use arrow::array::{ArrayRef, Int64Array}; - - let stats: ArrayRef = - Arc::new(Int64Array::from(vec![Some(100), Some(50), Some(80)])); - let rg_meta = make_rg_metadata(&[100_000, 100_000, 100_000]); - - let result = - compute_best_threshold_from_stats(&stats, &rg_meta, 100, false).unwrap(); - assert_eq!(result, Some(ScalarValue::Int64(Some(50)))); - } - - #[test] - fn test_compute_threshold_skips_small_rgs() { - use arrow::array::{ArrayRef, Int64Array}; - - // RG 0: 100K rows, value=10 - // RG 1: 50 rows (< limit=100), value=999 — should be skipped - // RG 2: 100K rows, value=30 - let stats: ArrayRef = - Arc::new(Int64Array::from(vec![Some(10), Some(999), Some(30)])); - let rg_meta = make_rg_metadata(&[100_000, 50, 100_000]); - - let result = - compute_best_threshold_from_stats(&stats, &rg_meta, 100, true).unwrap(); - // want_max=true => max(10, 30) = 30 (skipping the 999 from the small RG) - assert_eq!(result, Some(ScalarValue::Int64(Some(30)))); - } - - #[test] - fn test_compute_threshold_skips_null_stats() { - use arrow::array::{ArrayRef, Int64Array}; - - let stats: ArrayRef = Arc::new(Int64Array::from(vec![Some(10), None, Some(30)])); - let rg_meta = make_rg_metadata(&[100_000, 100_000, 100_000]); - - let result = - compute_best_threshold_from_stats(&stats, &rg_meta, 100, true).unwrap(); - // Null entry is skipped, max(10, 30) = 30 - assert_eq!(result, Some(ScalarValue::Int64(Some(30)))); - } - - #[test] - fn test_compute_threshold_all_rgs_too_small() { - use arrow::array::{ArrayRef, Int64Array}; - - let stats: ArrayRef = Arc::new(Int64Array::from(vec![Some(10), Some(50)])); - let rg_meta = make_rg_metadata(&[5, 10]); // all < limit=100 - - let result = - compute_best_threshold_from_stats(&stats, &rg_meta, 100, true).unwrap(); - assert_eq!(result, None); - } - - #[test] - fn test_compute_threshold_empty_metadata() { - use arrow::array::{ArrayRef, Int64Array}; - - let stats: ArrayRef = Arc::new(Int64Array::from(Vec::>::new())); - let rg_meta: Vec = vec![]; - - let result = - compute_best_threshold_from_stats(&stats, &rg_meta, 100, true).unwrap(); - assert_eq!(result, None); - } - // --------------------------------------------------------------- // Tests for find_dynamic_filter // --------------------------------------------------------------- @@ -3401,7 +3000,7 @@ mod test { fn test_find_dynamic_filter_direct_match() { let col_expr = Arc::new(Column::new("id", 0)) as Arc; let dynamic_filter: Arc = Arc::new( - DynamicFilterPhysicalExpr::new(vec![col_expr], super::lit(true)), + DynamicFilterPhysicalExpr::new(vec![col_expr], physical_lit(true)), ); let result = find_dynamic_filter(&dynamic_filter); @@ -3415,13 +3014,13 @@ mod test { fn test_find_dynamic_filter_nested_in_conjunction() { let col_expr = Arc::new(Column::new("id", 0)) as Arc; let dynamic_filter: Arc = Arc::new( - DynamicFilterPhysicalExpr::new(vec![col_expr], super::lit(true)), + DynamicFilterPhysicalExpr::new(vec![col_expr], physical_lit(true)), ); // Wrap it: (id > 0) AND dynamic_filter let left: Arc = Arc::new(BinaryExpr::new( Arc::new(Column::new("id", 0)), Operator::Gt, - super::lit(0i64), + physical_lit(0i64), )); let conjunction: Arc = Arc::new(BinaryExpr::new(left, Operator::And, dynamic_filter)); @@ -3442,229 +3041,4 @@ mod test { "plain Column has no DynamicFilterPhysicalExpr" ); } - - // --------------------------------------------------------------- - // Tests for try_init_topk_threshold - // --------------------------------------------------------------- - - /// Write a small parquet file and return its metadata so we can build - /// realistic `RowGroupMetaData` with actual statistics. - fn make_parquet_metadata_with_stats( - row_counts: &[Vec], - ) -> ( - Vec, - Schema, - parquet::schema::types::SchemaDescPtr, - ) { - use parquet::arrow::ArrowWriter; - use parquet::file::reader::FileReader; - use parquet::file::serialized_reader::SerializedFileReader; - - let schema = - Arc::new(Schema::new(vec![Field::new("id", DataType::Int64, false)])); - - // Write each row_counts entry as a separate row group - let mut buf = Vec::new(); - let mut writer = - ArrowWriter::try_new(&mut buf, Arc::clone(&schema), None).unwrap(); - for values in row_counts { - let batch = RecordBatch::try_new( - Arc::clone(&schema), - vec![Arc::new(arrow::array::Int64Array::from(values.clone()))], - ) - .unwrap(); - writer.write(&batch).unwrap(); - writer.flush().unwrap(); // flush after each batch -> separate RG - } - writer.close().unwrap(); - - // Read back metadata - let reader = SerializedFileReader::new(bytes::Bytes::from(buf)).unwrap(); - let file_metadata = reader.metadata(); - let rg_metadata: Vec<_> = file_metadata.row_groups().to_vec(); - let parquet_schema = file_metadata.file_metadata().schema_descr_ptr(); - (rg_metadata, schema.as_ref().clone(), parquet_schema) - } - - #[test] - fn test_try_init_topk_threshold_desc() { - // Two RGs: - // RG0: values [1..=200] => min=1, max=200 - // RG1: values [100..=300] => min=100, max=300 - // DESC sort, limit=10: threshold = max(min) = max(1, 100) = 100 - // Filter should be: id > 100 - let rg0_values: Vec = (1..=200).collect(); - let rg1_values: Vec = (100..=300).collect(); - let (rg_metadata, arrow_schema, parquet_schema) = - make_parquet_metadata_with_stats(&[rg0_values, rg1_values]); - - let col_expr = Arc::new(Column::new("id", 0)) as Arc; - let desc_opts = arrow::compute::SortOptions { - descending: true, - nulls_first: true, - }; - let dynamic_filter = Arc::new(DynamicFilterPhysicalExpr::new_with_sort_options( - vec![Arc::clone(&col_expr)], - super::lit(true), - vec![desc_opts], - Some(10), - )); - let predicate: Arc = dynamic_filter.clone(); - - try_init_topk_threshold( - &predicate, - 10, - &rg_metadata, - &arrow_schema, - &parquet_schema, - ) - .unwrap(); - - // Verify the dynamic filter was updated - let current = dynamic_filter.current().unwrap(); - let display = format!("{current}"); - assert!( - display.contains(">"), - "DESC should produce Gt operator, got: {display}" - ); - assert!( - display.contains("100"), - "threshold should be 100 (max of mins), got: {display}" - ); - } - - #[test] - fn test_try_init_topk_threshold_asc() { - // Two RGs: - // RG0: values [1..=200] => min=1, max=200 - // RG1: values [100..=300] => min=100, max=300 - // ASC sort, limit=10: threshold = min(max) = min(200, 300) = 200 - // Filter should be: id < 200 - let rg0_values: Vec = (1..=200).collect(); - let rg1_values: Vec = (100..=300).collect(); - let (rg_metadata, arrow_schema, parquet_schema) = - make_parquet_metadata_with_stats(&[rg0_values, rg1_values]); - - let col_expr = Arc::new(Column::new("id", 0)) as Arc; - let asc_opts = arrow::compute::SortOptions { - descending: false, - nulls_first: false, - }; - let dynamic_filter = Arc::new(DynamicFilterPhysicalExpr::new_with_sort_options( - vec![Arc::clone(&col_expr)], - super::lit(true), - vec![asc_opts], - Some(10), - )); - let predicate: Arc = dynamic_filter.clone(); - - try_init_topk_threshold( - &predicate, - 10, - &rg_metadata, - &arrow_schema, - &parquet_schema, - ) - .unwrap(); - - let current = dynamic_filter.current().unwrap(); - let display = format!("{current}"); - assert!( - display.contains("<"), - "ASC should produce Lt operator, got: {display}" - ); - assert!( - display.contains("200"), - "threshold should be 200 (min of maxes), got: {display}" - ); - } - - #[test] - fn test_try_init_topk_threshold_no_dynamic_filter() { - let rg0_values: Vec = (1..=200).collect(); - let (rg_metadata, arrow_schema, parquet_schema) = - make_parquet_metadata_with_stats(&[rg0_values]); - - // Plain predicate with no DynamicFilterPhysicalExpr - let predicate: Arc = Arc::new(Column::new("id", 0)); - - let result = try_init_topk_threshold( - &predicate, - 10, - &rg_metadata, - &arrow_schema, - &parquet_schema, - ); - assert!(result.is_ok(), "should be a no-op when no dynamic filter"); - } - - #[test] - fn test_try_init_topk_threshold_no_sort_options() { - let rg0_values: Vec = (1..=200).collect(); - let (rg_metadata, arrow_schema, parquet_schema) = - make_parquet_metadata_with_stats(&[rg0_values]); - - // DynamicFilterPhysicalExpr WITHOUT sort_options => should be skipped - let col_expr = Arc::new(Column::new("id", 0)) as Arc; - let dynamic_filter = Arc::new(DynamicFilterPhysicalExpr::new( - vec![col_expr], - super::lit(true), - )); - let predicate: Arc = dynamic_filter.clone(); - - let result = try_init_topk_threshold( - &predicate, - 10, - &rg_metadata, - &arrow_schema, - &parquet_schema, - ); - assert!( - result.is_ok(), - "no sort_options should be skipped gracefully" - ); - - let current = dynamic_filter.current().unwrap(); - let display = format!("{current}"); - assert_eq!(display, "true", "filter should remain unchanged: {display}"); - } - - #[test] - fn test_try_init_topk_threshold_multi_column_skipped() { - let rg0_values: Vec = (1..=200).collect(); - let (rg_metadata, arrow_schema, parquet_schema) = - make_parquet_metadata_with_stats(&[rg0_values]); - - // DynamicFilterPhysicalExpr with 2 sort columns => should be skipped - let col1 = Arc::new(Column::new("id", 0)) as Arc; - let col2 = Arc::new(Column::new("id", 0)) as Arc; - let opts = arrow::compute::SortOptions { - descending: true, - nulls_first: true, - }; - let dynamic_filter = Arc::new(DynamicFilterPhysicalExpr::new_with_sort_options( - vec![col1, col2], - super::lit(true), - vec![opts, opts], - Some(10), - )); - let predicate: Arc = dynamic_filter.clone(); - - let result = try_init_topk_threshold( - &predicate, - 10, - &rg_metadata, - &arrow_schema, - &parquet_schema, - ); - assert!( - result.is_ok(), - "multi-column sort should be skipped gracefully" - ); - - // Filter should still be the original `super::lit(true)` - let current = dynamic_filter.current().unwrap(); - let display = format!("{current}"); - assert_eq!(display, "true", "filter should remain unchanged: {display}"); - } } diff --git a/datafusion/sqllogictest/test_files/sort_pushdown.slt b/datafusion/sqllogictest/test_files/sort_pushdown.slt index a9d512228fc14..c983dc4b9ee31 100644 --- a/datafusion/sqllogictest/test_files/sort_pushdown.slt +++ b/datafusion/sqllogictest/test_files/sort_pushdown.slt @@ -2376,6 +2376,288 @@ DROP TABLE th_mixed; statement ok DROP TABLE th_reorder; +# =========================================================== +# Test I: WITH ORDER + DESC LIMIT — stats init + cumulative prune +# Tests the full optimization chain on sorted non-overlapping data: +# file reorder → RG reorder → reverse → stats init → cumulative prune +# =========================================================== + +statement ok +SET datafusion.execution.target_partitions = 1; + +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; + +# Create sorted data with multiple small row groups +statement ok +CREATE TABLE ti_data(id INT, value INT) AS VALUES +(1,10),(2,20),(3,30),(4,40),(5,50),(6,60), +(7,70),(8,80),(9,90),(10,100),(11,110),(12,120); + +statement ok +SET datafusion.execution.parquet.max_row_group_size = 3; + +query I +COPY (SELECT * FROM ti_data ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/ti_sorted/data.parquet'; +---- +12 + +statement ok +SET datafusion.execution.parquet.max_row_group_size = 1048576; + +statement ok +CREATE EXTERNAL TABLE ti_sorted(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/ti_sorted/data.parquet' +WITH ORDER (id ASC); + +# Test I.1: DESC LIMIT with sort pushdown — EXPLAIN shows reverse_row_groups=true +query TT +EXPLAIN SELECT * FROM ti_sorted ORDER BY id DESC LIMIT 3; +---- +logical_plan +01)Sort: ti_sorted.id DESC NULLS FIRST, fetch=3 +02)--TableScan: ti_sorted projection=[id, value] +physical_plan +01)SortExec: TopK(fetch=3), expr=[id@0 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/ti_sorted/data.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +# Test I.2: DESC LIMIT results — should return largest values +query II +SELECT * FROM ti_sorted ORDER BY id DESC LIMIT 3; +---- +12 120 +11 110 +10 100 + +# Test I.3: ASC LIMIT (same direction as file order) — sort elimination, no TopK +query TT +EXPLAIN SELECT * FROM ti_sorted ORDER BY id ASC LIMIT 3; +---- +logical_plan +01)Sort: ti_sorted.id ASC NULLS LAST, fetch=3 +02)--TableScan: ti_sorted projection=[id, value] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/ti_sorted/data.parquet]]}, projection=[id, value], limit=3, output_ordering=[id@0 ASC NULLS LAST], file_type=parquet + +query II +SELECT * FROM ti_sorted ORDER BY id ASC LIMIT 3; +---- +1 10 +2 20 +3 30 + +# Test I.4: DESC LIMIT with WHERE — cumulative prune works, stats init skipped +query II +SELECT * FROM ti_sorted WHERE value > 50 ORDER BY id DESC LIMIT 2; +---- +12 120 +11 110 + +# Test I.5: Larger LIMIT spanning multiple RGs (4 RGs of 3 rows each) +query II +SELECT * FROM ti_sorted ORDER BY id DESC LIMIT 8; +---- +12 120 +11 110 +10 100 +9 90 +8 80 +7 70 +6 60 +5 50 + +# Test I.6: LIMIT larger than total rows — returns all rows +query II +SELECT * FROM ti_sorted ORDER BY id DESC LIMIT 100; +---- +12 120 +11 110 +10 100 +9 90 +8 80 +7 70 +6 60 +5 50 +4 40 +3 30 +2 20 +1 10 + +# Cleanup Test I +statement ok +DROP TABLE ti_data; + +statement ok +DROP TABLE ti_sorted; + +# =========================================================== +# Test J: Non-overlapping RGs without WITH ORDER — +# RG reorder + cumulative prune via DynamicFilter sort_options +# =========================================================== + +# Create data written in scrambled order but with non-overlapping RGs +statement ok +CREATE TABLE tj_high(id INT, value INT) AS VALUES (10,100),(11,110),(12,120); + +statement ok +CREATE TABLE tj_mid(id INT, value INT) AS VALUES (4,40),(5,50),(6,60); + +statement ok +CREATE TABLE tj_low(id INT, value INT) AS VALUES (1,10),(2,20),(3,30); + +statement ok +SET datafusion.execution.parquet.max_row_group_size = 3; + +# Write in scrambled order: high, low, mid → RGs are non-overlapping but not sorted +query I +COPY ( + SELECT * FROM tj_high + UNION ALL SELECT * FROM tj_low + UNION ALL SELECT * FROM tj_mid +) +TO 'test_files/scratch/sort_pushdown/tj_scrambled/data.parquet'; +---- +9 + +statement ok +SET datafusion.execution.parquet.max_row_group_size = 1048576; + +# No WITH ORDER — tests non-sort-pushdown TopK path +statement ok +CREATE EXTERNAL TABLE tj_scrambled(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/tj_scrambled/data.parquet'; + +# Test J.1: DESC LIMIT — RG reorder + reverse from DynamicFilter +# EXPLAIN shows DynamicFilter but NO reverse_row_groups (not sort pushdown) +query TT +EXPLAIN SELECT * FROM tj_scrambled ORDER BY id DESC LIMIT 3; +---- +logical_plan +01)Sort: tj_scrambled.id DESC NULLS FIRST, fetch=3 +02)--TableScan: tj_scrambled projection=[id, value] +physical_plan +01)SortExec: TopK(fetch=3), expr=[id@0 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tj_scrambled/data.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ] + +# Test J.2: Results must be correct +query II +SELECT * FROM tj_scrambled ORDER BY id DESC LIMIT 3; +---- +12 120 +11 110 +10 100 + +# Test J.3: ASC LIMIT +query II +SELECT * FROM tj_scrambled ORDER BY id ASC LIMIT 3; +---- +1 10 +2 20 +3 30 + +# Test J.4: DESC with WHERE +query II +SELECT * FROM tj_scrambled WHERE value >= 50 ORDER BY id DESC LIMIT 2; +---- +12 120 +11 110 + +# Test J.5: Full sort without LIMIT (no TopK, no reorder) +query II +SELECT * FROM tj_scrambled ORDER BY id ASC; +---- +1 10 +2 20 +3 30 +4 40 +5 50 +6 60 +10 100 +11 110 +12 120 + +# Cleanup Test J +statement ok +DROP TABLE tj_high; + +statement ok +DROP TABLE tj_mid; + +statement ok +DROP TABLE tj_low; + +statement ok +DROP TABLE tj_scrambled; + +# =========================================================== +# Test K: Overlapping RGs without WITH ORDER — +# cumulative prune must NOT trigger (overlapping = unsafe) +# =========================================================== + +# Create data with overlapping ranges +statement ok +CREATE TABLE tk_data(id INT, value INT) AS VALUES +(1,10),(5,50),(9,90), +(3,30),(7,70),(11,110), +(2,20),(6,60),(10,100); + +statement ok +SET datafusion.execution.parquet.max_row_group_size = 3; + +query I +COPY (SELECT * FROM tk_data) +TO 'test_files/scratch/sort_pushdown/tk_overlap/data.parquet'; +---- +9 + +statement ok +SET datafusion.execution.parquet.max_row_group_size = 1048576; + +statement ok +CREATE EXTERNAL TABLE tk_overlap(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/tk_overlap/data.parquet'; + +# Test K.1: DESC LIMIT on overlapping RGs — results must be correct +# (cumulative prune should not trigger due to overlap) +query II +SELECT * FROM tk_overlap ORDER BY id DESC LIMIT 3; +---- +11 110 +10 100 +9 90 + +# Test K.2: ASC LIMIT on overlapping RGs +query II +SELECT * FROM tk_overlap ORDER BY id ASC LIMIT 3; +---- +1 10 +2 20 +3 30 + +# Test K.3: Full sort (verify all data present) +query II +SELECT * FROM tk_overlap ORDER BY id ASC; +---- +1 10 +2 20 +3 30 +5 50 +6 60 +7 70 +9 90 +10 100 +11 110 + +# Cleanup Test K +statement ok +DROP TABLE tk_data; + +statement ok +DROP TABLE tk_overlap; + # Reset settings (SLT runner uses target_partitions=4, not system default) statement ok SET datafusion.execution.target_partitions = 4; From f7c42d8627b91fe6f4b2b97e771794743c18ec21 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 20:08:42 +0800 Subject: [PATCH 31/38] fix: cumulative prune only without WHERE to avoid under-returning rows --- datafusion/datasource-parquet/src/opener.rs | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 623f6eaafe1a3..9c9154fbb36f2 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -1232,11 +1232,22 @@ impl RowGroupsPrunedParquetOpen { // TopK cumulative pruning: after reorder + reverse, the RGs are in // optimal order. Accumulate rows from the front until >= K, prune rest. - // For sort pushdown: always safe (sorted = non-overlapping). - // For non-sort-pushdown: safe only if reordered RGs are non-overlapping - // (verified by checking max[i] <= min[i+1] in the reordered sequence). + // + // Only safe when predicate is DynamicFilter-only (no WHERE clause). + // With WHERE, raw num_rows overestimates qualifying rows — cumulative + // prune may keep too few RGs, returning fewer than K results. + // + // Additionally requires either sort pushdown (guaranteed non-overlapping) + // or verified non-overlap from statistics. + let is_pure_dynamic_filter = prepared.predicate.as_ref().is_some_and(|p| { + let any_ref: &dyn std::any::Any = p.as_ref(); + any_ref + .downcast_ref::() + .is_some() + }); let has_sort_pushdown = prepared.sort_order_for_reorder.is_some(); - if let Some(predicate) = &prepared.predicate + if is_pure_dynamic_filter + && let Some(predicate) = &prepared.predicate && let Some(df) = find_dynamic_filter(predicate) && let Some(fetch) = df.fetch() && (has_sort_pushdown From 332561008cde480e031fa409cbe61f4f86c56c60 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 20:36:08 +0800 Subject: [PATCH 32/38] feat: restore stats init with fixes (GtEq + df.fetch() + type cast) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Bring back stats init with all issues fixed: - GtEq/LtEq instead of Gt/Lt (include boundary values) - Use df.fetch() as limit (TopK K value, not scan limit) When K > single RG rows, stats init skips → cumulative prune handles it - Cast threshold to column data type (parquet vs table schema mismatch) - Null-aware filter for NULLS FIRST - Generation check prevents overwrite by later partitions - Restricted to sort pushdown + pure DynamicFilter (no WHERE) Stats init and cumulative prune are complementary: - Stats init: updates PruningPredicate → prunes at RG statistics level - Cumulative prune: truncates after reorder+reverse → prunes by row count Both work together without conflict when using df.fetch(). --- datafusion/datasource-parquet/src/opener.rs | 163 +++++++++++++++++++- 1 file changed, 158 insertions(+), 5 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 9c9154fbb36f2..ddbf17fa35945 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -27,7 +27,10 @@ use crate::{ use arrow::array::{Array, RecordBatch, RecordBatchOptions}; use arrow::datatypes::DataType; use datafusion_datasource::morsel::{Morsel, MorselPlan, MorselPlanner, Morselizer}; -use datafusion_physical_expr::expressions::{Column, DynamicFilterPhysicalExpr}; +use datafusion_expr::Operator; +use datafusion_physical_expr::expressions::{ + BinaryExpr, Column, DynamicFilterPhysicalExpr, lit, +}; use datafusion_physical_expr::projection::{ProjectionExprs, Projector}; use datafusion_physical_expr::utils::reassign_expr_columns; use datafusion_physical_expr_adapter::replace_columns_with_literals; @@ -831,10 +834,36 @@ impl MetadataLoadedParquetOpen { } prepared.physical_file_schema = Arc::clone(&physical_file_schema); - // Note: stats init (try_init_topk_threshold) was removed in favor of - // cumulative RG pruning which is safer and works with WHERE clauses. - // Stats init had issues: Gt vs GtEq boundary, type coercion, and - // conflicting with cumulative prune when limit spans multiple RGs. + // Initialize TopK threshold from RG statistics BEFORE building + // PruningPredicate. Uses GtEq/LtEq to include boundary values. + // Only for sort pushdown + no WHERE (pure DynamicFilter predicate). + // Uses df.fetch() as limit so stats init skips when K spans multiple + // RGs (no single RG has >= K rows), letting cumulative prune handle it. + if prepared.sort_order_for_reorder.is_some() + && let Some(pred) = &prepared.predicate + && let Some(df) = find_dynamic_filter(pred) + && df.sort_options().is_some_and(|opts| opts.len() == 1) + && let Some(fetch) = df.fetch() + { + // Only when predicate is pure DynamicFilter (no WHERE) + let any_ref: &dyn std::any::Any = pred.as_ref(); + if any_ref + .downcast_ref::() + .is_some() + { + let file_metadata = reader_metadata.metadata(); + let rg_metadata = file_metadata.row_groups(); + if let Err(e) = try_init_topk_threshold( + pred, + fetch, + rg_metadata, + &physical_file_schema, + reader_metadata.parquet_schema(), + ) { + debug!("Skipping TopK threshold init: {e}"); + } + } + } // Build predicates for this specific file let pruning_predicate = build_pruning_predicates( @@ -1370,6 +1399,130 @@ impl RowGroupsPrunedParquetOpen { /// checking the predicate itself and recursively walking its children. /// Check if row groups in the prepared plan are non-overlapping on the /// sort column. Adjacent RGs must satisfy `max(i) <= min(i+1)`. +/// Initialize TopK dynamic filter threshold from row group statistics. +/// +/// For DESC: `threshold = max(min)` across RGs with `num_rows >= fetch`. +/// For ASC: `threshold = min(max)` across qualifying RGs. +/// Uses GtEq/LtEq to include boundary values. +fn try_init_topk_threshold( + predicate: &Arc, + fetch: usize, + rg_metadata: &[parquet::file::metadata::RowGroupMetaData], + arrow_schema: &Schema, + parquet_schema: &parquet::schema::types::SchemaDescriptor, +) -> Result<()> { + let dynamic_filter = match find_dynamic_filter(predicate) { + Some(df) => df, + None => return Ok(()), + }; + + if dynamic_filter.snapshot_generation() > 1 { + return Ok(()); // Already initialized + } + + let sort_options = match dynamic_filter.sort_options() { + Some(opts) if opts.len() == 1 => opts, + _ => return Ok(()), + }; + + let is_descending = sort_options[0].descending; + let nulls_first = sort_options[0].nulls_first; + + let column = match find_column_in_expr(dynamic_filter.children()[0]) { + Some(col) => col, + None => return Ok(()), + }; + + let col_name = column.name(); + let converter = StatisticsConverter::try_new(col_name, arrow_schema, parquet_schema) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + + let threshold = if is_descending { + let mins = converter + .row_group_mins(rg_metadata.iter()) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + compute_best_threshold(&mins, rg_metadata, fetch, true)? + } else { + let maxes = converter + .row_group_maxes(rg_metadata.iter()) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + compute_best_threshold(&maxes, rg_metadata, fetch, false)? + }; + + let threshold = match threshold { + Some(t) => t, + None => return Ok(()), // No RG with >= fetch rows + }; + + // Cast threshold to column type + let col_expr: Arc = Arc::clone(dynamic_filter.children()[0]); + let col_data_type = col_expr.data_type(arrow_schema)?; + let threshold_casted = threshold.cast_to(&col_data_type)?; + + // GtEq/LtEq: boundary value IS a valid top-K value + let op = if is_descending { + Operator::GtEq + } else { + Operator::LtEq + }; + + let comparison: Arc = Arc::new(BinaryExpr::new( + Arc::clone(&col_expr), + op, + lit(threshold_casted), + )); + + let filter_expr: Arc = if nulls_first { + use datafusion_physical_expr::expressions::is_null; + let null_check = is_null(Arc::clone(&col_expr))?; + Arc::new(BinaryExpr::new(null_check, Operator::Or, comparison)) + } else { + comparison + }; + + debug!( + "TopK stats init: {col_name} {op} (fetch={fetch})", + op = if is_descending { ">=" } else { "<=" } + ); + + dynamic_filter.update(filter_expr)?; + Ok(()) +} + +/// Find the best threshold from RG statistics. +/// `want_max=true`: max of values (for DESC min). `want_max=false`: min of values (for ASC max). +fn compute_best_threshold( + stats: &arrow::array::ArrayRef, + rg_metadata: &[parquet::file::metadata::RowGroupMetaData], + fetch: usize, + want_max: bool, +) -> Result> { + let mut best: Option = None; + for (i, rg) in rg_metadata.iter().enumerate() { + if (rg.num_rows() as usize) < fetch { + continue; + } + if i >= stats.len() || stats.is_null(i) { + continue; + } + let value = ScalarValue::try_from_array(stats.as_ref(), i)?; + if value.is_null() { + continue; + } + best = Some(match best { + None => value, + Some(current) => { + if want_max { + if value > current { value } else { current } + } else { + if value < current { value } else { current } + } + } + }); + } + Ok(best) +} + fn rgs_are_non_overlapping( plan: &crate::access_plan::PreparedAccessPlan, file_metadata: &parquet::file::metadata::ParquetMetaData, From 76f63aa767271da33dfc7d9e2447714b4a67460b Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 21 Apr 2026 23:44:52 +0800 Subject: [PATCH 33/38] fix: SortExec.fetch was 0 when create_filter was called MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit create_filter() was called before new_sort.fetch was set, so DynamicFilterPhysicalExpr.fetch was always 0 (or None from old self). Fix by setting fetch before creating the filter. This was the root cause of stats init and cumulative prune not triggering on CI — fetch=0 meant "no rows needed" → skip. --- datafusion/physical-plan/src/sorts/sort.rs | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index a0e3ea523a892..eb1b7bbdd739e 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -962,14 +962,16 @@ impl SortExec { if fetch.is_some() && is_pipeline_friendly { cache = cache.with_boundedness(Boundedness::Bounded); } - let filter = fetch.is_some().then(|| { - // If we already have a filter, keep it. Otherwise, create a new one. - self.filter.clone().unwrap_or_else(|| self.create_filter()) - }); let mut new_sort = self.cloned(); new_sort.fetch = fetch; new_sort.cache = cache.into(); - new_sort.filter = filter; + new_sort.filter = fetch.is_some().then(|| { + // If we already have a filter, keep it. Otherwise, create a new one. + // Must be called after setting fetch so DynamicFilter gets the K value. + self.filter + .clone() + .unwrap_or_else(|| new_sort.create_filter()) + }); new_sort } From 93a8668b5aa8830e72e491a5564510e22d9652b9 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Wed, 22 Apr 2026 11:56:38 +0800 Subject: [PATCH 34/38] perf: skip RG reorder when sort column not in file schema MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit For GROUP BY + ORDER BY queries, the TopK sort column is an aggregate output (e.g. COUNT(*)) that doesn't exist in the parquet file schema. Previously we still created ReorderByStatistics which tried to look up the column in statistics — wasted work. Now check column existence in file schema before creating the optimizer. This eliminates overhead for non-scan-level TopK queries (ClickBench Q40-Q42 regression fix). --- datafusion/datasource-parquet/src/opener.rs | 42 +++++++++++++-------- 1 file changed, 27 insertions(+), 15 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index ddbf17fa35945..404acff23da0b 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -1197,23 +1197,35 @@ impl RowGroupsPrunedParquetOpen { && sort_options.len() == 1 { // Build a sort order from DynamicFilter for non-sort-pushdown TopK. - // Always ASC — reverse handles DESC separately. + // Quick bail: check if the sort column exists in file schema. + // For GROUP BY + ORDER BY, the sort column is an aggregate output + // (not in parquet) — skip to avoid wasted StatisticsConverter work. let children = df.children(); if !children.is_empty() { - let sort_expr = - datafusion_physical_expr_common::sort_expr::PhysicalSortExpr { - expr: Arc::clone(children[0]), - options: arrow::compute::SortOptions { - descending: false, - nulls_first: sort_options[0].nulls_first, - }, - }; - LexOrdering::new(vec![sort_expr]).map(|order| { - Box::new(crate::access_plan_optimizer::ReorderByStatistics::new( - order, - )) - as Box - }) + let col = find_column_in_expr(children[0]); + if let Some(ref c) = col + && prepared + .physical_file_schema + .field_with_name(c.name()) + .is_ok() + { + let sort_expr = + datafusion_physical_expr_common::sort_expr::PhysicalSortExpr { + expr: Arc::clone(children[0]), + options: arrow::compute::SortOptions { + descending: false, + nulls_first: sort_options[0].nulls_first, + }, + }; + LexOrdering::new(vec![sort_expr]).map(|order| { + Box::new(crate::access_plan_optimizer::ReorderByStatistics::new( + order, + )) + as Box + }) + } else { + None + } } else { None } From 8ded1f460128fc010247096e97230c9c88f30c82 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Wed, 22 Apr 2026 13:21:44 +0800 Subject: [PATCH 35/38] fix: use slt:ignore for non-deterministic output_rows_skew metric --- datafusion/sqllogictest/test_files/explain_analyze.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/explain_analyze.slt b/datafusion/sqllogictest/test_files/explain_analyze.slt index 561c1a9dcbc87..4b59e30762610 100644 --- a/datafusion/sqllogictest/test_files/explain_analyze.slt +++ b/datafusion/sqllogictest/test_files/explain_analyze.slt @@ -103,7 +103,7 @@ LOCATION 'test_files/scratch/explain_analyze/output_rows_skew'; query TT EXPLAIN ANALYZE SELECT * FROM skew_parquet; ---- -Plan with Metrics DataSourceExec: output_rows_skew=84.31% +Plan with Metrics DataSourceExec: output_rows_skew= # All partition's output_rows: [0] statement ok From 138bd8e8fa3a55ab12cdf0becdcb9d8039cb8c90 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Wed, 22 Apr 2026 20:27:06 +0800 Subject: [PATCH 36/38] fix: skip cumulative prune when row_selection exists + cleanup - truncate_row_groups now skips when row_selection is present to preserve page-level pruning state (xudong review feedback) - Remove incomplete DynamicRgPruner exploration code --- datafusion/datasource-parquet/src/access_plan.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/datafusion/datasource-parquet/src/access_plan.rs b/datafusion/datasource-parquet/src/access_plan.rs index 7fbe225ad7e02..346a6d4aadc51 100644 --- a/datafusion/datasource-parquet/src/access_plan.rs +++ b/datafusion/datasource-parquet/src/access_plan.rs @@ -389,11 +389,13 @@ impl PreparedAccessPlan { /// Keep only the first `count` row groups, dropping the rest. /// Used for TopK cumulative pruning after reorder + reverse. pub(crate) fn truncate_row_groups(mut self, count: usize) -> Self { - self.row_group_indexes.truncate(count); - // Clear row_selection since it's tied to the original RG set + // Skip truncation if row_selection exists — it contains page-level + // pruning state that would be lost. Cumulative prune is a best-effort + // optimization; page pruning is already reducing I/O within those RGs. if self.row_selection.is_some() { - self.row_selection = None; + return self; } + self.row_group_indexes.truncate(count); self } From 7acb19a523e703e7c9b77825079b21c58baba469 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Wed, 22 Apr 2026 20:45:14 +0800 Subject: [PATCH 37/38] feat: support multi-key ORDER BY + truncate row_selection safety Multi-key ORDER BY: use the first sort key for RG-level optimizations. Secondary keys only affect tie-breaking within RGs, not RG decisions. truncate_row_groups: skip truncation when row_selection exists to preserve page-level pruning state (xudong review). Tests: - Test L: multi-key DESC/ASC LIMIT (6 sub-tests) - truncate unit tests: basic, row_selection skip, no-op overflow --- .../datasource-parquet/src/access_plan.rs | 38 ++++++++ datafusion/datasource-parquet/src/opener.rs | 8 +- datafusion/datasource-parquet/src/source.rs | 2 +- .../sqllogictest/test_files/sort_pushdown.slt | 91 +++++++++++++++++++ 4 files changed, 134 insertions(+), 5 deletions(-) diff --git a/datafusion/datasource-parquet/src/access_plan.rs b/datafusion/datasource-parquet/src/access_plan.rs index 346a6d4aadc51..7b4484ae08e39 100644 --- a/datafusion/datasource-parquet/src/access_plan.rs +++ b/datafusion/datasource-parquet/src/access_plan.rs @@ -987,3 +987,41 @@ mod test { assert_eq!(plan.row_group_indexes, vec![0, 1]); } } + +#[test] +fn test_truncate_row_groups_basic() { + let plan = PreparedAccessPlan { + row_group_indexes: vec![5, 3, 1, 4, 2], + row_selection: None, + }; + let plan = plan.truncate_row_groups(3); + assert_eq!(plan.row_group_indexes, vec![5, 3, 1]); + assert!(plan.row_selection.is_none()); +} + +#[test] +fn test_truncate_row_groups_skips_when_row_selection_present() { + let selection = RowSelection::from(vec![ + RowSelector::select(100), + RowSelector::skip(50), + RowSelector::select(100), + ]); + let plan = PreparedAccessPlan { + row_group_indexes: vec![5, 3, 1, 4, 2], + row_selection: Some(selection), + }; + // Should NOT truncate because row_selection is present + let plan = plan.truncate_row_groups(2); + assert_eq!(plan.row_group_indexes, vec![5, 3, 1, 4, 2]); + assert!(plan.row_selection.is_some()); +} + +#[test] +fn test_truncate_row_groups_no_op_when_count_exceeds_len() { + let plan = PreparedAccessPlan { + row_group_indexes: vec![1, 2], + row_selection: None, + }; + let plan = plan.truncate_row_groups(10); + assert_eq!(plan.row_group_indexes, vec![1, 2]); +} diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 404acff23da0b..2dbbc53021170 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -842,7 +842,7 @@ impl MetadataLoadedParquetOpen { if prepared.sort_order_for_reorder.is_some() && let Some(pred) = &prepared.predicate && let Some(df) = find_dynamic_filter(pred) - && df.sort_options().is_some_and(|opts| opts.len() == 1) + && df.sort_options().is_some_and(|opts| !opts.is_empty()) && let Some(fetch) = df.fetch() { // Only when predicate is pure DynamicFilter (no WHERE) @@ -1194,7 +1194,7 @@ impl RowGroupsPrunedParquetOpen { } else if let Some(predicate) = &prepared.predicate && let Some(df) = find_dynamic_filter(predicate) && let Some(sort_options) = df.sort_options() - && sort_options.len() == 1 + && !sort_options.is_empty() { // Build a sort order from DynamicFilter for non-sort-pushdown TopK. // Quick bail: check if the sort column exists in file schema. @@ -1433,7 +1433,7 @@ fn try_init_topk_threshold( } let sort_options = match dynamic_filter.sort_options() { - Some(opts) if opts.len() == 1 => opts, + Some(opts) if !opts.is_empty() => opts, _ => return Ok(()), }; @@ -1542,7 +1542,7 @@ fn rgs_are_non_overlapping( dynamic_filter: &DynamicFilterPhysicalExpr, ) -> bool { let sort_options = match dynamic_filter.sort_options() { - Some(opts) if opts.len() == 1 => opts, + Some(opts) if !opts.is_empty() => opts, _ => return false, }; let children = dynamic_filter.children(); diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index e1dbc28cb00e2..81059e975de1a 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -528,7 +528,7 @@ impl ParquetSource { >(any_arc) { let sort_options = df.sort_options()?; - if sort_options.len() != 1 { + if sort_options.is_empty() { return None; } let children = df.children(); diff --git a/datafusion/sqllogictest/test_files/sort_pushdown.slt b/datafusion/sqllogictest/test_files/sort_pushdown.slt index c983dc4b9ee31..64ee49a715828 100644 --- a/datafusion/sqllogictest/test_files/sort_pushdown.slt +++ b/datafusion/sqllogictest/test_files/sort_pushdown.slt @@ -2658,6 +2658,97 @@ DROP TABLE tk_data; statement ok DROP TABLE tk_overlap; +# =========================================================== +# Test L: Multi-key ORDER BY — uses first key for RG reorder +# =========================================================== + +statement ok +CREATE TABLE tl_data(id INT, name VARCHAR, value INT) AS VALUES +(10, 'a', 100), (11, 'b', 110), (12, 'c', 120), +(4, 'd', 40), (5, 'e', 50), (6, 'f', 60), +(1, 'g', 10), (2, 'h', 20), (3, 'i', 30); + +statement ok +SET datafusion.execution.parquet.max_row_group_size = 3; + +query I +COPY (SELECT * FROM tl_data) +TO 'test_files/scratch/sort_pushdown/tl_multikey/data.parquet'; +---- +9 + +statement ok +SET datafusion.execution.parquet.max_row_group_size = 1048576; + +statement ok +CREATE EXTERNAL TABLE tl_multikey(id INT, name VARCHAR, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/tl_multikey/data.parquet'; + +# Test L.1: Multi-key DESC — first key determines RG reorder +query IIT +SELECT id, value, name FROM tl_multikey ORDER BY id DESC, name ASC LIMIT 3; +---- +12 120 c +11 110 b +10 100 a + +# Test L.2: Multi-key ASC +query IIT +SELECT id, value, name FROM tl_multikey ORDER BY id ASC, name DESC LIMIT 3; +---- +1 10 g +2 20 h +3 30 i + +# Test L.3: Multi-key with 3 sort columns +query IIT +SELECT id, value, name FROM tl_multikey ORDER BY id DESC, value DESC, name ASC LIMIT 5; +---- +12 120 c +11 110 b +10 100 a +6 60 f +5 50 e + +# Test L.4: WITH ORDER + multi-key DESC LIMIT +statement ok +CREATE EXTERNAL TABLE tl_sorted(id INT, name VARCHAR, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/tl_multikey/data.parquet' +WITH ORDER (id ASC); + +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; + +# Multi-key with sort pushdown — first key triggers reverse +query TT +EXPLAIN SELECT id, value FROM tl_sorted ORDER BY id DESC, value ASC LIMIT 3; +---- +logical_plan +01)Sort: tl_sorted.id DESC NULLS FIRST, tl_sorted.value ASC NULLS LAST, fetch=3 +02)--TableScan: tl_sorted projection=[id, value] +physical_plan +01)SortExec: TopK(fetch=3), expr=[id@0 DESC, value@1 ASC NULLS LAST], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tl_multikey/data.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] + +query II +SELECT id, value FROM tl_sorted ORDER BY id DESC, value ASC LIMIT 3; +---- +12 120 +11 110 +10 100 + +# Cleanup Test L +statement ok +DROP TABLE tl_data; + +statement ok +DROP TABLE tl_multikey; + +statement ok +DROP TABLE tl_sorted; + # Reset settings (SLT runner uses target_partitions=4, not system default) statement ok SET datafusion.execution.target_partitions = 4; From d725d84b9f05cc60fa11e0ea6b1ad4b27edb7760 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Wed, 22 Apr 2026 20:54:35 +0800 Subject: [PATCH 38/38] test: add multi-key Inexact sort pushdown test (Test M) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Test M: file declared WITH ORDER (id ASC, value ASC), multi-key queries testing: - M.1: EXPLAIN showing reverse_row_groups=true for fully reversed match - M.2: DESC, DESC LIMIT 3 — correct results - M.3: larger LIMIT spanning multiple RGs - M.4: ASC, ASC (same direction = Exact, sort elimination) - M.5: partial match (first key reversed, second key same) — NOT Inexact - M.6: full sort, data integrity check --- .../sqllogictest/test_files/sort_pushdown.slt | 105 ++++++++++++++++++ 1 file changed, 105 insertions(+) diff --git a/datafusion/sqllogictest/test_files/sort_pushdown.slt b/datafusion/sqllogictest/test_files/sort_pushdown.slt index 64ee49a715828..a968174a95b1a 100644 --- a/datafusion/sqllogictest/test_files/sort_pushdown.slt +++ b/datafusion/sqllogictest/test_files/sort_pushdown.slt @@ -2749,6 +2749,111 @@ DROP TABLE tl_multikey; statement ok DROP TABLE tl_sorted; +# =========================================================== +# Test M: Multi-key Inexact sort pushdown (WITH ORDER multi-key) +# File declared WITH ORDER (id ASC, value ASC), query ORDER BY +# id DESC, value DESC → reversed matches → Inexact path. +# =========================================================== + +statement ok +CREATE TABLE tm_data(id INT, value INT) AS VALUES +(1, 10), (1, 20), (1, 30), +(2, 40), (2, 50), (2, 60), +(3, 70), (3, 80), (3, 90), +(4, 100), (4, 110), (4, 120); + +statement ok +SET datafusion.execution.parquet.max_row_group_size = 3; + +# Write sorted by (id ASC, value ASC) +query I +COPY (SELECT * FROM tm_data ORDER BY id ASC, value ASC) +TO 'test_files/scratch/sort_pushdown/tm_multikey_sorted/data.parquet'; +---- +12 + +statement ok +SET datafusion.execution.parquet.max_row_group_size = 1048576; + +statement ok +CREATE EXTERNAL TABLE tm_sorted(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/tm_multikey_sorted/data.parquet' +WITH ORDER (id ASC, value ASC); + +# Test M.1: Multi-key DESC that fully matches reversed ordering +# ORDER BY id DESC, value DESC matches reversed (id ASC, value ASC) → Inexact +query TT +EXPLAIN SELECT * FROM tm_sorted ORDER BY id DESC, value DESC LIMIT 3; +---- +logical_plan +01)Sort: tm_sorted.id DESC NULLS FIRST, tm_sorted.value DESC NULLS FIRST, fetch=3 +02)--TableScan: tm_sorted projection=[id, value] +physical_plan +01)SortExec: TopK(fetch=3), expr=[id@0 DESC, value@1 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tm_multikey_sorted/data.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +# Test M.2: Results must be correct (largest id first, then largest value) +query II +SELECT * FROM tm_sorted ORDER BY id DESC, value DESC LIMIT 3; +---- +4 120 +4 110 +4 100 + +# Test M.3: Larger limit spanning multiple RGs +query II +SELECT * FROM tm_sorted ORDER BY id DESC, value DESC LIMIT 6; +---- +4 120 +4 110 +4 100 +3 90 +3 80 +3 70 + +# Test M.4: Multi-key ASC (same direction as file = Exact, sort elimination) +query II +SELECT * FROM tm_sorted ORDER BY id ASC, value ASC LIMIT 3; +---- +1 10 +1 20 +1 30 + +# Test M.5: Partial match — first key reversed, second key SAME direction +# ORDER BY id DESC, value ASC does NOT match reversed (id DESC, value DESC) +# → NOT Inexact path (second key mismatch) +query II +SELECT * FROM tm_sorted ORDER BY id DESC, value ASC LIMIT 3; +---- +4 100 +4 110 +4 120 + +# Test M.6: All rows, verify full data integrity +query II +SELECT * FROM tm_sorted ORDER BY id DESC, value DESC; +---- +4 120 +4 110 +4 100 +3 90 +3 80 +3 70 +2 60 +2 50 +2 40 +1 30 +1 20 +1 10 + +# Cleanup Test M +statement ok +DROP TABLE tm_data; + +statement ok +DROP TABLE tm_sorted; + # Reset settings (SLT runner uses target_partitions=4, not system default) statement ok SET datafusion.execution.target_partitions = 4;