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