diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 39cf7a9855de..2a5fd05594d1 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -837,6 +837,18 @@ config_namespace! { /// writing out already in-memory data, such as from a cached /// data frame. pub maximum_buffered_record_batches_per_stream: usize, default = 2 + + /// Enable sort pushdown optimization for Parquet files. + /// When enabled, optimizes queries with ORDER BY: + /// - Reordering files based on statistics + /// - Reversing row group read order when beneficial + /// Returns **inexact ordering**: Sort operator is kept for correctness, + /// but can terminate early for TopK queries (ORDER BY ... LIMIT N), + /// providing huge speedup. + /// Memory: No additional overhead (only changes read order). + /// Future TODO: Will add option to support detecting perfectly sorted data and eliminate Sort completely. + /// Default: true + pub enable_sort_pushdown: bool, default = true } } diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 8aa0134d09ec..e21b6e60e7b4 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -209,6 +209,7 @@ impl ParquetOptions { coerce_int96: _, // not used for writer props skip_arrow_metadata: _, max_predicate_cache_size: _, + enable_sort_pushdown: _, } = self; let mut builder = WriterProperties::builder() @@ -474,6 +475,7 @@ mod tests { skip_arrow_metadata: defaults.skip_arrow_metadata, coerce_int96: None, max_predicate_cache_size: defaults.max_predicate_cache_size, + enable_sort_pushdown: true, } } @@ -588,6 +590,7 @@ mod tests { binary_as_string: global_options_defaults.binary_as_string, skip_arrow_metadata: global_options_defaults.skip_arrow_metadata, coerce_int96: None, + enable_sort_pushdown: true, }, column_specific_options, key_value_metadata, diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index fe9db1975d27..d11322cd26be 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -32,6 +32,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; #[expect(clippy::needless_pass_by_value)] 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 000000000000..18bd9b80b2c4 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/pushdown_sort.rs @@ -0,0 +1,778 @@ +// 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_scan_inexact=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 arrow::compute::SortOptions; +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, sort_expr_options, 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 source_ordering = LexOrdering::new(vec![sort_expr("a", &schema)]).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![sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]) + .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], output_ordering=[a@0 ASC], file_type=parquet, reverse_scan_inexact=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 source_ordering = LexOrdering::new(vec![sort_expr("a", &schema)]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Request DESC ordering with limit + let desc_ordering = LexOrdering::new(vec![sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]) + .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], output_ordering=[a@0 ASC], file_type=parquet, reverse_scan_inexact=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 source_ordering = LexOrdering::new(vec![ + sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + sort_expr("b", &schema), + ]) + .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![ + sort_expr_options( + "a", + &schema, + SortOptions { + descending: false, + nulls_first: true, + }, + ), + sort_expr_options( + "b", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + ]) + .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], output_ordering=[a@0 DESC NULLS LAST, b@1 ASC], file_type=parquet, reverse_scan_inexact=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 source_ordering = LexOrdering::new(vec![ + sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + sort_expr("b", &schema), + ]) + .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![sort_expr_options( + "a", + &schema, + SortOptions { + descending: false, + nulls_first: true, + }, + )]) + .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], output_ordering=[a@0 DESC NULLS LAST, b@1 ASC], file_type=parquet, reverse_scan_inexact=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 source_ordering = LexOrdering::new(vec![ + sort_expr("a", &schema), + sort_expr_options( + "b", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + sort_expr("c", &schema), + ]) + .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![ + sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + sort_expr_options( + "b", + &schema, + SortOptions { + descending: false, + nulls_first: true, + }, + ), + ]) + .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], output_ordering=[a@0 ASC, b@1 DESC NULLS LAST, c@2 ASC], file_type=parquet, reverse_scan_inexact=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 source_ordering = LexOrdering::new(vec![ + sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + sort_expr("b", &schema), + sort_expr_options( + "c", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + ]) + .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![sort_expr_options( + "a", + &schema, + SortOptions { + descending: false, + nulls_first: true, + }, + )]) + .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, 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: + Ok: + - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + - 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, reverse_scan_inexact=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 source_ordering = LexOrdering::new(vec![ + sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + sort_expr("b", &schema), + ]) + .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![sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]) + .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 source_ordering = LexOrdering::new(vec![sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]) + .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![ + sort_expr_options( + "a", + &schema, + SortOptions { + descending: false, + nulls_first: true, + }, + ), + sort_expr_options( + "b", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + ]) + .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 source_ordering = LexOrdering::new(vec![sort_expr("a", &schema)]).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![sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]) + .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], output_ordering=[a@0 ASC], file_type=parquet, reverse_scan_inexact=true + " + ); +} + +#[test] +fn test_sort_through_repartition() { + // Sort should push through RepartitionExec + let schema = schema(); + let source_ordering = LexOrdering::new(vec![sort_expr("a", &schema)]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + let repartition = repartition_exec(source); + + let desc_ordering = LexOrdering::new(vec![sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]) + .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, 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: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - 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, reverse_scan_inexact=true + " + ); +} + +#[test] +fn test_nested_sorts() { + // Nested sort operations - only innermost can be optimized + let schema = schema(); + let source_ordering = LexOrdering::new(vec![sort_expr("a", &schema)]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + let desc_ordering = LexOrdering::new(vec![sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]) + .unwrap(); + let inner_sort = sort_exec(desc_ordering, source); + + let sort_exprs2 = LexOrdering::new(vec![sort_expr("b", &schema)]).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], output_ordering=[a@0 ASC], file_type=parquet, reverse_scan_inexact=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 source_ordering = LexOrdering::new(vec![sort_expr("a", &schema)]).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![sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]) + .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, 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: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - CoalescePartitionsExec + - 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, reverse_scan_inexact=true + "### + ); +} + +#[test] +fn test_complex_plan_with_multiple_operators() { + // Test a complex plan with multiple operators between sort and source + let schema = schema(); + let source_ordering = LexOrdering::new(vec![sort_expr("a", &schema)]).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![sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]) + .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, 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: + Ok: + - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] + - CoalescePartitionsExec + - 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, reverse_scan_inexact=true + "### + ); +} + +#[test] +fn test_multiple_sorts_different_columns() { + // Test nested sorts on different columns - only innermost can optimize + let schema = schema(); + let source_ordering = LexOrdering::new(vec![sort_expr("a", &schema)]).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![sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]) + .unwrap(); + let sort1 = sort_exec(desc_ordering, source); + + // Then sort by column 'c' (different column, can't optimize) + let sort_exprs2 = LexOrdering::new(vec![sort_expr("c", &schema)]).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], output_ordering=[a@0 ASC], file_type=parquet, reverse_scan_inexact=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 source_ordering = LexOrdering::new(vec![sort_expr("a", &schema)]).unwrap(); + let source = parquet_exec_with_sort(schema.clone(), vec![source_ordering]); + + // Request sort by 'b' (different column) + let sort_exprs = LexOrdering::new(vec![sort_expr("b", &schema)]).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 + "### + ); +} diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index e164e7617cf7..aaf04c5840a7 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; @@ -32,7 +32,9 @@ use datafusion::datasource::source::DataSourceExec; 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::config::SessionConfig; use datafusion_execution::object_store::ObjectStoreUrl; @@ -699,3 +701,80 @@ 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 session_config = SessionConfig::new(); + session_config + .options_mut() + .execution + .parquet + .enable_sort_pushdown = enable_sort_pushdown; + let optimizer_context = OptimizerContext::new(session_config.clone()); + let output_result = opt.optimize_plan(input_plan, &optimizer_context); + 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/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 83235dafdaf8..63d9b37eb74b 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -115,6 +115,7 @@ 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, + pub reverse_scan_inexact: bool, } impl FileOpener for ParquetOpener { @@ -168,6 +169,7 @@ impl FileOpener for ParquetOpener { let encryption_context = self.get_encryption_context(); let max_predicate_cache_size = self.max_predicate_cache_size; + let reverse_scan_inexact = self.reverse_scan_inexact; Ok(Box::pin(async move { #[cfg(feature = "parquet_encryption")] let file_decryption_properties = encryption_context @@ -434,10 +436,90 @@ 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); + + // Extract row selection before potentially reversing + let row_selection_opt = + access_plan.into_overall_row_selection(rg_metadata)?; + + if reverse_scan_inexact { + // Reverse the row groups + let reversed_indexes: Vec<_> = + row_group_indexes.clone().into_iter().rev().collect(); + + // If we have a row selection, we need to rebuild it for the reversed order + if let Some(row_selection) = row_selection_opt { + // Build a mapping of row group index to its row range in the file + let mut rg_row_ranges: Vec<(usize, usize, usize)> = Vec::new(); // (rg_index, start_row, end_row) + let mut current_row = 0; + for (rg_idx, rg_meta) in rg_metadata.iter().enumerate() { + let num_rows = rg_meta.num_rows() as usize; + rg_row_ranges.push((rg_idx, current_row, current_row + num_rows)); + current_row += num_rows; + } + + // Extract which rows are selected for each row group from the overall selection + use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; + + let mut rg_selections: std::collections::HashMap< + usize, + Vec, + > = std::collections::HashMap::new(); + + // Parse the overall row selection to determine which rows in each row group are selected + 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; + } + + // Now rebuild the overall selection in reversed row group order + let mut reversed_selectors = Vec::new(); + for &rg_idx in reversed_indexes.iter() { + 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)); + } + } + } + + if !reversed_selectors.is_empty() { + builder = builder + .with_row_selection(RowSelection::from(reversed_selectors)); + } + } + + builder = builder.with_row_groups(reversed_indexes); + } else { + // Normal forward scan + if let Some(row_selection) = row_selection_opt { + builder = builder.with_row_selection(row_selection); + } + builder = builder.with_row_groups(row_group_indexes); } if let Some(limit) = limit { @@ -454,7 +536,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()?; @@ -765,6 +846,7 @@ fn should_enable_page_index( mod test { use std::sync::Arc; + use arrow::array::Array; use arrow::{ compute::cast, datatypes::{DataType, Field, Schema, SchemaRef}, @@ -908,6 +990,7 @@ mod test { #[cfg(feature = "parquet_encryption")] encryption_factory: None, max_predicate_cache_size: None, + reverse_scan_inexact: false, } }; @@ -982,6 +1065,7 @@ mod test { #[cfg(feature = "parquet_encryption")] encryption_factory: None, max_predicate_cache_size: None, + reverse_scan_inexact: false, } }; @@ -1072,6 +1156,7 @@ mod test { #[cfg(feature = "parquet_encryption")] encryption_factory: None, max_predicate_cache_size: None, + reverse_scan_inexact: false, } }; @@ -1165,6 +1250,7 @@ mod test { #[cfg(feature = "parquet_encryption")] encryption_factory: None, max_predicate_cache_size: None, + reverse_scan_inexact: false, } }; @@ -1258,6 +1344,7 @@ mod test { #[cfg(feature = "parquet_encryption")] encryption_factory: None, max_predicate_cache_size: None, + reverse_scan_inexact: false, } }; @@ -1409,6 +1496,7 @@ mod test { #[cfg(feature = "parquet_encryption")] encryption_factory: None, max_predicate_cache_size: None, + reverse_scan_inexact: false, }; let predicate = logical2physical(&col("a").eq(lit(1u64)), &table_schema); @@ -1429,4 +1517,335 @@ 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 mut out = BytesMut::new().writer(); + { + let mut writer = + ArrowWriter::try_new(&mut out, batch1.schema(), Some(props)).unwrap(); + writer.write(&batch1).unwrap(); + writer.write(&batch2).unwrap(); + writer.write(&batch3).unwrap(); + writer.finish().unwrap(); + } + let data = out.into_inner().freeze(); + let data_len = data.len(); + store + .put(&Path::from("test.parquet"), data.into()) + .await + .unwrap(); + + 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 { + partition_index: 0, + projection: Arc::new([0]), + batch_size: 1024, + limit: None, + predicate: None, + logical_file_schema: schema.clone(), + metadata_size_hint: None, + metrics: ExecutionPlanMetricsSet::new(), + parquet_file_reader_factory: Arc::new(DefaultParquetFileReaderFactory::new( + Arc::clone(&store), + )), + partition_fields: vec![], + pushdown_filters: false, + reorder_filters: false, + force_filter_selections: false, + enable_page_index: false, + enable_bloom_filter: 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: None, + #[cfg(feature = "parquet_encryption")] + encryption_factory: None, + max_predicate_cache_size: None, + reverse_scan_inexact: reverse_scan, + }; + + // Test normal scan (forward) + let opener = make_opener(false); + let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let batches = collect_batches(stream).await; + + // Collect all values in order + let mut forward_values = vec![]; + for batch in &batches { + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + forward_values.push(array.value(i)); + } + } + } + + // Test reverse scan + let opener = make_opener(true); + let stream = opener.open(file.clone()).unwrap().await.unwrap(); + let batches = collect_batches(stream).await; + + // Collect all values in order + let mut reverse_values = vec![]; + for batch in &batches { + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + reverse_values.push(array.value(i)); + } + } + } + + // 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_single_row_group() { + let store = Arc::new(InMemory::new()) as Arc; + + // Create a single batch (single row group) + let batch = record_batch!(("a", Int32, vec![Some(1), Some(2), Some(3)])).unwrap(); + let data_size = + write_parquet(Arc::clone(&store), "test.parquet", batch.clone()).await; + + let schema = batch.schema(); + let file = PartitionedFile::new( + "test.parquet".to_string(), + u64::try_from(data_size).unwrap(), + ); + + let make_opener = |reverse_scan: bool| ParquetOpener { + partition_index: 0, + projection: Arc::new([0]), + batch_size: 1024, + limit: None, + predicate: None, + logical_file_schema: schema.clone(), + metadata_size_hint: None, + metrics: ExecutionPlanMetricsSet::new(), + parquet_file_reader_factory: Arc::new(DefaultParquetFileReaderFactory::new( + Arc::clone(&store), + )), + partition_fields: vec![], + pushdown_filters: false, + reorder_filters: false, + force_filter_selections: false, + enable_page_index: false, + enable_bloom_filter: 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: None, + #[cfg(feature = "parquet_encryption")] + encryption_factory: None, + max_predicate_cache_size: None, + reverse_scan_inexact: reverse_scan, + }; + + // With a single row group, forward and reverse should be the same + // (only the row group order is reversed, not the rows within) + let opener_forward = make_opener(false); + let stream_forward = opener_forward.open(file.clone()).unwrap().await.unwrap(); + let batches_forward = collect_batches(stream_forward).await; + + let opener_reverse = make_opener(true); + let stream_reverse = opener_reverse.open(file).unwrap().await.unwrap(); + let batches_reverse = collect_batches(stream_reverse).await; + + // Both should have the same data since there's only one row group + assert_eq!(batches_forward.len(), batches_reverse.len()); + for (b1, b2) in batches_forward.iter().zip(batches_reverse.iter()) { + assert_eq!(b1.num_rows(), b2.num_rows()); + } + } + + #[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 mut out = BytesMut::new().writer(); + { + let mut writer = + ArrowWriter::try_new(&mut out, batch1.schema(), Some(props)).unwrap(); + writer.write(&batch1).unwrap(); + writer.write(&batch2).unwrap(); + writer.write(&batch3).unwrap(); + writer.finish().unwrap(); + } + let data = out.into_inner().freeze(); + let data_len = data.len(); + store + .put(&Path::from("test.parquet"), data.into()) + .await + .unwrap(); + + 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 { + partition_index: 0, + projection: Arc::new([0]), + batch_size: 1024, + limit: None, + predicate: None, + logical_file_schema: schema.clone(), + metadata_size_hint: None, + metrics: ExecutionPlanMetricsSet::new(), + parquet_file_reader_factory: Arc::new(DefaultParquetFileReaderFactory::new( + Arc::clone(&store), + )), + partition_fields: vec![], + pushdown_filters: false, + reorder_filters: false, + force_filter_selections: false, + enable_page_index: false, + enable_bloom_filter: 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: None, + #[cfg(feature = "parquet_encryption")] + encryption_factory: None, + max_predicate_cache_size: None, + reverse_scan_inexact: reverse_scan, + }; + + // 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 batches = collect_batches(stream).await; + + let mut forward_values = vec![]; + for batch in &batches { + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + forward_values.push(array.value(i)); + } + } + } + + // 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 batches = collect_batches(stream).await; + + let mut reverse_values = vec![]; + for batch in &batches { + let array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + reverse_values.push(array.value(i)); + } + } + } + + // 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/source.rs b/datafusion/datasource-parquet/src/source.rs index d84ddf599379..9ce9433194df 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -39,7 +39,7 @@ use datafusion_datasource::schema_adapter::{ use arrow::datatypes::TimeUnit; use datafusion_common::config::TableParquetOptions; use datafusion_common::DataFusionError; -use datafusion_datasource::file::FileSource; +use datafusion_datasource::file::{FileSource, SortOrderPushdownResult}; use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::TableSchema; use datafusion_physical_expr::conjunction; @@ -57,6 +57,7 @@ use datafusion_physical_plan::DisplayFormatType; #[cfg(feature = "parquet_encryption")] use datafusion_execution::parquet_encryption::EncryptionFactory; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use itertools::Itertools; use object_store::ObjectStore; #[cfg(feature = "parquet_encryption")] @@ -292,6 +293,11 @@ pub struct ParquetSource { pub(crate) projection: SplitProjection, #[cfg(feature = "parquet_encryption")] pub(crate) encryption_factory: 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_scan_inexact: bool, } impl ParquetSource { @@ -314,6 +320,7 @@ impl ParquetSource { metadata_size_hint: None, #[cfg(feature = "parquet_encryption")] encryption_factory: None, + reverse_scan_inexact: false, } } @@ -490,6 +497,15 @@ impl ParquetSource { )), } } + + pub fn with_reverse_scan_inexact(mut self, reverse_scan_inexact: bool) -> Self { + self.reverse_scan_inexact = reverse_scan_inexact; + self + } + + pub fn reverse_scan_inexact(&self) -> bool { + self.reverse_scan_inexact + } } /// Parses datafusion.common.config.ParquetOptions.coerce_int96 String to a arrow_schema.datatype.TimeUnit @@ -612,6 +628,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_scan_inexact: self.reverse_scan_inexact, }) as Arc; opener = ProjectionOpener::try_new( split_projection.clone(), @@ -673,6 +690,11 @@ impl FileSource for ParquetSource { write!(f, "{predicate_string}")?; + // Add reverse_scan info if enabled + if self.reverse_scan_inexact { + write!(f, ", reverse_scan_inexact=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. @@ -708,6 +730,12 @@ impl FileSource for ParquetSource { if let Some(predicate) = self.filter() { writeln!(f, "predicate={}", fmt_sql(predicate.as_ref()))?; } + + // Add reverse_scan info if enabled + if self.reverse_scan_inexact { + writeln!(f, "reverse_scan_inexact=true")?; + } + Ok(()) } } @@ -794,6 +822,34 @@ impl FileSource for ParquetSource { fn schema_adapter_factory(&self) -> Option> { self.schema_adapter_factory.clone() } + + /// When push down to parquet source of a sort operation is possible, + /// create a new ParquetSource with reverse_scan enabled. + /// + /// # Phase 1 Behavior (Current) + /// Returns `Inexact` because we're only reversing the scan direction and reordering + /// files/row groups. We still need to verify ordering at a higher level. + /// + /// # Phase 2 (Future) + /// Could return `Exact` when we can guarantee that the scan order matches the requested order, and + /// we can remove any higher-level sort operations. + /// + /// TODO support more policies in addition to reversing the scan. + fn try_pushdown_sort( + &self, + _order: &[PhysicalSortExpr], + ) -> datafusion_common::Result>> { + // Note: We ignore the specific `order` parameter here because the decision + // about whether we can reverse is made at the FileScanConfig level. + // This method creates a reversed version of the current ParquetSource, + // and the FileScanConfig will reverse both the file list and the declared ordering. + let new_source = self.clone().with_reverse_scan_inexact(true); + + // Phase 1: Return Inexact + Ok(SortOrderPushdownResult::Inexact { + inner: Arc::new(new_source), + }) + } } #[cfg(test)] @@ -812,4 +868,87 @@ 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() { + use arrow::datatypes::Schema; + + let schema = Arc::new(Schema::empty()); + let source = ParquetSource::new(schema); + + assert!(!source.reverse_scan_inexact()); + } + + #[test] + fn test_reverse_scan_with_setter() { + use arrow::datatypes::Schema; + + let schema = Arc::new(Schema::empty()); + + let source = ParquetSource::new(schema.clone()).with_reverse_scan_inexact(true); + assert!(source.reverse_scan_inexact()); + + let source = source.with_reverse_scan_inexact(false); + assert!(!source.reverse_scan_inexact()); + } + + #[test] + fn test_reverse_scan_clone_preserves_value() { + use arrow::datatypes::Schema; + + let schema = Arc::new(Schema::empty()); + + let source = ParquetSource::new(schema).with_reverse_scan_inexact(true); + let cloned = source.clone(); + + assert!(cloned.reverse_scan_inexact()); + assert_eq!(source.reverse_scan_inexact(), cloned.reverse_scan_inexact()); + } + + #[test] + fn test_reverse_scan_with_other_options() { + use arrow::datatypes::Schema; + use datafusion_common::config::TableParquetOptions; + + let schema = Arc::new(Schema::empty()); + let options = TableParquetOptions::default(); + + let source = ParquetSource::new(schema) + .with_table_parquet_options(options) + .with_metadata_size_hint(8192) + .with_reverse_scan_inexact(true); + + assert!(source.reverse_scan_inexact()); + assert_eq!(source.metadata_size_hint, Some(8192)); + } + + #[test] + fn test_reverse_scan_builder_pattern() { + use arrow::datatypes::Schema; + + let schema = Arc::new(Schema::empty()); + + let source = ParquetSource::new(schema) + .with_reverse_scan_inexact(true) + .with_reverse_scan_inexact(false) + .with_reverse_scan_inexact(true); + + assert!(source.reverse_scan_inexact()); + } + + #[test] + fn test_reverse_scan_independent_of_predicate() { + use arrow::datatypes::Schema; + use datafusion_physical_expr::expressions::lit; + + let schema = Arc::new(Schema::empty()); + let predicate = lit(true); + + let source = ParquetSource::new(schema) + .with_predicate(predicate) + .with_reverse_scan_inexact(true); + + assert!(source.reverse_scan_inexact()); + assert!(source.filter().is_some()); + } } diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index 3668e0e4a77e..29e9e8c3e707 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -34,6 +34,7 @@ use datafusion_physical_plan::filter_pushdown::{FilterPushdownPropagation, Pushe use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion_physical_plan::DisplayFormatType; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use object_store::ObjectStore; /// Helper function to convert any type implementing FileSource to Arc<dyn FileSource> @@ -41,6 +42,43 @@ pub fn as_file_source(source: T) -> Arc Arc::new(source) } +/// Result of attempting to push down sort ordering to a file source +#[derive(Debug, Clone)] +pub enum SortOrderPushdownResult { + /// The source can guarantee exact ordering (data is perfectly sorted) + Exact { inner: T }, + /// The source has optimized for the ordering but cannot guarantee perfect sorting + /// (e.g., reordered files/row groups based on statistics) + Inexact { inner: T }, + /// The source cannot optimize for this ordering + Unsupported, +} + +impl SortOrderPushdownResult { + /// Returns true if the result is Exact + pub fn is_exact(&self) -> bool { + matches!(self, Self::Exact { .. }) + } + + /// Returns true if the result is Inexact + pub fn is_inexact(&self) -> bool { + matches!(self, Self::Inexact { .. }) + } + + /// Returns true if optimization was successful (Exact or Inexact) + pub fn is_supported(&self) -> bool { + !matches!(self, Self::Unsupported) + } + + /// 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, + } + } +} + /// file format specific behaviors for elements in [`DataSource`] /// /// See more details on specific implementations: @@ -129,6 +167,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_pushdown_sort( + &self, + _order: &[PhysicalSortExpr], + ) -> Result>> { + Ok(SortOrderPushdownResult::Unsupported) + } + /// Try to push down a projection into a this FileSource. /// /// `FileSource` implementations that support projection pushdown should diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 12654ee5b6af..f6ab4d58a2d8 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -37,25 +37,26 @@ use datafusion_execution::{ object_store::ObjectStoreUrl, SendableRecordBatchStream, TaskContext, }; use datafusion_expr::Operator; + +use crate::file::SortOrderPushdownResult; +use datafusion_physical_expr::equivalence::project_orderings; use datafusion_physical_expr::expressions::BinaryExpr; use datafusion_physical_expr::projection::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_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_plan::coop::cooperative; +use datafusion_physical_plan::execution_plan::SchedulingType; use datafusion_physical_plan::{ display::{display_orderings, ProjectSchemaDisplay}, filter_pushdown::FilterPushdownPropagation, metrics::ExecutionPlanMetricsSet, DisplayAs, DisplayFormatType, }; -use std::{any::Any, fmt::Debug, fmt::Formatter, fmt::Result as FmtResult, sync::Arc}; - -use datafusion_physical_expr::equivalence::project_orderings; -use datafusion_physical_plan::coop::cooperative; -use datafusion_physical_plan::execution_plan::SchedulingType; use log::{debug, warn}; +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. @@ -766,6 +767,107 @@ impl DataSource for FileScanConfig { } } } + + fn try_pushdown_sort( + &self, + order: &[PhysicalSortExpr], + ) -> Result>> { + let current_ordering = match self.output_ordering.first() { + Some(ordering) => ordering.as_ref(), + None => return Ok(None), + }; + + // Only support reverse ordering pushdown until now + if !is_reverse_ordering(order, current_ordering) { + return Ok(None); + } + + // Ask the file source if it can handle the sort pushdown + let pushdown_result = self.file_source.try_pushdown_sort(order)?; + + let new_file_source = match pushdown_result { + SortOrderPushdownResult::Exact { inner } + | SortOrderPushdownResult::Inexact { inner } => inner, + SortOrderPushdownResult::Unsupported => return Ok(None), + }; + + let mut new_config = self.clone(); + + // Reverse file groups: when scanning in reverse, we need to read files + // in reverse order to maintain the correct global ordering + new_config.file_groups = new_config + .file_groups + .into_iter() + .map(|group| { + let mut files = group.into_inner(); + files.reverse(); + files.into() + }) + .collect(); + + // Phase 1: DO NOT change output_ordering + // The ordering is still the same as before (e.g., ASC) because: + // 1. We're only reversing row groups, not rows within groups + // 2. This makes the scan "closer" to DESC but not guaranteed + // 3. The Sort operator above will still be needed + // + // Keep the original output_ordering unchanged + // new_config.output_ordering = ... (NO CHANGE) + + new_config.file_source = new_file_source; + + Ok(Some(Arc::new(new_config))) + } +} + +/// Check if the requested ordering can be satisfied by reversing the current ordering. +/// +/// This function supports **prefix matching**: if the file has ordering [A DESC, B ASC] +/// and we need [A ASC], reversing the scan gives us [A ASC, B DESC], which satisfies +/// the requirement since [A ASC] is a prefix. +/// +/// # Arguments +/// * `requested` - The ordering required by the query +/// * `current` - The natural ordering of the data source (e.g., from file metadata) +/// +/// # Returns +/// `true` if reversing the current ordering would satisfy the requested ordering +/// +/// # Example +/// ```text +/// Current: [number DESC, letter ASC] +/// Requested: [number ASC] +/// Reversed: [number ASC, letter DESC] ✓ Prefix match! +/// ``` +fn is_reverse_ordering( + requested: &[PhysicalSortExpr], + current: &[PhysicalSortExpr], +) -> bool { + // Allow prefix matching - we can satisfy a prefix of the current ordering + // by reversing the scan + if requested.len() > current.len() { + return false; + } + + requested.iter().zip(current.iter()).all(|(req, cur)| { + // Check if the expressions are semantically equivalent using PhysicalExpr::eq + // This is more robust than string comparison as it handles: + // - Expression equivalence (not just string representation) + // - Complex expressions that might have different string forms but same semantics + let exprs_match = req.expr.eq(&cur.expr); + + // Now check if the sort options are exactly reversed + // For a valid reverse scan: + // - descending must be opposite: ASC ↔ DESC + // - nulls_first must be opposite: NULLS FIRST ↔ NULLS LAST + let options_reversed = req.options.descending != cur.options.descending + && req.options.nulls_first != cur.options.nulls_first; + + // Both conditions must be true: + // 1. Expressions are semantically equivalent + // 2. Completely reversed sort options + exprs_match && options_reversed + }) } impl FileScanConfig { diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 0945ffc94c1d..9a4cb245cac0 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -40,7 +40,7 @@ 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, }; @@ -190,6 +190,24 @@ 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(Some(source))` - Created a source that satisfies the ordering + /// * `Ok(None)` - Cannot optimize for this ordering + /// * `Err(e)` - Error occurred + /// + /// Default implementation returns `Ok(None)`. + fn try_pushdown_sort( + &self, + _order: &[PhysicalSortExpr], + ) -> Result>> { + Ok(None) + } } /// [`ExecutionPlan`] that reads one or more files @@ -360,6 +378,19 @@ impl ExecutionPlan for DataSourceExec { }), } } + + fn try_pushdown_sort( + &self, + order: &[PhysicalSortExpr], + ) -> Result>> { + match self.data_source.try_pushdown_sort(order)? { + Some(new_data_source) => { + let new_exec = self.clone().with_data_source(new_data_source); + Ok(Some(Arc::new(new_exec))) + } + None => Ok(None), + } + } } impl DataSourceExec { diff --git a/datafusion/execution/src/config.rs b/datafusion/execution/src/config.rs index 2f78fb5532da..0b5ffab609a7 100644 --- a/datafusion/execution/src/config.rs +++ b/datafusion/execution/src/config.rs @@ -442,6 +442,12 @@ impl SessionConfig { self } + /// Enables or disables sort pushdown optimization for Parquet files + pub fn with_parquet_enable_sort_pushdown(mut self, enabled: bool) -> Self { + self.options_mut().execution.parquet.enable_sort_pushdown = enabled; + self + } + /// Enables or disables the collection of statistics after listing files pub fn with_collect_statistics(mut self, enabled: bool) -> Self { self.options_mut().execution.collect_statistics = enabled; diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index 07f61d3d0456..c3fc0cfefd2e 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 0f3467bfbbf8..22023b22fd47 100644 --- a/datafusion/physical-optimizer/src/optimizer.rs +++ b/datafusion/physical-optimizer/src/optimizer.rs @@ -37,6 +37,7 @@ use crate::topk_aggregation::TopKAggregation; use crate::update_aggr_exprs::OptimizeAggregateOrder; use crate::limit_pushdown_past_window::LimitPushPastWindows; +use crate::pushdown_sort::PushdownSort; use datafusion_common::config::ConfigOptions; use datafusion_common::{internal_err, Result}; use datafusion_execution::config::SessionConfig; @@ -212,6 +213,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 000000000000..0eb3eefc3b4f --- /dev/null +++ b/datafusion/physical-optimizer/src/pushdown_sort.rs @@ -0,0 +1,250 @@ +// 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 (Phase 1) +//! +//! Phase 1 supports reverse scan optimization: when the required sort order is +//! the reverse of the data source's output ordering (or a prefix of it), we perform +//! a reverse scan at the data source level (reading row groups in reverse order). +//! +//! **Prefix Matching**: If the data has ordering [A DESC, B ASC] and the query needs +//! [A ASC], reversing gives [A ASC, B DESC] which satisfies the requirement. +//! +//! This optimization: +//! 1. Detects SortExec nodes that require a specific ordering +//! 2. Recursively traverses through transparent nodes to find data sources +//! 3. Checks if required order is reverse of output order (supports prefix matching) +//! 4. If yes, pushes down reverse scan to data source +//! 5. Returns **Inexact** ordering (keeps Sort but enables early termination) +//! 6. Phase 2 will support more complex scenarios (file reordering) and detect perfect ordering + +use crate::{OptimizerContext, PhysicalOptimizerRule}; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_common::Result; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; +use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::ExecutionPlan; +use std::sync::Arc; + +/// A PhysicalOptimizerRule that attempts to push down sort requirements to data sources. +/// +/// # Phase 1 Behavior (Current) +/// +/// This optimization handles the case where the required sort order is the reverse +/// of the data source's output ordering (or a prefix of it): +/// - Detects when sort order is the reverse of natural output order +/// - **Supports prefix matching**: e.g., if data is [A DESC, B ASC] and query needs +/// [A ASC], reverse scan produces [A ASC, B DESC] which satisfies the requirement +/// - Pushes down reverse scan to data source (row groups read in reverse) +/// - Returns **Inexact** ordering (keeps Sort but enables early termination) +/// +/// Benefits: +/// - TopK queries with reverse order: huge faster due to early termination +/// - Memory: No additional overhead (only changes read order) +/// - Works with single column or multi-column reverse ordering +/// - Prefix matching allows partial sort pushdown for better performance +/// +/// # Phase 2 (Future) +/// +/// Will support more complex scenarios: +/// - File reordering based on statistics +/// - Partial ordering optimizations +/// - Detect when files are perfectly sorted and: +/// - Return **Exact** ordering guarantees +/// - Completely eliminate the Sort operator +/// +/// # Implementation +/// +/// 1. Detects SortExec nodes +/// 2. Recursively pushes through transparent nodes (CoalesceBatches, Repartition, etc.) +/// 3. Asks data sources to optimize via `try_pushdown_sort()` +/// 4. Data source checks if required order is reverse of natural order (with prefix matching) +/// 5. If yes, performs reverse scan; if no, returns None +/// 6. Keeps Sort operator (Phase 1 returns Inexact) +#[derive(Debug, Clone, Default)] +pub struct PushdownSort; + +impl PushdownSort { + pub fn new() -> Self { + Self {} + } +} + +impl PhysicalOptimizerRule for PushdownSort { + fn optimize_plan( + &self, + plan: Arc, + context: &OptimizerContext, + ) -> Result> { + // Check if sort pushdown optimization is enabled + let enable_sort_pushdown = context + .session_config() + .options() + .execution + .parquet + .enable_sort_pushdown; + + // Return early if not enabled + if !enable_sort_pushdown { + return Ok(plan); + } + + // Search for any SortExec nodes and try to optimize them + plan.transform_down(&|plan: Arc| { + // Check if this is a SortExec + let sort_exec = match plan.as_any().downcast_ref::() { + Some(sort_exec) => sort_exec, + None => return Ok(Transformed::no(plan)), + }; + + optimize_sort(sort_exec) + }) + .data() + } + + fn name(&self) -> &str { + "PushdownSort" + } + + fn schema_check(&self) -> bool { + true + } +} + +/// Optimize a SortExec by potentially pushing the sort down to the data source +/// +/// Phase 1: Optimizes when sort order is the reverse of natural output order. +/// Supports **prefix matching**: required order can be a prefix of the reversed order. +/// The data source will perform a reverse scan (read row groups in reverse). +fn optimize_sort(sort_exec: &SortExec) -> Result>> { + let sort_input = Arc::clone(sort_exec.input()); + let required_ordering = sort_exec.expr(); + + // Try to push the sort requirement down to the data source (with recursive traversal) + // Phase 1: Data source will only accept if required order is reverse of natural order + if let Some(optimized_input) = try_pushdown_sort(&sort_input, required_ordering)? { + // Phase 1: Always keep the Sort operator + // Even though we optimized the input (reverse scan), + // we return Inexact ordering to maintain correctness + // + // Benefits: + // - TopK queries with reverse order can terminate early + // - Less data needs to be sorted (data is approximately ordered) + // - Better cache locality + return Ok(Transformed::yes(Arc::new( + SortExec::new(required_ordering.clone(), optimized_input) + .with_fetch(sort_exec.fetch()) + .with_preserve_partitioning(sort_exec.preserve_partitioning()), + ))); + } + + Ok(Transformed::no(Arc::new(sort_exec.clone()))) +} + +/// Try to push down a sort requirement to an execution plan. +/// +/// This function recursively traverses through "transparent" nodes - nodes that don't +/// fundamentally change the ordering of data - to find data sources that can natively +/// handle the sort. +/// +/// **Transparent nodes** include: +/// - `CoalesceBatchesExec`: Combines small batches, preserves ordering +/// - `RepartitionExec`: May preserve ordering (if configured) +/// - `CoalescePartitionsExec`: Merges partitions, preserves ordering within partitions +/// +/// # Phase 1 Behavior +/// +/// In Phase 1, data sources will accept the pushdown if: +/// - The required ordering is the reverse of their natural output ordering +/// - **Supports prefix matching**: required ordering can be a prefix of the reversed order +/// (e.g., if data is [A DESC, B ASC], query needs [A ASC], reverse gives [A ASC, B DESC]) +/// - They can perform a reverse scan (read row groups in reverse order) +/// +/// If accepted, this returns `Some(optimized_plan)` with reverse scan enabled, +/// but does NOT guarantee perfect ordering (returns Inexact). The caller (optimize_sort) +/// will keep the Sort operator. +/// +/// # Returns +/// - `Ok(Some(plan))` - Successfully pushed sort down (reverse scan) and rebuilt the tree +/// - `Ok(None)` - Cannot push sort down through this node (not reverse order case) +/// - `Err(e)` - Error occurred during optimization +fn try_pushdown_sort( + plan: &Arc, + required_ordering: &[PhysicalSortExpr], +) -> Result>> { + // Base case: Check if the plan can natively handle the sort requirement + // Phase 1: Data source will check if required_ordering is reverse of natural order + let pushdown_result = plan.try_pushdown_sort(required_ordering)?; + + match pushdown_result { + Some(optimized) => { + // Phase 1: We got an optimized plan (reverse scan enabled) + // In future Phase 2, we could check if result is Exact and remove Sort + return Ok(Some(optimized)); + } + None => { + // Continue to recursive case + } + } + + // Recursive case: Try to push through transparent nodes + + // CoalesceBatchesExec - just combines batches, doesn't affect ordering + if let Some(coalesce_batches) = plan.as_any().downcast_ref::() { + let input = coalesce_batches.input(); + if let Some(optimized_input) = try_pushdown_sort(input, required_ordering)? { + return Ok(Some(Arc::new(CoalesceBatchesExec::new( + optimized_input, + coalesce_batches.target_batch_size(), + )))); + } + } + + // RepartitionExec - may preserve ordering in some cases + if let Some(repartition) = plan.as_any().downcast_ref::() { + let input = repartition.input(); + if let Some(optimized_input) = try_pushdown_sort(input, required_ordering)? { + // Rebuild the repartition with optimized input + let new_repartition = RepartitionExec::try_new( + optimized_input, + repartition.partitioning().clone(), + )?; + + // Preserve the preserve_order flag if it was set + if repartition.maintains_input_order()[0] { + return Ok(Some(Arc::new(new_repartition.with_preserve_order()))); + } + return Ok(Some(Arc::new(new_repartition))); + } + } + + // CoalescePartitionsExec - merges partitions + if let Some(coalesce_parts) = plan.as_any().downcast_ref::() { + let input = coalesce_parts.input(); + if let Some(optimized_input) = try_pushdown_sort(input, required_ordering)? { + return Ok(Some(Arc::new(CoalescePartitionsExec::new(optimized_input)))); + } + } + + // If we reach here, the node is not transparent or we couldn't optimize + // Phase 1: Most likely the required order is not the reverse of natural order + // (even considering prefix matching) + Ok(None) +} diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index d3043ee93c05..34d15c912e9a 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -54,7 +54,9 @@ use datafusion_common::{ 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}; @@ -682,6 +684,16 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { ) -> Option> { None } + + /// Try to create a new execution plan that satisfies the given sort ordering. + /// + /// Default implementation returns `Ok(None)`. + fn try_pushdown_sort( + &self, + _order: &[PhysicalSortExpr], + ) -> Result>> { + Ok(None) + } } /// [`ExecutionPlan`] Invariant Level diff --git a/datafusion/proto-common/proto/datafusion_common.proto b/datafusion/proto-common/proto/datafusion_common.proto index 15c82e948c95..ecba6a7aa813 100644 --- a/datafusion/proto-common/proto/datafusion_common.proto +++ b/datafusion/proto-common/proto/datafusion_common.proto @@ -532,6 +532,7 @@ message ParquetOptions { bool schema_force_view_types = 28; // default = false bool binary_as_string = 29; // default = false bool skip_arrow_metadata = 30; // default = false + bool enable_sort_pushdown = 35; // default = true oneof metadata_size_hint_opt { uint64 metadata_size_hint = 4; diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index dd6ed284c1b5..675704912822 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -1006,6 +1006,7 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { max_predicate_cache_size: value.max_predicate_cache_size_opt.map(|opt| match opt { protobuf::parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(v) => Some(v as usize), }).unwrap_or(None), + enable_sort_pushdown: value.enable_sort_pushdown, }) } } diff --git a/datafusion/proto-common/src/generated/pbjson.rs b/datafusion/proto-common/src/generated/pbjson.rs index 66659ad14cbb..2246200aaa84 100644 --- a/datafusion/proto-common/src/generated/pbjson.rs +++ b/datafusion/proto-common/src/generated/pbjson.rs @@ -5593,6 +5593,9 @@ impl serde::Serialize for ParquetOptions { if self.skip_arrow_metadata { len += 1; } + if self.enable_sort_pushdown { + len += 1; + } if self.dictionary_page_size_limit != 0 { len += 1; } @@ -5698,6 +5701,9 @@ impl serde::Serialize for ParquetOptions { if self.skip_arrow_metadata { struct_ser.serialize_field("skipArrowMetadata", &self.skip_arrow_metadata)?; } + if self.enable_sort_pushdown { + struct_ser.serialize_field("enableSortPushdown", &self.enable_sort_pushdown)?; + } if self.dictionary_page_size_limit != 0 { #[allow(clippy::needless_borrow)] #[allow(clippy::needless_borrows_for_generic_args)] @@ -5846,6 +5852,8 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "binaryAsString", "skip_arrow_metadata", "skipArrowMetadata", + "enable_sort_pushdown", + "enableSortPushdown", "dictionary_page_size_limit", "dictionaryPageSizeLimit", "data_page_row_count_limit", @@ -5895,6 +5903,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { SchemaForceViewTypes, BinaryAsString, SkipArrowMetadata, + EnableSortPushdown, DictionaryPageSizeLimit, DataPageRowCountLimit, MaxRowGroupSize, @@ -5948,6 +5957,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "schemaForceViewTypes" | "schema_force_view_types" => Ok(GeneratedField::SchemaForceViewTypes), "binaryAsString" | "binary_as_string" => Ok(GeneratedField::BinaryAsString), "skipArrowMetadata" | "skip_arrow_metadata" => Ok(GeneratedField::SkipArrowMetadata), + "enableSortPushdown" | "enable_sort_pushdown" => Ok(GeneratedField::EnableSortPushdown), "dictionaryPageSizeLimit" | "dictionary_page_size_limit" => Ok(GeneratedField::DictionaryPageSizeLimit), "dataPageRowCountLimit" | "data_page_row_count_limit" => Ok(GeneratedField::DataPageRowCountLimit), "maxRowGroupSize" | "max_row_group_size" => Ok(GeneratedField::MaxRowGroupSize), @@ -5999,6 +6009,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { let mut schema_force_view_types__ = None; let mut binary_as_string__ = None; let mut skip_arrow_metadata__ = None; + let mut enable_sort_pushdown__ = None; let mut dictionary_page_size_limit__ = None; let mut data_page_row_count_limit__ = None; let mut max_row_group_size__ = None; @@ -6126,6 +6137,12 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { } skip_arrow_metadata__ = Some(map_.next_value()?); } + GeneratedField::EnableSortPushdown => { + if enable_sort_pushdown__.is_some() { + return Err(serde::de::Error::duplicate_field("enableSortPushdown")); + } + enable_sort_pushdown__ = Some(map_.next_value()?); + } GeneratedField::DictionaryPageSizeLimit => { if dictionary_page_size_limit__.is_some() { return Err(serde::de::Error::duplicate_field("dictionaryPageSizeLimit")); @@ -6242,6 +6259,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { schema_force_view_types: schema_force_view_types__.unwrap_or_default(), binary_as_string: binary_as_string__.unwrap_or_default(), skip_arrow_metadata: skip_arrow_metadata__.unwrap_or_default(), + enable_sort_pushdown: enable_sort_pushdown__.unwrap_or_default(), dictionary_page_size_limit: dictionary_page_size_limit__.unwrap_or_default(), data_page_row_count_limit: data_page_row_count_limit__.unwrap_or_default(), max_row_group_size: max_row_group_size__.unwrap_or_default(), diff --git a/datafusion/proto-common/src/generated/prost.rs b/datafusion/proto-common/src/generated/prost.rs index eaeed5276b24..37a77d24c46d 100644 --- a/datafusion/proto-common/src/generated/prost.rs +++ b/datafusion/proto-common/src/generated/prost.rs @@ -801,6 +801,9 @@ pub struct ParquetOptions { /// default = false #[prost(bool, tag = "30")] pub skip_arrow_metadata: bool, + /// default = true + #[prost(bool, tag = "35")] + pub enable_sort_pushdown: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index 7addcde5956c..a2c2dd0a930c 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -882,6 +882,7 @@ impl TryFrom<&ParquetOptions> for protobuf::ParquetOptions { skip_arrow_metadata: value.skip_arrow_metadata, coerce_int96_opt: value.coerce_int96.clone().map(protobuf::parquet_options::CoerceInt96Opt::CoerceInt96), max_predicate_cache_size_opt: value.max_predicate_cache_size.map(|v| protobuf::parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(v as u64)), + enable_sort_pushdown: value.enable_sort_pushdown, }) } } diff --git a/datafusion/proto/src/generated/datafusion_proto_common.rs b/datafusion/proto/src/generated/datafusion_proto_common.rs index eaeed5276b24..37a77d24c46d 100644 --- a/datafusion/proto/src/generated/datafusion_proto_common.rs +++ b/datafusion/proto/src/generated/datafusion_proto_common.rs @@ -801,6 +801,9 @@ pub struct ParquetOptions { /// default = false #[prost(bool, tag = "30")] pub skip_arrow_metadata: bool, + /// default = true + #[prost(bool, tag = "35")] + pub enable_sort_pushdown: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index 20b3c6bb7aef..87d6fa3a0dc9 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -421,6 +421,7 @@ mod parquet { max_predicate_cache_size_opt: global_options.global.max_predicate_cache_size.map(|size| { parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(size as u64) }), + enable_sort_pushdown: global_options.global.enable_sort_pushdown, }), column_specific_options: column_specific_options.into_iter().map(|(column_name, options)| { ParquetColumnSpecificOptions { @@ -518,6 +519,7 @@ mod parquet { max_predicate_cache_size: proto.max_predicate_cache_size_opt.as_ref().map(|opt| match opt { parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(size) => *size as usize, }), + enable_sort_pushdown: proto.enable_sort_pushdown, } } } diff --git a/datafusion/sqllogictest/test_files/create_external_table.slt b/datafusion/sqllogictest/test_files/create_external_table.slt index 1e6183f48bac..70243a17a154 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], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet, reverse_scan_inexact=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 5b30599c4103..fd772dceb4ef 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -432,3 +432,231 @@ SET datafusion.optimizer.enable_aggregate_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_scan_inexact=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], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_scan_inexact=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.execution.parquet.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.execution.parquet.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], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_scan_inexact=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_scan_inexact=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], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet, predicate=(category@1 = alpha OR category@1 = gamma) AND DynamicFilter [ empty ], reverse_scan_inexact=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.execution.parquet.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.execution.parquet.enable_sort_pushdown = true; + +# 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.execution.parquet.enable_sort_pushdown = true; diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 918c01b5613a..c703efafd703 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -240,6 +240,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 @@ -320,6 +321,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 @@ -364,6 +366,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 @@ -599,6 +602,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 5e478de0416c..a347279ee97a 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -243,6 +243,7 @@ datafusion.execution.parquet.data_pagesize_limit 1048576 datafusion.execution.parquet.dictionary_enabled true datafusion.execution.parquet.dictionary_page_size_limit 1048576 datafusion.execution.parquet.enable_page_index true +datafusion.execution.parquet.enable_sort_pushdown true datafusion.execution.parquet.encoding NULL datafusion.execution.parquet.force_filter_selections false datafusion.execution.parquet.max_predicate_cache_size NULL @@ -371,6 +372,7 @@ datafusion.execution.parquet.data_pagesize_limit 1048576 (writing) Sets best eff datafusion.execution.parquet.dictionary_enabled true (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting datafusion.execution.parquet.dictionary_page_size_limit 1048576 (writing) Sets best effort maximum dictionary page size, in bytes datafusion.execution.parquet.enable_page_index true (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. +datafusion.execution.parquet.enable_sort_pushdown true Enable sort pushdown optimization for Parquet files. When enabled, optimizes queries with ORDER BY: - Reordering files based on statistics - Reversing row group read order when beneficial Returns **inexact ordering**: Sort operator is kept for correctness, but can terminate early for TopK queries (ORDER BY ... LIMIT N), providing huge speedup. Memory: No additional overhead (only changes read order). Future TODO: Will add option to support detecting perfectly sorted data and eliminate Sort completely. Default: true datafusion.execution.parquet.encoding NULL (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting datafusion.execution.parquet.force_filter_selections false (reading) Force the use of RowSelections for filter results, when pushdown_filters is enabled. If false, the reader will automatically choose between a RowSelection and a Bitmap based on the number and pattern of selected rows. datafusion.execution.parquet.max_predicate_cache_size NULL (reading) The maximum predicate cache size, in bytes. When `pushdown_filters` is enabled, sets the maximum memory used to cache the results of predicate evaluation between filter evaluation and output generation. Decreasing this value will reduce memory usage, but may increase IO and CPU usage. None means use the default parquet reader setting. 0 means no caching. diff --git a/datafusion/sqllogictest/test_files/slt_features.slt b/datafusion/sqllogictest/test_files/slt_features.slt index f3d467ea0d93..5075ed10eae9 100644 --- a/datafusion/sqllogictest/test_files/slt_features.slt +++ b/datafusion/sqllogictest/test_files/slt_features.slt @@ -71,4 +71,4 @@ select * from generate_series(3); 1 - \ No newline at end of file + diff --git a/datafusion/sqllogictest/test_files/spark/bitwise/bit_count.slt b/datafusion/sqllogictest/test_files/spark/bitwise/bit_count.slt index e07c96d7c121..8ec886d02e78 100644 --- a/datafusion/sqllogictest/test_files/spark/bitwise/bit_count.slt +++ b/datafusion/sqllogictest/test_files/spark/bitwise/bit_count.slt @@ -239,4 +239,4 @@ SELECT bit_count(false); query I SELECT bit_count(cast(null as boolean)); ---- -NULL \ No newline at end of file +NULL diff --git a/datafusion/sqllogictest/test_files/topk.slt b/datafusion/sqllogictest/test_files/topk.slt index 7364fccd8e57..009104112289 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], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_scan_inexact=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 77d6ff8be97e..e28583890252 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -110,6 +110,7 @@ The following configuration settings are available: | datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | | datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | | datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.parquet.enable_sort_pushdown | true | Enable sort pushdown optimization for Parquet files. When enabled, optimizes queries with ORDER BY: - Reordering files based on statistics - Reversing row group read order when beneficial Returns **inexact ordering**: Sort operator is kept for correctness, but can terminate early for TopK queries (ORDER BY ... LIMIT N), providing huge speedup. Memory: No additional overhead (only changes read order). Future TODO: Will add option to support detecting perfectly sorted data and eliminate Sort completely. Default: true | | datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | | datafusion.execution.skip_physical_aggregate_schema_check | false | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | | datafusion.execution.spill_compression | uncompressed | Sets the compression codec used when spilling data to disk. Since datafusion writes spill files using the Arrow IPC Stream format, only codecs supported by the Arrow IPC Stream Writer are allowed. Valid values are: uncompressed, lz4_frame, zstd. Note: lz4_frame offers faster (de)compression, but typically results in larger spill files. In contrast, zstd achieves higher compression ratios at the cost of slower (de)compression speed. |