diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index 6679405623d06..b3fa0c28145b1 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -102,6 +102,10 @@ 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) +# Sort Pushdown Benchmarks +sort_pushdown: Sort pushdown baseline (no WITH ORDER) on TPC-H data (SF=1) +sort_pushdown_sorted: Sort pushdown with WITH ORDER — tests sort elimination on non-overlapping files + # Sorted Data Benchmarks (ORDER BY Optimization) clickbench_sorted: ClickBench queries on pre-sorted data using prefer_existing_sort (tests sort elimination optimization) @@ -305,6 +309,10 @@ main() { # same data as for tpch data_tpch "1" "parquet" ;; + sort_pushdown|sort_pushdown_sorted) + # same data as for tpch + data_tpch "1" "parquet" + ;; sort_tpch) # same data as for tpch data_tpch "1" "parquet" @@ -505,6 +513,12 @@ main() { external_aggr) run_external_aggr ;; + sort_pushdown) + run_sort_pushdown + ;; + sort_pushdown_sorted) + run_sort_pushdown_sorted + ;; sort_tpch) run_sort_tpch "1" ;; @@ -1193,6 +1207,22 @@ run_external_aggr() { debug_run $CARGO_COMMAND --bin external_aggr -- benchmark --partitions 4 --iterations 5 --path "${TPCH_DIR}" -o "${RESULTS_FILE}" ${QUERY_ARG} } +# Runs the sort pushdown benchmark (without WITH ORDER) +run_sort_pushdown() { + TPCH_DIR="${DATA_DIR}/tpch_sf1" + RESULTS_FILE="${RESULTS_DIR}/sort_pushdown.json" + echo "Running sort pushdown benchmark (no WITH ORDER)..." + debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --iterations 5 --path "${TPCH_DIR}" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} +} + +# Runs the sort pushdown benchmark with WITH ORDER (enables sort elimination) +run_sort_pushdown_sorted() { + TPCH_DIR="${DATA_DIR}/tpch_sf1" + RESULTS_FILE="${RESULTS_DIR}/sort_pushdown_sorted.json" + echo "Running sort pushdown benchmark (with WITH ORDER)..." + debug_run $CARGO_COMMAND --bin dfbench -- sort-pushdown --sorted --iterations 5 --path "${TPCH_DIR}" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} +} + # Runs the sort integration benchmark run_sort_tpch() { SCALE_FACTOR=$1 diff --git a/benchmarks/queries/sort_pushdown/q1.sql b/benchmarks/queries/sort_pushdown/q1.sql new file mode 100644 index 0000000000000..f5f51a5d4043e --- /dev/null +++ b/benchmarks/queries/sort_pushdown/q1.sql @@ -0,0 +1,6 @@ +-- Sort elimination: ORDER BY sort key ASC (full scan) +-- With --sorted: SortExec removed, sequential scan in file order +-- Without --sorted: full SortExec required +SELECT l_orderkey, l_partkey, l_suppkey +FROM lineitem +ORDER BY l_orderkey diff --git a/benchmarks/queries/sort_pushdown/q2.sql b/benchmarks/queries/sort_pushdown/q2.sql new file mode 100644 index 0000000000000..29a0e127cb7c6 --- /dev/null +++ b/benchmarks/queries/sort_pushdown/q2.sql @@ -0,0 +1,7 @@ +-- Sort elimination + limit pushdown +-- With --sorted: SortExec removed + limit pushed to DataSourceExec +-- Without --sorted: TopK sort over all data +SELECT l_orderkey, l_partkey, l_suppkey +FROM lineitem +ORDER BY l_orderkey +LIMIT 100 diff --git a/benchmarks/queries/sort_pushdown/q3.sql b/benchmarks/queries/sort_pushdown/q3.sql new file mode 100644 index 0000000000000..e11b48659a2a2 --- /dev/null +++ b/benchmarks/queries/sort_pushdown/q3.sql @@ -0,0 +1,5 @@ +-- Sort elimination: wide projection (all columns) +-- Tests sort elimination benefit with larger row payload +SELECT * +FROM lineitem +ORDER BY l_orderkey diff --git a/benchmarks/queries/sort_pushdown/q4.sql b/benchmarks/queries/sort_pushdown/q4.sql new file mode 100644 index 0000000000000..99500c371991a --- /dev/null +++ b/benchmarks/queries/sort_pushdown/q4.sql @@ -0,0 +1,5 @@ +-- Sort elimination + limit: wide projection +SELECT * +FROM lineitem +ORDER BY l_orderkey +LIMIT 100 diff --git a/benchmarks/src/bin/dfbench.rs b/benchmarks/src/bin/dfbench.rs index 7e21890519fd1..3b1f54291e75c 100644 --- a/benchmarks/src/bin/dfbench.rs +++ b/benchmarks/src/bin/dfbench.rs @@ -34,7 +34,8 @@ static ALLOC: snmalloc_rs::SnMalloc = snmalloc_rs::SnMalloc; static ALLOC: mimalloc::MiMalloc = mimalloc::MiMalloc; use datafusion_benchmarks::{ - cancellation, clickbench, h2o, hj, imdb, nlj, smj, sort_tpch, tpcds, tpch, + cancellation, clickbench, h2o, hj, imdb, nlj, smj, sort_pushdown, sort_tpch, tpcds, + tpch, }; #[derive(Debug, Parser)] @@ -53,6 +54,7 @@ enum Options { Imdb(imdb::RunOpt), Nlj(nlj::RunOpt), Smj(smj::RunOpt), + SortPushdown(sort_pushdown::RunOpt), SortTpch(sort_tpch::RunOpt), Tpch(tpch::RunOpt), Tpcds(tpcds::RunOpt), @@ -72,6 +74,7 @@ pub async fn main() -> Result<()> { Options::Imdb(opt) => Box::pin(opt.run()).await, Options::Nlj(opt) => opt.run().await, Options::Smj(opt) => opt.run().await, + Options::SortPushdown(opt) => opt.run().await, Options::SortTpch(opt) => opt.run().await, Options::Tpch(opt) => Box::pin(opt.run()).await, Options::Tpcds(opt) => Box::pin(opt.run()).await, diff --git a/benchmarks/src/lib.rs b/benchmarks/src/lib.rs index a3bc221840ada..7e2196e89e592 100644 --- a/benchmarks/src/lib.rs +++ b/benchmarks/src/lib.rs @@ -23,6 +23,7 @@ pub mod hj; pub mod imdb; pub mod nlj; pub mod smj; +pub mod sort_pushdown; pub mod sort_tpch; pub mod tpcds; pub mod tpch; diff --git a/benchmarks/src/sort_pushdown.rs b/benchmarks/src/sort_pushdown.rs new file mode 100644 index 0000000000000..e7fce1921e7a8 --- /dev/null +++ b/benchmarks/src/sort_pushdown.rs @@ -0,0 +1,282 @@ +// 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. + +//! Benchmark for sort pushdown optimization. +//! +//! Tests performance of sort elimination when files are non-overlapping and +//! internally sorted (declared via `--sorted` / `WITH ORDER`). +//! +//! Queries are loaded from external SQL files under `queries/sort_pushdown/` +//! so they can also be run directly with `datafusion-cli`. +//! +//! # Usage +//! +//! ```text +//! # Prepare sorted TPCH lineitem data (SF=1) +//! ./bench.sh data sort_pushdown +//! +//! # Baseline (no WITH ORDER, full SortExec) +//! ./bench.sh run sort_pushdown +//! +//! # With sort elimination (WITH ORDER, SortExec removed) +//! ./bench.sh run sort_pushdown_sorted +//! ``` + +use clap::Args; +use futures::StreamExt; +use std::path::PathBuf; +use std::sync::Arc; + +use datafusion::datasource::TableProvider; +use datafusion::datasource::file_format::parquet::ParquetFormat; +use datafusion::datasource::listing::{ + ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, +}; +use datafusion::error::Result; +use datafusion::execution::SessionStateBuilder; +use datafusion::physical_plan::display::DisplayableExecutionPlan; +use datafusion::physical_plan::{displayable, execute_stream}; +use datafusion::prelude::*; +use datafusion_common::DEFAULT_PARQUET_EXTENSION; +use datafusion_common::instant::Instant; + +use crate::util::{BenchmarkRun, CommonOpt, QueryResult, print_memory_stats}; + +/// Default path to query files, relative to the benchmark root +const SORT_PUSHDOWN_QUERY_DIR: &str = "queries/sort_pushdown"; + +#[derive(Debug, Args)] +pub struct RunOpt { + /// Common options + #[command(flatten)] + common: CommonOpt, + + /// Sort pushdown query number (1-4). If not specified, runs all queries + #[arg(short, long)] + pub query: Option, + + /// Path to data files (lineitem). Only parquet format is supported. + #[arg(required = true, short = 'p', long = "path")] + path: PathBuf, + + /// Path to JSON benchmark result to be compared using `compare.py` + #[arg(short = 'o', long = "output")] + output_path: Option, + + /// Path to directory containing query SQL files (q1.sql, q2.sql, ...). + /// Defaults to `queries/sort_pushdown/` relative to current directory. + #[arg(long = "queries-path")] + queries_path: Option, + + /// Mark the first column (l_orderkey) as sorted via WITH ORDER. + /// When set, enables sort elimination for matching queries. + #[arg(short = 't', long = "sorted")] + sorted: bool, +} + +impl RunOpt { + const TABLES: [&'static str; 1] = ["lineitem"]; + + fn queries_dir(&self) -> PathBuf { + self.queries_path + .clone() + .unwrap_or_else(|| PathBuf::from(SORT_PUSHDOWN_QUERY_DIR)) + } + + fn load_query(&self, query_id: usize) -> Result { + let path = self.queries_dir().join(format!("q{query_id}.sql")); + std::fs::read_to_string(&path).map_err(|e| { + datafusion_common::DataFusionError::Execution(format!( + "Failed to read query file {}: {e}", + path.display() + )) + }) + } + + fn available_queries(&self) -> Vec { + let dir = self.queries_dir(); + let mut ids = Vec::new(); + if let Ok(entries) = std::fs::read_dir(&dir) { + for entry in entries.flatten() { + let name = entry.file_name(); + let name = name.to_string_lossy(); + if let Some(rest) = name.strip_prefix('q') + && let Some(num_str) = rest.strip_suffix(".sql") + && let Ok(id) = num_str.parse::() + { + ids.push(id); + } + } + } + ids.sort(); + ids + } + + pub async fn run(&self) -> Result<()> { + let mut benchmark_run = BenchmarkRun::new(); + + let query_ids = match self.query { + Some(query_id) => vec![query_id], + None => self.available_queries(), + }; + + for query_id in query_ids { + benchmark_run.start_new_case(&format!("{query_id}")); + + let query_results = self.benchmark_query(query_id).await; + match query_results { + Ok(query_results) => { + for iter in query_results { + benchmark_run.write_iter(iter.elapsed, iter.row_count); + } + } + Err(e) => { + benchmark_run.mark_failed(); + eprintln!("Query {query_id} failed: {e}"); + } + } + } + + benchmark_run.maybe_write_json(self.output_path.as_ref())?; + benchmark_run.maybe_print_failures(); + Ok(()) + } + + async fn benchmark_query(&self, query_id: usize) -> Result> { + let sql = self.load_query(query_id)?; + + let config = self.common.config()?; + let rt = self.common.build_runtime()?; + let state = SessionStateBuilder::new() + .with_config(config) + .with_runtime_env(rt) + .with_default_features() + .build(); + let ctx = SessionContext::from(state); + + self.register_tables(&ctx).await?; + + let mut millis = vec![]; + let mut query_results = vec![]; + for i in 0..self.iterations() { + let start = Instant::now(); + + let row_count = self.execute_query(&ctx, sql.as_str()).await?; + + let elapsed = start.elapsed(); + let ms = elapsed.as_secs_f64() * 1000.0; + millis.push(ms); + + println!( + "Query {query_id} iteration {i} took {ms:.1} ms and returned {row_count} rows" + ); + query_results.push(QueryResult { elapsed, row_count }); + } + + let avg = millis.iter().sum::() / millis.len() as f64; + println!("Query {query_id} avg time: {avg:.2} ms"); + + print_memory_stats(); + + Ok(query_results) + } + + async fn register_tables(&self, ctx: &SessionContext) -> Result<()> { + for table in Self::TABLES { + let table_provider = self.get_table(ctx, table).await?; + ctx.register_table(table, table_provider)?; + } + Ok(()) + } + + async fn execute_query(&self, ctx: &SessionContext, sql: &str) -> Result { + let debug = self.common.debug; + let plan = ctx.sql(sql).await?; + let (state, plan) = plan.into_parts(); + + if debug { + println!("=== Logical plan ===\n{plan}\n"); + } + + let plan = state.optimize(&plan)?; + if debug { + println!("=== Optimized logical plan ===\n{plan}\n"); + } + let physical_plan = state.create_physical_plan(&plan).await?; + if debug { + println!( + "=== Physical plan ===\n{}\n", + displayable(physical_plan.as_ref()).indent(true) + ); + } + + let mut row_count = 0; + let mut stream = execute_stream(physical_plan.clone(), state.task_ctx())?; + while let Some(batch) = stream.next().await { + row_count += batch?.num_rows(); + } + + if debug { + println!( + "=== Physical plan with metrics ===\n{}\n", + DisplayableExecutionPlan::with_metrics(physical_plan.as_ref()) + .indent(true) + ); + } + + Ok(row_count) + } + + async fn get_table( + &self, + ctx: &SessionContext, + table: &str, + ) -> Result> { + let path = self.path.to_str().unwrap(); + let state = ctx.state(); + let path = format!("{path}/{table}"); + let format = Arc::new( + ParquetFormat::default() + .with_options(ctx.state().table_options().parquet.clone()), + ); + let extension = DEFAULT_PARQUET_EXTENSION; + + let options = ListingOptions::new(format) + .with_file_extension(extension) + .with_collect_stat(true); // Always collect statistics for sort pushdown + + let table_path = ListingTableUrl::parse(path)?; + let schema = options.infer_schema(&state, &table_path).await?; + let options = if self.sorted { + let key_column_name = schema.fields()[0].name(); + options + .with_file_sort_order(vec![vec![col(key_column_name).sort(true, false)]]) + } else { + options + }; + + let config = ListingTableConfig::new(table_path) + .with_listing_options(options) + .with_schema(schema); + + Ok(Arc::new(ListingTable::try_new(config)?)) + } + + fn iterations(&self) -> usize { + self.common.iterations + } +} diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index 2b8385ac2d89c..abf068db0cca8 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -269,7 +269,7 @@ impl StatementExecutor { let options = task_ctx.session_config().options(); // Track memory usage for the query result if it's bounded - let mut reservation = + let reservation = MemoryConsumer::new("DataFusion-Cli").register(task_ctx.memory_pool()); if physical_plan.boundedness().is_unbounded() { diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 37824a722e86c..6a47b4b7a0d50 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -2173,7 +2173,7 @@ mod tests { // configure with same memory / disk manager let memory_pool = ctx1.runtime_env().memory_pool.clone(); - let mut reservation = MemoryConsumer::new("test").register(&memory_pool); + let reservation = MemoryConsumer::new("test").register(&memory_pool); reservation.grow(100); let disk_manager = ctx1.runtime_env().disk_manager.clone(); diff --git a/datafusion/core/tests/physical_optimizer/pushdown_sort.rs b/datafusion/core/tests/physical_optimizer/pushdown_sort.rs index efe3c53487fcc..7adca66649314 100644 --- a/datafusion/core/tests/physical_optimizer/pushdown_sort.rs +++ b/datafusion/core/tests/physical_optimizer/pushdown_sort.rs @@ -32,8 +32,8 @@ use datafusion_physical_optimizer::pushdown_sort::PushdownSort; use std::sync::Arc; use crate::physical_optimizer::test_utils::{ - OptimizationTest, coalesce_batches_exec, coalesce_partitions_exec, exact_test_scan, - parquet_exec, parquet_exec_with_sort, projection_exec, projection_exec_with_alias, + OptimizationTest, coalesce_batches_exec, coalesce_partitions_exec, parquet_exec, + parquet_exec_with_sort, projection_exec, projection_exec_with_alias, repartition_exec, schema, simple_projection_exec, sort_exec, sort_exec_with_fetch, sort_expr, sort_expr_named, test_scan_with_ordering, }; @@ -258,8 +258,9 @@ fn test_prefix_match_through_transparent_nodes() { } #[test] -fn test_no_prefix_match_wrong_direction() { - // Test that prefix matching does NOT work if the direction is wrong +fn test_exact_prefix_match_same_direction() { + // Test that when the requested sort [a DESC] matches a prefix of the source's + // natural ordering [a DESC, b ASC], the Sort is eliminated (Exact pushdown). let schema = schema(); // Source has [a DESC, b ASC] ordering @@ -268,7 +269,7 @@ fn test_no_prefix_match_wrong_direction() { 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 + // Request [a DESC] - same direction as source prefix, Sort should be eliminated let same_direction = LexOrdering::new(vec![a.clone().reverse()]).unwrap(); let plan = sort_exec(same_direction, source); @@ -281,8 +282,7 @@ fn test_no_prefix_match_wrong_direction() { - 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 + - 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 " ); } @@ -1038,58 +1038,3 @@ fn test_sort_pushdown_with_test_scan_arbitrary_ordering() { " ); } - -// ============================================================================ -// EXACT PUSHDOWN TESTS (source guarantees ordering, SortExec removed) -// ============================================================================ - -#[test] -fn test_sort_pushdown_exact_no_fetch_no_limit() { - let schema = schema(); - let a = sort_expr("a", &schema); - let b = sort_expr("b", &schema); - let source = exact_test_scan(schema.clone()); - - let ordering = LexOrdering::new(vec![a, b.reverse()]).unwrap(); - let plan = sort_exec(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] - - ExactTestScan - output: - Ok: - - ExactTestScan: ordered=[a@0 ASC, b@1 DESC NULLS LAST] - " - ); -} - -#[test] -fn test_sort_pushdown_exact_preserves_fetch() { - // When a source returns Exact and the SortExec has fetch (LIMIT), - // the optimizer tries to push the limit into the source via with_fetch(). - // ExactTestScan supports with_fetch(), so the limit should appear - // directly on the source (no GlobalLimitExec wrapper needed). - let schema = schema(); - let a = sort_expr("a", &schema); - let source = exact_test_scan(schema.clone()); - - let ordering = LexOrdering::new(vec![a]).unwrap(); - let plan = sort_exec_with_fetch(ordering, Some(10), source); - - insta::assert_snapshot!( - OptimizationTest::new(plan, PushdownSort::new(), true), - @r" - OptimizationTest: - input: - - SortExec: TopK(fetch=10), expr=[a@0 ASC], preserve_partitioning=[false] - - ExactTestScan - output: - Ok: - - ExactTestScan: ordered=[a@0 ASC], fetch=10 - " - ); -} diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index f05baa094069c..5b50181d7fd3e 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -979,144 +979,3 @@ pub fn test_scan_with_ordering( ) -> Arc { Arc::new(TestScan::with_ordering(schema, ordering)) } - -/// A test scan that returns `Exact` from `try_pushdown_sort`. -#[derive(Debug, Clone)] -pub struct ExactTestScan { - schema: SchemaRef, - plan_properties: PlanProperties, - requested_ordering: Option, - fetch: Option, -} - -impl ExactTestScan { - pub fn new(schema: SchemaRef) -> Self { - let eq_properties = EquivalenceProperties::new(Arc::clone(&schema)); - let plan_properties = PlanProperties::new( - eq_properties, - Partitioning::UnknownPartitioning(1), - EmissionType::Incremental, - Boundedness::Bounded, - ); - Self { - schema, - plan_properties, - requested_ordering: None, - fetch: None, - } - } -} - -impl DisplayAs for ExactTestScan { - fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!(f, "ExactTestScan")?; - if let Some(ref req) = self.requested_ordering { - write!(f, ": ordered=[")?; - for (i, sort_expr) in req.iter().enumerate() { - if i > 0 { - write!(f, ", ")?; - } - write!(f, "{sort_expr}")?; - } - write!(f, "]")?; - } - if let Some(fetch) = self.fetch { - write!(f, ", fetch={fetch}")?; - } - Ok(()) - } - DisplayFormatType::TreeRender => { - write!(f, "ExactTestScan") - } - } - } -} - -impl ExecutionPlan for ExactTestScan { - fn name(&self) -> &str { - "ExactTestScan" - } - - 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!("ExactTestScan should have no children") - } - } - - fn execute( - &self, - _partition: usize, - _context: Arc, - ) -> Result { - internal_err!("ExactTestScan is for testing optimizer only, not for execution") - } - - fn partition_statistics(&self, _partition: Option) -> Result { - Ok(Statistics::new_unknown(&self.schema)) - } - - fn with_fetch(&self, fetch: Option) -> Option> { - let mut new_scan = self.clone(); - new_scan.fetch = fetch; - Some(Arc::new(new_scan)) - } - - fn fetch(&self) -> Option { - self.fetch - } - - fn try_pushdown_sort( - &self, - order: &[PhysicalSortExpr], - ) -> Result>> { - let requested_ordering = LexOrdering::new(order.to_vec()); - - let orderings: Vec> = vec![order.to_vec()]; - let eq_properties = EquivalenceProperties::new_with_orderings( - Arc::clone(&self.schema), - orderings, - ); - let plan_properties = PlanProperties::new( - eq_properties, - Partitioning::UnknownPartitioning(1), - EmissionType::Incremental, - Boundedness::Bounded, - ); - - let new_scan = ExactTestScan { - schema: Arc::clone(&self.schema), - plan_properties, - requested_ordering, - fetch: self.fetch, - }; - - // Return Exact: this source guarantees the requested ordering - Ok(SortOrderPushdownResult::Exact { - inner: Arc::new(new_scan), - }) - } -} - -/// Helper function to create an ExactTestScan -pub fn exact_test_scan(schema: SchemaRef) -> Arc { - Arc::new(ExactTestScan::new(schema)) -} diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 368502836602c..9dc2ca853d2a3 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -1278,7 +1278,7 @@ impl FileSink for ParquetSink { parquet_props.clone(), ) .await?; - let mut reservation = MemoryConsumer::new(format!("ParquetSink[{path}]")) + let reservation = MemoryConsumer::new(format!("ParquetSink[{path}]")) .register(context.memory_pool()); file_write_tasks.spawn(async move { while let Some(batch) = rx.recv().await { @@ -1383,7 +1383,7 @@ impl DataSink for ParquetSink { async fn column_serializer_task( mut rx: Receiver, mut writer: ArrowColumnWriter, - mut reservation: MemoryReservation, + reservation: MemoryReservation, ) -> Result<(ArrowColumnWriter, MemoryReservation)> { while let Some(col) = rx.recv().await { writer.write(&col)?; @@ -1468,7 +1468,7 @@ fn spawn_rg_join_and_finalize_task( rg_rows: usize, pool: &Arc, ) -> SpawnedTask { - let mut rg_reservation = + let rg_reservation = MemoryConsumer::new("ParquetSink(SerializedRowGroupWriter)").register(pool); SpawnedTask::spawn(async move { @@ -1600,12 +1600,12 @@ async fn concatenate_parallel_row_groups( mut object_store_writer: Box, pool: Arc, ) -> Result { - let mut file_reservation = + let file_reservation = MemoryConsumer::new("ParquetSink(SerializedFileWriter)").register(&pool); while let Some(task) = serialize_rx.recv().await { let result = task.join_unwind().await; - let (serialized_columns, mut rg_reservation, _cnt) = + let (serialized_columns, rg_reservation, _cnt) = result.map_err(|e| DataFusionError::ExecutionJoin(Box::new(e)))??; let mut rg_out = parquet_writer.next_row_group()?; diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 2e0919b1447de..0f36b81b8dc8e 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -741,19 +741,17 @@ impl FileSource for ParquetSource { /// /// 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. + /// # Behavior + /// - Returns `Exact` when the file's natural ordering (from Parquet metadata) already + /// satisfies the requested ordering. This allows the Sort operator to be eliminated + /// if the files within each group are also non-overlapping (checked by FileScanConfig). + /// - 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 /// /// # Returns + /// - `Exact`: The file's natural ordering satisfies the request (within-file ordering guaranteed) /// - `Inexact`: Created an optimized source (e.g., reversed scan) that approximates the order /// - `Unsupported`: Cannot optimize for this ordering fn try_reverse_output( @@ -765,6 +763,16 @@ impl FileSource for ParquetSource { return Ok(SortOrderPushdownResult::Unsupported); } + // Check if the natural (non-reversed) ordering already satisfies the request. + // Parquet metadata guarantees within-file ordering, so if the ordering matches + // we can return Exact. FileScanConfig will verify that files within each group + // are non-overlapping before declaring the entire scan as Exact. + if eq_properties.ordering_satisfy(order.iter().cloned())? { + return Ok(SortOrderPushdownResult::Exact { + inner: Arc::new(self.clone()) as Arc, + }); + } + // Build new equivalence properties with the reversed ordering. // This allows us to check if the reversed ordering satisfies the request // by leveraging: @@ -809,11 +817,6 @@ impl FileSource for ParquetSource { 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 } } diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 34085c72855a2..58eca8fa5ffa5 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -34,6 +34,7 @@ use arrow::datatypes::{ }; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::config::ConfigOptions; +use datafusion_common::stats::Precision; use datafusion_common::{ ColumnStatistics, Constraints, Result, ScalarValue, Statistics, exec_datafusion_err, exec_err, internal_datafusion_err, internal_err, @@ -1038,33 +1039,84 @@ impl DataSource for FileScanConfig { } } + /// Push sort requirements into file-based data sources. + /// + /// # Sort Pushdown Architecture + /// + /// When a partition (file group) contains multiple files in wrong order, + /// `validated_output_ordering()` strips the ordering and `EnforceSorting` + /// inserts a `SortExec`. This optimizer fixes the file order by sorting + /// files within each group by min/max statistics, enabling sort elimination. + /// + /// This applies to both single-partition and multi-partition plans — any + /// file group with multiple files in wrong order benefits. + /// + /// ```text + /// PushdownSort optimizer finds SortExec + /// │ + /// ▼ + /// FileScanConfig::try_pushdown_sort() + /// │ + /// ├─► FileSource returns Exact + /// │ (natural ordering satisfies request) + /// │ → rebuild_with_source: sort files by stats, verify non-overlapping + /// │ → SortExec removed, fetch (LIMIT) pushed to DataSourceExec + /// │ + /// ├─► FileSource returns Inexact + /// │ (reverse_row_groups=true) + /// │ → SortExec kept, scan optimized + /// │ + /// └─► FileSource returns Unsupported + /// (ordering stripped because files in wrong order) + /// → try_sort_file_groups_by_statistics(): + /// 1. Sort files within each group by min/max statistics + /// 2. Re-check: non-overlapping + ordering valid? + /// YES → Exact → SortExec removed + /// NO → Inexact (files reordered, Sort stays) + /// ``` 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)?, - }) + let config = self.rebuild_with_source(inner, true, order)?; + // rebuild_with_source keeps output_ordering only when all groups + // are non-overlapping. If output_ordering was cleared, files + // overlap despite within-file ordering → downgrade to Inexact. + if config.output_ordering.is_empty() { + Ok(SortOrderPushdownResult::Inexact { + inner: Arc::new(config), + }) + } else { + Ok(SortOrderPushdownResult::Exact { + inner: Arc::new(config), + }) + } } SortOrderPushdownResult::Inexact { inner } => { Ok(SortOrderPushdownResult::Inexact { - inner: self.rebuild_with_source(inner, false)?, + inner: Arc::new(self.rebuild_with_source(inner, false, order)?), }) } SortOrderPushdownResult::Unsupported => { - Ok(SortOrderPushdownResult::Unsupported) + self.try_sort_file_groups_by_statistics(order) } } } } +/// Result of sorting files within groups by their min/max statistics. +struct SortedFileGroups { + file_groups: Vec, + any_reordered: bool, + all_non_overlapping: bool, +} + impl FileScanConfig { /// Get the file schema (schema of the files without partition columns) pub fn file_schema(&self) -> &SchemaRef { @@ -1353,34 +1405,335 @@ impl FileScanConfig { &self.file_source } - /// Helper: Rebuild FileScanConfig with new file source + /// Rebuild FileScanConfig after sort pushdown, applying file-level optimizations. + /// + /// This is the core of sort pushdown for file-based sources. It performs + /// three optimizations depending on the pushdown result: + /// + /// ```text + /// ┌─────────────────────────────────────────────────────────────┐ + /// │ rebuild_with_source │ + /// │ │ + /// │ 1. Reverse file groups (if DESC matches reversed ordering) │ + /// │ 2. Sort files within groups by min/max statistics │ + /// │ 3. If Exact + non-overlapping: │ + /// │ Keep output_ordering → SortExec eliminated │ + /// │ Otherwise: clear output_ordering → SortExec stays │ + /// └─────────────────────────────────────────────────────────────┘ + /// ``` + /// + /// # Why sort files by statistics? + /// + /// Files within a partition (file group) are read sequentially. By sorting + /// them so that file_i.max <= file_{i+1}.min, the combined output stream + /// is already in order — no SortExec needed for that partition. + /// + /// Even when files overlap (Inexact), statistics-based ordering helps + /// TopK/LIMIT queries: reading low-value files first lets dynamic filters + /// prune high-value files earlier. fn rebuild_with_source( &self, new_file_source: Arc, is_exact: bool, - ) -> Result> { + order: &[PhysicalSortExpr], + ) -> 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(); + // Reverse file order (within each group) if the caller is requesting a reversal of this + // scan's declared output ordering. + let reverse_file_groups = if self.output_ordering.is_empty() { + false + } else if let Some(requested) = LexOrdering::new(order.iter().cloned()) { + let projected_schema = self.projected_schema()?; + let orderings = project_orderings(&self.output_ordering, &projected_schema); + orderings + .iter() + .any(|ordering| ordering.is_reverse(&requested)) + } else { + false + }; + + if reverse_file_groups { + 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 { + // Sort files within groups by statistics when not reversing + let all_non_overlapping = if !reverse_file_groups { + if let Some(sort_order) = LexOrdering::new(order.iter().cloned()) { + let projected_schema = new_config.projected_schema()?; + let projection_indices = new_config + .file_source + .projection() + .as_ref() + .and_then(|p| ordered_column_indices_from_projection(p)); + let result = Self::sort_files_within_groups_by_statistics( + &new_config.file_groups, + &sort_order, + &projected_schema, + projection_indices.as_deref(), + ); + new_config.file_groups = result.file_groups; + result.all_non_overlapping + } else { + false + } + } else { + // When reversing, files are already reversed above. We skip + // statistics-based sorting here because it would undo the reversal. + // Note: reverse path is always Inexact, so all_non_overlapping + // is not used (is_exact is false). + false + }; + + if is_exact && all_non_overlapping { + // Truly exact: within-file ordering guaranteed and files are non-overlapping. + // Keep output_ordering so SortExec can be eliminated for each partition. + // + // We intentionally do NOT redistribute files across groups here. + // The planning-phase bin-packing may interleave file ranges across groups: + // + // Group 0: [f1(1-10), f3(21-30)] ← interleaved with group 1 + // Group 1: [f2(11-20), f4(31-40)] + // + // This interleaving is actually beneficial because SPM pulls from both + // partitions concurrently, keeping parallel I/O active: + // + // SPM: pull P0 [1-10] → pull P1 [11-20] → pull P0 [21-30] → pull P1 [31-40] + // ^^^^^^^^^^^^ ^^^^^^^^^^^^ + // both partitions scanning files simultaneously + // + // If we were to redistribute files consecutively: + // Group 0: [f1(1-10), f2(11-20)] ← all values < group 1 + // Group 1: [f3(21-30), f4(31-40)] + // + // SPM would read ALL of group 0 first (values always smaller), then group 1. + // This degrades to single-threaded sequential I/O — the other partition + // sits idle the entire time, losing the parallelism benefit. + } else { new_config.output_ordering = vec![]; } - Ok(Arc::new(new_config)) + Ok(new_config) + } + + /// Sort files within each file group by their min/max statistics. + /// + /// No files are moved between groups — parallelism and group composition + /// are unchanged. Groups where statistics are unavailable are kept as-is. + /// + /// ```text + /// Before: Group [file_c(20-30), file_a(0-9), file_b(10-19)] + /// After: Group [file_a(0-9), file_b(10-19), file_c(20-30)] + /// ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + /// sorted by min value, non-overlapping → Exact + /// ``` + fn sort_files_within_groups_by_statistics( + file_groups: &[FileGroup], + sort_order: &LexOrdering, + projected_schema: &SchemaRef, + projection_indices: Option<&[usize]>, + ) -> SortedFileGroups { + let mut any_reordered = false; + let mut confirmed_non_overlapping: usize = 0; + let mut new_groups = Vec::with_capacity(file_groups.len()); + + for group in file_groups { + if group.len() <= 1 { + new_groups.push(group.clone()); + confirmed_non_overlapping += 1; + continue; + } + + let files: Vec<_> = group.iter().collect(); + + let statistics = match MinMaxStatistics::new_from_files( + sort_order, + projected_schema, + projection_indices, + files.iter().copied(), + ) { + Ok(stats) => stats, + Err(e) => { + log::trace!( + "Cannot sort file group by statistics: {e}. Keeping original order." + ); + new_groups.push(group.clone()); + continue; + } + }; + + let sorted_indices = statistics.min_values_sorted(); + + let already_sorted = sorted_indices + .iter() + .enumerate() + .all(|(pos, (idx, _))| pos == *idx); + + let sorted_group: FileGroup = if already_sorted { + group.clone() + } else { + any_reordered = true; + sorted_indices + .iter() + .map(|(idx, _)| files[*idx].clone()) + .collect() + }; + + let sorted_files: Vec<_> = sorted_group.iter().collect(); + let is_non_overlapping = match MinMaxStatistics::new_from_files( + sort_order, + projected_schema, + projection_indices, + sorted_files.iter().copied(), + ) { + Ok(stats) => stats.is_sorted(), + Err(_) => false, + }; + + if is_non_overlapping { + confirmed_non_overlapping += 1; + } + + new_groups.push(sorted_group); + } + + SortedFileGroups { + file_groups: new_groups, + any_reordered, + all_non_overlapping: confirmed_non_overlapping == file_groups.len(), + } + } + + /// Last-resort optimization when FileSource returns `Unsupported`. + /// + /// FileSource may return `Unsupported` because `eq_properties` had no + /// ordering — which happens when `validated_output_ordering()` stripped + /// the ordering because files were in the wrong order. After sorting + /// files by statistics, the ordering may become valid again. + /// + /// This method: + /// 1. Sorts files within groups by min/max statistics + /// 2. Re-checks if the sorted file order makes `output_ordering` valid + /// 3. If valid AND non-overlapping → `Exact` (SortExec eliminated!) + /// 4. If files were reordered but ordering not valid → `Inexact` + /// 5. If no files were reordered → `Unsupported` + /// + /// This handles the key case where files have correct within-file ordering + /// (e.g., Parquet sorting_columns metadata) but were listed in wrong order + /// (e.g., alphabetical order doesn't match sort key order). + fn try_sort_file_groups_by_statistics( + &self, + order: &[PhysicalSortExpr], + ) -> Result>> { + let Some(sort_order) = LexOrdering::new(order.iter().cloned()) else { + return Ok(SortOrderPushdownResult::Unsupported); + }; + + let projected_schema = self.projected_schema()?; + let projection_indices = self + .file_source + .projection() + .as_ref() + .and_then(|p| ordered_column_indices_from_projection(p)); + + let result = Self::sort_files_within_groups_by_statistics( + &self.file_groups, + &sort_order, + &projected_schema, + projection_indices.as_deref(), + ); + + if !result.any_reordered { + return Ok(SortOrderPushdownResult::Unsupported); + } + + let mut new_config = self.clone(); + new_config.file_groups = result.file_groups; + + // Re-check: now that files are sorted, does output_ordering become valid? + // This handles the case where validated_output_ordering() previously + // stripped the ordering because files were in the wrong order. + // + // IMPORTANT: We cannot claim Exact if any file in a non-last position + // contains NULLs in the sort columns. With NULLS LAST, NULLs within + // a file are placed after all non-null values. If the next file has + // non-null values smaller than the previous file's max, those values + // would incorrectly appear after the NULLs. Similarly for NULLS FIRST. + // + // Conservative approach: if any file has nulls in the sort columns, + // do not claim Exact. The SortExec will handle NULL ordering correctly. + if result.all_non_overlapping + && !self.output_ordering.is_empty() + && !Self::any_file_has_nulls_in_sort_columns( + &new_config.file_groups, + order, + &projected_schema, + projection_indices.as_deref(), + ) + { + // Files are now non-overlapping, no NULLs in sort columns. + // Re-ask the FileSource if this ordering satisfies the request, + // using eq_properties computed from the NEW (sorted) file groups. + let new_eq_props = new_config.eq_properties(); + if new_eq_props.ordering_satisfy(order.iter().cloned())? { + // The sorted file order makes the ordering valid → Exact! + return Ok(SortOrderPushdownResult::Exact { + inner: Arc::new(new_config), + }); + } + } + + new_config.output_ordering = vec![]; + Ok(SortOrderPushdownResult::Inexact { + inner: Arc::new(new_config), + }) + } + + /// Check if any file in any group has nulls in the sort columns. + fn any_file_has_nulls_in_sort_columns( + file_groups: &[FileGroup], + order: &[PhysicalSortExpr], + projected_schema: &SchemaRef, + projection_indices: Option<&[usize]>, + ) -> bool { + let Some(sort_columns) = + sort_columns_from_physical_sort_exprs_nullable(order, projected_schema) + else { + return true; // Can't determine, assume nulls exist + }; + + for group in file_groups { + for file in group.iter() { + let Some(stats) = file.statistics.as_ref() else { + return true; // No stats, assume nulls exist + }; + for col in &sort_columns { + let stat_idx = projection_indices + .map(|p| p[col.index()]) + .unwrap_or_else(|| col.index()); + if stat_idx >= stats.column_statistics.len() { + return true; + } + let col_stats = &stats.column_statistics[stat_idx]; + match &col_stats.null_count { + Precision::Exact(0) => {} // No nulls, safe + Precision::Exact(_) => return true, // Has nulls + _ => return true, // Unknown null count, assume nulls + } + } + } + } + false } } @@ -1437,6 +1790,62 @@ fn ordered_column_indices_from_projection( .collect::>>() } +/// Extract Column references from sort expressions for null checking. +fn sort_columns_from_physical_sort_exprs_nullable( + order: &[PhysicalSortExpr], + _schema: &SchemaRef, +) -> Option> { + order + .iter() + .map(|expr| expr.expr.as_any().downcast_ref::().cloned()) + .collect() +} + +/// Check whether a given ordering is valid for all file groups by verifying +/// that files within each group are sorted according to their min/max statistics. +/// +/// For single-file (or empty) groups, the ordering is trivially valid. +/// For multi-file groups, we check that the min/max statistics for the sort +/// columns are in order and non-overlapping (or touching at boundaries). +/// +/// `projection` maps projected column indices back to table-schema indices +/// when validating after projection; pass `None` when validating at +/// table-schema level. +fn is_ordering_valid_for_file_groups( + file_groups: &[FileGroup], + ordering: &LexOrdering, + schema: &SchemaRef, + projection: Option<&[usize]>, +) -> bool { + file_groups.iter().all(|group| { + if group.len() <= 1 { + return true; // single-file groups are trivially sorted + } + match MinMaxStatistics::new_from_files(ordering, schema, projection, group.iter()) + { + Ok(stats) => stats.is_sorted(), + Err(_) => false, // can't prove sorted → reject + } + }) +} + +/// Filters orderings to retain only those valid for all file groups, +/// verified via min/max statistics. +fn validate_orderings( + orderings: &[LexOrdering], + schema: &SchemaRef, + file_groups: &[FileGroup], + projection: Option<&[usize]>, +) -> Vec { + orderings + .iter() + .filter(|ordering| { + is_ordering_valid_for_file_groups(file_groups, ordering, schema, projection) + }) + .cloned() + .collect() +} + /// A helper that projects partition columns into the file record batches. /// /// One interesting trick is the usage of a cache for the key buffers of the partition column @@ -1936,6 +2345,22 @@ fn create_output_array( /// /// DataSourceExec /// ``` +/// +/// **Exception**: When files within a partition are **non-overlapping** (verified +/// via min/max statistics) and each file is internally sorted, the combined +/// output is still correctly sorted. Sort pushdown +/// ([`FileScanConfig::try_pushdown_sort`]) detects this case and preserves +/// `output_ordering`, allowing `SortExec` to be eliminated entirely. +/// +/// ```text +/// Partition 1 (files sorted by stats, non-overlapping): +/// ┌──────────────────┐ ┌──────────────────┐ ┌──────────────────┐ +/// │ 1.parquet │ │ 2.parquet │ │ 3.parquet │ +/// │ A: [1..100] │ │ A: [101..200] │ │ A: [201..300] │ +/// │ Sort: A, B, C │ │ Sort: A, B, C │ │ Sort: A, B, C │ +/// └──────────────────┘ └──────────────────┘ └──────────────────┘ +/// max(1) <= min(2) ✓ max(2) <= min(3) ✓ → output_ordering preserved +/// ``` fn get_projected_output_ordering( base_config: &FileScanConfig, projected_schema: &SchemaRef, @@ -1943,52 +2368,47 @@ fn get_projected_output_ordering( let projected_orderings = project_orderings(&base_config.output_ordering, projected_schema); - let mut all_orderings = vec![]; - for new_ordering in projected_orderings { - // Check if any file groups are not sorted - if base_config.file_groups.iter().any(|group| { - if group.len() <= 1 { - // File groups with <= 1 files are always sorted - return false; - } - - let Some(indices) = base_config - .file_source - .projection() - .as_ref() - .map(|p| ordered_column_indices_from_projection(p)) - else { - // Can't determine if ordered without a simple projection - return true; - }; - - let statistics = match MinMaxStatistics::new_from_files( - &new_ordering, + let indices = base_config + .file_source + .projection() + .as_ref() + .map(|p| ordered_column_indices_from_projection(p)); + + match indices { + Some(Some(indices)) => { + // Simple column projection — validate with statistics + validate_orderings( + &projected_orderings, projected_schema, - indices.as_deref(), - group.iter(), - ) { - Ok(statistics) => statistics, - Err(e) => { - log::trace!("Error fetching statistics for file group: {e}"); - // we can't prove that it's ordered, so we have to reject it - return true; - } - }; - - !statistics.is_sorted() - }) { - debug!( - "Skipping specified output ordering {:?}. \ - Some file groups couldn't be determined to be sorted: {:?}", - base_config.output_ordering[0], base_config.file_groups - ); - continue; + &base_config.file_groups, + Some(indices.as_slice()), + ) + } + None => { + // No projection — validate with statistics (no remapping needed) + validate_orderings( + &projected_orderings, + projected_schema, + &base_config.file_groups, + None, + ) + } + Some(None) => { + // Complex projection (expressions, not simple columns) — can't + // determine column indices for statistics. Still valid if all + // file groups have at most one file. + if base_config.file_groups.iter().all(|g| g.len() <= 1) { + projected_orderings + } else { + debug!( + "Skipping specified output orderings. \ + Some file groups couldn't be determined to be sorted: {:?}", + base_config.file_groups + ); + vec![] + } } - - all_orderings.push(new_ordering); } - all_orderings } /// Convert type to a type suitable for use as a `ListingTable` @@ -4501,4 +4921,675 @@ mod tests { contains an expression that references a metadata column" ); } + + fn make_file_with_stats(name: &str, min: f64, max: f64) -> PartitionedFile { + PartitionedFile::new(name.to_string(), 1024).with_statistics(Arc::new( + Statistics { + num_rows: Precision::Exact(100), + total_byte_size: Precision::Exact(1024), + column_statistics: vec![ColumnStatistics { + null_count: Precision::Exact(0), + min_value: Precision::Exact(ScalarValue::Float64(Some(min))), + max_value: Precision::Exact(ScalarValue::Float64(Some(max))), + ..Default::default() + }], + }, + )) + } + + #[derive(Clone)] + struct ExactSortPushdownSource { + metrics: ExecutionPlanMetricsSet, + table_schema: TableSchema, + } + + impl ExactSortPushdownSource { + fn new(table_schema: TableSchema) -> Self { + Self { + metrics: ExecutionPlanMetricsSet::new(), + table_schema, + } + } + } + + impl FileSource for ExactSortPushdownSource { + fn create_file_opener( + &self, + _object_store: Arc, + _base_config: &FileScanConfig, + _partition: usize, + ) -> Result> { + unimplemented!() + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn table_schema(&self) -> &TableSchema { + &self.table_schema + } + + fn with_batch_size(&self, _batch_size: usize) -> Arc { + Arc::new(self.clone()) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + &self.metrics + } + + fn file_type(&self) -> &str { + "mock_exact" + } + + fn try_reverse_output( + &self, + _order: &[PhysicalSortExpr], + _eq_properties: &EquivalenceProperties, + ) -> Result>> { + Ok(SortOrderPushdownResult::Exact { + inner: Arc::new(self.clone()) as Arc, + }) + } + } + + #[derive(Clone)] + struct InexactSortPushdownSource { + metrics: ExecutionPlanMetricsSet, + table_schema: TableSchema, + } + + impl InexactSortPushdownSource { + fn new(table_schema: TableSchema) -> Self { + Self { + metrics: ExecutionPlanMetricsSet::new(), + table_schema, + } + } + } + + impl FileSource for InexactSortPushdownSource { + fn create_file_opener( + &self, + _object_store: Arc, + _base_config: &FileScanConfig, + _partition: usize, + ) -> Result> { + unimplemented!() + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn table_schema(&self) -> &TableSchema { + &self.table_schema + } + + fn with_batch_size(&self, _batch_size: usize) -> Arc { + Arc::new(self.clone()) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + &self.metrics + } + + fn file_type(&self) -> &str { + "mock_inexact" + } + + fn try_reverse_output( + &self, + _order: &[PhysicalSortExpr], + _eq_properties: &EquivalenceProperties, + ) -> Result>> { + Ok(SortOrderPushdownResult::Inexact { + inner: Arc::new(self.clone()) as Arc, + }) + } + } + + #[test] + fn sort_pushdown_unsupported_source_files_get_sorted() -> Result<()> { + let file_schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, false)])); + let table_schema = TableSchema::new(Arc::clone(&file_schema), vec![]); + let file_source = Arc::new(MockSource::new(table_schema)); + + let file_groups = vec![FileGroup::new(vec![ + make_file_with_stats("file3", 20.0, 30.0), + make_file_with_stats("file1", 0.0, 9.0), + make_file_with_stats("file2", 10.0, 19.0), + ])]; + + let sort_expr = PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source) + .with_file_groups(file_groups) + .build(); + + let result = config.try_pushdown_sort(&[sort_expr])?; + let SortOrderPushdownResult::Inexact { inner } = result else { + panic!("Expected Inexact result, got {result:?}"); + }; + let pushed_config = inner + .as_any() + .downcast_ref::() + .expect("Expected FileScanConfig"); + let files = pushed_config.file_groups[0].files(); + assert_eq!(files[0].object_meta.location.as_ref(), "file1"); + assert_eq!(files[1].object_meta.location.as_ref(), "file2"); + assert_eq!(files[2].object_meta.location.as_ref(), "file3"); + assert!(pushed_config.output_ordering.is_empty()); + Ok(()) + } + + #[test] + fn sort_pushdown_unsupported_source_already_sorted() -> Result<()> { + let file_schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, false)])); + let table_schema = TableSchema::new(Arc::clone(&file_schema), vec![]); + let file_source = Arc::new(MockSource::new(table_schema)); + + let file_groups = vec![FileGroup::new(vec![ + make_file_with_stats("file1", 0.0, 9.0), + make_file_with_stats("file2", 10.0, 19.0), + make_file_with_stats("file3", 20.0, 30.0), + ])]; + + let sort_expr = PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source) + .with_file_groups(file_groups) + .build(); + + let result = config.try_pushdown_sort(&[sort_expr])?; + assert!(matches!(result, SortOrderPushdownResult::Unsupported)); + Ok(()) + } + + #[test] + fn sort_pushdown_unsupported_source_descending_sort() -> Result<()> { + let file_schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, false)])); + let table_schema = TableSchema::new(Arc::clone(&file_schema), vec![]); + let file_source = Arc::new(MockSource::new(table_schema)); + + let file_groups = vec![FileGroup::new(vec![ + make_file_with_stats("file1", 0.0, 9.0), + make_file_with_stats("file3", 20.0, 30.0), + make_file_with_stats("file2", 10.0, 19.0), + ])]; + + let sort_expr = PhysicalSortExpr::new( + Arc::new(Column::new("a", 0)), + arrow::compute::SortOptions { + descending: true, + nulls_first: true, + }, + ); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source) + .with_file_groups(file_groups) + .build(); + + let result = config.try_pushdown_sort(&[sort_expr])?; + let SortOrderPushdownResult::Inexact { inner } = result else { + panic!("Expected Inexact result"); + }; + let pushed_config = inner + .as_any() + .downcast_ref::() + .expect("Expected FileScanConfig"); + let files = pushed_config.file_groups[0].files(); + assert_eq!(files[0].object_meta.location.as_ref(), "file3"); + assert_eq!(files[1].object_meta.location.as_ref(), "file2"); + assert_eq!(files[2].object_meta.location.as_ref(), "file1"); + Ok(()) + } + + #[test] + fn sort_pushdown_exact_source_non_overlapping_returns_exact() -> Result<()> { + let file_schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, false)])); + let table_schema = TableSchema::new(Arc::clone(&file_schema), vec![]); + let file_source = Arc::new(ExactSortPushdownSource::new(table_schema)); + + let sort_expr = PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))); + + let file_groups = vec![FileGroup::new(vec![ + make_file_with_stats("file1", 0.0, 9.0), + make_file_with_stats("file2", 10.0, 19.0), + make_file_with_stats("file3", 20.0, 30.0), + ])]; + + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source) + .with_file_groups(file_groups) + .with_output_ordering(vec![ + LexOrdering::new(vec![sort_expr.clone()]).unwrap(), + ]) + .build(); + + let result = config.try_pushdown_sort(&[sort_expr])?; + let SortOrderPushdownResult::Exact { inner } = result else { + panic!("Expected Exact result, got {result:?}"); + }; + let pushed_config = inner + .as_any() + .downcast_ref::() + .expect("Expected FileScanConfig"); + assert!(!pushed_config.output_ordering.is_empty()); + Ok(()) + } + + #[test] + fn sort_pushdown_exact_source_overlapping_downgraded_to_inexact() -> Result<()> { + let file_schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, false)])); + let table_schema = TableSchema::new(Arc::clone(&file_schema), vec![]); + let file_source = Arc::new(ExactSortPushdownSource::new(table_schema)); + + let sort_expr = PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))); + + let file_groups = vec![FileGroup::new(vec![ + make_file_with_stats("file1", 0.0, 15.0), + make_file_with_stats("file2", 10.0, 25.0), + make_file_with_stats("file3", 20.0, 30.0), + ])]; + + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source) + .with_file_groups(file_groups) + .with_output_ordering(vec![ + LexOrdering::new(vec![sort_expr.clone()]).unwrap(), + ]) + .build(); + + let result = config.try_pushdown_sort(&[sort_expr])?; + let SortOrderPushdownResult::Inexact { inner } = result else { + panic!("Expected Inexact (downgraded), got {result:?}"); + }; + let pushed_config = inner + .as_any() + .downcast_ref::() + .expect("Expected FileScanConfig"); + assert!(pushed_config.output_ordering.is_empty()); + Ok(()) + } + + #[test] + fn sort_pushdown_exact_source_out_of_order_returns_exact() -> Result<()> { + let file_schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, false)])); + let table_schema = TableSchema::new(Arc::clone(&file_schema), vec![]); + let file_source = Arc::new(ExactSortPushdownSource::new(table_schema)); + + let sort_expr = PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))); + + let file_groups = vec![FileGroup::new(vec![ + make_file_with_stats("file3", 20.0, 30.0), + make_file_with_stats("file1", 0.0, 9.0), + make_file_with_stats("file2", 10.0, 19.0), + ])]; + + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source) + .with_file_groups(file_groups) + .with_output_ordering(vec![ + LexOrdering::new(vec![sort_expr.clone()]).unwrap(), + ]) + .build(); + + let result = config.try_pushdown_sort(&[sort_expr])?; + let SortOrderPushdownResult::Exact { inner } = result else { + panic!("Expected Exact result, got {result:?}"); + }; + let pushed_config = inner + .as_any() + .downcast_ref::() + .expect("Expected FileScanConfig"); + let files = pushed_config.file_groups[0].files(); + assert_eq!(files[0].object_meta.location.as_ref(), "file1"); + assert_eq!(files[1].object_meta.location.as_ref(), "file2"); + assert_eq!(files[2].object_meta.location.as_ref(), "file3"); + assert!(!pushed_config.output_ordering.is_empty()); + Ok(()) + } + + #[test] + fn sort_pushdown_unsupported_source_single_file_groups() -> Result<()> { + let file_schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, false)])); + let table_schema = TableSchema::new(Arc::clone(&file_schema), vec![]); + let file_source = Arc::new(MockSource::new(table_schema)); + + let file_groups = vec![ + FileGroup::new(vec![make_file_with_stats("file1", 0.0, 9.0)]), + FileGroup::new(vec![make_file_with_stats("file2", 10.0, 19.0)]), + ]; + + let sort_expr = PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source) + .with_file_groups(file_groups) + .build(); + + let result = config.try_pushdown_sort(&[sort_expr])?; + assert!( + matches!(result, SortOrderPushdownResult::Unsupported), + "Expected Unsupported for single-file groups" + ); + Ok(()) + } + + #[test] + fn sort_pushdown_unsupported_source_multiple_groups() -> Result<()> { + let file_schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, false)])); + let table_schema = TableSchema::new(Arc::clone(&file_schema), vec![]); + let file_source = Arc::new(MockSource::new(table_schema)); + + let file_groups = vec![ + FileGroup::new(vec![ + make_file_with_stats("file_b", 10.0, 19.0), + make_file_with_stats("file_a", 0.0, 9.0), + ]), + FileGroup::new(vec![ + make_file_with_stats("file_d", 30.0, 39.0), + make_file_with_stats("file_c", 20.0, 29.0), + ]), + ]; + + let sort_expr = PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source) + .with_file_groups(file_groups) + .build(); + + let result = config.try_pushdown_sort(&[sort_expr])?; + let SortOrderPushdownResult::Inexact { inner } = result else { + panic!("Expected Inexact result"); + }; + let pushed_config = inner + .as_any() + .downcast_ref::() + .expect("Expected FileScanConfig"); + let files0 = pushed_config.file_groups[0].files(); + assert_eq!(files0[0].object_meta.location.as_ref(), "file_a"); + assert_eq!(files0[1].object_meta.location.as_ref(), "file_b"); + let files1 = pushed_config.file_groups[1].files(); + assert_eq!(files1[0].object_meta.location.as_ref(), "file_c"); + assert_eq!(files1[1].object_meta.location.as_ref(), "file_d"); + Ok(()) + } + + #[test] + fn sort_pushdown_unsupported_source_partial_statistics() -> Result<()> { + let file_schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, false)])); + let table_schema = TableSchema::new(Arc::clone(&file_schema), vec![]); + let file_source = Arc::new(MockSource::new(table_schema)); + + let file_groups = vec![ + FileGroup::new(vec![ + make_file_with_stats("file_b", 10.0, 19.0), + make_file_with_stats("file_a", 0.0, 9.0), + ]), + FileGroup::new(vec![ + PartitionedFile::new("file_d".to_string(), 1024), + PartitionedFile::new("file_c".to_string(), 1024), + ]), + ]; + + let sort_expr = PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source) + .with_file_groups(file_groups) + .build(); + + let result = config.try_pushdown_sort(&[sort_expr])?; + let SortOrderPushdownResult::Inexact { inner } = result else { + panic!("Expected Inexact result"); + }; + let pushed_config = inner + .as_any() + .downcast_ref::() + .expect("Expected FileScanConfig"); + let files0 = pushed_config.file_groups[0].files(); + assert_eq!(files0[0].object_meta.location.as_ref(), "file_a"); + assert_eq!(files0[1].object_meta.location.as_ref(), "file_b"); + let files1 = pushed_config.file_groups[1].files(); + assert_eq!(files1[0].object_meta.location.as_ref(), "file_d"); + assert_eq!(files1[1].object_meta.location.as_ref(), "file_c"); + Ok(()) + } + + #[test] + fn sort_pushdown_inexact_source_with_statistics_sorting() -> Result<()> { + let file_schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, false)])); + let table_schema = TableSchema::new(Arc::clone(&file_schema), vec![]); + let file_source = Arc::new(InexactSortPushdownSource::new(table_schema)); + + let file_groups = vec![FileGroup::new(vec![ + make_file_with_stats("file2", 10.0, 19.0), + make_file_with_stats("file1", 0.0, 9.0), + ])]; + + let sort_expr = PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source) + .with_file_groups(file_groups) + .build(); + + let result = config.try_pushdown_sort(&[sort_expr])?; + let SortOrderPushdownResult::Inexact { inner } = result else { + panic!("Expected Inexact result"); + }; + let pushed_config = inner + .as_any() + .downcast_ref::() + .expect("Expected FileScanConfig"); + let files = pushed_config.file_groups[0].files(); + assert_eq!(files[0].object_meta.location.as_ref(), "file1"); + assert_eq!(files[1].object_meta.location.as_ref(), "file2"); + assert!(pushed_config.output_ordering.is_empty()); + Ok(()) + } + + #[test] + fn sort_pushdown_exact_multi_group_preserves_parallelism() -> Result<()> { + // ExactSortPushdownSource + 4 non-overlapping files in 2 interleaved groups. + // Groups should NOT be redistributed — interleaved groups allow SPM to + // pull from both partitions concurrently, keeping parallel I/O active. + // Redistributing consecutively would make SPM read one partition at a + // time (all values in group 0 < group 1), degrading to single-threaded I/O. + let file_schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, false)])); + let table_schema = TableSchema::new(Arc::clone(&file_schema), vec![]); + let file_source = Arc::new(ExactSortPushdownSource::new(table_schema)); + + let sort_expr = PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))); + + // 2 groups with interleaved ranges (simulating bin-packing result): + // Group 0: [file_01(0-9), file_03(20-29)] + // Group 1: [file_02(10-19), file_04(30-39)] + let file_groups = vec![ + FileGroup::new(vec![ + make_file_with_stats("file_01", 0.0, 9.0), + make_file_with_stats("file_03", 20.0, 29.0), + ]), + FileGroup::new(vec![ + make_file_with_stats("file_02", 10.0, 19.0), + make_file_with_stats("file_04", 30.0, 39.0), + ]), + ]; + + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source) + .with_file_groups(file_groups) + .with_output_ordering(vec![ + LexOrdering::new(vec![sort_expr.clone()]).unwrap(), + ]) + .build(); + + let result = config.try_pushdown_sort(&[sort_expr])?; + let SortOrderPushdownResult::Exact { inner } = result else { + panic!("Expected Exact result, got {result:?}"); + }; + let pushed_config = inner + .as_any() + .downcast_ref::() + .expect("Expected FileScanConfig"); + + // 2 groups preserved (parallelism maintained) + assert_eq!(pushed_config.file_groups.len(), 2); + + // Files within each group are sorted by stats, but groups are NOT + // redistributed — interleaved assignment from bin-packing is kept + let files0 = pushed_config.file_groups[0].files(); + assert_eq!(files0[0].object_meta.location.as_ref(), "file_01"); + assert_eq!(files0[1].object_meta.location.as_ref(), "file_03"); + let files1 = pushed_config.file_groups[1].files(); + assert_eq!(files1[0].object_meta.location.as_ref(), "file_02"); + assert_eq!(files1[1].object_meta.location.as_ref(), "file_04"); + + // output_ordering preserved (Exact, each group internally non-overlapping) + assert!(!pushed_config.output_ordering.is_empty()); + Ok(()) + } + + #[test] + fn sort_pushdown_reverse_preserves_file_order_with_stats() -> Result<()> { + // Reverse scan should reverse file order but NOT apply statistics-based + // sorting (which would undo the reversal). The result is Inexact. + let file_schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, false)])); + let table_schema = TableSchema::new(Arc::clone(&file_schema), vec![]); + let file_source = Arc::new(InexactSortPushdownSource::new(table_schema)); + + let sort_expr = PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))); + + // Files with stats, in ASC order. Output ordering is [a ASC]. + let file_groups = vec![FileGroup::new(vec![ + make_file_with_stats("file1", 0.0, 9.0), + make_file_with_stats("file2", 10.0, 19.0), + make_file_with_stats("file3", 20.0, 30.0), + ])]; + + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source) + .with_file_groups(file_groups) + .with_output_ordering(vec![ + LexOrdering::new(vec![sort_expr.clone()]).unwrap(), + ]) + .build(); + + // Request DESC → reverse path + let result = config.try_pushdown_sort(&[sort_expr.reverse()])?; + let SortOrderPushdownResult::Inexact { inner } = result else { + panic!("Expected Inexact for reverse scan, got {result:?}"); + }; + let pushed_config = inner + .as_any() + .downcast_ref::() + .expect("Expected FileScanConfig"); + + // Files should be reversed (not re-sorted by stats) + let files = pushed_config.file_groups[0].files(); + assert_eq!(files[0].object_meta.location.as_ref(), "file3"); + assert_eq!(files[1].object_meta.location.as_ref(), "file2"); + assert_eq!(files[2].object_meta.location.as_ref(), "file1"); + + // output_ordering cleared (Inexact) + assert!(pushed_config.output_ordering.is_empty()); + Ok(()) + } + + /// Helper: create a PartitionedFile with stats including null count + fn make_file_with_null_stats( + name: &str, + min: f64, + max: f64, + null_count: usize, + ) -> PartitionedFile { + PartitionedFile::new(name.to_string(), 1024).with_statistics(Arc::new( + Statistics { + num_rows: Precision::Exact(100), + total_byte_size: Precision::Exact(1024), + column_statistics: vec![ColumnStatistics { + null_count: Precision::Exact(null_count), + min_value: Precision::Exact(ScalarValue::Float64(Some(min))), + max_value: Precision::Exact(ScalarValue::Float64(Some(max))), + ..Default::default() + }], + }, + )) + } + + #[test] + fn sort_pushdown_unsupported_with_nulls_does_not_upgrade_to_exact() -> Result<()> { + // Files are non-overlapping but one has NULLs. + // Should NOT upgrade to Exact — NULLs would appear in wrong position. + let file_schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, true)])); + let table_schema = TableSchema::new(Arc::clone(&file_schema), vec![]); + let file_source = Arc::new(MockSource::new(table_schema)); + + let sort_expr = PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))); + + // Files in wrong order (high min first) to trigger reordering + let file_groups = vec![FileGroup::new(vec![ + make_file_with_null_stats("b_no_nulls", 10.0, 19.0, 0), + make_file_with_null_stats("a_with_nulls", 0.0, 9.0, 5), // has NULLs + ])]; + + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source) + .with_file_groups(file_groups) + .with_output_ordering(vec![ + LexOrdering::new(vec![sort_expr.clone()]).unwrap(), + ]) + .build(); + + let result = config.try_pushdown_sort(&[sort_expr])?; + // Should be Inexact (not Exact) because of NULLs + assert!( + matches!(result, SortOrderPushdownResult::Inexact { .. }), + "Expected Inexact due to NULLs, got {result:?}" + ); + Ok(()) + } + + #[test] + fn sort_pushdown_unsupported_no_nulls_upgrades_to_exact() -> Result<()> { + // Files are non-overlapping, no NULLs → should upgrade to Exact + let file_schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, true)])); + let table_schema = TableSchema::new(Arc::clone(&file_schema), vec![]); + let file_source = Arc::new(MockSource::new(table_schema)); + + let sort_expr = PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))); + + let file_groups = vec![FileGroup::new(vec![ + make_file_with_null_stats("b_high", 10.0, 19.0, 0), + make_file_with_null_stats("a_low", 0.0, 9.0, 0), + ])]; + + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source) + .with_file_groups(file_groups) + .with_output_ordering(vec![ + LexOrdering::new(vec![sort_expr.clone()]).unwrap(), + ]) + .build(); + + let result = config.try_pushdown_sort(&[sort_expr])?; + assert!( + matches!(result, SortOrderPushdownResult::Exact { .. }), + "Expected Exact (no NULLs), got {result:?}" + ); + Ok(()) + } } diff --git a/datafusion/execution/src/memory_pool/mod.rs b/datafusion/execution/src/memory_pool/mod.rs index fbf9ce41da8fe..b23eede2a054e 100644 --- a/datafusion/execution/src/memory_pool/mod.rs +++ b/datafusion/execution/src/memory_pool/mod.rs @@ -18,7 +18,7 @@ //! [`MemoryPool`] for memory management during query execution, [`proxy`] for //! help with allocation accounting. -use datafusion_common::{Result, internal_err}; +use datafusion_common::{Result, internal_datafusion_err}; use std::hash::{Hash, Hasher}; use std::{cmp::Ordering, sync::Arc, sync::atomic}; @@ -322,7 +322,7 @@ impl MemoryConsumer { pool: Arc::clone(pool), consumer: self, }), - size: 0, + size: atomic::AtomicUsize::new(0), } } } @@ -351,13 +351,13 @@ impl Drop for SharedRegistration { #[derive(Debug)] pub struct MemoryReservation { registration: Arc, - size: usize, + size: atomic::AtomicUsize, } impl MemoryReservation { /// Returns the size of this reservation in bytes pub fn size(&self) -> usize { - self.size + self.size.load(atomic::Ordering::Relaxed) } /// Returns [MemoryConsumer] for this [MemoryReservation] @@ -367,8 +367,8 @@ impl MemoryReservation { /// Frees all bytes from this reservation back to the underlying /// pool, returning the number of bytes freed. - pub fn free(&mut self) -> usize { - let size = self.size; + pub fn free(&self) -> usize { + let size = self.size.load(atomic::Ordering::Relaxed); if size != 0 { self.shrink(size) } @@ -380,60 +380,62 @@ impl MemoryReservation { /// # Panics /// /// Panics if `capacity` exceeds [`Self::size`] - pub fn shrink(&mut self, capacity: usize) { - let new_size = self.size.checked_sub(capacity).unwrap(); + pub fn shrink(&self, capacity: usize) { + self.size.fetch_sub(capacity, atomic::Ordering::Relaxed); self.registration.pool.shrink(self, capacity); - self.size = new_size } /// Tries to free `capacity` bytes from this reservation /// if `capacity` does not exceed [`Self::size`] /// Returns new reservation size /// or error if shrinking capacity is more than allocated size - pub fn try_shrink(&mut self, capacity: usize) -> Result { - if let Some(new_size) = self.size.checked_sub(capacity) { - self.registration.pool.shrink(self, capacity); - self.size = new_size; - Ok(new_size) - } else { - internal_err!( - "Cannot free the capacity {capacity} out of allocated size {}", - self.size + pub fn try_shrink(&self, capacity: usize) -> Result { + let updated = self.size.fetch_update( + atomic::Ordering::Relaxed, + atomic::Ordering::Relaxed, + |prev| prev.checked_sub(capacity), + ); + updated.map_err(|_| { + let prev = self.size.load(atomic::Ordering::Relaxed); + internal_datafusion_err!( + "Cannot free the capacity {capacity} out of allocated size {prev}" ) - } + }) } /// Sets the size of this reservation to `capacity` - pub fn resize(&mut self, capacity: usize) { - match capacity.cmp(&self.size) { - Ordering::Greater => self.grow(capacity - self.size), - Ordering::Less => self.shrink(self.size - capacity), + pub fn resize(&self, capacity: usize) { + let size = self.size.load(atomic::Ordering::Relaxed); + match capacity.cmp(&size) { + Ordering::Greater => self.grow(capacity - size), + Ordering::Less => self.shrink(size - capacity), _ => {} } } /// Try to set the size of this reservation to `capacity` - pub fn try_resize(&mut self, capacity: usize) -> Result<()> { - match capacity.cmp(&self.size) { - Ordering::Greater => self.try_grow(capacity - self.size)?, - Ordering::Less => self.shrink(self.size - capacity), + pub fn try_resize(&self, capacity: usize) -> Result<()> { + let size = self.size.load(atomic::Ordering::Relaxed); + match capacity.cmp(&size) { + Ordering::Greater => self.try_grow(capacity - size)?, + Ordering::Less => self.shrink(size - capacity), _ => {} }; Ok(()) } /// Increase the size of this reservation by `capacity` bytes - pub fn grow(&mut self, capacity: usize) { + pub fn grow(&self, capacity: usize) { self.registration.pool.grow(self, capacity); - self.size += capacity; + self.size.fetch_add(capacity, atomic::Ordering::Relaxed); } /// Try to increase the size of this reservation by `capacity` /// bytes, returning error if there is insufficient capacity left /// in the pool. - pub fn try_grow(&mut self, capacity: usize) -> Result<()> { + pub fn try_grow(&self, capacity: usize) -> Result<()> { self.registration.pool.try_grow(self, capacity)?; - self.size += capacity; + self.size.fetch_add(capacity, atomic::Ordering::Relaxed); Ok(()) } @@ -447,10 +449,16 @@ impl MemoryReservation { /// # Panics /// /// Panics if `capacity` exceeds [`Self::size`] - pub fn split(&mut self, capacity: usize) -> MemoryReservation { - self.size = self.size.checked_sub(capacity).unwrap(); + pub fn split(&self, capacity: usize) -> MemoryReservation { + self.size + .fetch_update( + atomic::Ordering::Relaxed, + atomic::Ordering::Relaxed, + |prev| prev.checked_sub(capacity), + ) + .unwrap(); Self { - size: capacity, + size: atomic::AtomicUsize::new(capacity), registration: Arc::clone(&self.registration), } } @@ -458,7 +466,7 @@ impl MemoryReservation { /// Returns a new empty [`MemoryReservation`] with the same [`MemoryConsumer`] pub fn new_empty(&self) -> Self { Self { - size: 0, + size: atomic::AtomicUsize::new(0), registration: Arc::clone(&self.registration), } } @@ -466,7 +474,7 @@ impl MemoryReservation { /// Splits off all the bytes from this [`MemoryReservation`] into /// a new [`MemoryReservation`] with the same [`MemoryConsumer`] pub fn take(&mut self) -> MemoryReservation { - self.split(self.size) + self.split(self.size.load(atomic::Ordering::Relaxed)) } } @@ -492,7 +500,7 @@ mod tests { #[test] fn test_memory_pool_underflow() { let pool = Arc::new(GreedyMemoryPool::new(50)) as _; - let mut a1 = MemoryConsumer::new("a1").register(&pool); + let a1 = MemoryConsumer::new("a1").register(&pool); assert_eq!(pool.reserved(), 0); a1.grow(100); @@ -507,7 +515,7 @@ mod tests { a1.try_grow(30).unwrap(); assert_eq!(pool.reserved(), 30); - let mut a2 = MemoryConsumer::new("a2").register(&pool); + let a2 = MemoryConsumer::new("a2").register(&pool); a2.try_grow(25).unwrap_err(); assert_eq!(pool.reserved(), 30); @@ -521,7 +529,7 @@ mod tests { #[test] fn test_split() { let pool = Arc::new(GreedyMemoryPool::new(50)) as _; - let mut r1 = MemoryConsumer::new("r1").register(&pool); + let r1 = MemoryConsumer::new("r1").register(&pool); r1.try_grow(20).unwrap(); assert_eq!(r1.size(), 20); @@ -542,10 +550,10 @@ mod tests { #[test] fn test_new_empty() { let pool = Arc::new(GreedyMemoryPool::new(50)) as _; - let mut r1 = MemoryConsumer::new("r1").register(&pool); + let r1 = MemoryConsumer::new("r1").register(&pool); r1.try_grow(20).unwrap(); - let mut r2 = r1.new_empty(); + let r2 = r1.new_empty(); r2.try_grow(5).unwrap(); assert_eq!(r1.size(), 20); @@ -559,7 +567,7 @@ mod tests { let mut r1 = MemoryConsumer::new("r1").register(&pool); r1.try_grow(20).unwrap(); - let mut r2 = r1.take(); + let r2 = r1.take(); r2.try_grow(5).unwrap(); assert_eq!(r1.size(), 0); diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index bf74b5f6f4c6b..b10270851cc06 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -212,7 +212,7 @@ impl MemoryPool for FairSpillPool { .checked_div(state.num_spill) .unwrap_or(spill_available); - if reservation.size + additional > available { + if reservation.size() + additional > available { return Err(insufficient_capacity_err( reservation, additional, @@ -264,7 +264,7 @@ fn insufficient_capacity_err( "Failed to allocate additional {} for {} with {} already allocated for this reservation - {} remain available for the total pool", human_readable_size(additional), reservation.registration.consumer.name, - human_readable_size(reservation.size), + human_readable_size(reservation.size()), human_readable_size(available) ) } @@ -526,12 +526,12 @@ mod tests { fn test_fair() { let pool = Arc::new(FairSpillPool::new(100)) as _; - let mut r1 = MemoryConsumer::new("unspillable").register(&pool); + let r1 = MemoryConsumer::new("unspillable").register(&pool); // Can grow beyond capacity of pool r1.grow(2000); assert_eq!(pool.reserved(), 2000); - let mut r2 = MemoryConsumer::new("r2") + let r2 = MemoryConsumer::new("r2") .with_can_spill(true) .register(&pool); // Can grow beyond capacity of pool @@ -563,7 +563,7 @@ mod tests { assert_eq!(r2.size(), 10); assert_eq!(pool.reserved(), 30); - let mut r3 = MemoryConsumer::new("r3") + let r3 = MemoryConsumer::new("r3") .with_can_spill(true) .register(&pool); @@ -584,7 +584,7 @@ mod tests { r1.free(); assert_eq!(pool.reserved(), 80); - let mut r4 = MemoryConsumer::new("s4").register(&pool); + let r4 = MemoryConsumer::new("s4").register(&pool); let err = r4.try_grow(30).unwrap_err().strip_backtrace(); assert_snapshot!(err, @"Resources exhausted: Failed to allocate additional 30.0 B for s4 with 0.0 B already allocated for this reservation - 20.0 B remain available for the total pool"); } @@ -601,18 +601,18 @@ mod tests { // Test: use all the different interfaces to change reservation size // set r1=50, using grow and shrink - let mut r1 = MemoryConsumer::new("r1").register(&pool); + let r1 = MemoryConsumer::new("r1").register(&pool); r1.grow(50); r1.grow(20); r1.shrink(20); // set r2=15 using try_grow - let mut r2 = MemoryConsumer::new("r2").register(&pool); + let r2 = MemoryConsumer::new("r2").register(&pool); r2.try_grow(15) .expect("should succeed in memory allotment for r2"); // set r3=20 using try_resize - let mut r3 = MemoryConsumer::new("r3").register(&pool); + let r3 = MemoryConsumer::new("r3").register(&pool); r3.try_resize(25) .expect("should succeed in memory allotment for r3"); r3.try_resize(20) @@ -620,12 +620,12 @@ mod tests { // set r4=10 // this should not be reported in top 3 - let mut r4 = MemoryConsumer::new("r4").register(&pool); + let r4 = MemoryConsumer::new("r4").register(&pool); r4.grow(10); // Test: reports if new reservation causes error // using the previously set sizes for other consumers - let mut r5 = MemoryConsumer::new("r5").register(&pool); + let r5 = MemoryConsumer::new("r5").register(&pool); let res = r5.try_grow(150); assert!(res.is_err()); let error = res.unwrap_err().strip_backtrace(); @@ -650,7 +650,7 @@ mod tests { let same_name = "foo"; // Test: see error message when no consumers recorded yet - let mut r0 = MemoryConsumer::new(same_name).register(&pool); + let r0 = MemoryConsumer::new(same_name).register(&pool); let res = r0.try_grow(150); assert!(res.is_err()); let error = res.unwrap_err().strip_backtrace(); @@ -665,7 +665,7 @@ mod tests { r0.grow(10); // make r0=10, pool available=90 let new_consumer_same_name = MemoryConsumer::new(same_name); - let mut r1 = new_consumer_same_name.register(&pool); + let r1 = new_consumer_same_name.register(&pool); // TODO: the insufficient_capacity_err() message is per reservation, not per consumer. // a followup PR will clarify this message "0 bytes already allocated for this reservation" let res = r1.try_grow(150); @@ -695,7 +695,7 @@ mod tests { // will be recognized as different in the TrackConsumersPool let consumer_with_same_name_but_different_hash = MemoryConsumer::new(same_name).with_can_spill(true); - let mut r2 = consumer_with_same_name_but_different_hash.register(&pool); + let r2 = consumer_with_same_name_but_different_hash.register(&pool); let res = r2.try_grow(150); assert!(res.is_err()); let error = res.unwrap_err().strip_backtrace(); @@ -714,10 +714,10 @@ mod tests { // Baseline: see the 2 memory consumers let setting = make_settings(); let _bound = setting.bind_to_scope(); - let mut r0 = MemoryConsumer::new("r0").register(&pool); + let r0 = MemoryConsumer::new("r0").register(&pool); r0.grow(10); let r1_consumer = MemoryConsumer::new("r1"); - let mut r1 = r1_consumer.register(&pool); + let r1 = r1_consumer.register(&pool); r1.grow(20); let res = r0.try_grow(150); @@ -791,13 +791,13 @@ mod tests { .downcast::>() .unwrap(); // set r1=20 - let mut r1 = MemoryConsumer::new("r1").register(&pool); + let r1 = MemoryConsumer::new("r1").register(&pool); r1.grow(20); // set r2=15 - let mut r2 = MemoryConsumer::new("r2").register(&pool); + let r2 = MemoryConsumer::new("r2").register(&pool); r2.grow(15); // set r3=45 - let mut r3 = MemoryConsumer::new("r3").register(&pool); + let r3 = MemoryConsumer::new("r3").register(&pool); r3.grow(45); let downcasted = upcasted diff --git a/datafusion/physical-optimizer/src/pushdown_sort.rs b/datafusion/physical-optimizer/src/pushdown_sort.rs index 9f5f590385393..ef7da517789d8 100644 --- a/datafusion/physical-optimizer/src/pushdown_sort.rs +++ b/datafusion/physical-optimizer/src/pushdown_sort.rs @@ -35,19 +35,24 @@ //! - `Inexact`: Keep Sort but use optimized input (enables early termination for TopK) //! - `Unsupported`: No change //! -//! ## Current capabilities (Phase 1) +//! ## Capabilities //! -//! - Reverse scan optimization: when required sort is the reverse of the data source's +//! - **Sort elimination**: when a data source's natural ordering satisfies the +//! request, return `Exact` and remove the `SortExec` entirely. Preserves +//! `fetch` (LIMIT) from the eliminated `SortExec` for early termination. +//! - **Statistics-based file sorting**: sort files within each partition by +//! min/max statistics. When files are non-overlapping but listed in wrong +//! order (e.g., alphabetical order ≠ sort key order), this fixes the ordering +//! and enables sort elimination. Works for both single-partition and +//! multi-partition plans with multi-file groups. +//! - **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 +//! - **Prefix matching**: if data has ordering [A DESC, B ASC] and query needs +//! [A DESC], the existing ordering satisfies the requirement (`Exact`). +//! If the query needs [A ASC] (reverse of the prefix), a reverse scan is +//! used (`Inexact`, `SortExec` retained) //! -//! 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 +//! Related issue: use crate::PhysicalOptimizerRule; use datafusion_common::Result; @@ -55,10 +60,25 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_physical_plan::ExecutionPlan; use datafusion_physical_plan::SortOrderPushdownResult; -use datafusion_physical_plan::limit::GlobalLimitExec; +use datafusion_physical_plan::buffer::BufferExec; use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use std::sync::Arc; +/// Per-partition buffer capacity (in bytes) inserted between SPM and +/// DataSourceExec when sort elimination removes the buffering SortExec. +/// +/// SortExec buffers all input data in memory (potentially GB per partition) +/// before outputting sorted results. When we eliminate SortExec, SPM reads +/// directly from I/O-bound sources. BufferExec compensates with bounded +/// buffering, allowing I/O to pipeline with merge computation. +/// +/// This is strictly less memory than the SortExec it replaces, and only +/// inserted when PushdownSort eliminates a SortExec — no impact on other +/// query plans. BufferExec also integrates with MemoryPool, so it respects +/// the global memory limit and won't cause OOM. +const BUFFER_CAPACITY_AFTER_SORT_ELIMINATION: usize = 64 * 1024 * 1024; // 64 MB + /// A PhysicalOptimizerRule that attempts to push down sort requirements to data sources. /// /// See module-level documentation for details. @@ -83,8 +103,54 @@ impl PhysicalOptimizerRule for PushdownSort { } // Use transform_down to find and optimize all SortExec nodes (including nested ones) + // Also handles SPM → SortExec pattern to insert BufferExec when sort is eliminated plan.transform_down(|plan: Arc| { - // Check if this is a SortExec + // Pattern 1: SPM → SortExec(preserve_partitioning) + // When we eliminate the SortExec, SPM loses its memory buffer and reads + // directly from I/O-bound sources. Insert a BufferExec to compensate. + if let Some(spm) = plan.as_any().downcast_ref::() + && let Some(sort_child) = spm.input().as_any().downcast_ref::() + && sort_child.preserve_partitioning() + { + let sort_input = Arc::clone(sort_child.input()); + let required_ordering = sort_child.expr(); + match sort_input.try_pushdown_sort(required_ordering)? { + SortOrderPushdownResult::Exact { inner } => { + let inner = if let Some(fetch) = sort_child.fetch() { + inner.with_fetch(Some(fetch)).unwrap_or(inner) + } else { + inner + }; + // Insert BufferExec to replace SortExec's buffering role. + // SortExec buffered all data in memory; BufferExec provides + // bounded buffering so SPM doesn't stall on I/O. + let buffered: Arc = Arc::new(BufferExec::new( + inner, + BUFFER_CAPACITY_AFTER_SORT_ELIMINATION, + )); + let new_spm = + SortPreservingMergeExec::new(spm.expr().clone(), buffered) + .with_fetch(spm.fetch()); + return Ok(Transformed::yes(Arc::new(new_spm))); + } + SortOrderPushdownResult::Inexact { inner } => { + let new_sort = SortExec::new(required_ordering.clone(), inner) + .with_fetch(sort_child.fetch()) + .with_preserve_partitioning(true); + let new_spm = SortPreservingMergeExec::new( + spm.expr().clone(), + Arc::new(new_sort), + ) + .with_fetch(spm.fetch()); + return Ok(Transformed::yes(Arc::new(new_spm))); + } + SortOrderPushdownResult::Unsupported => { + return Ok(Transformed::no(plan)); + } + } + } + + // Pattern 2: Standalone SortExec (no SPM parent) let Some(sort_exec) = plan.as_any().downcast_ref::() else { return Ok(Transformed::no(plan)); }; @@ -97,26 +163,14 @@ impl PhysicalOptimizerRule for PushdownSort { match sort_input.try_pushdown_sort(required_ordering)? { SortOrderPushdownResult::Exact { inner } => { // Data source guarantees perfect ordering - remove the Sort operator. - // - // If the SortExec carried a fetch (LIMIT), we must preserve it. - // First try pushing the limit into the source via `with_fetch()` - // If the source doesn't support `with_fetch`, fall back to - // wrapping with GlobalLimitExec. - // - // Note: LimitPushdown runs *before* PushdownSort in the optimizer - // pipeline, so we need to handle the limit manually here. - if let Some(fetch) = sort_exec.fetch() { - let limited = inner - .with_fetch(Some(fetch)) - .unwrap_or_else(|| { - Arc::new(GlobalLimitExec::new( - inner, 0, Some(fetch), - )) - }); - Ok(Transformed::yes(limited)) + // Preserve the fetch (LIMIT) from the original SortExec so the + // data source can stop reading early. + let inner = if let Some(fetch) = sort_exec.fetch() { + inner.with_fetch(Some(fetch)).unwrap_or(inner) } else { - Ok(Transformed::yes(inner)) - } + inner + }; + Ok(Transformed::yes(inner)) } SortOrderPushdownResult::Inexact { inner } => { // Data source is optimized for the ordering but not perfectly sorted diff --git a/datafusion/physical-plan/src/buffer.rs b/datafusion/physical-plan/src/buffer.rs new file mode 100644 index 0000000000000..3b80f9924e311 --- /dev/null +++ b/datafusion/physical-plan/src/buffer.rs @@ -0,0 +1,629 @@ +// 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. + +//! [`BufferExec`] decouples production and consumption on messages by buffering the input in the +//! background up to a certain capacity. + +use crate::execution_plan::{CardinalityEffect, SchedulingType}; +use crate::filter_pushdown::{ + ChildPushdownResult, FilterDescription, FilterPushdownPhase, + FilterPushdownPropagation, +}; +use crate::projection::ProjectionExec; +use crate::stream::RecordBatchStreamAdapter; +use crate::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, SortOrderPushdownResult, +}; +use arrow::array::RecordBatch; +use datafusion_common::config::ConfigOptions; +use datafusion_common::{Result, Statistics, internal_err, plan_err}; +use datafusion_common_runtime::SpawnedTask; +use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr_common::metrics::{ + ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, +}; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use futures::{Stream, StreamExt, TryStreamExt}; +use pin_project_lite::pin_project; +use std::any::Any; +use std::fmt; +use std::pin::Pin; +use std::sync::Arc; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::task::{Context, Poll}; +use tokio::sync::mpsc::UnboundedReceiver; +use tokio::sync::{OwnedSemaphorePermit, Semaphore}; + +/// WARNING: EXPERIMENTAL +/// +/// Decouples production and consumption of record batches with an internal queue per partition, +/// eagerly filling up the capacity of the queues even before any message is requested. +/// +/// ```text +/// ┌───────────────────────────┐ +/// │ BufferExec │ +/// │ │ +/// │┌────── Partition 0 ──────┐│ +/// ││ ┌────┐ ┌────┐││ ┌────┐ +/// ──background poll────────▶│ │ │ ├┼┼───────▶ │ +/// ││ └────┘ └────┘││ └────┘ +/// │└─────────────────────────┘│ +/// │┌────── Partition 1 ──────┐│ +/// ││ ┌────┐ ┌────┐ ┌────┐││ ┌────┐ +/// ──background poll─▶│ │ │ │ │ ├┼┼───────▶ │ +/// ││ └────┘ └────┘ └────┘││ └────┘ +/// │└─────────────────────────┘│ +/// │ │ +/// │ ... │ +/// │ │ +/// │┌────── Partition N ──────┐│ +/// ││ ┌────┐││ ┌────┐ +/// ──background poll───────────────▶│ ├┼┼───────▶ │ +/// ││ └────┘││ └────┘ +/// │└─────────────────────────┘│ +/// └───────────────────────────┘ +/// ``` +/// +/// The capacity is provided in bytes, and for each buffered record batch it will take into account +/// the size reported by [RecordBatch::get_array_memory_size]. +/// +/// If a single record batch exceeds the maximum capacity set in the `capacity` argument, it's still +/// allowed to pass in order to not deadlock the buffer. +/// +/// This is useful for operators that conditionally start polling one of their children only after +/// other child has finished, allowing to perform some early work and accumulating batches in +/// memory so that they can be served immediately when requested. +#[derive(Debug, Clone)] +pub struct BufferExec { + input: Arc, + properties: PlanProperties, + capacity: usize, + metrics: ExecutionPlanMetricsSet, +} + +impl BufferExec { + /// Builds a new [BufferExec] with the provided capacity in bytes. + pub fn new(input: Arc, capacity: usize) -> Self { + let properties = input + .properties() + .clone() + .with_scheduling_type(SchedulingType::Cooperative); + + Self { + input, + properties, + capacity, + metrics: ExecutionPlanMetricsSet::new(), + } + } + + /// Returns the input [ExecutionPlan] of this [BufferExec]. + pub fn input(&self) -> &Arc { + &self.input + } + + /// Returns the per-partition capacity in bytes for this [BufferExec]. + pub fn capacity(&self) -> usize { + self.capacity + } +} + +impl DisplayAs for BufferExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "BufferExec: capacity={}", self.capacity) + } + DisplayFormatType::TreeRender => { + writeln!(f, "target_batch_size={}", self.capacity) + } + } + } +} + +impl ExecutionPlan for BufferExec { + fn name(&self) -> &str { + "BufferExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.properties + } + + fn maintains_input_order(&self) -> Vec { + vec![true] + } + + fn benefits_from_input_partitioning(&self) -> Vec { + vec![false] + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.input] + } + + fn with_new_children( + self: Arc, + mut children: Vec>, + ) -> Result> { + if children.len() != 1 { + return plan_err!("BufferExec can only have one child"); + } + Ok(Arc::new(Self::new(children.swap_remove(0), self.capacity))) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> Result { + let mem_reservation = MemoryConsumer::new(format!("BufferExec[{partition}]")) + .register(context.memory_pool()); + let in_stream = self.input.execute(partition, context)?; + + // Set up the metrics for the stream. + let curr_mem_in = Arc::new(AtomicUsize::new(0)); + let curr_mem_out = Arc::clone(&curr_mem_in); + let mut max_mem_in = 0; + let max_mem = MetricBuilder::new(&self.metrics).gauge("max_mem_used", partition); + + let curr_queued_in = Arc::new(AtomicUsize::new(0)); + let curr_queued_out = Arc::clone(&curr_queued_in); + let mut max_queued_in = 0; + let max_queued = MetricBuilder::new(&self.metrics).gauge("max_queued", partition); + + // Capture metrics when an element is queued on the stream. + let in_stream = in_stream.inspect_ok(move |v| { + let size = v.get_array_memory_size(); + let curr_size = curr_mem_in.fetch_add(size, Ordering::Relaxed) + size; + if curr_size > max_mem_in { + max_mem_in = curr_size; + max_mem.set(max_mem_in); + } + + let curr_queued = curr_queued_in.fetch_add(1, Ordering::Relaxed) + 1; + if curr_queued > max_queued_in { + max_queued_in = curr_queued; + max_queued.set(max_queued_in); + } + }); + // Buffer the input. + let out_stream = + MemoryBufferedStream::new(in_stream, self.capacity, mem_reservation); + // Update in the metrics that when an element gets out, some memory gets freed. + let out_stream = out_stream.inspect_ok(move |v| { + curr_mem_out.fetch_sub(v.get_array_memory_size(), Ordering::Relaxed); + curr_queued_out.fetch_sub(1, Ordering::Relaxed); + }); + + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + out_stream, + ))) + } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } + + fn partition_statistics(&self, partition: Option) -> Result { + self.input.partition_statistics(partition) + } + + fn supports_limit_pushdown(&self) -> bool { + self.input.supports_limit_pushdown() + } + + fn cardinality_effect(&self) -> CardinalityEffect { + CardinalityEffect::Equal + } + + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + match self.input.try_swapping_with_projection(projection)? { + Some(new_input) => Ok(Some( + Arc::new(self.clone()).with_new_children(vec![new_input])?, + )), + None => Ok(None), + } + } + + fn gather_filters_for_pushdown( + &self, + _phase: FilterPushdownPhase, + parent_filters: Vec>, + _config: &ConfigOptions, + ) -> Result { + FilterDescription::from_children(parent_filters, &self.children()) + } + + fn handle_child_pushdown_result( + &self, + _phase: FilterPushdownPhase, + child_pushdown_result: ChildPushdownResult, + _config: &ConfigOptions, + ) -> 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(Self::new(new_input, self.capacity)) as Arc) + }) + } +} + +/// Represents anything that occupies a capacity in a [MemoryBufferedStream]. +pub trait SizedMessage { + fn size(&self) -> usize; +} + +impl SizedMessage for RecordBatch { + fn size(&self) -> usize { + self.get_array_memory_size() + } +} + +pin_project! { +/// Decouples production and consumption of messages in a stream with an internal queue, eagerly +/// filling it up to the specified maximum capacity even before any message is requested. +/// +/// Allows each message to have a different size, which is taken into account for determining if +/// the queue is full or not. +pub struct MemoryBufferedStream { + task: SpawnedTask<()>, + batch_rx: UnboundedReceiver>, + memory_reservation: Arc, +}} + +impl MemoryBufferedStream { + /// Builds a new [MemoryBufferedStream] with the provided capacity and event handler. + /// + /// This immediately spawns a Tokio task that will start consumption of the input stream. + pub fn new( + mut input: impl Stream> + Unpin + Send + 'static, + capacity: usize, + memory_reservation: MemoryReservation, + ) -> Self { + let semaphore = Arc::new(Semaphore::new(capacity)); + let (batch_tx, batch_rx) = tokio::sync::mpsc::unbounded_channel(); + + let memory_reservation = Arc::new(memory_reservation); + let memory_reservation_clone = Arc::clone(&memory_reservation); + let task = SpawnedTask::spawn(async move { + loop { + // Select on both the input stream and the channel being closed. + // By down this, we abort polling the input as soon as the consumer channel is + // closed. Otherwise, we would need to wait for a full new message to be available + // in order to consider aborting the stream + let item_or_err = tokio::select! { + biased; + _ = batch_tx.closed() => break, + item_or_err = input.next() => { + let Some(item_or_err) = item_or_err else { + break; // stream finished + }; + item_or_err + } + }; + + let item = match item_or_err { + Ok(batch) => batch, + Err(err) => { + let _ = batch_tx.send(Err(err)); // If there's an error it means the channel was closed, which is fine. + break; + } + }; + + let size = item.size(); + if let Err(err) = memory_reservation.try_grow(size) { + let _ = batch_tx.send(Err(err)); // If there's an error it means the channel was closed, which is fine. + break; + } + + // We need to cap the minimum between amount of permits and the actual size of the + // message. If at any point we try to acquire more permits than the capacity of the + // semaphore, the stream will deadlock. + let capped_size = size.min(capacity) as u32; + + let semaphore = Arc::clone(&semaphore); + let Ok(permit) = semaphore.acquire_many_owned(capped_size).await else { + let _ = batch_tx.send(internal_err!("Closed semaphore in MemoryBufferedStream. This is a bug in DataFusion, please report it!")); + break; + }; + + if batch_tx.send(Ok((item, permit))).is_err() { + break; // stream was closed + }; + } + }); + + Self { + task, + batch_rx, + memory_reservation: memory_reservation_clone, + } + } + + /// Returns the number of queued messages. + pub fn messages_queued(&self) -> usize { + self.batch_rx.len() + } +} + +impl Stream for MemoryBufferedStream { + type Item = Result; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let self_project = self.project(); + match self_project.batch_rx.poll_recv(cx) { + Poll::Ready(Some(Ok((item, _semaphore_permit)))) => { + self_project.memory_reservation.shrink(item.size()); + Poll::Ready(Some(Ok(item))) + } + Poll::Ready(Some(Err(err))) => Poll::Ready(Some(Err(err))), + Poll::Ready(None) => Poll::Ready(None), + Poll::Pending => Poll::Pending, + } + } + + fn size_hint(&self) -> (usize, Option) { + if self.batch_rx.is_closed() { + let len = self.batch_rx.len(); + (len, Some(len)) + } else { + (self.batch_rx.len(), None) + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use datafusion_common::{DataFusionError, assert_contains}; + use datafusion_execution::memory_pool::{ + GreedyMemoryPool, MemoryPool, UnboundedMemoryPool, + }; + use std::error::Error; + use std::fmt::Debug; + use std::sync::Arc; + use std::time::Duration; + use tokio::time::timeout; + + #[tokio::test] + async fn buffers_only_some_messages() -> Result<(), Box> { + let input = futures::stream::iter([1, 2, 3, 4]).map(Ok); + let (_, res) = memory_pool_and_reservation(); + + let buffered = MemoryBufferedStream::new(input, 4, res); + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 2); + Ok(()) + } + + #[tokio::test] + async fn yields_all_messages() -> Result<(), Box> { + let input = futures::stream::iter([1, 2, 3, 4]).map(Ok); + let (_, res) = memory_pool_and_reservation(); + + let mut buffered = MemoryBufferedStream::new(input, 10, res); + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 4); + + pull_ok_msg(&mut buffered).await?; + pull_ok_msg(&mut buffered).await?; + pull_ok_msg(&mut buffered).await?; + pull_ok_msg(&mut buffered).await?; + finished(&mut buffered).await?; + Ok(()) + } + + #[tokio::test] + async fn yields_first_msg_even_if_big() -> Result<(), Box> { + let input = futures::stream::iter([25, 1, 2, 3]).map(Ok); + let (_, res) = memory_pool_and_reservation(); + + let mut buffered = MemoryBufferedStream::new(input, 10, res); + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 1); + pull_ok_msg(&mut buffered).await?; + Ok(()) + } + + #[tokio::test] + async fn memory_pool_kills_stream() -> Result<(), Box> { + let input = futures::stream::iter([1, 2, 3, 4]).map(Ok); + let (_, res) = bounded_memory_pool_and_reservation(7); + + let mut buffered = MemoryBufferedStream::new(input, 10, res); + wait_for_buffering().await; + + pull_ok_msg(&mut buffered).await?; + pull_ok_msg(&mut buffered).await?; + pull_ok_msg(&mut buffered).await?; + let msg = pull_err_msg(&mut buffered).await?; + + assert_contains!(msg.to_string(), "Failed to allocate additional 4.0 B"); + Ok(()) + } + + #[tokio::test] + async fn memory_pool_does_not_kill_stream() -> Result<(), Box> { + let input = futures::stream::iter([1, 2, 3, 4]).map(Ok); + let (_, res) = bounded_memory_pool_and_reservation(7); + + let mut buffered = MemoryBufferedStream::new(input, 3, res); + wait_for_buffering().await; + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + finished(&mut buffered).await?; + Ok(()) + } + + #[tokio::test] + async fn messages_pass_even_if_all_exceed_limit() -> Result<(), Box> { + let input = futures::stream::iter([3, 3, 3, 3]).map(Ok); + let (_, res) = memory_pool_and_reservation(); + + let mut buffered = MemoryBufferedStream::new(input, 2, res); + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 1); + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 1); + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 1); + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 1); + pull_ok_msg(&mut buffered).await?; + + wait_for_buffering().await; + finished(&mut buffered).await?; + Ok(()) + } + + #[tokio::test] + async fn errors_get_propagated() -> Result<(), Box> { + let input = futures::stream::iter([1, 2, 3, 4]).map(|v| { + if v == 3 { + return internal_err!("Error on 3"); + } + Ok(v) + }); + let (_, res) = memory_pool_and_reservation(); + + let mut buffered = MemoryBufferedStream::new(input, 10, res); + wait_for_buffering().await; + + pull_ok_msg(&mut buffered).await?; + pull_ok_msg(&mut buffered).await?; + pull_err_msg(&mut buffered).await?; + + Ok(()) + } + + #[tokio::test] + async fn memory_gets_released_if_stream_drops() -> Result<(), Box> { + let input = futures::stream::iter([1, 2, 3, 4]).map(Ok); + let (pool, res) = memory_pool_and_reservation(); + + let mut buffered = MemoryBufferedStream::new(input, 10, res); + wait_for_buffering().await; + assert_eq!(buffered.messages_queued(), 4); + assert_eq!(pool.reserved(), 10); + + pull_ok_msg(&mut buffered).await?; + assert_eq!(buffered.messages_queued(), 3); + assert_eq!(pool.reserved(), 9); + + pull_ok_msg(&mut buffered).await?; + assert_eq!(buffered.messages_queued(), 2); + assert_eq!(pool.reserved(), 7); + + drop(buffered); + assert_eq!(pool.reserved(), 0); + Ok(()) + } + + fn memory_pool_and_reservation() -> (Arc, MemoryReservation) { + let pool = Arc::new(UnboundedMemoryPool::default()) as _; + let reservation = MemoryConsumer::new("test").register(&pool); + (pool, reservation) + } + + fn bounded_memory_pool_and_reservation( + size: usize, + ) -> (Arc, MemoryReservation) { + let pool = Arc::new(GreedyMemoryPool::new(size)) as _; + let reservation = MemoryConsumer::new("test").register(&pool); + (pool, reservation) + } + + async fn wait_for_buffering() { + // We do not have control over the spawned task, so the best we can do is to yield some + // cycles to the tokio runtime and let the task make progress on its own. + tokio::time::sleep(Duration::from_millis(1)).await; + } + + async fn pull_ok_msg( + buffered: &mut MemoryBufferedStream, + ) -> Result> { + Ok(timeout(Duration::from_millis(1), buffered.next()) + .await? + .unwrap_or_else(|| internal_err!("Stream should not have finished"))?) + } + + async fn pull_err_msg( + buffered: &mut MemoryBufferedStream, + ) -> Result> { + Ok(timeout(Duration::from_millis(1), buffered.next()) + .await? + .map(|v| match v { + Ok(v) => internal_err!( + "Stream should not have failed, but succeeded with {v:?}" + ), + Err(err) => Ok(err), + }) + .unwrap_or_else(|| internal_err!("Stream should not have finished"))?) + } + + async fn finished( + buffered: &mut MemoryBufferedStream, + ) -> Result<(), Box> { + match timeout(Duration::from_millis(1), buffered.next()) + .await? + .is_none() + { + true => Ok(()), + false => internal_err!("Stream should have finished")?, + } + } + + impl SizedMessage for usize { + fn size(&self) -> usize { + *self + } + } +} diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 4f32b6176ec39..7ada14be66543 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -206,7 +206,7 @@ async fn load_left_input( let (batches, _metrics, reservation) = stream .try_fold( (Vec::new(), metrics, reservation), - |(mut batches, metrics, mut reservation), batch| async { + |(mut batches, metrics, reservation), batch| async { let batch_size = batch.get_array_memory_size(); // Reserve memory for incoming batch reservation.try_grow(batch_size)?; diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 44637321a7e35..b57f9132253bf 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -682,10 +682,10 @@ async fn collect_left_input( let schema = stream.schema(); // Load all batches and count the rows - let (batches, metrics, mut reservation) = stream + let (batches, metrics, reservation) = stream .try_fold( (Vec::new(), join_metrics, reservation), - |(mut batches, metrics, mut reservation), batch| async { + |(mut batches, metrics, reservation), batch| async { let batch_size = batch.get_array_memory_size(); // Reserve memory for incoming batch reservation.try_grow(batch_size)?; diff --git a/datafusion/physical-plan/src/joins/piecewise_merge_join/exec.rs b/datafusion/physical-plan/src/joins/piecewise_merge_join/exec.rs index 508be2e3984f4..d7ece845e943c 100644 --- a/datafusion/physical-plan/src/joins/piecewise_merge_join/exec.rs +++ b/datafusion/physical-plan/src/joins/piecewise_merge_join/exec.rs @@ -620,7 +620,7 @@ async fn build_buffered_data( // Combine batches and record number of rows let initial = (Vec::new(), 0, metrics, reservation); - let (batches, num_rows, metrics, mut reservation) = buffered + let (batches, num_rows, metrics, reservation) = buffered .try_fold(initial, |mut acc, batch| async { let batch_size = get_record_batch_memory_size(&batch); acc.3.try_grow(batch_size)?; diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index ec8e154caec91..ba870ba65cb92 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -65,6 +65,7 @@ mod visitor; pub mod aggregates; pub mod analyze; pub mod async_func; +pub mod buffer; pub mod coalesce; pub mod coalesce_batches; pub mod coalesce_partitions; diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 30a0e301823f7..1b1dce9827b2a 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -709,7 +709,7 @@ impl ExternalSorter { &self, batch: RecordBatch, metrics: &BaselineMetrics, - mut reservation: MemoryReservation, + reservation: MemoryReservation, ) -> Result { assert_eq!( get_reserved_bytes_for_record_batch(&batch)?, diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 4b26f84099505..bcef141ec5ad4 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -241,7 +241,7 @@ impl ExecutionPlan for SortPreservingMergeExec { metrics: self.metrics.clone(), fetch: limit, cache: self.cache.clone(), - enable_round_robin_repartition: true, + enable_round_robin_repartition: self.enable_round_robin_repartition, })) } diff --git a/datafusion/physical-plan/src/sorts/stream.rs b/datafusion/physical-plan/src/sorts/stream.rs index a510f44e4f4df..779511a865b6a 100644 --- a/datafusion/physical-plan/src/sorts/stream.rs +++ b/datafusion/physical-plan/src/sorts/stream.rs @@ -180,7 +180,7 @@ impl RowCursorStream { self.rows.save(stream_idx, &rows); // track the memory in the newly created Rows. - let mut rows_reservation = self.reservation.new_empty(); + let rows_reservation = self.reservation.new_empty(); rows_reservation.try_grow(rows.size())?; Ok(RowValues::new(rows, rows_reservation)) } @@ -246,7 +246,7 @@ impl FieldCursorStream { let array = value.into_array(batch.num_rows())?; let size_in_mem = array.get_buffer_memory_size(); let array = array.as_any().downcast_ref::().expect("field values"); - let mut array_reservation = self.reservation.new_empty(); + let array_reservation = self.reservation.new_empty(); array_reservation.try_grow(size_in_mem)?; Ok(ArrayValues::new( self.sort.options, diff --git a/datafusion/physical-plan/src/stream.rs b/datafusion/physical-plan/src/stream.rs index 80c2233d05db6..4b7e707fccedd 100644 --- a/datafusion/physical-plan/src/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -1005,7 +1005,7 @@ mod test { .build_arc() .unwrap(); - let mut reservation = MemoryConsumer::new("test").register(&runtime.memory_pool); + let reservation = MemoryConsumer::new("test").register(&runtime.memory_pool); let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); @@ -1071,7 +1071,7 @@ mod test { .build_arc() .unwrap(); - let mut reservation = MemoryConsumer::new("test").register(&runtime.memory_pool); + let reservation = MemoryConsumer::new("test").register(&runtime.memory_pool); let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index f1b9e3e88d123..1313909adbba2 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -283,7 +283,7 @@ mod tests { assert!(work_table.take().is_err()); let pool = Arc::new(UnboundedMemoryPool::default()) as _; - let mut reservation = MemoryConsumer::new("test_work_table").register(&pool); + let reservation = MemoryConsumer::new("test_work_table").register(&pool); // Update batch to work_table let array: ArrayRef = Arc::new((0..5).collect::()); diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 7a770324e9878..459a88800ad70 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -750,6 +750,7 @@ message PhysicalPlanNode { SortMergeJoinExecNode sort_merge_join = 34; MemoryScanExecNode memory_scan = 35; AsyncFuncExecNode async_func = 36; + BufferExecNode buffer = 37; } } @@ -1413,3 +1414,8 @@ message AsyncFuncExecNode { repeated PhysicalExprNode async_exprs = 2; repeated string async_expr_names = 3; } + +message BufferExecNode { + PhysicalPlanNode input = 1; + uint64 capacity = 2; +} \ No newline at end of file diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 19eca16c7899e..cb27f0dd27c63 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -1838,6 +1838,118 @@ impl<'de> serde::Deserialize<'de> for BinaryExprNode { deserializer.deserialize_struct("datafusion.BinaryExprNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for BufferExecNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.input.is_some() { + len += 1; + } + if self.capacity != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.BufferExecNode", len)?; + if let Some(v) = self.input.as_ref() { + struct_ser.serialize_field("input", v)?; + } + if self.capacity != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("capacity", ToString::to_string(&self.capacity).as_str())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for BufferExecNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "input", + "capacity", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Input, + Capacity, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl serde::de::Visitor<'_> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "input" => Ok(GeneratedField::Input), + "capacity" => Ok(GeneratedField::Capacity), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = BufferExecNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.BufferExecNode") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut input__ = None; + let mut capacity__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Input => { + if input__.is_some() { + return Err(serde::de::Error::duplicate_field("input")); + } + input__ = map_.next_value()?; + } + GeneratedField::Capacity => { + if capacity__.is_some() { + return Err(serde::de::Error::duplicate_field("capacity")); + } + capacity__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + } + } + Ok(BufferExecNode { + input: input__, + capacity: capacity__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.BufferExecNode", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for CaseNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -17504,6 +17616,9 @@ impl serde::Serialize for PhysicalPlanNode { physical_plan_node::PhysicalPlanType::AsyncFunc(v) => { struct_ser.serialize_field("asyncFunc", v)?; } + physical_plan_node::PhysicalPlanType::Buffer(v) => { + struct_ser.serialize_field("buffer", v)?; + } } } struct_ser.end() @@ -17571,6 +17686,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "memoryScan", "async_func", "asyncFunc", + "buffer", ]; #[allow(clippy::enum_variant_names)] @@ -17610,6 +17726,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { SortMergeJoin, MemoryScan, AsyncFunc, + Buffer, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -17666,6 +17783,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "sortMergeJoin" | "sort_merge_join" => Ok(GeneratedField::SortMergeJoin), "memoryScan" | "memory_scan" => Ok(GeneratedField::MemoryScan), "asyncFunc" | "async_func" => Ok(GeneratedField::AsyncFunc), + "buffer" => Ok(GeneratedField::Buffer), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -17931,6 +18049,13 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { return Err(serde::de::Error::duplicate_field("asyncFunc")); } physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::AsyncFunc) +; + } + GeneratedField::Buffer => { + if physical_plan_type__.is_some() { + return Err(serde::de::Error::duplicate_field("buffer")); + } + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Buffer) ; } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 51f3389a422c8..424b49a4742e5 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1078,7 +1078,7 @@ pub mod table_reference { pub struct PhysicalPlanNode { #[prost( oneof = "physical_plan_node::PhysicalPlanType", - tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36" + tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37" )] pub physical_plan_type: ::core::option::Option, } @@ -1158,6 +1158,8 @@ pub mod physical_plan_node { MemoryScan(super::MemoryScanExecNode), #[prost(message, tag = "36")] AsyncFunc(::prost::alloc::boxed::Box), + #[prost(message, tag = "37")] + Buffer(::prost::alloc::boxed::Box), } } #[derive(Clone, PartialEq, ::prost::Message)] @@ -2136,6 +2138,13 @@ pub struct AsyncFuncExecNode { #[prost(string, repeated, tag = "3")] pub async_expr_names: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, } +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct BufferExecNode { + #[prost(message, optional, boxed, tag = "1")] + pub input: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(uint64, tag = "2")] + pub capacity: u64, +} #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum WindowFrameUnits { diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 4ff90b61eed9c..25f5e55cbe0ac 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -74,6 +74,7 @@ use datafusion_physical_expr::{LexOrdering, LexRequirement, PhysicalExprRef}; use datafusion_physical_plan::aggregates::AggregateMode; use datafusion_physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; use datafusion_physical_plan::analyze::AnalyzeExec; +use datafusion_physical_plan::buffer::BufferExec; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::coop::CooperativeExec; @@ -254,6 +255,9 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { PhysicalPlanType::AsyncFunc(async_func) => { self.try_into_async_func_physical_plan(async_func, ctx, extension_codec) } + PhysicalPlanType::Buffer(buffer) => { + self.try_into_buffer_physical_plan(buffer, ctx, extension_codec) + } } } @@ -471,6 +475,13 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { ); } + if let Some(exec) = plan.downcast_ref::() { + return protobuf::PhysicalPlanNode::try_from_buffer_exec( + exec, + extension_codec, + ); + } + let mut buf: Vec = vec![]; match extension_codec.try_encode(Arc::clone(&plan_clone), &mut buf) { Ok(_) => { @@ -2024,6 +2035,18 @@ impl protobuf::PhysicalPlanNode { Ok(Arc::new(AsyncFuncExec::try_new(async_exprs, input)?)) } + fn try_into_buffer_physical_plan( + &self, + buffer: &protobuf::BufferExecNode, + ctx: &TaskContext, + extension_codec: &dyn PhysicalExtensionCodec, + ) -> Result> { + let input: Arc = + into_physical_plan(&buffer.input, ctx, extension_codec)?; + + Ok(Arc::new(BufferExec::new(input, buffer.capacity as usize))) + } + fn try_from_explain_exec( exec: &ExplainExec, _extension_codec: &dyn PhysicalExtensionCodec, @@ -3303,6 +3326,25 @@ impl protobuf::PhysicalPlanNode { ))), }) } + + fn try_from_buffer_exec( + exec: &BufferExec, + extension_codec: &dyn PhysicalExtensionCodec, + ) -> Result { + let input = protobuf::PhysicalPlanNode::try_from_physical_plan( + Arc::clone(exec.input()), + extension_codec, + )?; + + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::Buffer(Box::new( + protobuf::BufferExecNode { + input: Some(Box::new(input)), + capacity: exec.capacity() as u64, + }, + ))), + }) + } } pub trait AsExecutionPlan: Debug + Send + Sync + Clone { diff --git a/datafusion/sqllogictest/test_files/sort_pushdown.slt b/datafusion/sqllogictest/test_files/sort_pushdown.slt index 99f26b66d458b..e9d4e221e1ddb 100644 --- a/datafusion/sqllogictest/test_files/sort_pushdown.slt +++ b/datafusion/sqllogictest/test_files/sort_pushdown.slt @@ -1100,16 +1100,16 @@ CREATE EXTERNAL TABLE reversed_parquet(id INT, value INT) STORED AS PARQUET LOCATION 'test_files/scratch/sort_pushdown/reversed/'; -# Test 4.1: SortExec must be present because files are not in inter-file order +# Test 4.1: PushdownSort reorders files by min/max statistics so they are +# already in correct sort order → non-overlapping → no SortExec needed. +# (files reordered from [a_high, b_mid, c_low] to [c_low, b_mid, a_high]) query TT EXPLAIN SELECT * FROM reversed_parquet ORDER BY id ASC; ---- logical_plan 01)Sort: reversed_parquet.id ASC NULLS LAST 02)--TableScan: reversed_parquet projection=[id, value] -physical_plan -01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/a_high.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/c_low.parquet]]}, projection=[id, value], file_type=parquet +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/c_low.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/a_high.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet # Test 4.2: Results must be correct query II @@ -1184,16 +1184,15 @@ STORED AS PARQUET LOCATION 'test_files/scratch/sort_pushdown/reversed/' WITH ORDER (id ASC); -# Test 6.1: SortExec must be present despite WITH ORDER +# Test 6.1: Files sorted by statistics → non-overlapping → SortExec eliminated +# WITH ORDER declared + files reordered to correct order query TT EXPLAIN SELECT * FROM reversed_with_order_parquet ORDER BY id ASC; ---- logical_plan 01)Sort: reversed_with_order_parquet.id ASC NULLS LAST 02)--TableScan: reversed_with_order_parquet projection=[id, value] -physical_plan -01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/a_high.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/c_low.parquet]]}, projection=[id, value], file_type=parquet +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/c_low.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/a_high.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet # Test 6.2: Results must be correct query II @@ -1322,17 +1321,15 @@ STORED AS PARQUET LOCATION 'test_files/scratch/sort_pushdown/desc_reversed/' WITH ORDER (id DESC); -# Test 8.1: SortExec must be present — files are in wrong inter-file DESC order -# (a_low has 1-3, b_high has 7-9; for DESC, b_high should come first) +# Test 8.1: Files sorted by statistics → non-overlapping → SortExec eliminated +# (files reordered: b_high(7-9) before a_low(1-3) for DESC order) query TT EXPLAIN SELECT * FROM desc_reversed_parquet ORDER BY id DESC; ---- logical_plan 01)Sort: desc_reversed_parquet.id DESC NULLS FIRST 02)--TableScan: desc_reversed_parquet projection=[id, value] -physical_plan -01)SortExec: expr=[id@0 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/desc_reversed/a_low.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/desc_reversed/b_high.parquet]]}, projection=[id, value], file_type=parquet +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/desc_reversed/b_high.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/desc_reversed/a_low.parquet]]}, projection=[id, value], output_ordering=[id@0 DESC], file_type=parquet # Test 8.2: Results must be correct query II @@ -1520,9 +1517,12 @@ SELECT * FROM multi_partition_parquet ORDER BY id ASC; 8 800 9 900 -# Restore target_partitions to 1 for remaining cleanup +# Config reset + +# The SLT runner sets `target_partitions` to 4 instead of using the default, so +# reset it explicitly. statement ok -SET datafusion.execution.target_partitions = 2; +SET datafusion.execution.target_partitions = 4; # Cleanup statement ok @@ -1630,5 +1630,653 @@ DROP TABLE agg_expr_data; statement ok DROP TABLE agg_expr_parquet; +statement ok +RESET datafusion.execution.parquet.pushdown_filters; + +statement ok +RESET datafusion.execution.parquet.max_row_group_size; + +statement ok +RESET datafusion.execution.parquet.enable_page_index; + +statement ok +RESET datafusion.optimizer.enable_sort_pushdown; + + +############################################################### +# Statistics-based file sorting and sort elimination tests +############################################################### + +statement ok +SET datafusion.optimizer.enable_sort_pushdown = true; + +statement ok +SET datafusion.execution.target_partitions = 1; + +statement ok +SET datafusion.execution.collect_statistics = true; + +# Test A: Non-overlapping files with matching within-file ordering → Sort eliminated + +statement ok +CREATE TABLE ta_src_a(id INT, value INT) AS VALUES (1, 100), (2, 200), (3, 300); + +statement ok +CREATE TABLE ta_src_b(id INT, value INT) AS VALUES (4, 400), (5, 500), (6, 600); + +statement ok +CREATE TABLE ta_src_c(id INT, value INT) AS VALUES (7, 700), (8, 800), (9, 900), (10, 1000); + +query I +COPY (SELECT * FROM ta_src_a ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/ta_nonoverlap/file_a.parquet'; +---- +3 + +query I +COPY (SELECT * FROM ta_src_b ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/ta_nonoverlap/file_b.parquet'; +---- +3 + +query I +COPY (SELECT * FROM ta_src_c ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/ta_nonoverlap/file_c.parquet'; +---- +4 + +statement ok +CREATE EXTERNAL TABLE ta_sorted(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/ta_nonoverlap/' +WITH ORDER (id ASC); + +# Test A.1: SortExec eliminated — Exact pushdown +query TT +EXPLAIN SELECT * FROM ta_sorted ORDER BY id ASC; +---- +logical_plan +01)Sort: ta_sorted.id ASC NULLS LAST +02)--TableScan: ta_sorted projection=[id, value] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/ta_nonoverlap/file_a.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/ta_nonoverlap/file_b.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/ta_nonoverlap/file_c.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet + +query II +SELECT * FROM ta_sorted ORDER BY id ASC; +---- +1 100 +2 200 +3 300 +4 400 +5 500 +6 600 +7 700 +8 800 +9 900 +10 1000 + +# Cleanup Test A +statement ok +DROP TABLE ta_src_a; + +statement ok +DROP TABLE ta_src_b; + +statement ok +DROP TABLE ta_src_c; + +statement ok +DROP TABLE ta_sorted; + + +# Test B: Overlapping files → statistics-based reorder, SortExec retained + +statement ok +CREATE TABLE tb_src_x(id INT, value INT) AS VALUES (1, 10), (2, 20), (3, 30), (4, 40), (5, 50); + +statement ok +CREATE TABLE tb_src_y(id INT, value INT) AS VALUES (3, 31), (4, 41), (5, 51), (6, 61), (7, 71), (8, 81); + +statement ok +CREATE TABLE tb_src_z(id INT, value INT) AS VALUES (6, 62), (7, 72), (8, 82), (9, 92), (10, 102); + +query I +COPY (SELECT * FROM tb_src_x ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/tb_overlap/file_x.parquet'; +---- +5 + +query I +COPY (SELECT * FROM tb_src_y ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/tb_overlap/file_y.parquet'; +---- +6 + +query I +COPY (SELECT * FROM tb_src_z ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/tb_overlap/file_z.parquet'; +---- +5 + +statement ok +CREATE EXTERNAL TABLE tb_overlap(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/tb_overlap/' +WITH ORDER (id ASC); + +# Test B.1: Multi-column DESC sort — statistics fallback sorts files [z, y, x] +query TT +EXPLAIN SELECT * FROM tb_overlap ORDER BY id DESC, value DESC LIMIT 5; +---- +logical_plan +01)Sort: tb_overlap.id DESC NULLS FIRST, tb_overlap.value DESC NULLS FIRST, fetch=5 +02)--TableScan: tb_overlap projection=[id, value] +physical_plan +01)SortExec: TopK(fetch=5), expr=[id@0 DESC, value@1 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tb_overlap/file_z.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tb_overlap/file_y.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tb_overlap/file_x.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ] + +query II +SELECT * FROM tb_overlap ORDER BY id DESC, value DESC LIMIT 5; +---- +10 102 +9 92 +8 82 +8 81 +7 72 + +# Cleanup Test B +statement ok +DROP TABLE tb_src_x; + +statement ok +DROP TABLE tb_src_y; + +statement ok +DROP TABLE tb_src_z; + +statement ok +DROP TABLE tb_overlap; + + +# Test C: Non-overlapping files with LIMIT — sort elimination + limit pushdown + +statement ok +CREATE TABLE tc_src_a(id INT, value INT) AS VALUES (1, 100), (2, 200), (3, 300); + +statement ok +CREATE TABLE tc_src_b(id INT, value INT) AS VALUES (4, 400), (5, 500), (6, 600); + +statement ok +CREATE TABLE tc_src_c(id INT, value INT) AS VALUES (7, 700), (8, 800), (9, 900), (10, 1000); + +query I +COPY (SELECT * FROM tc_src_a ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/tc_limit/file_a.parquet'; +---- +3 + +query I +COPY (SELECT * FROM tc_src_b ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/tc_limit/file_b.parquet'; +---- +3 + +query I +COPY (SELECT * FROM tc_src_c ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/tc_limit/file_c.parquet'; +---- +4 + +statement ok +CREATE EXTERNAL TABLE tc_limit(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/tc_limit/' +WITH ORDER (id ASC); + +# Test C.1: ASC LIMIT — sort eliminated, limit pushed down +query TT +EXPLAIN SELECT * FROM tc_limit ORDER BY id ASC LIMIT 3; +---- +logical_plan +01)Sort: tc_limit.id ASC NULLS LAST, fetch=3 +02)--TableScan: tc_limit projection=[id, value] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tc_limit/file_a.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tc_limit/file_b.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tc_limit/file_c.parquet]]}, projection=[id, value], limit=3, output_ordering=[id@0 ASC NULLS LAST], file_type=parquet + +query II +SELECT * FROM tc_limit ORDER BY id ASC LIMIT 3; +---- +1 100 +2 200 +3 300 + +# Test C.2: DESC LIMIT — reverse scan path, SortExec stays +query TT +EXPLAIN SELECT * FROM tc_limit ORDER BY id DESC LIMIT 3; +---- +logical_plan +01)Sort: tc_limit.id DESC NULLS FIRST, fetch=3 +02)--TableScan: tc_limit projection=[id, value] +physical_plan +01)SortExec: TopK(fetch=3), expr=[id@0 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tc_limit/file_c.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tc_limit/file_b.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tc_limit/file_a.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], reverse_row_groups=true + +query II +SELECT * FROM tc_limit ORDER BY id DESC LIMIT 3; +---- +10 1000 +9 900 +8 800 + +# Cleanup Test C +statement ok +DROP TABLE tc_src_a; + +statement ok +DROP TABLE tc_src_b; + +statement ok +DROP TABLE tc_src_c; + +statement ok +DROP TABLE tc_limit; + + +# Test D: Multi-group case with target_partitions=2 + +statement ok +CREATE TABLE td_src_a(id INT, value INT) AS VALUES (1, 100), (2, 200), (3, 300); + +statement ok +CREATE TABLE td_src_b(id INT, value INT) AS VALUES (4, 400), (5, 500), (6, 600); + +statement ok +CREATE TABLE td_src_c(id INT, value INT) AS VALUES (7, 700), (8, 800), (9, 900), (10, 1000); + +query I +COPY (SELECT * FROM td_src_a ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/td_multi/file_a.parquet'; +---- +3 + +query I +COPY (SELECT * FROM td_src_b ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/td_multi/file_b.parquet'; +---- +3 + +query I +COPY (SELECT * FROM td_src_c ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/td_multi/file_c.parquet'; +---- +4 + +statement ok +SET datafusion.execution.target_partitions = 2; + +statement ok +CREATE EXTERNAL TABLE td_multi(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/td_multi/' +WITH ORDER (id ASC); + +# Test D.1: With 2 partitions, files split across 2 groups. +# Each group's sort is eliminated; SortPreservingMergeExec merges groups. +query TT +EXPLAIN SELECT * FROM td_multi ORDER BY id ASC; +---- +logical_plan +01)Sort: td_multi.id ASC NULLS LAST +02)--TableScan: td_multi projection=[id, value] +physical_plan +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/td_multi/file_a.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/td_multi/file_b.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/td_multi/file_c.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet + +query II +SELECT * FROM td_multi ORDER BY id ASC; +---- +1 100 +2 200 +3 300 +4 400 +5 500 +6 600 +7 700 +8 800 +9 900 +10 1000 + +# Cleanup Test D +statement ok +DROP TABLE td_src_a; + +statement ok +DROP TABLE td_src_b; + +statement ok +DROP TABLE td_src_c; + +statement ok +DROP TABLE td_multi; + +# Restore target_partitions=1 for remaining tests +statement ok +SET datafusion.execution.target_partitions = 1; + +# =========================================================== +# Test E: Inferred ordering from Parquet metadata (no WITH ORDER) +# Parquet files written with ORDER BY have sorting_columns in metadata. +# DataFusion should automatically infer the ordering and eliminate Sort. +# =========================================================== + +# Create sorted parquet files — COPY with ORDER BY writes sorting_columns metadata +statement ok +CREATE TABLE te_src_a(id INT, value INT) AS VALUES (1, 100), (2, 200), (3, 300); + +statement ok +CREATE TABLE te_src_b(id INT, value INT) AS VALUES (4, 400), (5, 500), (6, 600); + +statement ok +CREATE TABLE te_src_c(id INT, value INT) AS VALUES (7, 700), (8, 800), (9, 900), (10, 1000); + +query I +COPY (SELECT * FROM te_src_a ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/te_inferred/file_a.parquet'; +---- +3 + +query I +COPY (SELECT * FROM te_src_b ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/te_inferred/file_b.parquet'; +---- +3 + +query I +COPY (SELECT * FROM te_src_c ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/te_inferred/file_c.parquet'; +---- +4 + +# Create external table WITHOUT "WITH ORDER" — ordering should be inferred +# from Parquet sorting_columns metadata +statement ok +CREATE EXTERNAL TABLE te_inferred(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/te_inferred/'; + +# Test E.1: Ordering should be inferred and SortExec eliminated +# Even without WITH ORDER, the optimizer detects that files are sorted +# (from Parquet sorting_columns metadata) and non-overlapping. +# SortExec is completely eliminated — just DataSourceExec with output_ordering. +query TT +EXPLAIN SELECT * FROM te_inferred ORDER BY id ASC; +---- +logical_plan +01)Sort: te_inferred.id ASC NULLS LAST +02)--TableScan: te_inferred projection=[id, value] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/te_inferred/file_a.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/te_inferred/file_b.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/te_inferred/file_c.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet + +# Test E.2: Verify result correctness +query II +SELECT * FROM te_inferred ORDER BY id ASC; +---- +1 100 +2 200 +3 300 +4 400 +5 500 +6 600 +7 700 +8 800 +9 900 +10 1000 + +# Test E.3: LIMIT should also work with inferred ordering +query II +SELECT * FROM te_inferred ORDER BY id ASC LIMIT 3; +---- +1 100 +2 200 +3 300 + +# Test E.4: Inferred ordering with multiple partitions +# With target_partitions=2, files split into 2 groups. +# Each group has SortExec eliminated, SPM merges the sorted streams. +statement ok +DROP TABLE te_inferred; + +statement ok +SET datafusion.execution.target_partitions = 2; + +statement ok +CREATE EXTERNAL TABLE te_inferred_multi(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/te_inferred/'; + +query TT +EXPLAIN SELECT * FROM te_inferred_multi ORDER BY id ASC; +---- +logical_plan +01)Sort: te_inferred_multi.id ASC NULLS LAST +02)--TableScan: te_inferred_multi projection=[id, value] +physical_plan +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/te_inferred/file_a.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/te_inferred/file_b.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/te_inferred/file_c.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet + +# Verify correctness with multi partition +query II +SELECT * FROM te_inferred_multi ORDER BY id ASC; +---- +1 100 +2 200 +3 300 +4 400 +5 500 +6 600 +7 700 +8 800 +9 900 +10 1000 + +# Cleanup Test E +statement ok +DROP TABLE te_src_a; + +statement ok +DROP TABLE te_src_b; + +statement ok +DROP TABLE te_src_c; + +statement ok +DROP TABLE te_inferred_multi; + +# =========================================================== +# Test F: NULL handling — sort pushdown must not eliminate Sort +# when files contain NULLs in sort columns, because NULL ordering +# (NULLS FIRST/LAST) across files requires a full sort. +# =========================================================== + +# Test F.1: NULLS LAST — file with NULL must not cause wrong ordering +statement ok +CREATE TABLE null_src_a(id INT) AS VALUES (1), (NULL); + +statement ok +CREATE TABLE null_src_b(id INT) AS VALUES (2), (3); + +query I +COPY (SELECT * FROM null_src_a ORDER BY id ASC NULLS LAST) +TO 'test_files/scratch/sort_pushdown/tf_nulls/b_null_tail.parquet'; +---- +2 + +query I +COPY (SELECT * FROM null_src_b ORDER BY id ASC NULLS LAST) +TO 'test_files/scratch/sort_pushdown/tf_nulls/a_nonnull.parquet'; +---- +2 + +statement ok +CREATE EXTERNAL TABLE tf_nulls_last(id INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/tf_nulls/' +WITH ORDER (id ASC NULLS LAST); + +# With target_partitions=1, files end up in separate groups via +# split_groups_by_statistics. EnforceSorting eliminates SortExec, +# SPM merges the two sorted streams. +query TT +EXPLAIN SELECT * FROM tf_nulls_last ORDER BY id ASC NULLS LAST; +---- +logical_plan +01)Sort: tf_nulls_last.id ASC NULLS LAST +02)--TableScan: tf_nulls_last projection=[id] +physical_plan +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tf_nulls/a_nonnull.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tf_nulls/b_null_tail.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet + +# Results must be correct: NULLs at the end +query I +SELECT * FROM tf_nulls_last ORDER BY id ASC NULLS LAST; +---- +1 +2 +3 +NULL + +# Test F.2: NULLS FIRST — NULLs should come first +query I +SELECT * FROM tf_nulls_last ORDER BY id ASC NULLS FIRST; +---- +NULL +1 +2 +3 + +# Cleanup Test F +statement ok +DROP TABLE null_src_a; + +statement ok +DROP TABLE null_src_b; + +statement ok +DROP TABLE tf_nulls_last; + +# =========================================================== +# Test G: BufferExec insertion when sort elimination removes +# SortExec under SortPreservingMergeExec. +# +# When PushdownSort eliminates SortExec(preserve_partitioning=true), +# SPM loses SortExec's memory buffer. A BufferExec is inserted to +# compensate, allowing I/O pipelining with merge computation. +# =========================================================== + +# Create files with reversed naming: c_low has smallest values, +# a_high has largest — alphabetical order ≠ sort key order. +statement ok +CREATE TABLE tg_src_low(id INT, value INT) AS VALUES (1, 100), (2, 200), (3, 300); + +statement ok +CREATE TABLE tg_src_mid(id INT, value INT) AS VALUES (4, 400), (5, 500), (6, 600); + +statement ok +CREATE TABLE tg_src_high(id INT, value INT) AS VALUES (7, 700), (8, 800), (9, 900), (10, 1000); + +query I +COPY (SELECT * FROM tg_src_low ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/tg_buffer/c_low.parquet'; +---- +3 + +query I +COPY (SELECT * FROM tg_src_mid ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/tg_buffer/b_mid.parquet'; +---- +3 + +query I +COPY (SELECT * FROM tg_src_high ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/tg_buffer/a_high.parquet'; +---- +4 + +# Use target_partitions=2 so files are split across 2 groups. +# Files are in wrong alphabetical order → validated_output_ordering strips ordering +# → EnforceSorting adds SortExec(preserve_partitioning) + SPM +# → PushdownSort eliminates SortExec and inserts BufferExec +statement ok +SET datafusion.execution.target_partitions = 2; + +statement ok +CREATE EXTERNAL TABLE tg_buffer(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/tg_buffer/' +WITH ORDER (id ASC); + +# Test G.1: BufferExec appears between SPM and DataSourceExec +query TT +EXPLAIN SELECT * FROM tg_buffer ORDER BY id ASC; +---- +logical_plan +01)Sort: tg_buffer.id ASC NULLS LAST +02)--TableScan: tg_buffer projection=[id, value] +physical_plan +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] +02)--BufferExec: capacity=67108864 +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tg_buffer/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tg_buffer/a_high.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tg_buffer/c_low.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet + +# Verify correctness +query II +SELECT * FROM tg_buffer ORDER BY id ASC; +---- +1 100 +2 200 +3 300 +4 400 +5 500 +6 600 +7 700 +8 800 +9 900 +10 1000 + +# Test G.2: LIMIT query with BufferExec +query TT +EXPLAIN SELECT * FROM tg_buffer ORDER BY id ASC LIMIT 3; +---- +logical_plan +01)Sort: tg_buffer.id ASC NULLS LAST, fetch=3 +02)--TableScan: tg_buffer projection=[id, value] +physical_plan +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST], fetch=3 +02)--BufferExec: capacity=67108864 +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tg_buffer/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tg_buffer/a_high.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tg_buffer/c_low.parquet]]}, projection=[id, value], limit=3, output_ordering=[id@0 ASC NULLS LAST], file_type=parquet + +query II +SELECT * FROM tg_buffer ORDER BY id ASC LIMIT 3; +---- +1 100 +2 200 +3 300 + +# Cleanup Test G +statement ok +DROP TABLE tg_src_low; + +statement ok +DROP TABLE tg_src_mid; + +statement ok +DROP TABLE tg_src_high; + +statement ok +DROP TABLE tg_buffer; + +# Reset settings (SLT runner uses target_partitions=4, not system default) +statement ok +SET datafusion.execution.target_partitions = 4; + +statement ok +SET datafusion.execution.collect_statistics = true; + statement ok SET datafusion.optimizer.enable_sort_pushdown = true;