From 4ed0668e843472e151b7906990aa66b9e8955f81 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Thu, 18 Dec 2025 11:45:38 +0800 Subject: [PATCH 01/10] support reverse files and row groups for dynamic topk --- datafusion/common/src/config.rs | 15 + .../core/tests/physical_optimizer/mod.rs | 1 + .../tests/physical_optimizer/pushdown_sort.rs | 672 ++++++++++++++++++ .../tests/physical_optimizer/test_utils.rs | 78 +- datafusion/datasource-parquet/src/mod.rs | 1 + datafusion/datasource-parquet/src/opener.rs | 350 ++++++++- datafusion/datasource-parquet/src/sort.rs | 407 +++++++++++ datafusion/datasource-parquet/src/source.rs | 183 ++++- datafusion/datasource/src/file.rs | 36 +- datafusion/datasource/src/file_scan_config.rs | 89 ++- datafusion/datasource/src/source.rs | 35 +- datafusion/execution/src/config.rs | 7 + .../physical-expr-common/src/sort_expr.rs | 103 +++ datafusion/physical-optimizer/src/lib.rs | 1 + .../physical-optimizer/src/optimizer.rs | 3 + .../physical-optimizer/src/pushdown_sort.rs | 129 ++++ .../physical-plan/src/coalesce_batches.rs | 16 + .../physical-plan/src/coalesce_partitions.rs | 39 + .../physical-plan/src/execution_plan.rs | 28 +- datafusion/physical-plan/src/lib.rs | 2 + .../physical-plan/src/repartition/mod.rs | 24 +- datafusion/physical-plan/src/sort_pushdown.rs | 120 ++++ .../test_files/create_external_table.slt | 2 +- .../dynamic_filter_pushdown_config.slt | 326 +++++++++ .../sqllogictest/test_files/explain.slt | 4 + .../test_files/information_schema.slt | 2 + datafusion/sqllogictest/test_files/topk.slt | 2 +- docs/source/user-guide/configs.md | 1 + 28 files changed, 2628 insertions(+), 48 deletions(-) create mode 100644 datafusion/core/tests/physical_optimizer/pushdown_sort.rs create mode 100644 datafusion/datasource-parquet/src/sort.rs create mode 100644 datafusion/physical-optimizer/src/pushdown_sort.rs create mode 100644 datafusion/physical-plan/src/sort_pushdown.rs diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index a77fd764eea06..3931208e14f88 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -983,6 +983,21 @@ config_namespace! { /// then the output will be coerced to a non-view. /// Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. pub expand_views_at_output: bool, default = false + + /// Enable sort pushdown optimization. + /// When enabled, attempts to push sort requirements down to data sources + /// that can natively handle them (e.g., by reversing file/row group read order). + /// + /// Returns **inexact ordering**: Sort operator is kept for correctness, + /// but optimized input enables early termination for TopK queries (ORDER BY ... LIMIT N), + /// providing significant speedup. + /// + /// Memory: No additional overhead (only changes read order). + /// + /// Future: Will add option to detect perfectly sorted data and eliminate Sort completely. + /// + /// Default: true + pub enable_sort_pushdown: bool, default = true } } diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index 936c02eb2a02d..29c305e5e2bf0 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -28,6 +28,7 @@ mod limit_pushdown; mod limited_distinct_aggregation; mod partition_statistics; mod projection_pushdown; +mod pushdown_sort; mod replace_with_order_preserving_variants; mod sanity_checker; mod test_utils; diff --git a/datafusion/core/tests/physical_optimizer/pushdown_sort.rs b/datafusion/core/tests/physical_optimizer/pushdown_sort.rs new file mode 100644 index 0000000000000..26e3146b77bc9 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/pushdown_sort.rs @@ -0,0 +1,672 @@ +// 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. + +//! Tests for sort pushdown optimizer rule (Phase 1) +//! +//! Phase 1 tests verify that: +//! 1. Reverse scan is enabled (reverse_row_groups=true) +//! 2. SortExec is kept (because ordering is inexact) +//! 3. output_ordering remains unchanged +//! 4. Early termination is enabled for TopK queries +//! 5. Prefix matching works correctly + +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_optimizer::pushdown_sort::PushdownSort; +use datafusion_physical_optimizer::PhysicalOptimizerRule; + +use crate::physical_optimizer::test_utils::{ + coalesce_batches_exec, coalesce_partitions_exec, parquet_exec, + parquet_exec_with_sort, repartition_exec, schema, sort_exec, sort_exec_with_fetch, + sort_expr, OptimizationTest, +}; + +#[test] +fn test_sort_pushdown_disabled() { + // When pushdown is disabled, plan should remain unchanged + let schema = schema(); + let source = parquet_exec(schema.clone()); + let sort_exprs = LexOrdering::new(vec![sort_expr("a", &schema)]).unwrap(); + let plan = sort_exec(sort_exprs, source); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), false), + @r###" + OptimizationTest: + input: + - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + "### + ); +} + +#[test] +fn test_sort_pushdown_basic_phase1() { + // Phase 1: Reverse scan enabled, Sort kept, output_ordering unchanged + let schema = schema(); + + // Source has ASC NULLS LAST ordering (default) + let a = sort_expr("a", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Request DESC NULLS LAST ordering (exact reverse) + let desc_ordering = LexOrdering::new(vec![a.reverse()]).unwrap(); + let plan = sort_exec(desc_ordering, source); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_sort_with_limit_phase1() { + // Phase 1: Sort with fetch enables early termination but keeps Sort + let schema = schema(); + + // Source has ASC ordering + let a = sort_expr("a", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Request DESC ordering with limit + let desc_ordering = LexOrdering::new(vec![a.reverse()]).unwrap(); + let plan = sort_exec_with_fetch(desc_ordering, Some(10), source); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: TopK(fetch=10), expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: TopK(fetch=10), expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_sort_multiple_columns_phase1() { + // Phase 1: Sort on multiple columns - reverse multi-column ordering + let schema = schema(); + + // Source has [a DESC NULLS LAST, b ASC] ordering + let a = sort_expr("a", &schema); + let b = sort_expr("b", &schema); + let source_ordering = LexOrdering::new(vec![a.clone().reverse(), b.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Request [a ASC NULLS FIRST, b DESC] ordering (exact reverse) + let reverse_ordering = + LexOrdering::new(vec![a.clone().asc().nulls_first(), b.reverse()]).unwrap(); + let plan = sort_exec(reverse_ordering, source); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 ASC, b@1 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST, b@1 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 ASC, b@1 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +// ============================================================================ +// PREFIX MATCHING TESTS +// ============================================================================ + +#[test] +fn test_prefix_match_single_column() { + // Test prefix matching: source has [a DESC, b ASC], query needs [a ASC] + // After reverse: [a ASC, b DESC] which satisfies [a ASC] prefix + let schema = schema(); + + // Source has [a DESC NULLS LAST, b ASC NULLS LAST] ordering + let a = sort_expr("a", &schema); + let b = sort_expr("b", &schema); + let source_ordering = LexOrdering::new(vec![a.clone().reverse(), b]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Request only [a ASC NULLS FIRST] - a prefix of the reversed ordering + let prefix_ordering = LexOrdering::new(vec![a.clone().asc().nulls_first()]).unwrap(); + let plan = sort_exec(prefix_ordering, source); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST, b@1 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_prefix_match_with_limit() { + // Test prefix matching with LIMIT - important for TopK optimization + let schema = schema(); + + // Source has [a ASC, b DESC, c ASC] ordering + let a = sort_expr("a", &schema); + let b = sort_expr("b", &schema); + let c = sort_expr("c", &schema); + let source_ordering = + LexOrdering::new(vec![a.clone(), b.clone().reverse(), c]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Request [a DESC NULLS LAST, b ASC NULLS FIRST] with LIMIT 100 + // This is a prefix (2 columns) of the reversed 3-column ordering + let prefix_ordering = + LexOrdering::new(vec![a.reverse(), b.clone().asc().nulls_first()]).unwrap(); + let plan = sort_exec_with_fetch(prefix_ordering, Some(100), source); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: TopK(fetch=100), expr=[a@0 DESC NULLS LAST, b@1 ASC], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 DESC NULLS LAST, c@2 ASC], file_type=parquet + output: + Ok: + - SortExec: TopK(fetch=100), expr=[a@0 DESC NULLS LAST, b@1 ASC], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_prefix_match_through_transparent_nodes() { + // Test prefix matching works through transparent nodes + let schema = schema(); + + // Source has [a DESC NULLS LAST, b ASC, c DESC] ordering + let a = sort_expr("a", &schema); + let b = sort_expr("b", &schema); + let c = sort_expr("c", &schema); + let source_ordering = + LexOrdering::new(vec![a.clone().reverse(), b, c.reverse()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + let coalesce = coalesce_batches_exec(source, 1024); + let repartition = repartition_exec(coalesce); + + // Request only [a ASC NULLS FIRST] - prefix of reversed ordering + let prefix_ordering = LexOrdering::new(vec![a.clone().asc().nulls_first()]).unwrap(); + let plan = sort_exec(prefix_ordering, repartition); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + - CoalesceBatchesExec: target_batch_size=1024 + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST, b@1 ASC, c@2 DESC NULLS LAST], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + - CoalesceBatchesExec: target_batch_size=1024 + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_no_prefix_match_wrong_direction() { + // Test that prefix matching does NOT work if the direction is wrong + let schema = schema(); + + // Source has [a DESC, b ASC] ordering + let a = sort_expr("a", &schema); + let b = sort_expr("b", &schema); + let source_ordering = LexOrdering::new(vec![a.clone().reverse(), b]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Request [a DESC] - same direction as source, NOT a reverse prefix + let same_direction = LexOrdering::new(vec![a.clone().reverse()]).unwrap(); + let plan = sort_exec(same_direction, source); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r###" + OptimizationTest: + input: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST, b@1 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST, b@1 ASC], file_type=parquet + "### + ); +} + +#[test] +fn test_no_prefix_match_longer_than_source() { + // Test that prefix matching does NOT work if requested is longer than source + let schema = schema(); + + // Source has [a DESC] ordering (single column) + let a = sort_expr("a", &schema); + let b = sort_expr("b", &schema); + let source_ordering = LexOrdering::new(vec![a.clone().reverse()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Request [a ASC, b DESC] - longer than source, can't be a prefix + let longer_ordering = + LexOrdering::new(vec![a.clone().asc().nulls_first(), b.reverse()]).unwrap(); + let plan = sort_exec(longer_ordering, source); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r###" + OptimizationTest: + input: + - SortExec: expr=[a@0 ASC, b@1 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 ASC, b@1 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST], file_type=parquet + "### + ); +} + +// ============================================================================ +// ORIGINAL TESTS +// ============================================================================ + +#[test] +fn test_sort_through_coalesce_batches() { + // Sort pushes through CoalesceBatchesExec + let schema = schema(); + let a = sort_expr("a", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + let coalesce = coalesce_batches_exec(source, 1024); + + let desc_ordering = LexOrdering::new(vec![a.reverse()]).unwrap(); + let plan = sort_exec(desc_ordering, coalesce); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - CoalesceBatchesExec: target_batch_size=1024 + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - CoalesceBatchesExec: target_batch_size=1024 + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_sort_through_repartition() { + // Sort should push through RepartitionExec + let schema = schema(); + let a = sort_expr("a", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + let repartition = repartition_exec(source); + + let desc_ordering = LexOrdering::new(vec![a.reverse()]).unwrap(); + let plan = sort_exec(desc_ordering, repartition); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_nested_sorts() { + // Nested sort operations - only innermost can be optimized + let schema = schema(); + let a = sort_expr("a", &schema); + let b = sort_expr("b", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + let desc_ordering = LexOrdering::new(vec![a.reverse()]).unwrap(); + let inner_sort = sort_exec(desc_ordering, source); + + let sort_exprs2 = LexOrdering::new(vec![b]).unwrap(); + let plan = sort_exec(sort_exprs2, inner_sort); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[b@1 ASC], preserve_partitioning=[false] + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[b@1 ASC], preserve_partitioning=[false] + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_non_sort_plans_unchanged() { + // Plans without SortExec should pass through unchanged + let schema = schema(); + let source = parquet_exec(schema.clone()); + let plan = coalesce_batches_exec(source, 1024); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r###" + OptimizationTest: + input: + - CoalesceBatchesExec: target_batch_size=1024 + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + output: + Ok: + - CoalesceBatchesExec: target_batch_size=1024 + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + "### + ); +} + +#[test] +fn test_optimizer_properties() { + // Test optimizer metadata + let optimizer = PushdownSort::new(); + + assert_eq!(optimizer.name(), "PushdownSort"); + assert!(optimizer.schema_check()); +} + +#[test] +fn test_sort_through_coalesce_partitions() { + // Sort should push through CoalescePartitionsExec + let schema = schema(); + let a = sort_expr("a", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + let repartition = repartition_exec(source); + let coalesce_parts = coalesce_partitions_exec(repartition); + + let desc_ordering = LexOrdering::new(vec![a.reverse()]).unwrap(); + let plan = sort_exec(desc_ordering, coalesce_parts); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - CoalescePartitionsExec + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - CoalescePartitionsExec + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_complex_plan_with_multiple_operators() { + // Test a complex plan with multiple operators between sort and source + let schema = schema(); + let a = sort_expr("a", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + let coalesce_batches = coalesce_batches_exec(source, 1024); + let repartition = repartition_exec(coalesce_batches); + let coalesce_parts = coalesce_partitions_exec(repartition); + + let desc_ordering = LexOrdering::new(vec![a.reverse()]).unwrap(); + let plan = sort_exec(desc_ordering, coalesce_parts); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - CoalescePartitionsExec + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + - CoalesceBatchesExec: target_batch_size=1024 + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - CoalescePartitionsExec + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + - CoalesceBatchesExec: target_batch_size=1024 + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_multiple_sorts_different_columns() { + // Test nested sorts on different columns - only innermost can optimize + let schema = schema(); + let a = sort_expr("a", &schema); + let c = sort_expr("c", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // First sort by column 'a' DESC (reverse of source) + let desc_ordering = LexOrdering::new(vec![a.reverse()]).unwrap(); + let sort1 = sort_exec(desc_ordering, source); + + // Then sort by column 'c' (different column, can't optimize) + let sort_exprs2 = LexOrdering::new(vec![c]).unwrap(); + let plan = sort_exec(sort_exprs2, sort1); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_no_pushdown_for_unordered_source() { + // Verify pushdown does NOT happen for sources without ordering + let schema = schema(); + let source = parquet_exec(schema.clone()); // No output_ordering + let sort_exprs = LexOrdering::new(vec![sort_expr("a", &schema)]).unwrap(); + let plan = sort_exec(sort_exprs, source); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r###" + OptimizationTest: + input: + - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + "### + ); +} + +#[test] +fn test_no_pushdown_for_non_reverse_sort() { + // Verify pushdown does NOT happen when sort doesn't reverse source ordering + let schema = schema(); + + // Source sorted by 'a' ASC + let a = sort_expr("a", &schema); + let b = sort_expr("b", &schema); + let source_ordering = LexOrdering::new(vec![a]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Request sort by 'b' (different column) + let sort_exprs = LexOrdering::new(vec![b]).unwrap(); + let plan = sort_exec(sort_exprs, source); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r###" + OptimizationTest: + input: + - SortExec: expr=[b@1 ASC], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[b@1 ASC], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + "### + ); +} + +#[test] +fn test_pushdown_through_blocking_node() { + // Test that pushdown works for inner sort even when outer sort is blocked + // Structure: Sort -> Aggregate (blocks pushdown) -> Sort -> Scan + // The outer sort can't push through aggregate, but the inner sort should still optimize + use datafusion_functions_aggregate::count::count_udaf; + use datafusion_physical_expr::aggregate::AggregateExprBuilder; + use datafusion_physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, + }; + use std::sync::Arc; + + let schema = schema(); + + // Bottom: DataSource with [a ASC NULLS LAST] ordering + let a = sort_expr("a", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Inner Sort: [a DESC NULLS FIRST] - exact reverse, CAN push down to source + let inner_sort_ordering = LexOrdering::new(vec![a.clone().reverse()]).unwrap(); + let inner_sort = sort_exec(inner_sort_ordering, source); + + // Middle: Aggregate (blocks pushdown from outer sort) + // GROUP BY a, COUNT(b) + let group_by = PhysicalGroupBy::new_single(vec![( + Arc::new(datafusion_physical_expr::expressions::Column::new("a", 0)) as _, + "a".to_string(), + )]); + + let count_expr = Arc::new( + AggregateExprBuilder::new( + count_udaf(), + vec![ + Arc::new(datafusion_physical_expr::expressions::Column::new("b", 1)) as _, + ], + ) + .schema(Arc::clone(&schema)) + .alias("COUNT(b)") + .build() + .unwrap(), + ); + + let aggregate = Arc::new( + AggregateExec::try_new( + AggregateMode::Final, + group_by, + vec![count_expr], + vec![None], + inner_sort, + Arc::clone(&schema), + ) + .unwrap(), + ); + + // Outer Sort: [a ASC] - this CANNOT push down through aggregate + let outer_sort_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let plan = sort_exec(outer_sort_ordering, aggregate); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + - AggregateExec: mode=Final, gby=[a@0 as a], aggr=[COUNT(b)], ordering_mode=Sorted + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + - AggregateExec: mode=Final, gby=[a@0 as a], aggr=[COUNT(b)], ordering_mode=Sorted + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 8ca33f3d4abb9..a589caa817943 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -18,7 +18,7 @@ //! Test utilities for physical optimizer tests use std::any::Any; -use std::fmt::Formatter; +use std::fmt::{Display, Formatter}; use std::sync::{Arc, LazyLock}; use arrow::array::Int32Array; @@ -33,7 +33,9 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::stats::Precision; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; -use datafusion_common::{ColumnStatistics, JoinType, NullEquality, Result, Statistics}; +use datafusion_common::{ + internal_err, ColumnStatistics, JoinType, NullEquality, Result, Statistics, +}; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; @@ -704,3 +706,75 @@ impl TestAggregate { } } } + +/// A harness for testing physical optimizers. +#[derive(Debug)] +pub struct OptimizationTest { + input: Vec, + output: Result, String>, +} + +impl OptimizationTest { + pub fn new( + input_plan: Arc, + opt: O, + enable_sort_pushdown: bool, + ) -> Self + where + O: PhysicalOptimizerRule, + { + let input = format_execution_plan(&input_plan); + let input_schema = input_plan.schema(); + + let mut config = ConfigOptions::new(); + config.optimizer.enable_sort_pushdown = enable_sort_pushdown; + let output_result = opt.optimize(input_plan, &config); + let output = output_result + .and_then(|plan| { + if opt.schema_check() && (plan.schema() != input_schema) { + internal_err!( + "Schema mismatch:\n\nBefore:\n{:?}\n\nAfter:\n{:?}", + input_schema, + plan.schema() + ) + } else { + Ok(plan) + } + }) + .map(|plan| format_execution_plan(&plan)) + .map_err(|e| e.to_string()); + + Self { input, output } + } +} + +impl Display for OptimizationTest { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + writeln!(f, "OptimizationTest:")?; + writeln!(f, " input:")?; + for line in &self.input { + writeln!(f, " - {line}")?; + } + writeln!(f, " output:")?; + match &self.output { + Ok(output) => { + writeln!(f, " Ok:")?; + for line in output { + writeln!(f, " - {line}")?; + } + } + Err(err) => { + writeln!(f, " Err: {err}")?; + } + } + Ok(()) + } +} + +pub fn format_execution_plan(plan: &Arc) -> Vec { + format_lines(&displayable(plan.as_ref()).indent(false).to_string()) +} + +fn format_lines(s: &str) -> Vec { + s.trim().split('\n').map(|s| s.to_string()).collect() +} diff --git a/datafusion/datasource-parquet/src/mod.rs b/datafusion/datasource-parquet/src/mod.rs index 2f64f34bc09b4..8bd1ea7936559 100644 --- a/datafusion/datasource-parquet/src/mod.rs +++ b/datafusion/datasource-parquet/src/mod.rs @@ -28,6 +28,7 @@ mod page_filter; mod reader; mod row_filter; mod row_group_filter; +mod sort; pub mod source; mod writer; diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 5701b22ccbcc5..f2165a95b86ce 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -45,6 +45,7 @@ use datafusion_physical_plan::metrics::{ }; use datafusion_pruning::{build_pruning_predicate, FilePruner, PruningPredicate}; +use crate::sort::reverse_row_selection; #[cfg(feature = "parquet_encryption")] use datafusion_common::config::EncryptionFactoryOptions; #[cfg(feature = "parquet_encryption")] @@ -56,7 +57,7 @@ use parquet::arrow::arrow_reader::metrics::ArrowReaderMetrics; use parquet::arrow::arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions}; use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; -use parquet::file::metadata::{PageIndexPolicy, ParquetMetaDataReader}; +use parquet::file::metadata::{PageIndexPolicy, ParquetMetaDataReader, RowGroupMetaData}; /// Implements [`FileOpener`] for a parquet file pub(super) struct ParquetOpener { @@ -113,6 +114,60 @@ pub(super) struct ParquetOpener { /// Maximum size of the predicate cache, in bytes. If none, uses /// the arrow-rs default. pub max_predicate_cache_size: Option, + /// Whether to read row groups in reverse order + pub reverse_row_groups: bool, +} + +/// Represents a prepared access plan with optional row selection +struct PreparedAccessPlan { + /// Row group indexes to read + row_group_indexes: Vec, + /// Optional row selection for filtering within row groups + row_selection: Option, +} + +impl PreparedAccessPlan { + /// Create a new prepared access plan from a ParquetAccessPlan + fn from_access_plan( + access_plan: ParquetAccessPlan, + rg_metadata: &[RowGroupMetaData], + ) -> Result { + let row_group_indexes = access_plan.row_group_indexes(); + let row_selection = access_plan.into_overall_row_selection(rg_metadata)?; + + Ok(Self { + row_group_indexes, + row_selection, + }) + } + + /// Reverse the access plan for reverse scanning + fn reverse( + mut self, + file_metadata: &parquet::file::metadata::ParquetMetaData, + ) -> Result { + // Reverse the row group indexes + self.row_group_indexes = self.row_group_indexes.into_iter().rev().collect(); + + // If we have a row selection, reverse it to match the new row group order + if let Some(row_selection) = self.row_selection { + self.row_selection = + Some(reverse_row_selection(&row_selection, file_metadata)?); + } + + Ok(self) + } + + /// Apply this access plan to a ParquetRecordBatchStreamBuilder + fn apply_to_builder( + self, + mut builder: ParquetRecordBatchStreamBuilder>, + ) -> ParquetRecordBatchStreamBuilder> { + if let Some(row_selection) = self.row_selection { + builder = builder.with_row_selection(row_selection); + } + builder.with_row_groups(self.row_group_indexes) + } } impl FileOpener for ParquetOpener { @@ -166,6 +221,7 @@ impl FileOpener for ParquetOpener { let encryption_context = self.get_encryption_context(); let max_predicate_cache_size = self.max_predicate_cache_size; + let reverse_row_groups = self.reverse_row_groups; Ok(Box::pin(async move { #[cfg(feature = "parquet_encryption")] let file_decryption_properties = encryption_context @@ -433,13 +489,18 @@ impl FileOpener for ParquetOpener { } } - let row_group_indexes = access_plan.row_group_indexes(); - if let Some(row_selection) = - access_plan.into_overall_row_selection(rg_metadata)? - { - builder = builder.with_row_selection(row_selection); + // Prepare the access plan (extract row groups and row selection) + let mut prepared_plan = + PreparedAccessPlan::from_access_plan(access_plan, rg_metadata)?; + + // If reverse scanning is enabled, reverse the prepared plan + if reverse_row_groups { + prepared_plan = prepared_plan.reverse(file_metadata.as_ref())?; } + // Apply the prepared plan to the builder + builder = prepared_plan.apply_to_builder(builder); + if let Some(limit) = limit { builder = builder.with_limit(limit) } @@ -454,7 +515,6 @@ impl FileOpener for ParquetOpener { let stream = builder .with_projection(mask) .with_batch_size(batch_size) - .with_row_groups(row_group_indexes) .with_metrics(arrow_reader_metrics.clone()) .build()?; @@ -763,23 +823,19 @@ fn should_enable_page_index( #[cfg(test)] mod test { - use std::sync::Arc; - - use arrow::{ - compute::cast, - datatypes::{DataType, Field, Schema, SchemaRef}, - }; + use crate::{opener::ParquetOpener, DefaultParquetFileReaderFactory}; + use arrow::compute::cast; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use bytes::{BufMut, BytesMut}; use datafusion_common::{ assert_batches_eq, record_batch, stats::Precision, ColumnStatistics, DataFusionError, ScalarValue, Statistics, }; + use datafusion_datasource::schema_adapter::{ + SchemaAdapter, SchemaAdapterFactory, SchemaMapper, + }; use datafusion_datasource::{ - file_stream::FileOpener, - schema_adapter::{ - DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory, - SchemaMapper, - }, + file_stream::FileOpener, schema_adapter::DefaultSchemaAdapterFactory, PartitionedFile, }; use datafusion_expr::{col, lit}; @@ -791,8 +847,8 @@ mod test { use futures::{Stream, StreamExt}; use object_store::{memory::InMemory, path::Path, ObjectStore}; use parquet::arrow::ArrowWriter; - - use crate::{opener::ParquetOpener, DefaultParquetFileReaderFactory}; + use parquet::file::properties::WriterProperties; + use std::sync::Arc; async fn count_batches_and_rows( mut stream: std::pin::Pin< @@ -811,31 +867,54 @@ mod test { (num_batches, num_rows) } - async fn collect_batches( + /// Helper to collect all int32 values from the first column of batches + async fn collect_int32_values( mut stream: std::pin::Pin< Box< dyn Stream> + Send, >, >, - ) -> Vec { - let mut batches = vec![]; + ) -> Vec { + use arrow::array::Array; + let mut values = vec![]; while let Some(Ok(batch)) = stream.next().await { - batches.push(batch); + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + values.push(array.value(i)); + } + } } - batches + values } async fn write_parquet( store: Arc, filename: &str, batch: arrow::record_batch::RecordBatch, + ) -> usize { + write_parquet_batches(store, filename, vec![batch], None).await + } + + /// Write multiple batches to a parquet file with optional writer properties + async fn write_parquet_batches( + store: Arc, + filename: &str, + batches: Vec, + props: Option, ) -> usize { let mut out = BytesMut::new().writer(); { - let mut writer = - ArrowWriter::try_new(&mut out, batch.schema(), None).unwrap(); - writer.write(&batch).unwrap(); + let schema = batches[0].schema(); + let mut writer = ArrowWriter::try_new(&mut out, schema, props).unwrap(); + for batch in batches { + writer.write(&batch).unwrap(); + } writer.finish().unwrap(); } let data = out.into_inner().freeze(); @@ -908,6 +987,7 @@ mod test { #[cfg(feature = "parquet_encryption")] encryption_factory: None, max_predicate_cache_size: None, + reverse_row_groups: false, } }; @@ -982,6 +1062,7 @@ mod test { #[cfg(feature = "parquet_encryption")] encryption_factory: None, max_predicate_cache_size: None, + reverse_row_groups: false, } }; @@ -1072,6 +1153,7 @@ mod test { #[cfg(feature = "parquet_encryption")] encryption_factory: None, max_predicate_cache_size: None, + reverse_row_groups: false, } }; @@ -1165,6 +1247,7 @@ mod test { #[cfg(feature = "parquet_encryption")] encryption_factory: None, max_predicate_cache_size: None, + reverse_row_groups: false, } }; @@ -1258,6 +1341,7 @@ mod test { #[cfg(feature = "parquet_encryption")] encryption_factory: None, max_predicate_cache_size: None, + reverse_row_groups: false, } }; @@ -1278,6 +1362,20 @@ mod test { assert_eq!(num_batches, 0); assert_eq!(num_rows, 0); } + async fn collect_batches( + mut stream: std::pin::Pin< + Box< + dyn Stream> + + Send, + >, + >, + ) -> Vec { + let mut batches = vec![]; + while let Some(Ok(batch)) = stream.next().await { + batches.push(batch); + } + batches + } fn get_value(metrics: &MetricsSet, metric_name: &str) -> usize { match metrics.sum_by_name(metric_name) { @@ -1409,6 +1507,7 @@ mod test { #[cfg(feature = "parquet_encryption")] encryption_factory: None, max_predicate_cache_size: None, + reverse_row_groups: false, }; let predicate = logical2physical(&col("a").eq(lit(1u64)), &table_schema); @@ -1429,4 +1528,199 @@ mod test { assert_eq!(get_value(&metrics, "row_groups_pruned_statistics"), 0); assert_eq!(get_value(&metrics, "pushdown_rows_pruned"), 2); } + + #[tokio::test] + async fn test_reverse_scan_row_groups() { + use parquet::file::properties::WriterProperties; + + let store = Arc::new(InMemory::new()) as Arc; + + // Create multiple batches to ensure multiple row groups + let batch1 = + record_batch!(("a", Int32, vec![Some(1), Some(2), Some(3)])).unwrap(); + let batch2 = + record_batch!(("a", Int32, vec![Some(4), Some(5), Some(6)])).unwrap(); + let batch3 = + record_batch!(("a", Int32, vec![Some(7), Some(8), Some(9)])).unwrap(); + + // Write parquet file with multiple row groups + // Force small row groups by setting max_row_group_size + let props = WriterProperties::builder() + .set_max_row_group_size(3) // Force each batch into its own row group + .build(); + + let data_len = write_parquet_batches( + Arc::clone(&store), + "test.parquet", + vec![batch1.clone(), batch2, batch3], + Some(props), + ) + .await; + + let schema = batch1.schema(); + let file = PartitionedFile::new( + "test.parquet".to_string(), + u64::try_from(data_len).unwrap(), + ); + + let make_opener = |reverse_scan: bool| ParquetOpener { + projection: Arc::new([0]), + partition_index: 0, + batch_size: 1024, + limit: None, + predicate: None, + metadata_size_hint: None, + metrics: ExecutionPlanMetricsSet::new(), + parquet_file_reader_factory: Arc::new(DefaultParquetFileReaderFactory::new( + Arc::clone(&store), + )), + partition_fields: vec![], + pushdown_filters: true, + reorder_filters: false, + enable_page_index: false, + enable_bloom_filter: false, + enable_limit_pruning: false, + schema_adapter_factory: Arc::new(DefaultSchemaAdapterFactory), + enable_row_group_stats_pruning: false, + coerce_int96: None, + #[cfg(feature = "parquet_encryption")] + file_decryption_properties: None, + expr_adapter_factory: Some(Arc::new(DefaultPhysicalExprAdapterFactory)), + #[cfg(feature = "parquet_encryption")] + encryption_factory: None, + max_predicate_cache_size: None, + reverse_row_groups: reverse_scan, + logical_file_schema: schema.clone(), + }; + + // Test normal scan (forward) + let opener = make_opener(false); + let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let forward_values = collect_int32_values(stream).await; + + // Test reverse scan + let opener = make_opener(true); + let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let reverse_values = collect_int32_values(stream).await; + + // The forward scan should return data in the order written + assert_eq!(forward_values, vec![1, 2, 3, 4, 5, 6, 7, 8, 9]); + + // With reverse scan, row groups are reversed, so we expect: + // Row group 3 (7,8,9), then row group 2 (4,5,6), then row group 1 (1,2,3) + assert_eq!(reverse_values, vec![7, 8, 9, 4, 5, 6, 1, 2, 3]); + } + + #[tokio::test] + async fn test_reverse_scan_with_row_selection() { + use parquet::file::properties::WriterProperties; + + let store = Arc::new(InMemory::new()) as Arc; + + // Create 3 batches with DIFFERENT selection patterns + let batch1 = + record_batch!(("a", Int32, vec![Some(1), Some(2), Some(3), Some(4)])) + .unwrap(); // 4 rows + let batch2 = + record_batch!(("a", Int32, vec![Some(5), Some(6), Some(7), Some(8)])) + .unwrap(); // 4 rows + let batch3 = + record_batch!(("a", Int32, vec![Some(9), Some(10), Some(11), Some(12)])) + .unwrap(); // 4 rows + + let props = WriterProperties::builder() + .set_max_row_group_size(4) + .build(); + + let data_len = write_parquet_batches( + Arc::clone(&store), + "test.parquet", + vec![batch1.clone(), batch2, batch3], + Some(props), + ) + .await; + + let schema = batch1.schema(); + + use crate::ParquetAccessPlan; + use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; + + let mut access_plan = ParquetAccessPlan::new_all(3); + // Row group 0: skip first 2, select last 2 (should get: 3, 4) + access_plan.scan_selection( + 0, + RowSelection::from(vec![RowSelector::skip(2), RowSelector::select(2)]), + ); + // Row group 1: select all (should get: 5, 6, 7, 8) + // Row group 2: select first 2, skip last 2 (should get: 9, 10) + access_plan.scan_selection( + 2, + RowSelection::from(vec![RowSelector::select(2), RowSelector::skip(2)]), + ); + + let file = PartitionedFile::new( + "test.parquet".to_string(), + u64::try_from(data_len).unwrap(), + ) + .with_extensions(Arc::new(access_plan)); + + let make_opener = |reverse_scan: bool| ParquetOpener { + projection: Arc::new([0]), + partition_index: 0, + batch_size: 1024, + limit: None, + predicate: None, + metadata_size_hint: None, + metrics: ExecutionPlanMetricsSet::new(), + parquet_file_reader_factory: Arc::new(DefaultParquetFileReaderFactory::new( + Arc::clone(&store), + )), + partition_fields: vec![], + pushdown_filters: true, + reorder_filters: false, + enable_page_index: false, + enable_bloom_filter: false, + enable_limit_pruning: false, + schema_adapter_factory: Arc::new(DefaultSchemaAdapterFactory), + enable_row_group_stats_pruning: false, + coerce_int96: None, + #[cfg(feature = "parquet_encryption")] + file_decryption_properties: None, + expr_adapter_factory: Some(Arc::new(DefaultPhysicalExprAdapterFactory)), + #[cfg(feature = "parquet_encryption")] + encryption_factory: None, + max_predicate_cache_size: None, + reverse_row_groups: reverse_scan, + logical_file_schema: schema.clone(), + }; + + // Forward scan: RG0(3,4), RG1(5,6,7,8), RG2(9,10) + let opener = make_opener(false); + let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let forward_values = collect_int32_values(stream).await; + + // Forward scan should produce: RG0(3,4), RG1(5,6,7,8), RG2(9,10) + assert_eq!( + forward_values, + vec![3, 4, 5, 6, 7, 8, 9, 10], + "Forward scan should select correct rows based on RowSelection" + ); + + // Reverse scan + // CORRECT behavior: reverse row groups AND their corresponding selections + // - RG2 is read first, WITH RG2's selection (select 2, skip 2) -> 9, 10 + // - RG1 is read second, WITH RG1's selection (select all) -> 5, 6, 7, 8 + // - RG0 is read third, WITH RG0's selection (skip 2, select 2) -> 3, 4 + let opener = make_opener(true); + let stream = opener.open(file).unwrap().await.unwrap(); + let reverse_values = collect_int32_values(stream).await; + + // Correct expected result: row groups reversed but each keeps its own selection + // RG2 with its selection (9,10), RG1 with its selection (5,6,7,8), RG0 with its selection (3,4) + assert_eq!( + reverse_values, + vec![9, 10, 5, 6, 7, 8, 3, 4], + "Reverse scan should reverse row group order while maintaining correct RowSelection for each group" + ); + } } diff --git a/datafusion/datasource-parquet/src/sort.rs b/datafusion/datasource-parquet/src/sort.rs new file mode 100644 index 0000000000000..4255d4d6960b1 --- /dev/null +++ b/datafusion/datasource-parquet/src/sort.rs @@ -0,0 +1,407 @@ +// 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. + +//! Sort-related utilities for Parquet scanning + +use datafusion_common::Result; +use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; +use parquet::file::metadata::ParquetMetaData; +use std::collections::HashMap; + +/// Reverse a row selection to match reversed row group order. +/// +/// When scanning row groups in reverse order, we need to adjust the row selection +/// to account for the new ordering. This function: +/// 1. Maps each selection to its corresponding row group +/// 2. Reverses the order of row groups +/// 3. Reconstructs the row selection for the new order +/// +/// # Arguments +/// * `row_selection` - Original row selection +/// * `parquet_metadata` - Metadata containing row group information +/// +/// # Returns +/// A new `RowSelection` adjusted for reversed row group order +pub fn reverse_row_selection( + row_selection: &RowSelection, + parquet_metadata: &ParquetMetaData, +) -> Result { + let rg_metadata = parquet_metadata.row_groups(); + + // Build a mapping of row group index to its row range in the file + let mut rg_row_ranges: Vec<(usize, usize, usize)> = + Vec::with_capacity(rg_metadata.len()); + let mut current_row = 0; + for (rg_idx, rg) in rg_metadata.iter().enumerate() { + let num_rows = rg.num_rows() as usize; + rg_row_ranges.push((rg_idx, current_row, current_row + num_rows)); + current_row += num_rows; + } + + // Map selections to row groups + let mut rg_selections: HashMap> = HashMap::new(); + + let mut current_file_row = 0; + for selector in row_selection.iter() { + let selector_end = current_file_row + selector.row_count; + + // Find which row groups this selector spans + for (rg_idx, rg_start, rg_end) in rg_row_ranges.iter() { + if current_file_row < *rg_end && selector_end > *rg_start { + // This selector overlaps with this row group + let overlap_start = current_file_row.max(*rg_start); + let overlap_end = selector_end.min(*rg_end); + let overlap_count = overlap_end - overlap_start; + + if overlap_count > 0 { + let entry = rg_selections.entry(*rg_idx).or_default(); + if selector.skip { + entry.push(RowSelector::skip(overlap_count)); + } else { + entry.push(RowSelector::select(overlap_count)); + } + } + } + } + + current_file_row = selector_end; + } + + // Build new selection for reversed row group order + let mut reversed_selectors = Vec::new(); + for rg_idx in (0..rg_metadata.len()).rev() { + if let Some(selectors) = rg_selections.get(&rg_idx) { + reversed_selectors.extend(selectors.iter().cloned()); + } else { + // No specific selection for this row group means select all + if let Some((_, start, end)) = + rg_row_ranges.iter().find(|(idx, _, _)| *idx == rg_idx) + { + reversed_selectors.push(RowSelector::select(end - start)); + } + } + } + + Ok(RowSelection::from(reversed_selectors)) +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::datatypes::{DataType, Field, Schema}; + use bytes::Bytes; + use parquet::arrow::ArrowWriter; + use parquet::file::reader::FileReader; + use parquet::file::serialized_reader::SerializedFileReader; + use std::sync::Arc; + + /// Helper function to create a ParquetMetaData with specified row group sizes + /// by actually writing a parquet file in memory + fn create_test_metadata(row_group_sizes: Vec) -> ParquetMetaData { + // Create a simple schema + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + + // Create in-memory parquet file with the specified row groups + let mut buffer = Vec::new(); + { + let props = parquet::file::properties::WriterProperties::builder() + .set_max_row_group_size(row_group_sizes[0] as usize) + .build(); + + let mut writer = + ArrowWriter::try_new(&mut buffer, schema.clone(), Some(props)).unwrap(); + + for &size in &row_group_sizes { + // Create a batch with the specified number of rows + let array = arrow::array::Int32Array::from(vec![1; size as usize]); + let batch = arrow::record_batch::RecordBatch::try_new( + schema.clone(), + vec![Arc::new(array)], + ) + .unwrap(); + writer.write(&batch).unwrap(); + } + writer.close().unwrap(); + } + + // Read back the metadata + let bytes = Bytes::from(buffer); + let reader = SerializedFileReader::new(bytes).unwrap(); + reader.metadata().clone() + } + + #[test] + fn test_reverse_simple_selection() { + // 3 row groups with 100 rows each + let metadata = create_test_metadata(vec![100, 100, 100]); + + // Select first 50 rows from first row group + let selection = + RowSelection::from(vec![RowSelector::select(50), RowSelector::skip(250)]); + + let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + + // Verify total selected rows remain the same + let original_selected: usize = selection + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + let reversed_selected: usize = reversed + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + + assert_eq!(original_selected, reversed_selected); + } + + #[test] + fn test_reverse_multi_row_group_selection() { + let metadata = create_test_metadata(vec![100, 100, 100]); + + // Select rows spanning multiple row groups + let selection = RowSelection::from(vec![ + RowSelector::skip(50), + RowSelector::select(100), // Spans RG0 and RG1 + RowSelector::skip(150), + ]); + + let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + + // Verify total selected rows remain the same + let original_selected: usize = selection + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + let reversed_selected: usize = reversed + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + + assert_eq!(original_selected, reversed_selected); + } + + #[test] + fn test_reverse_full_selection() { + let metadata = create_test_metadata(vec![100, 100, 100]); + + // Select all rows + let selection = RowSelection::from(vec![RowSelector::select(300)]); + + let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + + // Should still select all rows, just in reversed row group order + let total_selected: usize = reversed + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + + assert_eq!(total_selected, 300); + } + + #[test] + fn test_reverse_empty_selection() { + let metadata = create_test_metadata(vec![100, 100, 100]); + + // Skip all rows + let selection = RowSelection::from(vec![RowSelector::skip(300)]); + + let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + + // Should still skip all rows + let total_selected: usize = reversed + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + + assert_eq!(total_selected, 0); + } + + #[test] + fn test_reverse_with_different_row_group_sizes() { + let metadata = create_test_metadata(vec![50, 150, 100]); + + let selection = RowSelection::from(vec![ + RowSelector::skip(25), + RowSelector::select(200), // Spans all row groups + RowSelector::skip(75), + ]); + + let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + + let original_selected: usize = selection + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + let reversed_selected: usize = reversed + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + + assert_eq!(original_selected, reversed_selected); + } + + #[test] + fn test_reverse_single_row_group() { + let metadata = create_test_metadata(vec![100]); + + let selection = + RowSelection::from(vec![RowSelector::select(50), RowSelector::skip(50)]); + + let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + + // With single row group, selection should remain the same + let original_selected: usize = selection + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + let reversed_selected: usize = reversed + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + + assert_eq!(original_selected, reversed_selected); + } + + #[test] + fn test_reverse_complex_pattern() { + let metadata = create_test_metadata(vec![100, 100, 100]); + + // Complex pattern: select some, skip some, select some more + let selection = RowSelection::from(vec![ + RowSelector::select(30), + RowSelector::skip(40), + RowSelector::select(80), + RowSelector::skip(50), + RowSelector::select(100), + ]); + + let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + + let original_selected: usize = selection + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + let reversed_selected: usize = reversed + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + + assert_eq!(original_selected, reversed_selected); + assert_eq!(original_selected, 210); // 30 + 80 + 100 + } + + #[test] + fn test_reverse_with_skipped_row_group() { + // This test covers the "no specific selection" code path (lines 90-95) + let metadata = create_test_metadata(vec![100, 100, 100]); + + // Select only from first and third row groups, skip middle one entirely + let selection = RowSelection::from(vec![ + RowSelector::select(50), // First 50 of RG0 + RowSelector::skip(150), // Rest of RG0 + all of RG1 + half of RG2 + RowSelector::select(50), // Last 50 of RG2 + ]); + + let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + + // Verify total selected rows remain the same + let original_selected: usize = selection + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + let reversed_selected: usize = reversed + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + + assert_eq!(original_selected, reversed_selected); + assert_eq!(original_selected, 100); // 50 + 50 + } + + #[test] + fn test_reverse_middle_row_group_only() { + // Another test to ensure skipped row groups are handled correctly + let metadata = create_test_metadata(vec![100, 100, 100]); + + // Select only middle row group + let selection = RowSelection::from(vec![ + RowSelector::skip(100), // Skip RG0 + RowSelector::select(100), // Select all of RG1 + RowSelector::skip(100), // Skip RG2 + ]); + + let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + + let original_selected: usize = selection + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + let reversed_selected: usize = reversed + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + + assert_eq!(original_selected, reversed_selected); + assert_eq!(original_selected, 100); + } + + #[test] + fn test_reverse_alternating_row_groups() { + // Test with more complex skipping pattern + let metadata = create_test_metadata(vec![100, 100, 100, 100]); + + // Select first and third row groups, skip second and fourth + let selection = RowSelection::from(vec![ + RowSelector::select(100), // RG0 + RowSelector::skip(100), // RG1 + RowSelector::select(100), // RG2 + RowSelector::skip(100), // RG3 + ]); + + let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + + let original_selected: usize = selection + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + let reversed_selected: usize = reversed + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + + assert_eq!(original_selected, reversed_selected); + assert_eq!(original_selected, 200); + } +} diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 339d36b57cc35..145b08e4595a3 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -43,8 +43,7 @@ use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::TableSchema; use datafusion_physical_expr::conjunction; use datafusion_physical_expr_adapter::DefaultPhysicalExprAdapterFactory; -use datafusion_physical_expr_common::physical_expr::fmt_sql; -use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::physical_expr::{fmt_sql, PhysicalExpr}; use datafusion_physical_plan::filter_pushdown::PushedDown; use datafusion_physical_plan::filter_pushdown::{ FilterPushdownPropagation, PushedDownPredicate, @@ -52,9 +51,11 @@ use datafusion_physical_plan::filter_pushdown::{ use datafusion_physical_plan::metrics::Count; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion_physical_plan::DisplayFormatType; +use datafusion_physical_plan::SortOrderPushdownResult; #[cfg(feature = "parquet_encryption")] use datafusion_execution::parquet_encryption::EncryptionFactory; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use itertools::Itertools; use object_store::ObjectStore; #[cfg(feature = "parquet_encryption")] @@ -289,6 +290,14 @@ pub struct ParquetSource { pub(crate) projected_statistics: Option, #[cfg(feature = "parquet_encryption")] pub(crate) encryption_factory: Option>, + /// The ordering of data within the files + /// This is set by FileScanConfig when it knows the file ordering + file_ordering: Option, + /// If true, read files in reverse order and reverse row groups within files. + /// But it's not guaranteed that rows within row groups are in reverse order, + /// 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, } impl ParquetSource { @@ -297,8 +306,19 @@ impl ParquetSource { /// if default values are going to be used, use `ParguetConfig::default()` instead pub fn new(table_parquet_options: TableParquetOptions) -> Self { Self { + table_schema: None, table_parquet_options, - ..Self::default() + metrics: ExecutionPlanMetricsSet::new(), + predicate: None, + parquet_file_reader_factory: None, + schema_adapter_factory: None, + batch_size: None, + metadata_size_hint: None, + projected_statistics: None, + #[cfg(feature = "parquet_encryption")] + encryption_factory: None, + file_ordering: None, + reverse_row_groups: false, } } @@ -364,6 +384,12 @@ impl ParquetSource { self } + /// If set, indicates the ordering of data within the files being read. + pub fn with_file_ordering(mut self, ordering: Option) -> Self { + self.file_ordering = ordering; + self + } + /// Return the value described in [`Self::with_pushdown_filters`] pub(crate) fn pushdown_filters(&self) -> bool { self.table_parquet_options.global.pushdown_filters @@ -460,6 +486,15 @@ impl ParquetSource { )), } } + + pub(crate) fn with_reverse_row_groups(mut self, reverse_row_groups: bool) -> Self { + self.reverse_row_groups = reverse_row_groups; + self + } + #[cfg(test)] + pub(crate) fn reverse_row_groups(&self) -> bool { + self.reverse_row_groups + } } /// Parses datafusion.common.config.ParquetOptions.coerce_int96 String to a arrow_schema.datatype.TimeUnit @@ -584,6 +619,7 @@ impl FileSource for ParquetSource { #[cfg(feature = "parquet_encryption")] encryption_factory: self.get_encryption_factory_with_config(), max_predicate_cache_size: self.max_predicate_cache_size(), + reverse_row_groups: self.reverse_row_groups, }) } @@ -653,6 +689,11 @@ impl FileSource for ParquetSource { write!(f, "{predicate_string}")?; + // Add reverse_scan info if enabled + if self.reverse_row_groups { + write!(f, ", reverse_row_groups=true")?; + } + // Try to build a the pruning predicates. // These are only generated here because it's useful to have *some* // idea of what pushdown is happening when viewing plans. @@ -786,6 +827,72 @@ impl FileSource for ParquetSource { fn schema_adapter_factory(&self) -> Option> { self.schema_adapter_factory.clone() } + + /// Try to optimize the scan to produce data in the requested sort order. + /// + /// This method receives: + /// 1. The query's required ordering (`order` parameter) + /// 2. The file's natural ordering (via `self.file_ordering`, set by FileScanConfig) + /// + /// With both pieces of information, ParquetSource can decide what optimizations to apply. + /// + /// # Phase 1 Behavior (Current) + /// Returns `Inexact` when reversing the row group scan order would help satisfy the + /// requested ordering. We still need a Sort operator at a higher level because: + /// - We only reverse row group read order, not rows within row groups + /// - This provides approximate ordering that benefits limit pushdown + /// + /// # Phase 2 (Future) + /// Could return `Exact` when we can guarantee perfect ordering through techniques like: + /// - File reordering based on statistics + /// - Detecting already-sorted data + /// This would allow removing the Sort operator entirely. + /// + /// # Returns + /// - `Inexact`: Created an optimized source (e.g., reversed scan) that approximates the order + /// - `Unsupported`: Cannot optimize for this ordering + fn try_reverse_output( + &self, + order: &[PhysicalSortExpr], + ) -> datafusion_common::Result>> { + // Check if we have file ordering information + let file_ordering = match &self.file_ordering { + Some(ordering) => ordering, + None => return Ok(SortOrderPushdownResult::Unsupported), + }; + + // Create a LexOrdering from the requested order to use the is_reverse method + let Some(requested_ordering) = LexOrdering::new(order.to_vec()) else { + // Empty ordering requested, cannot optimize + return Ok(SortOrderPushdownResult::Unsupported); + }; + + // Check if reversing the file ordering would satisfy the requested ordering + if file_ordering.is_reverse(&requested_ordering) { + // Phase 1: Enable reverse row group scanning + let new_source = self.clone().with_reverse_row_groups(true); + + // Return Inexact because we're only reversing row group order, + // not guaranteeing perfect row-level ordering + return Ok(SortOrderPushdownResult::Inexact { + inner: Arc::new(new_source) as Arc, + }); + } + + // TODO Phase 2: Add support for other optimizations: + // - File reordering based on min/max statistics + // - Detection of exact ordering (return Exact to remove Sort operator) + // - Partial sort pushdown for prefix matches + + Ok(SortOrderPushdownResult::Unsupported) + } + + fn with_file_ordering_info( + &self, + ordering: Option, + ) -> datafusion_common::Result> { + Ok(Arc::new(self.clone().with_file_ordering(ordering))) + } } #[cfg(test)] @@ -802,4 +909,74 @@ mod tests { // same value. but filter() call Arc::clone internally assert_eq!(parquet_source.predicate(), parquet_source.filter().as_ref()); } + + #[test] + fn test_reverse_scan_default_value() { + let options = TableParquetOptions::default(); + let source = ParquetSource::new(options); + + assert!(!source.reverse_row_groups()); + } + + #[test] + fn test_reverse_scan_with_setter() { + let options = TableParquetOptions::default(); + + let source = ParquetSource::new(options).with_reverse_row_groups(true); + assert!(source.reverse_row_groups()); + + let source = source.with_reverse_row_groups(false); + assert!(!source.reverse_row_groups()); + } + + #[test] + fn test_reverse_scan_clone_preserves_value() { + let options = TableParquetOptions::default(); + + let source = ParquetSource::new(options).with_reverse_row_groups(true); + let cloned = source.clone(); + + assert!(cloned.reverse_row_groups()); + assert_eq!(source.reverse_row_groups(), cloned.reverse_row_groups()); + } + + #[test] + fn test_reverse_scan_with_other_options() { + use datafusion_common::config::TableParquetOptions; + + let options = TableParquetOptions::default(); + + let source = ParquetSource::new(options) + .with_metadata_size_hint(8192) + .with_reverse_row_groups(true); + + assert!(source.reverse_row_groups()); + assert_eq!(source.metadata_size_hint, Some(8192)); + } + + #[test] + fn test_reverse_scan_builder_pattern() { + let options = TableParquetOptions::default(); + + let source = ParquetSource::new(options) + .with_reverse_row_groups(true) + .with_reverse_row_groups(false) + .with_reverse_row_groups(true); + + assert!(source.reverse_row_groups()); + } + + #[test] + fn test_reverse_scan_independent_of_predicate() { + let options = TableParquetOptions::default(); + + let predicate = lit(true); + + let source = ParquetSource::new(options) + .with_predicate(predicate) + .with_reverse_row_groups(true); + + assert!(source.reverse_row_groups()); + assert!(source.filter().is_some()); + } } diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index d6ade3b8b2107..db781910c81f6 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -30,10 +30,11 @@ use crate::TableSchema; use datafusion_common::config::ConfigOptions; use datafusion_common::{not_impl_err, Result, Statistics}; use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use datafusion_physical_plan::filter_pushdown::{FilterPushdownPropagation, PushedDown}; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion_physical_plan::DisplayFormatType; - +use datafusion_physical_plan::SortOrderPushdownResult; use object_store::ObjectStore; /// Helper function to convert any type implementing FileSource to Arc<dyn FileSource> @@ -129,6 +130,21 @@ pub trait FileSource: Send + Sync { )) } + /// Try to create a new FileSource that can produce data in the specified sort order. + /// + /// # Returns + /// * `Exact` - Created a source that guarantees perfect ordering + /// * `Inexact` - Created a source optimized for ordering (e.g., reordered files) but not perfectly sorted + /// * `Unsupported` - Cannot optimize for this ordering + /// + /// Default implementation returns `Unsupported`. + fn try_reverse_output( + &self, + _order: &[PhysicalSortExpr], + ) -> Result>> { + Ok(SortOrderPushdownResult::Unsupported) + } + /// Set optional schema adapter factory. /// /// [`SchemaAdapterFactory`] allows user to specify how fields from the @@ -154,4 +170,22 @@ pub trait FileSource: Send + Sync { fn schema_adapter_factory(&self) -> Option> { None } + + /// Set the file ordering information + /// + /// This allows the file source to know how the files are sorted, + /// enabling it to make informed decisions about sort pushdown. + /// + /// # Default Implementation + /// + /// Returns `not_impl_err!`. FileSource implementations that support + /// sort optimization should override this method. + fn with_file_ordering_info( + &self, + _ordering: Option, + ) -> Result> { + // Default: clone self without modification + // ParquetSource will override this + not_impl_err!("with_file_ordering_info not implemented for this FileSource") + } } diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 02d9762a4a396..fb1e66e026a1a 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -44,31 +44,33 @@ use datafusion_execution::{ object_store::ObjectStoreUrl, SendableRecordBatchStream, TaskContext, }; use datafusion_expr::Operator; + use datafusion_physical_expr::expressions::{BinaryExpr, Column}; -use datafusion_physical_expr::projection::ProjectionExprs; +use datafusion_physical_expr::projection::{ProjectionExpr, ProjectionExprs}; use datafusion_physical_expr::utils::reassign_expr_columns; use datafusion_physical_expr::{split_conjunction, EquivalenceProperties, Partitioning}; use datafusion_physical_expr_adapter::PhysicalExprAdapterFactory; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_plan::projection::{ - all_alias_free_columns, new_projections_for_columns, ProjectionExpr, -}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_plan::SortOrderPushdownResult; use datafusion_physical_plan::{ display::{display_orderings, ProjectSchemaDisplay}, filter_pushdown::FilterPushdownPropagation, metrics::ExecutionPlanMetricsSet, DisplayAs, DisplayFormatType, }; -use std::{ - any::Any, borrow::Cow, collections::HashMap, fmt::Debug, fmt::Formatter, - fmt::Result as FmtResult, marker::PhantomData, sync::Arc, -}; use datafusion_physical_expr::equivalence::project_orderings; use datafusion_physical_plan::coop::cooperative; use datafusion_physical_plan::execution_plan::SchedulingType; +use datafusion_physical_plan::projection::{ + all_alias_free_columns, new_projections_for_columns, +}; use log::{debug, warn}; +use std::borrow::Cow; +use std::collections::HashMap; +use std::marker::PhantomData; +use std::{any::Any, fmt::Debug, fmt::Formatter, fmt::Result as FmtResult, sync::Arc}; /// The base configurations for a [`DataSourceExec`], the a physical plan for /// any given file format. @@ -757,6 +759,45 @@ impl DataSource for FileScanConfig { } } } + + fn try_pushdown_sort( + &self, + order: &[PhysicalSortExpr], + ) -> Result>> { + let file_ordering = self.output_ordering.first().cloned(); + + if file_ordering.is_none() { + return Ok(SortOrderPushdownResult::Unsupported); + } + + // Use the trait method instead of downcasting + // Try to provide file ordering info to the source + // If not supported (e.g., CsvSource), fall back to original source + let file_source_with_ordering = self + .file_source + .with_file_ordering_info(file_ordering) + .unwrap_or_else(|_| Arc::clone(&self.file_source)); + + // Try to reverse the datasource with ordering info, + // and currently only ParquetSource supports it with inexact reverse with row groups. + let pushdown_result = file_source_with_ordering.try_reverse_output(order)?; + + match pushdown_result { + SortOrderPushdownResult::Exact { inner } => { + Ok(SortOrderPushdownResult::Exact { + inner: self.rebuild_with_source(inner, true)?, + }) + } + SortOrderPushdownResult::Inexact { inner } => { + Ok(SortOrderPushdownResult::Inexact { + inner: self.rebuild_with_source(inner, false)?, + }) + } + SortOrderPushdownResult::Unsupported => { + Ok(SortOrderPushdownResult::Unsupported) + } + } + } } impl FileScanConfig { @@ -1100,6 +1141,36 @@ impl FileScanConfig { pub fn file_source(&self) -> &Arc { &self.file_source } + + /// Helper: Rebuild FileScanConfig with new file source + fn rebuild_with_source( + &self, + new_file_source: Arc, + is_exact: bool, + ) -> Result> { + let mut new_config = self.clone(); + + // Reverse file groups (FileScanConfig's responsibility) + new_config.file_groups = new_config + .file_groups + .into_iter() + .map(|group| { + let mut files = group.into_inner(); + files.reverse(); + files.into() + }) + .collect(); + + new_config.file_source = new_file_source; + + // Phase 1: Clear output_ordering for Inexact + // (we're only reversing row groups, not guaranteeing perfect ordering) + if !is_exact { + new_config.output_ordering = vec![]; + } + + Ok(Arc::new(new_config)) + } } impl Debug for FileScanConfig { diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 7169997bd0316..cdff6ce2ee874 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -39,10 +39,11 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::{Constraints, Result, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning, PhysicalExpr}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_plan::filter_pushdown::{ ChildPushdownResult, FilterPushdownPhase, FilterPushdownPropagation, PushedDown, }; +use datafusion_physical_plan::SortOrderPushdownResult; /// A source of data, typically a list of files or memory /// @@ -189,6 +190,25 @@ pub trait DataSource: Send + Sync + Debug { vec![PushedDown::No; filters.len()], )) } + + /// Try to create a new DataSource that produces data in the specified sort order. + /// + /// # Arguments + /// * `order` - The desired output ordering + /// + /// # Returns + /// * `Ok(SortOrderPushdownResult::Exact { .. })` - Created a source that guarantees exact ordering + /// * `Ok(SortOrderPushdownResult::Inexact { .. })` - Created a source optimized for the ordering + /// * `Ok(SortOrderPushdownResult::Unsupported)` - Cannot optimize for this ordering + /// * `Err(e)` - Error occurred + /// + /// Default implementation returns `Unsupported`. + fn try_pushdown_sort( + &self, + _order: &[PhysicalSortExpr], + ) -> Result>> { + Ok(SortOrderPushdownResult::Unsupported) + } } /// [`ExecutionPlan`] that reads one or more files @@ -371,6 +391,19 @@ impl ExecutionPlan for DataSourceExec { }), } } + + fn try_pushdown_sort( + &self, + order: &[PhysicalSortExpr], + ) -> Result>> { + // Delegate to the data source and wrap result with DataSourceExec + self.data_source + .try_pushdown_sort(order)? + .try_map(|new_data_source| { + let new_exec = self.clone().with_data_source(new_data_source); + Ok(Arc::new(new_exec) as Arc) + }) + } } impl DataSourceExec { diff --git a/datafusion/execution/src/config.rs b/datafusion/execution/src/config.rs index a0b180bf40206..f3173e3514612 100644 --- a/datafusion/execution/src/config.rs +++ b/datafusion/execution/src/config.rs @@ -424,6 +424,13 @@ impl SessionConfig { self.options.optimizer.enable_round_robin_repartition } + /// Enables or disables sort pushdown optimization, and currently only + /// applies to Parquet data source. + pub fn with_enable_sort_pushdown(mut self, enabled: bool) -> Self { + self.options_mut().optimizer.enable_sort_pushdown = enabled; + self + } + /// Set the size of [`sort_spill_reservation_bytes`] to control /// memory pre-reservation /// diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index d19d7024a516e..5a7ca48337dae 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -426,6 +426,62 @@ impl LexOrdering { self.exprs.truncate(len); true } + + /// Check if reversing this ordering would satisfy another ordering requirement. + /// + /// This supports **prefix matching**: if this ordering is `[A DESC, B ASC]` + /// and `other` is `[A ASC]`, reversing this gives `[A ASC, B DESC]`, which + /// satisfies `other` since `[A ASC]` is a prefix. + /// + /// # Arguments + /// * `other` - The ordering requirement to check against + /// + /// # Returns + /// `true` if reversing this ordering would satisfy `other` + /// + /// # Example + /// ```text + /// self: [number DESC, letter ASC] + /// other: [number ASC] + /// After reversing self: [number ASC, letter DESC] ✓ Prefix match! + /// ``` + pub fn is_reverse(&self, other: &LexOrdering) -> bool { + let self_exprs = self.as_ref(); + let other_exprs = other.as_ref(); + + if other_exprs.len() > self_exprs.len() { + return false; + } + + other_exprs.iter().zip(self_exprs.iter()).all(|(req, cur)| { + req.expr.eq(&cur.expr) && is_reversed_sort_options(&req.options, &cur.options) + }) + } +} + +/// Check if two SortOptions represent reversed orderings. +/// +/// Returns `true` if both `descending` and `nulls_first` are opposite. +/// +/// # Example +/// ``` +/// use arrow::compute::SortOptions; +/// # use datafusion_physical_expr_common::sort_expr::is_reversed_sort_options; +/// +/// let asc_nulls_last = SortOptions { +/// descending: false, +/// nulls_first: false, +/// }; +/// let desc_nulls_first = SortOptions { +/// descending: true, +/// nulls_first: true, +/// }; +/// +/// assert!(is_reversed_sort_options(&asc_nulls_last, &desc_nulls_first)); +/// assert!(is_reversed_sort_options(&desc_nulls_first, &asc_nulls_last)); +/// ``` +pub fn is_reversed_sort_options(lhs: &SortOptions, rhs: &SortOptions) -> bool { + lhs.descending != rhs.descending && lhs.nulls_first != rhs.nulls_first } impl PartialEq for LexOrdering { @@ -732,3 +788,50 @@ impl DerefMut for OrderingRequirements { } } } + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_is_reversed_sort_options() { + // Test basic reversal: ASC NULLS LAST ↔ DESC NULLS FIRST + let asc_nulls_last = SortOptions { + descending: false, + nulls_first: false, + }; + let desc_nulls_first = SortOptions { + descending: true, + nulls_first: true, + }; + assert!(is_reversed_sort_options(&asc_nulls_last, &desc_nulls_first)); + assert!(is_reversed_sort_options(&desc_nulls_first, &asc_nulls_last)); + + // Test another reversal: ASC NULLS FIRST ↔ DESC NULLS LAST + let asc_nulls_first = SortOptions { + descending: false, + nulls_first: true, + }; + let desc_nulls_last = SortOptions { + descending: true, + nulls_first: false, + }; + assert!(is_reversed_sort_options(&asc_nulls_first, &desc_nulls_last)); + assert!(is_reversed_sort_options(&desc_nulls_last, &asc_nulls_first)); + + // Test non-reversal: same options + assert!(!is_reversed_sort_options(&asc_nulls_last, &asc_nulls_last)); + assert!(!is_reversed_sort_options( + &desc_nulls_first, + &desc_nulls_first + )); + + // Test non-reversal: only descending differs + assert!(!is_reversed_sort_options(&asc_nulls_last, &desc_nulls_last)); + assert!(!is_reversed_sort_options(&desc_nulls_last, &asc_nulls_last)); + + // Test non-reversal: only nulls_first differs + assert!(!is_reversed_sort_options(&asc_nulls_last, &asc_nulls_first)); + assert!(!is_reversed_sort_options(&asc_nulls_first, &asc_nulls_last)); + } +} diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index 79db43c1cbe94..08fa859df54a7 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -40,6 +40,7 @@ pub mod optimizer; pub mod output_requirements; pub mod projection_pushdown; pub use datafusion_pruning as pruning; +pub mod pushdown_sort; pub mod sanity_checker; pub mod topk_aggregation; pub mod update_aggr_exprs; diff --git a/datafusion/physical-optimizer/src/optimizer.rs b/datafusion/physical-optimizer/src/optimizer.rs index 4d00f1029db71..7c3d6ac527704 100644 --- a/datafusion/physical-optimizer/src/optimizer.rs +++ b/datafusion/physical-optimizer/src/optimizer.rs @@ -38,6 +38,7 @@ use crate::update_aggr_exprs::OptimizeAggregateOrder; use crate::coalesce_async_exec_input::CoalesceAsyncExecInput; use crate::limit_pushdown_past_window::LimitPushPastWindows; +use crate::pushdown_sort::PushdownSort; use datafusion_common::config::ConfigOptions; use datafusion_common::Result; use datafusion_physical_plan::ExecutionPlan; @@ -147,6 +148,8 @@ impl PhysicalOptimizer { // are not present, the load of executors such as join or union will be // reduced by narrowing their input tables. Arc::new(ProjectionPushdown::new()), + // PushdownSort: Detect sorts that can be pushed down to data sources. + Arc::new(PushdownSort::new()), Arc::new(EnsureCooperative::new()), // This FilterPushdown handles dynamic filters that may have references to the source ExecutionPlan. // Therefore it should be run at the end of the optimization process since any changes to the plan may break the dynamic filter's references. diff --git a/datafusion/physical-optimizer/src/pushdown_sort.rs b/datafusion/physical-optimizer/src/pushdown_sort.rs new file mode 100644 index 0000000000000..51eb3722e98ff --- /dev/null +++ b/datafusion/physical-optimizer/src/pushdown_sort.rs @@ -0,0 +1,129 @@ +// 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. + +//! Sort Pushdown Optimization +//! +//! This optimizer attempts to push sort requirements down through the execution plan +//! tree to data sources that can natively handle them (e.g., by scanning files in +//! reverse order). +//! +//! ## How it works +//! +//! 1. Detects `SortExec` nodes in the plan +//! 2. Calls `try_pushdown_sort()` on the input to recursively push the sort requirement +//! 3. Each node type defines its own pushdown behavior: +//! - **Transparent nodes** (CoalesceBatchesExec, RepartitionExec, etc.) delegate to +//! their children and wrap the result +//! - **Data sources** (DataSourceExec) check if they can optimize for the ordering +//! - **Blocking nodes** return `Unsupported` to stop pushdown +//! 4. Based on the result: +//! - `Exact`: Remove the Sort operator (data source guarantees perfect ordering) +//! - `Inexact`: Keep Sort but use optimized input (enables early termination for TopK) +//! - `Unsupported`: No change +//! +//! ## Current capabilities (Phase 1) +//! +//! - Reverse scan optimization: when required sort is the reverse of the data source's +//! natural ordering, enable reverse scanning (reading row groups in reverse order) +//! - Supports prefix matching: if data has ordering [A DESC, B ASC] and query needs +//! [A ASC], reversing gives [A ASC, B DESC] which satisfies the requirement +//! +//! TODO Issue: +//! ## Future enhancements (Phase 2), +//! +//! - File reordering based on statistics +//! - Return `Exact` when files are known to be perfectly sorted +//! - Complete Sort elimination when ordering is guaranteed + +use crate::PhysicalOptimizerRule; +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_common::Result; +use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::ExecutionPlan; +use datafusion_physical_plan::SortOrderPushdownResult; +use std::sync::Arc; + +/// A PhysicalOptimizerRule that attempts to push down sort requirements to data sources. +/// +/// See module-level documentation for details. +#[derive(Debug, Clone, Default)] +pub struct PushdownSort; + +impl PushdownSort { + pub fn new() -> Self { + Self {} + } +} + +impl PhysicalOptimizerRule for PushdownSort { + fn optimize( + &self, + plan: Arc, + config: &ConfigOptions, + ) -> Result> { + // Check if sort pushdown optimization is enabled + if !config.optimizer.enable_sort_pushdown { + return Ok(plan); + } + + // Use transform_down to find and optimize all SortExec nodes (including nested ones) + plan.transform_down(|plan: Arc| { + // Check if this is a SortExec + let Some(sort_exec) = plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(plan)); + }; + + let sort_input = Arc::clone(sort_exec.input()); + let required_ordering = sort_exec.expr(); + + // Try to push the sort requirement down through the plan tree + // Each node type defines its own pushdown behavior via try_pushdown_sort() + match sort_input.try_pushdown_sort(required_ordering)? { + SortOrderPushdownResult::Exact { inner } => { + // Data source guarantees perfect ordering - remove the Sort operator + Ok(Transformed::yes(inner)) + } + SortOrderPushdownResult::Inexact { inner } => { + // Data source is optimized for the ordering but not perfectly sorted + // Keep the Sort operator but use the optimized input + // Benefits: TopK queries can terminate early, better cache locality + Ok(Transformed::yes(Arc::new( + SortExec::new(required_ordering.clone(), inner) + .with_fetch(sort_exec.fetch()) + .with_preserve_partitioning( + sort_exec.preserve_partitioning(), + ), + ))) + } + SortOrderPushdownResult::Unsupported => { + // Cannot optimize for this ordering - no change + Ok(Transformed::no(plan)) + } + } + }) + .data() + } + + fn name(&self) -> &str { + "PushdownSort" + } + + fn schema_check(&self) -> bool { + true + } +} diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 1eeb477a39b21..86a08411cebf7 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -40,7 +40,9 @@ use crate::filter_pushdown::{ ChildPushdownResult, FilterDescription, FilterPushdownPhase, FilterPushdownPropagation, }; +use crate::sort_pushdown::SortOrderPushdownResult; use datafusion_common::config::ConfigOptions; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use futures::ready; use futures::stream::{Stream, StreamExt}; @@ -251,6 +253,20 @@ impl ExecutionPlan for CoalesceBatchesExec { ) -> Result>> { Ok(FilterPushdownPropagation::if_all(child_pushdown_result)) } + + fn try_pushdown_sort( + &self, + order: &[PhysicalSortExpr], + ) -> Result>> { + // CoalesceBatchesExec is transparent for sort ordering - it preserves order + // Delegate to the child and wrap with a new CoalesceBatchesExec + self.input.try_pushdown_sort(order)?.try_map(|new_input| { + Ok(Arc::new( + CoalesceBatchesExec::new(new_input, self.target_batch_size) + .with_fetch(self.fetch), + ) as Arc) + }) + } } /// Stream for [`CoalesceBatchesExec`]. See [`CoalesceBatchesExec`] for more details. diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 31e5a7369cab3..ce0d90eec2ec1 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -29,8 +29,11 @@ use super::{ }; use crate::execution_plan::{CardinalityEffect, EvaluationType, SchedulingType}; use crate::filter_pushdown::{FilterDescription, FilterPushdownPhase}; + use crate::projection::{make_with_child, ProjectionExec}; +use crate::sort_pushdown::SortOrderPushdownResult; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use datafusion_common::config::ConfigOptions; use datafusion_common::{internal_err, Result}; @@ -292,6 +295,42 @@ impl ExecutionPlan for CoalescePartitionsExec { ) -> Result { FilterDescription::from_children(parent_filters, &self.children()) } + + fn try_pushdown_sort( + &self, + order: &[PhysicalSortExpr], + ) -> Result>> { + // CoalescePartitionsExec merges multiple partitions into one, which loses + // global ordering. However, we can still push the sort requirement down + // to optimize individual partitions - the Sort operator above will handle + // the global ordering. + // + // Note: The result will always be at most Inexact (never Exact) when there + // are multiple partitions, because merging destroys global ordering. + let result = self.input.try_pushdown_sort(order)?; + + // If we have multiple partitions, we can't return Exact even if the + // underlying source claims Exact - merging destroys global ordering + let has_multiple_partitions = + self.input.output_partitioning().partition_count() > 1; + + result + .try_map(|new_input| { + Ok( + Arc::new( + CoalescePartitionsExec::new(new_input).with_fetch(self.fetch), + ) as Arc, + ) + }) + .map(|r| { + if has_multiple_partitions { + // Downgrade Exact to Inexact when merging multiple partitions + r.into_inexact() + } else { + r + } + }) + } } #[cfg(test)] diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 340e5662cebcb..9d998c4ffb17d 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -22,6 +22,7 @@ use crate::filter_pushdown::{ }; pub use crate::metrics::Metric; pub use crate::ordering::InputOrderMode; +use crate::sort_pushdown::SortOrderPushdownResult; pub use crate::stream::EmptyRecordBatchStream; pub use datafusion_common::hash_utils; @@ -51,7 +52,9 @@ use datafusion_common::{exec_err, Constraints, DataFusionError, Result}; use datafusion_common_runtime::JoinSet; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, OrderingRequirements}; +use datafusion_physical_expr_common::sort_expr::{ + LexOrdering, OrderingRequirements, PhysicalSortExpr, +}; use futures::stream::{StreamExt, TryStreamExt}; @@ -688,6 +691,29 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { ) -> Option> { None } + + /// Try to push down sort ordering requirements to this node. + /// + /// This method is called during sort pushdown optimization to determine if this + /// node can optimize for a requested sort ordering. Implementations should: + /// + /// - Return [`SortOrderPushdownResult::Exact`] if the node can guarantee the exact + /// ordering (allowing the Sort operator to be removed) + /// - Return [`SortOrderPushdownResult::Inexact`] if the node can optimize for the + /// ordering but cannot guarantee perfect sorting (Sort operator is kept) + /// - Return [`SortOrderPushdownResult::Unsupported`] if the node cannot optimize + /// for the ordering + /// + /// For transparent nodes (that preserve ordering), implement this to delegate to + /// children and wrap the result with a new instance of this node. + /// + /// Default implementation returns `Unsupported`. + fn try_pushdown_sort( + &self, + _order: &[PhysicalSortExpr], + ) -> Result>> { + Ok(SortOrderPushdownResult::Unsupported) + } } /// [`ExecutionPlan`] Invariant Level diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index c8e8049c3b324..88178307b56b7 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -47,6 +47,7 @@ pub use crate::execution_plan::{ }; pub use crate::metrics::Metric; pub use crate::ordering::InputOrderMode; +pub use crate::sort_pushdown::SortOrderPushdownResult; pub use crate::stream::EmptyRecordBatchStream; pub use crate::topk::TopK; pub use crate::visitor::{accept, visit_execution_plan, ExecutionPlanVisitor}; @@ -81,6 +82,7 @@ pub mod placeholder_row; pub mod projection; pub mod recursive_query; pub mod repartition; +pub mod sort_pushdown; pub mod sorts; pub mod spill; pub mod stream; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index f215c6233b3a0..cd93741c6300d 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -29,6 +29,7 @@ use super::common::SharedMemoryReservation; use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use super::{ DisplayAs, ExecutionPlanProperties, RecordBatchStream, SendableRecordBatchStream, + SortOrderPushdownResult, }; use crate::execution_plan::{CardinalityEffect, EvaluationType, SchedulingType}; use crate::hash_utils::create_hashes; @@ -52,7 +53,7 @@ use datafusion_common_runtime::SpawnedTask; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use crate::filter_pushdown::{ ChildPushdownResult, FilterDescription, FilterPushdownPhase, @@ -1087,6 +1088,27 @@ impl ExecutionPlan for RepartitionExec { Ok(FilterPushdownPropagation::if_all(child_pushdown_result)) } + fn try_pushdown_sort( + &self, + order: &[PhysicalSortExpr], + ) -> Result>> { + // RepartitionExec only maintains input order if preserve_order is set + // or if there's only one partition + if !self.maintains_input_order()[0] { + return Ok(SortOrderPushdownResult::Unsupported); + } + + // Delegate to the child and wrap with a new RepartitionExec + self.input.try_pushdown_sort(order)?.try_map(|new_input| { + let mut new_repartition = + RepartitionExec::try_new(new_input, self.partitioning().clone())?; + if self.preserve_order { + new_repartition = new_repartition.with_preserve_order(); + } + Ok(Arc::new(new_repartition) as Arc) + }) + } + fn repartitioned( &self, target_partitions: usize, diff --git a/datafusion/physical-plan/src/sort_pushdown.rs b/datafusion/physical-plan/src/sort_pushdown.rs new file mode 100644 index 0000000000000..8432fd5dabee7 --- /dev/null +++ b/datafusion/physical-plan/src/sort_pushdown.rs @@ -0,0 +1,120 @@ +// 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. + +//! Sort pushdown types for physical execution plans. +//! +//! This module provides types used for pushing sort ordering requirements +//! down through the execution plan tree to data sources. + +/// Result of attempting to push down sort ordering to a node. +/// +/// Used by [`ExecutionPlan::try_pushdown_sort`] to communicate +/// whether and how sort ordering was successfully pushed down. +/// +/// [`ExecutionPlan::try_pushdown_sort`]: crate::ExecutionPlan::try_pushdown_sort +#[derive(Debug, Clone)] +pub enum SortOrderPushdownResult { + /// The source can guarantee exact ordering (data is perfectly sorted). + /// + /// When this is returned, the optimizer can safely remove the Sort operator + /// entirely since the data source guarantees the requested ordering. + Exact { + /// The optimized node that provides exact ordering + inner: T, + }, + /// The source has optimized for the ordering but cannot guarantee perfect sorting. + /// + /// This indicates the data source has been optimized (e.g., reordered files/row groups + /// based on statistics, enabled reverse scanning) but the data may not be perfectly + /// sorted. The optimizer should keep the Sort operator but benefits from the + /// optimization (e.g., faster TopK queries due to early termination). + Inexact { + /// The optimized node that provides approximate ordering + inner: T, + }, + /// The source cannot optimize for this ordering. + /// + /// The data source does not support the requested sort ordering and no + /// optimization was applied. + Unsupported, +} + +impl SortOrderPushdownResult { + /// Extract the inner value if present + pub fn into_inner(self) -> Option { + match self { + Self::Exact { inner } | Self::Inexact { inner } => Some(inner), + Self::Unsupported => None, + } + } + + /// Map the inner value to a different type while preserving the variant. + pub fn map U>(self, f: F) -> SortOrderPushdownResult { + match self { + Self::Exact { inner } => SortOrderPushdownResult::Exact { inner: f(inner) }, + Self::Inexact { inner } => { + SortOrderPushdownResult::Inexact { inner: f(inner) } + } + Self::Unsupported => SortOrderPushdownResult::Unsupported, + } + } + + /// Try to map the inner value, returning an error if the function fails. + pub fn try_map Result>( + self, + f: F, + ) -> Result, E> { + match self { + Self::Exact { inner } => { + Ok(SortOrderPushdownResult::Exact { inner: f(inner)? }) + } + Self::Inexact { inner } => { + Ok(SortOrderPushdownResult::Inexact { inner: f(inner)? }) + } + Self::Unsupported => Ok(SortOrderPushdownResult::Unsupported), + } + } + + /// Convert this result to `Inexact`, downgrading `Exact` if present. + /// + /// This is useful when an operation (like merging multiple partitions) + /// cannot guarantee exact ordering even if the input provides it. + /// + /// # Examples + /// + /// ``` + /// # use datafusion_physical_plan::SortOrderPushdownResult; + /// let exact = SortOrderPushdownResult::Exact { inner: 42 }; + /// let inexact = exact.into_inexact(); + /// assert!(matches!(inexact, SortOrderPushdownResult::Inexact { inner: 42 })); + /// + /// let already_inexact = SortOrderPushdownResult::Inexact { inner: 42 }; + /// let still_inexact = already_inexact.into_inexact(); + /// assert!(matches!(still_inexact, SortOrderPushdownResult::Inexact { inner: 42 })); + /// + /// let unsupported = SortOrderPushdownResult::::Unsupported; + /// let still_unsupported = unsupported.into_inexact(); + /// assert!(matches!(still_unsupported, SortOrderPushdownResult::Unsupported)); + /// ``` + pub fn into_inexact(self) -> Self { + match self { + Self::Exact { inner } => Self::Inexact { inner }, + Self::Inexact { inner } => Self::Inexact { inner }, + Self::Unsupported => Self::Unsupported, + } + } +} diff --git a/datafusion/sqllogictest/test_files/create_external_table.slt b/datafusion/sqllogictest/test_files/create_external_table.slt index 1e6183f48bac7..0b15a7f8ec5dd 100644 --- a/datafusion/sqllogictest/test_files/create_external_table.slt +++ b/datafusion/sqllogictest/test_files/create_external_table.slt @@ -264,7 +264,7 @@ logical_plan 02)--TableScan: t projection=[id] physical_plan 01)SortExec: expr=[id@0 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], file_type=parquet, reverse_row_groups=true statement ok DROP TABLE t; diff --git a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt index e5cd6d88b08f4..57047412980c5 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -337,3 +337,329 @@ SET datafusion.optimizer.enable_join_dynamic_filter_pushdown = true; statement ok SET datafusion.optimizer.enable_dynamic_filter_pushdown = true; + +# Test 6: Sort Pushdown for ordered Parquet files + +# Create a sorted dataset +statement ok +CREATE TABLE sorted_data(id INT, value INT, name VARCHAR) AS VALUES +(1, 100, 'a'), +(2, 200, 'b'), +(3, 300, 'c'), +(4, 400, 'd'), +(5, 500, 'e'), +(6, 600, 'f'), +(7, 700, 'g'), +(8, 800, 'h'), +(9, 900, 'i'), +(10, 1000, 'j'); + +# Copy to parquet with sorting +query I +COPY (SELECT * FROM sorted_data ORDER BY id ASC) +TO 'test_files/scratch/dynamic_filter_pushdown_config/sorted_data.parquet'; +---- +10 + +statement ok +CREATE EXTERNAL TABLE sorted_parquet(id INT, value INT, name VARCHAR) +STORED AS PARQUET +LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/sorted_data.parquet' +WITH ORDER (id ASC); + +# Test 6.1: Sort pushdown with DESC (opposite of ASC) +# Should show reverse_row_groups=true +query TT +EXPLAIN SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3; +---- +logical_plan +01)Sort: sorted_parquet.id DESC NULLS FIRST, fetch=3 +02)--TableScan: sorted_parquet projection=[id, value, name] +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/dynamic_filter_pushdown_config/sorted_data.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +# Test 6.2: Verify results are correct +query IIT +SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3; +---- +10 1000 j +9 900 i +8 800 h + +# Test 6.3: Should NOT apply for ASC (same direction) +query TT +EXPLAIN SELECT * FROM sorted_parquet ORDER BY id ASC LIMIT 3; +---- +logical_plan +01)Sort: sorted_parquet.id ASC NULLS LAST, fetch=3 +02)--TableScan: sorted_parquet projection=[id, value, name] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/sorted_data.parquet]]}, projection=[id, value, name], limit=3, output_ordering=[id@0 ASC NULLS LAST], file_type=parquet + +# Test 6.4: Disable sort pushdown +statement ok +SET datafusion.optimizer.enable_sort_pushdown = false; + +query TT +EXPLAIN SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3; +---- +logical_plan +01)Sort: sorted_parquet.id DESC NULLS FIRST, fetch=3 +02)--TableScan: sorted_parquet projection=[id, value, name] +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/dynamic_filter_pushdown_config/sorted_data.parquet]]}, projection=[id, value, name], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] + +# Re-enable +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; + +# Test 6.5: With OFFSET +query TT +EXPLAIN SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3 OFFSET 2; +---- +logical_plan +01)Limit: skip=2, fetch=3 +02)--Sort: sorted_parquet.id DESC NULLS FIRST, fetch=5 +03)----TableScan: sorted_parquet projection=[id, value, name] +physical_plan +01)GlobalLimitExec: skip=2, fetch=3 +02)--SortExec: TopK(fetch=5), expr=[id@0 DESC], preserve_partitioning=[false] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/sorted_data.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +query IIT +SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3 OFFSET 2; +---- +8 800 h +7 700 g +6 600 f + +# Test 6.6: Reverse scan with row selection (page index pruning) +# This tests that when reverse_row_groups=true, the RowSelection is also properly reversed + +# Create a dataset with multiple row groups and enable page index +statement ok +CREATE TABLE multi_rg_data(id INT, category VARCHAR, value INT) AS VALUES +(1, 'alpha', 10), +(2, 'alpha', 20), +(3, 'beta', 30), +(4, 'beta', 40), +(5, 'gamma', 50), +(6, 'gamma', 60), +(7, 'delta', 70), +(8, 'delta', 80); + +# Write with small row groups (2 rows each = 4 row groups) +statement ok +SET datafusion.execution.parquet.max_row_group_size = 2; + +query I +COPY (SELECT * FROM multi_rg_data ORDER BY id ASC) +TO 'test_files/scratch/dynamic_filter_pushdown_config/multi_rg_sorted.parquet'; +---- +8 + +# Reset row group size +statement ok +SET datafusion.execution.parquet.max_row_group_size = 1048576; + +statement ok +CREATE EXTERNAL TABLE multi_rg_sorted(id INT, category VARCHAR, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/multi_rg_sorted.parquet' +WITH ORDER (id ASC); + +# Enable page index for better pruning +statement ok +SET datafusion.execution.parquet.enable_page_index = true; + +statement ok +SET datafusion.execution.parquet.pushdown_filters = true; + +# Test with reverse scan and filter that prunes some row groups +# This will create a RowSelection with partial row group scans +query TT +EXPLAIN SELECT * FROM multi_rg_sorted +WHERE category IN ('alpha', 'gamma') +ORDER BY id DESC LIMIT 5; +---- +logical_plan +01)Sort: multi_rg_sorted.id DESC NULLS FIRST, fetch=5 +02)--Filter: multi_rg_sorted.category = Utf8View("alpha") OR multi_rg_sorted.category = Utf8View("gamma") +03)----TableScan: multi_rg_sorted projection=[id, category, value], partial_filters=[multi_rg_sorted.category = Utf8View("alpha") OR multi_rg_sorted.category = Utf8View("gamma")] +physical_plan +01)SortExec: TopK(fetch=5), expr=[id@0 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/multi_rg_sorted.parquet]]}, projection=[id, category, value], file_type=parquet, predicate=(category@1 = alpha OR category@1 = gamma) AND DynamicFilter [ empty ], reverse_row_groups=true, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1 OR category_null_count@2 != row_count@3 AND category_min@0 <= gamma AND gamma <= category_max@1, required_guarantees=[category in (alpha, gamma)] + +# Verify the results are correct despite reverse scanning with row selection +# Expected: gamma values (6, 5) then alpha values (2, 1), in DESC order by id +query ITI +SELECT * FROM multi_rg_sorted +WHERE category IN ('alpha', 'gamma') +ORDER BY id DESC LIMIT 5; +---- +6 gamma 60 +5 gamma 50 +2 alpha 20 +1 alpha 10 + +# Test with more complex selection pattern +query ITI +SELECT * FROM multi_rg_sorted +WHERE category IN ('beta', 'delta') +ORDER BY id DESC; +---- +8 delta 80 +7 delta 70 +4 beta 40 +3 beta 30 + +# Test forward scan for comparison (should give same logical results in ASC order) +query ITI +SELECT * FROM multi_rg_sorted +WHERE category IN ('alpha', 'gamma') +ORDER BY id ASC; +---- +1 alpha 10 +2 alpha 20 +5 gamma 50 +6 gamma 60 + +# Disable reverse scan and verify it still works +statement ok +SET datafusion.optimizer.enable_sort_pushdown = false; + +query ITI +SELECT * FROM multi_rg_sorted +WHERE category IN ('alpha', 'gamma') +ORDER BY id DESC LIMIT 5; +---- +6 gamma 60 +5 gamma 50 +2 alpha 20 +1 alpha 10 + +# Re-enable +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; + +# Test 6.7: Sort pushdown with more than one partition +# Create multiple parquet files to trigger it + +# Split data into multiple files +statement ok +CREATE TABLE sorted_data_part1(id INT, value INT, name VARCHAR) AS VALUES +(1, 100, 'a'), +(2, 200, 'b'), +(3, 300, 'c'); + +statement ok +CREATE TABLE sorted_data_part2(id INT, value INT, name VARCHAR) AS VALUES +(4, 400, 'd'), +(5, 500, 'e'), +(6, 600, 'f'); + +statement ok +CREATE TABLE sorted_data_part3(id INT, value INT, name VARCHAR) AS VALUES +(7, 700, 'g'), +(8, 800, 'h'), +(9, 900, 'i'), +(10, 1000, 'j'); + +# Create directory for multi-file parquet +query I +COPY (SELECT * FROM sorted_data_part1 ORDER BY id ASC) +TO 'test_files/scratch/dynamic_filter_pushdown_config/sorted_multi/part1.parquet'; +---- +3 + +query I +COPY (SELECT * FROM sorted_data_part2 ORDER BY id ASC) +TO 'test_files/scratch/dynamic_filter_pushdown_config/sorted_multi/part2.parquet'; +---- +3 + +query I +COPY (SELECT * FROM sorted_data_part3 ORDER BY id ASC) +TO 'test_files/scratch/dynamic_filter_pushdown_config/sorted_multi/part3.parquet'; +---- +4 + +# Create external table pointing to directory with multiple files +statement ok +CREATE EXTERNAL TABLE sorted_parquet_multi(id INT, value INT, name VARCHAR) +STORED AS PARQUET +LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/sorted_multi/' +WITH ORDER (id ASC); + +# Enable multiple partitions +statement ok +SET datafusion.execution.target_partitions = 4; + +# Now we should see RepartitionExec because we have 3 input partitions (3 files) +query TT +EXPLAIN SELECT * FROM sorted_parquet_multi ORDER BY id DESC LIMIT 3; +---- +logical_plan +01)Sort: sorted_parquet_multi.id DESC NULLS FIRST, fetch=3 +02)--TableScan: sorted_parquet_multi projection=[id, value, name] +physical_plan +01)SortPreservingMergeExec: [id@0 DESC], fetch=3 +02)--SortExec: TopK(fetch=3), expr=[id@0 DESC], preserve_partitioning=[true] +03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/sorted_multi/part1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/sorted_multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/sorted_multi/part3.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +# Verify correctness with repartitioning and multiple files +query IIT +SELECT * FROM sorted_parquet_multi ORDER BY id DESC LIMIT 3; +---- +10 1000 j +9 900 i +8 800 h + +# Test ASC order (should not trigger reverse scan) +query IIT +SELECT * FROM sorted_parquet_multi ORDER BY id ASC LIMIT 3; +---- +1 100 a +2 200 b +3 300 c + +# Cleanup +statement ok +DROP TABLE sorted_data_part1; + +statement ok +DROP TABLE sorted_data_part2; + +statement ok +DROP TABLE sorted_data_part3; + +statement ok +DROP TABLE sorted_parquet_multi; + +# Reset to default +statement ok +SET datafusion.execution.target_partitions = 4; + +# Cleanup +statement ok +DROP TABLE multi_rg_data; + +statement ok +DROP TABLE multi_rg_sorted; + +statement ok +SET datafusion.execution.parquet.enable_page_index = false; + +statement ok +SET datafusion.execution.parquet.pushdown_filters = false; + +# Cleanup +statement ok +DROP TABLE sorted_data; + +statement ok +DROP TABLE sorted_parquet; + +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 41585a5d2cc6d..902d6172b99f5 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -244,6 +244,7 @@ physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushPastWindows SAME TEXT AS ABOVE physical_plan after LimitPushdown SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after PushdownSort SAME TEXT AS ABOVE physical_plan after EnsureCooperative SAME TEXT AS ABOVE physical_plan after FilterPushdown(Post) SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE @@ -325,6 +326,7 @@ physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushPastWindows SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after PushdownSort SAME TEXT AS ABOVE physical_plan after EnsureCooperative SAME TEXT AS ABOVE physical_plan after FilterPushdown(Post) SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE @@ -370,6 +372,7 @@ physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushPastWindows SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after PushdownSort SAME TEXT AS ABOVE physical_plan after EnsureCooperative SAME TEXT AS ABOVE physical_plan after FilterPushdown(Post) SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE @@ -608,6 +611,7 @@ physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushPastWindows SAME TEXT AS ABOVE physical_plan after LimitPushdown SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after PushdownSort SAME TEXT AS ABOVE physical_plan after EnsureCooperative SAME TEXT AS ABOVE physical_plan after FilterPushdown(Post) SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index dcf336c9be86e..7b6a6517945e7 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -294,6 +294,7 @@ datafusion.optimizer.enable_dynamic_filter_pushdown true datafusion.optimizer.enable_join_dynamic_filter_pushdown true datafusion.optimizer.enable_piecewise_merge_join false datafusion.optimizer.enable_round_robin_repartition true +datafusion.optimizer.enable_sort_pushdown true datafusion.optimizer.enable_topk_aggregation true datafusion.optimizer.enable_topk_dynamic_filter_pushdown true datafusion.optimizer.enable_window_limits true @@ -415,6 +416,7 @@ datafusion.optimizer.enable_dynamic_filter_pushdown true When set to true attemp datafusion.optimizer.enable_join_dynamic_filter_pushdown true When set to true, the optimizer will attempt to push down Join dynamic filters into the file scan phase. datafusion.optimizer.enable_piecewise_merge_join false When set to true, piecewise merge join is enabled. PiecewiseMergeJoin is currently experimental. Physical planner will opt for PiecewiseMergeJoin when there is only one range filter. datafusion.optimizer.enable_round_robin_repartition true When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores +datafusion.optimizer.enable_sort_pushdown true Enable sort pushdown optimization. When enabled, attempts to push sort requirements down to data sources that can natively handle them (e.g., by reversing file/row group read order). Returns **inexact ordering**: Sort operator is kept for correctness, but optimized input enables early termination for TopK queries (ORDER BY ... LIMIT N), providing significant speedup. Memory: No additional overhead (only changes read order). Future: Will add option to detect perfectly sorted data and eliminate Sort completely. Default: true datafusion.optimizer.enable_topk_aggregation true When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible datafusion.optimizer.enable_topk_dynamic_filter_pushdown true When set to true, the optimizer will attempt to push down TopK dynamic filters into the file scan phase. datafusion.optimizer.enable_window_limits true When set to true, the optimizer will attempt to push limit operations past window functions, if possible diff --git a/datafusion/sqllogictest/test_files/topk.slt b/datafusion/sqllogictest/test_files/topk.slt index 8a08cc17d4172..b167523485333 100644 --- a/datafusion/sqllogictest/test_files/topk.slt +++ b/datafusion/sqllogictest/test_files/topk.slt @@ -340,7 +340,7 @@ explain select number, letter, age from partial_sorted order by number asc limit ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true query TT explain select number, letter, age from partial_sorted order by letter asc, number desc limit 3; diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index fa7ad5c6c393b..078df208ae1f9 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -155,6 +155,7 @@ The following configuration settings are available: | datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | | datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | | datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | +| datafusion.optimizer.enable_sort_pushdown | true | Enable sort pushdown optimization. When enabled, attempts to push sort requirements down to data sources that can natively handle them (e.g., by reversing file/row group read order). Returns **inexact ordering**: Sort operator is kept for correctness, but optimized input enables early termination for TopK queries (ORDER BY ... LIMIT N), providing significant speedup. Memory: No additional overhead (only changes read order). Future: Will add option to detect perfectly sorted data and eliminate Sort completely. Default: true | | datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | | datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | | datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | From 1ae6efa908f1afc438dd50408fa3417f672ae3e7 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 23 Dec 2025 15:40:44 +0800 Subject: [PATCH 02/10] use new design --- Cargo.lock | 1 + datafusion/datasource-parquet/src/opener.rs | 5 +- datafusion/datasource-parquet/src/source.rs | 91 ++++++++++--------- datafusion/datasource/src/file.rs | 37 +++++++- datafusion/datasource/src/file_scan_config.rs | 21 +---- datafusion/datasource/src/source.rs | 3 +- datafusion/physical-optimizer/Cargo.toml | 1 + .../physical-optimizer/src/pushdown_sort.rs | 5 + datafusion/physical-plan/src/coop.rs | 16 +++- datafusion/physical-plan/src/projection.rs | 28 +++++- 10 files changed, 136 insertions(+), 72 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index d712eecfcc72e..ab3548fa06f50 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2533,6 +2533,7 @@ dependencies = [ "datafusion-pruning", "insta", "itertools 0.14.0", + "log", "recursive", "tokio", ] diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index f2165a95b86ce..fe9b65ef4d0c6 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -52,7 +52,7 @@ use datafusion_common::config::EncryptionFactoryOptions; use datafusion_execution::parquet_encryption::EncryptionFactory; use futures::{ready, Stream, StreamExt, TryStreamExt}; use itertools::Itertools; -use log::debug; +use log::{debug, info}; use parquet::arrow::arrow_reader::metrics::ArrowReaderMetrics; use parquet::arrow::arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions}; use parquet::arrow::async_reader::AsyncFileReader; @@ -495,9 +495,12 @@ impl FileOpener for ParquetOpener { // If reverse scanning is enabled, reverse the prepared plan if reverse_row_groups { + info!("reversing parquet file scan for file {}", file_name); prepared_plan = prepared_plan.reverse(file_metadata.as_ref())?; } + info!("parquet file scan for file {}", file_name); + // Apply the prepared plan to the builder builder = prepared_plan.apply_to_builder(builder); diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 145b08e4595a3..8f22eaab480f0 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -41,7 +41,7 @@ use datafusion_common::{DataFusionError, Statistics}; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::TableSchema; -use datafusion_physical_expr::conjunction; +use datafusion_physical_expr::{conjunction, EquivalenceProperties}; use datafusion_physical_expr_adapter::DefaultPhysicalExprAdapterFactory; use datafusion_physical_expr_common::physical_expr::{fmt_sql, PhysicalExpr}; use datafusion_physical_plan::filter_pushdown::PushedDown; @@ -57,6 +57,7 @@ use datafusion_physical_plan::SortOrderPushdownResult; use datafusion_execution::parquet_encryption::EncryptionFactory; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use itertools::Itertools; +use log::info; use object_store::ObjectStore; #[cfg(feature = "parquet_encryption")] use parquet::encryption::decrypt::FileDecryptionProperties; @@ -290,9 +291,6 @@ pub struct ParquetSource { pub(crate) projected_statistics: Option, #[cfg(feature = "parquet_encryption")] pub(crate) encryption_factory: Option>, - /// The ordering of data within the files - /// This is set by FileScanConfig when it knows the file ordering - file_ordering: Option, /// If true, read files in reverse order and reverse row groups within files. /// But it's not guaranteed that rows within row groups are in reverse order, /// so we still need to sort them after reading, so the reverse scan is inexact. @@ -317,7 +315,6 @@ impl ParquetSource { projected_statistics: None, #[cfg(feature = "parquet_encryption")] encryption_factory: None, - file_ordering: None, reverse_row_groups: false, } } @@ -384,11 +381,6 @@ impl ParquetSource { self } - /// If set, indicates the ordering of data within the files being read. - pub fn with_file_ordering(mut self, ordering: Option) -> Self { - self.file_ordering = ordering; - self - } /// Return the value described in [`Self::with_pushdown_filters`] pub(crate) fn pushdown_filters(&self) -> bool { @@ -487,12 +479,12 @@ impl ParquetSource { } } - pub(crate) fn with_reverse_row_groups(mut self, reverse_row_groups: bool) -> Self { + pub fn with_reverse_row_groups(mut self, reverse_row_groups: bool) -> Self { self.reverse_row_groups = reverse_row_groups; self } - #[cfg(test)] - pub(crate) fn reverse_row_groups(&self) -> bool { + + pub fn reverse_row_groups(&self) -> bool { self.reverse_row_groups } } @@ -854,44 +846,61 @@ impl FileSource for ParquetSource { fn try_reverse_output( &self, order: &[PhysicalSortExpr], + eq_properties: &EquivalenceProperties, ) -> datafusion_common::Result>> { - // Check if we have file ordering information - let file_ordering = match &self.file_ordering { - Some(ordering) => ordering, - None => return Ok(SortOrderPushdownResult::Unsupported), - }; - - // Create a LexOrdering from the requested order to use the is_reverse method - let Some(requested_ordering) = LexOrdering::new(order.to_vec()) else { - // Empty ordering requested, cannot optimize + if order.is_empty() { return Ok(SortOrderPushdownResult::Unsupported); - }; + } - // Check if reversing the file ordering would satisfy the requested ordering - if file_ordering.is_reverse(&requested_ordering) { - // Phase 1: Enable reverse row group scanning - let new_source = self.clone().with_reverse_row_groups(true); + // Build new equivalence properties with the reversed ordering. + // This allows us to check if the reversed ordering satisfies the request + // by leveraging: + // - Function monotonicity (e.g., extract_year_month preserves ordering) + // - Constant columns (from filters) + // - Other equivalence relationships + // + // Example flow: + // 1. File ordering: [extract_year_month(ws) DESC, ws DESC] + // 2. After reversal: [extract_year_month(ws) ASC, ws ASC] + // 3. Requested: [ws ASC] + // 4. Through extract_year_month's monotonicity property, the reversed + // ordering satisfies [ws ASC] even though it has additional prefix + let reversed_eq_properties = { + let mut new = eq_properties.clone(); + new.clear_orderings(); + + // Reverse each ordering in the equivalence properties + let reversed_orderings = eq_properties + .oeq_class() + .iter() + .map(|ordering| { + ordering + .iter() + .map(|expr| expr.reverse()) + .collect::>() + }) + .collect::>(); + + new.add_orderings(reversed_orderings); + new + }; - // Return Inexact because we're only reversing row group order, - // not guaranteeing perfect row-level ordering - return Ok(SortOrderPushdownResult::Inexact { - inner: Arc::new(new_source) as Arc, - }); + // Check if the reversed ordering satisfies the requested ordering + if !reversed_eq_properties.ordering_satisfy(order.iter().cloned())? { + return Ok(SortOrderPushdownResult::Unsupported); } + // 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); + Ok(SortOrderPushdownResult::Inexact { + inner: Arc::new(new_source) as Arc, + }) + // TODO Phase 2: Add support for other optimizations: // - File reordering based on min/max statistics // - Detection of exact ordering (return Exact to remove Sort operator) // - Partial sort pushdown for prefix matches - - Ok(SortOrderPushdownResult::Unsupported) - } - - fn with_file_ordering_info( - &self, - ordering: Option, - ) -> datafusion_common::Result> { - Ok(Arc::new(self.clone().with_file_ordering(ordering))) } } diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index db781910c81f6..1bbe5b75dccb9 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -29,7 +29,7 @@ use crate::schema_adapter::SchemaAdapterFactory; use crate::TableSchema; use datafusion_common::config::ConfigOptions; use datafusion_common::{not_impl_err, Result, Statistics}; -use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use datafusion_physical_plan::filter_pushdown::{FilterPushdownPropagation, PushedDown}; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; @@ -132,15 +132,46 @@ pub trait FileSource: Send + Sync { /// Try to create a new FileSource that can produce data in the specified sort order. /// + /// This method attempts to optimize data retrieval to match the requested ordering. + /// It receives both the requested ordering and equivalence properties that describe + /// relationships between expressions (e.g., constant columns, monotonic functions). + /// + /// # Parameters + /// * `order` - The requested sort ordering + /// * `eq_properties` - Equivalence properties that can be used to determine if a reversed + /// ordering satisfies the request. This includes information about: + /// - Constant columns (e.g., from filters like `ticker = 'AAPL'`) + /// - Monotonic functions (e.g., `extract_year_month(timestamp)`) + /// - Other equivalence relationships + /// + /// # Examples + /// + /// ## Example 1: Simple reverse + /// ```text + /// File ordering: [a ASC, b DESC] + /// Requested: [a DESC] + /// Reversed file: [a DESC, b ASC] + /// Result: Satisfies request (prefix match) → Inexact + /// ``` + /// + /// ## Example 2: Monotonic function + /// ```text + /// File ordering: [extract_year_month(ts) ASC, ts ASC] + /// Requested: [ts DESC] + /// Reversed file: [extract_year_month(ts) DESC, ts DESC] + /// Result: Through monotonicity, satisfies [ts DESC] → Inexact + /// ``` + /// /// # Returns - /// * `Exact` - Created a source that guarantees perfect ordering - /// * `Inexact` - Created a source optimized for ordering (e.g., reordered files) but not perfectly sorted + /// * `Exact` - Created a source that guarantees perfect ordering (e.g., file reordering) + /// * `Inexact` - Created a source optimized for ordering (e.g., reversed row groups) but not perfectly sorted /// * `Unsupported` - Cannot optimize for this ordering /// /// Default implementation returns `Unsupported`. fn try_reverse_output( &self, _order: &[PhysicalSortExpr], + _eq_properties: &EquivalenceProperties, ) -> Result>> { Ok(SortOrderPushdownResult::Unsupported) } diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index fb1e66e026a1a..f7fd0bdb5c914 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -66,7 +66,7 @@ use datafusion_physical_plan::execution_plan::SchedulingType; use datafusion_physical_plan::projection::{ all_alias_free_columns, new_projections_for_columns, }; -use log::{debug, warn}; +use log::{debug, info, warn}; use std::borrow::Cow; use std::collections::HashMap; use std::marker::PhantomData; @@ -764,23 +764,10 @@ impl DataSource for FileScanConfig { &self, order: &[PhysicalSortExpr], ) -> Result>> { - let file_ordering = self.output_ordering.first().cloned(); - - if file_ordering.is_none() { - return Ok(SortOrderPushdownResult::Unsupported); - } - - // Use the trait method instead of downcasting - // Try to provide file ordering info to the source - // If not supported (e.g., CsvSource), fall back to original source - let file_source_with_ordering = self + // Delegate to FileSource to check if reverse scanning can satisfy the request. + let pushdown_result = self .file_source - .with_file_ordering_info(file_ordering) - .unwrap_or_else(|_| Arc::clone(&self.file_source)); - - // Try to reverse the datasource with ordering info, - // and currently only ParquetSource supports it with inexact reverse with row groups. - let pushdown_result = file_source_with_ordering.try_reverse_output(order)?; + .try_reverse_output(order, &self.eq_properties())?; match pushdown_result { SortOrderPushdownResult::Exact { inner } => { diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index cdff6ce2ee874..4a9fff2ecc760 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -33,7 +33,7 @@ use datafusion_physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, }; use itertools::Itertools; - +use log::info; use crate::file_scan_config::FileScanConfig; use datafusion_common::config::ConfigOptions; use datafusion_common::{Constraints, Result, Statistics}; @@ -396,6 +396,7 @@ impl ExecutionPlan for DataSourceExec { &self, order: &[PhysicalSortExpr], ) -> Result>> { + info!("trying to pushdown sort in DataSource: {:?}", order); // Delegate to the data source and wrap result with DataSourceExec self.data_source .try_pushdown_sort(order)? diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index 395da10d629ba..88a1ff30d80cf 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -52,6 +52,7 @@ datafusion-physical-plan = { workspace = true } datafusion-pruning = { workspace = true } itertools = { workspace = true } recursive = { workspace = true, optional = true } +log = "0.4.28" [dev-dependencies] datafusion-expr = { workspace = true } diff --git a/datafusion/physical-optimizer/src/pushdown_sort.rs b/datafusion/physical-optimizer/src/pushdown_sort.rs index 51eb3722e98ff..cb009053f5913 100644 --- a/datafusion/physical-optimizer/src/pushdown_sort.rs +++ b/datafusion/physical-optimizer/src/pushdown_sort.rs @@ -57,6 +57,7 @@ use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::ExecutionPlan; use datafusion_physical_plan::SortOrderPushdownResult; use std::sync::Arc; +use log::info; /// A PhysicalOptimizerRule that attempts to push down sort requirements to data sources. /// @@ -91,6 +92,8 @@ impl PhysicalOptimizerRule for PushdownSort { let sort_input = Arc::clone(sort_exec.input()); let required_ordering = sort_exec.expr(); + info!("trying to pushdown sort: {:?}", required_ordering); + // Try to push the sort requirement down through the plan tree // Each node type defines its own pushdown behavior via try_pushdown_sort() match sort_input.try_pushdown_sort(required_ordering)? { @@ -99,6 +102,7 @@ impl PhysicalOptimizerRule for PushdownSort { Ok(Transformed::yes(inner)) } SortOrderPushdownResult::Inexact { inner } => { + info!("inexact pushdown sort : {:?}", required_ordering); // Data source is optimized for the ordering but not perfectly sorted // Keep the Sort operator but use the optimized input // Benefits: TopK queries can terminate early, better cache locality @@ -111,6 +115,7 @@ impl PhysicalOptimizerRule for PushdownSort { ))) } SortOrderPushdownResult::Unsupported => { + info!("unsupported pushdown sort : {:?}", required_ordering); // Cannot optimize for this ordering - no change Ok(Transformed::no(plan)) } diff --git a/datafusion/physical-plan/src/coop.rs b/datafusion/physical-plan/src/coop.rs index b62d15e6d2f17..56b1b6e5dca2f 100644 --- a/datafusion/physical-plan/src/coop.rs +++ b/datafusion/physical-plan/src/coop.rs @@ -79,10 +79,7 @@ use crate::filter_pushdown::{ ChildPushdownResult, FilterDescription, FilterPushdownPhase, FilterPushdownPropagation, }; -use crate::{ - DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, RecordBatchStream, - SendableRecordBatchStream, -}; +use crate::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, RecordBatchStream, SendableRecordBatchStream, SortOrderPushdownResult}; use arrow::record_batch::RecordBatch; use arrow_schema::Schema; use datafusion_common::{internal_err, Result, Statistics}; @@ -91,6 +88,8 @@ use datafusion_execution::TaskContext; use crate::execution_plan::SchedulingType; use crate::stream::RecordBatchStreamAdapter; use futures::{Stream, StreamExt}; +use log::info; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; /// A stream that passes record batches through unchanged while cooperating with the Tokio runtime. /// It consumes cooperative scheduling budget for each returned [`RecordBatch`], @@ -313,6 +312,15 @@ impl ExecutionPlan for CooperativeExec { ) -> Result>> { Ok(FilterPushdownPropagation::if_all(child_pushdown_result)) } + + fn try_pushdown_sort(&self, order: &[PhysicalSortExpr]) -> Result>> { + info!("CooperativeExec trying to pushdown sort: {:?}", order); + self.input + .try_pushdown_sort(order)? + .try_map(|inner| { + Ok(Arc::new(CooperativeExec::new(inner)) as Arc) + }) + } } /// Creates a [`CooperativeStream`] wrapper around the given [`RecordBatchStream`]. diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index cfdaa4e9d9fd4..de5d130efac5e 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -22,10 +22,7 @@ use super::expressions::{Column, Literal}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; -use super::{ - DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, - SendableRecordBatchStream, Statistics, -}; +use super::{DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, SortOrderPushdownResult, Statistics}; use crate::execution_plan::CardinalityEffect; use crate::filter_pushdown::{ ChildPushdownResult, FilterDescription, FilterPushdownPhase, @@ -50,7 +47,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr_common::physical_expr::{fmt_sql, PhysicalExprRef}; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement, PhysicalSortExpr}; // Re-exported from datafusion-physical-expr for backwards compatibility // We recommend updating your imports to use datafusion-physical-expr directly pub use datafusion_physical_expr::projection::{ @@ -346,6 +343,27 @@ impl ExecutionPlan for ProjectionExec { ) -> Result>> { Ok(FilterPushdownPropagation::if_all(child_pushdown_result)) } + + fn try_pushdown_sort(&self, order: &[PhysicalSortExpr]) -> Result>> { + let child = self.input(); + + println!("try pushdown_sort in ProjectionExec: {:?}", order); + match child.try_pushdown_sort(order)? { + SortOrderPushdownResult::Exact { inner } => { + let new_exec = Arc::new(self.clone()) + .with_new_children(vec![inner])?; + Ok(SortOrderPushdownResult::Exact { inner: new_exec }) + } + SortOrderPushdownResult::Inexact { inner } => { + let new_exec = Arc::new(self.clone()) + .with_new_children(vec![inner])?; + Ok(SortOrderPushdownResult::Inexact { inner: new_exec }) + } + SortOrderPushdownResult::Unsupported => { + Ok(SortOrderPushdownResult::Unsupported) + } + } + } } impl ProjectionStream { From a4b686cca973f9b41fcf823231e9301a531395ec Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 23 Dec 2025 16:29:39 +0800 Subject: [PATCH 03/10] fix --- datafusion/datasource-parquet/src/opener.rs | 4 +- datafusion/datasource-parquet/src/source.rs | 4 +- datafusion/datasource/src/file_scan_config.rs | 2 +- datafusion/datasource/src/source.rs | 22 +- .../physical-optimizer/src/pushdown_sort.rs | 5 - datafusion/physical-plan/src/coop.rs | 22 +- datafusion/physical-plan/src/filter.rs | 22 +- datafusion/physical-plan/src/projection.rs | 22 +- .../dynamic_filter_pushdown_config.slt | 579 ++++++++++++++++++ 9 files changed, 641 insertions(+), 41 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index fe9b65ef4d0c6..59a9f76b15cfb 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -495,11 +495,11 @@ impl FileOpener for ParquetOpener { // If reverse scanning is enabled, reverse the prepared plan if reverse_row_groups { - info!("reversing parquet file scan for file {}", file_name); + info!("reversing parquet file scan for file {file_name}"); prepared_plan = prepared_plan.reverse(file_metadata.as_ref())?; } - info!("parquet file scan for file {}", file_name); + info!("parquet file scan for file {file_name}"); // Apply the prepared plan to the builder builder = prepared_plan.apply_to_builder(builder); diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 8f22eaab480f0..5dd153ff8847d 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -55,9 +55,8 @@ use datafusion_physical_plan::SortOrderPushdownResult; #[cfg(feature = "parquet_encryption")] use datafusion_execution::parquet_encryption::EncryptionFactory; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use itertools::Itertools; -use log::info; use object_store::ObjectStore; #[cfg(feature = "parquet_encryption")] use parquet::encryption::decrypt::FileDecryptionProperties; @@ -381,7 +380,6 @@ impl ParquetSource { self } - /// Return the value described in [`Self::with_pushdown_filters`] pub(crate) fn pushdown_filters(&self) -> bool { self.table_parquet_options.global.pushdown_filters diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index f7fd0bdb5c914..a5c47493df367 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -66,7 +66,7 @@ use datafusion_physical_plan::execution_plan::SchedulingType; use datafusion_physical_plan::projection::{ all_alias_free_columns, new_projections_for_columns, }; -use log::{debug, info, warn}; +use log::{debug, warn}; use std::borrow::Cow; use std::collections::HashMap; use std::marker::PhantomData; diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 4a9fff2ecc760..a53928c936827 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -22,28 +22,28 @@ use std::fmt; use std::fmt::{Debug, Formatter}; use std::sync::Arc; +use crate::file_scan_config::FileScanConfig; +use datafusion_common::config::ConfigOptions; +use datafusion_common::{Constraints, Result, Statistics}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning, PhysicalExpr}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_plan::execution_plan::{ Boundedness, EmissionType, SchedulingType, }; +use datafusion_physical_plan::filter_pushdown::{ + ChildPushdownResult, FilterPushdownPhase, FilterPushdownPropagation, PushedDown, +}; use datafusion_physical_plan::metrics::SplitMetrics; use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_plan::projection::{ProjectionExec, ProjectionExpr}; use datafusion_physical_plan::stream::BatchSplitStream; +use datafusion_physical_plan::SortOrderPushdownResult; use datafusion_physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, }; use itertools::Itertools; use log::info; -use crate::file_scan_config::FileScanConfig; -use datafusion_common::config::ConfigOptions; -use datafusion_common::{Constraints, Result, Statistics}; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr::{EquivalenceProperties, Partitioning, PhysicalExpr}; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; -use datafusion_physical_plan::filter_pushdown::{ - ChildPushdownResult, FilterPushdownPhase, FilterPushdownPropagation, PushedDown, -}; -use datafusion_physical_plan::SortOrderPushdownResult; /// A source of data, typically a list of files or memory /// @@ -396,7 +396,7 @@ impl ExecutionPlan for DataSourceExec { &self, order: &[PhysicalSortExpr], ) -> Result>> { - info!("trying to pushdown sort in DataSource: {:?}", order); + info!("trying to pushdown sort in DataSource: {order:?}"); // Delegate to the data source and wrap result with DataSourceExec self.data_source .try_pushdown_sort(order)? diff --git a/datafusion/physical-optimizer/src/pushdown_sort.rs b/datafusion/physical-optimizer/src/pushdown_sort.rs index cb009053f5913..51eb3722e98ff 100644 --- a/datafusion/physical-optimizer/src/pushdown_sort.rs +++ b/datafusion/physical-optimizer/src/pushdown_sort.rs @@ -57,7 +57,6 @@ use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::ExecutionPlan; use datafusion_physical_plan::SortOrderPushdownResult; use std::sync::Arc; -use log::info; /// A PhysicalOptimizerRule that attempts to push down sort requirements to data sources. /// @@ -92,8 +91,6 @@ impl PhysicalOptimizerRule for PushdownSort { let sort_input = Arc::clone(sort_exec.input()); let required_ordering = sort_exec.expr(); - info!("trying to pushdown sort: {:?}", required_ordering); - // Try to push the sort requirement down through the plan tree // Each node type defines its own pushdown behavior via try_pushdown_sort() match sort_input.try_pushdown_sort(required_ordering)? { @@ -102,7 +99,6 @@ impl PhysicalOptimizerRule for PushdownSort { Ok(Transformed::yes(inner)) } SortOrderPushdownResult::Inexact { inner } => { - info!("inexact pushdown sort : {:?}", required_ordering); // Data source is optimized for the ordering but not perfectly sorted // Keep the Sort operator but use the optimized input // Benefits: TopK queries can terminate early, better cache locality @@ -115,7 +111,6 @@ impl PhysicalOptimizerRule for PushdownSort { ))) } SortOrderPushdownResult::Unsupported => { - info!("unsupported pushdown sort : {:?}", required_ordering); // Cannot optimize for this ordering - no change Ok(Transformed::no(plan)) } diff --git a/datafusion/physical-plan/src/coop.rs b/datafusion/physical-plan/src/coop.rs index 56b1b6e5dca2f..9d2a0c62e6c3a 100644 --- a/datafusion/physical-plan/src/coop.rs +++ b/datafusion/physical-plan/src/coop.rs @@ -79,7 +79,10 @@ use crate::filter_pushdown::{ ChildPushdownResult, FilterDescription, FilterPushdownPhase, FilterPushdownPropagation, }; -use crate::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, RecordBatchStream, SendableRecordBatchStream, SortOrderPushdownResult}; +use crate::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, RecordBatchStream, + SendableRecordBatchStream, SortOrderPushdownResult, +}; use arrow::record_batch::RecordBatch; use arrow_schema::Schema; use datafusion_common::{internal_err, Result, Statistics}; @@ -87,9 +90,9 @@ use datafusion_execution::TaskContext; use crate::execution_plan::SchedulingType; use crate::stream::RecordBatchStreamAdapter; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use futures::{Stream, StreamExt}; use log::info; -use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; /// A stream that passes record batches through unchanged while cooperating with the Tokio runtime. /// It consumes cooperative scheduling budget for each returned [`RecordBatch`], @@ -313,13 +316,14 @@ impl ExecutionPlan for CooperativeExec { Ok(FilterPushdownPropagation::if_all(child_pushdown_result)) } - fn try_pushdown_sort(&self, order: &[PhysicalSortExpr]) -> Result>> { - info!("CooperativeExec trying to pushdown sort: {:?}", order); - self.input - .try_pushdown_sort(order)? - .try_map(|inner| { - Ok(Arc::new(CooperativeExec::new(inner)) as Arc) - }) + fn try_pushdown_sort( + &self, + order: &[PhysicalSortExpr], + ) -> Result>> { + info!("CooperativeExec trying to pushdown sort: {order:?}"); + self.input.try_pushdown_sort(order)?.try_map(|inner| { + Ok(Arc::new(CooperativeExec::new(inner)) as Arc) + }) } } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 2ee7d283d8ad8..e5ffe6934e9f5 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -24,7 +24,7 @@ use itertools::Itertools; use super::{ ColumnStatistics, DisplayAs, ExecutionPlanProperties, PlanProperties, - RecordBatchStream, SendableRecordBatchStream, Statistics, + RecordBatchStream, SendableRecordBatchStream, SortOrderPushdownResult, Statistics, }; use crate::common::can_project; use crate::execution_plan::CardinalityEffect; @@ -63,6 +63,7 @@ use datafusion_physical_expr::{ }; use datafusion_physical_expr_common::physical_expr::fmt_sql; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -570,6 +571,25 @@ impl ExecutionPlan for FilterExec { updated_node, }) } + fn try_pushdown_sort( + &self, + order: &[PhysicalSortExpr], + ) -> Result>> { + let child = self.input(); + match child.try_pushdown_sort(order)? { + SortOrderPushdownResult::Exact { inner } => { + let new_exec = Arc::new(self.clone()).with_new_children(vec![inner])?; + Ok(SortOrderPushdownResult::Exact { inner: new_exec }) + } + SortOrderPushdownResult::Inexact { inner } => { + let new_exec = Arc::new(self.clone()).with_new_children(vec![inner])?; + Ok(SortOrderPushdownResult::Inexact { inner: new_exec }) + } + SortOrderPushdownResult::Unsupported => { + Ok(SortOrderPushdownResult::Unsupported) + } + } + } } impl EmbeddedProjection for FilterExec { diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index de5d130efac5e..1979939587c27 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -22,7 +22,10 @@ use super::expressions::{Column, Literal}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; -use super::{DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, SortOrderPushdownResult, Statistics}; +use super::{ + DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, + SendableRecordBatchStream, SortOrderPushdownResult, Statistics, +}; use crate::execution_plan::CardinalityEffect; use crate::filter_pushdown::{ ChildPushdownResult, FilterDescription, FilterPushdownPhase, @@ -47,7 +50,9 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr_common::physical_expr::{fmt_sql, PhysicalExprRef}; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::{ + LexOrdering, LexRequirement, PhysicalSortExpr, +}; // Re-exported from datafusion-physical-expr for backwards compatibility // We recommend updating your imports to use datafusion-physical-expr directly pub use datafusion_physical_expr::projection::{ @@ -344,19 +349,18 @@ impl ExecutionPlan for ProjectionExec { Ok(FilterPushdownPropagation::if_all(child_pushdown_result)) } - fn try_pushdown_sort(&self, order: &[PhysicalSortExpr]) -> Result>> { + fn try_pushdown_sort( + &self, + order: &[PhysicalSortExpr], + ) -> Result>> { let child = self.input(); - - println!("try pushdown_sort in ProjectionExec: {:?}", order); match child.try_pushdown_sort(order)? { SortOrderPushdownResult::Exact { inner } => { - let new_exec = Arc::new(self.clone()) - .with_new_children(vec![inner])?; + let new_exec = Arc::new(self.clone()).with_new_children(vec![inner])?; Ok(SortOrderPushdownResult::Exact { inner: new_exec }) } SortOrderPushdownResult::Inexact { inner } => { - let new_exec = Arc::new(self.clone()) - .with_new_children(vec![inner])?; + let new_exec = Arc::new(self.clone()).with_new_children(vec![inner])?; Ok(SortOrderPushdownResult::Inexact { inner: new_exec }) } SortOrderPushdownResult::Unsupported => { diff --git a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt index 57047412980c5..678697f3c12fa 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -663,3 +663,582 @@ DROP TABLE sorted_parquet; statement ok SET datafusion.optimizer.enable_sort_pushdown = true; + +# Test 7: Sort pushdown with constant column filtering +# This tests the case where a leading sort column becomes constant through WHERE filtering + +# Create a multi-column sorted dataset (like time-series data) +statement ok +CREATE TABLE timeseries_data(timeframe VARCHAR, period_end INT, value DOUBLE) AS VALUES +('daily', 1, 100.0), +('daily', 2, 150.0), +('daily', 3, 200.0), +('weekly', 1, 500.0), +('weekly', 2, 600.0), +('weekly', 3, 700.0), +('monthly', 1, 1000.0), +('monthly', 2, 1100.0), +('monthly', 3, 1200.0), +('quarterly', 1, 5000.0), +('quarterly', 2, 5500.0), +('quarterly', 3, 6000.0); + +# Copy to parquet with multi-column sorting (timeframe ASC, period_end ASC) +query I +COPY (SELECT * FROM timeseries_data ORDER BY timeframe ASC, period_end ASC) +TO 'test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet'; +---- +12 + +statement ok +CREATE EXTERNAL TABLE timeseries_parquet(timeframe VARCHAR, period_end INT, value DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet' +WITH ORDER (timeframe ASC, period_end ASC); + +# Test 7.1: Query with constant prefix filter and DESC on remaining column +# WHERE timeframe='quarterly' makes the first sort column constant +# ORDER BY period_end DESC should trigger reverse scan because: +# File ordering: [timeframe ASC, period_end ASC] +# After filtering timeframe='quarterly': effectively [period_end ASC] +# Request: [period_end DESC] -> exact reverse! +query TT +EXPLAIN SELECT * FROM timeseries_parquet +WHERE timeframe = 'quarterly' +ORDER BY period_end DESC +LIMIT 2; +---- +logical_plan +01)Sort: timeseries_parquet.period_end DESC NULLS FIRST, fetch=2 +02)--Filter: timeseries_parquet.timeframe = Utf8View("quarterly") +03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] +physical_plan +01)SortPreservingMergeExec: [period_end@1 DESC], fetch=2 +02)--SortExec: TopK(fetch=2), expr=[period_end@1 DESC], preserve_partitioning=[true] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------FilterExec: timeframe@0 = quarterly +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=timeframe@0 = quarterly AND DynamicFilter [ empty ], reverse_row_groups=true, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] + +# Test 7.2: Verify the results are correct +query TIR +SELECT * FROM timeseries_parquet +WHERE timeframe = 'quarterly' +ORDER BY period_end DESC +LIMIT 2; +---- +quarterly 3 6000 +quarterly 2 5500 + +# Test 7.3: Same filter but ASC order (should not trigger reverse scan, ordering already satisfied) +query TT +EXPLAIN SELECT * FROM timeseries_parquet +WHERE timeframe = 'quarterly' +ORDER BY period_end ASC +LIMIT 2; +---- +logical_plan +01)Sort: timeseries_parquet.period_end ASC NULLS LAST, fetch=2 +02)--Filter: timeseries_parquet.timeframe = Utf8View("quarterly") +03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] +physical_plan +01)SortPreservingMergeExec: [period_end@1 ASC NULLS LAST], fetch=2 +02)--CoalesceBatchesExec: target_batch_size=8192, fetch=2 +03)----FilterExec: timeframe@0 = quarterly +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=timeframe@0 = quarterly, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] + +# Test 7.4: Verify ASC results +query TIR +SELECT * FROM timeseries_parquet +WHERE timeframe = 'quarterly' +ORDER BY period_end ASC +LIMIT 2; +---- +quarterly 1 5000 +quarterly 2 5500 + +# Test 7.5: Test with different constant value +query TIR +SELECT * FROM timeseries_parquet +WHERE timeframe = 'weekly' +ORDER BY period_end DESC; +---- +weekly 3 700 +weekly 2 600 +weekly 1 500 + +# Test 7.6: Test without constant filter (no reverse scan because need both columns) +# Request: [timeframe ASC, period_end DESC] +# File has: [timeframe ASC, period_end ASC] +# These are NOT reverse of each other - only second column is reversed +query TT +EXPLAIN SELECT * FROM timeseries_parquet +ORDER BY timeframe ASC, period_end DESC +LIMIT 3; +---- +logical_plan +01)Sort: timeseries_parquet.timeframe ASC NULLS LAST, timeseries_parquet.period_end DESC NULLS FIRST, fetch=3 +02)--TableScan: timeseries_parquet projection=[timeframe, period_end, value] +physical_plan +01)SortExec: TopK(fetch=3), expr=[timeframe@0 ASC NULLS LAST, period_end@1 DESC], preserve_partitioning=[false], sort_prefix=[timeframe@0 ASC NULLS LAST] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] + +# Test 7.7: Disable sort pushdown and verify filter still works +statement ok +SET datafusion.optimizer.enable_sort_pushdown = false; + +query TT +EXPLAIN SELECT * FROM timeseries_parquet +WHERE timeframe = 'quarterly' +ORDER BY period_end DESC +LIMIT 2; +---- +logical_plan +01)Sort: timeseries_parquet.period_end DESC NULLS FIRST, fetch=2 +02)--Filter: timeseries_parquet.timeframe = Utf8View("quarterly") +03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] +physical_plan +01)SortPreservingMergeExec: [period_end@1 DESC], fetch=2 +02)--SortExec: TopK(fetch=2), expr=[period_end@1 DESC], preserve_partitioning=[true] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------FilterExec: timeframe@0 = quarterly +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=timeframe@0 = quarterly AND DynamicFilter [ empty ], pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] + +# Results should still be correct +query TIR +SELECT * FROM timeseries_parquet +WHERE timeframe = 'quarterly' +ORDER BY period_end DESC +LIMIT 2; +---- +quarterly 3 6000 +quarterly 2 5500 + +# Re-enable +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; + +# Test 7.8: Test with IN clause (multiple constant values) +# Note: IN clause with multiple values means timeframe is NOT constant +# (could be 'daily' or 'weekly'), so the first sort column cannot be eliminated. +# Without a constant first column, we cannot reverse scan based on just period_end DESC. +# The physical plan should NOT show reverse_row_groups=true +query TT +EXPLAIN SELECT * FROM timeseries_parquet +WHERE timeframe IN ('daily', 'weekly') +ORDER BY period_end DESC +LIMIT 3; +---- +logical_plan +01)Sort: timeseries_parquet.period_end DESC NULLS FIRST, fetch=3 +02)--Filter: timeseries_parquet.timeframe = Utf8View("daily") OR timeseries_parquet.timeframe = Utf8View("weekly") +03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("daily") OR timeseries_parquet.timeframe = Utf8View("weekly")] +physical_plan +01)SortPreservingMergeExec: [period_end@1 DESC], fetch=3 +02)--SortExec: TopK(fetch=3), expr=[period_end@1 DESC], preserve_partitioning=[true] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------FilterExec: timeframe@0 = daily OR timeframe@0 = weekly +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=(timeframe@0 = daily OR timeframe@0 = weekly) AND DynamicFilter [ empty ], pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= daily AND daily <= timeframe_max@1 OR timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= weekly AND weekly <= timeframe_max@1, required_guarantees=[timeframe in (daily, weekly)] + +# Test 7.9: Complex case - literal constant in sort expression itself +# The literal 'constant' is ignored in sort analysis +# After stripping: ORDER BY period_end DESC +# With WHERE timeframe='monthly' making first column constant +# File: [period_end ASC] (after constant column removal) +# Request: [period_end DESC] -> exact reverse, triggers reverse scan +query TT +EXPLAIN SELECT * FROM timeseries_parquet +WHERE timeframe = 'monthly' +ORDER BY 'constant', period_end DESC +LIMIT 2; +---- +logical_plan +01)Sort: Utf8("constant") ASC NULLS LAST, timeseries_parquet.period_end DESC NULLS FIRST, fetch=2 +02)--Filter: timeseries_parquet.timeframe = Utf8View("monthly") +03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("monthly")] +physical_plan +01)SortPreservingMergeExec: [constant ASC NULLS LAST, period_end@1 DESC], fetch=2 +02)--SortExec: TopK(fetch=2), expr=[period_end@1 DESC], preserve_partitioning=[true] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------FilterExec: timeframe@0 = monthly +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=timeframe@0 = monthly AND DynamicFilter [ empty ], reverse_row_groups=true, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= monthly AND monthly <= timeframe_max@1, required_guarantees=[timeframe in (monthly)] + +# Verify results +query TIR +SELECT * FROM timeseries_parquet +WHERE timeframe = 'monthly' +ORDER BY period_end DESC +LIMIT 2; +---- +monthly 3 1200 +monthly 2 1100 + +# Test 7.10: Filter on non-leading sort column +# File order: [timeframe ASC, period_end ASC] +# Filter: period_end = 2 (makes second column constant) +# Request: [timeframe DESC] +# After constant column removal: File has [timeframe ASC], Request wants [timeframe DESC] +# This is exact reverse -> triggers reverse scan +query TT +EXPLAIN SELECT * FROM timeseries_parquet +WHERE period_end = 2 +ORDER BY timeframe DESC; +---- +logical_plan +01)Sort: timeseries_parquet.timeframe DESC NULLS FIRST +02)--Filter: timeseries_parquet.period_end = Int32(2) +03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.period_end = Int32(2)] +physical_plan +01)SortPreservingMergeExec: [timeframe@0 DESC] +02)--SortExec: expr=[timeframe@0 DESC], preserve_partitioning=[true] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------FilterExec: period_end@1 = 2 +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=period_end@1 = 2, reverse_row_groups=true, pruning_predicate=period_end_null_count@2 != row_count@3 AND period_end_min@0 <= 2 AND 2 <= period_end_max@1, required_guarantees=[period_end in (2)] + +# Cleanup +statement ok +DROP TABLE timeseries_data; + +statement ok +DROP TABLE timeseries_parquet; + +# Reset to default +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; + + +# Test 8: Sort pushdown with monotonic functions +# This tests that reverse scan works when sort expressions involve monotonic functions + +# Create test data with timestamp column +statement ok +CREATE TABLE timestamp_data(id INT, ts TIMESTAMP, volume BIGINT, price DOUBLE) AS VALUES +(1, TIMESTAMP '2024-01-15 10:00:00', 1000, 100.0), +(2, TIMESTAMP '2024-01-20 11:00:00', 1500, 105.0), +(3, TIMESTAMP '2024-01-25 12:00:00', 2000, 110.0), +(4, TIMESTAMP '2024-02-05 09:00:00', 1200, 108.0), +(5, TIMESTAMP '2024-02-15 14:00:00', 1800, 112.0), +(6, TIMESTAMP '2024-02-25 15:00:00', 2200, 115.0), +(7, TIMESTAMP '2024-03-10 09:00:00', 1300, 113.0), +(8, TIMESTAMP '2024-03-18 14:00:00', 1900, 118.0), +(9, TIMESTAMP '2024-03-28 15:00:00', 2300, 120.0); + +# Copy to parquet with sorting by timestamp ASC +query I +COPY (SELECT * FROM timestamp_data ORDER BY ts ASC) +TO 'test_files/scratch/dynamic_filter_pushdown_config/timestamp_sorted.parquet'; +---- +9 + +# Test 8.1: Simple monotonic function - date_trunc +# Create external table with file ordering that conceptually includes date_trunc +# File is actually sorted by [ts ASC], but conceptually [date_trunc('month', ts) ASC, ts ASC] +statement ok +CREATE EXTERNAL TABLE timestamp_parquet(id INT, ts TIMESTAMP, volume BIGINT, price DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/timestamp_sorted.parquet' +WITH ORDER (ts ASC); + +# Query with ORDER BY ts DESC +# File ordering: [ts ASC] +# Request: [ts DESC] +# This should trigger reverse_row_groups=true +query TT +EXPLAIN SELECT * FROM timestamp_parquet +ORDER BY ts DESC +LIMIT 3; +---- +logical_plan +01)Sort: timestamp_parquet.ts DESC NULLS FIRST, fetch=3 +02)--TableScan: timestamp_parquet projection=[id, ts, volume, price] +physical_plan +01)SortExec: TopK(fetch=3), expr=[ts@1 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timestamp_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +# Verify results +query IPIR +SELECT * FROM timestamp_parquet +ORDER BY ts DESC +LIMIT 3; +---- +9 2024-03-28T15:00:00 2300 120 +8 2024-03-18T14:00:00 1900 118 +7 2024-03-10T09:00:00 1300 113 + +# Test 8.2: Monotonic function in ORDER BY - date_trunc DESC +# File ordering: [ts ASC] +# Request: [date_trunc('day', ts) DESC] +# Since date_trunc is monotonic with ts, reversed file ordering [ts DESC] satisfies [date_trunc DESC] +query TT +EXPLAIN SELECT * FROM timestamp_parquet +ORDER BY date_trunc('day', ts) DESC +LIMIT 3; +---- +logical_plan +01)Sort: date_trunc(Utf8("day"), timestamp_parquet.ts) DESC NULLS FIRST, fetch=3 +02)--TableScan: timestamp_parquet projection=[id, ts, volume, price] +physical_plan +01)SortExec: TopK(fetch=3), expr=[date_trunc(day, ts@1) DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timestamp_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +# Verify results (descending day) +query IPIR +SELECT * FROM timestamp_parquet +ORDER BY date_trunc('day', ts) DESC +LIMIT 3; +---- +9 2024-03-28T15:00:00 2300 120 +8 2024-03-18T14:00:00 1900 118 +7 2024-03-10T09:00:00 1300 113 + +# Test 8.3: Multi-column scenario with explicit monotonic function in file ordering +# Create a table where we explicitly declare the ordering includes the monotonic function +# This simulates files that are partitioned/sorted by [date_trunc('month', ts) ASC, ts ASC] + +# Create a new parquet file sorted by [ts ASC] (which implies date_trunc ordering) +statement ok +CREATE TABLE multi_month_data(id INT, ts TIMESTAMP, volume BIGINT, price DOUBLE) AS VALUES +-- January 2024 +(1, TIMESTAMP '2024-01-05 09:30:00', 1000, 100.0), +(2, TIMESTAMP '2024-01-15 14:30:00', 1500, 105.0), +(3, TIMESTAMP '2024-01-25 15:59:00', 2000, 110.0), +-- February 2024 +(4, TIMESTAMP '2024-02-03 09:30:00', 1200, 108.0), +(5, TIMESTAMP '2024-02-14 12:00:00', 1800, 112.0), +(6, TIMESTAMP '2024-02-28 15:59:00', 2200, 115.0), +-- March 2024 +(7, TIMESTAMP '2024-03-01 09:30:00', 1300, 113.0), +(8, TIMESTAMP '2024-03-15 14:00:00', 1900, 118.0), +(9, TIMESTAMP '2024-03-29 15:59:00', 2300, 120.0); + +query I +COPY (SELECT * FROM multi_month_data ORDER BY ts ASC) +TO 'test_files/scratch/dynamic_filter_pushdown_config/multi_month_sorted.parquet'; +---- +9 + +# Declare the file has ordering [ts ASC] +# Conceptually this means [date_trunc('month', ts) ASC, ts ASC] due to monotonicity +statement ok +CREATE EXTERNAL TABLE multi_month_parquet(id INT, ts TIMESTAMP, volume BIGINT, price DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/multi_month_sorted.parquet' +WITH ORDER (ts ASC); + +# Test 8.3a: Request ORDER BY ts DESC (opposite direction) +# File: [ts ASC] +# Request: [ts DESC] +# Should trigger reverse_row_groups=true +query TT +EXPLAIN SELECT * FROM multi_month_parquet +ORDER BY ts DESC +LIMIT 2; +---- +logical_plan +01)Sort: multi_month_parquet.ts DESC NULLS FIRST, fetch=2 +02)--TableScan: multi_month_parquet projection=[id, ts, volume, price] +physical_plan +01)SortExec: TopK(fetch=2), expr=[ts@1 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/multi_month_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +query IPIR +SELECT * FROM multi_month_parquet +ORDER BY ts DESC +LIMIT 2; +---- +9 2024-03-29T15:59:00 2300 120 +8 2024-03-15T14:00:00 1900 118 + +# Test 8.3b: Request ORDER BY date_trunc('month', ts) DESC, ts DESC +# File: [ts ASC] (which implies [date_trunc('month', ts) ASC, ts ASC]) +# Request: [date_trunc('month', ts) DESC, ts DESC] +# The reversed file ordering [ts DESC] satisfies this because: +# - date_trunc is monotonic with ts +# - So [ts DESC] implies [date_trunc('month', ts) DESC, ts DESC] +query TT +EXPLAIN SELECT * FROM multi_month_parquet +ORDER BY date_trunc('month', ts) DESC, ts DESC +LIMIT 2; +---- +logical_plan +01)Sort: date_trunc(Utf8("month"), multi_month_parquet.ts) DESC NULLS FIRST, multi_month_parquet.ts DESC NULLS FIRST, fetch=2 +02)--TableScan: multi_month_parquet projection=[id, ts, volume, price] +physical_plan +01)SortExec: TopK(fetch=2), expr=[date_trunc(month, ts@1) DESC, ts@1 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/multi_month_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +query IPIR +SELECT * FROM multi_month_parquet +ORDER BY date_trunc('month', ts) DESC, ts DESC +LIMIT 2; +---- +9 2024-03-29T15:59:00 2300 120 +8 2024-03-15T14:00:00 1900 118 + +# Test 8.4: CAST as a monotonic function +statement ok +CREATE TABLE int_data(id INT, small_val SMALLINT, big_val BIGINT) AS VALUES +(1, 10, 100), +(2, 20, 200), +(3, 30, 300), +(4, 40, 400), +(5, 50, 500); + +query I +COPY (SELECT * FROM int_data ORDER BY small_val ASC) +TO 'test_files/scratch/dynamic_filter_pushdown_config/int_sorted.parquet'; +---- +5 + +statement ok +CREATE EXTERNAL TABLE int_parquet(id INT, small_val SMALLINT, big_val BIGINT) +STORED AS PARQUET +LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/int_sorted.parquet' +WITH ORDER (small_val ASC); + +# CAST preserves ordering: CAST(small_val AS BIGINT) is monotonic with small_val +query TT +EXPLAIN SELECT * FROM int_parquet +ORDER BY CAST(small_val AS BIGINT) DESC +LIMIT 2; +---- +logical_plan +01)Sort: CAST(int_parquet.small_val AS Int64) DESC NULLS FIRST, fetch=2 +02)--TableScan: int_parquet projection=[id, small_val, big_val] +physical_plan +01)SortExec: TopK(fetch=2), expr=[CAST(small_val@1 AS Int64) DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/int_sorted.parquet]]}, projection=[id, small_val, big_val], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +query III +SELECT * FROM int_parquet +ORDER BY CAST(small_val AS BIGINT) DESC +LIMIT 2; +---- +5 50 500 +4 40 400 + +# Test 8.5: CEIL as a monotonic function +statement ok +CREATE TABLE float_data(id INT, value DOUBLE) AS VALUES +(1, 1.1), +(2, 2.3), +(3, 3.5), +(4, 4.7), +(5, 5.9); + +query I +COPY (SELECT * FROM float_data ORDER BY value ASC) +TO 'test_files/scratch/dynamic_filter_pushdown_config/float_sorted.parquet'; +---- +5 + +statement ok +CREATE EXTERNAL TABLE float_parquet(id INT, value DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/float_sorted.parquet' +WITH ORDER (value ASC); + +# CEIL is monotonic increasing +query TT +EXPLAIN SELECT * FROM float_parquet +ORDER BY CEIL(value) DESC +LIMIT 3; +---- +logical_plan +01)Sort: ceil(float_parquet.value) DESC NULLS FIRST, fetch=3 +02)--TableScan: float_parquet projection=[id, value] +physical_plan +01)SortExec: TopK(fetch=3), expr=[ceil(value@1) DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/float_sorted.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +query IR +SELECT * FROM float_parquet +ORDER BY CEIL(value) DESC +LIMIT 3; +---- +5 5.9 +4 4.7 +3 3.5 + +# Test 8.6: Negative case - ABS is NOT monotonic over mixed positive/negative range +statement ok +CREATE TABLE signed_data(id INT, value DOUBLE) AS VALUES +(1, -5.0), +(2, -3.0), +(3, -1.0), +(4, 2.0), +(5, 4.0); + +query I +COPY (SELECT * FROM signed_data ORDER BY value ASC) +TO 'test_files/scratch/dynamic_filter_pushdown_config/signed_sorted.parquet'; +---- +5 + +statement ok +CREATE EXTERNAL TABLE signed_parquet(id INT, value DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/signed_sorted.parquet' +WITH ORDER (value ASC); + +# ABS is NOT monotonic over the full range [-5, 4], so should NOT trigger reverse scan +query TT +EXPLAIN SELECT * FROM signed_parquet +ORDER BY ABS(value) DESC +LIMIT 3; +---- +logical_plan +01)Sort: abs(signed_parquet.value) DESC NULLS FIRST, fetch=3 +02)--TableScan: signed_parquet projection=[id, value] +physical_plan +01)SortExec: TopK(fetch=3), expr=[abs(value@1) DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/signed_sorted.parquet]]}, projection=[id, value], output_ordering=[value@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] + +# Results should still be correct (no optimization applied) +query IR +SELECT * FROM signed_parquet +ORDER BY ABS(value) DESC +LIMIT 3; +---- +1 -5 +5 4 +2 -3 + +# Cleanup +statement ok +DROP TABLE timestamp_data; + +statement ok +DROP TABLE timestamp_parquet; + +statement ok +DROP TABLE multi_month_data; + +statement ok +DROP TABLE multi_month_parquet; + +statement ok +DROP TABLE int_data; + +statement ok +DROP TABLE int_parquet; + +statement ok +DROP TABLE float_data; + +statement ok +DROP TABLE float_parquet; + +statement ok +DROP TABLE signed_data; + +statement ok +DROP TABLE signed_parquet; + +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; From f4fa2665ce5b28cc22d97bddfa7a948e7f8a6739 Mon Sep 17 00:00:00 2001 From: Qi Zhu Date: Wed, 31 Dec 2025 14:35:33 +0800 Subject: [PATCH 04/10] fix: Reverse row selection should respect the row group index (#19557) - Closes [#19535](https://github.com/apache/datafusion/issues/19535) Reverse row selection should respect the row group index, this PR will fix the issue. Reverse row selection should respect the row group index, this PR will fix the issue. Yes No (cherry picked from commit 27de50d0553b7bbf1ec4172ec87bc23c7f2a5acb) --- datafusion/datasource-parquet/src/opener.rs | 142 +++- datafusion/datasource-parquet/src/sort.rs | 865 +++++++++++++++++--- 2 files changed, 873 insertions(+), 134 deletions(-) diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 59a9f76b15cfb..7774c49832a0e 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -119,16 +119,16 @@ pub(super) struct ParquetOpener { } /// Represents a prepared access plan with optional row selection -struct PreparedAccessPlan { +pub(crate) struct PreparedAccessPlan { /// Row group indexes to read - row_group_indexes: Vec, + pub(crate) row_group_indexes: Vec, /// Optional row selection for filtering within row groups - row_selection: Option, + pub(crate) row_selection: Option, } impl PreparedAccessPlan { /// Create a new prepared access plan from a ParquetAccessPlan - fn from_access_plan( + pub(crate) fn from_access_plan( access_plan: ParquetAccessPlan, rg_metadata: &[RowGroupMetaData], ) -> Result { @@ -142,17 +142,23 @@ impl PreparedAccessPlan { } /// Reverse the access plan for reverse scanning - fn reverse( + pub(crate) fn reverse( mut self, file_metadata: &parquet::file::metadata::ParquetMetaData, ) -> Result { + // Get the row group indexes before reversing + let row_groups_to_scan = self.row_group_indexes.clone(); + // Reverse the row group indexes self.row_group_indexes = self.row_group_indexes.into_iter().rev().collect(); // If we have a row selection, reverse it to match the new row group order if let Some(row_selection) = self.row_selection { - self.row_selection = - Some(reverse_row_selection(&row_selection, file_metadata)?); + self.row_selection = Some(reverse_row_selection( + &row_selection, + file_metadata, + &row_groups_to_scan, // Pass the original (non-reversed) row group indexes + )?); } Ok(self) @@ -826,8 +832,9 @@ fn should_enable_page_index( #[cfg(test)] mod test { - use crate::{opener::ParquetOpener, DefaultParquetFileReaderFactory}; + use std::sync::Arc; use arrow::compute::cast; + use crate::{DefaultParquetFileReaderFactory, RowGroupAccess, opener::ParquetOpener}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use bytes::{BufMut, BytesMut}; use datafusion_common::{ @@ -851,7 +858,6 @@ mod test { use object_store::{memory::InMemory, path::Path, ObjectStore}; use parquet::arrow::ArrowWriter; use parquet::file::properties::WriterProperties; - use std::sync::Arc; async fn count_batches_and_rows( mut stream: std::pin::Pin< @@ -1726,4 +1732,122 @@ mod test { "Reverse scan should reverse row group order while maintaining correct RowSelection for each group" ); } + + #[tokio::test] + async fn test_reverse_scan_with_non_contiguous_row_groups() { + use parquet::file::properties::WriterProperties; + + let store = Arc::new(InMemory::new()) as Arc; + + // Create 4 batches (4 row groups) + let batch0 = record_batch!(("a", Int32, vec![Some(1), Some(2)])).unwrap(); + let batch1 = record_batch!(("a", Int32, vec![Some(3), Some(4)])).unwrap(); + let batch2 = record_batch!(("a", Int32, vec![Some(5), Some(6)])).unwrap(); + let batch3 = record_batch!(("a", Int32, vec![Some(7), Some(8)])).unwrap(); + + let props = WriterProperties::builder() + .set_max_row_group_size(2) + .build(); + + let data_len = write_parquet_batches( + Arc::clone(&store), + "test.parquet", + vec![batch0.clone(), batch1, batch2, batch3], + Some(props), + ) + .await; + + let schema = batch0.schema(); + + use crate::ParquetAccessPlan; + use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; + + // KEY: Skip RG1 (non-contiguous!) + // Only scan row groups: [0, 2, 3] + let mut access_plan = ParquetAccessPlan::new(vec![ + RowGroupAccess::Scan, // RG0 + RowGroupAccess::Skip, // RG1 - SKIPPED! + RowGroupAccess::Scan, // RG2 + RowGroupAccess::Scan, // RG3 + ]); + + // Add RowSelection for each scanned row group + // RG0: select first row (1), skip second (2) + access_plan.scan_selection( + 0, + RowSelection::from(vec![RowSelector::select(1), RowSelector::skip(1)]), + ); + // RG1: skipped, no selection needed + // RG2: select first row (5), skip second (6) + access_plan.scan_selection( + 2, + RowSelection::from(vec![RowSelector::select(1), RowSelector::skip(1)]), + ); + // RG3: select first row (7), skip second (8) + access_plan.scan_selection( + 3, + RowSelection::from(vec![RowSelector::select(1), RowSelector::skip(1)]), + ); + + let file = PartitionedFile::new( + "test.parquet".to_string(), + u64::try_from(data_len).unwrap(), + ) + .with_extensions(Arc::new(access_plan)); + + let make_opener = |reverse_scan: bool| ParquetOpener { + projection: Arc::new([0]), + partition_index: 0, + batch_size: 1024, + limit: None, + predicate: None, + metadata_size_hint: None, + metrics: ExecutionPlanMetricsSet::new(), + parquet_file_reader_factory: Arc::new(DefaultParquetFileReaderFactory::new( + Arc::clone(&store), + )), + partition_fields: vec![], + pushdown_filters: true, + reorder_filters: false, + enable_page_index: false, + enable_bloom_filter: false, + enable_limit_pruning: false, + schema_adapter_factory: Arc::new(DefaultSchemaAdapterFactory), + enable_row_group_stats_pruning: false, + coerce_int96: None, + #[cfg(feature = "parquet_encryption")] + file_decryption_properties: None, + expr_adapter_factory: Some(Arc::new(DefaultPhysicalExprAdapterFactory)), + #[cfg(feature = "parquet_encryption")] + encryption_factory: None, + max_predicate_cache_size: None, + reverse_row_groups: reverse_scan, + logical_file_schema: schema.clone(), + }; + + // Forward scan: RG0(1), RG2(5), RG3(7) + // Note: RG1 is completely skipped + let opener = make_opener(false); + let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let forward_values = collect_int32_values(stream).await; + + assert_eq!( + forward_values, + vec![1, 5, 7], + "Forward scan with non-contiguous row groups" + ); + + // Reverse scan: RG3(7), RG2(5), RG0(1) + // WITHOUT the bug fix, this would return WRONG values + // because the RowSelection would be incorrectly mapped + let opener = make_opener(true); + let stream = opener.open(file).unwrap().await.unwrap(); + let reverse_values = collect_int32_values(stream).await; + + assert_eq!( + reverse_values, + vec![7, 5, 1], + "Reverse scan with non-contiguous row groups should correctly map RowSelection" + ); + } } diff --git a/datafusion/datasource-parquet/src/sort.rs b/datafusion/datasource-parquet/src/sort.rs index 4255d4d6960b1..abc50eeb317d5 100644 --- a/datafusion/datasource-parquet/src/sort.rs +++ b/datafusion/datasource-parquet/src/sort.rs @@ -31,25 +31,43 @@ use std::collections::HashMap; /// 3. Reconstructs the row selection for the new order /// /// # Arguments -/// * `row_selection` - Original row selection +/// * `row_selection` - Original row selection (only covers row groups that are scanned) /// * `parquet_metadata` - Metadata containing row group information +/// * `row_groups_to_scan` - Indexes of row groups that will be scanned (in original order) /// /// # Returns /// A new `RowSelection` adjusted for reversed row group order +/// +/// # Important Notes +/// The input `row_selection` only covers the row groups specified in `row_groups_to_scan`. +/// Row groups that are skipped (not in `row_groups_to_scan`) are not represented in the +/// `row_selection` at all. This function needs `row_groups_to_scan` to correctly map +/// the selection back to the original row groups. pub fn reverse_row_selection( row_selection: &RowSelection, parquet_metadata: &ParquetMetaData, + row_groups_to_scan: &[usize], ) -> Result { let rg_metadata = parquet_metadata.row_groups(); - // Build a mapping of row group index to its row range in the file + // Build a mapping of row group index to its row range, but ONLY for + // the row groups that are actually being scanned. + // + // IMPORTANT: The row numbers in this mapping are RELATIVE to the scanned row groups, + // not absolute positions in the file. + // + // Example: If row_groups_to_scan = [0, 2, 3] and each has 100 rows: + // RG0: rows 0-99 (relative to scanned data) + // RG2: rows 100-199 (relative to scanned data, NOT 200-299 in file!) + // RG3: rows 200-299 (relative to scanned data, NOT 300-399 in file!) let mut rg_row_ranges: Vec<(usize, usize, usize)> = - Vec::with_capacity(rg_metadata.len()); + Vec::with_capacity(row_groups_to_scan.len()); let mut current_row = 0; - for (rg_idx, rg) in rg_metadata.iter().enumerate() { + for &rg_idx in row_groups_to_scan { + let rg = &rg_metadata[rg_idx]; let num_rows = rg.num_rows() as usize; rg_row_ranges.push((rg_idx, current_row, current_row + num_rows)); - current_row += num_rows; + current_row += num_rows; // This is relative row number, NOT absolute file position } // Map selections to row groups @@ -82,12 +100,13 @@ pub fn reverse_row_selection( } // Build new selection for reversed row group order + // Only iterate over the row groups that are being scanned, in reverse order let mut reversed_selectors = Vec::new(); - for rg_idx in (0..rg_metadata.len()).rev() { + for &rg_idx in row_groups_to_scan.iter().rev() { if let Some(selectors) = rg_selections.get(&rg_idx) { reversed_selectors.extend(selectors.iter().cloned()); } else { - // No specific selection for this row group means select all + // No specific selection for this row group means select all rows in it if let Some((_, start, end)) = rg_row_ranges.iter().find(|(idx, _, _)| *idx == rg_idx) { @@ -101,32 +120,30 @@ pub fn reverse_row_selection( #[cfg(test)] mod tests { - use super::*; + use crate::ParquetAccessPlan; + use crate::RowGroupAccess; + use crate::opener::PreparedAccessPlan; use arrow::datatypes::{DataType, Field, Schema}; use bytes::Bytes; use parquet::arrow::ArrowWriter; + use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; use parquet::file::reader::FileReader; use parquet::file::serialized_reader::SerializedFileReader; use std::sync::Arc; /// Helper function to create a ParquetMetaData with specified row group sizes /// by actually writing a parquet file in memory - fn create_test_metadata(row_group_sizes: Vec) -> ParquetMetaData { - // Create a simple schema + fn create_test_metadata( + row_group_sizes: Vec, + ) -> parquet::file::metadata::ParquetMetaData { let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); - - // Create in-memory parquet file with the specified row groups let mut buffer = Vec::new(); { - let props = parquet::file::properties::WriterProperties::builder() - .set_max_row_group_size(row_group_sizes[0] as usize) - .build(); - + let props = parquet::file::properties::WriterProperties::builder().build(); let mut writer = ArrowWriter::try_new(&mut buffer, schema.clone(), Some(props)).unwrap(); for &size in &row_group_sizes { - // Create a batch with the specified number of rows let array = arrow::array::Int32Array::from(vec![1; size as usize]); let batch = arrow::record_batch::RecordBatch::try_new( schema.clone(), @@ -134,34 +151,131 @@ mod tests { ) .unwrap(); writer.write(&batch).unwrap(); + writer.flush().unwrap(); } writer.close().unwrap(); } - // Read back the metadata let bytes = Bytes::from(buffer); let reader = SerializedFileReader::new(bytes).unwrap(); reader.metadata().clone() } #[test] - fn test_reverse_simple_selection() { - // 3 row groups with 100 rows each + fn test_prepared_access_plan_reverse_simple() { + // Test: all row groups are scanned, no row selection + let metadata = create_test_metadata(vec![100, 100, 100]); + + let access_plan = ParquetAccessPlan::new_all(3); + let rg_metadata = metadata.row_groups(); + + let prepared_plan = + PreparedAccessPlan::from_access_plan(access_plan, rg_metadata) + .expect("Failed to create PreparedAccessPlan"); + + // Verify original plan + assert_eq!(prepared_plan.row_group_indexes, vec![0, 1, 2]); + + // No row selection originally due to scanning all rows + assert_eq!(prepared_plan.row_selection, None); + + let reversed_plan = prepared_plan + .reverse(&metadata) + .expect("Failed to reverse PreparedAccessPlan"); + + // Verify row groups are reversed + assert_eq!(reversed_plan.row_group_indexes, vec![2, 1, 0]); + + // If no selection originally, after reversal should still select all rows, + // and the selection should be None + assert_eq!(reversed_plan.row_selection, None); + } + + #[test] + fn test_prepared_access_plan_reverse_with_selection() { + // Test: simple row selection that spans multiple row groups let metadata = create_test_metadata(vec![100, 100, 100]); - // Select first 50 rows from first row group - let selection = - RowSelection::from(vec![RowSelector::select(50), RowSelector::skip(250)]); + let mut access_plan = ParquetAccessPlan::new_all(3); + + // Select first 50 rows from first row group, skip rest + access_plan.scan_selection( + 0, + RowSelection::from(vec![RowSelector::select(50), RowSelector::skip(50)]), + ); + + let rg_metadata = metadata.row_groups(); + let prepared_plan = + PreparedAccessPlan::from_access_plan(access_plan, rg_metadata) + .expect("Failed to create PreparedAccessPlan"); - let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + let original_selected: usize = prepared_plan + .row_selection + .as_ref() + .unwrap() + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + + let reversed_plan = prepared_plan + .reverse(&metadata) + .expect("Failed to reverse PreparedAccessPlan"); + + let reversed_selected: usize = reversed_plan + .row_selection + .as_ref() + .unwrap() + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + + assert_eq!( + original_selected, reversed_selected, + "Total selected rows should remain the same" + ); + } + + #[test] + fn test_prepared_access_plan_reverse_multi_row_group_selection() { + // Test: row selection spanning multiple row groups + let metadata = create_test_metadata(vec![100, 100, 100]); - // Verify total selected rows remain the same - let original_selected: usize = selection + let mut access_plan = ParquetAccessPlan::new_all(3); + + // Create selection that spans RG0 and RG1 + access_plan.scan_selection( + 0, + RowSelection::from(vec![RowSelector::skip(50), RowSelector::select(50)]), + ); + access_plan.scan_selection( + 1, + RowSelection::from(vec![RowSelector::select(50), RowSelector::skip(50)]), + ); + + let rg_metadata = metadata.row_groups(); + let prepared_plan = + PreparedAccessPlan::from_access_plan(access_plan, rg_metadata) + .expect("Failed to create PreparedAccessPlan"); + + let original_selected: usize = prepared_plan + .row_selection + .as_ref() + .unwrap() .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); - let reversed_selected: usize = reversed + + let reversed_plan = prepared_plan + .reverse(&metadata) + .expect("Failed to reverse PreparedAccessPlan"); + + let reversed_selected: usize = reversed_plan + .row_selection + .as_ref() + .unwrap() .iter() .filter(|s| !s.skip) .map(|s| s.row_count) @@ -171,25 +285,80 @@ mod tests { } #[test] - fn test_reverse_multi_row_group_selection() { + fn test_prepared_access_plan_reverse_empty_selection() { + // Test: all rows are skipped let metadata = create_test_metadata(vec![100, 100, 100]); - // Select rows spanning multiple row groups - let selection = RowSelection::from(vec![ - RowSelector::skip(50), - RowSelector::select(100), // Spans RG0 and RG1 - RowSelector::skip(150), - ]); + let mut access_plan = ParquetAccessPlan::new_all(3); + + // Skip all rows in all row groups + for i in 0..3 { + access_plan + .scan_selection(i, RowSelection::from(vec![RowSelector::skip(100)])); + } + + let rg_metadata = metadata.row_groups(); + let prepared_plan = + PreparedAccessPlan::from_access_plan(access_plan, rg_metadata) + .expect("Failed to create PreparedAccessPlan"); - let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + let reversed_plan = prepared_plan + .reverse(&metadata) + .expect("Failed to reverse PreparedAccessPlan"); - // Verify total selected rows remain the same - let original_selected: usize = selection + // Should still skip all rows + let total_selected: usize = reversed_plan + .row_selection + .as_ref() + .unwrap() .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); - let reversed_selected: usize = reversed + + assert_eq!(total_selected, 0); + } + + #[test] + fn test_prepared_access_plan_reverse_different_row_group_sizes() { + // Test: row groups with different sizes + let metadata = create_test_metadata(vec![50, 150, 100]); + + let mut access_plan = ParquetAccessPlan::new_all(3); + + // Create complex selection pattern + access_plan.scan_selection( + 0, + RowSelection::from(vec![RowSelector::skip(25), RowSelector::select(25)]), + ); + access_plan.scan_selection(1, RowSelection::from(vec![RowSelector::select(150)])); + access_plan.scan_selection( + 2, + RowSelection::from(vec![RowSelector::select(50), RowSelector::skip(50)]), + ); + + let rg_metadata = metadata.row_groups(); + let prepared_plan = + PreparedAccessPlan::from_access_plan(access_plan, rg_metadata) + .expect("Failed to create PreparedAccessPlan"); + + let original_selected: usize = prepared_plan + .row_selection + .as_ref() + .unwrap() + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); + + let reversed_plan = prepared_plan + .reverse(&metadata) + .expect("Failed to reverse PreparedAccessPlan"); + + let reversed_selected: usize = reversed_plan + .row_selection + .as_ref() + .unwrap() .iter() .filter(|s| !s.skip) .map(|s| s.row_count) @@ -199,209 +368,655 @@ mod tests { } #[test] - fn test_reverse_full_selection() { - let metadata = create_test_metadata(vec![100, 100, 100]); + fn test_prepared_access_plan_reverse_single_row_group() { + // Test: single row group case + let metadata = create_test_metadata(vec![100]); - // Select all rows - let selection = RowSelection::from(vec![RowSelector::select(300)]); + let mut access_plan = ParquetAccessPlan::new_all(1); + access_plan.scan_selection( + 0, + RowSelection::from(vec![RowSelector::select(50), RowSelector::skip(50)]), + ); + + let rg_metadata = metadata.row_groups(); + let prepared_plan = + PreparedAccessPlan::from_access_plan(access_plan, rg_metadata) + .expect("Failed to create PreparedAccessPlan"); + + let original_selected: usize = prepared_plan + .row_selection + .as_ref() + .unwrap() + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); - let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + let reversed_plan = prepared_plan + .reverse(&metadata) + .expect("Failed to reverse PreparedAccessPlan"); - // Should still select all rows, just in reversed row group order - let total_selected: usize = reversed + // With single row group, row_group_indexes should remain [0] + assert_eq!(reversed_plan.row_group_indexes, vec![0]); + + let reversed_selected: usize = reversed_plan + .row_selection + .as_ref() + .unwrap() .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); - assert_eq!(total_selected, 300); + assert_eq!(original_selected, reversed_selected); + assert_eq!(original_selected, 50); } #[test] - fn test_reverse_empty_selection() { + fn test_prepared_access_plan_reverse_complex_pattern() { + // Test: complex pattern with multiple select/skip segments let metadata = create_test_metadata(vec![100, 100, 100]); - // Skip all rows - let selection = RowSelection::from(vec![RowSelector::skip(300)]); + let mut access_plan = ParquetAccessPlan::new_all(3); + + // Complex pattern: select some, skip some, select some more + access_plan.scan_selection( + 0, + RowSelection::from(vec![ + RowSelector::select(30), + RowSelector::skip(40), + RowSelector::select(30), + ]), + ); + access_plan.scan_selection( + 1, + RowSelection::from(vec![RowSelector::skip(50), RowSelector::select(50)]), + ); + access_plan.scan_selection(2, RowSelection::from(vec![RowSelector::select(100)])); + + let rg_metadata = metadata.row_groups(); + let prepared_plan = + PreparedAccessPlan::from_access_plan(access_plan, rg_metadata) + .expect("Failed to create PreparedAccessPlan"); + + let original_selected: usize = prepared_plan + .row_selection + .as_ref() + .unwrap() + .iter() + .filter(|s| !s.skip) + .map(|s| s.row_count) + .sum(); - let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + let reversed_plan = prepared_plan + .reverse(&metadata) + .expect("Failed to reverse PreparedAccessPlan"); - // Should still skip all rows - let total_selected: usize = reversed + let reversed_selected: usize = reversed_plan + .row_selection + .as_ref() + .unwrap() .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); - assert_eq!(total_selected, 0); + assert_eq!(original_selected, reversed_selected); + assert_eq!(original_selected, 210); // 30 + 30 + 50 + 100 } #[test] - fn test_reverse_with_different_row_group_sizes() { - let metadata = create_test_metadata(vec![50, 150, 100]); + fn test_prepared_access_plan_reverse_with_skipped_row_groups() { + // This is the KEY test case for the bug fix! + // Test scenario where some row groups are completely skipped (not in scan plan) + let metadata = create_test_metadata(vec![100, 100, 100, 100]); - let selection = RowSelection::from(vec![ - RowSelector::skip(25), - RowSelector::select(200), // Spans all row groups - RowSelector::skip(75), + // Scenario: RG0 (scan all), RG1 (completely skipped), RG2 (partial), RG3 (scan all) + // Only row groups [0, 2, 3] are in the scan plan + let mut access_plan = ParquetAccessPlan::new(vec![ + RowGroupAccess::Scan, // RG0 + RowGroupAccess::Skip, // RG1 - NOT in scan plan! + RowGroupAccess::Scan, // RG2 + RowGroupAccess::Scan, // RG3 ]); - let reversed = reverse_row_selection(&selection, &metadata).unwrap(); - - let original_selected: usize = selection + // Add row selections for the scanned row groups + // Note: The RowSelection only covers row groups [0, 2, 3] (300 rows total) + access_plan.scan_selection( + 0, + RowSelection::from(vec![RowSelector::select(100)]), // RG0: all 100 rows + ); + // RG1 is skipped, no selection needed + access_plan.scan_selection( + 2, + RowSelection::from(vec![ + RowSelector::select(25), // RG2: first 25 rows + RowSelector::skip(75), // RG2: skip last 75 rows + ]), + ); + access_plan.scan_selection( + 3, + RowSelection::from(vec![RowSelector::select(100)]), // RG3: all 100 rows + ); + + let rg_metadata = metadata.row_groups(); + + // Step 1: Create PreparedAccessPlan + let prepared_plan = + PreparedAccessPlan::from_access_plan(access_plan, rg_metadata) + .expect("Failed to create PreparedAccessPlan"); + + // Verify original plan + assert_eq!(prepared_plan.row_group_indexes, vec![0, 2, 3]); + let original_selected: usize = prepared_plan + .row_selection + .as_ref() + .unwrap() .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); - let reversed_selected: usize = reversed + assert_eq!(original_selected, 225); // 100 + 25 + 100 + + // Step 2: Reverse the plan (this is the production code path) + let reversed_plan = prepared_plan + .reverse(&metadata) + .expect("Failed to reverse PreparedAccessPlan"); + + // Verify reversed results + // Row group order should be reversed: [3, 2, 0] + assert_eq!( + reversed_plan.row_group_indexes, + vec![3, 2, 0], + "Row groups should be reversed" + ); + + // Verify row selection is also correctly reversed + let reversed_selected: usize = reversed_plan + .row_selection + .as_ref() + .unwrap() .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); - assert_eq!(original_selected, reversed_selected); + assert_eq!( + reversed_selected, 225, + "Total selected rows should remain the same" + ); + + // Verify the reversed selection structure + // After reversal, the order becomes: RG3, RG2, RG0 + // - RG3: select(100) + // - RG2: select(25), skip(75) (note: internal order preserved, not reversed) + // - RG0: select(100) + // + // After RowSelection::from() merges adjacent selectors of the same type: + // - RG3's select(100) + RG2's select(25) = select(125) + // - RG2's skip(75) remains as skip(75) + // - RG0's select(100) remains as select(100) + let selectors: Vec<_> = reversed_plan + .row_selection + .as_ref() + .unwrap() + .iter() + .collect(); + assert_eq!(selectors.len(), 3); + + // RG3 (100) + RG2 first part (25) merged into select(125) + assert!(!selectors[0].skip); + assert_eq!(selectors[0].row_count, 125); + + // RG2: skip last 75 rows + assert!(selectors[1].skip); + assert_eq!(selectors[1].row_count, 75); + + // RG0: select all 100 rows + assert!(!selectors[2].skip); + assert_eq!(selectors[2].row_count, 100); } #[test] - fn test_reverse_single_row_group() { - let metadata = create_test_metadata(vec![100]); + fn test_prepared_access_plan_reverse_alternating_row_groups() { + // Test with alternating scan/skip pattern + let metadata = create_test_metadata(vec![100, 100, 100, 100]); - let selection = - RowSelection::from(vec![RowSelector::select(50), RowSelector::skip(50)]); + // Scan RG0 and RG2, skip RG1 and RG3 + let mut access_plan = ParquetAccessPlan::new(vec![ + RowGroupAccess::Scan, // RG0 + RowGroupAccess::Skip, // RG1 + RowGroupAccess::Scan, // RG2 + RowGroupAccess::Skip, // RG3 + ]); + + access_plan.scan_selection(0, RowSelection::from(vec![RowSelector::select(100)])); + access_plan.scan_selection(2, RowSelection::from(vec![RowSelector::select(100)])); - let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + let rg_metadata = metadata.row_groups(); + let prepared_plan = + PreparedAccessPlan::from_access_plan(access_plan, rg_metadata) + .expect("Failed to create PreparedAccessPlan"); - // With single row group, selection should remain the same - let original_selected: usize = selection + let original_selected: usize = prepared_plan + .row_selection + .as_ref() + .unwrap() .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); - let reversed_selected: usize = reversed + + // Original: [0, 2] + assert_eq!(prepared_plan.row_group_indexes, vec![0, 2]); + + let reversed_plan = prepared_plan + .reverse(&metadata) + .expect("Failed to reverse PreparedAccessPlan"); + + // After reverse: [2, 0] + assert_eq!(reversed_plan.row_group_indexes, vec![2, 0]); + + let reversed_selected: usize = reversed_plan + .row_selection + .as_ref() + .unwrap() .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); assert_eq!(original_selected, reversed_selected); + assert_eq!(original_selected, 200); } #[test] - fn test_reverse_complex_pattern() { + fn test_prepared_access_plan_reverse_middle_row_group_only() { + // Test selecting only the middle row group let metadata = create_test_metadata(vec![100, 100, 100]); - // Complex pattern: select some, skip some, select some more - let selection = RowSelection::from(vec![ - RowSelector::select(30), - RowSelector::skip(40), - RowSelector::select(80), - RowSelector::skip(50), - RowSelector::select(100), + let mut access_plan = ParquetAccessPlan::new(vec![ + RowGroupAccess::Skip, // RG0 + RowGroupAccess::Scan, // RG1 + RowGroupAccess::Skip, // RG2 ]); - let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + access_plan.scan_selection( + 1, + RowSelection::from(vec![RowSelector::select(100)]), // Select all of RG1 + ); - let original_selected: usize = selection + let rg_metadata = metadata.row_groups(); + let prepared_plan = + PreparedAccessPlan::from_access_plan(access_plan, rg_metadata) + .expect("Failed to create PreparedAccessPlan"); + + let original_selected: usize = prepared_plan + .row_selection + .as_ref() + .unwrap() .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); - let reversed_selected: usize = reversed + + // Original: [1] + assert_eq!(prepared_plan.row_group_indexes, vec![1]); + + let reversed_plan = prepared_plan + .reverse(&metadata) + .expect("Failed to reverse PreparedAccessPlan"); + + // After reverse: still [1] (only one row group) + assert_eq!(reversed_plan.row_group_indexes, vec![1]); + + let reversed_selected: usize = reversed_plan + .row_selection + .as_ref() + .unwrap() .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); assert_eq!(original_selected, reversed_selected); - assert_eq!(original_selected, 210); // 30 + 80 + 100 + assert_eq!(original_selected, 100); } #[test] - fn test_reverse_with_skipped_row_group() { - // This test covers the "no specific selection" code path (lines 90-95) - let metadata = create_test_metadata(vec![100, 100, 100]); + fn test_prepared_access_plan_reverse_with_skipped_row_groups_detailed() { + // This is the KEY test case for the bug fix! + // Test scenario where some row groups are completely skipped (not in scan plan) + // This version includes DETAILED verification of the selector distribution + let metadata = create_test_metadata(vec![100, 100, 100, 100]); - // Select only from first and third row groups, skip middle one entirely - let selection = RowSelection::from(vec![ - RowSelector::select(50), // First 50 of RG0 - RowSelector::skip(150), // Rest of RG0 + all of RG1 + half of RG2 - RowSelector::select(50), // Last 50 of RG2 + // Scenario: RG0 (scan all), RG1 (completely skipped), RG2 (partial), RG3 (scan all) + // Only row groups [0, 2, 3] are in the scan plan + let mut access_plan = ParquetAccessPlan::new(vec![ + RowGroupAccess::Scan, // RG0 + RowGroupAccess::Skip, // RG1 - NOT in scan plan! + RowGroupAccess::Scan, // RG2 + RowGroupAccess::Scan, // RG3 ]); - let reversed = reverse_row_selection(&selection, &metadata).unwrap(); - - // Verify total selected rows remain the same - let original_selected: usize = selection + // Add row selections for the scanned row groups + access_plan.scan_selection( + 0, + RowSelection::from(vec![RowSelector::select(100)]), // RG0: all 100 rows + ); + // RG1 is skipped, no selection needed + access_plan.scan_selection( + 2, + RowSelection::from(vec![ + RowSelector::select(25), // RG2: first 25 rows + RowSelector::skip(75), // RG2: skip last 75 rows + ]), + ); + access_plan.scan_selection( + 3, + RowSelection::from(vec![RowSelector::select(100)]), // RG3: all 100 rows + ); + + let rg_metadata = metadata.row_groups(); + + // Step 1: Create PreparedAccessPlan + let prepared_plan = + PreparedAccessPlan::from_access_plan(access_plan, rg_metadata) + .expect("Failed to create PreparedAccessPlan"); + + // Verify original plan in detail + assert_eq!(prepared_plan.row_group_indexes, vec![0, 2, 3]); + + // Detailed verification of original selection + let orig_selectors: Vec<_> = prepared_plan + .row_selection + .as_ref() + .unwrap() + .iter() + .collect(); + + // Original structure should be: + // RG0: select(100) + // RG2: select(25), skip(75) + // RG3: select(100) + // After merging by RowSelection::from(): select(125), skip(75), select(100) + assert_eq!( + orig_selectors.len(), + 3, + "Original should have 3 selectors after merging" + ); + assert!( + !orig_selectors[0].skip && orig_selectors[0].row_count == 125, + "Original: First selector should be select(125) from RG0(100) + RG2(25)" + ); + assert!( + orig_selectors[1].skip && orig_selectors[1].row_count == 75, + "Original: Second selector should be skip(75) from RG2" + ); + assert!( + !orig_selectors[2].skip && orig_selectors[2].row_count == 100, + "Original: Third selector should be select(100) from RG3" + ); + + let original_selected: usize = orig_selectors .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); - let reversed_selected: usize = reversed + assert_eq!(original_selected, 225); // 100 + 25 + 100 + + // Step 2: Reverse the plan (this is the production code path) + let reversed_plan = prepared_plan + .reverse(&metadata) + .expect("Failed to reverse PreparedAccessPlan"); + + // Verify reversed results + // Row group order should be reversed: [3, 2, 0] + assert_eq!( + reversed_plan.row_group_indexes, + vec![3, 2, 0], + "Row groups should be reversed" + ); + + // Detailed verification of reversed selection + let rev_selectors: Vec<_> = reversed_plan + .row_selection + .as_ref() + .unwrap() + .iter() + .collect(); + + // After reversal, the order becomes: RG3, RG2, RG0 + // - RG3: select(100) + // - RG2: select(25), skip(75) (note: internal order preserved, not reversed) + // - RG0: select(100) + // + // After RowSelection::from() merges adjacent selectors of the same type: + // - RG3's select(100) + RG2's select(25) = select(125) + // - RG2's skip(75) remains as skip(75) + // - RG0's select(100) remains as select(100) + + assert_eq!( + rev_selectors.len(), + 3, + "Reversed should have 3 selectors after merging" + ); + + // First selector: RG3 (100) + RG2 first part (25) merged into select(125) + assert!( + !rev_selectors[0].skip && rev_selectors[0].row_count == 125, + "Reversed: First selector should be select(125) from RG3(100) + RG2(25), got skip={} count={}", + rev_selectors[0].skip, + rev_selectors[0].row_count + ); + + // Second selector: RG2 skip last 75 rows + assert!( + rev_selectors[1].skip && rev_selectors[1].row_count == 75, + "Reversed: Second selector should be skip(75) from RG2, got skip={} count={}", + rev_selectors[1].skip, + rev_selectors[1].row_count + ); + + // Third selector: RG0 select all 100 rows + assert!( + !rev_selectors[2].skip && rev_selectors[2].row_count == 100, + "Reversed: Third selector should be select(100) from RG0, got skip={} count={}", + rev_selectors[2].skip, + rev_selectors[2].row_count + ); + + // Verify row selection is also correctly reversed (total count) + let reversed_selected: usize = rev_selectors .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); - assert_eq!(original_selected, reversed_selected); - assert_eq!(original_selected, 100); // 50 + 50 + assert_eq!( + reversed_selected, 225, + "Total selected rows should remain the same" + ); } #[test] - fn test_reverse_middle_row_group_only() { - // Another test to ensure skipped row groups are handled correctly + fn test_prepared_access_plan_reverse_complex_pattern_detailed() { + // Test: complex pattern with detailed verification let metadata = create_test_metadata(vec![100, 100, 100]); - // Select only middle row group - let selection = RowSelection::from(vec![ - RowSelector::skip(100), // Skip RG0 - RowSelector::select(100), // Select all of RG1 - RowSelector::skip(100), // Skip RG2 - ]); + let mut access_plan = ParquetAccessPlan::new_all(3); + + // Complex pattern: select some, skip some, select some more + access_plan.scan_selection( + 0, + RowSelection::from(vec![ + RowSelector::select(30), + RowSelector::skip(40), + RowSelector::select(30), + ]), + ); + access_plan.scan_selection( + 1, + RowSelection::from(vec![RowSelector::skip(50), RowSelector::select(50)]), + ); + access_plan.scan_selection(2, RowSelection::from(vec![RowSelector::select(100)])); + + let rg_metadata = metadata.row_groups(); + let prepared_plan = + PreparedAccessPlan::from_access_plan(access_plan, rg_metadata) + .expect("Failed to create PreparedAccessPlan"); + + // Verify original selection structure in detail + let orig_selectors: Vec<_> = prepared_plan + .row_selection + .as_ref() + .unwrap() + .iter() + .collect(); - let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + // RG0: select(30), skip(40), select(30) + // RG1: skip(50), select(50) + // RG2: select(100) + // Sequential: sel(30), skip(40), sel(30), skip(50), sel(50), sel(100) + // After merge: sel(30), skip(40), sel(30), skip(50), sel(150) - let original_selected: usize = selection + let original_selected: usize = orig_selectors .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); - let reversed_selected: usize = reversed + assert_eq!(original_selected, 210); // 30 + 30 + 50 + 100 + + let reversed_plan = prepared_plan + .reverse(&metadata) + .expect("Failed to reverse PreparedAccessPlan"); + + // Verify reversed selection structure + let rev_selectors: Vec<_> = reversed_plan + .row_selection + .as_ref() + .unwrap() + .iter() + .collect(); + + // After reversal: RG2, RG1, RG0 + // RG2: select(100) + // RG1: skip(50), select(50) + // RG0: select(30), skip(40), select(30) + // Sequential: sel(100), skip(50), sel(50), sel(30), skip(40), sel(30) + // After merge: sel(100), skip(50), sel(80), skip(40), sel(30) + + let reversed_selected: usize = rev_selectors .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); - assert_eq!(original_selected, reversed_selected); - assert_eq!(original_selected, 100); + assert_eq!( + reversed_selected, 210, + "Total selected rows should remain the same (30 + 30 + 50 + 100)" + ); + + // Verify row group order + assert_eq!(reversed_plan.row_group_indexes, vec![2, 1, 0]); } #[test] - fn test_reverse_alternating_row_groups() { - // Test with more complex skipping pattern + fn test_prepared_access_plan_reverse_alternating_detailed() { + // Test with alternating scan/skip pattern with detailed verification let metadata = create_test_metadata(vec![100, 100, 100, 100]); - // Select first and third row groups, skip second and fourth - let selection = RowSelection::from(vec![ - RowSelector::select(100), // RG0 - RowSelector::skip(100), // RG1 - RowSelector::select(100), // RG2 - RowSelector::skip(100), // RG3 + // Scan RG0 and RG2, skip RG1 and RG3 + let mut access_plan = ParquetAccessPlan::new(vec![ + RowGroupAccess::Scan, // RG0 + RowGroupAccess::Skip, // RG1 + RowGroupAccess::Scan, // RG2 + RowGroupAccess::Skip, // RG3 ]); - let reversed = reverse_row_selection(&selection, &metadata).unwrap(); + access_plan.scan_selection( + 0, + RowSelection::from(vec![RowSelector::select(30), RowSelector::skip(70)]), + ); + access_plan.scan_selection( + 2, + RowSelection::from(vec![RowSelector::skip(20), RowSelector::select(80)]), + ); + + let rg_metadata = metadata.row_groups(); + let prepared_plan = + PreparedAccessPlan::from_access_plan(access_plan, rg_metadata) + .expect("Failed to create PreparedAccessPlan"); + + // Original: [0, 2] + assert_eq!(prepared_plan.row_group_indexes, vec![0, 2]); + + // Verify original selection + let orig_selectors: Vec<_> = prepared_plan + .row_selection + .as_ref() + .unwrap() + .iter() + .collect(); + + // Original: + // RG0: select(30), skip(70) + // RG2: skip(20), select(80) + // Sequential: sel(30), skip(90), sel(80) + // (RG0's skip(70) + RG2's skip(20) = skip(90)) - let original_selected: usize = selection + let original_selected: usize = orig_selectors .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); - let reversed_selected: usize = reversed + assert_eq!(original_selected, 110); // 30 + 80 + + let reversed_plan = prepared_plan + .reverse(&metadata) + .expect("Failed to reverse PreparedAccessPlan"); + + // After reverse: [2, 0] + assert_eq!(reversed_plan.row_group_indexes, vec![2, 0]); + + // Verify reversed selection + let rev_selectors: Vec<_> = reversed_plan + .row_selection + .as_ref() + .unwrap() + .iter() + .collect(); + + // After reversal: RG2, RG0 + // RG2: skip(20), select(80) + // RG0: select(30), skip(70) + // Sequential: skip(20), sel(110), skip(70) + // (RG2's select(80) + RG0's select(30) = select(110)) + + let reversed_selected: usize = rev_selectors .iter() .filter(|s| !s.skip) .map(|s| s.row_count) .sum(); - assert_eq!(original_selected, reversed_selected); - assert_eq!(original_selected, 200); + assert_eq!(reversed_selected, 110); // Should still be 30 + 80 + + // Detailed verification of structure + assert_eq!(rev_selectors.len(), 3, "Reversed should have 3 selectors"); + + assert!( + rev_selectors[0].skip && rev_selectors[0].row_count == 20, + "First selector should be skip(20) from RG2" + ); + + assert!( + !rev_selectors[1].skip && rev_selectors[1].row_count == 110, + "Second selector should be select(110) from RG2(80) + RG0(30)" + ); + + assert!( + rev_selectors[2].skip && rev_selectors[2].row_count == 70, + "Third selector should be skip(70) from RG0" + ); } } From 97a7174a689068b928c0f118bbdb575f710028cd Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 6 Jan 2026 23:02:35 +0800 Subject: [PATCH 05/10] fix --- .../tests/physical_optimizer/pushdown_sort.rs | 426 ++++++++- .../tests/physical_optimizer/test_utils.rs | 210 +++- datafusion/physical-plan/src/filter.rs | 19 - datafusion/physical-plan/src/projection.rs | 48 +- .../dynamic_filter_pushdown_config.slt | 905 ------------------ .../sqllogictest/test_files/sort_pushdown.slt | 886 +++++++++++++++++ 6 files changed, 1536 insertions(+), 958 deletions(-) create mode 100644 datafusion/sqllogictest/test_files/sort_pushdown.slt diff --git a/datafusion/core/tests/physical_optimizer/pushdown_sort.rs b/datafusion/core/tests/physical_optimizer/pushdown_sort.rs index 26e3146b77bc9..e2eb36586431e 100644 --- a/datafusion/core/tests/physical_optimizer/pushdown_sort.rs +++ b/datafusion/core/tests/physical_optimizer/pushdown_sort.rs @@ -24,14 +24,18 @@ //! 4. Early termination is enabled for TopK queries //! 5. Prefix matching works correctly +use datafusion_physical_expr::expressions; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_optimizer::pushdown_sort::PushdownSort; use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_optimizer::pushdown_sort::PushdownSort; +use std::sync::Arc; use crate::physical_optimizer::test_utils::{ - coalesce_batches_exec, coalesce_partitions_exec, parquet_exec, - parquet_exec_with_sort, repartition_exec, schema, sort_exec, sort_exec_with_fetch, - sort_expr, OptimizationTest, + OptimizationTest, coalesce_batches_exec, coalesce_partitions_exec, parquet_exec, + parquet_exec_with_sort, projection_exec, projection_exec_with_alias, + repartition_exec, schema, simple_projection_exec, sort_exec, sort_exec_with_fetch, + sort_expr, sort_expr_named, test_scan_with_ordering, }; #[test] @@ -44,7 +48,7 @@ fn test_sort_pushdown_disabled() { insta::assert_snapshot!( OptimizationTest::new(plan, PushdownSort::new(), false), - @r###" + @r" OptimizationTest: input: - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] @@ -53,7 +57,7 @@ fn test_sort_pushdown_disabled() { Ok: - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - "### + " ); } @@ -240,7 +244,7 @@ fn test_prefix_match_through_transparent_nodes() { OptimizationTest: input: - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] - - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1, maintains_sort_order=true - CoalesceBatchesExec: target_batch_size=1024 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST, b@1 ASC, c@2 DESC NULLS LAST], file_type=parquet output: @@ -270,7 +274,7 @@ fn test_no_prefix_match_wrong_direction() { insta::assert_snapshot!( OptimizationTest::new(plan, PushdownSort::new(), true), - @r###" + @r" OptimizationTest: input: - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] @@ -279,7 +283,7 @@ fn test_no_prefix_match_wrong_direction() { Ok: - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST, b@1 ASC], file_type=parquet - "### + " ); } @@ -301,7 +305,7 @@ fn test_no_prefix_match_longer_than_source() { insta::assert_snapshot!( OptimizationTest::new(plan, PushdownSort::new(), true), - @r###" + @r" OptimizationTest: input: - SortExec: expr=[a@0 ASC, b@1 DESC NULLS LAST], preserve_partitioning=[false] @@ -310,7 +314,7 @@ fn test_no_prefix_match_longer_than_source() { Ok: - SortExec: expr=[a@0 ASC, b@1 DESC NULLS LAST], preserve_partitioning=[false] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST], file_type=parquet - "### + " ); } @@ -365,7 +369,7 @@ fn test_sort_through_repartition() { OptimizationTest: input: - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] - - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1, maintains_sort_order=true - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet output: Ok: @@ -417,7 +421,7 @@ fn test_non_sort_plans_unchanged() { insta::assert_snapshot!( OptimizationTest::new(plan, PushdownSort::new(), true), - @r###" + @r" OptimizationTest: input: - CoalesceBatchesExec: target_batch_size=1024 @@ -426,7 +430,7 @@ fn test_non_sort_plans_unchanged() { Ok: - CoalesceBatchesExec: target_batch_size=1024 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - "### + " ); } @@ -459,7 +463,7 @@ fn test_sort_through_coalesce_partitions() { input: - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] - CoalescePartitionsExec - - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1, maintains_sort_order=true - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet output: Ok: @@ -492,7 +496,7 @@ fn test_complex_plan_with_multiple_operators() { input: - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] - CoalescePartitionsExec - - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1, maintains_sort_order=true - CoalesceBatchesExec: target_batch_size=1024 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet output: @@ -550,7 +554,7 @@ fn test_no_pushdown_for_unordered_source() { insta::assert_snapshot!( OptimizationTest::new(plan, PushdownSort::new(), true), - @r###" + @r" OptimizationTest: input: - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] @@ -559,7 +563,7 @@ fn test_no_pushdown_for_unordered_source() { Ok: - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - "### + " ); } @@ -580,7 +584,7 @@ fn test_no_pushdown_for_non_reverse_sort() { insta::assert_snapshot!( OptimizationTest::new(plan, PushdownSort::new(), true), - @r###" + @r" OptimizationTest: input: - SortExec: expr=[b@1 ASC], preserve_partitioning=[false] @@ -589,7 +593,7 @@ fn test_no_pushdown_for_non_reverse_sort() { Ok: - SortExec: expr=[b@1 ASC], preserve_partitioning=[false] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet - "### + " ); } @@ -619,21 +623,19 @@ fn test_pushdown_through_blocking_node() { // Middle: Aggregate (blocks pushdown from outer sort) // GROUP BY a, COUNT(b) let group_by = PhysicalGroupBy::new_single(vec![( - Arc::new(datafusion_physical_expr::expressions::Column::new("a", 0)) as _, + Arc::new(expressions::Column::new("a", 0)) as _, "a".to_string(), )]); let count_expr = Arc::new( AggregateExprBuilder::new( count_udaf(), - vec![ - Arc::new(datafusion_physical_expr::expressions::Column::new("b", 1)) as _, - ], + vec![Arc::new(expressions::Column::new("b", 1)) as _], ) - .schema(Arc::clone(&schema)) - .alias("COUNT(b)") - .build() - .unwrap(), + .schema(Arc::clone(&schema)) + .alias("COUNT(b)") + .build() + .unwrap(), ); let aggregate = Arc::new( @@ -645,7 +647,7 @@ fn test_pushdown_through_blocking_node() { inner_sort, Arc::clone(&schema), ) - .unwrap(), + .unwrap(), ); // Outer Sort: [a ASC] - this CANNOT push down through aggregate @@ -670,3 +672,369 @@ fn test_pushdown_through_blocking_node() { " ); } + +// ============================================================================ +// PROJECTION TESTS +// ============================================================================ + +#[test] +fn test_sort_pushdown_through_simple_projection() { + // Sort pushes through projection with simple column references + let schema = schema(); + + // Source has [a ASC] ordering + let a = sort_expr("a", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Projection: SELECT a, b (simple column references) + let projection = simple_projection_exec(source, vec![0, 1]); // columns a, b + + // Request [a DESC] - should push through projection to source + let desc_ordering = LexOrdering::new(vec![a.reverse()]).unwrap(); + let plan = sort_exec(desc_ordering, projection); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - ProjectionExec: expr=[a@0 as a, b@1 as b] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - ProjectionExec: expr=[a@0 as a, b@1 as b] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_sort_pushdown_through_projection_with_alias() { + // Sort pushes through projection with column aliases + let schema = schema(); + + // Source has [a ASC] ordering + let a = sort_expr("a", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Projection: SELECT a AS id, b AS value + let projection = projection_exec_with_alias(source, vec![(0, "id"), (1, "value")]); + + // Request [id DESC] - should map to [a DESC] and push down + let id_expr = sort_expr_named("id", 0); + let desc_ordering = LexOrdering::new(vec![id_expr.reverse()]).unwrap(); + let plan = sort_exec(desc_ordering, projection); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[id@0 DESC NULLS LAST], preserve_partitioning=[false] + - ProjectionExec: expr=[a@0 as id, b@1 as value] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[id@0 DESC NULLS LAST], preserve_partitioning=[false] + - ProjectionExec: expr=[a@0 as id, b@1 as value] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_no_sort_pushdown_through_computed_projection() { + use datafusion_expr::Operator; + + // Sort should NOT push through projection with computed columns + let schema = schema(); + + // Source has [a ASC] ordering + let a = sort_expr("a", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Projection: SELECT a+b as sum, c + let projection = projection_exec( + vec![ + ( + Arc::new(expressions::BinaryExpr::new( + Arc::new(expressions::Column::new("a", 0)), + Operator::Plus, + Arc::new(expressions::Column::new("b", 1)), + )) as Arc, + "sum".to_string(), + ), + ( + Arc::new(expressions::Column::new("c", 2)) as Arc, + "c".to_string(), + ), + ], + source, + ) + .unwrap(); + + // Request [sum DESC] - should NOT push down (sum is computed) + let sum_expr = sort_expr_named("sum", 0); + let desc_ordering = LexOrdering::new(vec![sum_expr.reverse()]).unwrap(); + let plan = sort_exec(desc_ordering, projection); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[sum@0 DESC NULLS LAST], preserve_partitioning=[false] + - ProjectionExec: expr=[a@0 + b@1 as sum, c@2 as c] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[sum@0 DESC NULLS LAST], preserve_partitioning=[false] + - ProjectionExec: expr=[a@0 + b@1 as sum, c@2 as c] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + " + ); +} + +#[test] +fn test_sort_pushdown_projection_reordered_columns() { + // Sort pushes through projection that reorders columns + let schema = schema(); + + // Source has [a ASC] ordering + let a = sort_expr("a", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Projection: SELECT c, b, a (columns reordered) + let projection = simple_projection_exec(source, vec![2, 1, 0]); // c, b, a + + // Request [a DESC] where a is now at index 2 in projection output + let a_expr_at_2 = sort_expr_named("a", 2); + let desc_ordering = LexOrdering::new(vec![a_expr_at_2.reverse()]).unwrap(); + let plan = sort_exec(desc_ordering, projection); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@2 DESC NULLS LAST], preserve_partitioning=[false] + - ProjectionExec: expr=[c@2 as c, b@1 as b, a@0 as a] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[a@2 DESC NULLS LAST], preserve_partitioning=[false] + - ProjectionExec: expr=[c@2 as c, b@1 as b, a@0 as a] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_sort_pushdown_projection_with_limit() { + // Sort with LIMIT pushes through simple projection + let schema = schema(); + + // Source has [a ASC] ordering + let a = sort_expr("a", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Projection: SELECT a, b + let projection = simple_projection_exec(source, vec![0, 1]); + + // Request [a DESC] with LIMIT 10 + let desc_ordering = LexOrdering::new(vec![a.reverse()]).unwrap(); + let plan = sort_exec_with_fetch(desc_ordering, Some(10), projection); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: TopK(fetch=10), expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - ProjectionExec: expr=[a@0 as a, b@1 as b] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: TopK(fetch=10), expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - ProjectionExec: expr=[a@0 as a, b@1 as b] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_sort_pushdown_through_projection_and_coalesce() { + // Sort pushes through both projection and coalesce batches + let schema = schema(); + + // Source has [a ASC] ordering + let a = sort_expr("a", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + let coalesce = coalesce_batches_exec(source, 1024); + + // Projection: SELECT a, b + let projection = simple_projection_exec(coalesce, vec![0, 1]); + + // Request [a DESC] + let desc_ordering = LexOrdering::new(vec![a.reverse()]).unwrap(); + let plan = sort_exec(desc_ordering, projection); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - ProjectionExec: expr=[a@0 as a, b@1 as b] + - CoalesceBatchesExec: target_batch_size=1024 + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - ProjectionExec: expr=[a@0 as a, b@1 as b] + - CoalesceBatchesExec: target_batch_size=1024 + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +#[test] +fn test_sort_pushdown_projection_subset_of_columns() { + // Sort pushes through projection that selects subset of columns + let schema = schema(); + + // Source has [a ASC, b ASC] ordering + let a = sort_expr("a", &schema); + let b = sort_expr("b", &schema); + let source_ordering = LexOrdering::new(vec![a.clone(), b.clone()]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Projection: SELECT a (subset of columns) + let projection = simple_projection_exec(source, vec![0]); + + // Request [a DESC] + let desc_ordering = LexOrdering::new(vec![a.reverse()]).unwrap(); + let plan = sort_exec(desc_ordering, projection); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - ProjectionExec: expr=[a@0 as a] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=parquet + output: + Ok: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - ProjectionExec: expr=[a@0 as a] + - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet, reverse_row_groups=true + " + ); +} + +// ============================================================================ +// TESTSCAN DEMONSTRATION TESTS +// ============================================================================ +// These tests use TestScan to demonstrate how sort pushdown works more clearly +// than ParquetExec. TestScan can accept ANY ordering (not just reverse) and +// displays the requested ordering explicitly in the output. + +#[test] +fn test_sort_pushdown_with_test_scan_basic() { + // Demonstrates TestScan showing requested ordering clearly + let schema = schema(); + + // Source has [a ASC] ordering + let a = sort_expr("a", &schema); + let source_ordering = LexOrdering::new(vec![a.clone()]).unwrap(); + let source = test_scan_with_ordering(schema.clone(), source_ordering); + + // Request [a DESC] ordering + let desc_ordering = LexOrdering::new(vec![a.reverse()]).unwrap(); + let plan = sort_exec(desc_ordering, source); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - TestScan: output_ordering=[a@0 ASC] + output: + Ok: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - TestScan: output_ordering=[a@0 ASC], requested_ordering=[a@0 DESC NULLS LAST] + " + ); +} + +#[test] +fn test_sort_pushdown_with_test_scan_multi_column() { + // Demonstrates TestScan with multi-column ordering + let schema = schema(); + + // Source has [a ASC, b DESC] ordering + let a = sort_expr("a", &schema); + let b = sort_expr("b", &schema); + let source_ordering = LexOrdering::new(vec![a.clone(), b.clone().reverse()]).unwrap(); + let source = test_scan_with_ordering(schema.clone(), source_ordering); + + // Request [a DESC, b ASC] ordering (reverse of source) + let reverse_ordering = LexOrdering::new(vec![a.reverse(), b]).unwrap(); + let plan = sort_exec(reverse_ordering, source); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 DESC NULLS LAST, b@1 ASC], preserve_partitioning=[false] + - TestScan: output_ordering=[a@0 ASC, b@1 DESC NULLS LAST] + output: + Ok: + - SortExec: expr=[a@0 DESC NULLS LAST, b@1 ASC], preserve_partitioning=[false] + - TestScan: output_ordering=[a@0 ASC, b@1 DESC NULLS LAST], requested_ordering=[a@0 DESC NULLS LAST, b@1 ASC] + " + ); +} + +#[test] +fn test_sort_pushdown_with_test_scan_arbitrary_ordering() { + // Demonstrates that TestScan can accept ANY ordering (not just reverse) + // This is different from ParquetExec which only supports reverse scans + let schema = schema(); + + // Source has [a ASC, b ASC] ordering + let a = sort_expr("a", &schema); + let b = sort_expr("b", &schema); + let source_ordering = LexOrdering::new(vec![a.clone(), b.clone()]).unwrap(); + let source = test_scan_with_ordering(schema.clone(), source_ordering); + + // Request [a ASC, b DESC] - NOT a simple reverse, but TestScan accepts it + let mixed_ordering = LexOrdering::new(vec![a, b.reverse()]).unwrap(); + let plan = sort_exec(mixed_ordering, source); + + insta::assert_snapshot!( + OptimizationTest::new(plan, PushdownSort::new(), true), + @r" + OptimizationTest: + input: + - SortExec: expr=[a@0 ASC, b@1 DESC NULLS LAST], preserve_partitioning=[false] + - TestScan: output_ordering=[a@0 ASC, b@1 ASC] + output: + Ok: + - SortExec: expr=[a@0 ASC, b@1 DESC NULLS LAST], preserve_partitioning=[false] + - TestScan: output_ordering=[a@0 ASC, b@1 ASC], requested_ordering=[a@0 ASC, b@1 DESC NULLS LAST] + " + ); +} diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index a589caa817943..0f02e6b4b9c9c 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -42,6 +42,7 @@ use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; use datafusion_functions_aggregate::count::count_udaf; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; +use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_expr::expressions::{self, col}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{ @@ -66,10 +67,8 @@ use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::windows::{create_window_expr, BoundedWindowAggExec}; -use datafusion_physical_plan::{ - displayable, DisplayAs, DisplayFormatType, ExecutionPlan, InputOrderMode, - Partitioning, PlanProperties, -}; +use datafusion_physical_plan::{displayable, DisplayAs, DisplayFormatType, ExecutionPlan, InputOrderMode, Partitioning, PlanProperties, SortOrderPushdownResult}; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; /// Create a non sorted parquet exec pub fn parquet_exec(schema: SchemaRef) -> Arc { @@ -778,3 +777,206 @@ pub fn format_execution_plan(plan: &Arc) -> Vec { fn format_lines(s: &str) -> Vec { s.trim().split('\n').map(|s| s.to_string()).collect() } + +/// Create a simple ProjectionExec with column indices (simplified version) +pub fn simple_projection_exec( + input: Arc, + columns: Vec, +) -> Arc { + let schema = input.schema(); + let exprs: Vec<(Arc, String)> = columns + .iter() + .map(|&i| { + let field = schema.field(i); + ( + Arc::new(expressions::Column::new(field.name(), i)) + as Arc, + field.name().to_string(), + ) + }) + .collect(); + + projection_exec(exprs, input).unwrap() +} + +/// Create a ProjectionExec with column aliases +pub fn projection_exec_with_alias( + input: Arc, + columns: Vec<(usize, &str)>, +) -> Arc { + let schema = input.schema(); + let exprs: Vec<(Arc, String)> = columns + .iter() + .map(|&(i, alias)| { + ( + Arc::new(expressions::Column::new(schema.field(i).name(), i)) + as Arc, + alias.to_string(), + ) + }) + .collect(); + + projection_exec(exprs, input).unwrap() +} + +/// Create a sort expression with custom name and index +pub fn sort_expr_named(name: &str, index: usize) -> PhysicalSortExpr { + PhysicalSortExpr { + expr: Arc::new(expressions::Column::new(name, index)), + options: SortOptions::default(), + } +} + +/// A test data source that can display any requested ordering +/// This is useful for testing sort pushdown behavior +#[derive(Debug, Clone)] +pub struct TestScan { + schema: SchemaRef, + output_ordering: Vec, + plan_properties: PlanProperties, + // Store the requested ordering for display + requested_ordering: Option, +} + +impl TestScan { + /// Create a new TestScan with the given schema and output ordering + pub fn new(schema: SchemaRef, output_ordering: Vec) -> Self { + let eq_properties = if !output_ordering.is_empty() { + // Convert Vec to the format expected by new_with_orderings + // We need to extract the inner Vec from each LexOrdering + let orderings: Vec> = output_ordering + .iter() + .map(|lex_ordering| { + // LexOrdering implements IntoIterator, so we can collect it + lex_ordering.iter().cloned().collect() + }) + .collect(); + + EquivalenceProperties::new_with_orderings(Arc::clone(&schema), orderings) + } else { + EquivalenceProperties::new(Arc::clone(&schema)) + }; + + let plan_properties = PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + EmissionType::Incremental, + Boundedness::Bounded, + ); + + Self { + schema, + output_ordering, + plan_properties, + requested_ordering: None, + } + } + + /// Create a TestScan with a single output ordering + pub fn with_ordering(schema: SchemaRef, ordering: LexOrdering) -> Self { + Self::new(schema, vec![ordering]) + } +} + +impl DisplayAs for TestScan { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "TestScan")?; + if !self.output_ordering.is_empty() { + write!(f, ": output_ordering=[")?; + // Format the ordering in a readable way + for (i, sort_expr) in self.output_ordering[0].iter().enumerate() { + if i > 0 { + write!(f, ", ")?; + } + write!(f, "{sort_expr}")?; + } + write!(f, "]")?; + } + // This is the key part - show what ordering was requested + if let Some(ref req) = self.requested_ordering { + write!(f, ", requested_ordering=[")?; + for (i, sort_expr) in req.iter().enumerate() { + if i > 0 { + write!(f, ", ")?; + } + write!(f, "{sort_expr}")?; + } + write!(f, "]")?; + } + Ok(()) + } + DisplayFormatType::TreeRender => { + write!(f, "TestScan") + } + } + } +} + +impl ExecutionPlan for TestScan { + fn name(&self) -> &str { + "TestScan" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.plan_properties + } + + fn children(&self) -> Vec<&Arc> { + vec![] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result> { + if children.is_empty() { + Ok(self) + } else { + internal_err!("TestScan should have no children") + } + } + + fn execute( + &self, + _partition: usize, + _context: Arc, + ) -> Result { + internal_err!("TestScan is for testing optimizer only, not for execution") + } + + fn partition_statistics(&self, _partition: Option) -> Result { + Ok(Statistics::new_unknown(&self.schema)) + } + + // This is the key method - implement sort pushdown + fn try_pushdown_sort( + &self, + order: &[PhysicalSortExpr], + ) -> Result>> { + // For testing purposes, accept ANY ordering request + // and create a new TestScan that shows what was requested + let requested_ordering = LexOrdering::new(order.to_vec()); + + let mut new_scan = self.clone(); + new_scan.requested_ordering = requested_ordering; + + // Always return Inexact to keep the Sort node (like Phase 1 behavior) + Ok(SortOrderPushdownResult::Inexact { + inner: Arc::new(new_scan), + }) + } +} + +/// Helper function to create a TestScan with ordering +pub fn test_scan_with_ordering( + schema: SchemaRef, + ordering: LexOrdering, +) -> Arc { + Arc::new(TestScan::with_ordering(schema, ordering)) +} diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index e5ffe6934e9f5..48e8efd336abd 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -571,25 +571,6 @@ impl ExecutionPlan for FilterExec { updated_node, }) } - fn try_pushdown_sort( - &self, - order: &[PhysicalSortExpr], - ) -> Result>> { - let child = self.input(); - match child.try_pushdown_sort(order)? { - SortOrderPushdownResult::Exact { inner } => { - let new_exec = Arc::new(self.clone()).with_new_children(vec![inner])?; - Ok(SortOrderPushdownResult::Exact { inner: new_exec }) - } - SortOrderPushdownResult::Inexact { inner } => { - let new_exec = Arc::new(self.clone()).with_new_children(vec![inner])?; - Ok(SortOrderPushdownResult::Inexact { inner: new_exec }) - } - SortOrderPushdownResult::Unsupported => { - Ok(SortOrderPushdownResult::Unsupported) - } - } - } } impl EmbeddedProjection for FilterExec { diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 1979939587c27..4e54bdb81b10a 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -354,7 +354,53 @@ impl ExecutionPlan for ProjectionExec { order: &[PhysicalSortExpr], ) -> Result>> { let child = self.input(); - match child.try_pushdown_sort(order)? { + let mut child_order = Vec::new(); + + // Check and transform sort expressions + for sort_expr in order { + // Recursively transform the expression + let mut can_pushdown = true; + let transformed = Arc::clone(&sort_expr.expr).transform(|expr| { + if let Some(col) = expr.as_any().downcast_ref::() { + // Check if column index is valid. + // This should always be true but fail gracefully if it's not. + if col.index() >= self.expr().len() { + can_pushdown = false; + return Ok(Transformed::no(expr)); + } + + let proj_expr = &self.expr()[col.index()]; + + // Check if projection expression is a simple column + // We cannot push down order by clauses that depend on + // projected computations as they would have nothing to reference. + if let Some(child_col) = + proj_expr.expr.as_any().downcast_ref::() + { + // Replace with the child column + Ok(Transformed::yes(Arc::new(child_col.clone()) as _)) + } else { + // Projection involves computation, cannot push down + can_pushdown = false; + Ok(Transformed::no(expr)) + } + } else { + Ok(Transformed::no(expr)) + } + })?; + + if !can_pushdown { + return Ok(SortOrderPushdownResult::Unsupported); + } + + child_order.push(PhysicalSortExpr { + expr: transformed.data, + options: sort_expr.options, + }); + } + + // Recursively push down to child node + match child.try_pushdown_sort(&child_order)? { SortOrderPushdownResult::Exact { inner } => { let new_exec = Arc::new(self.clone()).with_new_children(vec![inner])?; Ok(SortOrderPushdownResult::Exact { inner: new_exec }) diff --git a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt index 678697f3c12fa..e5cd6d88b08f4 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -337,908 +337,3 @@ SET datafusion.optimizer.enable_join_dynamic_filter_pushdown = true; statement ok SET datafusion.optimizer.enable_dynamic_filter_pushdown = true; - -# Test 6: Sort Pushdown for ordered Parquet files - -# Create a sorted dataset -statement ok -CREATE TABLE sorted_data(id INT, value INT, name VARCHAR) AS VALUES -(1, 100, 'a'), -(2, 200, 'b'), -(3, 300, 'c'), -(4, 400, 'd'), -(5, 500, 'e'), -(6, 600, 'f'), -(7, 700, 'g'), -(8, 800, 'h'), -(9, 900, 'i'), -(10, 1000, 'j'); - -# Copy to parquet with sorting -query I -COPY (SELECT * FROM sorted_data ORDER BY id ASC) -TO 'test_files/scratch/dynamic_filter_pushdown_config/sorted_data.parquet'; ----- -10 - -statement ok -CREATE EXTERNAL TABLE sorted_parquet(id INT, value INT, name VARCHAR) -STORED AS PARQUET -LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/sorted_data.parquet' -WITH ORDER (id ASC); - -# Test 6.1: Sort pushdown with DESC (opposite of ASC) -# Should show reverse_row_groups=true -query TT -EXPLAIN SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3; ----- -logical_plan -01)Sort: sorted_parquet.id DESC NULLS FIRST, fetch=3 -02)--TableScan: sorted_parquet projection=[id, value, name] -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/dynamic_filter_pushdown_config/sorted_data.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true - -# Test 6.2: Verify results are correct -query IIT -SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3; ----- -10 1000 j -9 900 i -8 800 h - -# Test 6.3: Should NOT apply for ASC (same direction) -query TT -EXPLAIN SELECT * FROM sorted_parquet ORDER BY id ASC LIMIT 3; ----- -logical_plan -01)Sort: sorted_parquet.id ASC NULLS LAST, fetch=3 -02)--TableScan: sorted_parquet projection=[id, value, name] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/sorted_data.parquet]]}, projection=[id, value, name], limit=3, output_ordering=[id@0 ASC NULLS LAST], file_type=parquet - -# Test 6.4: Disable sort pushdown -statement ok -SET datafusion.optimizer.enable_sort_pushdown = false; - -query TT -EXPLAIN SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3; ----- -logical_plan -01)Sort: sorted_parquet.id DESC NULLS FIRST, fetch=3 -02)--TableScan: sorted_parquet projection=[id, value, name] -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/dynamic_filter_pushdown_config/sorted_data.parquet]]}, projection=[id, value, name], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] - -# Re-enable -statement ok -SET datafusion.optimizer.enable_sort_pushdown = true; - -# Test 6.5: With OFFSET -query TT -EXPLAIN SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3 OFFSET 2; ----- -logical_plan -01)Limit: skip=2, fetch=3 -02)--Sort: sorted_parquet.id DESC NULLS FIRST, fetch=5 -03)----TableScan: sorted_parquet projection=[id, value, name] -physical_plan -01)GlobalLimitExec: skip=2, fetch=3 -02)--SortExec: TopK(fetch=5), expr=[id@0 DESC], preserve_partitioning=[false] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/sorted_data.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true - -query IIT -SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3 OFFSET 2; ----- -8 800 h -7 700 g -6 600 f - -# Test 6.6: Reverse scan with row selection (page index pruning) -# This tests that when reverse_row_groups=true, the RowSelection is also properly reversed - -# Create a dataset with multiple row groups and enable page index -statement ok -CREATE TABLE multi_rg_data(id INT, category VARCHAR, value INT) AS VALUES -(1, 'alpha', 10), -(2, 'alpha', 20), -(3, 'beta', 30), -(4, 'beta', 40), -(5, 'gamma', 50), -(6, 'gamma', 60), -(7, 'delta', 70), -(8, 'delta', 80); - -# Write with small row groups (2 rows each = 4 row groups) -statement ok -SET datafusion.execution.parquet.max_row_group_size = 2; - -query I -COPY (SELECT * FROM multi_rg_data ORDER BY id ASC) -TO 'test_files/scratch/dynamic_filter_pushdown_config/multi_rg_sorted.parquet'; ----- -8 - -# Reset row group size -statement ok -SET datafusion.execution.parquet.max_row_group_size = 1048576; - -statement ok -CREATE EXTERNAL TABLE multi_rg_sorted(id INT, category VARCHAR, value INT) -STORED AS PARQUET -LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/multi_rg_sorted.parquet' -WITH ORDER (id ASC); - -# Enable page index for better pruning -statement ok -SET datafusion.execution.parquet.enable_page_index = true; - -statement ok -SET datafusion.execution.parquet.pushdown_filters = true; - -# Test with reverse scan and filter that prunes some row groups -# This will create a RowSelection with partial row group scans -query TT -EXPLAIN SELECT * FROM multi_rg_sorted -WHERE category IN ('alpha', 'gamma') -ORDER BY id DESC LIMIT 5; ----- -logical_plan -01)Sort: multi_rg_sorted.id DESC NULLS FIRST, fetch=5 -02)--Filter: multi_rg_sorted.category = Utf8View("alpha") OR multi_rg_sorted.category = Utf8View("gamma") -03)----TableScan: multi_rg_sorted projection=[id, category, value], partial_filters=[multi_rg_sorted.category = Utf8View("alpha") OR multi_rg_sorted.category = Utf8View("gamma")] -physical_plan -01)SortExec: TopK(fetch=5), expr=[id@0 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/multi_rg_sorted.parquet]]}, projection=[id, category, value], file_type=parquet, predicate=(category@1 = alpha OR category@1 = gamma) AND DynamicFilter [ empty ], reverse_row_groups=true, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1 OR category_null_count@2 != row_count@3 AND category_min@0 <= gamma AND gamma <= category_max@1, required_guarantees=[category in (alpha, gamma)] - -# Verify the results are correct despite reverse scanning with row selection -# Expected: gamma values (6, 5) then alpha values (2, 1), in DESC order by id -query ITI -SELECT * FROM multi_rg_sorted -WHERE category IN ('alpha', 'gamma') -ORDER BY id DESC LIMIT 5; ----- -6 gamma 60 -5 gamma 50 -2 alpha 20 -1 alpha 10 - -# Test with more complex selection pattern -query ITI -SELECT * FROM multi_rg_sorted -WHERE category IN ('beta', 'delta') -ORDER BY id DESC; ----- -8 delta 80 -7 delta 70 -4 beta 40 -3 beta 30 - -# Test forward scan for comparison (should give same logical results in ASC order) -query ITI -SELECT * FROM multi_rg_sorted -WHERE category IN ('alpha', 'gamma') -ORDER BY id ASC; ----- -1 alpha 10 -2 alpha 20 -5 gamma 50 -6 gamma 60 - -# Disable reverse scan and verify it still works -statement ok -SET datafusion.optimizer.enable_sort_pushdown = false; - -query ITI -SELECT * FROM multi_rg_sorted -WHERE category IN ('alpha', 'gamma') -ORDER BY id DESC LIMIT 5; ----- -6 gamma 60 -5 gamma 50 -2 alpha 20 -1 alpha 10 - -# Re-enable -statement ok -SET datafusion.optimizer.enable_sort_pushdown = true; - -# Test 6.7: Sort pushdown with more than one partition -# Create multiple parquet files to trigger it - -# Split data into multiple files -statement ok -CREATE TABLE sorted_data_part1(id INT, value INT, name VARCHAR) AS VALUES -(1, 100, 'a'), -(2, 200, 'b'), -(3, 300, 'c'); - -statement ok -CREATE TABLE sorted_data_part2(id INT, value INT, name VARCHAR) AS VALUES -(4, 400, 'd'), -(5, 500, 'e'), -(6, 600, 'f'); - -statement ok -CREATE TABLE sorted_data_part3(id INT, value INT, name VARCHAR) AS VALUES -(7, 700, 'g'), -(8, 800, 'h'), -(9, 900, 'i'), -(10, 1000, 'j'); - -# Create directory for multi-file parquet -query I -COPY (SELECT * FROM sorted_data_part1 ORDER BY id ASC) -TO 'test_files/scratch/dynamic_filter_pushdown_config/sorted_multi/part1.parquet'; ----- -3 - -query I -COPY (SELECT * FROM sorted_data_part2 ORDER BY id ASC) -TO 'test_files/scratch/dynamic_filter_pushdown_config/sorted_multi/part2.parquet'; ----- -3 - -query I -COPY (SELECT * FROM sorted_data_part3 ORDER BY id ASC) -TO 'test_files/scratch/dynamic_filter_pushdown_config/sorted_multi/part3.parquet'; ----- -4 - -# Create external table pointing to directory with multiple files -statement ok -CREATE EXTERNAL TABLE sorted_parquet_multi(id INT, value INT, name VARCHAR) -STORED AS PARQUET -LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/sorted_multi/' -WITH ORDER (id ASC); - -# Enable multiple partitions -statement ok -SET datafusion.execution.target_partitions = 4; - -# Now we should see RepartitionExec because we have 3 input partitions (3 files) -query TT -EXPLAIN SELECT * FROM sorted_parquet_multi ORDER BY id DESC LIMIT 3; ----- -logical_plan -01)Sort: sorted_parquet_multi.id DESC NULLS FIRST, fetch=3 -02)--TableScan: sorted_parquet_multi projection=[id, value, name] -physical_plan -01)SortPreservingMergeExec: [id@0 DESC], fetch=3 -02)--SortExec: TopK(fetch=3), expr=[id@0 DESC], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/sorted_multi/part1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/sorted_multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/sorted_multi/part3.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true - -# Verify correctness with repartitioning and multiple files -query IIT -SELECT * FROM sorted_parquet_multi ORDER BY id DESC LIMIT 3; ----- -10 1000 j -9 900 i -8 800 h - -# Test ASC order (should not trigger reverse scan) -query IIT -SELECT * FROM sorted_parquet_multi ORDER BY id ASC LIMIT 3; ----- -1 100 a -2 200 b -3 300 c - -# Cleanup -statement ok -DROP TABLE sorted_data_part1; - -statement ok -DROP TABLE sorted_data_part2; - -statement ok -DROP TABLE sorted_data_part3; - -statement ok -DROP TABLE sorted_parquet_multi; - -# Reset to default -statement ok -SET datafusion.execution.target_partitions = 4; - -# Cleanup -statement ok -DROP TABLE multi_rg_data; - -statement ok -DROP TABLE multi_rg_sorted; - -statement ok -SET datafusion.execution.parquet.enable_page_index = false; - -statement ok -SET datafusion.execution.parquet.pushdown_filters = false; - -# Cleanup -statement ok -DROP TABLE sorted_data; - -statement ok -DROP TABLE sorted_parquet; - -statement ok -SET datafusion.optimizer.enable_sort_pushdown = true; - -# Test 7: Sort pushdown with constant column filtering -# This tests the case where a leading sort column becomes constant through WHERE filtering - -# Create a multi-column sorted dataset (like time-series data) -statement ok -CREATE TABLE timeseries_data(timeframe VARCHAR, period_end INT, value DOUBLE) AS VALUES -('daily', 1, 100.0), -('daily', 2, 150.0), -('daily', 3, 200.0), -('weekly', 1, 500.0), -('weekly', 2, 600.0), -('weekly', 3, 700.0), -('monthly', 1, 1000.0), -('monthly', 2, 1100.0), -('monthly', 3, 1200.0), -('quarterly', 1, 5000.0), -('quarterly', 2, 5500.0), -('quarterly', 3, 6000.0); - -# Copy to parquet with multi-column sorting (timeframe ASC, period_end ASC) -query I -COPY (SELECT * FROM timeseries_data ORDER BY timeframe ASC, period_end ASC) -TO 'test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet'; ----- -12 - -statement ok -CREATE EXTERNAL TABLE timeseries_parquet(timeframe VARCHAR, period_end INT, value DOUBLE) -STORED AS PARQUET -LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet' -WITH ORDER (timeframe ASC, period_end ASC); - -# Test 7.1: Query with constant prefix filter and DESC on remaining column -# WHERE timeframe='quarterly' makes the first sort column constant -# ORDER BY period_end DESC should trigger reverse scan because: -# File ordering: [timeframe ASC, period_end ASC] -# After filtering timeframe='quarterly': effectively [period_end ASC] -# Request: [period_end DESC] -> exact reverse! -query TT -EXPLAIN SELECT * FROM timeseries_parquet -WHERE timeframe = 'quarterly' -ORDER BY period_end DESC -LIMIT 2; ----- -logical_plan -01)Sort: timeseries_parquet.period_end DESC NULLS FIRST, fetch=2 -02)--Filter: timeseries_parquet.timeframe = Utf8View("quarterly") -03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] -physical_plan -01)SortPreservingMergeExec: [period_end@1 DESC], fetch=2 -02)--SortExec: TopK(fetch=2), expr=[period_end@1 DESC], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: timeframe@0 = quarterly -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=timeframe@0 = quarterly AND DynamicFilter [ empty ], reverse_row_groups=true, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] - -# Test 7.2: Verify the results are correct -query TIR -SELECT * FROM timeseries_parquet -WHERE timeframe = 'quarterly' -ORDER BY period_end DESC -LIMIT 2; ----- -quarterly 3 6000 -quarterly 2 5500 - -# Test 7.3: Same filter but ASC order (should not trigger reverse scan, ordering already satisfied) -query TT -EXPLAIN SELECT * FROM timeseries_parquet -WHERE timeframe = 'quarterly' -ORDER BY period_end ASC -LIMIT 2; ----- -logical_plan -01)Sort: timeseries_parquet.period_end ASC NULLS LAST, fetch=2 -02)--Filter: timeseries_parquet.timeframe = Utf8View("quarterly") -03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] -physical_plan -01)SortPreservingMergeExec: [period_end@1 ASC NULLS LAST], fetch=2 -02)--CoalesceBatchesExec: target_batch_size=8192, fetch=2 -03)----FilterExec: timeframe@0 = quarterly -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=timeframe@0 = quarterly, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] - -# Test 7.4: Verify ASC results -query TIR -SELECT * FROM timeseries_parquet -WHERE timeframe = 'quarterly' -ORDER BY period_end ASC -LIMIT 2; ----- -quarterly 1 5000 -quarterly 2 5500 - -# Test 7.5: Test with different constant value -query TIR -SELECT * FROM timeseries_parquet -WHERE timeframe = 'weekly' -ORDER BY period_end DESC; ----- -weekly 3 700 -weekly 2 600 -weekly 1 500 - -# Test 7.6: Test without constant filter (no reverse scan because need both columns) -# Request: [timeframe ASC, period_end DESC] -# File has: [timeframe ASC, period_end ASC] -# These are NOT reverse of each other - only second column is reversed -query TT -EXPLAIN SELECT * FROM timeseries_parquet -ORDER BY timeframe ASC, period_end DESC -LIMIT 3; ----- -logical_plan -01)Sort: timeseries_parquet.timeframe ASC NULLS LAST, timeseries_parquet.period_end DESC NULLS FIRST, fetch=3 -02)--TableScan: timeseries_parquet projection=[timeframe, period_end, value] -physical_plan -01)SortExec: TopK(fetch=3), expr=[timeframe@0 ASC NULLS LAST, period_end@1 DESC], preserve_partitioning=[false], sort_prefix=[timeframe@0 ASC NULLS LAST] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] - -# Test 7.7: Disable sort pushdown and verify filter still works -statement ok -SET datafusion.optimizer.enable_sort_pushdown = false; - -query TT -EXPLAIN SELECT * FROM timeseries_parquet -WHERE timeframe = 'quarterly' -ORDER BY period_end DESC -LIMIT 2; ----- -logical_plan -01)Sort: timeseries_parquet.period_end DESC NULLS FIRST, fetch=2 -02)--Filter: timeseries_parquet.timeframe = Utf8View("quarterly") -03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] -physical_plan -01)SortPreservingMergeExec: [period_end@1 DESC], fetch=2 -02)--SortExec: TopK(fetch=2), expr=[period_end@1 DESC], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: timeframe@0 = quarterly -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=timeframe@0 = quarterly AND DynamicFilter [ empty ], pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] - -# Results should still be correct -query TIR -SELECT * FROM timeseries_parquet -WHERE timeframe = 'quarterly' -ORDER BY period_end DESC -LIMIT 2; ----- -quarterly 3 6000 -quarterly 2 5500 - -# Re-enable -statement ok -SET datafusion.optimizer.enable_sort_pushdown = true; - -# Test 7.8: Test with IN clause (multiple constant values) -# Note: IN clause with multiple values means timeframe is NOT constant -# (could be 'daily' or 'weekly'), so the first sort column cannot be eliminated. -# Without a constant first column, we cannot reverse scan based on just period_end DESC. -# The physical plan should NOT show reverse_row_groups=true -query TT -EXPLAIN SELECT * FROM timeseries_parquet -WHERE timeframe IN ('daily', 'weekly') -ORDER BY period_end DESC -LIMIT 3; ----- -logical_plan -01)Sort: timeseries_parquet.period_end DESC NULLS FIRST, fetch=3 -02)--Filter: timeseries_parquet.timeframe = Utf8View("daily") OR timeseries_parquet.timeframe = Utf8View("weekly") -03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("daily") OR timeseries_parquet.timeframe = Utf8View("weekly")] -physical_plan -01)SortPreservingMergeExec: [period_end@1 DESC], fetch=3 -02)--SortExec: TopK(fetch=3), expr=[period_end@1 DESC], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: timeframe@0 = daily OR timeframe@0 = weekly -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=(timeframe@0 = daily OR timeframe@0 = weekly) AND DynamicFilter [ empty ], pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= daily AND daily <= timeframe_max@1 OR timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= weekly AND weekly <= timeframe_max@1, required_guarantees=[timeframe in (daily, weekly)] - -# Test 7.9: Complex case - literal constant in sort expression itself -# The literal 'constant' is ignored in sort analysis -# After stripping: ORDER BY period_end DESC -# With WHERE timeframe='monthly' making first column constant -# File: [period_end ASC] (after constant column removal) -# Request: [period_end DESC] -> exact reverse, triggers reverse scan -query TT -EXPLAIN SELECT * FROM timeseries_parquet -WHERE timeframe = 'monthly' -ORDER BY 'constant', period_end DESC -LIMIT 2; ----- -logical_plan -01)Sort: Utf8("constant") ASC NULLS LAST, timeseries_parquet.period_end DESC NULLS FIRST, fetch=2 -02)--Filter: timeseries_parquet.timeframe = Utf8View("monthly") -03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("monthly")] -physical_plan -01)SortPreservingMergeExec: [constant ASC NULLS LAST, period_end@1 DESC], fetch=2 -02)--SortExec: TopK(fetch=2), expr=[period_end@1 DESC], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: timeframe@0 = monthly -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=timeframe@0 = monthly AND DynamicFilter [ empty ], reverse_row_groups=true, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= monthly AND monthly <= timeframe_max@1, required_guarantees=[timeframe in (monthly)] - -# Verify results -query TIR -SELECT * FROM timeseries_parquet -WHERE timeframe = 'monthly' -ORDER BY period_end DESC -LIMIT 2; ----- -monthly 3 1200 -monthly 2 1100 - -# Test 7.10: Filter on non-leading sort column -# File order: [timeframe ASC, period_end ASC] -# Filter: period_end = 2 (makes second column constant) -# Request: [timeframe DESC] -# After constant column removal: File has [timeframe ASC], Request wants [timeframe DESC] -# This is exact reverse -> triggers reverse scan -query TT -EXPLAIN SELECT * FROM timeseries_parquet -WHERE period_end = 2 -ORDER BY timeframe DESC; ----- -logical_plan -01)Sort: timeseries_parquet.timeframe DESC NULLS FIRST -02)--Filter: timeseries_parquet.period_end = Int32(2) -03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.period_end = Int32(2)] -physical_plan -01)SortPreservingMergeExec: [timeframe@0 DESC] -02)--SortExec: expr=[timeframe@0 DESC], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: period_end@1 = 2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=period_end@1 = 2, reverse_row_groups=true, pruning_predicate=period_end_null_count@2 != row_count@3 AND period_end_min@0 <= 2 AND 2 <= period_end_max@1, required_guarantees=[period_end in (2)] - -# Cleanup -statement ok -DROP TABLE timeseries_data; - -statement ok -DROP TABLE timeseries_parquet; - -# Reset to default -statement ok -SET datafusion.optimizer.enable_sort_pushdown = true; - - -# Test 8: Sort pushdown with monotonic functions -# This tests that reverse scan works when sort expressions involve monotonic functions - -# Create test data with timestamp column -statement ok -CREATE TABLE timestamp_data(id INT, ts TIMESTAMP, volume BIGINT, price DOUBLE) AS VALUES -(1, TIMESTAMP '2024-01-15 10:00:00', 1000, 100.0), -(2, TIMESTAMP '2024-01-20 11:00:00', 1500, 105.0), -(3, TIMESTAMP '2024-01-25 12:00:00', 2000, 110.0), -(4, TIMESTAMP '2024-02-05 09:00:00', 1200, 108.0), -(5, TIMESTAMP '2024-02-15 14:00:00', 1800, 112.0), -(6, TIMESTAMP '2024-02-25 15:00:00', 2200, 115.0), -(7, TIMESTAMP '2024-03-10 09:00:00', 1300, 113.0), -(8, TIMESTAMP '2024-03-18 14:00:00', 1900, 118.0), -(9, TIMESTAMP '2024-03-28 15:00:00', 2300, 120.0); - -# Copy to parquet with sorting by timestamp ASC -query I -COPY (SELECT * FROM timestamp_data ORDER BY ts ASC) -TO 'test_files/scratch/dynamic_filter_pushdown_config/timestamp_sorted.parquet'; ----- -9 - -# Test 8.1: Simple monotonic function - date_trunc -# Create external table with file ordering that conceptually includes date_trunc -# File is actually sorted by [ts ASC], but conceptually [date_trunc('month', ts) ASC, ts ASC] -statement ok -CREATE EXTERNAL TABLE timestamp_parquet(id INT, ts TIMESTAMP, volume BIGINT, price DOUBLE) -STORED AS PARQUET -LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/timestamp_sorted.parquet' -WITH ORDER (ts ASC); - -# Query with ORDER BY ts DESC -# File ordering: [ts ASC] -# Request: [ts DESC] -# This should trigger reverse_row_groups=true -query TT -EXPLAIN SELECT * FROM timestamp_parquet -ORDER BY ts DESC -LIMIT 3; ----- -logical_plan -01)Sort: timestamp_parquet.ts DESC NULLS FIRST, fetch=3 -02)--TableScan: timestamp_parquet projection=[id, ts, volume, price] -physical_plan -01)SortExec: TopK(fetch=3), expr=[ts@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timestamp_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true - -# Verify results -query IPIR -SELECT * FROM timestamp_parquet -ORDER BY ts DESC -LIMIT 3; ----- -9 2024-03-28T15:00:00 2300 120 -8 2024-03-18T14:00:00 1900 118 -7 2024-03-10T09:00:00 1300 113 - -# Test 8.2: Monotonic function in ORDER BY - date_trunc DESC -# File ordering: [ts ASC] -# Request: [date_trunc('day', ts) DESC] -# Since date_trunc is monotonic with ts, reversed file ordering [ts DESC] satisfies [date_trunc DESC] -query TT -EXPLAIN SELECT * FROM timestamp_parquet -ORDER BY date_trunc('day', ts) DESC -LIMIT 3; ----- -logical_plan -01)Sort: date_trunc(Utf8("day"), timestamp_parquet.ts) DESC NULLS FIRST, fetch=3 -02)--TableScan: timestamp_parquet projection=[id, ts, volume, price] -physical_plan -01)SortExec: TopK(fetch=3), expr=[date_trunc(day, ts@1) DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/timestamp_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true - -# Verify results (descending day) -query IPIR -SELECT * FROM timestamp_parquet -ORDER BY date_trunc('day', ts) DESC -LIMIT 3; ----- -9 2024-03-28T15:00:00 2300 120 -8 2024-03-18T14:00:00 1900 118 -7 2024-03-10T09:00:00 1300 113 - -# Test 8.3: Multi-column scenario with explicit monotonic function in file ordering -# Create a table where we explicitly declare the ordering includes the monotonic function -# This simulates files that are partitioned/sorted by [date_trunc('month', ts) ASC, ts ASC] - -# Create a new parquet file sorted by [ts ASC] (which implies date_trunc ordering) -statement ok -CREATE TABLE multi_month_data(id INT, ts TIMESTAMP, volume BIGINT, price DOUBLE) AS VALUES --- January 2024 -(1, TIMESTAMP '2024-01-05 09:30:00', 1000, 100.0), -(2, TIMESTAMP '2024-01-15 14:30:00', 1500, 105.0), -(3, TIMESTAMP '2024-01-25 15:59:00', 2000, 110.0), --- February 2024 -(4, TIMESTAMP '2024-02-03 09:30:00', 1200, 108.0), -(5, TIMESTAMP '2024-02-14 12:00:00', 1800, 112.0), -(6, TIMESTAMP '2024-02-28 15:59:00', 2200, 115.0), --- March 2024 -(7, TIMESTAMP '2024-03-01 09:30:00', 1300, 113.0), -(8, TIMESTAMP '2024-03-15 14:00:00', 1900, 118.0), -(9, TIMESTAMP '2024-03-29 15:59:00', 2300, 120.0); - -query I -COPY (SELECT * FROM multi_month_data ORDER BY ts ASC) -TO 'test_files/scratch/dynamic_filter_pushdown_config/multi_month_sorted.parquet'; ----- -9 - -# Declare the file has ordering [ts ASC] -# Conceptually this means [date_trunc('month', ts) ASC, ts ASC] due to monotonicity -statement ok -CREATE EXTERNAL TABLE multi_month_parquet(id INT, ts TIMESTAMP, volume BIGINT, price DOUBLE) -STORED AS PARQUET -LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/multi_month_sorted.parquet' -WITH ORDER (ts ASC); - -# Test 8.3a: Request ORDER BY ts DESC (opposite direction) -# File: [ts ASC] -# Request: [ts DESC] -# Should trigger reverse_row_groups=true -query TT -EXPLAIN SELECT * FROM multi_month_parquet -ORDER BY ts DESC -LIMIT 2; ----- -logical_plan -01)Sort: multi_month_parquet.ts DESC NULLS FIRST, fetch=2 -02)--TableScan: multi_month_parquet projection=[id, ts, volume, price] -physical_plan -01)SortExec: TopK(fetch=2), expr=[ts@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/multi_month_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true - -query IPIR -SELECT * FROM multi_month_parquet -ORDER BY ts DESC -LIMIT 2; ----- -9 2024-03-29T15:59:00 2300 120 -8 2024-03-15T14:00:00 1900 118 - -# Test 8.3b: Request ORDER BY date_trunc('month', ts) DESC, ts DESC -# File: [ts ASC] (which implies [date_trunc('month', ts) ASC, ts ASC]) -# Request: [date_trunc('month', ts) DESC, ts DESC] -# The reversed file ordering [ts DESC] satisfies this because: -# - date_trunc is monotonic with ts -# - So [ts DESC] implies [date_trunc('month', ts) DESC, ts DESC] -query TT -EXPLAIN SELECT * FROM multi_month_parquet -ORDER BY date_trunc('month', ts) DESC, ts DESC -LIMIT 2; ----- -logical_plan -01)Sort: date_trunc(Utf8("month"), multi_month_parquet.ts) DESC NULLS FIRST, multi_month_parquet.ts DESC NULLS FIRST, fetch=2 -02)--TableScan: multi_month_parquet projection=[id, ts, volume, price] -physical_plan -01)SortExec: TopK(fetch=2), expr=[date_trunc(month, ts@1) DESC, ts@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/multi_month_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true - -query IPIR -SELECT * FROM multi_month_parquet -ORDER BY date_trunc('month', ts) DESC, ts DESC -LIMIT 2; ----- -9 2024-03-29T15:59:00 2300 120 -8 2024-03-15T14:00:00 1900 118 - -# Test 8.4: CAST as a monotonic function -statement ok -CREATE TABLE int_data(id INT, small_val SMALLINT, big_val BIGINT) AS VALUES -(1, 10, 100), -(2, 20, 200), -(3, 30, 300), -(4, 40, 400), -(5, 50, 500); - -query I -COPY (SELECT * FROM int_data ORDER BY small_val ASC) -TO 'test_files/scratch/dynamic_filter_pushdown_config/int_sorted.parquet'; ----- -5 - -statement ok -CREATE EXTERNAL TABLE int_parquet(id INT, small_val SMALLINT, big_val BIGINT) -STORED AS PARQUET -LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/int_sorted.parquet' -WITH ORDER (small_val ASC); - -# CAST preserves ordering: CAST(small_val AS BIGINT) is monotonic with small_val -query TT -EXPLAIN SELECT * FROM int_parquet -ORDER BY CAST(small_val AS BIGINT) DESC -LIMIT 2; ----- -logical_plan -01)Sort: CAST(int_parquet.small_val AS Int64) DESC NULLS FIRST, fetch=2 -02)--TableScan: int_parquet projection=[id, small_val, big_val] -physical_plan -01)SortExec: TopK(fetch=2), expr=[CAST(small_val@1 AS Int64) DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/int_sorted.parquet]]}, projection=[id, small_val, big_val], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true - -query III -SELECT * FROM int_parquet -ORDER BY CAST(small_val AS BIGINT) DESC -LIMIT 2; ----- -5 50 500 -4 40 400 - -# Test 8.5: CEIL as a monotonic function -statement ok -CREATE TABLE float_data(id INT, value DOUBLE) AS VALUES -(1, 1.1), -(2, 2.3), -(3, 3.5), -(4, 4.7), -(5, 5.9); - -query I -COPY (SELECT * FROM float_data ORDER BY value ASC) -TO 'test_files/scratch/dynamic_filter_pushdown_config/float_sorted.parquet'; ----- -5 - -statement ok -CREATE EXTERNAL TABLE float_parquet(id INT, value DOUBLE) -STORED AS PARQUET -LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/float_sorted.parquet' -WITH ORDER (value ASC); - -# CEIL is monotonic increasing -query TT -EXPLAIN SELECT * FROM float_parquet -ORDER BY CEIL(value) DESC -LIMIT 3; ----- -logical_plan -01)Sort: ceil(float_parquet.value) DESC NULLS FIRST, fetch=3 -02)--TableScan: float_parquet projection=[id, value] -physical_plan -01)SortExec: TopK(fetch=3), expr=[ceil(value@1) DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/float_sorted.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true - -query IR -SELECT * FROM float_parquet -ORDER BY CEIL(value) DESC -LIMIT 3; ----- -5 5.9 -4 4.7 -3 3.5 - -# Test 8.6: Negative case - ABS is NOT monotonic over mixed positive/negative range -statement ok -CREATE TABLE signed_data(id INT, value DOUBLE) AS VALUES -(1, -5.0), -(2, -3.0), -(3, -1.0), -(4, 2.0), -(5, 4.0); - -query I -COPY (SELECT * FROM signed_data ORDER BY value ASC) -TO 'test_files/scratch/dynamic_filter_pushdown_config/signed_sorted.parquet'; ----- -5 - -statement ok -CREATE EXTERNAL TABLE signed_parquet(id INT, value DOUBLE) -STORED AS PARQUET -LOCATION 'test_files/scratch/dynamic_filter_pushdown_config/signed_sorted.parquet' -WITH ORDER (value ASC); - -# ABS is NOT monotonic over the full range [-5, 4], so should NOT trigger reverse scan -query TT -EXPLAIN SELECT * FROM signed_parquet -ORDER BY ABS(value) DESC -LIMIT 3; ----- -logical_plan -01)Sort: abs(signed_parquet.value) DESC NULLS FIRST, fetch=3 -02)--TableScan: signed_parquet projection=[id, value] -physical_plan -01)SortExec: TopK(fetch=3), expr=[abs(value@1) DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/signed_sorted.parquet]]}, projection=[id, value], output_ordering=[value@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] - -# Results should still be correct (no optimization applied) -query IR -SELECT * FROM signed_parquet -ORDER BY ABS(value) DESC -LIMIT 3; ----- -1 -5 -5 4 -2 -3 - -# Cleanup -statement ok -DROP TABLE timestamp_data; - -statement ok -DROP TABLE timestamp_parquet; - -statement ok -DROP TABLE multi_month_data; - -statement ok -DROP TABLE multi_month_parquet; - -statement ok -DROP TABLE int_data; - -statement ok -DROP TABLE int_parquet; - -statement ok -DROP TABLE float_data; - -statement ok -DROP TABLE float_parquet; - -statement ok -DROP TABLE signed_data; - -statement ok -DROP TABLE signed_parquet; - -statement ok -SET datafusion.optimizer.enable_sort_pushdown = true; diff --git a/datafusion/sqllogictest/test_files/sort_pushdown.slt b/datafusion/sqllogictest/test_files/sort_pushdown.slt new file mode 100644 index 0000000000000..58d9915a24be2 --- /dev/null +++ b/datafusion/sqllogictest/test_files/sort_pushdown.slt @@ -0,0 +1,886 @@ +#Sort Pushdown for ordered Parquet files +statement ok +SET datafusion.execution.parquet.pushdown_filters = true; + +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; + +# Test 1: Sort Pushdown for ordered Parquet files +# Create a sorted dataset +statement ok +CREATE TABLE sorted_data(id INT, value INT, name VARCHAR) AS VALUES +(1, 100, 'a'), +(2, 200, 'b'), +(3, 300, 'c'), +(4, 400, 'd'), +(5, 500, 'e'), +(6, 600, 'f'), +(7, 700, 'g'), +(8, 800, 'h'), +(9, 900, 'i'), +(10, 1000, 'j'); + +# Copy to parquet with sorting +query I +COPY (SELECT * FROM sorted_data ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/sorted_data.parquet'; +---- +10 + +statement ok +CREATE EXTERNAL TABLE sorted_parquet(id INT, value INT, name VARCHAR) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/sorted_data.parquet' +WITH ORDER (id ASC); + +# Test 1.1: Sort pushdown with DESC (opposite of ASC) +# Should show reverse_row_groups=true +query TT +EXPLAIN SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3; +---- +logical_plan +01)Sort: sorted_parquet.id DESC NULLS FIRST, fetch=3 +02)--TableScan: sorted_parquet projection=[id, value, name] +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/sorted_data.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +# Test 1.2: Verify results are correct +query IIT +SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3; +---- +10 1000 j +9 900 i +8 800 h + +# Test 1.3: Should NOT apply for ASC (same direction) +query TT +EXPLAIN SELECT * FROM sorted_parquet ORDER BY id ASC LIMIT 3; +---- +logical_plan +01)Sort: sorted_parquet.id ASC NULLS LAST, fetch=3 +02)--TableScan: sorted_parquet projection=[id, value, name] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_data.parquet]]}, projection=[id, value, name], limit=3, output_ordering=[id@0 ASC NULLS LAST], file_type=parquet + +# Test 1.4: Disable sort pushdown +statement ok +SET datafusion.optimizer.enable_sort_pushdown = false; + +query TT +EXPLAIN SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3; +---- +logical_plan +01)Sort: sorted_parquet.id DESC NULLS FIRST, fetch=3 +02)--TableScan: sorted_parquet projection=[id, value, name] +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/sorted_data.parquet]]}, projection=[id, value, name], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] + +# Re-enable +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; + +# Test 1.5: With OFFSET +query TT +EXPLAIN SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3 OFFSET 2; +---- +logical_plan +01)Limit: skip=2, fetch=3 +02)--Sort: sorted_parquet.id DESC NULLS FIRST, fetch=5 +03)----TableScan: sorted_parquet projection=[id, value, name] +physical_plan +01)GlobalLimitExec: skip=2, fetch=3 +02)--SortExec: TopK(fetch=5), expr=[id@0 DESC], preserve_partitioning=[false] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_data.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +query IIT +SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3 OFFSET 2; +---- +8 800 h +7 700 g +6 600 f + +# Test 1.6: Reverse scan with row selection (page index pruning) +# This tests that when reverse_row_groups=true, the RowSelection is also properly reversed + +# Create a dataset with multiple row groups and enable page index +statement ok +CREATE TABLE multi_rg_data(id INT, category VARCHAR, value INT) AS VALUES +(1, 'alpha', 10), +(2, 'alpha', 20), +(3, 'beta', 30), +(4, 'beta', 40), +(5, 'gamma', 50), +(6, 'gamma', 60), +(7, 'delta', 70), +(8, 'delta', 80); + +# Write with small row groups (2 rows each = 4 row groups) +statement ok +SET datafusion.execution.parquet.max_row_group_size = 2; + +query I +COPY (SELECT * FROM multi_rg_data ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/multi_rg_sorted.parquet'; +---- +8 + +# Reset row group size +statement ok +SET datafusion.execution.parquet.max_row_group_size = 1048576; + +statement ok +CREATE EXTERNAL TABLE multi_rg_sorted(id INT, category VARCHAR, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/multi_rg_sorted.parquet' +WITH ORDER (id ASC); + +# Enable page index for better pruning +statement ok +SET datafusion.execution.parquet.enable_page_index = true; + +statement ok +SET datafusion.execution.parquet.pushdown_filters = true; + +# Test with reverse scan and filter that prunes some row groups +# This will create a RowSelection with partial row group scans +query TT +EXPLAIN SELECT * FROM multi_rg_sorted +WHERE category IN ('alpha', 'gamma') +ORDER BY id DESC LIMIT 5; +---- +logical_plan +01)Sort: multi_rg_sorted.id DESC NULLS FIRST, fetch=5 +02)--Filter: multi_rg_sorted.category = Utf8View("alpha") OR multi_rg_sorted.category = Utf8View("gamma") +03)----TableScan: multi_rg_sorted projection=[id, category, value], partial_filters=[multi_rg_sorted.category = Utf8View("alpha") OR multi_rg_sorted.category = Utf8View("gamma")] +physical_plan +01)SortExec: TopK(fetch=5), expr=[id@0 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_rg_sorted.parquet]]}, projection=[id, category, value], file_type=parquet, predicate=(category@1 = alpha OR category@1 = gamma) AND DynamicFilter [ empty ], reverse_row_groups=true, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1 OR category_null_count@2 != row_count@3 AND category_min@0 <= gamma AND gamma <= category_max@1, required_guarantees=[category in (alpha, gamma)] + +# Verify the results are correct despite reverse scanning with row selection +# Expected: gamma values (6, 5) then alpha values (2, 1), in DESC order by id +query ITI +SELECT * FROM multi_rg_sorted +WHERE category IN ('alpha', 'gamma') +ORDER BY id DESC LIMIT 5; +---- +6 gamma 60 +5 gamma 50 +2 alpha 20 +1 alpha 10 + +# Test with more complex selection pattern +query ITI +SELECT * FROM multi_rg_sorted +WHERE category IN ('beta', 'delta') +ORDER BY id DESC; +---- +8 delta 80 +7 delta 70 +4 beta 40 +3 beta 30 + +# Test forward scan for comparison (should give same logical results in ASC order) +query ITI +SELECT * FROM multi_rg_sorted +WHERE category IN ('alpha', 'gamma') +ORDER BY id ASC; +---- +1 alpha 10 +2 alpha 20 +5 gamma 50 +6 gamma 60 + +# Disable reverse scan and verify it still works +statement ok +SET datafusion.optimizer.enable_sort_pushdown = false; + +query ITI +SELECT * FROM multi_rg_sorted +WHERE category IN ('alpha', 'gamma') +ORDER BY id DESC LIMIT 5; +---- +6 gamma 60 +5 gamma 50 +2 alpha 20 +1 alpha 10 + +# Re-enable +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; + +# Test 1.7: Sort pushdown with more than one partition +# Create multiple parquet files to trigger it + +# Split data into multiple files +statement ok +CREATE TABLE sorted_data_part1(id INT, value INT, name VARCHAR) AS VALUES +(1, 100, 'a'), +(2, 200, 'b'), +(3, 300, 'c'); + +statement ok +CREATE TABLE sorted_data_part2(id INT, value INT, name VARCHAR) AS VALUES +(4, 400, 'd'), +(5, 500, 'e'), +(6, 600, 'f'); + +statement ok +CREATE TABLE sorted_data_part3(id INT, value INT, name VARCHAR) AS VALUES +(7, 700, 'g'), +(8, 800, 'h'), +(9, 900, 'i'), +(10, 1000, 'j'); + +# Create directory for multi-file parquet +query I +COPY (SELECT * FROM sorted_data_part1 ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/sorted_multi/part1.parquet'; +---- +3 + +query I +COPY (SELECT * FROM sorted_data_part2 ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/sorted_multi/part2.parquet'; +---- +3 + +query I +COPY (SELECT * FROM sorted_data_part3 ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/sorted_multi/part3.parquet'; +---- +4 + +# Create external table pointing to directory with multiple files +statement ok +CREATE EXTERNAL TABLE sorted_parquet_multi(id INT, value INT, name VARCHAR) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/sorted_multi/' +WITH ORDER (id ASC); + +# Enable multiple partitions +statement ok +SET datafusion.execution.target_partitions = 4; + +# Now we should see RepartitionExec because we have 3 input partitions (3 files) +query TT +EXPLAIN SELECT * FROM sorted_parquet_multi ORDER BY id DESC LIMIT 3; +---- +logical_plan +01)Sort: sorted_parquet_multi.id DESC NULLS FIRST, fetch=3 +02)--TableScan: sorted_parquet_multi projection=[id, value, name] +physical_plan +01)SortPreservingMergeExec: [id@0 DESC], fetch=3 +02)--SortExec: TopK(fetch=3), expr=[id@0 DESC], preserve_partitioning=[true] +03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_multi/part1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_multi/part3.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +# Verify correctness with repartitioning and multiple files +query IIT +SELECT * FROM sorted_parquet_multi ORDER BY id DESC LIMIT 3; +---- +10 1000 j +9 900 i +8 800 h + +# Test ASC order (should not trigger reverse scan) +query IIT +SELECT * FROM sorted_parquet_multi ORDER BY id ASC LIMIT 3; +---- +1 100 a +2 200 b +3 300 c + +# Cleanup +statement ok +DROP TABLE sorted_data_part1; + +statement ok +DROP TABLE sorted_data_part2; + +statement ok +DROP TABLE sorted_data_part3; + +statement ok +DROP TABLE sorted_parquet_multi; + +# Reset to default +statement ok +SET datafusion.execution.target_partitions = 4; + +# Cleanup +statement ok +DROP TABLE multi_rg_data; + +statement ok +DROP TABLE multi_rg_sorted; + +statement ok +SET datafusion.execution.parquet.enable_page_index = false; + +statement ok +SET datafusion.execution.parquet.pushdown_filters = true; + +# Cleanup +statement ok +DROP TABLE sorted_data; + +statement ok +DROP TABLE sorted_parquet; + +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; + + +# Test 2: Sort pushdown with constant column filtering +# This tests the case where a leading sort column becomes constant through WHERE filtering + +# Create a multi-column sorted dataset (like time-series data) +statement ok +CREATE TABLE timeseries_data(timeframe VARCHAR, period_end INT, value DOUBLE) AS VALUES +('daily', 1, 100.0), +('daily', 2, 150.0), +('daily', 3, 200.0), +('weekly', 1, 500.0), +('weekly', 2, 600.0), +('weekly', 3, 700.0), +('monthly', 1, 1000.0), +('monthly', 2, 1100.0), +('monthly', 3, 1200.0), +('quarterly', 1, 5000.0), +('quarterly', 2, 5500.0), +('quarterly', 3, 6000.0); + +# Copy to parquet with multi-column sorting (timeframe ASC, period_end ASC) +query I +COPY (SELECT * FROM timeseries_data ORDER BY timeframe ASC, period_end ASC) +TO 'test_files/scratch/sort_pushdown/timeseries_sorted.parquet'; +---- +12 + +statement ok +CREATE EXTERNAL TABLE timeseries_parquet(timeframe VARCHAR, period_end INT, value DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/timeseries_sorted.parquet' +WITH ORDER (timeframe ASC, period_end ASC); + +# Test 2.1: Query with constant prefix filter and DESC on remaining column +# WHERE timeframe='quarterly' makes the first sort column constant +# ORDER BY period_end DESC should trigger reverse scan because: +# File ordering: [timeframe ASC, period_end ASC] +# After filtering timeframe='quarterly': effectively [period_end ASC] +# Request: [period_end DESC] -> exact reverse! +query TT +EXPLAIN SELECT * FROM timeseries_parquet +WHERE timeframe = 'quarterly' +ORDER BY period_end DESC +LIMIT 2; +---- +logical_plan +01)Sort: timeseries_parquet.period_end DESC NULLS FIRST, fetch=2 +02)--Filter: timeseries_parquet.timeframe = Utf8View("quarterly") +03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] +physical_plan +01)SortExec: TopK(fetch=2), expr=[period_end@1 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=timeframe@0 = quarterly AND DynamicFilter [ empty ], reverse_row_groups=true, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] + +# Test 2.2: Verify the results are correct +query TIR +SELECT * FROM timeseries_parquet +WHERE timeframe = 'quarterly' +ORDER BY period_end DESC +LIMIT 2; +---- +quarterly 3 6000 +quarterly 2 5500 + +# Test 2.3: Same filter but ASC order (should not trigger reverse scan, ordering already satisfied) +query TT +EXPLAIN SELECT * FROM timeseries_parquet +WHERE timeframe = 'quarterly' +ORDER BY period_end ASC +LIMIT 2; +---- +logical_plan +01)Sort: timeseries_parquet.period_end ASC NULLS LAST, fetch=2 +02)--Filter: timeseries_parquet.timeframe = Utf8View("quarterly") +03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], limit=2, output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=timeframe@0 = quarterly, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] + +# Test 2.4: Verify ASC results +query TIR +SELECT * FROM timeseries_parquet +WHERE timeframe = 'quarterly' +ORDER BY period_end ASC +LIMIT 2; +---- +quarterly 1 5000 +quarterly 2 5500 + +# Test 2.5: Test with different constant value +query TIR +SELECT * FROM timeseries_parquet +WHERE timeframe = 'weekly' +ORDER BY period_end DESC; +---- +weekly 3 700 +weekly 2 600 +weekly 1 500 + +# Test 2.6: Test without constant filter (no reverse scan because need both columns) +# Request: [timeframe ASC, period_end DESC] +# File has: [timeframe ASC, period_end ASC] +# These are NOT reverse of each other - only second column is reversed +query TT +EXPLAIN SELECT * FROM timeseries_parquet +ORDER BY timeframe ASC, period_end DESC +LIMIT 3; +---- +logical_plan +01)Sort: timeseries_parquet.timeframe ASC NULLS LAST, timeseries_parquet.period_end DESC NULLS FIRST, fetch=3 +02)--TableScan: timeseries_parquet projection=[timeframe, period_end, value] +physical_plan +01)SortExec: TopK(fetch=3), expr=[timeframe@0 ASC NULLS LAST, period_end@1 DESC], preserve_partitioning=[false], sort_prefix=[timeframe@0 ASC NULLS LAST] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] + +# Test 2.7: Disable sort pushdown and verify filter still works +statement ok +SET datafusion.optimizer.enable_sort_pushdown = false; + +query TT +EXPLAIN SELECT * FROM timeseries_parquet +WHERE timeframe = 'quarterly' +ORDER BY period_end DESC +LIMIT 2; +---- +logical_plan +01)Sort: timeseries_parquet.period_end DESC NULLS FIRST, fetch=2 +02)--Filter: timeseries_parquet.timeframe = Utf8View("quarterly") +03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] +physical_plan +01)SortExec: TopK(fetch=2), expr=[period_end@1 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=timeframe@0 = quarterly AND DynamicFilter [ empty ], pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] + +# Results should still be correct +query TIR +SELECT * FROM timeseries_parquet +WHERE timeframe = 'quarterly' +ORDER BY period_end DESC +LIMIT 2; +---- +quarterly 3 6000 +quarterly 2 5500 + +# Re-enable +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; + +# Test 2.8: Test with IN clause (multiple constant values) +# Note: IN clause with multiple values means timeframe is NOT constant +# (could be 'daily' or 'weekly'), so the first sort column cannot be eliminated. +# Without a constant first column, we cannot reverse scan based on just period_end DESC. +# The physical plan should NOT show reverse_row_groups=true +query TT +EXPLAIN SELECT * FROM timeseries_parquet +WHERE timeframe IN ('daily', 'weekly') +ORDER BY period_end DESC +LIMIT 3; +---- +logical_plan +01)Sort: timeseries_parquet.period_end DESC NULLS FIRST, fetch=3 +02)--Filter: timeseries_parquet.timeframe = Utf8View("daily") OR timeseries_parquet.timeframe = Utf8View("weekly") +03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("daily") OR timeseries_parquet.timeframe = Utf8View("weekly")] +physical_plan +01)SortExec: TopK(fetch=3), expr=[period_end@1 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=(timeframe@0 = daily OR timeframe@0 = weekly) AND DynamicFilter [ empty ], pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= daily AND daily <= timeframe_max@1 OR timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= weekly AND weekly <= timeframe_max@1, required_guarantees=[timeframe in (daily, weekly)] + +# Test 2.9: Complex case - literal constant in sort expression itself +# The literal 'constant' is ignored in sort analysis +# After stripping: ORDER BY period_end DESC +# With WHERE timeframe='monthly' making first column constant +# File: [period_end ASC] (after constant column removal) +# Request: [period_end DESC] -> exact reverse, triggers reverse scan +query TT +EXPLAIN SELECT * FROM timeseries_parquet +WHERE timeframe = 'monthly' +ORDER BY 'constant', period_end DESC +LIMIT 2; +---- +logical_plan +01)Sort: Utf8("constant") ASC NULLS LAST, timeseries_parquet.period_end DESC NULLS FIRST, fetch=2 +02)--Filter: timeseries_parquet.timeframe = Utf8View("monthly") +03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("monthly")] +physical_plan +01)SortExec: TopK(fetch=2), expr=[period_end@1 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=timeframe@0 = monthly AND DynamicFilter [ empty ], reverse_row_groups=true, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= monthly AND monthly <= timeframe_max@1, required_guarantees=[timeframe in (monthly)] + +# Verify results +query TIR +SELECT * FROM timeseries_parquet +WHERE timeframe = 'monthly' +ORDER BY period_end DESC +LIMIT 2; +---- +monthly 3 1200 +monthly 2 1100 + +# Test 2.10: Filter on non-leading sort column +# File order: [timeframe ASC, period_end ASC] +# Filter: period_end = 2 (makes second column constant) +# Request: [timeframe DESC] +# After constant column removal: File has [timeframe ASC], Request wants [timeframe DESC] +# This is exact reverse -> triggers reverse scan +query TT +EXPLAIN SELECT * FROM timeseries_parquet +WHERE period_end = 2 +ORDER BY timeframe DESC; +---- +logical_plan +01)Sort: timeseries_parquet.timeframe DESC NULLS FIRST +02)--Filter: timeseries_parquet.period_end = Int32(2) +03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.period_end = Int32(2)] +physical_plan +01)SortExec: expr=[timeframe@0 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=period_end@1 = 2, reverse_row_groups=true, pruning_predicate=period_end_null_count@2 != row_count@3 AND period_end_min@0 <= 2 AND 2 <= period_end_max@1, required_guarantees=[period_end in (2)] + +# Cleanup +statement ok +DROP TABLE timeseries_data; + +statement ok +DROP TABLE timeseries_parquet; + +# Reset to default +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; + + +# Test 3: Sort pushdown with monotonic functions +# This tests that reverse scan works when sort expressions involve monotonic functions + +# Create test data with timestamp column +statement ok +CREATE TABLE timestamp_data(id INT, ts TIMESTAMP, volume BIGINT, price DOUBLE) AS VALUES +(1, TIMESTAMP '2024-01-15 10:00:00', 1000, 100.0), +(2, TIMESTAMP '2024-01-20 11:00:00', 1500, 105.0), +(3, TIMESTAMP '2024-01-25 12:00:00', 2000, 110.0), +(4, TIMESTAMP '2024-02-05 09:00:00', 1200, 108.0), +(5, TIMESTAMP '2024-02-15 14:00:00', 1800, 112.0), +(6, TIMESTAMP '2024-02-25 15:00:00', 2200, 115.0), +(7, TIMESTAMP '2024-03-10 09:00:00', 1300, 113.0), +(8, TIMESTAMP '2024-03-18 14:00:00', 1900, 118.0), +(9, TIMESTAMP '2024-03-28 15:00:00', 2300, 120.0); + +# Copy to parquet with sorting by timestamp ASC +query I +COPY (SELECT * FROM timestamp_data ORDER BY ts ASC) +TO 'test_files/scratch/sort_pushdown/timestamp_sorted.parquet'; +---- +9 + +# Test 3.1: Simple monotonic function - date_trunc +# Create external table with file ordering that conceptually includes date_trunc +# File is actually sorted by [ts ASC], but conceptually [date_trunc('month', ts) ASC, ts ASC] +statement ok +CREATE EXTERNAL TABLE timestamp_parquet(id INT, ts TIMESTAMP, volume BIGINT, price DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/timestamp_sorted.parquet' +WITH ORDER (ts ASC); + +# Query with ORDER BY ts DESC +# File ordering: [ts ASC] +# Request: [ts DESC] +# This should trigger reverse_row_groups=true +query TT +EXPLAIN SELECT * FROM timestamp_parquet +ORDER BY ts DESC +LIMIT 3; +---- +logical_plan +01)Sort: timestamp_parquet.ts DESC NULLS FIRST, fetch=3 +02)--TableScan: timestamp_parquet projection=[id, ts, volume, price] +physical_plan +01)SortExec: TopK(fetch=3), expr=[ts@1 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timestamp_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +# Verify results +query IPIR +SELECT * FROM timestamp_parquet +ORDER BY ts DESC +LIMIT 3; +---- +9 2024-03-28T15:00:00 2300 120 +8 2024-03-18T14:00:00 1900 118 +7 2024-03-10T09:00:00 1300 113 + +# Test 3.2: Monotonic function in ORDER BY - date_trunc DESC +# File ordering: [ts ASC] +# Request: [date_trunc('day', ts) DESC] +# Since date_trunc is monotonic with ts, reversed file ordering [ts DESC] satisfies [date_trunc DESC] +query TT +EXPLAIN SELECT * FROM timestamp_parquet +ORDER BY date_trunc('day', ts) DESC +LIMIT 3; +---- +logical_plan +01)Sort: date_trunc(Utf8("day"), timestamp_parquet.ts) DESC NULLS FIRST, fetch=3 +02)--TableScan: timestamp_parquet projection=[id, ts, volume, price] +physical_plan +01)SortExec: TopK(fetch=3), expr=[date_trunc(day, ts@1) DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timestamp_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +# Verify results (descending day) +query IPIR +SELECT * FROM timestamp_parquet +ORDER BY date_trunc('day', ts) DESC +LIMIT 3; +---- +9 2024-03-28T15:00:00 2300 120 +8 2024-03-18T14:00:00 1900 118 +7 2024-03-10T09:00:00 1300 113 + +# Test 3.3: Multi-column scenario with explicit monotonic function in file ordering +# Create a table where we explicitly declare the ordering includes the monotonic function +# This simulates files that are partitioned/sorted by [date_trunc('month', ts) ASC, ts ASC] + +# Create a new parquet file sorted by [ts ASC] (which implies date_trunc ordering) +statement ok +CREATE TABLE multi_month_data(id INT, ts TIMESTAMP, volume BIGINT, price DOUBLE) AS VALUES +-- January 2024 +(1, TIMESTAMP '2024-01-05 09:30:00', 1000, 100.0), +(2, TIMESTAMP '2024-01-15 14:30:00', 1500, 105.0), +(3, TIMESTAMP '2024-01-25 15:59:00', 2000, 110.0), +-- February 2024 +(4, TIMESTAMP '2024-02-03 09:30:00', 1200, 108.0), +(5, TIMESTAMP '2024-02-14 12:00:00', 1800, 112.0), +(6, TIMESTAMP '2024-02-28 15:59:00', 2200, 115.0), +-- March 2024 +(7, TIMESTAMP '2024-03-01 09:30:00', 1300, 113.0), +(8, TIMESTAMP '2024-03-15 14:00:00', 1900, 118.0), +(9, TIMESTAMP '2024-03-29 15:59:00', 2300, 120.0); + +query I +COPY (SELECT * FROM multi_month_data ORDER BY ts ASC) +TO 'test_files/scratch/sort_pushdown/multi_month_sorted.parquet'; +---- +9 + +# Declare the file has ordering [ts ASC] +# Conceptually this means [date_trunc('month', ts) ASC, ts ASC] due to monotonicity +statement ok +CREATE EXTERNAL TABLE multi_month_parquet(id INT, ts TIMESTAMP, volume BIGINT, price DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/multi_month_sorted.parquet' +WITH ORDER (ts ASC); + +# Test 3.3a: Request ORDER BY ts DESC (opposite direction) +# File: [ts ASC] +# Request: [ts DESC] +# Should trigger reverse_row_groups=true +query TT +EXPLAIN SELECT * FROM multi_month_parquet +ORDER BY ts DESC +LIMIT 2; +---- +logical_plan +01)Sort: multi_month_parquet.ts DESC NULLS FIRST, fetch=2 +02)--TableScan: multi_month_parquet projection=[id, ts, volume, price] +physical_plan +01)SortExec: TopK(fetch=2), expr=[ts@1 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_month_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +query IPIR +SELECT * FROM multi_month_parquet +ORDER BY ts DESC +LIMIT 2; +---- +9 2024-03-29T15:59:00 2300 120 +8 2024-03-15T14:00:00 1900 118 + +# Test 3.3b: Request ORDER BY date_trunc('month', ts) DESC, ts DESC +# File: [ts ASC] (which implies [date_trunc('month', ts) ASC, ts ASC]) +# Request: [date_trunc('month', ts) DESC, ts DESC] +# The reversed file ordering [ts DESC] satisfies this because: +# - date_trunc is monotonic with ts +# - So [ts DESC] implies [date_trunc('month', ts) DESC, ts DESC] +query TT +EXPLAIN SELECT * FROM multi_month_parquet +ORDER BY date_trunc('month', ts) DESC, ts DESC +LIMIT 2; +---- +logical_plan +01)Sort: date_trunc(Utf8("month"), multi_month_parquet.ts) DESC NULLS FIRST, multi_month_parquet.ts DESC NULLS FIRST, fetch=2 +02)--TableScan: multi_month_parquet projection=[id, ts, volume, price] +physical_plan +01)SortExec: TopK(fetch=2), expr=[date_trunc(month, ts@1) DESC, ts@1 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_month_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +query IPIR +SELECT * FROM multi_month_parquet +ORDER BY date_trunc('month', ts) DESC, ts DESC +LIMIT 2; +---- +9 2024-03-29T15:59:00 2300 120 +8 2024-03-15T14:00:00 1900 118 + +# Test 3.4: CAST as a monotonic function +statement ok +CREATE TABLE int_data(id INT, small_val SMALLINT, big_val BIGINT) AS VALUES +(1, 10, 100), +(2, 20, 200), +(3, 30, 300), +(4, 40, 400), +(5, 50, 500); + +query I +COPY (SELECT * FROM int_data ORDER BY small_val ASC) +TO 'test_files/scratch/sort_pushdown/int_sorted.parquet'; +---- +5 + +statement ok +CREATE EXTERNAL TABLE int_parquet(id INT, small_val SMALLINT, big_val BIGINT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/int_sorted.parquet' +WITH ORDER (small_val ASC); + +# CAST preserves ordering: CAST(small_val AS BIGINT) is monotonic with small_val +query TT +EXPLAIN SELECT * FROM int_parquet +ORDER BY CAST(small_val AS BIGINT) DESC +LIMIT 2; +---- +logical_plan +01)Sort: CAST(int_parquet.small_val AS Int64) DESC NULLS FIRST, fetch=2 +02)--TableScan: int_parquet projection=[id, small_val, big_val] +physical_plan +01)SortExec: TopK(fetch=2), expr=[CAST(small_val@1 AS Int64) DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/int_sorted.parquet]]}, projection=[id, small_val, big_val], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +query III +SELECT * FROM int_parquet +ORDER BY CAST(small_val AS BIGINT) DESC +LIMIT 2; +---- +5 50 500 +4 40 400 + +# Test 3.5: CEIL as a monotonic function +statement ok +CREATE TABLE float_data(id INT, value DOUBLE) AS VALUES +(1, 1.1), +(2, 2.3), +(3, 3.5), +(4, 4.7), +(5, 5.9); + +query I +COPY (SELECT * FROM float_data ORDER BY value ASC) +TO 'test_files/scratch/sort_pushdown/float_sorted.parquet'; +---- +5 + +statement ok +CREATE EXTERNAL TABLE float_parquet(id INT, value DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/float_sorted.parquet' +WITH ORDER (value ASC); + +# CEIL is monotonic increasing +query TT +EXPLAIN SELECT * FROM float_parquet +ORDER BY CEIL(value) DESC +LIMIT 3; +---- +logical_plan +01)Sort: ceil(float_parquet.value) DESC NULLS FIRST, fetch=3 +02)--TableScan: float_parquet projection=[id, value] +physical_plan +01)SortExec: TopK(fetch=3), expr=[ceil(value@1) DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/float_sorted.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +query IR +SELECT * FROM float_parquet +ORDER BY CEIL(value) DESC +LIMIT 3; +---- +5 5.9 +4 4.7 +3 3.5 + +# Test 3.6: Negative case - ABS is NOT monotonic over mixed positive/negative range +statement ok +CREATE TABLE signed_data(id INT, value DOUBLE) AS VALUES +(1, -5.0), +(2, -3.0), +(3, -1.0), +(4, 2.0), +(5, 4.0); + +query I +COPY (SELECT * FROM signed_data ORDER BY value ASC) +TO 'test_files/scratch/sort_pushdown/signed_sorted.parquet'; +---- +5 + +statement ok +CREATE EXTERNAL TABLE signed_parquet(id INT, value DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/signed_sorted.parquet' +WITH ORDER (value ASC); + +# ABS is NOT monotonic over the full range [-5, 4], so should NOT trigger reverse scan +query TT +EXPLAIN SELECT * FROM signed_parquet +ORDER BY ABS(value) DESC +LIMIT 3; +---- +logical_plan +01)Sort: abs(signed_parquet.value) DESC NULLS FIRST, fetch=3 +02)--TableScan: signed_parquet projection=[id, value] +physical_plan +01)SortExec: TopK(fetch=3), expr=[abs(value@1) DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/signed_sorted.parquet]]}, projection=[id, value], output_ordering=[value@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] + +# Results should still be correct (no optimization applied) +query IR +SELECT * FROM signed_parquet +ORDER BY ABS(value) DESC +LIMIT 3; +---- +1 -5 +5 4 +2 -3 + +# Cleanup +statement ok +DROP TABLE timestamp_data; + +statement ok +DROP TABLE timestamp_parquet; + +statement ok +DROP TABLE multi_month_data; + +statement ok +DROP TABLE multi_month_parquet; + +statement ok +DROP TABLE int_data; + +statement ok +DROP TABLE int_parquet; + +statement ok +DROP TABLE float_data; + +statement ok +DROP TABLE float_parquet; + +statement ok +DROP TABLE signed_data; + +statement ok +DROP TABLE signed_parquet; + +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; From 667c3a2118f02e7bdfd926543b0f2a0bb9bf23d4 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 6 Jan 2026 23:06:57 +0800 Subject: [PATCH 06/10] fix --- .../tests/physical_optimizer/pushdown_sort.rs | 18 +++++++++--------- .../tests/physical_optimizer/test_utils.rs | 9 ++++++--- datafusion/datasource-parquet/src/opener.rs | 4 ++-- datafusion/datasource-parquet/src/sort.rs | 4 ++-- datafusion/physical-plan/src/filter.rs | 3 +-- 5 files changed, 20 insertions(+), 18 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/pushdown_sort.rs b/datafusion/core/tests/physical_optimizer/pushdown_sort.rs index e2eb36586431e..fca3006c3f8d7 100644 --- a/datafusion/core/tests/physical_optimizer/pushdown_sort.rs +++ b/datafusion/core/tests/physical_optimizer/pushdown_sort.rs @@ -27,15 +27,15 @@ use datafusion_physical_expr::expressions; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_optimizer::pushdown_sort::PushdownSort; +use datafusion_physical_optimizer::PhysicalOptimizerRule; use std::sync::Arc; use crate::physical_optimizer::test_utils::{ - OptimizationTest, coalesce_batches_exec, coalesce_partitions_exec, parquet_exec, + coalesce_batches_exec, coalesce_partitions_exec, parquet_exec, parquet_exec_with_sort, projection_exec, projection_exec_with_alias, repartition_exec, schema, simple_projection_exec, sort_exec, sort_exec_with_fetch, - sort_expr, sort_expr_named, test_scan_with_ordering, + sort_expr, sort_expr_named, test_scan_with_ordering, OptimizationTest, }; #[test] @@ -632,10 +632,10 @@ fn test_pushdown_through_blocking_node() { count_udaf(), vec![Arc::new(expressions::Column::new("b", 1)) as _], ) - .schema(Arc::clone(&schema)) - .alias("COUNT(b)") - .build() - .unwrap(), + .schema(Arc::clone(&schema)) + .alias("COUNT(b)") + .build() + .unwrap(), ); let aggregate = Arc::new( @@ -647,7 +647,7 @@ fn test_pushdown_through_blocking_node() { inner_sort, Arc::clone(&schema), ) - .unwrap(), + .unwrap(), ); // Outer Sort: [a ASC] - this CANNOT push down through aggregate @@ -776,7 +776,7 @@ fn test_no_sort_pushdown_through_computed_projection() { ], source, ) - .unwrap(); + .unwrap(); // Request [sum DESC] - should NOT push down (sum is computed) let sum_expr = sort_expr_named("sum", 0); diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 0f02e6b4b9c9c..4c1c7c8016791 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -42,8 +42,8 @@ use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; use datafusion_functions_aggregate::count::count_udaf; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; -use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_expr::expressions::{self, col}; +use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{ LexOrdering, OrderingRequirements, PhysicalSortExpr, @@ -55,6 +55,7 @@ use datafusion_physical_plan::aggregates::{ }; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::{JoinFilter, JoinOn}; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode, SortMergeJoinExec}; @@ -67,8 +68,10 @@ use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::windows::{create_window_expr, BoundedWindowAggExec}; -use datafusion_physical_plan::{displayable, DisplayAs, DisplayFormatType, ExecutionPlan, InputOrderMode, Partitioning, PlanProperties, SortOrderPushdownResult}; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::{ + displayable, DisplayAs, DisplayFormatType, ExecutionPlan, InputOrderMode, + Partitioning, PlanProperties, SortOrderPushdownResult, +}; /// Create a non sorted parquet exec pub fn parquet_exec(schema: SchemaRef) -> Arc { diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 7774c49832a0e..2e69596a649e1 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -832,9 +832,8 @@ fn should_enable_page_index( #[cfg(test)] mod test { - use std::sync::Arc; + use crate::{opener::ParquetOpener, DefaultParquetFileReaderFactory, RowGroupAccess}; use arrow::compute::cast; - use crate::{DefaultParquetFileReaderFactory, RowGroupAccess, opener::ParquetOpener}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use bytes::{BufMut, BytesMut}; use datafusion_common::{ @@ -858,6 +857,7 @@ mod test { use object_store::{memory::InMemory, path::Path, ObjectStore}; use parquet::arrow::ArrowWriter; use parquet::file::properties::WriterProperties; + use std::sync::Arc; async fn count_batches_and_rows( mut stream: std::pin::Pin< diff --git a/datafusion/datasource-parquet/src/sort.rs b/datafusion/datasource-parquet/src/sort.rs index abc50eeb317d5..b74afad443e67 100644 --- a/datafusion/datasource-parquet/src/sort.rs +++ b/datafusion/datasource-parquet/src/sort.rs @@ -120,13 +120,13 @@ pub fn reverse_row_selection( #[cfg(test)] mod tests { + use crate::opener::PreparedAccessPlan; use crate::ParquetAccessPlan; use crate::RowGroupAccess; - use crate::opener::PreparedAccessPlan; use arrow::datatypes::{DataType, Field, Schema}; use bytes::Bytes; - use parquet::arrow::ArrowWriter; use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; + use parquet::arrow::ArrowWriter; use parquet::file::reader::FileReader; use parquet::file::serialized_reader::SerializedFileReader; use std::sync::Arc; diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 48e8efd336abd..2ee7d283d8ad8 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -24,7 +24,7 @@ use itertools::Itertools; use super::{ ColumnStatistics, DisplayAs, ExecutionPlanProperties, PlanProperties, - RecordBatchStream, SendableRecordBatchStream, SortOrderPushdownResult, Statistics, + RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::common::can_project; use crate::execution_plan::CardinalityEffect; @@ -63,7 +63,6 @@ use datafusion_physical_expr::{ }; use datafusion_physical_expr_common::physical_expr::fmt_sql; -use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use futures::stream::{Stream, StreamExt}; use log::trace; From 419fea5015597ee432e9e4811cd876844784cc84 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Tue, 6 Jan 2026 23:20:41 +0800 Subject: [PATCH 07/10] fix --- datafusion/physical-optimizer/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index 88a1ff30d80cf..7f63a619a14e3 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -51,8 +51,8 @@ datafusion-physical-expr-common = { workspace = true } datafusion-physical-plan = { workspace = true } datafusion-pruning = { workspace = true } itertools = { workspace = true } -recursive = { workspace = true, optional = true } log = "0.4.28" +recursive = { workspace = true, optional = true } [dev-dependencies] datafusion-expr = { workspace = true } From f76f58a3e64b2117fd6e495d878a603369ea2c7c Mon Sep 17 00:00:00 2001 From: Qi Zhu Date: Tue, 9 Dec 2025 02:13:11 +0800 Subject: [PATCH 08/10] Add sorted data benchmark. (#19042) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Which issue does this PR close? Add sorted data benchmark. - Closes[ #18976](https://github.com/apache/datafusion/issues/18976) ## Rationale for this change ## What changes are included in this PR? ## Are these changes tested? Yes, test results for reverse parquet PR, it's 30X faster than main branch for sorted data: https://github.com/apache/datafusion/pull/18817 ```rust Running `/Users/zhuqi/arrow-datafusion/target/release/dfbench clickbench --iterations 5 --path /Users/zhuqi/arrow-datafusion/benchmarks/data/hits_0_sorted.parquet --queries-path /Users/zhuqi/arrow-datafusion/benchmarks/queries/clickbench/queries/sorted_data --sorted-by EventTime --sort-order ASC -o /Users/zhuqi/arrow-datafusion/benchmarks/results/reverse_parquet/data_sorted_clickbench.json` Running benchmarks with the following options: RunOpt { query: None, pushdown: false, common: CommonOpt { iterations: 5, partitions: None, batch_size: None, mem_pool_type: "fair", memory_limit: None, sort_spill_reservation_bytes: None, debug: false }, path: "/Users/zhuqi/arrow-datafusion/benchmarks/data/hits_0_sorted.parquet", queries_path: "/Users/zhuqi/arrow-datafusion/benchmarks/queries/clickbench/queries/sorted_data", output_path: Some("/Users/zhuqi/arrow-datafusion/benchmarks/results/reverse_parquet/data_sorted_clickbench.json"), sorted_by: Some("EventTime"), sort_order: "ASC" } âš ī¸ Forcing target_partitions=1 to preserve sort order âš ī¸ (Because we want to get the pure performance benefit of sorted data to compare) 📊 Session config target_partitions: 1 Registering table with sort order: EventTime ASC Executing: CREATE EXTERNAL TABLE hits STORED AS PARQUET LOCATION '/Users/zhuqi/arrow-datafusion/benchmarks/data/hits_0_sorted.parquet' WITH ORDER ("EventTime" ASC) Q0: -- Must set for ClickBench hits_partitioned dataset. See https://github.com/apache/datafusion/issues/16591 -- set datafusion.execution.parquet.binary_as_string = true SELECT * FROM hits ORDER BY "EventTime" DESC limit 10; Query 0 iteration 0 took 14.7 ms and returned 10 rows Query 0 iteration 1 took 10.2 ms and returned 10 rows Query 0 iteration 2 took 8.7 ms and returned 10 rows Query 0 iteration 3 took 7.9 ms and returned 10 rows Query 0 iteration 4 took 7.9 ms and returned 10 rows Query 0 avg time: 9.85 ms + set +x Done ``` And the main branch result: ```rust Running `/Users/zhuqi/arrow-datafusion/target/release/dfbench clickbench --iterations 5 --path /Users/zhuqi/arrow-datafusion/benchmarks/data/hits_0_sorted.parquet --queries-path /Users/zhuqi/arrow-datafusion/benchmarks/queries/clickbench/queries/sorted_data --sorted-by EventTime --sort-order ASC -o /Users/zhuqi/arrow-datafusion/benchmarks/results/issue_18976/data_sorted_clickbench.json` Running benchmarks with the following options: RunOpt { query: None, pushdown: false, common: CommonOpt { iterations: 5, partitions: None, batch_size: None, mem_pool_type: "fair", memory_limit: None, sort_spill_reservation_bytes: None, debug: false }, path: "/Users/zhuqi/arrow-datafusion/benchmarks/data/hits_0_sorted.parquet", queries_path: "/Users/zhuqi/arrow-datafusion/benchmarks/queries/clickbench/queries/sorted_data", output_path: Some("/Users/zhuqi/arrow-datafusion/benchmarks/results/issue_18976/data_sorted_clickbench.json"), sorted_by: Some("EventTime"), sort_order: "ASC" } âš ī¸ Forcing target_partitions=1 to preserve sort order âš ī¸ (Because we want to get the pure performance benefit of sorted data to compare) 📊 Session config target_partitions: 1 Registering table with sort order: EventTime ASC Executing: CREATE EXTERNAL TABLE hits STORED AS PARQUET LOCATION '/Users/zhuqi/arrow-datafusion/benchmarks/data/hits_0_sorted.parquet' WITH ORDER ("EventTime" ASC) Q0: -- Must set for ClickBench hits_partitioned dataset. See https://github.com/apache/datafusion/issues/16591 -- set datafusion.execution.parquet.binary_as_string = true SELECT * FROM hits ORDER BY "EventTime" DESC limit 10; Query 0 iteration 0 took 331.1 ms and returned 10 rows Query 0 iteration 1 took 286.0 ms and returned 10 rows Query 0 iteration 2 took 283.3 ms and returned 10 rows Query 0 iteration 3 took 283.8 ms and returned 10 rows Query 0 iteration 4 took 286.5 ms and returned 10 rows Query 0 avg time: 294.13 ms + set +x Done ``` ## Are there any user-facing changes? --------- Co-authored-by: Martin Grigorov Co-authored-by: Yongting You <2010youy01@gmail.com> Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> Co-authored-by: Andrew Lamb (cherry picked from commit cde6dfab1c5cc9c6832db9aa383df7ec85e046b0) --- benchmarks/README.md | 38 ++++++ benchmarks/bench.sh | 118 +++++++++++++++++- .../clickbench/queries/sorted_data/q0.sql | 3 + benchmarks/src/clickbench.rs | 115 +++++++++++++++-- 4 files changed, 264 insertions(+), 10 deletions(-) create mode 100644 benchmarks/queries/clickbench/queries/sorted_data/q0.sql diff --git a/benchmarks/README.md b/benchmarks/README.md index 8fed85fa02b80..aaf2261074639 100644 --- a/benchmarks/README.md +++ b/benchmarks/README.md @@ -804,3 +804,41 @@ Getting results... cancelling thread done dropping runtime in 83.531417ms ``` + +## Sorted Data Benchmarks + +### Data Sorted ClickBench + +Benchmark for queries on pre-sorted data to test sort order optimization. +This benchmark uses a subset of the ClickBench dataset (hits.parquet, ~14GB) that has been pre-sorted by the EventTime column. The queries are designed to test DataFusion's performance when the data is already sorted as is common in timeseries workloads. + +The benchmark includes queries that: +- Scan pre-sorted data with ORDER BY clauses that match the sort order +- Test reverse scans on sorted data +- Verify the performance result + +#### Generating Sorted Data + +The sorted dataset is automatically generated from the ClickBench partitioned dataset. You can configure the memory used during the sorting process with the `DATAFUSION_MEMORY_GB` environment variable. The default memory limit is 12GB. +```bash +./bench.sh data data_sorted_clickbench +``` + +To create the sorted dataset, for example with 16GB of memory, run: + +```bash +DATAFUSION_MEMORY_GB=16 ./bench.sh data data_sorted_clickbench +``` + +This command will: +1. Download the ClickBench partitioned dataset if not present +2. Sort hits.parquet by EventTime in ascending order +3. Save the sorted file as hits_sorted.parquet + +#### Running the Benchmark + +```bash +./bench.sh run data_sorted_clickbench +``` + +This runs queries against the pre-sorted dataset with the `--sorted-by EventTime` flag, which informs DataFusion that the data is pre-sorted, allowing it to optimize away redundant sort operations. diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index dbfd319dd9ad4..0ff631078e537 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -99,6 +99,9 @@ clickbench_partitioned: ClickBench queries against partitioned (100 files) parqu clickbench_pushdown: ClickBench queries against partitioned (100 files) parquet w/ filter_pushdown enabled clickbench_extended: ClickBench \"inspired\" queries against a single parquet (DataFusion specific) +# Sorted Data Benchmarks (ORDER BY Optimization) +clickbench_sorted: ClickBench queries on pre-sorted data using prefer_existing_sort (tests sort elimination optimization) + # H2O.ai Benchmarks (Group By, Join, Window) h2o_small: h2oai benchmark with small dataset (1e7 rows) for groupby, default file format is csv h2o_medium: h2oai benchmark with medium dataset (1e8 rows) for groupby, default file format is csv @@ -314,6 +317,9 @@ main() { compile_profile) data_tpch "1" ;; + clickbench_sorted) + clickbench_sorted + ;; *) echo "Error: unknown benchmark '$BENCHMARK' for data generation" usage @@ -445,7 +451,7 @@ main() { h2o_medium_window) run_h2o_window "MEDIUM" "CSV" "window" ;; - h2o_big_window) + h2o_big_window) run_h2o_window "BIG" "CSV" "window" ;; h2o_small_parquet) @@ -497,6 +503,9 @@ main() { compile_profile) run_compile_profile "${PROFILE_ARGS[@]}" ;; + clickbench_sorted) + run_clickbench_sorted + ;; *) echo "Error: unknown benchmark '$BENCHMARK' for run" usage @@ -1189,6 +1198,113 @@ compare_benchmarks() { } +# Creates sorted ClickBench data from hits.parquet (full dataset) +# The data is sorted by EventTime in ascending order +# Uses datafusion-cli to reduce dependencies +clickbench_sorted() { + SORTED_FILE="${DATA_DIR}/hits_sorted.parquet" + ORIGINAL_FILE="${DATA_DIR}/hits.parquet" + + # Default memory limit is 12GB, can be overridden with DATAFUSION_MEMORY_GB env var + MEMORY_LIMIT_GB=${DATAFUSION_MEMORY_GB:-12} + + echo "Creating sorted ClickBench dataset from hits.parquet..." + echo "Configuration:" + echo " Memory limit: ${MEMORY_LIMIT_GB}G" + echo " Row group size: 64K rows" + echo " Compression: uncompressed" + + if [ ! -f "${ORIGINAL_FILE}" ]; then + echo "hits.parquet not found. Running data_clickbench_1 first..." + data_clickbench_1 + fi + + if [ -f "${SORTED_FILE}" ]; then + echo "Sorted hits.parquet already exists at ${SORTED_FILE}" + return 0 + fi + + echo "Sorting hits.parquet by EventTime (this may take several minutes)..." + + pushd "${DATAFUSION_DIR}" > /dev/null + echo "Building datafusion-cli..." + cargo build --release --bin datafusion-cli + DATAFUSION_CLI="${DATAFUSION_DIR}/target/release/datafusion-cli" + popd > /dev/null + + + START_TIME=$(date +%s) + echo "Start time: $(date '+%Y-%m-%d %H:%M:%S')" + echo "Using datafusion-cli to create sorted parquet file..." + "${DATAFUSION_CLI}" << EOF +-- Memory and performance configuration +SET datafusion.runtime.memory_limit = '${MEMORY_LIMIT_GB}G'; +SET datafusion.execution.spill_compression = 'uncompressed'; +SET datafusion.execution.sort_spill_reservation_bytes = 10485760; -- 10MB +SET datafusion.execution.batch_size = 8192; +SET datafusion.execution.target_partitions = 1; + +-- Parquet output configuration +SET datafusion.execution.parquet.max_row_group_size = 65536; +SET datafusion.execution.parquet.compression = 'uncompressed'; + +-- Execute sort and write +COPY (SELECT * FROM '${ORIGINAL_FILE}' ORDER BY "EventTime") +TO '${SORTED_FILE}' +STORED AS PARQUET; +EOF + + local result=$? + + END_TIME=$(date +%s) + DURATION=$((END_TIME - START_TIME)) + echo "End time: $(date '+%Y-%m-%d %H:%M:%S')" + + if [ $result -eq 0 ]; then + echo "✓ Successfully created sorted ClickBench dataset" + + INPUT_SIZE=$(stat -f%z "${ORIGINAL_FILE}" 2>/dev/null || stat -c%s "${ORIGINAL_FILE}" 2>/dev/null) + OUTPUT_SIZE=$(stat -f%z "${SORTED_FILE}" 2>/dev/null || stat -c%s "${SORTED_FILE}" 2>/dev/null) + INPUT_MB=$((INPUT_SIZE / 1024 / 1024)) + OUTPUT_MB=$((OUTPUT_SIZE / 1024 / 1024)) + + echo " Input: ${INPUT_MB} MB" + echo " Output: ${OUTPUT_MB} MB" + + echo "" + echo "Time Statistics:" + echo " Total duration: ${DURATION} seconds ($(printf '%02d:%02d:%02d' $((DURATION/3600)) $((DURATION%3600/60)) $((DURATION%60))))" + echo " Throughput: $((INPUT_MB / DURATION)) MB/s" + + return 0 + else + echo "✗ Error: Failed to create sorted dataset" + echo "💡 Tip: Try increasing memory with: DATAFUSION_MEMORY_GB=16 ./bench.sh data clickbench_sorted" + return 1 + fi +} + +# Runs the sorted data benchmark with prefer_existing_sort configuration +run_clickbench_sorted() { + RESULTS_FILE="${RESULTS_DIR}/clickbench_sorted.json" + echo "RESULTS_FILE: ${RESULTS_FILE}" + echo "Running sorted data benchmark with prefer_existing_sort optimization..." + + # Ensure sorted data exists + clickbench_sorted + + # Run benchmark with prefer_existing_sort configuration + # This allows DataFusion to optimize away redundant sorts while maintaining parallelism + debug_run $CARGO_COMMAND --bin dfbench -- clickbench \ + --iterations 5 \ + --path "${DATA_DIR}/hits_sorted.parquet" \ + --queries-path "${SCRIPT_DIR}/queries/clickbench/queries/sorted_data" \ + --sorted-by "EventTime" \ + -c datafusion.optimizer.prefer_existing_sort=true \ + -o "${RESULTS_FILE}" \ + ${QUERY_ARG} +} + setup_venv() { python3 -m venv "$VIRTUAL_ENV" PATH=$VIRTUAL_ENV/bin:$PATH python3 -m pip install -r requirements.txt diff --git a/benchmarks/queries/clickbench/queries/sorted_data/q0.sql b/benchmarks/queries/clickbench/queries/sorted_data/q0.sql new file mode 100644 index 0000000000000..1170a383bcb22 --- /dev/null +++ b/benchmarks/queries/clickbench/queries/sorted_data/q0.sql @@ -0,0 +1,3 @@ +-- Must set for ClickBench hits_partitioned dataset. See https://github.com/apache/datafusion/issues/16591 +-- set datafusion.execution.parquet.binary_as_string = true +SELECT * FROM hits ORDER BY "EventTime" DESC limit 10; diff --git a/benchmarks/src/clickbench.rs b/benchmarks/src/clickbench.rs index a550503390c54..796b6a4dec525 100644 --- a/benchmarks/src/clickbench.rs +++ b/benchmarks/src/clickbench.rs @@ -78,6 +78,27 @@ pub struct RunOpt { /// If present, write results json here #[structopt(parse(from_os_str), short = "o", long = "output")] output_path: Option, + + /// Column name that the data is sorted by (e.g., "EventTime") + /// If specified, DataFusion will be informed that the data has this sort order + /// using CREATE EXTERNAL TABLE with WITH ORDER clause. + /// + /// Recommended to use with: -c datafusion.optimizer.prefer_existing_sort=true + /// This allows DataFusion to optimize away redundant sorts while maintaining + /// multi-core parallelism for other operations. + #[structopt(long = "sorted-by")] + sorted_by: Option, + + /// Sort order: ASC or DESC (default: ASC) + #[structopt(long = "sort-order", default_value = "ASC")] + sort_order: String, + + /// Configuration options in the format key=value + /// Can be specified multiple times. + /// + /// Example: -c datafusion.optimizer.prefer_existing_sort=true + #[structopt(short = "c", long = "config")] + config_options: Vec, } /// Get the SQL file path @@ -125,6 +146,37 @@ impl RunOpt { // configure parquet options let mut config = self.common.config()?; + + if self.sorted_by.is_some() { + println!("â„šī¸ Data is registered with sort order"); + + let has_prefer_sort = self + .config_options + .iter() + .any(|opt| opt.contains("prefer_existing_sort=true")); + + if !has_prefer_sort { + println!("â„šī¸ Consider using -c datafusion.optimizer.prefer_existing_sort=true"); + println!("â„šī¸ to optimize queries while maintaining parallelism"); + } + } + + // Apply user-provided configuration options + for config_opt in &self.config_options { + let parts: Vec<&str> = config_opt.splitn(2, '=').collect(); + if parts.len() != 2 { + return Err(exec_datafusion_err!( + "Invalid config option format: '{}'. Expected 'key=value'", + config_opt + )); + } + let key = parts[0]; + let value = parts[1]; + + println!("Setting config: {key} = {value}"); + config = config.set_str(key, value); + } + { let parquet_options = &mut config.options_mut().execution.parquet; // The hits_partitioned dataset specifies string columns @@ -136,10 +188,18 @@ impl RunOpt { parquet_options.pushdown_filters = true; parquet_options.reorder_filters = true; } + + if self.sorted_by.is_some() { + // We should compare the dynamic topk optimization when data is sorted, so we make the + // assumption that filter pushdown is also enabled in this case. + parquet_options.pushdown_filters = true; + parquet_options.reorder_filters = true; + } } let rt_builder = self.common.runtime_env_builder()?; let ctx = SessionContext::new_with_config_rt(config, rt_builder.build_arc()?); + self.register_hits(&ctx).await?; let mut benchmark_run = BenchmarkRun::new(); @@ -214,17 +274,54 @@ impl RunOpt { } /// Registers the `hits.parquet` as a table named `hits` + /// If sorted_by is specified, uses CREATE EXTERNAL TABLE with WITH ORDER async fn register_hits(&self, ctx: &SessionContext) -> Result<()> { - let options = Default::default(); let path = self.path.as_os_str().to_str().unwrap(); - ctx.register_parquet("hits", path, options) - .await - .map_err(|e| { - DataFusionError::Context( - format!("Registering 'hits' as {path}"), - Box::new(e), - ) - }) + + // If sorted_by is specified, use CREATE EXTERNAL TABLE with WITH ORDER + if let Some(ref sort_column) = self.sorted_by { + println!( + "Registering table with sort order: {} {}", + sort_column, self.sort_order + ); + + // Escape column name with double quotes + let escaped_column = if sort_column.contains('"') { + sort_column.clone() + } else { + format!("\"{sort_column}\"") + }; + + // Build CREATE EXTERNAL TABLE DDL with WITH ORDER clause + // Schema will be automatically inferred from the Parquet file + let create_table_sql = format!( + "CREATE EXTERNAL TABLE hits \ + STORED AS PARQUET \ + LOCATION '{}' \ + WITH ORDER ({} {})", + path, + escaped_column, + self.sort_order.to_uppercase() + ); + + println!("Executing: {create_table_sql}"); + + // Execute the CREATE EXTERNAL TABLE statement + ctx.sql(&create_table_sql).await?.collect().await?; + + Ok(()) + } else { + // Original registration without sort order + let options = Default::default(); + ctx.register_parquet("hits", path, options) + .await + .map_err(|e| { + DataFusionError::Context( + format!("Registering 'hits' as {path}"), + Box::new(e), + ) + }) + } } fn iterations(&self) -> usize { From 36682cb44191e291e6dfe4c7eb51dc13fd7ca932 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Wed, 7 Jan 2026 14:56:38 +0800 Subject: [PATCH 09/10] fix --- datafusion/core/tests/physical_optimizer/pushdown_sort.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/pushdown_sort.rs b/datafusion/core/tests/physical_optimizer/pushdown_sort.rs index fca3006c3f8d7..58730c6e09a30 100644 --- a/datafusion/core/tests/physical_optimizer/pushdown_sort.rs +++ b/datafusion/core/tests/physical_optimizer/pushdown_sort.rs @@ -244,7 +244,7 @@ fn test_prefix_match_through_transparent_nodes() { OptimizationTest: input: - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] - - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1, maintains_sort_order=true + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - CoalesceBatchesExec: target_batch_size=1024 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST, b@1 ASC, c@2 DESC NULLS LAST], file_type=parquet output: @@ -369,7 +369,7 @@ fn test_sort_through_repartition() { OptimizationTest: input: - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] - - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1, maintains_sort_order=true + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet output: Ok: @@ -463,7 +463,7 @@ fn test_sort_through_coalesce_partitions() { input: - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] - CoalescePartitionsExec - - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1, maintains_sort_order=true + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet output: Ok: @@ -496,7 +496,7 @@ fn test_complex_plan_with_multiple_operators() { input: - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] - CoalescePartitionsExec - - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1, maintains_sort_order=true + - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - CoalesceBatchesExec: target_batch_size=1024 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet output: From f5f96f7f0e2b3a6e0e71fb10dcdbcc81a227fe4e Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Wed, 7 Jan 2026 16:08:47 +0800 Subject: [PATCH 10/10] fix --- Cargo.lock | 1 - datafusion/physical-optimizer/Cargo.toml | 1 - 2 files changed, 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ab3548fa06f50..d712eecfcc72e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2533,7 +2533,6 @@ dependencies = [ "datafusion-pruning", "insta", "itertools 0.14.0", - "log", "recursive", "tokio", ] diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index 7f63a619a14e3..395da10d629ba 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -51,7 +51,6 @@ datafusion-physical-expr-common = { workspace = true } datafusion-physical-plan = { workspace = true } datafusion-pruning = { workspace = true } itertools = { workspace = true } -log = "0.4.28" recursive = { workspace = true, optional = true } [dev-dependencies]