From 26c6121f037bf347cad6b71fa81c67de81a9f1c5 Mon Sep 17 00:00:00 2001 From: Oleks V Date: Wed, 15 Apr 2026 07:01:49 -0700 Subject: [PATCH 1/6] feat: support `array_compact` builtin function (#21522) ## Which issue does this PR close? - Closes #. ## Rationale for this change Create `array_compact` function which removes NULLs from input array. There is no direct counterparty in DuckDB however the function used in Spark, SnowFlake ## What changes are included in this PR? ## Are these changes tested? ## Are there any user-facing changes? --- .../functions-nested/src/array_compact.rs | 191 ++++++++++++++++++ datafusion/functions-nested/src/lib.rs | 3 + datafusion/sqllogictest/test_files/array.slt | 93 +++++++++ .../source/user-guide/sql/scalar_functions.md | 33 +++ 4 files changed, 320 insertions(+) create mode 100644 datafusion/functions-nested/src/array_compact.rs diff --git a/datafusion/functions-nested/src/array_compact.rs b/datafusion/functions-nested/src/array_compact.rs new file mode 100644 index 0000000000000..11be494b5b20f --- /dev/null +++ b/datafusion/functions-nested/src/array_compact.rs @@ -0,0 +1,191 @@ +// 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. + +//! [`ScalarUDFImpl`] definitions for array_compact function. + +use crate::utils::make_scalar_function; +use arrow::array::{ + Array, ArrayRef, Capacities, GenericListArray, MutableArrayData, OffsetSizeTrait, + make_array, +}; +use arrow::buffer::OffsetBuffer; +use arrow::datatypes::DataType; +use arrow::datatypes::DataType::{LargeList, List, Null}; +use datafusion_common::cast::{as_large_list_array, as_list_array}; +use datafusion_common::{Result, exec_err, utils::take_function_args}; +use datafusion_expr::{ + ColumnarValue, Documentation, ScalarFunctionArgs, ScalarUDFImpl, Signature, + Volatility, +}; +use datafusion_macros::user_doc; +use std::sync::Arc; + +make_udf_expr_and_func!( + ArrayCompact, + array_compact, + array, + "removes null values from the array.", + array_compact_udf +); + +#[user_doc( + doc_section(label = "Array Functions"), + description = "Removes null values from the array.", + syntax_example = "array_compact(array)", + sql_example = r#"```sql +> select array_compact([1, NULL, 2, NULL, 3]) arr; ++-----------+ +| arr | ++-----------+ +| [1, 2, 3] | ++-----------+ +```"#, + argument( + name = "array", + description = "Array expression. Can be a constant, column, or function, and any combination of array operators." + ) +)] +#[derive(Debug, PartialEq, Eq, Hash)] +pub struct ArrayCompact { + signature: Signature, + aliases: Vec, +} + +impl Default for ArrayCompact { + fn default() -> Self { + Self::new() + } +} + +impl ArrayCompact { + pub fn new() -> Self { + Self { + signature: Signature::array(Volatility::Immutable), + aliases: vec!["list_compact".to_string()], + } + } +} + +impl ScalarUDFImpl for ArrayCompact { + fn name(&self) -> &str { + "array_compact" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> Result { + Ok(arg_types[0].clone()) + } + + fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { + make_scalar_function(array_compact_inner)(&args.args) + } + + fn aliases(&self) -> &[String] { + &self.aliases + } + + fn documentation(&self) -> Option<&Documentation> { + self.doc() + } +} + +/// array_compact SQL function +fn array_compact_inner(arg: &[ArrayRef]) -> Result { + let [input_array] = take_function_args("array_compact", arg)?; + + match &input_array.data_type() { + List(field) => { + let array = as_list_array(input_array)?; + compact_list::(array, field) + } + LargeList(field) => { + let array = as_large_list_array(input_array)?; + compact_list::(array, field) + } + Null => Ok(Arc::clone(input_array)), + array_type => exec_err!("array_compact does not support type '{array_type}'."), + } +} + +/// Remove null elements from each row of a list array. +fn compact_list( + list_array: &GenericListArray, + field: &Arc, +) -> Result { + let values = list_array.values(); + + // Fast path: no nulls in values, return input unchanged + if values.null_count() == 0 { + return Ok(Arc::new(list_array.clone())); + } + + let original_data = values.to_data(); + let capacity = original_data.len() - values.null_count(); + let mut offsets = Vec::::with_capacity(list_array.len() + 1); + offsets.push(O::zero()); + let mut mutable = MutableArrayData::with_capacities( + vec![&original_data], + false, + Capacities::Array(capacity), + ); + + for row_index in 0..list_array.len() { + if list_array.nulls().is_some_and(|n| n.is_null(row_index)) { + offsets.push(offsets[row_index]); + continue; + } + + let start = list_array.offsets()[row_index].as_usize(); + let end = list_array.offsets()[row_index + 1].as_usize(); + let mut copied = 0usize; + + // Batch consecutive non-null elements into single extend() calls + // to reduce per-element overhead. For [1, 2, NULL, 3, 4] this + // produces 2 extend calls (0..2, 3..5) instead of 4 individual ones. + let mut batch_start: Option = None; + for i in start..end { + if values.is_null(i) { + // Null breaks the current batch — flush it + if let Some(bs) = batch_start { + mutable.extend(0, bs, i); + copied += i - bs; + batch_start = None; + } + } else if batch_start.is_none() { + batch_start = Some(i); + } + } + // Flush any remaining batch after the loop + if let Some(bs) = batch_start { + mutable.extend(0, bs, end); + copied += end - bs; + } + + offsets.push(offsets[row_index] + O::usize_as(copied)); + } + + let new_values = make_array(mutable.freeze()); + Ok(Arc::new(GenericListArray::::try_new( + Arc::clone(field), + OffsetBuffer::new(offsets.into()), + new_values, + list_array.nulls().cloned(), + )?)) +} diff --git a/datafusion/functions-nested/src/lib.rs b/datafusion/functions-nested/src/lib.rs index 99b25ec96454b..99c33fb64039f 100644 --- a/datafusion/functions-nested/src/lib.rs +++ b/datafusion/functions-nested/src/lib.rs @@ -37,6 +37,7 @@ #[macro_use] pub mod macros; +pub mod array_compact; pub mod array_has; pub mod arrays_zip; pub mod cardinality; @@ -77,6 +78,7 @@ use std::sync::Arc; /// Fluent-style API for creating `Expr`s pub mod expr_fn { + pub use super::array_compact::array_compact; pub use super::array_has::array_has; pub use super::array_has::array_has_all; pub use super::array_has::array_has_any; @@ -128,6 +130,7 @@ pub mod expr_fn { /// Return all default nested type functions pub fn all_default_nested_functions() -> Vec> { vec![ + array_compact::array_compact_udf(), string::array_to_string_udf(), string::string_to_array_udf(), range::range_udf(), diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 81d5c8f91a5bc..7b6779abe486f 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -7685,6 +7685,99 @@ from array_distinct_table_2D_fixed; [[1, 2], [3, 4], [5, 6], [7, 8], [9, 10]] [[5, 6], NULL] +## array_compact (aliases: `list_compact`) + +# basic: remove nulls from integer array +query ? +select array_compact([1, NULL, 2, NULL, 3]); +---- +[1, 2, 3] + +# no nulls present +query ? +select array_compact([1, 2, 3]); +---- +[1, 2, 3] + +# all nulls +query ? +select array_compact(arrow_cast([NULL, NULL, NULL], 'List(Int64)')); +---- +[] + +# empty array +query ? +select array_compact([]); +---- +[] + +# NULL input returns NULL +query ? +select array_compact(NULL::INT[]); +---- +NULL + +# string array +query ? +select array_compact(['a', NULL, 'b', NULL, 'c']); +---- +[a, b, c] + +# float array +query ? +select array_compact([1.0, NULL, 2.0, NULL]); +---- +[1.0, 2.0] + +# nested array (2D) +query ? +select array_compact([make_array(1, 2), NULL, make_array(3, 4)]); +---- +[[1, 2], [3, 4]] + +# LargeList +query ? +select array_compact(arrow_cast([1, NULL, 2, NULL, 3], 'LargeList(Int64)')); +---- +[1, 2, 3] + +# alias list_compact +query ? +select list_compact([1, NULL, 2]); +---- +[1, 2] + +# table-based test +statement ok +CREATE TABLE array_compact_table AS VALUES + (make_array(1, NULL, 2, NULL, 3)), + (make_array(NULL, NULL, NULL)), + (make_array(4, 5, 6)), + (NULL::INT[]) +; + +query ? +select array_compact(column1) from array_compact_table; +---- +[1, 2, 3] +[] +[4, 5, 6] +NULL + +statement ok +DROP TABLE array_compact_table; + +# FixedSizeList (coerced to List) +query ? +select array_compact(arrow_cast(make_array(1, NULL, 2, NULL, 3), 'FixedSizeList(5, Int64)')); +---- +[1, 2, 3] + +query ? +select array_compact(arrow_cast(make_array(NULL, NULL, NULL), 'FixedSizeList(3, Int64)')); +---- +[] + ## arrays_zip (aliases: `list_zip`) # Spark example: arrays_zip(array(1, 2, 3), array(2, 3, 4)) diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index ca8707270fe18..9f5814447b7dc 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -3237,6 +3237,7 @@ _Alias of [current_date](#current_date)._ - [array_any_value](#array_any_value) - [array_append](#array_append) - [array_cat](#array_cat) +- [array_compact](#array_compact) - [array_concat](#array_concat) - [array_contains](#array_contains) - [array_dims](#array_dims) @@ -3285,6 +3286,7 @@ _Alias of [current_date](#current_date)._ - [list_any_value](#list_any_value) - [list_append](#list_append) - [list_cat](#list_cat) +- [list_compact](#list_compact) - [list_concat](#list_concat) - [list_contains](#list_contains) - [list_dims](#list_dims) @@ -3391,6 +3393,33 @@ array_append(array, element) _Alias of [array_concat](#array_concat)._ +### `array_compact` + +Removes null values from the array. + +```sql +array_compact(array) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select array_compact([1, NULL, 2, NULL, 3]) arr; ++-----------+ +| arr | ++-----------+ +| [1, 2, 3] | ++-----------+ +``` + +#### Aliases + +- list_compact + ### `array_concat` Concatenates arrays. @@ -4535,6 +4564,10 @@ _Alias of [array_append](#array_append)._ _Alias of [array_concat](#array_concat)._ +### `list_compact` + +_Alias of [array_compact](#array_compact)._ + ### `list_concat` _Alias of [array_concat](#array_concat)._ From d0692b8ee999ed72cd2a8f7000884618adfc06ee Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 15 Apr 2026 10:50:24 -0400 Subject: [PATCH 2/6] bench: Scale sort benchmarks to 1M rows to exercise merge path (#21630) ## Which issue does this PR close? - Partially addresses #21543. Also needed to properly evaluate the ExternalSorter refactor in #21629, which improves the merge path. ## Rationale for this change Current sort benchmarks use 100K rows across 8 partitions (~12.5K rows per partition, ~100KB for integers). This falls below the `sort_in_place_threshold_bytes` (1MB), so the "sort partitioned" benchmarks always take the concat-and-sort-in-place path and never exercise the sort-then-merge path that dominates real workloads. ## What changes are included in this PR? Parameterizes the sort benchmark on input size, running each case at both 100K rows (existing) and 1M rows (new). At 1M rows, each partition holds ~125K rows (~1MB for integers), which exercises the merge path. - `INPUT_SIZE` constant replaced with `INPUT_SIZES` array: `[(100_000, "100k"), (1_000_000, "1M")]` - `DataGenerator` takes `input_size` as a constructor parameter - All stream generator functions accept `input_size` - Benchmark names include size label (e.g. `sort partitioned i64 100k`, `sort partitioned i64 10M`) - Data distribution and cardinality ratios are preserved across sizes ## Are these changes tested? Benchmark compiles and runs. No functional test changes. ## Are there any user-facing changes? No. --- datafusion/core/benches/sort.rs | 218 ++++++++++++++++++++------------ 1 file changed, 136 insertions(+), 82 deletions(-) diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index 4ba57a1530e81..7544f7ae26d43 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -102,61 +102,104 @@ const NUM_STREAMS: usize = 8; /// The size of each batch within each stream const BATCH_SIZE: usize = 1024; -/// Total number of input rows to generate -const INPUT_SIZE: u64 = 100000; +/// Input sizes to benchmark. The small size (100K) exercises the +/// in-memory concat-and-sort path; the large size (10M) exercises +/// the sort-then-merge path with high fan-in. +const INPUT_SIZES: &[(u64, &str)] = &[(100_000, "100k"), (1_000_000, "1M")]; type PartitionedBatches = Vec>; +type StreamGenerator = Box PartitionedBatches>; fn criterion_benchmark(c: &mut Criterion) { - let cases: Vec<(&str, &dyn Fn(bool) -> PartitionedBatches)> = vec![ - ("i64", &i64_streams), - ("f64", &f64_streams), - ("utf8 low cardinality", &utf8_low_cardinality_streams), - ("utf8 high cardinality", &utf8_high_cardinality_streams), - ( - "utf8 view low cardinality", - &utf8_view_low_cardinality_streams, - ), - ( - "utf8 view high cardinality", - &utf8_view_high_cardinality_streams, - ), - ("utf8 tuple", &utf8_tuple_streams), - ("utf8 view tuple", &utf8_view_tuple_streams), - ("utf8 dictionary", &dictionary_streams), - ("utf8 dictionary tuple", &dictionary_tuple_streams), - ("mixed dictionary tuple", &mixed_dictionary_tuple_streams), - ("mixed tuple", &mixed_tuple_streams), - ( - "mixed tuple with utf8 view", - &mixed_tuple_with_utf8_view_streams, - ), - ]; - - for (name, f) in cases { - c.bench_function(&format!("merge sorted {name}"), |b| { - let data = f(true); - let case = BenchCase::merge_sorted(&data); - b.iter(move || case.run()) - }); - - c.bench_function(&format!("sort merge {name}"), |b| { - let data = f(false); - let case = BenchCase::sort_merge(&data); - b.iter(move || case.run()) - }); - - c.bench_function(&format!("sort {name}"), |b| { - let data = f(false); - let case = BenchCase::sort(&data); - b.iter(move || case.run()) - }); - - c.bench_function(&format!("sort partitioned {name}"), |b| { - let data = f(false); - let case = BenchCase::sort_partitioned(&data); - b.iter(move || case.run()) - }); + for &(input_size, size_label) in INPUT_SIZES { + let cases: Vec<(&str, StreamGenerator)> = vec![ + ( + "i64", + Box::new(move |sorted| i64_streams(sorted, input_size)), + ), + ( + "f64", + Box::new(move |sorted| f64_streams(sorted, input_size)), + ), + ( + "utf8 low cardinality", + Box::new(move |sorted| utf8_low_cardinality_streams(sorted, input_size)), + ), + ( + "utf8 high cardinality", + Box::new(move |sorted| utf8_high_cardinality_streams(sorted, input_size)), + ), + ( + "utf8 view low cardinality", + Box::new(move |sorted| { + utf8_view_low_cardinality_streams(sorted, input_size) + }), + ), + ( + "utf8 view high cardinality", + Box::new(move |sorted| { + utf8_view_high_cardinality_streams(sorted, input_size) + }), + ), + ( + "utf8 tuple", + Box::new(move |sorted| utf8_tuple_streams(sorted, input_size)), + ), + ( + "utf8 view tuple", + Box::new(move |sorted| utf8_view_tuple_streams(sorted, input_size)), + ), + ( + "utf8 dictionary", + Box::new(move |sorted| dictionary_streams(sorted, input_size)), + ), + ( + "utf8 dictionary tuple", + Box::new(move |sorted| dictionary_tuple_streams(sorted, input_size)), + ), + ( + "mixed dictionary tuple", + Box::new(move |sorted| { + mixed_dictionary_tuple_streams(sorted, input_size) + }), + ), + ( + "mixed tuple", + Box::new(move |sorted| mixed_tuple_streams(sorted, input_size)), + ), + ( + "mixed tuple with utf8 view", + Box::new(move |sorted| { + mixed_tuple_with_utf8_view_streams(sorted, input_size) + }), + ), + ]; + + for (name, f) in &cases { + c.bench_function(&format!("merge sorted {name} {size_label}"), |b| { + let data = f(true); + let case = BenchCase::merge_sorted(&data); + b.iter(move || case.run()) + }); + + c.bench_function(&format!("sort merge {name} {size_label}"), |b| { + let data = f(false); + let case = BenchCase::sort_merge(&data); + b.iter(move || case.run()) + }); + + c.bench_function(&format!("sort {name} {size_label}"), |b| { + let data = f(false); + let case = BenchCase::sort(&data); + b.iter(move || case.run()) + }); + + c.bench_function(&format!("sort partitioned {name} {size_label}"), |b| { + let data = f(false); + let case = BenchCase::sort_partitioned(&data); + b.iter(move || case.run()) + }); + } } } @@ -279,8 +322,8 @@ fn make_sort_exprs(schema: &Schema) -> LexOrdering { } /// Create streams of int64 (where approximately 1/3 values is repeated) -fn i64_streams(sorted: bool) -> PartitionedBatches { - let mut values = DataGenerator::new().i64_values(); +fn i64_streams(sorted: bool, input_size: u64) -> PartitionedBatches { + let mut values = DataGenerator::new(input_size).i64_values(); if sorted { values.sort_unstable(); } @@ -293,8 +336,8 @@ fn i64_streams(sorted: bool) -> PartitionedBatches { /// Create streams of f64 (where approximately 1/3 values are repeated) /// with the same distribution as i64_streams -fn f64_streams(sorted: bool) -> PartitionedBatches { - let mut values = DataGenerator::new().f64_values(); +fn f64_streams(sorted: bool, input_size: u64) -> PartitionedBatches { + let mut values = DataGenerator::new(input_size).f64_values(); if sorted { values.sort_unstable_by(|a, b| a.total_cmp(b)); } @@ -306,8 +349,8 @@ fn f64_streams(sorted: bool) -> PartitionedBatches { } /// Create streams of random low cardinality utf8 values -fn utf8_low_cardinality_streams(sorted: bool) -> PartitionedBatches { - let mut values = DataGenerator::new().utf8_low_cardinality_values(); +fn utf8_low_cardinality_streams(sorted: bool, input_size: u64) -> PartitionedBatches { + let mut values = DataGenerator::new(input_size).utf8_low_cardinality_values(); if sorted { values.sort_unstable(); } @@ -318,8 +361,11 @@ fn utf8_low_cardinality_streams(sorted: bool) -> PartitionedBatches { } /// Create streams of random low cardinality utf8_view values -fn utf8_view_low_cardinality_streams(sorted: bool) -> PartitionedBatches { - let mut values = DataGenerator::new().utf8_low_cardinality_values(); +fn utf8_view_low_cardinality_streams( + sorted: bool, + input_size: u64, +) -> PartitionedBatches { + let mut values = DataGenerator::new(input_size).utf8_low_cardinality_values(); if sorted { values.sort_unstable(); } @@ -330,8 +376,11 @@ fn utf8_view_low_cardinality_streams(sorted: bool) -> PartitionedBatches { } /// Create streams of high cardinality (~ no duplicates) utf8_view values -fn utf8_view_high_cardinality_streams(sorted: bool) -> PartitionedBatches { - let mut values = DataGenerator::new().utf8_high_cardinality_values(); +fn utf8_view_high_cardinality_streams( + sorted: bool, + input_size: u64, +) -> PartitionedBatches { + let mut values = DataGenerator::new(input_size).utf8_high_cardinality_values(); if sorted { values.sort_unstable(); } @@ -342,8 +391,8 @@ fn utf8_view_high_cardinality_streams(sorted: bool) -> PartitionedBatches { } /// Create streams of high cardinality (~ no duplicates) utf8 values -fn utf8_high_cardinality_streams(sorted: bool) -> PartitionedBatches { - let mut values = DataGenerator::new().utf8_high_cardinality_values(); +fn utf8_high_cardinality_streams(sorted: bool, input_size: u64) -> PartitionedBatches { + let mut values = DataGenerator::new(input_size).utf8_high_cardinality_values(); if sorted { values.sort_unstable(); } @@ -354,8 +403,8 @@ fn utf8_high_cardinality_streams(sorted: bool) -> PartitionedBatches { } /// Create a batch of (utf8_low, utf8_low, utf8_high) -fn utf8_tuple_streams(sorted: bool) -> PartitionedBatches { - let mut data_gen = DataGenerator::new(); +fn utf8_tuple_streams(sorted: bool, input_size: u64) -> PartitionedBatches { + let mut data_gen = DataGenerator::new(input_size); // need to sort by the combined key, so combine them together let mut tuples: Vec<_> = data_gen @@ -387,8 +436,8 @@ fn utf8_tuple_streams(sorted: bool) -> PartitionedBatches { } /// Create a batch of (utf8_view_low, utf8_view_low, utf8_view_high) -fn utf8_view_tuple_streams(sorted: bool) -> PartitionedBatches { - let mut data_gen = DataGenerator::new(); +fn utf8_view_tuple_streams(sorted: bool, input_size: u64) -> PartitionedBatches { + let mut data_gen = DataGenerator::new(input_size); // need to sort by the combined key, so combine them together let mut tuples: Vec<_> = data_gen @@ -420,8 +469,8 @@ fn utf8_view_tuple_streams(sorted: bool) -> PartitionedBatches { } /// Create a batch of (f64, utf8_low, utf8_low, i64) -fn mixed_tuple_streams(sorted: bool) -> PartitionedBatches { - let mut data_gen = DataGenerator::new(); +fn mixed_tuple_streams(sorted: bool, input_size: u64) -> PartitionedBatches { + let mut data_gen = DataGenerator::new(input_size); // need to sort by the combined key, so combine them together let mut tuples: Vec<_> = data_gen @@ -458,8 +507,11 @@ fn mixed_tuple_streams(sorted: bool) -> PartitionedBatches { } /// Create a batch of (f64, utf8_view_low, utf8_view_low, i64) -fn mixed_tuple_with_utf8_view_streams(sorted: bool) -> PartitionedBatches { - let mut data_gen = DataGenerator::new(); +fn mixed_tuple_with_utf8_view_streams( + sorted: bool, + input_size: u64, +) -> PartitionedBatches { + let mut data_gen = DataGenerator::new(input_size); // need to sort by the combined key, so combine them together let mut tuples: Vec<_> = data_gen @@ -496,8 +548,8 @@ fn mixed_tuple_with_utf8_view_streams(sorted: bool) -> PartitionedBatches { } /// Create a batch of (utf8_dict) -fn dictionary_streams(sorted: bool) -> PartitionedBatches { - let mut data_gen = DataGenerator::new(); +fn dictionary_streams(sorted: bool, input_size: u64) -> PartitionedBatches { + let mut data_gen = DataGenerator::new(input_size); let mut values = data_gen.utf8_low_cardinality_values(); if sorted { values.sort_unstable(); @@ -511,8 +563,8 @@ fn dictionary_streams(sorted: bool) -> PartitionedBatches { } /// Create a batch of (utf8_dict, utf8_dict, utf8_dict) -fn dictionary_tuple_streams(sorted: bool) -> PartitionedBatches { - let mut data_gen = DataGenerator::new(); +fn dictionary_tuple_streams(sorted: bool, input_size: u64) -> PartitionedBatches { + let mut data_gen = DataGenerator::new(input_size); let mut tuples: Vec<_> = data_gen .utf8_low_cardinality_values() .into_iter() @@ -542,8 +594,8 @@ fn dictionary_tuple_streams(sorted: bool) -> PartitionedBatches { } /// Create a batch of (utf8_dict, utf8_dict, utf8_dict, i64) -fn mixed_dictionary_tuple_streams(sorted: bool) -> PartitionedBatches { - let mut data_gen = DataGenerator::new(); +fn mixed_dictionary_tuple_streams(sorted: bool, input_size: u64) -> PartitionedBatches { + let mut data_gen = DataGenerator::new(input_size); let mut tuples: Vec<_> = data_gen .utf8_low_cardinality_values() .into_iter() @@ -579,19 +631,21 @@ fn mixed_dictionary_tuple_streams(sorted: bool) -> PartitionedBatches { /// Encapsulates creating data for this test struct DataGenerator { rng: StdRng, + input_size: u64, } impl DataGenerator { - fn new() -> Self { + fn new(input_size: u64) -> Self { Self { rng: StdRng::seed_from_u64(42), + input_size, } } /// Create an array of i64 sorted values (where approximately 1/3 values is repeated) fn i64_values(&mut self) -> Vec { - let mut vec: Vec<_> = (0..INPUT_SIZE) - .map(|_| self.rng.random_range(0..INPUT_SIZE as i64)) + let mut vec: Vec<_> = (0..self.input_size) + .map(|_| self.rng.random_range(0..self.input_size as i64)) .collect(); vec.sort_unstable(); @@ -614,7 +668,7 @@ impl DataGenerator { .collect::>(); // pick from the 100 strings randomly - let mut input = (0..INPUT_SIZE) + let mut input = (0..self.input_size) .map(|_| { let idx = self.rng.random_range(0..strings.len()); let s = Arc::clone(&strings[idx]); @@ -629,7 +683,7 @@ impl DataGenerator { /// Create sorted values of high cardinality (~ no duplicates) utf8 values fn utf8_high_cardinality_values(&mut self) -> Vec> { // make random strings - let mut input = (0..INPUT_SIZE) + let mut input = (0..self.input_size) .map(|_| Some(self.random_string())) .collect::>(); From 7b2f28400bb994b3c541e35d1ea8e199c20a2d33 Mon Sep 17 00:00:00 2001 From: Ariel Miculas-Trif Date: Wed, 15 Apr 2026 18:16:22 +0300 Subject: [PATCH 3/6] fix: json scan performance on local files (#21478) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Which issue does this PR close? - Closes #21450 ## Rationale for this change The into_stream() implementation of GetResult (from arrow-rs-objectstore) fetches every 8KiB chunk using a spawn_blocking() task, resulting in a lot of scheduling overhead. Fix this by reading the data directly from the async context, using a buffer size of 8KiBs. This avoids any context switch. ## What changes are included in this PR? ## Are these changes tested? ``` Validated that the initial reported overhead is now much smaller: Comparing json-test-on-main and test-json-improvement -------------------- Benchmark clickbench_2.json -------------------- ┏━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━┓ ┃ Query     ┃ json-test-on-main ┃ test-json-improvement ┃       Change ┃ ┡━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━┩ │ QQuery 0  │        2421.62 ms │            2521.19 ms │    no change │ │ QQuery 1  │        2584.29 ms │            2729.98 ms │ 1.06x slower │ │ QQuery 2  │        2662.11 ms │            2782.29 ms │    no change │ │ QQuery 3  │              FAIL │                  FAIL │ incomparable │ │ QQuery 4  │        2764.78 ms │            2896.46 ms │    no change │ │ QQuery 5  │        2676.46 ms │            2758.01 ms │    no change │ │ QQuery 6  │              FAIL │                  FAIL │ incomparable │ │ QQuery 7  │        2684.50 ms │            2752.37 ms │    no change │ │ QQuery 8  │        2781.21 ms │            2827.46 ms │    no change │ │ QQuery 9  │        3039.17 ms │            3165.29 ms │    no change │ │ QQuery 10 │        2791.32 ms │            2843.44 ms │    no change │ │ QQuery 11 │        2839.05 ms │            3011.84 ms │ 1.06x slower │ │ QQuery 12 │        2691.51 ms │            2839.97 ms │ 1.06x slower │ │ QQuery 13 │        2768.57 ms │            2860.68 ms │    no change │ │ QQuery 14 │        2712.50 ms │            2856.80 ms │ 1.05x slower │ │ QQuery 15 │        2807.64 ms │            2888.94 ms │    no change │ │ QQuery 16 │        2774.87 ms │            2875.44 ms │    no change │ │ QQuery 17 │        2797.28 ms │            2850.17 ms │    no change │ │ QQuery 18 │        3017.75 ms │            3111.64 ms │    no change │ │ QQuery 19 │        2801.30 ms │            2927.25 ms │    no change │ │ QQuery 20 │        2743.43 ms │            2862.10 ms │    no change │ │ QQuery 21 │        2811.41 ms │            2906.42 ms │    no change │ │ QQuery 22 │        2953.66 ms │            3038.23 ms │    no change │ │ QQuery 23 │              FAIL │                  FAIL │ incomparable │ │ QQuery 24 │        2862.27 ms │            2940.31 ms │    no change │ │ QQuery 25 │        2763.40 ms │            2848.55 ms │    no change │ │ QQuery 26 │        2840.39 ms │            2950.47 ms │    no change │ │ QQuery 27 │        2886.70 ms │            2921.28 ms │    no change │ │ QQuery 28 │        3145.39 ms │            3221.27 ms │    no change │ │ QQuery 29 │        2821.87 ms │            2869.85 ms │    no change │ │ QQuery 30 │        2953.55 ms │            2990.15 ms │    no change │ │ QQuery 31 │        2997.81 ms │            3049.28 ms │    no change │ │ QQuery 32 │        2969.14 ms │            3126.79 ms │ 1.05x slower │ │ QQuery 33 │        2764.80 ms │            2866.63 ms │    no change │ │ QQuery 34 │        2828.77 ms │            2848.54 ms │    no change │ │ QQuery 35 │        2812.55 ms │            2793.79 ms │    no change │ │ QQuery 36 │              FAIL │                  FAIL │ incomparable │ │ QQuery 37 │              FAIL │                  FAIL │ incomparable │ │ QQuery 38 │              FAIL │                  FAIL │ incomparable │ │ QQuery 39 │              FAIL │                  FAIL │ incomparable │ │ QQuery 40 │              FAIL │                  FAIL │ incomparable │ │ QQuery 41 │              FAIL │                  FAIL │ incomparable │ │ QQuery 42 │              FAIL │                  FAIL │ incomparable │ └───────────┴───────────────────┴───────────────────────┴──────────────┘ ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━┓ ┃ Benchmark Summary                    ┃            ┃ ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━┩ │ Total Time (json-test-on-main)       │ 92771.07ms │ │ Total Time (test-json-improvement)   │ 95732.89ms │ │ Average Time (json-test-on-main)     │  2811.24ms │ │ Average Time (test-json-improvement) │  2901.00ms │ │ Queries Faster                       │          0 │ │ Queries Slower                       │          5 │ │ Queries with No Change               │         28 │ │ Queries with Failure                 │         10 │ └──────────────────────────────────────┴────────────┘ ``` and with SIMULATE_LATENCY: ``` -------------------- Benchmark clickbench_2.json -------------------- ┏━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━┓ ┃ Query     ┃ json-test-on-main ┃ test-json-improvement ┃        Change ┃ ┡━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━┩ │ QQuery 0  │        2795.68 ms │            2687.68 ms │     no change │ │ QQuery 1  │        2880.50 ms │            2768.30 ms │     no change │ │ QQuery 2  │        2960.75 ms │            2826.89 ms │     no change │ │ QQuery 3  │              FAIL │                  FAIL │  incomparable │ │ QQuery 4  │        3140.38 ms │            2963.15 ms │ +1.06x faster │ │ QQuery 5  │        2926.66 ms │            2830.43 ms │     no change │ │ QQuery 6  │              FAIL │                  FAIL │  incomparable │ │ QQuery 7  │        3026.29 ms │            2858.30 ms │ +1.06x faster │ │ QQuery 8  │        4302.35 ms │            2954.96 ms │ +1.46x faster │ │ QQuery 9  │        4439.83 ms │            3200.43 ms │ +1.39x faster │ │ QQuery 10 │        3028.32 ms │            2969.32 ms │     no change │ │ QQuery 11 │        3147.81 ms │            3040.74 ms │     no change │ │ QQuery 12 │        4169.45 ms │            2886.59 ms │ +1.44x faster │ │ QQuery 13 │        3839.01 ms │            2997.80 ms │ +1.28x faster │ │ QQuery 14 │        4086.30 ms │            2907.42 ms │ +1.41x faster │ │ QQuery 15 │        4308.07 ms │            3025.22 ms │ +1.42x faster │ │ QQuery 16 │        3084.89 ms │            2984.34 ms │     no change │ │ QQuery 17 │        4287.89 ms │            2984.27 ms │ +1.44x faster │ │ QQuery 18 │        3542.80 ms │            3144.98 ms │ +1.13x faster │ │ QQuery 19 │        4388.70 ms │            3014.37 ms │ +1.46x faster │ │ QQuery 20 │        3149.54 ms │            2986.73 ms │ +1.05x faster │ │ QQuery 21 │        3250.81 ms │            2906.60 ms │ +1.12x faster │ │ QQuery 22 │        3265.98 ms │            3122.25 ms │     no change │ │ QQuery 23 │              FAIL │                  FAIL │  incomparable │ │ QQuery 24 │        3066.52 ms │            2997.55 ms │     no change │ │ QQuery 25 │        4289.31 ms │            2884.22 ms │ +1.49x faster │ │ QQuery 26 │        4223.03 ms │            2933.16 ms │ +1.44x faster │ │ QQuery 27 │        3156.86 ms │            3001.17 ms │     no change │ │ QQuery 28 │        4831.42 ms │            3318.89 ms │ +1.46x faster │ │ QQuery 29 │        3252.45 ms │            4375.90 ms │  1.35x slower │ │ QQuery 30 │        4460.06 ms │            3153.77 ms │ +1.41x faster │ │ QQuery 31 │        4235.85 ms │            3171.58 ms │ +1.34x faster │ │ QQuery 32 │        3435.14 ms │            3202.64 ms │ +1.07x faster │ │ QQuery 33 │        3147.21 ms │            3031.54 ms │     no change │ │ QQuery 34 │        4378.41 ms │            3008.79 ms │ +1.46x faster │ │ QQuery 35 │        4224.36 ms │            2897.53 ms │ +1.46x faster │ │ QQuery 36 │              FAIL │                  FAIL │  incomparable │ │ QQuery 37 │              FAIL │                  FAIL │  incomparable │ │ QQuery 38 │              FAIL │                  FAIL │  incomparable │ │ QQuery 39 │              FAIL │                  FAIL │  incomparable │ │ QQuery 40 │              FAIL │                  FAIL │  incomparable │ │ QQuery 41 │              FAIL │                  FAIL │  incomparable │ │ QQuery 42 │              FAIL │                  FAIL │  incomparable │ └───────────┴───────────────────┴───────────────────────┴───────────────┘ ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━┓ ┃ Benchmark Summary                    ┃             ┃ ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━┩ │ Total Time (json-test-on-main)       │ 120722.63ms │ │ Total Time (test-json-improvement)   │ 100037.48ms │ │ Average Time (json-test-on-main)     │   3658.26ms │ │ Average Time (test-json-improvement) │   3031.44ms │ │ Queries Faster                       │          21 │ │ Queries Slower                       │           1 │ │ Queries with No Change               │          11 │ │ Queries with Failure                 │          10 │ └──────────────────────────────────────┴─────────────┘ ``` For the tests I've used a c7a.16xlarge ec2 instance, with a trimmed down version of hits.json to 51G (original has 217 GiB), with a warm cache (by running `cat hits_50.json > /dev/null`) ## Are there any user-facing changes? No --- .../datasource-json/src/boundary_stream.rs | 46 ++++++++++++++++++- 1 file changed, 44 insertions(+), 2 deletions(-) diff --git a/datafusion/datasource-json/src/boundary_stream.rs b/datafusion/datasource-json/src/boundary_stream.rs index fc40feda6b80f..847c80279a53e 100644 --- a/datafusion/datasource-json/src/boundary_stream.rs +++ b/datafusion/datasource-json/src/boundary_stream.rs @@ -28,7 +28,7 @@ use std::task::{Context, Poll}; use bytes::Bytes; use futures::stream::{BoxStream, Stream}; use futures::{StreamExt, TryFutureExt}; -use object_store::{GetOptions, GetRange, ObjectStore}; +use object_store::{GetOptions, GetRange, GetResultPayload, ObjectStore}; /// How far past `raw_end` the initial bounded fetch covers. If the terminating /// newline is not found within this window, `ScanningLastTerminator` issues @@ -90,10 +90,52 @@ async fn get_stream( range: std::ops::Range, ) -> object_store::Result>> { let opts = GetOptions { - range: Some(GetRange::Bounded(range)), + range: Some(GetRange::Bounded(range.clone())), ..Default::default() }; let result = store.get_opts(&location, opts).await?; + + #[cfg(not(target_arch = "wasm32"))] + if let GetResultPayload::File(mut file, _path) = result.payload { + use std::io::{Read, Seek, SeekFrom}; + const CHUNK_SIZE: u64 = 8 * 1024; + + file.seek(SeekFrom::Start(range.start)).map_err(|e| { + object_store::Error::Generic { + store: "local", + source: Box::new(e), + } + })?; + + return Ok(futures::stream::try_unfold( + (file, range.end - range.start), + move |(mut file, remaining)| async move { + if remaining == 0 { + return Ok(None); + } + let to_read = remaining.min(CHUNK_SIZE); + let cap = usize::try_from(to_read).map_err(|e| { + object_store::Error::Generic { + store: "local", + source: Box::new(e), + } + })?; + + let mut buf = Vec::with_capacity(cap); + let read = + (&mut file) + .take(to_read) + .read_to_end(&mut buf) + .map_err(|e| object_store::Error::Generic { + store: "local", + source: Box::new(e), + })?; + Ok(Some((Bytes::from(buf), (file, remaining - read as u64)))) + }, + ) + .boxed()); + } + Ok(result.into_stream()) } From edf8ad3930d57b435d6a2acbe7d39be918232148 Mon Sep 17 00:00:00 2001 From: Kumar Ujjawal Date: Wed, 15 Apr 2026 20:56:11 +0530 Subject: [PATCH 4/6] fix(benchmarks): correct TPC-H benchmark SQL (#21615) ## Which issue does this PR close? - Closes #21368. ## Rationale for this change TPC-H query 11 was using a fixed `0.0001` threshold in the HAVING clause. Per the TPC-H spec, this value must be `0.0001 / SF`. This means the benchmark query is only correct for scale factor 1. For larger scale factors the filter becomes too strict, and for smaller scale factors it becomes too loose. There are also few benchmark queries using fixed end dates where the spec uses `date + interval`. Those are equivalent but using intervals matches the TPC-H query definitions more closely. ## What changes are included in this PR? - Make TPC-H query 11 use scale-factor substitution. - Add scale factor support in the benchmark runner. - Infer scale factor from dataset paths like `tpch_sf10`. - Pass the scale factor from `bench.sh`. - Keep the old query-loading entry point working with the default scale factor of 1. - Update query 5, 6, 10, 12, and 14 to use interval-based date ranges. - Add regression tests for scale-factor substitution, scale-factor parsing, and invalid scale factors. ## Are these changes tested? Yes ## Are there any user-facing changes? TPC-H benchmark query 11 now returns correct results when the scale factor is not 1. --- benchmarks/bench.sh | 4 +- benchmarks/queries/q10.sql | 2 +- benchmarks/queries/q11.sql | 4 +- benchmarks/queries/q12.sql | 4 +- benchmarks/queries/q14.sql | 2 +- benchmarks/queries/q5.sql | 4 +- benchmarks/queries/q6.sql | 4 +- benchmarks/src/tpch/mod.rs | 86 +++++++++++++++++++++++++++++++++ benchmarks/src/tpch/run.rs | 98 +++++++++++++++++++++++++++++++++++--- 9 files changed, 190 insertions(+), 18 deletions(-) diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index abd1801213710..9dce4cf77b933 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -677,7 +677,7 @@ run_tpch() { echo "Running tpch benchmark..." FORMAT=$2 - debug_run $CARGO_COMMAND --bin dfbench -- tpch --iterations 5 --path "${TPCH_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --format ${FORMAT} -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} + debug_run $CARGO_COMMAND --bin dfbench -- tpch --iterations 5 --path "${TPCH_DIR}" --scale-factor "${SCALE_FACTOR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --format ${FORMAT} -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} } # Runs the tpch in memory (needs tpch parquet data) @@ -693,7 +693,7 @@ run_tpch_mem() { echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running tpch_mem benchmark..." # -m means in memory - debug_run $CARGO_COMMAND --bin dfbench -- tpch --iterations 5 --path "${TPCH_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" -m --format parquet -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} + debug_run $CARGO_COMMAND --bin dfbench -- tpch --iterations 5 --path "${TPCH_DIR}" --scale-factor "${SCALE_FACTOR}" --prefer_hash_join "${PREFER_HASH_JOIN}" -m --format parquet -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} } # Runs the tpcds benchmark diff --git a/benchmarks/queries/q10.sql b/benchmarks/queries/q10.sql index 8613fd4962837..8ac2fd90798c9 100644 --- a/benchmarks/queries/q10.sql +++ b/benchmarks/queries/q10.sql @@ -16,7 +16,7 @@ where c_custkey = o_custkey and l_orderkey = o_orderkey and o_orderdate >= date '1993-10-01' - and o_orderdate < date '1994-01-01' + and o_orderdate < date '1993-10-01' + interval '3' month and l_returnflag = 'R' and c_nationkey = n_nationkey group by diff --git a/benchmarks/queries/q11.sql b/benchmarks/queries/q11.sql index c23ed1c71bfb3..9a9710d09ec35 100644 --- a/benchmarks/queries/q11.sql +++ b/benchmarks/queries/q11.sql @@ -13,7 +13,7 @@ group by ps_partkey having sum(ps_supplycost * ps_availqty) > ( select - sum(ps_supplycost * ps_availqty) * 0.0001 + sum(ps_supplycost * ps_availqty) * 0.0001 /* __TPCH_Q11_FRACTION__ */ from partsupp, supplier, @@ -24,4 +24,4 @@ group by and n_name = 'GERMANY' ) order by - value desc; \ No newline at end of file + value desc; diff --git a/benchmarks/queries/q12.sql b/benchmarks/queries/q12.sql index f8e6d960c8420..c3f4d62344701 100644 --- a/benchmarks/queries/q12.sql +++ b/benchmarks/queries/q12.sql @@ -23,8 +23,8 @@ where and l_commitdate < l_receiptdate and l_shipdate < l_commitdate and l_receiptdate >= date '1994-01-01' - and l_receiptdate < date '1995-01-01' + and l_receiptdate < date '1994-01-01' + interval '1' year group by l_shipmode order by - l_shipmode; \ No newline at end of file + l_shipmode; diff --git a/benchmarks/queries/q14.sql b/benchmarks/queries/q14.sql index d8ef6afaca9bb..6fe88c42662d0 100644 --- a/benchmarks/queries/q14.sql +++ b/benchmarks/queries/q14.sql @@ -10,4 +10,4 @@ from where l_partkey = p_partkey and l_shipdate >= date '1995-09-01' - and l_shipdate < date '1995-10-01'; \ No newline at end of file + and l_shipdate < date '1995-09-01' + interval '1' month; diff --git a/benchmarks/queries/q5.sql b/benchmarks/queries/q5.sql index 5a336b231184b..146980ccd6f76 100644 --- a/benchmarks/queries/q5.sql +++ b/benchmarks/queries/q5.sql @@ -17,8 +17,8 @@ where and n_regionkey = r_regionkey and r_name = 'ASIA' and o_orderdate >= date '1994-01-01' - and o_orderdate < date '1995-01-01' + and o_orderdate < date '1994-01-01' + interval '1' year group by n_name order by - revenue desc; \ No newline at end of file + revenue desc; diff --git a/benchmarks/queries/q6.sql b/benchmarks/queries/q6.sql index 5806f980f8088..5a13fe7df765a 100644 --- a/benchmarks/queries/q6.sql +++ b/benchmarks/queries/q6.sql @@ -4,6 +4,6 @@ from lineitem where l_shipdate >= date '1994-01-01' - and l_shipdate < date '1995-01-01' + and l_shipdate < date '1994-01-01' + interval '1' year and l_discount between 0.06 - 0.01 and 0.06 + 0.01 - and l_quantity < 24; \ No newline at end of file + and l_quantity < 24; diff --git a/benchmarks/src/tpch/mod.rs b/benchmarks/src/tpch/mod.rs index 681aa0a403ee1..08cedc0e5b4c3 100644 --- a/benchmarks/src/tpch/mod.rs +++ b/benchmarks/src/tpch/mod.rs @@ -33,6 +33,7 @@ pub const TPCH_TABLES: &[&str] = &[ pub const TPCH_QUERY_START_ID: usize = 1; pub const TPCH_QUERY_END_ID: usize = 22; +const TPCH_Q11_FRACTION_SENTINEL: &str = "0.0001 /* __TPCH_Q11_FRACTION__ */"; /// The `.tbl` file contains a trailing column pub fn get_tbl_tpch_table_schema(table: &str) -> Schema { @@ -139,6 +140,21 @@ pub fn get_tpch_table_schema(table: &str) -> Schema { /// Get the SQL statements from the specified query file pub fn get_query_sql(query: usize) -> Result> { + get_query_sql_for_scale_factor(query, 1.0) +} + +/// Get the SQL statements from the specified query file using the provided scale factor for +/// TPC-H substitutions such as Q11 FRACTION. +pub fn get_query_sql_for_scale_factor( + query: usize, + scale_factor: f64, +) -> Result> { + if !(scale_factor.is_finite() && scale_factor > 0.0) { + return plan_err!( + "invalid scale factor. Expected a positive finite value, got {scale_factor}" + ); + } + if query > 0 && query < 23 { let possibilities = vec![ format!("queries/q{query}.sql"), @@ -148,6 +164,7 @@ pub fn get_query_sql(query: usize) -> Result> { for filename in possibilities { match fs::read_to_string(&filename) { Ok(contents) => { + let contents = customize_query_sql(query, contents, scale_factor)?; return Ok(contents .split(';') .map(|s| s.trim()) @@ -164,6 +181,27 @@ pub fn get_query_sql(query: usize) -> Result> { } } +fn customize_query_sql( + query: usize, + contents: String, + scale_factor: f64, +) -> Result { + if query != 11 { + return Ok(contents); + } + + if !contents.contains(TPCH_Q11_FRACTION_SENTINEL) { + return plan_err!( + "invalid query 11. Missing fraction marker {TPCH_Q11_FRACTION_SENTINEL}" + ); + } + + Ok(contents.replace( + TPCH_Q11_FRACTION_SENTINEL, + &format!("(0.0001 / {scale_factor})"), + )) +} + pub const QUERY_LIMIT: [Option; 22] = [ None, Some(100), @@ -188,3 +226,51 @@ pub const QUERY_LIMIT: [Option; 22] = [ Some(100), None, ]; + +#[cfg(test)] +mod tests { + use super::{get_query_sql, get_query_sql_for_scale_factor}; + use datafusion::error::Result; + + fn get_single_query(query: usize) -> Result { + let mut queries = get_query_sql(query)?; + assert_eq!(queries.len(), 1); + Ok(queries.remove(0)) + } + + fn get_single_query_for_scale_factor( + query: usize, + scale_factor: f64, + ) -> Result { + let mut queries = get_query_sql_for_scale_factor(query, scale_factor)?; + assert_eq!(queries.len(), 1); + Ok(queries.remove(0)) + } + + #[test] + fn q11_uses_scale_factor_substitution() -> Result<()> { + let sf1_sql = get_single_query(11)?; + assert!(sf1_sql.contains("(0.0001 / 1)")); + + let sf01_sql = get_single_query_for_scale_factor(11, 0.1)?; + assert!(sf01_sql.contains("(0.0001 / 0.1)")); + + let sf10_sql = get_single_query_for_scale_factor(11, 10.0)?; + assert!(sf10_sql.contains("(0.0001 / 10)")); + + let sf30_sql = get_single_query_for_scale_factor(11, 30.0)?; + assert!(sf30_sql.contains("(0.0001 / 30)")); + assert!(!sf10_sql.contains("__TPCH_Q11_FRACTION__")); + Ok(()) + } + + #[test] + fn interval_queries_use_interval_arithmetic() -> Result<()> { + assert!(get_single_query(5)?.contains("date '1994-01-01' + interval '1' year")); + assert!(get_single_query(6)?.contains("date '1994-01-01' + interval '1' year")); + assert!(get_single_query(10)?.contains("date '1993-10-01' + interval '3' month")); + assert!(get_single_query(12)?.contains("date '1994-01-01' + interval '1' year")); + assert!(get_single_query(14)?.contains("date '1995-09-01' + interval '1' month")); + Ok(()) + } +} diff --git a/benchmarks/src/tpch/run.rs b/benchmarks/src/tpch/run.rs index 0d1268013c168..ec7aa8c554a28 100644 --- a/benchmarks/src/tpch/run.rs +++ b/benchmarks/src/tpch/run.rs @@ -15,17 +15,18 @@ // specific language governing permissions and limitations // under the License. -use std::path::PathBuf; +use std::path::{Path, PathBuf}; use std::sync::Arc; use super::{ - TPCH_QUERY_END_ID, TPCH_QUERY_START_ID, TPCH_TABLES, get_query_sql, + TPCH_QUERY_END_ID, TPCH_QUERY_START_ID, TPCH_TABLES, get_query_sql_for_scale_factor, get_tbl_tpch_table_schema, get_tpch_table_schema, }; use crate::util::{BenchmarkRun, CommonOpt, QueryResult, print_memory_stats}; use arrow::record_batch::RecordBatch; use arrow::util::pretty::{self, pretty_format_batches}; +use datafusion::common::exec_err; use datafusion::datasource::file_format::FileFormat; use datafusion::datasource::file_format::csv::CsvFormat; use datafusion::datasource::file_format::parquet::ParquetFormat; @@ -71,6 +72,11 @@ pub struct RunOpt { #[arg(required = true, short = 'p', long = "path")] path: PathBuf, + /// TPC-H scale factor used for query substitutions such as Q11 FRACTION. + /// If omitted, the benchmark tries to infer it from paths like `.../tpch_sf10/...`. + #[arg(long)] + scale_factor: Option, + /// File format: `csv` or `parquet` #[arg(short = 'f', long = "format", default_value = "csv")] file_format: String, @@ -133,10 +139,11 @@ impl RunOpt { let ctx = SessionContext::new_with_config_rt(config, rt); // register tables self.register_tables(&ctx).await?; + let scale_factor = self.scale_factor()?; for query_id in query_range { benchmark_run.start_new_case(&format!("Query {query_id}")); - let query_run = self.benchmark_query(query_id, &ctx).await; + let query_run = self.benchmark_query(query_id, scale_factor, &ctx).await; match query_run { Ok(query_results) => { for iter in query_results { @@ -157,13 +164,14 @@ impl RunOpt { async fn benchmark_query( &self, query_id: usize, + scale_factor: f64, ctx: &SessionContext, ) -> Result> { let mut millis = vec![]; // run benchmark let mut query_results = vec![]; - let sql = &get_query_sql(query_id)?; + let sql = &get_query_sql_for_scale_factor(query_id, scale_factor)?; for i in 0..self.iterations() { let start = Instant::now(); @@ -346,6 +354,82 @@ impl RunOpt { .partitions .unwrap_or_else(get_available_parallelism) } + + fn scale_factor(&self) -> Result { + resolve_scale_factor(self.scale_factor, &self.path) + } +} + +fn resolve_scale_factor(scale_factor: Option, path: &Path) -> Result { + let scale_factor = scale_factor + .or_else(|| infer_scale_factor_from_path(path)) + .unwrap_or(1.0); + + if scale_factor.is_finite() && scale_factor > 0.0 { + Ok(scale_factor) + } else { + exec_err!( + "Invalid TPC-H scale factor {scale_factor}. Expected a positive finite value" + ) + } +} + +fn infer_scale_factor_from_path(path: &Path) -> Option { + path.iter().find_map(|component| { + component + .to_str()? + .strip_prefix("tpch_sf")? + .parse::() + .ok() + }) +} + +#[cfg(test)] +mod scale_factor_tests { + use std::path::Path; + + use super::{infer_scale_factor_from_path, resolve_scale_factor}; + use datafusion::error::Result; + + #[test] + fn uses_explicit_scale_factor_when_provided() -> Result<()> { + let scale_factor = + resolve_scale_factor(Some(30.0), Path::new("benchmarks/data/tpch_sf10"))?; + assert_eq!(scale_factor, 30.0); + Ok(()) + } + + #[test] + fn infers_scale_factor_from_standard_tpch_path() -> Result<()> { + let scale_factor = + resolve_scale_factor(None, Path::new("benchmarks/data/tpch_sf10"))?; + assert_eq!(scale_factor, 10.0); + assert_eq!( + infer_scale_factor_from_path(Path::new("benchmarks/data/tpch_sf0.1")), + Some(0.1) + ); + Ok(()) + } + + #[test] + fn defaults_to_sf1_when_path_has_no_scale_factor() -> Result<()> { + let scale_factor = resolve_scale_factor(None, Path::new("benchmarks/data"))?; + assert_eq!(scale_factor, 1.0); + Ok(()) + } + + #[test] + fn rejects_invalid_scale_factors() { + assert!(resolve_scale_factor(Some(0.0), Path::new("benchmarks/data")).is_err()); + assert!(resolve_scale_factor(Some(-1.0), Path::new("benchmarks/data")).is_err()); + assert!( + resolve_scale_factor(Some(f64::NAN), Path::new("benchmarks/data")).is_err() + ); + assert!( + resolve_scale_factor(Some(f64::INFINITY), Path::new("benchmarks/data")) + .is_err() + ); + } } #[cfg(test)] @@ -392,6 +476,7 @@ mod tests { query: Some(query), common, path: PathBuf::from(path.to_string()), + scale_factor: Some(1.0), file_format: "tbl".to_string(), mem_table: false, output_path: None, @@ -402,7 +487,7 @@ mod tests { hash_join_buffering_capacity: 0, }; opt.register_tables(&ctx).await?; - let queries = get_query_sql(query)?; + let queries = crate::tpch::get_query_sql(query)?; for query in queries { let plan = ctx.sql(&query).await?; let plan = plan.into_optimized_plan()?; @@ -432,6 +517,7 @@ mod tests { query: Some(query), common, path: PathBuf::from(path.to_string()), + scale_factor: Some(1.0), file_format: "tbl".to_string(), mem_table: false, output_path: None, @@ -442,7 +528,7 @@ mod tests { hash_join_buffering_capacity: 0, }; opt.register_tables(&ctx).await?; - let queries = get_query_sql(query)?; + let queries = crate::tpch::get_query_sql(query)?; for query in queries { let plan = ctx.sql(&query).await?; let plan = plan.create_physical_plan().await?; From f99ba69c618947a21cd0b9af282cbddda20394c4 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 15 Apr 2026 11:40:05 -0400 Subject: [PATCH 5/6] Add release management page to the documentation (#21001) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Which issue does this PR close? - Related to #19692 ## Rationale for this change As we see more people helping with releases (thank you @timsaucer and @comphead ❤️ ) I think having the process documented more will help Also, I spent quite a while back porting PRs and wanted to document it better ## What changes are included in this PR? 1. Add a release management page 2. Leave links to relevant sections ## Are these changes tested? By CI ## Are there any user-facing changes? Docs only --- dev/release/README.md | 166 +++++++----------- docs/source/contributor-guide/index.md | 5 + .../contributor-guide/release_management.md | 123 +++++++++++++ docs/source/index.rst | 1 + 4 files changed, 195 insertions(+), 100 deletions(-) create mode 100644 docs/source/contributor-guide/release_management.md diff --git a/dev/release/README.md b/dev/release/README.md index 533bfc20ee7c9..2f99d52b200d2 100644 --- a/dev/release/README.md +++ b/dev/release/README.md @@ -19,59 +19,11 @@ # Release Process -DataFusion typically has major releases around once per month, including breaking API changes. +For contributor-facing guidance on release branches and backports, see the +[Contributor Guide Release Management page](../../docs/source/contributor-guide/release_management.md). -Patch releases are made on an adhoc basis, but we try and avoid them given the frequent major releases. - -## Release Process Overview - -New development happens on the `main` branch. -Releases are made from branches, e.g. `branch-50` for the `50.x.y` release series. - -To prepare for a new release series, we: - -- Create a new branch from `main`, such as `branch-50` in the Apache repository (not in a fork) -- Continue merging new features changes to `main` branch -- Prepare the release branch for release: - - Update version numbers in `Cargo.toml` files and create `CHANGELOG.md` - - Add additional changes to the release branch as needed -- When the code is ready, create GitHub tags release candidate (rc) artifacts from the release branch. -- After the release is approved, publish to [crates.io], the ASF distribution servers, and GitHub tags. - -To add changes to the release branch, depending on the change we either: - -- Fix the issue on `main` and then backport the change to the release branch (e.g. [#18129]) -- Fix the issue on the release branch and then forward-port the change back to `main` (e.g.[#18057]) - -[crates.io]: https://crates.io/crates/datafusion -[#18129]: https://github.com/apache/datafusion/pull/18129 -[#18057]: https://github.com/apache/datafusion/pull/18057 - -## Backporting (add changes) to `branch-*` branch - -If you would like to propose your change for inclusion in a patch release, the -change must be applied to the relevant release branch. To do so please follow -these steps: - -1. Find (or create) the issue for the incremental release ([example release issue]) and discuss the proposed change there with the maintainers. -2. Follow normal workflow to create PR to `main` branch and wait for its approval and merge. -3. After PR is squash merged to `main`, branch from most recent release branch (e.g. `branch-50`), cherry-pick the commit and create a PR targeting the release branch [example backport PR]. - -For example, to backport commit `12345` from `main` to `branch-50`: - -```shell -git checkout branch-50 -git checkout -b backport_to_50 -git cherry-pick 12345 # your git commit hash -git push -u -# make a PR as normal targeting branch-50, prefixed with [branch-50] -``` - -It is also acceptable to fix the issue directly on the release branch first -and then cherry-pick the change back to `main` branch in a new PR. - -[example release issue]: https://github.com/apache/datafusion/issues/18072 -[example backport pr]: https://github.com/apache/datafusion/pull/18131 +This guide is for maintainers to create release candidates and run the release +process. ## Release Prerequisites @@ -160,33 +112,35 @@ branch-50: First, prepare a PR to update the changelog and versions to reflect the planned release. See [#18173](https://github.com/apache/datafusion/pull/18173) for an example. -#### Update Version Numbers - -Manually update the DataFusion version in the root `Cargo.toml` to reflect the new release version. +- This PR should be the last one merged before creating a release + candidate, so the changelog includes all changes. +- If there are code changes between RCs, create and merge a new PR to update + the changelog before creating the next RC. -Ensure Cargo.lock is updated accordingly by running: +**Step 1**: Manually update the DataFusion version in the root `Cargo.toml` to +reflect the new release version. Ensure `Cargo.lock` is updated accordingly by +running: ```shell cargo check -p datafusion ``` -#### Changelog Generation - -We maintain a [changelog] so our users know what has been changed between releases. - -[changelog]: ../changelog - -The changelog is generated using a Python script. +**Step 2**: Update the [changelog] in `dev/changelog/`. Each release has its +own file, such as `dev/changelog/50.0.0.md`, which should include all changes +since the previous release. -To run the script, you will need a GitHub Personal Access Token (described in the prerequisites section) and the `PyGitHub` library. First install the dev dependencies via `uv`: +The changelog is generated using a Python script, which requires a GitHub +Personal Access Token (described in the prerequisites section) and the +`PyGitHub` library. First install the dev dependencies via `uv`: ```shell uv sync ``` -To generate the changelog, set the `GITHUB_TOKEN` environment variable and then run `./dev/release/generate-changelog.py` -providing two commit ids or tags followed by the version number of the release being created. For example, -to generate a change log of all changes between the `50.3.0` tag and `branch-51`, in preparation for release `51.0.0`: +To generate the changelog, set the `GITHUB_TOKEN` environment variable and run +`./dev/release/generate-changelog.py` with two commit IDs or tags followed by +the release version. For example, to generate a changelog of all changes +between the `50.3.0` tag and `branch-51` for release `51.0.0`: > [!NOTE] > @@ -205,52 +159,52 @@ uv run ./dev/release/generate-changelog.py 50.3.0 branch-51 51.0.0 > dev/changel This script creates a changelog from GitHub PRs based on the labels associated with them as well as looking for titles starting with `feat:`, `fix:`, or `docs:`. -Once the change log is generated, run `prettier` to format the document: +Once the changelog is generated, run `prettier` to format the document: ```shell prettier -w dev/changelog/51.0.0.md ``` -#### Commit and PR - Then commit the changes and create a PR targeting the release branch. ```shell git commit -a -m 'Update version' ``` -Remember to merge any fixes back to `main` branch as well. +Remember to merge any fixes back to the `main` branch as well. ### 4. Prepare Release Candidate Artifacts After the PR gets merged, you are ready to create release artifacts based off the merged commit. -(Note you need to be a committer to run these scripts as they upload to the apache svn distribution servers) +- You must be a committer to run these scripts because they upload to the + Apache SVN distribution servers. +- If there are code changes between RCs, create and merge a changelog PR before + creating the next RC. #### Pick a Release Candidate (RC) number Pick numbers in sequential order, with `1` for `rc1`, `2` for `rc2`, etc. -#### Create git Tag for the Release: +#### Create Git tag for the Release Candidate While the official release artifacts are signed tarballs and zip files, we also -tag the commit it was created for convenience and code archaeology. Release tags -have the format `` (e.g. `38.0.0`), and release candidates have the -format `-rc` (e.g. `38.0.0-rc0`). See [the list of existing +tag the commit it was created from for convenience and code archaeology. Release tags +look like `38.0.0`, and release candidate tags look like `38.0.0-rc0`. See [the list of existing tags]. [the list of existing tags]: https://github.com/apache/datafusion/tags -Using a string such as `38.0.0` as the ``, create and push the rc tag by running these commands: +Create and push the RC tag: ```shell git fetch apache -git tag - apache/branch-X # create tag from the release branch -git push apache - # push tag to Github remote +git tag -rc apache/branch-X +git push apache -rc ``` -For example, to create the `50.3.0-rc1 tag from `branch-50`: +For example, to create the `50.3.0-rc1` tag from `branch-50`, use: ```shell git fetch apache @@ -260,7 +214,7 @@ git push apache 50.3.0-rc1 #### Create, Sign, and Upload Artifacts -Run the `create-tarball.sh` script with the `` tag and `` and you determined in previous steps: +Run the `create-tarball.sh` script with the `` tag and `` number you determined in previous steps: For example, to create the `50.3.0-rc1` artifacts: @@ -275,18 +229,20 @@ The `create-tarball.sh` script apache distribution SVN server 2. Provides you an email template to - send to dev@datafusion.apache.org for release voting. + send to `dev@datafusion.apache.org` for release voting. ### 5. Vote on Release Candidate Artifacts -Send the email output from the script to dev@datafusion.apache.org. +Send the email output from the script to `dev@datafusion.apache.org`. -In order to publish the release on crates.io, it must be "official". To become -official it needs at least three PMC members to vote +1 on it. +In order to publish the release on crates.io, it must be "official." To become +official, it needs at least three PMC members to vote +1 on it and no -1 votes. +The vote must remain open for at least 72 hours to give everyone a chance to +review the release candidate. #### Verifying Release Candidates -The `dev/release/verify-release-candidate.sh` is a script in this repository that can assist in the verification process. Run it like: +`dev/release/verify-release-candidate.sh` is a script in this repository that can assist in the verification process. Run it like this: ```shell ./dev/release/verify-release-candidate.sh 50.3.0 1 @@ -294,15 +250,18 @@ The `dev/release/verify-release-candidate.sh` is a script in this repository tha #### If the Release is not Approved -If the release is not approved, fix whatever the problem is, merge changelog -changes into the release branch and try again with the next RC number. +If the release is not approved, fix whatever the problem is, make a new release +candidate and try again. + +Reminders: -Remember to merge any fixes back to `main` branch as well. +1. Update the changelog and create a new PR if there are any code changes, and merge it before creating the next RC. +2. Merge any fixes back to the `main` branch as well. -#### If the Release is Approved: Call the Vote +#### If the Vote Passes: Announce the Result Call the vote on the Arrow dev list by replying to the RC voting thread. The -reply should have a new subject constructed by adding `[RESULT]` prefix to the +reply should have a new subject constructed by adding the `[RESULT]` prefix to the old subject line. Sample announcement template: @@ -415,39 +374,46 @@ the examples from previous releases. The release information is used to generate a template for a board report (see example from Apache Arrow project [here](https://github.com/apache/arrow/pull/14357)). -### 10: Delete old RCs and Releases +### 10: Delete Old RCs and Releases See the ASF documentation on [when to archive](https://www.apache.org/legal/release-policy.html#when-to-archive) for more information. -#### Deleting old release candidates from `dev` svn - Release candidates should be deleted once the release is published. -Get a list of DataFusion release candidates: +To get a list of DataFusion release candidates: ```shell svn ls https://dist.apache.org/repos/dist/dev/datafusion ``` -Delete a release candidate: +To delete a release candidate: ```shell svn delete -m "delete old DataFusion RC" https://dist.apache.org/repos/dist/dev/datafusion/apache-datafusion-50.0.0-rc1/ ``` -#### Deleting old releases from `release` svn +#### Delete old releases from `release` SVN -Only the latest release should be available. Delete old releases after publishing the new release. +Only the latest release should be available. Delete old releases after +publishing the new release. -Get a list of DataFusion releases: +To get a list of DataFusion releases: ```shell svn ls https://dist.apache.org/repos/dist/release/datafusion ``` -Delete a release: +To delete a release: ```shell svn delete -m "delete old DataFusion release" https://dist.apache.org/repos/dist/release/datafusion/datafusion-50.0.0 ``` + +### 11. Forward-port changelog to `main` + +After the release is published, forward port the version update and changelog to +`main` so that it is up to date for the next release. See [#21053] for an +example PR that forward-ports the changelog to `main`. + +[#21053]: https://github.com/apache/datafusion/pull/21053 diff --git a/docs/source/contributor-guide/index.md b/docs/source/contributor-guide/index.md index 4ace4be49499b..6ec1efa4d99fa 100644 --- a/docs/source/contributor-guide/index.md +++ b/docs/source/contributor-guide/index.md @@ -101,6 +101,11 @@ If you are concerned that a larger design will be lost in a string of small PRs, Note all commits in a PR are squashed when merged to the `main` branch so there is one commit per PR after merge. +## Release Management and Backports + +Contributor-facing guidance for release branches, patch releases, and backports +is documented in the [Release Management](release_management.md) guide. + ## Before Submitting a PR Before submitting a PR, run the standard non-functional checks. PRs must pass diff --git a/docs/source/contributor-guide/release_management.md b/docs/source/contributor-guide/release_management.md new file mode 100644 index 0000000000000..0515204a5ecbc --- /dev/null +++ b/docs/source/contributor-guide/release_management.md @@ -0,0 +1,123 @@ + + +# Release Management + +This page describes DataFusion release branches and backports. For the +maintainer release guide, including release candidate artifacts, voting, and +publication, see [the release process README in `dev/release`]. + +## Overview + +DataFusion typically has a major release about once per month, including +breaking API changes. Patch releases are made on an ad hoc basis, but we try to +avoid them because major releases are frequent. + +New development happens on the [`main` branch]. Releases are made from release +branches named `branch-NN`, such as [`branch-50`] for the `50.x.y` release +series. + +In general: + +- New features land on [`main`] +- Patch releases are cut from the corresponding `branch-NN` +- Only targeted, low-risk fixes should be added to a release branch + +Changes reach a release branch in one of two ways: + +- (Most common) Fix the issue on `main` and then backport the merged change to the release branch +- Fix the issue on the release branch and then forward-port the change to `main` + +Releases are coordinated in a GitHub issue, such as the +[release issue for 50.3.0]. If you think a fix should be included in a patch +release, discuss it on the relevant tracking issue first. You can also open the +backport PR first and then link it from the tracking issue. + +To prepare for a new release series, maintainers: + +- Create a new branch from `main`, such as `branch-50`, in the Apache repository +- Continue merging new features to `main` +- Prepare the release branch for release by updating versions, changelog content, + and any additional release-specific fixes via the + [Backport Workflow](#backport-workflow) +- Create release candidate artifacts from the release branch +- After approval, publish to crates.io, ASF distribution servers, and Git tags + +## Backport Workflow + +The usual workflow is: + +1. Fix on `main` first, and merge the fix via a normal PR workflow. +2. Cherry-pick the merged commit onto the release branch. +3. Open a backport PR targeting the release branch (examples below). + +- [Example backport PR] +- [Additional backport PR example] + +### Inputs + +To backport a change, gather the following information: + +- Target branch, such as `apache/branch-52` +- The release tracking issue URL, such as https://github.com/apache/datafusion/issues/19692 +- The original PR URL, such as https://github.com/apache/datafusion/pull/20192 +- Optional explicit commit SHA to backport + +### Apply the Backport + +Start from the target release branch, create a dedicated backport branch, and +use `git cherry-pick`. For example, to backport PR #1234 to `branch-52` when +the commit SHA is `abc123`, run: + +```bash +git checkout apache/branch-52 +git checkout -b alamb/backport_1234 +git cherry-pick abc123 +``` + +### Test + +Run tests as described in the [testing documentation]. + +### Open the PR + +Create a PR against the release branch, not `main`, and prefix it with +`[branch-NN]` to show which release branch the backport targets. For example: + +- `[branch-52] fix: validate inter-file ordering in eq_properties() (#20329)` + +Use a PR description that links the tracking issue, original PR, and target +branch, for example: + +```markdown +- Part of +- Closes on + +This PR: + +- Backports from @ to the line +``` + +[`main` branch]: https://github.com/apache/datafusion/tree/main +[`branch-50`]: https://github.com/apache/datafusion/tree/branch-50 +[the release process readme in `dev/release`]: https://github.com/apache/datafusion/blob/main/dev/release/README.md +[release issue for 50.3.0]: https://github.com/apache/datafusion/issues/18072 +[example backport pr]: https://github.com/apache/datafusion/pull/18131 +[additional backport pr example]: https://github.com/apache/datafusion/pull/20792 +[testing documentation]: testing.md diff --git a/docs/source/index.rst b/docs/source/index.rst index 4d57faa0cbf73..cc0da3c44473e 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -163,6 +163,7 @@ To get started, see contributor-guide/testing contributor-guide/api-health contributor-guide/howtos + contributor-guide/release_management contributor-guide/roadmap contributor-guide/governance contributor-guide/inviting From 5c653bee5da64003915f6dfeb3da15759b091a8d Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Wed, 15 Apr 2026 11:54:23 -0500 Subject: [PATCH 6/6] Port filter_pushdown.rs async tests to sqllogictest (#21620) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Which issue does this PR close? - Closes #. ## Rationale for this change [#21160](https://github.com/apache/datafusion/pull/21160) added `datafusion.explain.analyze_categories`, which lets `EXPLAIN ANALYZE` emit only deterministic metric categories (e.g. `'rows'`). That unlocked a long-standing blocker on porting tests out of `datafusion/core/tests/physical_optimizer/filter_pushdown.rs`: previously these tests had to assert on execution state via `insta` snapshots over hand-wired `ExecutionPlan` trees and mock `TestSource` data, which kept them expensive to read, expensive to update, and impossible to test from the user-facing SQL path. With `analyze_categories = 'rows'`, the `predicate=DynamicFilter [ ... ]` text on a parquet scan is stable across runs, so the same invariants can now be expressed as plain `EXPLAIN ANALYZE` SQL in sqllogictest, where they are easier to read, easier to update, and exercise the full SQL → logical optimizer → physical optimizer → execution pipeline rather than a single optimizer rule in isolation. ## What changes are included in this PR? 24 end-to-end filter-pushdown tests are ported out of `filter_pushdown.rs` and deleted. The helpers `run_aggregate_dyn_filter_case` and `run_projection_dyn_filter_case` (and their supporting structs) are deleted along with the tests that used them. The 24 synchronous `#[test]` optimizer-rule-in-isolation tests are untouched — they stay in Rust because they specifically exercise `FilterPushdown::new()` / `OptimizationTest` over a hand-built plan. ### `datafusion/sqllogictest/test_files/push_down_filter_parquet.slt` New tests covering: - TopK dynamic filter pushdown integration (100k-row parquet, `max_row_group_size = 128`, asserting on `pushdown_rows_matched = 128` / `pushdown_rows_pruned = 99.87 K`) - TopK single-column and multi-column (compound-sort) dynamic filter shapes - HashJoin CollectLeft dynamic filter with `struct(a, b) IN (SET) ([...])` content - Nested hash joins propagating filters to both inner scans - Parent `WHERE` filter splitting across the two sides of a HashJoin - TopK above HashJoin, with both dynamic filters ANDed on the probe scan - Dynamic filter flowing through a `GROUP BY` sitting between a HashJoin and the probe scan - TopK projection rewrite — reorder, prune, expression, alias shadowing - NULL-bearing build-side join keys - `LEFT JOIN` and `LEFT SEMI JOIN` dynamic filter pushdown - HashTable strategy (`hash_lookup`) via `hash_join_inlist_pushdown_max_size = 1`, on both string and integer multi-column keys ### `datafusion/sqllogictest/test_files/push_down_filter_regression.slt` New tests covering: - Aggregate dynamic filter baseline: `MIN(a)`, `MAX(a)`, `MIN(a), MAX(a)`, `MIN(a), MAX(b)`, mixed `MIN/MAX` with an unsupported expression input, all-NULL input (filter stays `true`), `MIN(a+1)` (no filter emitted) - `WHERE` filter on a grouping column pushes through `AggregateExec` - `HAVING count(b) > 5` filter stays above the aggregate - End-to-end aggregate dynamic filter actually pruning a multi-file parquet scan The aggregate baseline tests run under `analyze_level = summary` + `analyze_categories = 'none'` so that metrics render empty and only the `predicate=DynamicFilter [ ... ]` content remains — the filter text is deterministic even though the pruning counts are subject to parallel-execution scheduling. ### What stayed in Rust Ten async tests now carry a short `// Not portable to sqllogictest: …` header explaining why. In short, they either: - Hand-wire `PartitionMode::Partitioned` or a `RepartitionExec` boundary that SQL never constructs for the sizes of data these tests use - Assert via debug-only APIs (`HashJoinExec::dynamic_filter_for_test().is_used()`, `ExecutionPlan::apply_expressions()` + `downcast_ref::`) that are not observable from SQL - Target the specific stacked-`FilterExec` shape (#20109 regression) that the logical optimizer collapses before physical planning ## Are these changes tested? Yes — the ported tests _are_ the tests. Each ported slt case was generated with `cargo test -p datafusion-sqllogictest --test sqllogictests -- --complete`, then re-run twice back-to-back without `--complete` to confirm determinism. The remaining Rust `filter_pushdown` tests continue to pass (`cargo test -p datafusion --test core_integration filter_pushdown` → 47 passed, 0 failed). `cargo clippy --tests -D warnings` and `cargo fmt --all` are clean. ## Test plan - [x] `cargo test -p datafusion-sqllogictest --test sqllogictests -- push_down_filter` - [x] `cargo test -p datafusion --test core_integration filter_pushdown` - [x] `cargo clippy -p datafusion --tests -- -D warnings` - [x] `cargo fmt --all` ## Are there any user-facing changes? No. This is a test-only refactor. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-authored-by: Claude Opus 4.6 (1M context) --- .../physical_optimizer/filter_pushdown.rs | 2356 +---------------- .../test_files/push_down_filter_parquet.slt | 857 ++++++ .../push_down_filter_regression.slt | 311 ++- 3 files changed, 1295 insertions(+), 2229 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index a255c07545829..d058e44a85d00 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -18,7 +18,7 @@ use std::sync::{Arc, LazyLock}; use arrow::{ - array::{Float64Array, Int32Array, RecordBatch, StringArray, record_batch}, + array::record_batch, datatypes::{DataType, Field, Schema, SchemaRef}, util::pretty::pretty_format_batches, }; @@ -30,11 +30,10 @@ use datafusion::{ PhysicalExpr, expressions::{BinaryExpr, Column, Literal}, }, - prelude::{ParquetReadOptions, SessionConfig, SessionContext}, + prelude::{SessionConfig, SessionContext}, scalar::ScalarValue, }; use datafusion_catalog::memory::DataSourceExec; -use datafusion_common::JoinType; use datafusion_common::config::ConfigOptions; use datafusion_datasource::{ PartitionedFile, file_groups::FileGroup, file_scan_config::FileScanConfigBuilder, @@ -42,19 +41,14 @@ use datafusion_datasource::{ use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_expr::ScalarUDF; use datafusion_functions::math::random::RandomFunc; -use datafusion_functions_aggregate::{ - count::count_udaf, - min_max::{max_udaf, min_udaf}, -}; +use datafusion_functions_aggregate::{count::count_udaf, min_max::min_udaf}; use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr, expressions::col}; use datafusion_physical_expr::{ - Partitioning, ScalarFunctionExpr, - aggregate::{AggregateExprBuilder, AggregateFunctionExpr}, + Partitioning, ScalarFunctionExpr, aggregate::AggregateExprBuilder, }; use datafusion_physical_optimizer::{ PhysicalOptimizerRule, filter_pushdown::FilterPushdown, }; -use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; use datafusion_physical_plan::{ ExecutionPlan, aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, @@ -70,9 +64,7 @@ use super::pushdown_utils::{ OptimizationTest, TestNode, TestScanBuilder, TestSource, format_plan_for_test, }; use datafusion_physical_plan::union::UnionExec; -use futures::StreamExt; -use object_store::{ObjectStore, memory::InMemory}; -use regex::Regex; +use object_store::memory::InMemory; #[test] fn test_pushdown_into_scan() { @@ -176,128 +168,11 @@ fn test_pushdown_into_scan_with_config_options() { ); } -#[tokio::test] -async fn test_dynamic_filter_pushdown_through_hash_join_with_topk() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - - // Create build side with limited values - let build_batches = vec![ - record_batch!( - ("a", Utf8, ["aa", "ab"]), - ("b", Utf8View, ["ba", "bb"]), - ("c", Float64, [1.0, 2.0]) - ) - .unwrap(), - ]; - let build_side_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, false), - Field::new("b", DataType::Utf8View, false), - Field::new("c", DataType::Float64, false), - ])); - let build_scan = TestScanBuilder::new(Arc::clone(&build_side_schema)) - .with_support(true) - .with_batches(build_batches) - .build(); - - // Create probe side with more values - let probe_batches = vec![ - record_batch!( - ("d", Utf8, ["aa", "ab", "ac", "ad"]), - ("e", Utf8View, ["ba", "bb", "bc", "bd"]), - ("f", Float64, [1.0, 2.0, 3.0, 4.0]) - ) - .unwrap(), - ]; - let probe_side_schema = Arc::new(Schema::new(vec![ - Field::new("d", DataType::Utf8, false), - Field::new("e", DataType::Utf8View, false), - Field::new("f", DataType::Float64, false), - ])); - let probe_scan = TestScanBuilder::new(Arc::clone(&probe_side_schema)) - .with_support(true) - .with_batches(probe_batches) - .build(); - - // Create HashJoinExec - let on = vec![( - col("a", &build_side_schema).unwrap(), - col("d", &probe_side_schema).unwrap(), - )]; - let join = Arc::new( - HashJoinExec::try_new( - build_scan, - probe_scan, - on, - None, - &JoinType::Inner, - None, - PartitionMode::Partitioned, - datafusion_common::NullEquality::NullEqualsNothing, - false, - ) - .unwrap(), - ); - - let join_schema = join.schema(); - - // Finally let's add a SortExec on the outside to test pushdown of dynamic filters - let sort_expr = - PhysicalSortExpr::new(col("e", &join_schema).unwrap(), SortOptions::default()); - let plan = Arc::new( - SortExec::new(LexOrdering::new(vec![sort_expr]).unwrap(), join) - .with_fetch(Some(2)), - ) as Arc; - - let mut config = ConfigOptions::default(); - config.optimizer.enable_dynamic_filter_pushdown = true; - config.execution.parquet.pushdown_filters = true; - - // Apply the FilterPushdown optimizer rule - let plan = FilterPushdown::new_post_optimization() - .optimize(Arc::clone(&plan), &config) - .unwrap(); - - // Test that filters are pushed down correctly to each side of the join - insta::assert_snapshot!( - format_plan_for_test(&plan), - @r" - - SortExec: TopK(fetch=2), expr=[e@4 ASC], preserve_partitioning=[false] - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - " - ); - - // Put some data through the plan to check that the filter is updated to reflect the TopK state - let session_ctx = SessionContext::new_with_config(SessionConfig::new()); - session_ctx.register_object_store( - ObjectStoreUrl::parse("test://").unwrap().as_ref(), - Arc::new(InMemory::new()), - ); - let state = session_ctx.state(); - let task_ctx = state.task_ctx(); - let mut stream = plan.execute(0, Arc::clone(&task_ctx)).unwrap(); - // Iterate one batch - stream.next().await.unwrap().unwrap(); - - // Test that filters are pushed down correctly to each side of the join - // NOTE: We dropped the CASE expression here because we now optimize that away if there's only 1 partition - insta::assert_snapshot!( - format_plan_for_test(&plan), - @r" - - SortExec: TopK(fetch=2), expr=[e@4 ASC], preserve_partitioning=[false], filter=[e@4 IS NULL OR e@4 < bb] - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ d@0 >= aa AND d@0 <= ab AND d@0 IN (SET) ([aa, ab]) ] AND DynamicFilter [ e@1 IS NULL OR e@1 < bb ] - " - ); -} - -// Test both static and dynamic filter pushdown in HashJoinExec. -// Note that static filter pushdown is rare: it should have already happened in the logical optimizer phase. -// However users may manually construct plans that could result in a FilterExec -> HashJoinExec -> Scan setup. -// Dynamic filters arise in cases such as nested inner joins or TopK -> HashJoinExec -> Scan setups. +// Inner-join part is covered by push_down_filter_parquet.slt::test_hashjoin_parent_filter_pushdown. +// The Left-join part stays in Rust: SQL's outer-join-elimination rewrites +// `LEFT JOIN ... WHERE ` into an INNER JOIN +// before physical filter pushdown runs, so the preserved-vs-non-preserved +// distinction this test exercises is not reachable via SQL. #[tokio::test] async fn test_static_filter_pushdown_through_hash_join() { use datafusion_common::JoinType; @@ -926,175 +801,11 @@ fn test_node_handles_child_pushdown_result() { ); } -#[tokio::test] -async fn test_topk_dynamic_filter_pushdown() { - let batches = vec![ - record_batch!( - ("a", Utf8, ["aa", "ab"]), - ("b", Utf8, ["bd", "bc"]), - ("c", Float64, [1.0, 2.0]) - ) - .unwrap(), - record_batch!( - ("a", Utf8, ["ac", "ad"]), - ("b", Utf8, ["bb", "ba"]), - ("c", Float64, [2.0, 1.0]) - ) - .unwrap(), - ]; - let scan = TestScanBuilder::new(schema()) - .with_support(true) - .with_batches(batches) - .build(); - let plan = Arc::new( - SortExec::new( - LexOrdering::new(vec![PhysicalSortExpr::new( - col("b", &schema()).unwrap(), - SortOptions::new(true, false), // descending, nulls_first - )]) - .unwrap(), - Arc::clone(&scan), - ) - .with_fetch(Some(1)), - ) as Arc; - - // expect the predicate to be pushed down into the DataSource - insta::assert_snapshot!( - OptimizationTest::new(Arc::clone(&plan), FilterPushdown::new_post_optimization(), true), - @r" - OptimizationTest: - input: - - SortExec: TopK(fetch=1), expr=[b@1 DESC NULLS LAST], preserve_partitioning=[false] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - output: - Ok: - - SortExec: TopK(fetch=1), expr=[b@1 DESC NULLS LAST], preserve_partitioning=[false] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ] - " - ); - - // Actually apply the optimization to the plan and put some data through it to check that the filter is updated to reflect the TopK state - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); - let config = SessionConfig::new().with_batch_size(2); - let session_ctx = SessionContext::new_with_config(config); - session_ctx.register_object_store( - ObjectStoreUrl::parse("test://").unwrap().as_ref(), - Arc::new(InMemory::new()), - ); - let state = session_ctx.state(); - let task_ctx = state.task_ctx(); - let mut stream = plan.execute(0, Arc::clone(&task_ctx)).unwrap(); - // Iterate one batch - stream.next().await.unwrap().unwrap(); - // Now check what our filter looks like - insta::assert_snapshot!( - format!("{}", format_plan_for_test(&plan)), - @r" - - SortExec: TopK(fetch=1), expr=[b@1 DESC NULLS LAST], preserve_partitioning=[false], filter=[b@1 > bd] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ b@1 > bd ] - " - ); -} - -#[tokio::test] -async fn test_topk_dynamic_filter_pushdown_multi_column_sort() { - let batches = vec![ - // We are going to do ORDER BY b ASC NULLS LAST, a DESC - // And we put the values in such a way that the first batch will fill the TopK - // and we skip the second batch. - record_batch!( - ("a", Utf8, ["ac", "ad"]), - ("b", Utf8, ["bb", "ba"]), - ("c", Float64, [2.0, 1.0]) - ) - .unwrap(), - record_batch!( - ("a", Utf8, ["aa", "ab"]), - ("b", Utf8, ["bc", "bd"]), - ("c", Float64, [1.0, 2.0]) - ) - .unwrap(), - ]; - let scan = TestScanBuilder::new(schema()) - .with_support(true) - .with_batches(batches) - .build(); - let plan = Arc::new( - SortExec::new( - LexOrdering::new(vec![ - PhysicalSortExpr::new( - col("b", &schema()).unwrap(), - SortOptions::default().asc().nulls_last(), - ), - PhysicalSortExpr::new( - col("a", &schema()).unwrap(), - SortOptions::default().desc().nulls_first(), - ), - ]) - .unwrap(), - Arc::clone(&scan), - ) - .with_fetch(Some(2)), - ) as Arc; - - // expect the predicate to be pushed down into the DataSource - insta::assert_snapshot!( - OptimizationTest::new(Arc::clone(&plan), FilterPushdown::new_post_optimization(), true), - @r" - OptimizationTest: - input: - - SortExec: TopK(fetch=2), expr=[b@1 ASC NULLS LAST, a@0 DESC], preserve_partitioning=[false] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - output: - Ok: - - SortExec: TopK(fetch=2), expr=[b@1 ASC NULLS LAST, a@0 DESC], preserve_partitioning=[false] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ] - " - ); - - // Actually apply the optimization to the plan and put some data through it to check that the filter is updated to reflect the TopK state - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); - let config = SessionConfig::new().with_batch_size(2); - let session_ctx = SessionContext::new_with_config(config); - session_ctx.register_object_store( - ObjectStoreUrl::parse("test://").unwrap().as_ref(), - Arc::new(InMemory::new()), - ); - let state = session_ctx.state(); - let task_ctx = state.task_ctx(); - let mut stream = plan.execute(0, Arc::clone(&task_ctx)).unwrap(); - // Iterate one batch - let res = stream.next().await.unwrap().unwrap(); - #[rustfmt::skip] - let expected = [ - "+----+----+-----+", - "| a | b | c |", - "+----+----+-----+", - "| ad | ba | 1.0 |", - "| ac | bb | 2.0 |", - "+----+----+-----+", - ]; - assert_batches_eq!(expected, &[res]); - // Now check what our filter looks like - insta::assert_snapshot!( - format!("{}", format_plan_for_test(&plan)), - @r" - - SortExec: TopK(fetch=2), expr=[b@1 ASC NULLS LAST, a@0 DESC], preserve_partitioning=[false], filter=[b@1 < bb OR b@1 = bb AND (a@0 IS NULL OR a@0 > ac)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ b@1 < bb OR b@1 = bb AND (a@0 IS NULL OR a@0 > ac) ] - " - ); - // There should be no more batches - assert!(stream.next().await.is_none()); -} - +// Not portable to sqllogictest: requires manually constructing +// `SortExec(CoalescePartitionsExec(scan))`. A SQL `ORDER BY ... LIMIT` over a +// multi-partition scan plans as `SortPreservingMergeExec(SortExec(scan))` +// instead, so the filter-through-coalesce path this test exercises is not +// reachable via SQL. #[tokio::test] async fn test_topk_filter_passes_through_coalesce_partitions() { // Create multiple batches for different partitions @@ -1163,129 +874,12 @@ async fn test_topk_filter_passes_through_coalesce_partitions() { ); } +// Not portable to sqllogictest: this test pins `PartitionMode::Partitioned` +// by hand-wiring `RepartitionExec(Hash, 12)` on both join sides. A SQL +// INNER JOIN over small parquet inputs plans as `CollectLeft`, so the +// per-partition CASE filter this test exercises is not reachable via SQL. #[tokio::test] -async fn test_hashjoin_dynamic_filter_pushdown() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - - // Create build side with limited values - let build_batches = vec![ - record_batch!( - ("a", Utf8, ["aa", "ab"]), - ("b", Utf8, ["ba", "bb"]), - ("c", Float64, [1.0, 2.0]) // Extra column not used in join - ) - .unwrap(), - ]; - let build_side_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, false), - Field::new("b", DataType::Utf8, false), - Field::new("c", DataType::Float64, false), - ])); - let build_scan = TestScanBuilder::new(Arc::clone(&build_side_schema)) - .with_support(true) - .with_batches(build_batches) - .build(); - - // Create probe side with more values - let probe_batches = vec![ - record_batch!( - ("a", Utf8, ["aa", "ab", "ac", "ad"]), - ("b", Utf8, ["ba", "bb", "bc", "bd"]), - ("e", Float64, [1.0, 2.0, 3.0, 4.0]) // Extra column not used in join - ) - .unwrap(), - ]; - let probe_side_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, false), - Field::new("b", DataType::Utf8, false), - Field::new("e", DataType::Float64, false), - ])); - let probe_scan = TestScanBuilder::new(Arc::clone(&probe_side_schema)) - .with_support(true) - .with_batches(probe_batches) - .build(); - - // Create HashJoinExec with dynamic filter - let on = vec![ - ( - col("a", &build_side_schema).unwrap(), - col("a", &probe_side_schema).unwrap(), - ), - ( - col("b", &build_side_schema).unwrap(), - col("b", &probe_side_schema).unwrap(), - ), - ]; - let plan = Arc::new( - HashJoinExec::try_new( - build_scan, - probe_scan, - on, - None, - &JoinType::Inner, - None, - PartitionMode::CollectLeft, - datafusion_common::NullEquality::NullEqualsNothing, - false, - ) - .unwrap(), - ) as Arc; - - // expect the predicate to be pushed down into the probe side DataSource - insta::assert_snapshot!( - OptimizationTest::new(Arc::clone(&plan), FilterPushdown::new_post_optimization(), true), - @r" - OptimizationTest: - input: - - HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true - output: - Ok: - - HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ] - ", - ); - - // Actually apply the optimization to the plan and execute to see the filter in action - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - config.optimizer.enable_dynamic_filter_pushdown = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); - - // Test for https://github.com/apache/datafusion/pull/17371: dynamic filter linking survives `with_new_children` - let children = plan.children().into_iter().map(Arc::clone).collect(); - let plan = plan.with_new_children(children).unwrap(); - - let config = SessionConfig::new().with_batch_size(10); - let session_ctx = SessionContext::new_with_config(config); - session_ctx.register_object_store( - ObjectStoreUrl::parse("test://").unwrap().as_ref(), - Arc::new(InMemory::new()), - ); - let state = session_ctx.state(); - let task_ctx = state.task_ctx(); - let mut stream = plan.execute(0, Arc::clone(&task_ctx)).unwrap(); - // Iterate one batch - stream.next().await.unwrap().unwrap(); - - // Now check what our filter looks like - insta::assert_snapshot!( - format!("{}", format_plan_for_test(&plan)), - @r" - - HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:ba}, {c0:ab,c1:bb}]) ] - " - ); -} - -#[tokio::test] -async fn test_hashjoin_dynamic_filter_pushdown_partitioned() { +async fn test_hashjoin_dynamic_filter_pushdown_partitioned() { use datafusion_common::JoinType; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; @@ -1533,6 +1127,12 @@ async fn test_hashjoin_dynamic_filter_pushdown_partitioned() { ); } +// Not portable to sqllogictest: this test specifically pins a +// `RepartitionExec(Hash, 12)` between `HashJoinExec(CollectLeft)` and the +// probe-side scan to verify the dynamic filter link survives that boundary +// (regression for #17451). The same CollectLeft filter content and +// pushdown counters are already covered by the simpler slt port +// (push_down_filter_parquet.slt::test_hashjoin_dynamic_filter_pushdown). #[tokio::test] async fn test_hashjoin_dynamic_filter_pushdown_collect_left() { use datafusion_common::JoinType; @@ -1705,265 +1305,6 @@ async fn test_hashjoin_dynamic_filter_pushdown_collect_left() { ); } -#[tokio::test] -async fn test_nested_hashjoin_dynamic_filter_pushdown() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - - // Create test data for three tables: t1, t2, t3 - // t1: small table with limited values (will be build side of outer join) - let t1_batches = vec![ - record_batch!(("a", Utf8, ["aa", "ab"]), ("x", Float64, [1.0, 2.0])).unwrap(), - ]; - let t1_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, false), - Field::new("x", DataType::Float64, false), - ])); - let t1_scan = TestScanBuilder::new(Arc::clone(&t1_schema)) - .with_support(true) - .with_batches(t1_batches) - .build(); - - // t2: larger table (will be probe side of inner join, build side of outer join) - let t2_batches = vec![ - record_batch!( - ("b", Utf8, ["aa", "ab", "ac", "ad", "ae"]), - ("c", Utf8, ["ca", "cb", "cc", "cd", "ce"]), - ("y", Float64, [1.0, 2.0, 3.0, 4.0, 5.0]) - ) - .unwrap(), - ]; - let t2_schema = Arc::new(Schema::new(vec![ - Field::new("b", DataType::Utf8, false), - Field::new("c", DataType::Utf8, false), - Field::new("y", DataType::Float64, false), - ])); - let t2_scan = TestScanBuilder::new(Arc::clone(&t2_schema)) - .with_support(true) - .with_batches(t2_batches) - .build(); - - // t3: largest table (will be probe side of inner join) - let t3_batches = vec![ - record_batch!( - ("d", Utf8, ["ca", "cb", "cc", "cd", "ce", "cf", "cg", "ch"]), - ("z", Float64, [1.0, 2.0, 3.0, 4.0, 5.0, 6.0, 7.0, 8.0]) - ) - .unwrap(), - ]; - let t3_schema = Arc::new(Schema::new(vec![ - Field::new("d", DataType::Utf8, false), - Field::new("z", DataType::Float64, false), - ])); - let t3_scan = TestScanBuilder::new(Arc::clone(&t3_schema)) - .with_support(true) - .with_batches(t3_batches) - .build(); - - // Create nested join structure: - // Join (t1.a = t2.b) - // / \ - // t1 Join(t2.c = t3.d) - // / \ - // t2 t3 - - // First create inner join: t2.c = t3.d - let inner_join_on = - vec![(col("c", &t2_schema).unwrap(), col("d", &t3_schema).unwrap())]; - let inner_join = Arc::new( - HashJoinExec::try_new( - t2_scan, - t3_scan, - inner_join_on, - None, - &JoinType::Inner, - None, - PartitionMode::Partitioned, - datafusion_common::NullEquality::NullEqualsNothing, - false, - ) - .unwrap(), - ); - - // Then create outer join: t1.a = t2.b (from inner join result) - let outer_join_on = vec![( - col("a", &t1_schema).unwrap(), - col("b", &inner_join.schema()).unwrap(), - )]; - let outer_join = Arc::new( - HashJoinExec::try_new( - t1_scan, - inner_join as Arc, - outer_join_on, - None, - &JoinType::Inner, - None, - PartitionMode::Partitioned, - datafusion_common::NullEquality::NullEqualsNothing, - false, - ) - .unwrap(), - ) as Arc; - - // Test that dynamic filters are pushed down correctly through nested joins - insta::assert_snapshot!( - OptimizationTest::new(Arc::clone(&outer_join), FilterPushdown::new_post_optimization(), true), - @r" - OptimizationTest: - input: - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true - output: - Ok: - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ] - ", - ); - - // Execute the plan to verify the dynamic filters are properly updated - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - config.optimizer.enable_dynamic_filter_pushdown = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(outer_join, &config) - .unwrap(); - let config = SessionConfig::new().with_batch_size(10); - let session_ctx = SessionContext::new_with_config(config); - session_ctx.register_object_store( - ObjectStoreUrl::parse("test://").unwrap().as_ref(), - Arc::new(InMemory::new()), - ); - let state = session_ctx.state(); - let task_ctx = state.task_ctx(); - let mut stream = plan.execute(0, Arc::clone(&task_ctx)).unwrap(); - // Execute to populate the dynamic filters - stream.next().await.unwrap().unwrap(); - - // Verify that both the inner and outer join have updated dynamic filters - insta::assert_snapshot!( - format!("{}", format_plan_for_test(&plan)), - @r" - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ b@0 >= aa AND b@0 <= ab AND b@0 IN (SET) ([aa, ab]) ] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ d@0 >= ca AND d@0 <= cb AND d@0 IN (SET) ([ca, cb]) ] - " - ); -} - -#[tokio::test] -async fn test_hashjoin_parent_filter_pushdown() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - - // Create build side with limited values - let build_batches = vec![ - record_batch!( - ("a", Utf8, ["aa", "ab"]), - ("b", Utf8, ["ba", "bb"]), - ("c", Float64, [1.0, 2.0]) - ) - .unwrap(), - ]; - let build_side_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, false), - Field::new("b", DataType::Utf8, false), - Field::new("c", DataType::Float64, false), - ])); - let build_scan = TestScanBuilder::new(Arc::clone(&build_side_schema)) - .with_support(true) - .with_batches(build_batches) - .build(); - - // Create probe side with more values - let probe_batches = vec![ - record_batch!( - ("d", Utf8, ["aa", "ab", "ac", "ad"]), - ("e", Utf8, ["ba", "bb", "bc", "bd"]), - ("f", Float64, [1.0, 2.0, 3.0, 4.0]) - ) - .unwrap(), - ]; - let probe_side_schema = Arc::new(Schema::new(vec![ - Field::new("d", DataType::Utf8, false), - Field::new("e", DataType::Utf8, false), - Field::new("f", DataType::Float64, false), - ])); - let probe_scan = TestScanBuilder::new(Arc::clone(&probe_side_schema)) - .with_support(true) - .with_batches(probe_batches) - .build(); - - // Create HashJoinExec - let on = vec![( - col("a", &build_side_schema).unwrap(), - col("d", &probe_side_schema).unwrap(), - )]; - let join = Arc::new( - HashJoinExec::try_new( - build_scan, - probe_scan, - on, - None, - &JoinType::Inner, - None, - PartitionMode::Partitioned, - datafusion_common::NullEquality::NullEqualsNothing, - false, - ) - .unwrap(), - ); - - // Create filters that can be pushed down to different sides - // We need to create filters in the context of the join output schema - let join_schema = join.schema(); - - // Filter on build side column: a = 'aa' - let left_filter = col_lit_predicate("a", "aa", &join_schema); - // Filter on probe side column: e = 'ba' - let right_filter = col_lit_predicate("e", "ba", &join_schema); - // Filter that references both sides: a = d (should not be pushed down) - let cross_filter = Arc::new(BinaryExpr::new( - col("a", &join_schema).unwrap(), - Operator::Eq, - col("d", &join_schema).unwrap(), - )) as Arc; - - let filter = - Arc::new(FilterExec::try_new(left_filter, Arc::clone(&join) as _).unwrap()); - let filter = Arc::new(FilterExec::try_new(right_filter, filter).unwrap()); - let plan = Arc::new(FilterExec::try_new(cross_filter, filter).unwrap()) - as Arc; - - // Test that filters are pushed down correctly to each side of the join - insta::assert_snapshot!( - OptimizationTest::new(Arc::clone(&plan), FilterPushdown::new(), true), - @r" - OptimizationTest: - input: - - FilterExec: a@0 = d@3 - - FilterExec: e@4 = ba - - FilterExec: a@0 = aa - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true - output: - Ok: - - FilterExec: a@0 = d@3 - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = aa - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate=e@1 = ba - " - ); -} - #[test] fn test_hashjoin_parent_filter_pushdown_same_column_names() { use datafusion_common::JoinType; @@ -2176,64 +1517,6 @@ fn test_hashjoin_parent_filter_pushdown_semi_anti_join() { ); } -/// Integration test for dynamic filter pushdown with TopK. -/// We use an integration test because there are complex interactions in the optimizer rules -/// that the unit tests applying a single optimizer rule do not cover. -#[tokio::test] -async fn test_topk_dynamic_filter_pushdown_integration() { - let store = Arc::new(InMemory::new()) as Arc; - let mut cfg = SessionConfig::new(); - cfg.options_mut().execution.parquet.pushdown_filters = true; - cfg.options_mut().execution.parquet.max_row_group_size = 128; - let ctx = SessionContext::new_with_config(cfg); - ctx.register_object_store( - ObjectStoreUrl::parse("memory://").unwrap().as_ref(), - Arc::clone(&store), - ); - ctx.sql( - r" -COPY ( - SELECT 1372708800 + value AS t - FROM generate_series(0, 99999) - ORDER BY t - ) TO 'memory:///1.parquet' -STORED AS PARQUET; - ", - ) - .await - .unwrap() - .collect() - .await - .unwrap(); - - // Register the file with the context - ctx.register_parquet( - "topk_pushdown", - "memory:///1.parquet", - ParquetReadOptions::default(), - ) - .await - .unwrap(); - - // Create a TopK query that will use dynamic filter pushdown - // Note that we use t * t as the order by expression to avoid - // the order pushdown optimizer from optimizing away the TopK. - let df = ctx - .sql(r"EXPLAIN ANALYZE SELECT t FROM topk_pushdown ORDER BY t * t LIMIT 10;") - .await - .unwrap(); - let batches = df.collect().await.unwrap(); - let explain = format!("{}", pretty_format_batches(&batches).unwrap()); - - assert!(explain.contains("output_rows=128")); // Read 1 row group - assert!(explain.contains("t@0 < 1884329474306198481")); // Dynamic filter was applied - assert!( - explain.contains("pushdown_rows_matched=128, pushdown_rows_pruned=99.87 K"), - "{explain}" - ); - // Pushdown pruned most rows -} - #[test] fn test_filter_pushdown_through_union() { let scan1 = TestScanBuilder::new(schema()).with_support(true).build(); @@ -2450,233 +1733,11 @@ fn schema() -> SchemaRef { Arc::clone(&TEST_SCHEMA) } -struct ProjectionDynFilterTestCase { - schema: SchemaRef, - batches: Vec, - projection: Vec<(Arc, String)>, - sort_expr: PhysicalSortExpr, - expected_plans: Vec, -} - -async fn run_projection_dyn_filter_case(case: ProjectionDynFilterTestCase) { - let ProjectionDynFilterTestCase { - schema, - batches, - projection, - sort_expr, - expected_plans, - } = case; - - let scan = TestScanBuilder::new(Arc::clone(&schema)) - .with_support(true) - .with_batches(batches) - .build(); - - let projection_exec = Arc::new(ProjectionExec::try_new(projection, scan).unwrap()); - - let sort = Arc::new( - SortExec::new(LexOrdering::new(vec![sort_expr]).unwrap(), projection_exec) - .with_fetch(Some(2)), - ) as Arc; - - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - config.optimizer.enable_dynamic_filter_pushdown = true; - - let optimized_plan = FilterPushdown::new_post_optimization() - .optimize(Arc::clone(&sort), &config) - .unwrap(); - - pretty_assertions::assert_eq!( - format_plan_for_test(&optimized_plan).trim(), - expected_plans[0].trim() - ); - - let config = SessionConfig::new().with_batch_size(2); - let session_ctx = SessionContext::new_with_config(config); - session_ctx.register_object_store( - ObjectStoreUrl::parse("test://").unwrap().as_ref(), - Arc::new(InMemory::new()), - ); - let state = session_ctx.state(); - let task_ctx = state.task_ctx(); - let mut stream = optimized_plan.execute(0, Arc::clone(&task_ctx)).unwrap(); - for (idx, expected_plan) in expected_plans.iter().enumerate().skip(1) { - stream.next().await.unwrap().unwrap(); - let formatted_plan = format_plan_for_test(&optimized_plan); - pretty_assertions::assert_eq!( - formatted_plan.trim(), - expected_plan.trim(), - "Mismatch at iteration {}", - idx - ); - } -} - -#[tokio::test] -async fn test_topk_with_projection_transformation_on_dyn_filter() { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, false), - Field::new("b", DataType::Utf8, false), - Field::new("c", DataType::Float64, false), - ])); - let simple_abc = vec![ - record_batch!( - ("a", Int32, [1, 2, 3]), - ("b", Utf8, ["x", "y", "z"]), - ("c", Float64, [1.0, 2.0, 3.0]) - ) - .unwrap(), - ]; - - // Case 1: Reordering [b, a] - run_projection_dyn_filter_case(ProjectionDynFilterTestCase { - schema: Arc::clone(&schema), - batches: simple_abc.clone(), - projection: vec![ - (col("b", &schema).unwrap(), "b".to_string()), - (col("a", &schema).unwrap(), "a".to_string()), - ], - sort_expr: PhysicalSortExpr::new( - Arc::new(Column::new("a", 1)), - SortOptions::default(), - ), - expected_plans: vec![ -r#" - SortExec: TopK(fetch=2), expr=[a@1 ASC], preserve_partitioning=[false] - - ProjectionExec: expr=[b@1 as b, a@0 as a] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ]"#.to_string(), -r#" - SortExec: TopK(fetch=2), expr=[a@1 ASC], preserve_partitioning=[false], filter=[a@1 IS NULL OR a@1 < 2] - - ProjectionExec: expr=[b@1 as b, a@0 as a] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 IS NULL OR a@0 < 2 ]"#.to_string()] - }) - .await; - - // Case 2: Pruning [a] - run_projection_dyn_filter_case(ProjectionDynFilterTestCase { - schema: Arc::clone(&schema), - batches: simple_abc.clone(), - projection: vec![(col("a", &schema).unwrap(), "a".to_string())], - sort_expr: PhysicalSortExpr::new( - Arc::new(Column::new("a", 0)), - SortOptions::default(), - ), - expected_plans: vec![ - r#" - SortExec: TopK(fetch=2), expr=[a@0 ASC], preserve_partitioning=[false] - - ProjectionExec: expr=[a@0 as a] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ]"#.to_string(), - r#" - SortExec: TopK(fetch=2), expr=[a@0 ASC], preserve_partitioning=[false], filter=[a@0 IS NULL OR a@0 < 2] - - ProjectionExec: expr=[a@0 as a] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 IS NULL OR a@0 < 2 ]"#.to_string(), - ], - }) - .await; - - // Case 3: Identity [a, b] - run_projection_dyn_filter_case(ProjectionDynFilterTestCase { - schema: Arc::clone(&schema), - batches: simple_abc.clone(), - projection: vec![ - (col("a", &schema).unwrap(), "a".to_string()), - (col("b", &schema).unwrap(), "b".to_string()), - ], - sort_expr: PhysicalSortExpr::new( - Arc::new(Column::new("a", 0)), - SortOptions::default(), - ), - expected_plans: vec![ - r#" - SortExec: TopK(fetch=2), expr=[a@0 ASC], preserve_partitioning=[false] - - ProjectionExec: expr=[a@0 as a, b@1 as b] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ]"#.to_string(), - r#" - SortExec: TopK(fetch=2), expr=[a@0 ASC], preserve_partitioning=[false], filter=[a@0 IS NULL OR a@0 < 2] - - ProjectionExec: expr=[a@0 as a, b@1 as b] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 IS NULL OR a@0 < 2 ]"#.to_string(), - ], - }) - .await; - - // Case 4: Expressions [a + 1, b] - run_projection_dyn_filter_case(ProjectionDynFilterTestCase { - schema: Arc::clone(&schema), - batches: simple_abc.clone(), - projection: vec![ - ( - Arc::new(BinaryExpr::new( - col("a", &schema).unwrap(), - Operator::Plus, - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), - )), - "a_plus_1".to_string(), - ), - (col("b", &schema).unwrap(), "b".to_string()), - ], - sort_expr: PhysicalSortExpr::new( - Arc::new(Column::new("a_plus_1", 0)), - SortOptions::default(), - ), - expected_plans: vec![ - r#" - SortExec: TopK(fetch=2), expr=[a_plus_1@0 ASC], preserve_partitioning=[false] - - ProjectionExec: expr=[a@0 + 1 as a_plus_1, b@1 as b] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ]"#.to_string(), - r#" - SortExec: TopK(fetch=2), expr=[a_plus_1@0 ASC], preserve_partitioning=[false], filter=[a_plus_1@0 IS NULL OR a_plus_1@0 < 3] - - ProjectionExec: expr=[a@0 + 1 as a_plus_1, b@1 as b] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 + 1 IS NULL OR a@0 + 1 < 3 ]"#.to_string(), - ], - }) - .await; - - // Case 5: [a as b, b as a] (swapped columns) - run_projection_dyn_filter_case(ProjectionDynFilterTestCase { - schema: Arc::clone(&schema), - batches: simple_abc.clone(), - projection: vec![ - (col("a", &schema).unwrap(), "b".to_string()), - (col("b", &schema).unwrap(), "a".to_string()), - ], - sort_expr: PhysicalSortExpr::new( - Arc::new(Column::new("b", 0)), - SortOptions::default(), - ), - expected_plans: vec![ - r#" - SortExec: TopK(fetch=2), expr=[b@0 ASC], preserve_partitioning=[false] - - ProjectionExec: expr=[a@0 as b, b@1 as a] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ]"#.to_string(), - r#" - SortExec: TopK(fetch=2), expr=[b@0 ASC], preserve_partitioning=[false], filter=[b@0 IS NULL OR b@0 < 2] - - ProjectionExec: expr=[a@0 as b, b@1 as a] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 IS NULL OR a@0 < 2 ]"#.to_string(), - ], - }) - .await; - - // Case 6: Confusing expr [a + 1 as a, b] - run_projection_dyn_filter_case(ProjectionDynFilterTestCase { - schema: Arc::clone(&schema), - batches: simple_abc.clone(), - projection: vec![ - ( - Arc::new(BinaryExpr::new( - col("a", &schema).unwrap(), - Operator::Plus, - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), - )), - "a".to_string(), - ), - (col("b", &schema).unwrap(), "b".to_string()), - ], - sort_expr: PhysicalSortExpr::new( - Arc::new(Column::new("a", 0)), - SortOptions::default(), - ), - expected_plans: vec![ - r#" - SortExec: TopK(fetch=2), expr=[a@0 ASC], preserve_partitioning=[false] - - ProjectionExec: expr=[a@0 + 1 as a, b@1 as b] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ]"#.to_string(), - r#" - SortExec: TopK(fetch=2), expr=[a@0 ASC], preserve_partitioning=[false], filter=[a@0 IS NULL OR a@0 < 3] - - ProjectionExec: expr=[a@0 + 1 as a, b@1 as b] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 + 1 IS NULL OR a@0 + 1 < 3 ]"#.to_string(), - ], - }) - .await; -} +// test_topk_with_projection_transformation_on_dyn_filter has been ported +// to datafusion/sqllogictest/test_files/push_down_filter_parquet.slt; see +// `topk_proj` fixture for the 4 representative cases (reorder, prune, +// expression, alias shadowing). The `run_projection_dyn_filter_case` +// harness was removed along with it. /// Returns a predicate that is a binary expression col = lit fn col_lit_predicate( @@ -2693,382 +1754,12 @@ fn col_lit_predicate( } // ==== Aggregate Dynamic Filter tests ==== - -// ---- Test Utilities ---- -struct AggregateDynFilterCase<'a> { - schema: SchemaRef, - batches: Vec, - aggr_exprs: Vec, - expected_before: Option<&'a str>, - expected_after: Option<&'a str>, - scan_support: bool, -} - -async fn run_aggregate_dyn_filter_case(case: AggregateDynFilterCase<'_>) { - let AggregateDynFilterCase { - schema, - batches, - aggr_exprs, - expected_before, - expected_after, - scan_support, - } = case; - - let scan = TestScanBuilder::new(Arc::clone(&schema)) - .with_support(scan_support) - .with_batches(batches) - .build(); - - let aggr_exprs: Vec<_> = aggr_exprs - .into_iter() - .map(|expr| Arc::new(expr) as Arc) - .collect(); - let aggr_len = aggr_exprs.len(); - - let plan: Arc = Arc::new( - AggregateExec::try_new( - AggregateMode::Partial, - PhysicalGroupBy::new_single(vec![]), - aggr_exprs, - vec![None; aggr_len], - scan, - Arc::clone(&schema), - ) - .unwrap(), - ); - - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - config.optimizer.enable_dynamic_filter_pushdown = true; - - let optimized = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); - - let before = format_plan_for_test(&optimized); - if let Some(expected) = expected_before { - assert!( - before.contains(expected), - "expected `{expected}` before execution, got: {before}" - ); - } else { - assert!( - !before.contains("DynamicFilter ["), - "dynamic filter unexpectedly present before execution: {before}" - ); - } - - let session_ctx = SessionContext::new(); - session_ctx.register_object_store( - ObjectStoreUrl::parse("test://").unwrap().as_ref(), - Arc::new(InMemory::new()), - ); - let task_ctx = session_ctx.state().task_ctx(); - let mut stream = optimized.execute(0, Arc::clone(&task_ctx)).unwrap(); - let _ = stream.next().await.transpose().unwrap(); - - let after = format_plan_for_test(&optimized); - if let Some(expected) = expected_after { - assert!( - after.contains(expected), - "expected `{expected}` after execution, got: {after}" - ); - } else { - assert!( - !after.contains("DynamicFilter ["), - "dynamic filter unexpectedly present after execution: {after}" - ); - } -} - -// ---- Test Cases ---- -// Cases covered below: -// 1. `min(a)` and `max(a)` baseline. -// 2. Unsupported expression input (`min(a+1)`). -// 3. Multiple supported columns (same column vs different columns). -// 4. Mixed supported + unsupported aggregates. -// 5. Entirely NULL input to surface current bound behavior. -// 6. End-to-end tests on parquet files - -/// `MIN(a)`: able to pushdown dynamic filter -#[tokio::test] -async fn test_aggregate_dynamic_filter_min_simple() { - // Single min(a) showcases the base case. - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, true)])); - let batches = vec![record_batch!(("a", Int32, [5, 1, 3, 8])).unwrap()]; - - let min_expr = - AggregateExprBuilder::new(min_udaf(), vec![col("a", &schema).unwrap()]) - .schema(Arc::clone(&schema)) - .alias("min_a") - .build() - .unwrap(); - - run_aggregate_dyn_filter_case(AggregateDynFilterCase { - schema, - batches, - aggr_exprs: vec![min_expr], - expected_before: Some("DynamicFilter [ empty ]"), - expected_after: Some("DynamicFilter [ a@0 < 1 ]"), - scan_support: true, - }) - .await; -} - -/// `MAX(a)`: able to pushdown dynamic filter -#[tokio::test] -async fn test_aggregate_dynamic_filter_max_simple() { - // Single max(a) mirrors the base case on the upper bound. - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, true)])); - let batches = vec![record_batch!(("a", Int32, [5, 1, 3, 8])).unwrap()]; - - let max_expr = - AggregateExprBuilder::new(max_udaf(), vec![col("a", &schema).unwrap()]) - .schema(Arc::clone(&schema)) - .alias("max_a") - .build() - .unwrap(); - - run_aggregate_dyn_filter_case(AggregateDynFilterCase { - schema, - batches, - aggr_exprs: vec![max_expr], - expected_before: Some("DynamicFilter [ empty ]"), - expected_after: Some("DynamicFilter [ a@0 > 8 ]"), - scan_support: true, - }) - .await; -} - -/// `MIN(a+1)`: Can't pushdown dynamic filter -#[tokio::test] -async fn test_aggregate_dynamic_filter_min_expression_not_supported() { - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, true)])); - let batches = vec![record_batch!(("a", Int32, [5, 1, 3, 8])).unwrap()]; - - let expr: Arc = Arc::new(BinaryExpr::new( - col("a", &schema).unwrap(), - Operator::Plus, - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), - )); - let min_expr = AggregateExprBuilder::new(min_udaf(), vec![expr]) - .schema(Arc::clone(&schema)) - .alias("min_a_plus_one") - .build() - .unwrap(); - - run_aggregate_dyn_filter_case(AggregateDynFilterCase { - schema, - batches, - aggr_exprs: vec![min_expr], - expected_before: None, - expected_after: None, - scan_support: true, - }) - .await; -} - -/// `MIN(a), MAX(a)`: Pushdown dynamic filter like `(a<1) or (a>8)` -#[tokio::test] -async fn test_aggregate_dynamic_filter_min_max_same_column() { - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, true)])); - let batches = vec![record_batch!(("a", Int32, [5, 1, 3, 8])).unwrap()]; - - let min_expr = - AggregateExprBuilder::new(min_udaf(), vec![col("a", &schema).unwrap()]) - .schema(Arc::clone(&schema)) - .alias("min_a") - .build() - .unwrap(); - let max_expr = - AggregateExprBuilder::new(max_udaf(), vec![col("a", &schema).unwrap()]) - .schema(Arc::clone(&schema)) - .alias("max_a") - .build() - .unwrap(); - - run_aggregate_dyn_filter_case(AggregateDynFilterCase { - schema, - batches, - aggr_exprs: vec![min_expr, max_expr], - expected_before: Some("DynamicFilter [ empty ]"), - expected_after: Some("DynamicFilter [ a@0 < 1 OR a@0 > 8 ]"), - scan_support: true, - }) - .await; -} - -/// `MIN(a), MAX(b)`: Pushdown dynamic filter like `(a<1) or (b>9)` -#[tokio::test] -async fn test_aggregate_dynamic_filter_min_max_different_columns() { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - ])); - let batches = vec![ - record_batch!(("a", Int32, [5, 1, 3, 8]), ("b", Int32, [7, 2, 4, 9])).unwrap(), - ]; - - let min_expr = - AggregateExprBuilder::new(min_udaf(), vec![col("a", &schema).unwrap()]) - .schema(Arc::clone(&schema)) - .alias("min_a") - .build() - .unwrap(); - let max_expr = - AggregateExprBuilder::new(max_udaf(), vec![col("b", &schema).unwrap()]) - .schema(Arc::clone(&schema)) - .alias("max_b") - .build() - .unwrap(); - - run_aggregate_dyn_filter_case(AggregateDynFilterCase { - schema, - batches, - aggr_exprs: vec![min_expr, max_expr], - expected_before: Some("DynamicFilter [ empty ]"), - expected_after: Some("DynamicFilter [ a@0 < 1 OR b@1 > 9 ]"), - scan_support: true, - }) - .await; -} - -/// Mix of supported/unsupported aggregates retains only the valid ones. -/// `MIN(a), MAX(a), MAX(b), MIN(c+1)`: Pushdown dynamic filter like `(a<1) or (a>8) OR (b>12)` -#[tokio::test] -async fn test_aggregate_dynamic_filter_multiple_mixed_expressions() { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - ])); - let batches = vec![ - record_batch!( - ("a", Int32, [5, 1, 3, 8]), - ("b", Int32, [10, 4, 6, 12]), - ("c", Int32, [100, 70, 90, 110]) - ) - .unwrap(), - ]; - - let min_a = AggregateExprBuilder::new(min_udaf(), vec![col("a", &schema).unwrap()]) - .schema(Arc::clone(&schema)) - .alias("min_a") - .build() - .unwrap(); - let max_a = AggregateExprBuilder::new(max_udaf(), vec![col("a", &schema).unwrap()]) - .schema(Arc::clone(&schema)) - .alias("max_a") - .build() - .unwrap(); - let max_b = AggregateExprBuilder::new(max_udaf(), vec![col("b", &schema).unwrap()]) - .schema(Arc::clone(&schema)) - .alias("max_b") - .build() - .unwrap(); - let expr_c: Arc = Arc::new(BinaryExpr::new( - col("c", &schema).unwrap(), - Operator::Plus, - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), - )); - let min_c_expr = AggregateExprBuilder::new(min_udaf(), vec![expr_c]) - .schema(Arc::clone(&schema)) - .alias("min_c_plus_one") - .build() - .unwrap(); - - run_aggregate_dyn_filter_case(AggregateDynFilterCase { - schema, - batches, - aggr_exprs: vec![min_a, max_a, max_b, min_c_expr], - expected_before: Some("DynamicFilter [ empty ]"), - expected_after: Some("DynamicFilter [ a@0 < 1 OR a@0 > 8 OR b@1 > 12 ]"), - scan_support: true, - }) - .await; -} - -/// Don't tighten the dynamic filter if all inputs are null -#[tokio::test] -async fn test_aggregate_dynamic_filter_min_all_nulls() { - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, true)])); - let batches = vec![record_batch!(("a", Int32, [None, None, None, None])).unwrap()]; - - let min_expr = - AggregateExprBuilder::new(min_udaf(), vec![col("a", &schema).unwrap()]) - .schema(Arc::clone(&schema)) - .alias("min_a") - .build() - .unwrap(); - - run_aggregate_dyn_filter_case(AggregateDynFilterCase { - schema, - batches, - aggr_exprs: vec![min_expr], - expected_before: Some("DynamicFilter [ empty ]"), - // After reading the input it hasn't a meaningful bound to update, so the - // predicate `true` means don't filter out anything - expected_after: Some("DynamicFilter [ true ]"), - scan_support: true, - }) - .await; -} - -/// Test aggregate dynamic filter is working when reading parquet files -/// -/// Runs 'select max(id) from test_table where id > 1', and ensure some file ranges -/// pruned by the dynamic filter. -#[tokio::test] -async fn test_aggregate_dynamic_filter_parquet_e2e() { - let config = SessionConfig::new() - .with_collect_statistics(true) - .with_target_partitions(2) - .set_bool("datafusion.optimizer.enable_dynamic_filter_pushdown", true) - .set_bool("datafusion.execution.parquet.pushdown_filters", true); - let ctx = SessionContext::new_with_config(config); - - let data_path = format!( - "{}/tests/data/test_statistics_per_partition/", - env!("CARGO_MANIFEST_DIR") - ); - - ctx.register_parquet("test_table", &data_path, ParquetReadOptions::default()) - .await - .unwrap(); - - // partition 1: - // files: ..03-01(id=4), ..03-02(id=3) - // partition 1: - // files: ..03-03(id=2), ..03-04(id=1) - // - // In partition 1, after reading the first file, the dynamic filter will be update - // to "id > 4", so the `..03-02` file must be able to get pruned out - let df = ctx - .sql("explain analyze select max(id) from test_table where id > 1") - .await - .unwrap(); - - let result = df.collect().await.unwrap(); - - let formatted = pretty_format_batches(&result).unwrap(); - let explain_analyze = format!("{formatted}"); - - // Capture "2" from "files_ranges_pruned_statistics=4 total → 2 matched" - let re = Regex::new( - r"files_ranges_pruned_statistics\s*=\s*(\d+)\s*total\s*[→>\-]\s*(\d+)\s*matched", - ) - .unwrap(); - - if let Some(caps) = re.captures(&explain_analyze) { - let matched_num: i32 = caps[2].parse().unwrap(); - assert!( - matched_num < 4, - "Total 4 files, if some pruned, the matched count is < 4" - ); - } else { - unreachable!("metrics should exist") - } -} +// +// The end-to-end min/max dynamic filter cases (simple/min/max/mixed/all-nulls) +// have been ported to +// `datafusion/sqllogictest/test_files/push_down_filter_regression.slt`. +// The `run_aggregate_dyn_filter_case` harness used to drive them was removed +// along with the test functions. /// Non-partial (Single) aggregates should skip dynamic filter initialization. #[test] @@ -3115,139 +1806,6 @@ fn test_aggregate_dynamic_filter_not_created_for_single_mode() { ); } -#[tokio::test] -async fn test_aggregate_filter_pushdown() { - // Test that filters can pass through AggregateExec even with aggregate functions - // when the filter references grouping columns - // Simulates: SELECT a, COUNT(b) FROM table WHERE a = 'x' GROUP BY a - - let batches = vec![ - record_batch!(("a", Utf8, ["x", "y"]), ("b", Utf8, ["foo", "bar"])).unwrap(), - ]; - - let scan = TestScanBuilder::new(schema()) - .with_support(true) - .with_batches(batches) - .build(); - - // Create an aggregate: GROUP BY a with COUNT(b) - let group_by = PhysicalGroupBy::new_single(vec![( - col("a", &schema()).unwrap(), - "a".to_string(), - )]); - - // Add COUNT aggregate - let count_expr = - AggregateExprBuilder::new(count_udaf(), vec![col("b", &schema()).unwrap()]) - .schema(schema()) - .alias("count") - .build() - .unwrap(); - - let aggregate = Arc::new( - AggregateExec::try_new( - AggregateMode::Partial, - group_by, - vec![count_expr.into()], // Has aggregate function - vec![None], // No filter on the aggregate function - Arc::clone(&scan), - schema(), - ) - .unwrap(), - ); - - // Add a filter on the grouping column 'a' - let predicate = col_lit_predicate("a", "x", &schema()); - let plan = Arc::new(FilterExec::try_new(predicate, aggregate).unwrap()) - as Arc; - - // Even with aggregate functions, filter on grouping column should be pushed through - insta::assert_snapshot!( - OptimizationTest::new(Arc::clone(&plan), FilterPushdown::new(), true), - @r" - OptimizationTest: - input: - - FilterExec: a@0 = x - - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - output: - Ok: - - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count], ordering_mode=Sorted - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = x - " - ); -} - -#[tokio::test] -async fn test_no_pushdown_filter_on_aggregate_result() { - // Test that filters on aggregate results (not grouping columns) are NOT pushed through - // SELECT a, COUNT(b) as cnt FROM table GROUP BY a HAVING cnt > 5 - // The filter on 'cnt' cannot be pushed down because it's an aggregate result - - let batches = vec![ - record_batch!(("a", Utf8, ["x", "y"]), ("b", Utf8, ["foo", "bar"])).unwrap(), - ]; - - let scan = TestScanBuilder::new(schema()) - .with_support(true) - .with_batches(batches) - .build(); - - // Create an aggregate: GROUP BY a with COUNT(b) - let group_by = PhysicalGroupBy::new_single(vec![( - col("a", &schema()).unwrap(), - "a".to_string(), - )]); - - // Add COUNT aggregate - let count_expr = - AggregateExprBuilder::new(count_udaf(), vec![col("b", &schema()).unwrap()]) - .schema(schema()) - .alias("count") - .build() - .unwrap(); - - let aggregate = Arc::new( - AggregateExec::try_new( - AggregateMode::Partial, - group_by, - vec![count_expr.into()], - vec![None], - Arc::clone(&scan), - schema(), - ) - .unwrap(), - ); - - // Add a filter on the aggregate output column - // This simulates filtering on COUNT result, which should NOT be pushed through - let agg_schema = aggregate.schema(); - let predicate = Arc::new(BinaryExpr::new( - Arc::new(Column::new_with_schema("count[count]", &agg_schema).unwrap()), - Operator::Gt, - Arc::new(Literal::new(ScalarValue::Int64(Some(5)))), - )); - let plan = Arc::new(FilterExec::try_new(predicate, aggregate).unwrap()) - as Arc; - - // The filter should NOT be pushed through the aggregate since it's on an aggregate result - insta::assert_snapshot!( - OptimizationTest::new(Arc::clone(&plan), FilterPushdown::new(), true), - @r" - OptimizationTest: - input: - - FilterExec: count[count]@1 > 5 - - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - output: - Ok: - - FilterExec: count[count]@1 > 5 - - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - " - ); -} - #[test] fn test_pushdown_filter_on_non_first_grouping_column() { // Test that filters on non-first grouping columns are still pushed down @@ -3628,209 +2186,61 @@ fn test_pushdown_through_aggregate_grouping_sets_with_reordered_input() { aggregate_expr, vec![None], projection, - reordered_schema, - ) - .unwrap(), - ); - - let agg_output_schema = aggregate.schema(); - - // Filter on b (present in all grouping sets) should be pushed down - let predicate = col_lit_predicate("b", "bar", &agg_output_schema); - let plan = Arc::new(FilterExec::try_new(predicate, aggregate.clone()).unwrap()); - - insta::assert_snapshot!( - OptimizationTest::new(plan, FilterPushdown::new(), true), - @r" - OptimizationTest: - input: - - FilterExec: b@1 = bar - - AggregateExec: mode=Final, gby=[(a@1 as a, b@2 as b), (NULL as a, b@2 as b)], aggr=[cnt] - - ProjectionExec: expr=[c@2 as c, a@0 as a, b@1 as b] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - output: - Ok: - - AggregateExec: mode=Final, gby=[(a@1 as a, b@2 as b), (NULL as a, b@2 as b)], aggr=[cnt], ordering_mode=PartiallySorted([1]) - - ProjectionExec: expr=[c@2 as c, a@0 as a, b@1 as b] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=b@1 = bar - " - ); - - // Filter on a (missing from second grouping set) should not be pushed down - let predicate = col_lit_predicate("a", "foo", &agg_output_schema); - let plan = Arc::new(FilterExec::try_new(predicate, aggregate).unwrap()); - - insta::assert_snapshot!( - OptimizationTest::new(plan, FilterPushdown::new(), true), - @r" - OptimizationTest: - input: - - FilterExec: a@0 = foo - - AggregateExec: mode=Final, gby=[(a@1 as a, b@2 as b), (NULL as a, b@2 as b)], aggr=[cnt] - - ProjectionExec: expr=[c@2 as c, a@0 as a, b@1 as b] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - output: - Ok: - - FilterExec: a@0 = foo - - AggregateExec: mode=Final, gby=[(a@1 as a, b@2 as b), (NULL as a, b@2 as b)], aggr=[cnt] - - ProjectionExec: expr=[c@2 as c, a@0 as a, b@1 as b] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - " - ); -} - -/// Regression test for https://github.com/apache/datafusion/issues/21065. -/// -/// Given a plan similar to the following, ensure that the filter is pushed down -/// through an AggregateExec whose input columns are reordered by a ProjectionExec. -#[tokio::test] -async fn test_hashjoin_dynamic_filter_pushdown_through_aggregate_with_reordered_input() { - // Build side - let build_batches = vec![record_batch!(("a", Utf8, ["h1", "h2"])).unwrap()]; - let build_schema = - Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, false)])); - let build_scan = TestScanBuilder::new(Arc::clone(&build_schema)) - .with_support(true) - .with_batches(build_batches) - .build(); - - // Probe side - let probe_batches = vec![ - record_batch!( - ("a", Utf8, ["h1", "h2", "h3", "h4"]), - ("value", Float64, [1.0, 2.0, 3.0, 4.0]) - ) - .unwrap(), - ]; - let probe_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, false), - Field::new("value", DataType::Float64, false), - ])); - let probe_scan = TestScanBuilder::new(Arc::clone(&probe_schema)) - .with_support(true) - .with_batches(probe_batches) - .build(); - - // ProjectionExec reorders (a, value) → (value, a) - let reordered_schema = Arc::new(Schema::new(vec![ - Field::new("value", DataType::Float64, false), - Field::new("a", DataType::Utf8, false), - ])); - let projection = Arc::new( - ProjectionExec::try_new( - vec![ - (col("value", &probe_schema).unwrap(), "value".to_string()), - (col("a", &probe_schema).unwrap(), "a".to_string()), - ], - probe_scan, - ) - .unwrap(), - ); - - // AggregateExec: GROUP BY a@1, min(value@0) - let aggregate_expr = vec![ - AggregateExprBuilder::new( - min_udaf(), - vec![col("value", &reordered_schema).unwrap()], - ) - .schema(reordered_schema.clone()) - .alias("min_value") - .build() - .map(Arc::new) - .unwrap(), - ]; - let group_by = PhysicalGroupBy::new_single(vec![( - col("a", &reordered_schema).unwrap(), // a@1 in input - "a".to_string(), - )]); - - let aggregate = Arc::new( - AggregateExec::try_new( - AggregateMode::Single, - group_by, - aggregate_expr, - vec![None], - projection, - reordered_schema, - ) - .unwrap(), - ); - - // Aggregate output schema: (a@0, min_value@1) - let agg_output_schema = aggregate.schema(); - - // Join the build and probe side - let plan = Arc::new( - HashJoinExec::try_new( - build_scan, - aggregate, - vec![( - col("a", &build_schema).unwrap(), - col("a", &agg_output_schema).unwrap(), - )], - None, - &JoinType::Inner, - None, - PartitionMode::CollectLeft, - datafusion_common::NullEquality::NullEqualsNothing, - false, + reordered_schema, ) .unwrap(), - ) as Arc; + ); + + let agg_output_schema = aggregate.schema(); + + // Filter on b (present in all grouping sets) should be pushed down + let predicate = col_lit_predicate("b", "bar", &agg_output_schema); + let plan = Arc::new(FilterExec::try_new(predicate, aggregate.clone()).unwrap()); - // The HashJoin's dynamic filter on `a` should push - // through the aggregate and reach the probe-side DataSource. insta::assert_snapshot!( - OptimizationTest::new(Arc::clone(&plan), FilterPushdown::new_post_optimization(), true), + OptimizationTest::new(plan, FilterPushdown::new(), true), @r" OptimizationTest: input: - - HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a], file_type=test, pushdown_supported=true - - AggregateExec: mode=Single, gby=[a@1 as a], aggr=[min_value] - - ProjectionExec: expr=[value@1 as value, a@0 as a] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, value], file_type=test, pushdown_supported=true + - FilterExec: b@1 = bar + - AggregateExec: mode=Final, gby=[(a@1 as a, b@2 as b), (NULL as a, b@2 as b)], aggr=[cnt] + - ProjectionExec: expr=[c@2 as c, a@0 as a, b@1 as b] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true output: Ok: - - HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a], file_type=test, pushdown_supported=true - - AggregateExec: mode=Single, gby=[a@1 as a], aggr=[min_value] - - ProjectionExec: expr=[value@1 as value, a@0 as a] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, value], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ] + - AggregateExec: mode=Final, gby=[(a@1 as a, b@2 as b), (NULL as a, b@2 as b)], aggr=[cnt], ordering_mode=PartiallySorted([1]) + - ProjectionExec: expr=[c@2 as c, a@0 as a, b@1 as b] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=b@1 = bar " ); - // Actually execute the plan to verify the dynamic filter is populated - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); - - let session_config = SessionConfig::new().with_batch_size(10); - let session_ctx = SessionContext::new_with_config(session_config); - session_ctx.register_object_store( - ObjectStoreUrl::parse("test://").unwrap().as_ref(), - Arc::new(InMemory::new()), - ); - let state = session_ctx.state(); - let task_ctx = state.task_ctx(); - let mut stream = plan.execute(0, Arc::clone(&task_ctx)).unwrap(); - stream.next().await.unwrap().unwrap(); + // Filter on a (missing from second grouping set) should not be pushed down + let predicate = col_lit_predicate("a", "foo", &agg_output_schema); + let plan = Arc::new(FilterExec::try_new(predicate, aggregate).unwrap()); - // After execution, the dynamic filter should be populated with values insta::assert_snapshot!( - format!("{}", format_plan_for_test(&plan)), + OptimizationTest::new(plan, FilterPushdown::new(), true), @r" - - HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a], file_type=test, pushdown_supported=true - - AggregateExec: mode=Single, gby=[a@1 as a], aggr=[min_value] - - ProjectionExec: expr=[value@1 as value, a@0 as a] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, value], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= h1 AND a@0 <= h2 AND a@0 IN (SET) ([h1, h2]) ] + OptimizationTest: + input: + - FilterExec: a@0 = foo + - AggregateExec: mode=Final, gby=[(a@1 as a, b@2 as b), (NULL as a, b@2 as b)], aggr=[cnt] + - ProjectionExec: expr=[c@2 as c, a@0 as a, b@1 as b] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + output: + Ok: + - FilterExec: a@0 = foo + - AggregateExec: mode=Final, gby=[(a@1 as a, b@2 as b), (NULL as a, b@2 as b)], aggr=[cnt] + - ProjectionExec: expr=[c@2 as c, a@0 as a, b@1 as b] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true " ); } +/// Regression test for https://github.com/apache/datafusion/issues/21065. +/// +/// Given a plan similar to the following, ensure that the filter is pushed down +/// through an AggregateExec whose input columns are reordered by a ProjectionExec. #[test] fn test_pushdown_with_computed_grouping_key() { // Test filter pushdown with computed grouping expression @@ -3892,6 +2302,11 @@ fn test_pushdown_with_computed_grouping_key() { ); } +// Not portable to sqllogictest: in CollectLeft (the mode SQL picks for small +// data), an empty build side short-circuits the HashJoin and the probe scan +// is never executed, so its dynamic filter stays at `[ empty ]` rather than +// collapsing to `[ false ]`. The Rust test uses PartitionMode::Partitioned +// on a hand-wired plan, which does trigger the `false` path. #[tokio::test] async fn test_hashjoin_dynamic_filter_all_partitions_empty() { use datafusion_common::JoinType; @@ -4024,145 +2439,9 @@ async fn test_hashjoin_dynamic_filter_all_partitions_empty() { ); } -#[tokio::test] -async fn test_hashjoin_dynamic_filter_with_nulls() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - - // Test scenario where build side has NULL values in join keys - // This validates NULL handling in bounds computation and filter generation - - // Create build side with NULL values - let build_batch = RecordBatch::try_new( - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, true), // nullable - Field::new("b", DataType::Int32, true), // nullable - ])), - vec![ - Arc::new(StringArray::from(vec![Some("aa"), None, Some("ab")])), - Arc::new(Int32Array::from(vec![Some(1), Some(2), None])), - ], - ) - .unwrap(); - let build_batches = vec![build_batch]; - let build_side_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, true), - Field::new("b", DataType::Int32, true), - ])); - let build_scan = TestScanBuilder::new(Arc::clone(&build_side_schema)) - .with_support(true) - .with_batches(build_batches) - .build(); - - // Create probe side with nullable fields - let probe_batch = RecordBatch::try_new( - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Float64, false), - ])), - vec![ - Arc::new(StringArray::from(vec![ - Some("aa"), - Some("ab"), - Some("ac"), - None, - ])), - Arc::new(Int32Array::from(vec![Some(1), Some(3), Some(4), Some(5)])), - Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0])), - ], - ) - .unwrap(); - let probe_batches = vec![probe_batch]; - let probe_side_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Float64, false), - ])); - let probe_scan = TestScanBuilder::new(Arc::clone(&probe_side_schema)) - .with_support(true) - .with_batches(probe_batches) - .build(); - - // Create HashJoinExec in CollectLeft mode (simpler for this test) - let on = vec![ - ( - col("a", &build_side_schema).unwrap(), - col("a", &probe_side_schema).unwrap(), - ), - ( - col("b", &build_side_schema).unwrap(), - col("b", &probe_side_schema).unwrap(), - ), - ]; - let plan = Arc::new( - HashJoinExec::try_new( - build_scan, - Arc::clone(&probe_scan), - on, - None, - &JoinType::Inner, - None, - PartitionMode::CollectLeft, - datafusion_common::NullEquality::NullEqualsNothing, - false, - ) - .unwrap(), - ); - - // Apply the filter pushdown optimizer - let mut config = SessionConfig::new(); - config.options_mut().execution.parquet.pushdown_filters = true; - let optimizer = FilterPushdown::new_post_optimization(); - let plan = optimizer.optimize(plan, config.options()).unwrap(); - - insta::assert_snapshot!( - format_plan_for_test(&plan), - @r" - - HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ] - " - ); - - // Put some data through the plan to check that the filter is updated to reflect the TopK state - let session_ctx = SessionContext::new_with_config(config); - session_ctx.register_object_store( - ObjectStoreUrl::parse("test://").unwrap().as_ref(), - Arc::new(InMemory::new()), - ); - let state = session_ctx.state(); - let task_ctx = state.task_ctx(); - // Execute all partitions (required for partitioned hash join coordination) - let batches = collect(Arc::clone(&plan), Arc::clone(&task_ctx)) - .await - .unwrap(); - - // Test that filters are pushed down correctly to each side of the join - insta::assert_snapshot!( - format_plan_for_test(&plan), - @r" - - HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= 1 AND b@1 <= 2 AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:1}, {c0:,c1:2}, {c0:ab,c1:}]) ] - " - ); - - #[rustfmt::skip] - let expected = [ - "+----+---+----+---+-----+", - "| a | b | a | b | c |", - "+----+---+----+---+-----+", - "| aa | 1 | aa | 1 | 1.0 |", - "+----+---+----+---+-----+", - ]; - assert_batches_eq!(&expected, &batches); -} - -/// Test that when hash_join_inlist_pushdown_max_size is set to a very small value, -/// the HashTable strategy is used instead of InList strategy, even with small build sides. -/// This test is identical to test_hashjoin_dynamic_filter_pushdown_partitioned except -/// for the config setting that forces the HashTable strategy. +// Not portable to sqllogictest: same reason as +// test_hashjoin_dynamic_filter_pushdown_partitioned — hand-wires +// PartitionMode::Partitioned, which SQL never picks for small parquet inputs. #[tokio::test] async fn test_hashjoin_hash_table_pushdown_partitioned() { use datafusion_common::JoinType; @@ -4325,10 +2604,10 @@ async fn test_hashjoin_hash_table_pushdown_partitioned() { ); } -/// Test that when hash_join_inlist_pushdown_max_size is set to a very small value, -/// the HashTable strategy is used instead of InList strategy in CollectLeft mode. -/// This test is identical to test_hashjoin_dynamic_filter_pushdown_collect_left except -/// for the config setting that forces the HashTable strategy. +// Ported to push_down_filter_parquet.slt (`hl_build`/`hl_probe` fixture). +// Rust version retained only because the slt port cannot hand-wire the +// RepartitionExec-above-probe shape this test uses; the hash_lookup vs +// IN (SET) invariant is captured in the slt port. #[tokio::test] async fn test_hashjoin_hash_table_pushdown_collect_left() { use datafusion_common::JoinType; @@ -4397,145 +2676,10 @@ async fn test_hashjoin_hash_table_pushdown_collect_left() { col("b", &probe_side_schema).unwrap(), ), ]; - let hash_join = Arc::new( - HashJoinExec::try_new( - build_scan, - probe_repartition, - on, - None, - &JoinType::Inner, - None, - PartitionMode::CollectLeft, - datafusion_common::NullEquality::NullEqualsNothing, - false, - ) - .unwrap(), - ); - - // Top-level CoalescePartitionsExec - let cp = Arc::new(CoalescePartitionsExec::new(hash_join)) as Arc; - // Add a sort for deterministic output - let plan = Arc::new(SortExec::new( - LexOrdering::new(vec![PhysicalSortExpr::new( - col("a", &probe_side_schema).unwrap(), - SortOptions::new(true, false), // descending, nulls_first - )]) - .unwrap(), - cp, - )) as Arc; - - // Apply the optimization with config setting that forces HashTable strategy - let session_config = SessionConfig::default() - .with_batch_size(10) - .set_usize("datafusion.optimizer.hash_join_inlist_pushdown_max_size", 1) - .set_bool("datafusion.execution.parquet.pushdown_filters", true) - .set_bool("datafusion.optimizer.enable_dynamic_filter_pushdown", true); - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, session_config.options()) - .unwrap(); - let session_ctx = SessionContext::new_with_config(session_config); - session_ctx.register_object_store( - ObjectStoreUrl::parse("test://").unwrap().as_ref(), - Arc::new(InMemory::new()), - ); - let state = session_ctx.state(); - let task_ctx = state.task_ctx(); - let batches = collect(Arc::clone(&plan), Arc::clone(&task_ctx)) - .await - .unwrap(); - - // Verify that hash_lookup is used instead of IN (SET) - let plan_str = format_plan_for_test(&plan).to_string(); - assert!( - plan_str.contains("hash_lookup"), - "Expected hash_lookup in plan but got: {plan_str}" - ); - assert!( - !plan_str.contains("IN (SET)"), - "Expected no IN (SET) in plan but got: {plan_str}" - ); - - let result = format!("{}", pretty_format_batches(&batches).unwrap()); - - let probe_scan_metrics = probe_scan.metrics().unwrap(); - - // The probe side had 4 rows, but after applying the dynamic filter only 2 rows should remain. - assert_eq!(probe_scan_metrics.output_rows().unwrap(), 2); - - // Results should be identical to the InList version - insta::assert_snapshot!( - result, - @r" - +----+----+-----+----+----+-----+ - | a | b | c | a | b | e | - +----+----+-----+----+----+-----+ - | ab | bb | 2.0 | ab | bb | 2.0 | - | aa | ba | 1.0 | aa | ba | 1.0 | - +----+----+-----+----+----+-----+ - ", - ); -} - -/// Test HashTable strategy with integer multi-column join keys. -/// Verifies that hash_lookup works correctly with integer data types. -#[tokio::test] -async fn test_hashjoin_hash_table_pushdown_integer_keys() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - - // Create build side with integer keys - let build_batches = vec![ - record_batch!( - ("id1", Int32, [1, 2]), - ("id2", Int32, [10, 20]), - ("value", Float64, [100.0, 200.0]) - ) - .unwrap(), - ]; - let build_side_schema = Arc::new(Schema::new(vec![ - Field::new("id1", DataType::Int32, false), - Field::new("id2", DataType::Int32, false), - Field::new("value", DataType::Float64, false), - ])); - let build_scan = TestScanBuilder::new(Arc::clone(&build_side_schema)) - .with_support(true) - .with_batches(build_batches) - .build(); - - // Create probe side with more integer rows - let probe_batches = vec![ - record_batch!( - ("id1", Int32, [1, 2, 3, 4]), - ("id2", Int32, [10, 20, 30, 40]), - ("data", Utf8, ["a", "b", "c", "d"]) - ) - .unwrap(), - ]; - let probe_side_schema = Arc::new(Schema::new(vec![ - Field::new("id1", DataType::Int32, false), - Field::new("id2", DataType::Int32, false), - Field::new("data", DataType::Utf8, false), - ])); - let probe_scan = TestScanBuilder::new(Arc::clone(&probe_side_schema)) - .with_support(true) - .with_batches(probe_batches) - .build(); - - // Create join on multiple integer columns - let on = vec![ - ( - col("id1", &build_side_schema).unwrap(), - col("id1", &probe_side_schema).unwrap(), - ), - ( - col("id2", &build_side_schema).unwrap(), - col("id2", &probe_side_schema).unwrap(), - ), - ]; - let plan = Arc::new( + let hash_join = Arc::new( HashJoinExec::try_new( build_scan, - Arc::clone(&probe_scan), + probe_repartition, on, None, &JoinType::Inner, @@ -4547,7 +2691,19 @@ async fn test_hashjoin_hash_table_pushdown_integer_keys() { .unwrap(), ); - // Apply optimization with forced HashTable strategy + // Top-level CoalescePartitionsExec + let cp = Arc::new(CoalescePartitionsExec::new(hash_join)) as Arc; + // Add a sort for deterministic output + let plan = Arc::new(SortExec::new( + LexOrdering::new(vec![PhysicalSortExpr::new( + col("a", &probe_side_schema).unwrap(), + SortOptions::new(true, false), // descending, nulls_first + )]) + .unwrap(), + cp, + )) as Arc; + + // Apply the optimization with config setting that forces HashTable strategy let session_config = SessionConfig::default() .with_batch_size(10) .set_usize("datafusion.optimizer.hash_join_inlist_pushdown_max_size", 1) @@ -4567,7 +2723,7 @@ async fn test_hashjoin_hash_table_pushdown_integer_keys() { .await .unwrap(); - // Verify hash_lookup is used + // Verify that hash_lookup is used instead of IN (SET) let plan_str = format_plan_for_test(&plan).to_string(); assert!( plan_str.contains("hash_lookup"), @@ -4581,22 +2737,30 @@ async fn test_hashjoin_hash_table_pushdown_integer_keys() { let result = format!("{}", pretty_format_batches(&batches).unwrap()); let probe_scan_metrics = probe_scan.metrics().unwrap(); - // Only 2 rows from probe side match the build side + + // The probe side had 4 rows, but after applying the dynamic filter only 2 rows should remain. assert_eq!(probe_scan_metrics.output_rows().unwrap(), 2); + // Results should be identical to the InList version insta::assert_snapshot!( result, @r" - +-----+-----+-------+-----+-----+------+ - | id1 | id2 | value | id1 | id2 | data | - +-----+-----+-------+-----+-----+------+ - | 1 | 10 | 100.0 | 1 | 10 | a | - | 2 | 20 | 200.0 | 2 | 20 | b | - +-----+-----+-------+-----+-----+------+ + +----+----+-----+----+----+-----+ + | a | b | c | a | b | e | + +----+----+-----+----+----+-----+ + | ab | bb | 2.0 | ab | bb | 2.0 | + | aa | ba | 1.0 | aa | ba | 1.0 | + +----+----+-----+----+----+-----+ ", ); } +// Not portable to sqllogictest: asserts on `HashJoinExec::dynamic_filter_for_test().is_used()` +// which is a debug-only API. The observable behavior (probe-side scan +// receiving the dynamic filter when the data source supports it) is +// already covered by the simpler CollectLeft port in push_down_filter_parquet.slt; +// the with_support(false) branch has no SQL analog (parquet always supports +// pushdown). #[tokio::test] async fn test_hashjoin_dynamic_filter_pushdown_is_used() { use datafusion_common::JoinType; @@ -4685,7 +2849,13 @@ async fn test_hashjoin_dynamic_filter_pushdown_is_used() { } } -/// Regression test for https://github.com/apache/datafusion/issues/20109 +/// Regression test for https://github.com/apache/datafusion/issues/20109. +/// +/// Not portable to sqllogictest: the regression specifically targets the +/// physical FilterPushdown rule running over *stacked* FilterExecs with +/// projections on a MemorySourceConfig. In SQL the logical optimizer +/// collapses the two filters before physical planning, so the stacked +/// FilterExec shape this test exercises is unreachable. #[tokio::test] async fn test_filter_with_projection_pushdown() { use arrow::array::{Int64Array, RecordBatch, StringArray}; @@ -4772,7 +2942,11 @@ async fn test_filter_with_projection_pushdown() { assert_batches_eq!(expected, &result); } -/// Test that ExecutionPlan::apply_expressions() can discover dynamic filters across the plan tree +/// Test that ExecutionPlan::apply_expressions() can discover dynamic filters across the plan tree. +/// +/// Not portable to sqllogictest: asserts by walking the plan tree with +/// `apply_expressions` + `downcast_ref::` and +/// counting nodes. Neither API is observable from SQL. #[tokio::test] async fn test_discover_dynamic_filters_via_expressions_api() { use datafusion_common::JoinType; @@ -4872,277 +3046,3 @@ async fn test_discover_dynamic_filters_via_expressions_api() { "After optimization, should discover exactly 2 dynamic filters (1 in HashJoinExec, 1 in DataSourceExec), found {count_after}" ); } - -#[tokio::test] -async fn test_hashjoin_dynamic_filter_pushdown_left_join() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - - // Create build side with limited values - let build_batches = vec![ - record_batch!( - ("a", Utf8, ["aa", "ab"]), - ("b", Utf8, ["ba", "bb"]), - ("c", Float64, [1.0, 2.0]) - ) - .unwrap(), - ]; - let build_side_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, false), - Field::new("b", DataType::Utf8, false), - Field::new("c", DataType::Float64, false), - ])); - let build_scan = TestScanBuilder::new(Arc::clone(&build_side_schema)) - .with_support(true) - .with_batches(build_batches) - .build(); - - // Create probe side with more values (some won't match) - let probe_batches = vec![ - record_batch!( - ("a", Utf8, ["aa", "ab", "ac", "ad"]), - ("b", Utf8, ["ba", "bb", "bc", "bd"]), - ("e", Float64, [1.0, 2.0, 3.0, 4.0]) - ) - .unwrap(), - ]; - let probe_side_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, false), - Field::new("b", DataType::Utf8, false), - Field::new("e", DataType::Float64, false), - ])); - let probe_scan = TestScanBuilder::new(Arc::clone(&probe_side_schema)) - .with_support(true) - .with_batches(probe_batches) - .build(); - - // Create HashJoinExec with Left join and CollectLeft mode - let on = vec![ - ( - col("a", &build_side_schema).unwrap(), - col("a", &probe_side_schema).unwrap(), - ), - ( - col("b", &build_side_schema).unwrap(), - col("b", &probe_side_schema).unwrap(), - ), - ]; - let plan = Arc::new( - HashJoinExec::try_new( - build_scan, - Arc::clone(&probe_scan), - on, - None, - &JoinType::Left, - None, - PartitionMode::CollectLeft, - datafusion_common::NullEquality::NullEqualsNothing, - false, - ) - .unwrap(), - ) as Arc; - - // Expect the dynamic filter predicate to be pushed down into the probe side DataSource - insta::assert_snapshot!( - OptimizationTest::new(Arc::clone(&plan), FilterPushdown::new_post_optimization(), true), - @r" - OptimizationTest: - input: - - HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@0), (b@1, b@1)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true - output: - Ok: - - HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@0), (b@1, b@1)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ] - ", - ); - - // Actually apply the optimization and execute the plan - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - config.optimizer.enable_dynamic_filter_pushdown = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); - - // Test that dynamic filter linking survives with_new_children - let children = plan.children().into_iter().map(Arc::clone).collect(); - let plan = plan.with_new_children(children).unwrap(); - - let config = SessionConfig::new().with_batch_size(10); - let session_ctx = SessionContext::new_with_config(config); - session_ctx.register_object_store( - ObjectStoreUrl::parse("test://").unwrap().as_ref(), - Arc::new(InMemory::new()), - ); - let state = session_ctx.state(); - let task_ctx = state.task_ctx(); - let batches = collect(Arc::clone(&plan), Arc::clone(&task_ctx)) - .await - .unwrap(); - - // After execution, verify the dynamic filter was populated with bounds and IN-list - insta::assert_snapshot!( - format!("{}", format_plan_for_test(&plan)), - @r" - - HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@0), (b@1, b@1)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:ba}, {c0:ab,c1:bb}]) ] - " - ); - - // Verify result correctness: left join preserves all build (left) rows. - // All build rows match probe rows here, so we get 2 matched rows. - // The dynamic filter pruned unmatched probe rows (ac, ad) at scan time, - // which is safe because those probe rows would never match any build row. - let result = format!("{}", pretty_format_batches(&batches).unwrap()); - insta::assert_snapshot!( - result, - @r" - +----+----+-----+----+----+-----+ - | a | b | c | a | b | e | - +----+----+-----+----+----+-----+ - | aa | ba | 1.0 | aa | ba | 1.0 | - | ab | bb | 2.0 | ab | bb | 2.0 | - +----+----+-----+----+----+-----+ - " - ); -} - -#[tokio::test] -async fn test_hashjoin_dynamic_filter_pushdown_left_semi_join() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - - // Create build side with limited values - let build_batches = vec![ - record_batch!( - ("a", Utf8, ["aa", "ab"]), - ("b", Utf8, ["ba", "bb"]), - ("c", Float64, [1.0, 2.0]) - ) - .unwrap(), - ]; - let build_side_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, false), - Field::new("b", DataType::Utf8, false), - Field::new("c", DataType::Float64, false), - ])); - let build_scan = TestScanBuilder::new(Arc::clone(&build_side_schema)) - .with_support(true) - .with_batches(build_batches) - .build(); - - // Create probe side with more values (some won't match) - let probe_batches = vec![ - record_batch!( - ("a", Utf8, ["aa", "ab", "ac", "ad"]), - ("b", Utf8, ["ba", "bb", "bc", "bd"]), - ("e", Float64, [1.0, 2.0, 3.0, 4.0]) - ) - .unwrap(), - ]; - let probe_side_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, false), - Field::new("b", DataType::Utf8, false), - Field::new("e", DataType::Float64, false), - ])); - let probe_scan = TestScanBuilder::new(Arc::clone(&probe_side_schema)) - .with_support(true) - .with_batches(probe_batches) - .build(); - - // Create HashJoinExec with LeftSemi join and CollectLeft mode - let on = vec![ - ( - col("a", &build_side_schema).unwrap(), - col("a", &probe_side_schema).unwrap(), - ), - ( - col("b", &build_side_schema).unwrap(), - col("b", &probe_side_schema).unwrap(), - ), - ]; - let plan = Arc::new( - HashJoinExec::try_new( - build_scan, - Arc::clone(&probe_scan), - on, - None, - &JoinType::LeftSemi, - None, - PartitionMode::CollectLeft, - datafusion_common::NullEquality::NullEqualsNothing, - false, - ) - .unwrap(), - ) as Arc; - - // Expect the dynamic filter predicate to be pushed down into the probe side DataSource - insta::assert_snapshot!( - OptimizationTest::new(Arc::clone(&plan), FilterPushdown::new_post_optimization(), true), - @r" - OptimizationTest: - input: - - HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(a@0, a@0), (b@1, b@1)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true - output: - Ok: - - HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(a@0, a@0), (b@1, b@1)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ] - ", - ); - - // Actually apply the optimization and execute the plan - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - config.optimizer.enable_dynamic_filter_pushdown = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); - - // Test that dynamic filter linking survives with_new_children - let children = plan.children().into_iter().map(Arc::clone).collect(); - let plan = plan.with_new_children(children).unwrap(); - - let config = SessionConfig::new().with_batch_size(10); - let session_ctx = SessionContext::new_with_config(config); - session_ctx.register_object_store( - ObjectStoreUrl::parse("test://").unwrap().as_ref(), - Arc::new(InMemory::new()), - ); - let state = session_ctx.state(); - let task_ctx = state.task_ctx(); - let batches = collect(Arc::clone(&plan), Arc::clone(&task_ctx)) - .await - .unwrap(); - - // After execution, verify the dynamic filter was populated with bounds and IN-list - insta::assert_snapshot!( - format!("{}", format_plan_for_test(&plan)), - @r" - - HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(a@0, a@0), (b@1, b@1)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:ba}, {c0:ab,c1:bb}]) ] - " - ); - - // Verify result correctness: left semi join returns only build (left) rows - // that have at least one matching probe row. Output schema is build-side columns only. - let result = format!("{}", pretty_format_batches(&batches).unwrap()); - insta::assert_snapshot!( - result, - @r" - +----+----+-----+ - | a | b | c | - +----+----+-----+ - | aa | ba | 1.0 | - | ab | bb | 2.0 | - +----+----+-----+ - " - ); -} diff --git a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt index ab6847e1c4834..8469c32a17033 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt @@ -166,6 +166,863 @@ drop table small_table; statement ok drop table large_table; + +######## +# Ported from datafusion/core/tests/physical_optimizer/filter_pushdown.rs:: +# test_topk_dynamic_filter_pushdown_integration +# +# Integration test for dynamic filter pushdown with TopK: +# a TopK over an ordered parquet file emits a dynamic filter that should +# get pushed into the parquet scan and prune every row group except the +# first (128 rows), visible via `pushdown_rows_matched` / `pushdown_rows_pruned`. +# +# `t * t` (rather than `t`) is used as the sort expression so the +# order-pushdown optimizer does not rewrite the TopK away. +######## + +statement ok +set datafusion.execution.parquet.max_row_group_size = 128; + +query I +COPY ( + SELECT 1372708800 + value AS t + FROM generate_series(0, 99999) + ORDER BY t +) TO 'test_files/scratch/push_down_filter_parquet/topk_pushdown.parquet' +STORED AS PARQUET; +---- +100000 + +statement ok +CREATE EXTERNAL TABLE topk_pushdown +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/topk_pushdown.parquet'; + +statement ok +set datafusion.explain.analyze_categories = 'rows'; + +query TT +EXPLAIN ANALYZE SELECT t FROM topk_pushdown ORDER BY t * t LIMIT 10; +---- +Plan with Metrics +01)SortExec: TopK(fetch=10), expr=[t@0 * t@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[t@0 * t@0 < 1884329474306198481], metrics=[output_rows=10, output_batches=1, row_replacements=10] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_pushdown.parquet]]}, projection=[t], output_ordering=[t@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ t@0 * t@0 < 1884329474306198481 ], metrics=[output_rows=128, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=782 total → 782 matched, row_groups_pruned_bloom_filter=782 total → 782 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=128, pushdown_rows_pruned=99.87 K, predicate_cache_inner_records=128, predicate_cache_records=128, scan_efficiency_ratio=64.87% (258.7 K/398.8 K)] + +statement ok +reset datafusion.explain.analyze_categories; + +statement ok +reset datafusion.execution.parquet.max_row_group_size; + +statement ok +drop table topk_pushdown; + + +######## +# Ported from datafusion/core/tests/physical_optimizer/filter_pushdown.rs:: +# test_topk_dynamic_filter_pushdown +# +# A `ORDER BY b DESC LIMIT 1` over a parquet file should emit a dynamic +# filter on the TopK that gets pushed into the scan's predicate, visible +# in EXPLAIN ANALYZE as `DynamicFilter [ b > ]` tightened to the +# running max seen by the TopK during execution. +######## + +statement ok +COPY ( + SELECT * FROM (VALUES + ('aa', 'bd', 1.0), + ('ab', 'bc', 2.0), + ('ac', 'bb', 2.0), + ('ad', 'ba', 1.0) + ) AS v(a, b, c) +) TO 'test_files/scratch/push_down_filter_parquet/topk_single_col.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE topk_single_col (a VARCHAR, b VARCHAR, c DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/topk_single_col.parquet'; + +# Data-correctness check — top row by b desc is (aa, bd, 1). +query TTR +SELECT * FROM topk_single_col ORDER BY b DESC LIMIT 1; +---- +aa bd 1 + +# Before execution: a dynamic filter is emitted but still empty +# (DynamicFilter [ empty ]) on the parquet scan. +query TT +EXPLAIN SELECT * FROM topk_single_col ORDER BY b DESC LIMIT 1; +---- +physical_plan +01)SortExec: TopK(fetch=1), expr=[b@1 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_single_col.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilter [ empty ] + +statement ok +set datafusion.explain.analyze_categories = 'rows'; + +# After execution: the dynamic filter is tightened to `b > `. +query TT +EXPLAIN ANALYZE SELECT * FROM topk_single_col ORDER BY b DESC LIMIT 1; +---- +Plan with Metrics +01)SortExec: TopK(fetch=1), expr=[b@1 DESC], preserve_partitioning=[false], filter=[b@1 IS NULL OR b@1 > bd], metrics=[output_rows=1, output_batches=1, row_replacements=1] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_single_col.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilter [ b@1 IS NULL OR b@1 > bd ], pruning_predicate=b_null_count@0 > 0 OR b_null_count@0 != row_count@2 AND b_max@1 > bd, required_guarantees=[], metrics=[output_rows=4, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=4, pushdown_rows_pruned=0, predicate_cache_inner_records=4, predicate_cache_records=4, scan_efficiency_ratio=22.37% (240/1.07 K)] + +statement ok +reset datafusion.explain.analyze_categories; + +statement ok +drop table topk_single_col; + + +######## +# Ported from datafusion/core/tests/physical_optimizer/filter_pushdown.rs:: +# test_topk_dynamic_filter_pushdown_multi_column_sort +# +# TopK with a compound sort key (b ASC NULLS LAST, a DESC). The dynamic +# filter emitted to the scan should encode the tiebreaker as a compound +# predicate of the form `b < OR (b = AND a > )`. +# With fetch=2 the top-2 rows are (ad, ba) and (ac, bb), so the final +# filter reads `b < bb OR (b = bb AND a > ac)`. +######## + +statement ok +COPY ( + SELECT * FROM (VALUES + ('ac', 'bb', 2.0), + ('ad', 'ba', 1.0), + ('aa', 'bc', 1.0), + ('ab', 'bd', 2.0) + ) AS v(a, b, c) +) TO 'test_files/scratch/push_down_filter_parquet/topk_multi_col.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE topk_multi_col (a VARCHAR, b VARCHAR, c DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/topk_multi_col.parquet'; + +# Data-correctness: confirm the top-2 rows come out in the expected order. +query TTR +SELECT * FROM topk_multi_col ORDER BY b ASC NULLS LAST, a DESC LIMIT 2; +---- +ad ba 1 +ac bb 2 + +statement ok +set datafusion.explain.analyze_categories = 'rows'; + +query TT +EXPLAIN ANALYZE SELECT * FROM topk_multi_col ORDER BY b ASC NULLS LAST, a DESC LIMIT 2; +---- +Plan with Metrics +01)SortExec: TopK(fetch=2), expr=[b@1 ASC NULLS LAST, a@0 DESC], preserve_partitioning=[false], filter=[b@1 < bb OR b@1 = bb AND (a@0 IS NULL OR a@0 > ac)], metrics=[output_rows=2, output_batches=1, row_replacements=2] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_multi_col.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilter [ b@1 < bb OR b@1 = bb AND (a@0 IS NULL OR a@0 > ac) ], pruning_predicate=b_null_count@1 != row_count@2 AND b_min@0 < bb OR b_null_count@1 != row_count@2 AND b_min@0 <= bb AND bb <= b_max@3 AND (a_null_count@4 > 0 OR a_null_count@4 != row_count@2 AND a_max@5 > ac), required_guarantees=[], metrics=[output_rows=4, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=4, pushdown_rows_pruned=0, predicate_cache_inner_records=8, predicate_cache_records=8, scan_efficiency_ratio=22.37% (240/1.07 K)] + +statement ok +reset datafusion.explain.analyze_categories; + +statement ok +drop table topk_multi_col; + + +######## +# Ported from datafusion/core/tests/physical_optimizer/filter_pushdown.rs:: +# test_hashjoin_dynamic_filter_pushdown +# +# CollectLeft hash join on two equi-join keys: the dynamic filter emitted +# by the build side should cover both keys and include an `IN (SET)` +# predicate over `struct(a, b)` for the rows it collected. +######## + +statement ok +COPY ( + SELECT * FROM (VALUES + ('aa', 'ba', 1.0), + ('ab', 'bb', 2.0) + ) AS v(a, b, c) +) TO 'test_files/scratch/push_down_filter_parquet/join_build.parquet' +STORED AS PARQUET; + +statement ok +COPY ( + SELECT * FROM (VALUES + ('aa', 'ba', 1.0), + ('ab', 'bb', 2.0), + ('ac', 'bc', 3.0), + ('ad', 'bd', 4.0) + ) AS v(a, b, e) +) TO 'test_files/scratch/push_down_filter_parquet/join_probe.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE join_build (a VARCHAR, b VARCHAR, c DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/join_build.parquet'; + +statement ok +CREATE EXTERNAL TABLE join_probe (a VARCHAR, b VARCHAR, e DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/join_probe.parquet'; + +# Data-correctness: only the 2 build rows should match. +query TTRR +SELECT p.a, p.b, build.c, p.e +FROM join_probe p INNER JOIN join_build AS build + ON p.a = build.a AND p.b = build.b +ORDER BY p.a; +---- +aa ba 1 1 +ab bb 2 2 + +statement ok +set datafusion.explain.analyze_categories = 'rows'; + +query TT +EXPLAIN ANALYZE +SELECT p.a, p.b, build.c, p.e +FROM join_probe p INNER JOIN join_build AS build + ON p.a = build.a AND p.b = build.b; +---- +Plan with Metrics +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)], projection=[a@3, b@4, c@2, e@5], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=1, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/join_build.parquet]]}, projection=[a, b, c], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=20.48% (214/1.04 K)] +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/join_probe.parquet]]}, projection=[a, b, e], file_type=parquet, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:ba}, {c0:ab,c1:bb}]) ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= aa AND a_null_count@1 != row_count@2 AND a_min@3 <= ab AND b_null_count@5 != row_count@2 AND b_max@4 >= ba AND b_null_count@5 != row_count@2 AND b_min@6 <= bb, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=22.78% (246/1.08 K)] + +statement ok +reset datafusion.explain.analyze_categories; + +statement ok +drop table join_build; + +statement ok +drop table join_probe; + + +######## +# Ported from datafusion/core/tests/physical_optimizer/filter_pushdown.rs:: +# test_nested_hashjoin_dynamic_filter_pushdown +# +# Nested hash joins: `t1 JOIN (t2 JOIN t3 ON t2.c = t3.d) ON t1.a = t2.b` +# should push dynamic filters down to BOTH the t2 and t3 scans +# (t2 gets a filter on `b`, t3 gets a filter on `d`). +######## + +statement ok +COPY ( + SELECT * FROM (VALUES + ('aa', 1.0), + ('ab', 2.0) + ) AS v(a, x) +) TO 'test_files/scratch/push_down_filter_parquet/nested_t1.parquet' +STORED AS PARQUET; + +statement ok +COPY ( + SELECT * FROM (VALUES + ('aa', 'ca', 1.0), + ('ab', 'cb', 2.0), + ('ac', 'cc', 3.0), + ('ad', 'cd', 4.0), + ('ae', 'ce', 5.0) + ) AS v(b, c, y) +) TO 'test_files/scratch/push_down_filter_parquet/nested_t2.parquet' +STORED AS PARQUET; + +statement ok +COPY ( + SELECT * FROM (VALUES + ('ca', 1.0), + ('cb', 2.0), + ('cc', 3.0), + ('cd', 4.0), + ('ce', 5.0), + ('cf', 6.0), + ('cg', 7.0), + ('ch', 8.0) + ) AS v(d, z) +) TO 'test_files/scratch/push_down_filter_parquet/nested_t3.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE nested_t1 (a VARCHAR, x DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/nested_t1.parquet'; + +statement ok +CREATE EXTERNAL TABLE nested_t2 (b VARCHAR, c VARCHAR, y DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/nested_t2.parquet'; + +statement ok +CREATE EXTERNAL TABLE nested_t3 (d VARCHAR, z DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/nested_t3.parquet'; + +statement ok +set datafusion.explain.analyze_categories = 'rows'; + +query TT +EXPLAIN ANALYZE +SELECT * +FROM nested_t1 +INNER JOIN nested_t2 ON nested_t1.a = nested_t2.b +INNER JOIN nested_t3 ON nested_t2.c = nested_t3.d; +---- +Plan with Metrics +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@3, d@0)], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=1, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, b@0)], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=1, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/nested_t1.parquet]]}, projection=[a, x], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=18.23% (144/790)] +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/nested_t2.parquet]]}, projection=[b, c, y], file_type=parquet, predicate=DynamicFilter [ b@0 >= aa AND b@0 <= ab AND b@0 IN (SET) ([aa, ab]) ], pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 >= aa AND b_null_count@1 != row_count@2 AND b_min@3 <= ab AND (b_null_count@1 != row_count@2 AND b_min@3 <= aa AND aa <= b_max@0 OR b_null_count@1 != row_count@2 AND b_min@3 <= ab AND ab <= b_max@0), required_guarantees=[b in (aa, ab)], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, page_index_rows_pruned=5 total → 5 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=3, predicate_cache_inner_records=5, predicate_cache_records=2, scan_efficiency_ratio=23.2% (252/1.09 K)] +05)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/nested_t3.parquet]]}, projection=[d, z], file_type=parquet, predicate=DynamicFilter [ d@0 >= ca AND d@0 <= cb AND hash_lookup ], pruning_predicate=d_null_count@1 != row_count@2 AND d_max@0 >= ca AND d_null_count@1 != row_count@2 AND d_min@3 <= cb, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, page_index_rows_pruned=8 total → 8 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=6, predicate_cache_inner_records=8, predicate_cache_records=2, scan_efficiency_ratio=22.12% (184/832)] + +statement ok +reset datafusion.explain.analyze_categories; + +statement ok +drop table nested_t1; + +statement ok +drop table nested_t2; + +statement ok +drop table nested_t3; + + +######## +# Ported from datafusion/core/tests/physical_optimizer/filter_pushdown.rs:: +# test_hashjoin_parent_filter_pushdown +# +# A FilterExec above a HashJoin should split: the single-side predicates +# should be pushed down to each scan, while any cross-side predicate +# stays above the join. +######## + +statement ok +COPY ( + SELECT * FROM (VALUES + ('aa', 'ba', 1.0), + ('ab', 'bb', 2.0) + ) AS v(a, b, c) +) TO 'test_files/scratch/push_down_filter_parquet/parent_build.parquet' +STORED AS PARQUET; + +statement ok +COPY ( + SELECT * FROM (VALUES + ('aa', 'ba', 1.0), + ('ab', 'bb', 2.0), + ('ac', 'bc', 3.0), + ('ad', 'bd', 4.0) + ) AS v(d, e, f) +) TO 'test_files/scratch/push_down_filter_parquet/parent_probe.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE parent_build (a VARCHAR, b VARCHAR, c DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/parent_build.parquet'; + +statement ok +CREATE EXTERNAL TABLE parent_probe (d VARCHAR, e VARCHAR, f DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/parent_probe.parquet'; + +query TT +EXPLAIN +SELECT * +FROM parent_build +INNER JOIN parent_probe ON parent_build.a = parent_probe.d +WHERE parent_build.a = 'aa' AND parent_probe.e = 'ba'; +---- +physical_plan +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, d@0)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/parent_build.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=a@0 = aa, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= aa AND aa <= a_max@1, required_guarantees=[a in (aa)] +03)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/parent_probe.parquet]]}, projection=[d, e, f], file_type=parquet, predicate=e@1 = ba AND d@0 = aa AND DynamicFilter [ empty ], pruning_predicate=e_null_count@2 != row_count@3 AND e_min@0 <= ba AND ba <= e_max@1 AND d_null_count@6 != row_count@3 AND d_min@4 <= aa AND aa <= d_max@5, required_guarantees=[d in (aa), e in (ba)] + +statement ok +drop table parent_build; + +statement ok +drop table parent_probe; + + +######## +# Ported from datafusion/core/tests/physical_optimizer/filter_pushdown.rs:: +# test_dynamic_filter_pushdown_through_hash_join_with_topk +# +# Composition: a TopK sitting above an InnerJoin should push its dynamic +# filter THROUGH the hash join and into the probe-side scan, where it +# gets combined (AND) with the hash join's own dynamic filter. The +# probe-side predicate should contain BOTH a `DynamicFilter [ d IN (...) ]` +# (from the join) and a `DynamicFilter [ e < ]` (from the TopK). +######## + +statement ok +COPY ( + SELECT * FROM (VALUES + ('aa', 'ba', 1.0), + ('ab', 'bb', 2.0) + ) AS v(a, b, c) +) TO 'test_files/scratch/push_down_filter_parquet/topk_join_build.parquet' +STORED AS PARQUET; + +statement ok +COPY ( + SELECT * FROM (VALUES + ('aa', 'ba', 1.0), + ('ab', 'bb', 2.0), + ('ac', 'bc', 3.0), + ('ad', 'bd', 4.0) + ) AS v(d, e, f) +) TO 'test_files/scratch/push_down_filter_parquet/topk_join_probe.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE topk_join_build (a VARCHAR, b VARCHAR, c DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/topk_join_build.parquet'; + +statement ok +CREATE EXTERNAL TABLE topk_join_probe (d VARCHAR, e VARCHAR, f DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/topk_join_probe.parquet'; + +statement ok +set datafusion.explain.analyze_categories = 'rows'; + +query TT +EXPLAIN ANALYZE +SELECT topk_join_probe.e +FROM topk_join_build +INNER JOIN topk_join_probe ON topk_join_build.a = topk_join_probe.d +ORDER BY topk_join_probe.e ASC +LIMIT 2; +---- +Plan with Metrics +01)SortExec: TopK(fetch=2), expr=[e@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[e@0 < bb], metrics=[output_rows=2, output_batches=1, row_replacements=2] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, d@0)], projection=[e@2], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=1, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_join_build.parquet]]}, projection=[a], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=6.7% (70/1.04 K)] +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_join_probe.parquet]]}, projection=[d, e], file_type=parquet, predicate=DynamicFilter [ d@0 >= aa AND d@0 <= ab AND d@0 IN (SET) ([aa, ab]) ] AND DynamicFilter [ e@1 < bb ], pruning_predicate=d_null_count@1 != row_count@2 AND d_max@0 >= aa AND d_null_count@1 != row_count@2 AND d_min@3 <= ab AND (d_null_count@1 != row_count@2 AND d_min@3 <= aa AND aa <= d_max@0 OR d_null_count@1 != row_count@2 AND d_min@3 <= ab AND ab <= d_max@0) AND e_null_count@5 != row_count@2 AND e_min@4 < bb, required_guarantees=[d in (aa, ab)], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, page_index_rows_pruned=4 total → 4 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=15.37% (166/1.08 K)] + +statement ok +reset datafusion.explain.analyze_categories; + +statement ok +drop table topk_join_build; + +statement ok +drop table topk_join_probe; + + +######## +# Ported from datafusion/core/tests/physical_optimizer/filter_pushdown.rs:: +# test_topk_with_projection_transformation_on_dyn_filter +# +# When a ProjectionExec sits between a TopK and the parquet scan, the +# dynamic filter emitted by the TopK (on the projected column) must be +# rewritten back to the original scan-side column before it reaches the +# scan. Covers: simple `SELECT a`, column reorder (`SELECT b, a`), +# expression projection (`SELECT a+1 AS a_plus_1`), and column alias +# shadowing (`SELECT a+1 AS a`). +######## + +statement ok +COPY ( + SELECT * FROM (VALUES + (1, 'x', 1.0), + (2, 'y', 2.0), + (3, 'z', 3.0) + ) AS v(a, b, c) +) TO 'test_files/scratch/push_down_filter_parquet/topk_proj.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE topk_proj (a INT, b VARCHAR, c DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/topk_proj.parquet'; + +statement ok +set datafusion.explain.analyze_categories = 'rows'; + +# Case 1: reorder — `SELECT b, a` — filter on the TopK uses projected column +# `a`, but the scan predicate must reference the original `a@0`. +query TT +EXPLAIN ANALYZE SELECT b, a FROM topk_proj ORDER BY a LIMIT 2; +---- +Plan with Metrics +01)SortExec: TopK(fetch=2), expr=[a@1 ASC NULLS LAST], preserve_partitioning=[false], filter=[a@1 < 2], metrics=[output_rows=2, output_batches=1, row_replacements=2] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_proj.parquet]]}, projection=[b, a], file_type=parquet, predicate=DynamicFilter [ a@0 < 2 ], pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 2, required_guarantees=[], metrics=[output_rows=3, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=3, pushdown_rows_pruned=0, predicate_cache_inner_records=3, predicate_cache_records=3, scan_efficiency_ratio=13.72% (153/1.11 K)] + +# Case 2: prune — `SELECT a` — filter stays as `a < 2` on the scan. +query TT +EXPLAIN ANALYZE SELECT a FROM topk_proj ORDER BY a LIMIT 2; +---- +Plan with Metrics +01)SortExec: TopK(fetch=2), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[a@0 < 2], metrics=[output_rows=2, output_batches=1, row_replacements=2] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_proj.parquet]]}, projection=[a], file_type=parquet, predicate=DynamicFilter [ a@0 < 2 ], pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 2, required_guarantees=[], metrics=[output_rows=3, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=3, pushdown_rows_pruned=0, predicate_cache_inner_records=3, predicate_cache_records=3, scan_efficiency_ratio=7.09% (79/1.11 K)] + +# Case 3: expression — `SELECT a+1 AS a_plus_1` — the TopK filter is on +# `a_plus_1`, the scan predicate must read `a@0 + 1`. +query TT +EXPLAIN ANALYZE SELECT a + 1 AS a_plus_1, b FROM topk_proj ORDER BY a_plus_1 LIMIT 2; +---- +Plan with Metrics +01)SortExec: TopK(fetch=2), expr=[a_plus_1@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[a_plus_1@0 < 3], metrics=[output_rows=2, output_batches=1, row_replacements=2] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_proj.parquet]]}, projection=[CAST(a@0 AS Int64) + 1 as a_plus_1, b], file_type=parquet, predicate=DynamicFilter [ CAST(a@0 AS Int64) + 1 < 3 ], metrics=[output_rows=3, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=3, pushdown_rows_pruned=0, predicate_cache_inner_records=3, predicate_cache_records=3, scan_efficiency_ratio=13.72% (153/1.11 K)] + +# Case 4: alias shadowing — `SELECT a+1 AS a` — the projection renames +# `a+1` to `a`, so the TopK's `a < 3` must still be rewritten to +# `a@0 + 1 < 3` on the scan. +query TT +EXPLAIN ANALYZE SELECT a + 1 AS a, b FROM topk_proj ORDER BY a LIMIT 2; +---- +Plan with Metrics +01)SortExec: TopK(fetch=2), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[a@0 < 3], metrics=[output_rows=2, output_batches=1, row_replacements=2] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_proj.parquet]]}, projection=[CAST(a@0 AS Int64) + 1 as a, b], file_type=parquet, predicate=DynamicFilter [ CAST(a@0 AS Int64) + 1 < 3 ], metrics=[output_rows=3, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=3, pushdown_rows_pruned=0, predicate_cache_inner_records=3, predicate_cache_records=3, scan_efficiency_ratio=13.72% (153/1.11 K)] + +statement ok +reset datafusion.explain.analyze_categories; + +statement ok +drop table topk_proj; + + +######## +# Ported from datafusion/core/tests/physical_optimizer/filter_pushdown.rs:: +# test_hashjoin_dynamic_filter_pushdown_through_aggregate_with_reordered_input +# +# The HashJoin's dynamic filter on the join key should propagate down +# through a GROUP BY aggregate and land on the underlying parquet scan. +######## + +statement ok +COPY ( + SELECT * FROM (VALUES ('h1'), ('h2')) AS v(a) +) TO 'test_files/scratch/push_down_filter_parquet/join_agg_build.parquet' +STORED AS PARQUET; + +statement ok +COPY ( + SELECT * FROM (VALUES + ('h1', 1.0), + ('h2', 2.0), + ('h3', 3.0), + ('h4', 4.0) + ) AS v(a, value) +) TO 'test_files/scratch/push_down_filter_parquet/join_agg_probe.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE join_agg_build (a VARCHAR) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/join_agg_build.parquet'; + +statement ok +CREATE EXTERNAL TABLE join_agg_probe (a VARCHAR, value DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/join_agg_probe.parquet'; + +statement ok +set datafusion.explain.analyze_categories = 'rows'; + +query TT +EXPLAIN ANALYZE +SELECT b.a, agg.min_value +FROM join_agg_build b +INNER JOIN ( + SELECT a, MIN(value) AS min_value FROM join_agg_probe GROUP BY a +) agg ON b.a = agg.a; +---- +Plan with Metrics +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)], projection=[a@0, min_value@2], metrics=[output_rows=2, output_batches=2, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=2, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/join_agg_build.parquet]]}, projection=[a], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=15.32% (70/457)] +03)--ProjectionExec: expr=[a@0 as a, min(join_agg_probe.value)@1 as min_value], metrics=[output_rows=2, output_batches=2] +04)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[min(join_agg_probe.value)], metrics=[output_rows=2, output_batches=2, spill_count=0, spilled_rows=0] +05)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1, metrics=[output_rows=2, output_batches=2, spill_count=0, spilled_rows=0] +06)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[min(join_agg_probe.value)], metrics=[output_rows=2, output_batches=1, spill_count=0, spilled_rows=0, skipped_aggregation_rows=0, reduction_factor=100% (2/2)] +07)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/join_agg_probe.parquet]]}, projection=[a, value], file_type=parquet, predicate=DynamicFilter [ a@0 >= h1 AND a@0 <= h2 AND a@0 IN (SET) ([h1, h2]) ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= h1 AND a_null_count@1 != row_count@2 AND a_min@3 <= h2 AND (a_null_count@1 != row_count@2 AND a_min@3 <= h1 AND h1 <= a_max@0 OR a_null_count@1 != row_count@2 AND a_min@3 <= h2 AND h2 <= a_max@0), required_guarantees=[a in (h1, h2)], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, page_index_rows_pruned=4 total → 4 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=4, predicate_cache_records=2, scan_efficiency_ratio=19.81% (163/823)] + +statement ok +reset datafusion.explain.analyze_categories; + +statement ok +drop table join_agg_build; + +statement ok +drop table join_agg_probe; + + +######## +# Ported from datafusion/core/tests/physical_optimizer/filter_pushdown.rs:: +# test_hashjoin_dynamic_filter_with_nulls +# +# HashJoin build side contains NULL join-key values. The dynamic filter +# emitted to the probe scan should still correctly handle NULLs. +######## + +statement ok +COPY ( + SELECT * FROM (VALUES + ('aa', 1), + (CAST(NULL AS VARCHAR), 2), + ('ab', CAST(NULL AS INT)) + ) AS v(a, b) +) TO 'test_files/scratch/push_down_filter_parquet/nulls_build.parquet' +STORED AS PARQUET; + +statement ok +COPY ( + SELECT * FROM (VALUES + ('aa', 1, 1.0), + ('ab', 3, 2.0), + ('ac', 4, 3.0), + (CAST(NULL AS VARCHAR), 5, 4.0) + ) AS v(a, b, c) +) TO 'test_files/scratch/push_down_filter_parquet/nulls_probe.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE nulls_build (a VARCHAR, b INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/nulls_build.parquet'; + +statement ok +CREATE EXTERNAL TABLE nulls_probe (a VARCHAR, b INT, c DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/nulls_probe.parquet'; + +statement ok +set datafusion.explain.analyze_categories = 'rows'; + +query TT +EXPLAIN ANALYZE +SELECT nulls_build.a, nulls_build.b, nulls_probe.a, nulls_probe.b, nulls_probe.c +FROM nulls_build INNER JOIN nulls_probe +ON nulls_build.a = nulls_probe.a AND nulls_build.b = nulls_probe.b; +---- +Plan with Metrics +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)], metrics=[output_rows=1, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=3, input_batches=1, input_rows=1, avg_fanout=100% (1/1), probe_hit_rate=100% (1/1)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/nulls_build.parquet]]}, projection=[a, b], file_type=parquet, metrics=[output_rows=3, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=18.6% (144/774)] +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/nulls_probe.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= 1 AND b@1 <= 2 AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:1}, {c0:,c1:2}, {c0:ab,c1:}]) ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= aa AND a_null_count@1 != row_count@2 AND a_min@3 <= ab AND b_null_count@5 != row_count@2 AND b_max@4 >= 1 AND b_null_count@5 != row_count@2 AND b_min@6 <= 2, required_guarantees=[], metrics=[output_rows=1, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=1, pushdown_rows_pruned=3, predicate_cache_inner_records=8, predicate_cache_records=2, scan_efficiency_ratio=21.1% (237/1.12 K)] + +statement ok +reset datafusion.explain.analyze_categories; + +statement ok +drop table nulls_build; + +statement ok +drop table nulls_probe; + + +######## +# Ported from datafusion/core/tests/physical_optimizer/filter_pushdown.rs:: +# test_hashjoin_dynamic_filter_pushdown_left_join +# test_hashjoin_dynamic_filter_pushdown_left_semi_join +# +# For LEFT and LEFT SEMI joins (without a WHERE that would convert them +# to INNER), the HashJoin's dynamic filter should still push into the +# probe scan on the join keys — unmatched probe rows that can never +# contribute to any build match are safely pruned there. +######## + +statement ok +COPY ( + SELECT * FROM (VALUES + ('aa', 'ba', 1.0), + ('ab', 'bb', 2.0) + ) AS v(a, b, c) +) TO 'test_files/scratch/push_down_filter_parquet/lj_build.parquet' +STORED AS PARQUET; + +statement ok +COPY ( + SELECT * FROM (VALUES + ('aa', 'ba', 1.0), + ('ab', 'bb', 2.0), + ('ac', 'bc', 3.0), + ('ad', 'bd', 4.0) + ) AS v(a, b, e) +) TO 'test_files/scratch/push_down_filter_parquet/lj_probe.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE lj_build (a VARCHAR, b VARCHAR, c DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/lj_build.parquet'; + +statement ok +CREATE EXTERNAL TABLE lj_probe (a VARCHAR, b VARCHAR, e DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/lj_probe.parquet'; + +statement ok +set datafusion.explain.analyze_categories = 'rows'; + +# LEFT JOIN: build is preserved; probe rows get dynamic filter based on build keys. +query TT +EXPLAIN ANALYZE +SELECT lj_build.a, lj_build.b, lj_build.c, lj_probe.a, lj_probe.b, lj_probe.e +FROM lj_build LEFT JOIN lj_probe +ON lj_build.a = lj_probe.a AND lj_build.b = lj_probe.b; +---- +Plan with Metrics +01)HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@0), (b@1, b@1)], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=2, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/lj_build.parquet]]}, projection=[a, b, c], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=20.48% (214/1.04 K)] +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/lj_probe.parquet]]}, projection=[a, b, e], file_type=parquet, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:ba}, {c0:ab,c1:bb}]) ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= aa AND a_null_count@1 != row_count@2 AND a_min@3 <= ab AND b_null_count@5 != row_count@2 AND b_max@4 >= ba AND b_null_count@5 != row_count@2 AND b_min@6 <= bb, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=22.78% (246/1.08 K)] + +# LEFT SEMI JOIN: only matching build rows are returned; probe scan still +# receives the dynamic filter. +query TT +EXPLAIN ANALYZE +SELECT lj_build.* +FROM lj_build +WHERE EXISTS ( + SELECT 1 FROM lj_probe + WHERE lj_probe.a = lj_build.a AND lj_probe.b = lj_build.b +); +---- +Plan with Metrics +01)HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(a@0, a@0), (b@1, b@1)], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=2, input_rows=4, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/lj_build.parquet]]}, projection=[a, b, c], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=20.48% (214/1.04 K)] +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/lj_probe.parquet]]}, projection=[a, b], file_type=parquet, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:ba}, {c0:ab,c1:bb}]) ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= aa AND a_null_count@1 != row_count@2 AND a_min@3 <= ab AND b_null_count@5 != row_count@2 AND b_max@4 >= ba AND b_null_count@5 != row_count@2 AND b_min@6 <= bb, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=15.37% (166/1.08 K)] + +statement ok +reset datafusion.explain.analyze_categories; + +statement ok +drop table lj_build; + +statement ok +drop table lj_probe; + + +######## +# Ported from datafusion/core/tests/physical_optimizer/filter_pushdown.rs:: +# test_hashjoin_hash_table_pushdown_collect_left +# test_hashjoin_hash_table_pushdown_integer_keys +# +# With `hash_join_inlist_pushdown_max_size = 1`, the HashJoin dynamic filter +# should use the HashTable strategy (`hash_lookup`) instead of the InList +# strategy (`IN (SET) ([...])`), even when the build side is tiny. +######## + +statement ok +set datafusion.optimizer.hash_join_inlist_pushdown_max_size = 1; + +# --- multi-column string keys --- + +statement ok +COPY ( + SELECT * FROM (VALUES + ('aa', 'ba', 1.0), + ('ab', 'bb', 2.0) + ) AS v(a, b, c) +) TO 'test_files/scratch/push_down_filter_parquet/hl_build.parquet' +STORED AS PARQUET; + +statement ok +COPY ( + SELECT * FROM (VALUES + ('aa', 'ba', 1.0), + ('ab', 'bb', 2.0), + ('ac', 'bc', 3.0), + ('ad', 'bd', 4.0) + ) AS v(a, b, e) +) TO 'test_files/scratch/push_down_filter_parquet/hl_probe.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE hl_build (a VARCHAR, b VARCHAR, c DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/hl_build.parquet'; + +statement ok +CREATE EXTERNAL TABLE hl_probe (a VARCHAR, b VARCHAR, e DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/hl_probe.parquet'; + +statement ok +set datafusion.explain.analyze_categories = 'rows'; + +# probe scan filter should contain `hash_lookup`, not `IN (SET)`. +query TT +EXPLAIN ANALYZE +SELECT p.a, p.b, build.c, p.e +FROM hl_probe p INNER JOIN hl_build AS build + ON p.a = build.a AND p.b = build.b; +---- +Plan with Metrics +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)], projection=[a@3, b@4, c@2, e@5], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=1, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/hl_build.parquet]]}, projection=[a, b, c], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=20.48% (214/1.04 K)] +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/hl_probe.parquet]]}, projection=[a, b, e], file_type=parquet, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND hash_lookup ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= aa AND a_null_count@1 != row_count@2 AND a_min@3 <= ab AND b_null_count@5 != row_count@2 AND b_max@4 >= ba AND b_null_count@5 != row_count@2 AND b_min@6 <= bb, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=22.78% (246/1.08 K)] + +statement ok +drop table hl_build; + +statement ok +drop table hl_probe; + +# --- multi-column integer keys --- + +statement ok +COPY ( + SELECT * FROM (VALUES + (1, 10, 100.0), + (2, 20, 200.0) + ) AS v(id1, id2, value) +) TO 'test_files/scratch/push_down_filter_parquet/int_build.parquet' +STORED AS PARQUET; + +statement ok +COPY ( + SELECT * FROM (VALUES + (1, 10, 'a'), + (2, 20, 'b'), + (3, 30, 'c'), + (4, 40, 'd') + ) AS v(id1, id2, data) +) TO 'test_files/scratch/push_down_filter_parquet/int_probe.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE int_build (id1 INT, id2 INT, value DOUBLE) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/int_build.parquet'; + +statement ok +CREATE EXTERNAL TABLE int_probe (id1 INT, id2 INT, data VARCHAR) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_parquet/int_probe.parquet'; + +query TT +EXPLAIN ANALYZE +SELECT b.id1, b.id2, b.value, p.data +FROM int_build b INNER JOIN int_probe p + ON b.id1 = p.id1 AND b.id2 = p.id2; +---- +Plan with Metrics +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id1@0, id1@0), (id2@1, id2@1)], projection=[id1@0, id2@1, value@2, data@5], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=1, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/int_build.parquet]]}, projection=[id1, id2, value], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=19.02% (222/1.17 K)] +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/int_probe.parquet]]}, projection=[id1, id2, data], file_type=parquet, predicate=DynamicFilter [ id1@0 >= 1 AND id1@0 <= 2 AND id2@1 >= 10 AND id2@1 <= 20 AND hash_lookup ], pruning_predicate=id1_null_count@1 != row_count@2 AND id1_max@0 >= 1 AND id1_null_count@1 != row_count@2 AND id1_min@3 <= 2 AND id2_null_count@5 != row_count@2 AND id2_max@4 >= 10 AND id2_null_count@5 != row_count@2 AND id2_min@6 <= 20, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=21.43% (239/1.11 K)] + +statement ok +reset datafusion.explain.analyze_categories; + +statement ok +reset datafusion.optimizer.hash_join_inlist_pushdown_max_size; + +statement ok +drop table int_build; + +statement ok +drop table int_probe; + + # Config reset statement ok RESET datafusion.explain.physical_plan_only; diff --git a/datafusion/sqllogictest/test_files/push_down_filter_regression.slt b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt index cfc564fa2b56c..51ad998c02307 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_regression.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt @@ -185,9 +185,318 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[(NULL as id), (id@0 as id)], aggr=[max(agg_dyn_test.id)] 05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 < 10, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 < 10, required_guarantees=[] +# Ported from datafusion/core/tests/physical_optimizer/filter_pushdown.rs:: +# test_aggregate_dynamic_filter_parquet_e2e +# +# End-to-end: `select max(id) from t where id > 1` should pick up a dynamic +# filter from the partial aggregate that prunes at least one file range +# (files_ranges_pruned_statistics must report < 4 matched out of 4 total). +# +# Build a self-contained fixture of 4 single-row parquet files, one per +# `id` value. With target_partitions=2, files are split into 2 groups of 2. +# Writing the files in decreasing-id order ensures the first file in each +# group has the larger id, so after reading it the dynamic filter tightens +# (e.g. to `id > 4` in group 1) and prunes the second file in the group. + +statement ok +COPY (VALUES (4)) TO 'test_files/scratch/push_down_filter_regression/agg_dyn/file_0.parquet' STORED AS PARQUET; + +statement ok +COPY (VALUES (3)) TO 'test_files/scratch/push_down_filter_regression/agg_dyn/file_1.parquet' STORED AS PARQUET; + +statement ok +COPY (VALUES (2)) TO 'test_files/scratch/push_down_filter_regression/agg_dyn/file_2.parquet' STORED AS PARQUET; + +statement ok +COPY (VALUES (1)) TO 'test_files/scratch/push_down_filter_regression/agg_dyn/file_3.parquet' STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE agg_dyn_e2e (column1 INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_regression/agg_dyn/'; + +statement ok +set datafusion.execution.collect_statistics = true; + +statement ok +set datafusion.explain.analyze_categories = 'rows'; + +query TT +EXPLAIN ANALYZE select max(column1) from agg_dyn_e2e where column1 > 1; +---- +Plan with Metrics +01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_dyn_e2e.column1)], metrics=[output_rows=1, output_batches=1] +02)--CoalescePartitionsExec, metrics=[output_rows=2, output_batches=2] +03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_dyn_e2e.column1)], metrics=[output_rows=2, output_batches=2] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn/file_0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn/file_1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn/file_2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn/file_3.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 > 1 AND DynamicFilter [ column1@0 > 4 ], pruning_predicate=column1_null_count@1 != row_count@2 AND column1_max@0 > 1 AND column1_null_count@1 != row_count@2 AND column1_max@0 > 4, required_guarantees=[], metrics=[output_rows=2, output_batches=2, files_ranges_pruned_statistics=4 total → 4 matched, row_groups_pruned_statistics=4 total → 2 matched -> 2 fully matched, row_groups_pruned_bloom_filter=2 total → 2 matched, page_index_pages_pruned=2 total → 2 matched, page_index_rows_pruned=2 total → 2 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=4, files_processed=4, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=0, predicate_cache_inner_records=2, predicate_cache_records=4, scan_efficiency_ratio=25.15% (130/517)] + +statement ok +reset datafusion.explain.analyze_categories; + +statement ok +reset datafusion.execution.collect_statistics; + +statement ok +drop table agg_dyn_e2e; + + +######## +# Ported from datafusion/core/tests/physical_optimizer/filter_pushdown.rs: +# test_aggregate_dynamic_filter_min_simple +# test_aggregate_dynamic_filter_max_simple +# test_aggregate_dynamic_filter_min_expression_not_supported +# test_aggregate_dynamic_filter_min_max_same_column +# test_aggregate_dynamic_filter_min_max_different_columns +# test_aggregate_dynamic_filter_multiple_mixed_expressions +# test_aggregate_dynamic_filter_min_all_nulls +# +# Baseline behavior of aggregate dynamic filter emission: +# MIN(a) -> filter `a < ` +# MAX(a) -> filter `a > ` +# MIN + MAX -> filter `a < OR a > ` (or across columns) +# MIN(a+1) -> no dynamic filter (expression input is unsupported) +# all-NULLs -> filter stays `true` (no meaningful bound to propagate) +# +# Dynamic aggregate filters are only emitted in Partial+Final mode, not +# Single mode, so each fixture is written to 2 files and runs with +# target_partitions=2 to force a multi-partition aggregate plan. + +statement ok +set datafusion.execution.target_partitions = 2; + +# --- single-column fixture ([5, 1, 3, 8]) split across 2 files --- + +statement ok +COPY ( + SELECT * FROM (VALUES (5), (1)) AS v(a) +) TO 'test_files/scratch/push_down_filter_regression/agg_dyn_single/file_0.parquet' +STORED AS PARQUET; + +statement ok +COPY ( + SELECT * FROM (VALUES (3), (8)) AS v(a) +) TO 'test_files/scratch/push_down_filter_regression/agg_dyn_single/file_1.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE agg_dyn_single (a INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_regression/agg_dyn_single/'; + +# Use `analyze_level = summary` + `analyze_categories = 'none'` so metrics +# render empty; we only care that the `predicate=DynamicFilter [ ... ]` text +# matches. Pruning metrics here are subject to a parallel-execution race +# (the order in which Partial aggregates publish filter updates vs. when the +# scan reads each partition), so the filter *content* is deterministic but +# the pruning counts are not. +statement ok +set datafusion.explain.analyze_level = summary; + +statement ok +set datafusion.explain.analyze_categories = 'none'; + +# MIN(a) -> DynamicFilter [ a < 1 ] +query TT +EXPLAIN ANALYZE SELECT MIN(a) FROM agg_dyn_single; +---- +Plan with Metrics +01)AggregateExec: mode=Final, gby=[], aggr=[min(agg_dyn_single.a)], metrics=[] +02)--CoalescePartitionsExec, metrics=[] +03)----AggregateExec: mode=Partial, gby=[], aggr=[min(agg_dyn_single.a)], metrics=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_1.parquet]]}, projection=[a], file_type=parquet, predicate=DynamicFilter [ a@0 < 1 ], pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 1, required_guarantees=[], metrics=[] + +# MAX(a) -> DynamicFilter [ a > 8 ] +query TT +EXPLAIN ANALYZE SELECT MAX(a) FROM agg_dyn_single; +---- +Plan with Metrics +01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_dyn_single.a)], metrics=[] +02)--CoalescePartitionsExec, metrics=[] +03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_dyn_single.a)], metrics=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_1.parquet]]}, projection=[a], file_type=parquet, predicate=DynamicFilter [ a@0 > 8 ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 > 8, required_guarantees=[], metrics=[] + +# MIN(a), MAX(a) -> DynamicFilter [ a < 1 OR a > 8 ] +query TT +EXPLAIN ANALYZE SELECT MIN(a), MAX(a) FROM agg_dyn_single; +---- +Plan with Metrics +01)AggregateExec: mode=Final, gby=[], aggr=[min(agg_dyn_single.a), max(agg_dyn_single.a)], metrics=[] +02)--CoalescePartitionsExec, metrics=[] +03)----AggregateExec: mode=Partial, gby=[], aggr=[min(agg_dyn_single.a), max(agg_dyn_single.a)], metrics=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_1.parquet]]}, projection=[a], file_type=parquet, predicate=DynamicFilter [ a@0 < 1 OR a@0 > 8 ], pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 1 OR a_null_count@1 != row_count@2 AND a_max@3 > 8, required_guarantees=[], metrics=[] + +# MIN(a+1) -> no dynamic filter (expression input is not a plain column) +query TT +EXPLAIN ANALYZE SELECT MIN(a + 1) FROM agg_dyn_single; +---- +Plan with Metrics +01)AggregateExec: mode=Final, gby=[], aggr=[min(agg_dyn_single.a + Int64(1))], metrics=[] +02)--CoalescePartitionsExec, metrics=[] +03)----AggregateExec: mode=Partial, gby=[], aggr=[min(agg_dyn_single.a + Int64(1))], metrics=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_1.parquet]]}, projection=[a], file_type=parquet, metrics=[] + +statement ok +drop table agg_dyn_single; + +# --- two-column fixture: MIN(a) + MAX(b) across columns --- + +statement ok +COPY ( + SELECT * FROM (VALUES (5, 7), (1, 2)) AS v(a, b) +) TO 'test_files/scratch/push_down_filter_regression/agg_dyn_two_col/file_0.parquet' +STORED AS PARQUET; + +statement ok +COPY ( + SELECT * FROM (VALUES (3, 4), (8, 9)) AS v(a, b) +) TO 'test_files/scratch/push_down_filter_regression/agg_dyn_two_col/file_1.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE agg_dyn_two_col (a INT, b INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_regression/agg_dyn_two_col/'; + +# MIN(a), MAX(b) -> DynamicFilter [ a < 1 OR b > 9 ] +query TT +EXPLAIN ANALYZE SELECT MIN(a), MAX(b) FROM agg_dyn_two_col; +---- +Plan with Metrics +01)AggregateExec: mode=Final, gby=[], aggr=[min(agg_dyn_two_col.a), max(agg_dyn_two_col.b)], metrics=[] +02)--CoalescePartitionsExec, metrics=[] +03)----AggregateExec: mode=Partial, gby=[], aggr=[min(agg_dyn_two_col.a), max(agg_dyn_two_col.b)], metrics=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_two_col/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_two_col/file_1.parquet]]}, projection=[a, b], file_type=parquet, predicate=DynamicFilter [ a@0 < 1 OR b@1 > 9 ], pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 1 OR b_null_count@4 != row_count@2 AND b_max@3 > 9, required_guarantees=[], metrics=[] + +statement ok +drop table agg_dyn_two_col; + +# --- mixed expressions: MIN(a), MAX(a), MAX(b), MIN(c+1) --- +# Supported aggregates (MIN(a), MAX(a), MAX(b)) should drive a filter; +# MIN(c+1) is unsupported and must not contribute. + +statement ok +COPY ( + SELECT * FROM (VALUES (5, 10, 100), (1, 4, 70)) AS v(a, b, c) +) TO 'test_files/scratch/push_down_filter_regression/agg_dyn_mixed/file_0.parquet' +STORED AS PARQUET; + +statement ok +COPY ( + SELECT * FROM (VALUES (3, 6, 90), (8, 12, 110)) AS v(a, b, c) +) TO 'test_files/scratch/push_down_filter_regression/agg_dyn_mixed/file_1.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE agg_dyn_mixed (a INT, b INT, c INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_regression/agg_dyn_mixed/'; + +# -> DynamicFilter [ a < 1 OR a > 8 OR b > 12 ] (MIN(c+1) dropped as unsupported) +query TT +EXPLAIN ANALYZE SELECT MIN(a), MAX(a), MAX(b), MIN(c + 1) FROM agg_dyn_mixed; +---- +Plan with Metrics +01)AggregateExec: mode=Final, gby=[], aggr=[min(agg_dyn_mixed.a), max(agg_dyn_mixed.a), max(agg_dyn_mixed.b), min(agg_dyn_mixed.c + Int64(1))], metrics=[] +02)--CoalescePartitionsExec, metrics=[] +03)----AggregateExec: mode=Partial, gby=[], aggr=[min(agg_dyn_mixed.a), max(agg_dyn_mixed.a), max(agg_dyn_mixed.b), min(agg_dyn_mixed.c + Int64(1))], metrics=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_mixed/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_mixed/file_1.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilter [ a@0 < 1 OR a@0 > 8 OR b@1 > 12 ], pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 1 OR a_null_count@1 != row_count@2 AND a_max@3 > 8 OR b_null_count@5 != row_count@2 AND b_max@4 > 12, required_guarantees=[], metrics=[] + +statement ok +drop table agg_dyn_mixed; + +# --- all-NULLs input: filter should stay `true` (no meaningful bound) --- + +statement ok +COPY ( + SELECT CAST(NULL AS INT) AS a FROM (VALUES (1), (2)) AS v(n) +) TO 'test_files/scratch/push_down_filter_regression/agg_dyn_nulls/file_0.parquet' +STORED AS PARQUET; + +statement ok +COPY ( + SELECT CAST(NULL AS INT) AS a FROM (VALUES (3), (4)) AS v(n) +) TO 'test_files/scratch/push_down_filter_regression/agg_dyn_nulls/file_1.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE agg_dyn_nulls (a INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_regression/agg_dyn_nulls/'; + +# MIN(a) over all-NULL input -> DynamicFilter [ true ] +query TT +EXPLAIN ANALYZE SELECT MIN(a) FROM agg_dyn_nulls; +---- +Plan with Metrics +01)AggregateExec: mode=Final, gby=[], aggr=[min(agg_dyn_nulls.a)], metrics=[] +02)--CoalescePartitionsExec, metrics=[] +03)----AggregateExec: mode=Partial, gby=[], aggr=[min(agg_dyn_nulls.a)], metrics=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_nulls/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_nulls/file_1.parquet]]}, projection=[a], file_type=parquet, predicate=DynamicFilter [ true ], metrics=[] + +statement ok +reset datafusion.explain.analyze_categories; + +statement ok +reset datafusion.explain.analyze_level; + +statement ok +drop table agg_dyn_nulls; + +statement ok +set datafusion.execution.target_partitions = 4; + + +######## +# Ported from datafusion/core/tests/physical_optimizer/filter_pushdown.rs:: +# test_aggregate_filter_pushdown +# test_no_pushdown_filter_on_aggregate_result +# +# - A filter that only references grouping columns (`WHERE a = 'x' GROUP BY a`) +# should push through the aggregate and land on the scan's predicate. +# - A filter on an aggregate *result* (`HAVING count(b) > 5`) must NOT push +# through — it has to stay above the aggregate. +######## + +statement ok +COPY ( + SELECT * FROM (VALUES ('x', 'foo'), ('y', 'bar')) AS v(a, b) +) TO 'test_files/scratch/push_down_filter_regression/agg_filter_pushdown.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE agg_filter_pushdown (a VARCHAR, b VARCHAR) +STORED AS PARQUET +LOCATION 'test_files/scratch/push_down_filter_regression/agg_filter_pushdown.parquet'; + +# Filter on grouping column pushes down to the scan. +query TT +EXPLAIN SELECT a, count(b) FROM agg_filter_pushdown WHERE a = 'x' GROUP BY a; +---- +physical_plan +01)AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(agg_filter_pushdown.b)], ordering_mode=Sorted +02)--RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 +03)----AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(agg_filter_pushdown.b)], ordering_mode=Sorted +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_filter_pushdown.parquet]]}, projection=[a, b], file_type=parquet, predicate=a@0 = x, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= x AND x <= a_max@1, required_guarantees=[a in (x)] + +# Filter on aggregate result (HAVING) stays above the aggregate. +query TT +EXPLAIN SELECT a, count(b) AS cnt FROM agg_filter_pushdown GROUP BY a HAVING count(b) > 5; +---- +physical_plan +01)ProjectionExec: expr=[a@0 as a, count(agg_filter_pushdown.b)@1 as cnt] +02)--FilterExec: count(agg_filter_pushdown.b)@1 > 5 +03)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(agg_filter_pushdown.b)] +04)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 +05)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(agg_filter_pushdown.b)] +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_filter_pushdown.parquet]]}, projection=[a, b], file_type=parquet + +statement ok +drop table agg_filter_pushdown; + # Config reset -# The SLT runner sets `target_partitions` to 4 instead of using the default, so +# The SLT runner sets `target_partitions` to 4 instead of using the default, so # reset it explicitly. statement ok set datafusion.execution.target_partitions = 4;