diff --git a/.github/workflows/audit.yml b/.github/workflows/audit.yml index 37f8f8a52540..9ba1fe39d324 100644 --- a/.github/workflows/audit.yml +++ b/.github/workflows/audit.yml @@ -48,4 +48,4 @@ jobs: - name: Run audit check # Note: you can ignore specific RUSTSEC issues using the `--ignore` flag ,for example: # run: cargo audit --ignore RUSTSEC-2026-0001 - run: cargo audit + run: cargo audit --ignore RUSTSEC-2024-0436 --ignore RUSTSEC-2024-0014 diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 4703b55ecc0d..dde40cc0603b 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -38,10 +38,10 @@ mod tests { use crate::prelude::{ParquetReadOptions, SessionConfig, SessionContext}; use crate::test::object_store::local_unpartitioned_file; use arrow::array::{ - ArrayRef, AsArray, Date64Array, Int8Array, Int32Array, Int64Array, StringArray, - StringViewArray, StructArray, TimestampNanosecondArray, + ArrayRef, AsArray, Date64Array, DictionaryArray, Int8Array, Int32Array, + Int64Array, StringArray, StringViewArray, StructArray, TimestampNanosecondArray, }; - use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaBuilder}; + use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaBuilder, UInt16Type}; use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; use arrow_schema::{SchemaRef, TimeUnit}; @@ -2249,6 +2249,48 @@ mod tests { Ok(()) } + /// Tests that constant dictionary columns (where min == max in statistics) + /// are correctly handled. This reproduced a bug where the constant value + /// from statistics had type Utf8 but the schema expected Dictionary. + #[tokio::test] + async fn test_constant_dictionary_column_parquet() -> Result<()> { + let tmp_dir = TempDir::new()?; + let path = tmp_dir.path().to_str().unwrap().to_string() + "/test.parquet"; + + // Write parquet with dictionary column where all values are the same + let schema = Arc::new(Schema::new(vec![Field::new( + "status", + DataType::Dictionary(Box::new(DataType::UInt16), Box::new(DataType::Utf8)), + false, + )])); + let status: DictionaryArray = + vec!["active", "active"].into_iter().collect(); + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(status)])?; + let file = File::create(&path)?; + let props = WriterProperties::builder() + .set_statistics_enabled(parquet::file::properties::EnabledStatistics::Page) + .build(); + let mut writer = ArrowWriter::try_new(file, schema, Some(props))?; + writer.write(&batch)?; + writer.close()?; + + // Query the constant dictionary column + let ctx = SessionContext::new(); + ctx.register_parquet("t", &path, ParquetReadOptions::default()) + .await?; + let result = ctx.sql("SELECT status FROM t").await?.collect().await?; + + insta::assert_snapshot!(batches_to_string(&result),@r" + +--------+ + | status | + +--------+ + | active | + | active | + +--------+ + "); + Ok(()) + } + fn write_file(file: &String) { let struct_fields = Fields::from(vec![ Field::new("id", DataType::Int64, false), diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index d6357fdf6bc7..ad0d22e24af3 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -3600,3 +3600,88 @@ async fn test_hashjoin_dynamic_filter_pushdown_is_used() { ); } } + +/// Regression test for https://github.com/apache/datafusion/issues/20109 +#[tokio::test] +async fn test_filter_with_projection_pushdown() { + use arrow::array::{Int64Array, RecordBatch, StringArray}; + use datafusion_physical_plan::collect; + use datafusion_physical_plan::filter::FilterExec; + + // Create schema: [time, event, size] + let schema = Arc::new(Schema::new(vec![ + Field::new("time", DataType::Int64, false), + Field::new("event", DataType::Utf8, false), + Field::new("size", DataType::Int64, false), + ])); + + // Create sample data + let timestamps = vec![100i64, 200, 300, 400, 500]; + let events = vec!["Ingestion", "Ingestion", "Query", "Ingestion", "Query"]; + let sizes = vec![10i64, 20, 30, 40, 50]; + + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(Int64Array::from(timestamps)), + Arc::new(StringArray::from(events)), + Arc::new(Int64Array::from(sizes)), + ], + ) + .unwrap(); + + // Create data source + let memory_exec = datafusion_datasource::memory::MemorySourceConfig::try_new_exec( + &[vec![batch]], + schema.clone(), + None, + ) + .unwrap(); + + // First FilterExec: time < 350 with projection=[event@1, size@2] + let time_col = col("time", &memory_exec.schema()).unwrap(); + let time_filter = Arc::new(BinaryExpr::new( + time_col, + Operator::Lt, + Arc::new(Literal::new(ScalarValue::Int64(Some(350)))), + )); + let filter1 = Arc::new( + FilterExec::try_new(time_filter, memory_exec) + .unwrap() + .with_projection(Some(vec![1, 2])) + .unwrap(), + ); + + // Second FilterExec: event = 'Ingestion' with projection=[size@1] + let event_col = col("event", &filter1.schema()).unwrap(); + let event_filter = Arc::new(BinaryExpr::new( + event_col, + Operator::Eq, + Arc::new(Literal::new(ScalarValue::Utf8(Some( + "Ingestion".to_string(), + )))), + )); + let filter2 = Arc::new( + FilterExec::try_new(event_filter, filter1) + .unwrap() + .with_projection(Some(vec![1])) + .unwrap(), + ); + + // Apply filter pushdown optimization + let config = ConfigOptions::default(); + let optimized_plan = FilterPushdown::new() + .optimize(Arc::clone(&filter2) as Arc, &config) + .unwrap(); + + // Execute the optimized plan - this should not error + let ctx = SessionContext::new(); + let result = collect(optimized_plan, ctx.task_ctx()).await.unwrap(); + + // Verify results: should return rows where time < 350 AND event = 'Ingestion' + // That's rows with time=100,200 (both have event='Ingestion'), so sizes 10,20 + let expected = [ + "+------+", "| size |", "+------+", "| 10 |", "| 20 |", "+------+", + ]; + assert_batches_eq!(expected, &result); +} diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index ba53d079e305..00827edc2413 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -387,17 +387,17 @@ mod test { column_statistics: vec![ ColumnStatistics { null_count: Precision::Exact(0), - max_value: Precision::Exact(ScalarValue::Null), - min_value: Precision::Exact(ScalarValue::Null), - sum_value: Precision::Exact(ScalarValue::Null), + max_value: Precision::Exact(ScalarValue::Int32(None)), + min_value: Precision::Exact(ScalarValue::Int32(None)), + sum_value: Precision::Exact(ScalarValue::Int32(None)), distinct_count: Precision::Exact(0), byte_size: Precision::Exact(16), }, ColumnStatistics { null_count: Precision::Exact(0), - max_value: Precision::Exact(ScalarValue::Null), - min_value: Precision::Exact(ScalarValue::Null), - sum_value: Precision::Exact(ScalarValue::Null), + max_value: Precision::Exact(ScalarValue::Date32(None)), + min_value: Precision::Exact(ScalarValue::Date32(None)), + sum_value: Precision::Exact(ScalarValue::Date32(None)), distinct_count: Precision::Exact(0), byte_size: Precision::Exact(16), // 4 rows * 4 bytes (Date32) }, @@ -416,17 +416,17 @@ mod test { column_statistics: vec![ ColumnStatistics { null_count: Precision::Exact(0), - max_value: Precision::Exact(ScalarValue::Null), - min_value: Precision::Exact(ScalarValue::Null), - sum_value: Precision::Exact(ScalarValue::Null), + max_value: Precision::Exact(ScalarValue::Int32(None)), + min_value: Precision::Exact(ScalarValue::Int32(None)), + sum_value: Precision::Exact(ScalarValue::Int32(None)), distinct_count: Precision::Exact(0), byte_size: Precision::Exact(8), }, ColumnStatistics { null_count: Precision::Exact(0), - max_value: Precision::Exact(ScalarValue::Null), - min_value: Precision::Exact(ScalarValue::Null), - sum_value: Precision::Exact(ScalarValue::Null), + max_value: Precision::Exact(ScalarValue::Date32(None)), + min_value: Precision::Exact(ScalarValue::Date32(None)), + sum_value: Precision::Exact(ScalarValue::Date32(None)), distinct_count: Precision::Exact(0), byte_size: Precision::Exact(8), // 2 rows * 4 bytes (Date32) }, @@ -864,7 +864,7 @@ mod test { let plan_string = get_plan_string(&aggregate_exec_partial).swap_remove(0); assert_snapshot!( plan_string, - @"AggregateExec: mode=Partial, gby=[id@0 as id, 1 + id@0 as expr], aggr=[COUNT(c)], ordering_mode=Sorted" + @"AggregateExec: mode=Partial, gby=[id@0 as id, 1 + id@0 as expr], aggr=[COUNT(c)]" ); let p0_statistics = aggregate_exec_partial.partition_statistics(Some(0))?; diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 83bdf79c8fcc..719a3afc764f 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -696,6 +696,10 @@ fn constant_value_from_stats( && !min.is_null() && matches!(column_stats.null_count, Precision::Exact(0)) { + // Cast to the expected data type if needed (e.g., Utf8 -> Dictionary) + if min.data_type() != *data_type { + return min.cast_to(data_type).ok(); + } return Some(min.clone()); } diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index c8636343ccc5..facd12d7cbdf 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -665,7 +665,7 @@ impl DataSource for FileScanConfig { let schema = self.file_source.table_schema().table_schema(); let mut eq_properties = EquivalenceProperties::new_with_orderings( Arc::clone(schema), - self.output_ordering.clone(), + self.validated_output_ordering(), ) .with_constraints(self.constraints.clone()); @@ -853,6 +853,40 @@ impl DataSource for FileScanConfig { } impl FileScanConfig { + /// Returns only the output orderings that are validated against actual + /// file group statistics. + /// + /// For example, individual files may be ordered by `col1 ASC`, + /// but if we have files with these min/max statistics in a single partition / file group: + /// + /// - file1: min(col1) = 10, max(col1) = 20 + /// - file2: min(col1) = 5, max(col1) = 15 + /// + /// Because reading file1 followed by file2 would produce out-of-order output (there is overlap + /// in the ranges), we cannot retain `col1 ASC` as a valid output ordering. + /// + /// Similarly this would not be a valid order (non-overlapping ranges but not ordered): + /// + /// - file1: min(col1) = 20, max(col1) = 30 + /// - file2: min(col1) = 10, max(col1) = 15 + /// + /// On the other hand if we had: + /// + /// - file1: min(col1) = 5, max(col1) = 15 + /// - file2: min(col1) = 16, max(col1) = 25 + /// + /// Then we know that reading file1 followed by file2 will produce ordered output, + /// so `col1 ASC` would be retained. + /// + /// Note that we are checking for ordering *within* *each* file group / partition, + /// files in different partitions are read independently and do not affect each other's ordering. + /// Merging of the multiple partition streams into a single ordered stream is handled + /// upstream e.g. by `SortPreservingMergeExec`. + fn validated_output_ordering(&self) -> Vec { + let schema = self.file_source.table_schema().table_schema(); + validate_orderings(&self.output_ordering, schema, &self.file_groups, None) + } + /// Get the file schema (schema of the files without partition columns) pub fn file_schema(&self) -> &SchemaRef { self.file_source.table_schema().file_schema() @@ -1202,6 +1236,51 @@ fn ordered_column_indices_from_projection( .collect::>>() } +/// Check whether a given ordering is valid for all file groups by verifying +/// that files within each group are sorted according to their min/max statistics. +/// +/// For single-file (or empty) groups, the ordering is trivially valid. +/// For multi-file groups, we check that the min/max statistics for the sort +/// columns are in order and non-overlapping (or touching at boundaries). +/// +/// `projection` maps projected column indices back to table-schema indices +/// when validating after projection; pass `None` when validating at +/// table-schema level. +fn is_ordering_valid_for_file_groups( + file_groups: &[FileGroup], + ordering: &LexOrdering, + schema: &SchemaRef, + projection: Option<&[usize]>, +) -> bool { + file_groups.iter().all(|group| { + if group.len() <= 1 { + return true; // single-file groups are trivially sorted + } + match MinMaxStatistics::new_from_files(ordering, schema, projection, group.iter()) + { + Ok(stats) => stats.is_sorted(), + Err(_) => false, // can't prove sorted → reject + } + }) +} + +/// Filters orderings to retain only those valid for all file groups, +/// verified via min/max statistics. +fn validate_orderings( + orderings: &[LexOrdering], + schema: &SchemaRef, + file_groups: &[FileGroup], + projection: Option<&[usize]>, +) -> Vec { + orderings + .iter() + .filter(|ordering| { + is_ordering_valid_for_file_groups(file_groups, ordering, schema, projection) + }) + .cloned() + .collect() +} + /// The various listing tables does not attempt to read all files /// concurrently, instead they will read files in sequence within a /// partition. This is an important property as it allows plans to @@ -1268,52 +1347,47 @@ fn get_projected_output_ordering( let projected_orderings = project_orderings(&base_config.output_ordering, projected_schema); - let mut all_orderings = vec![]; - for new_ordering in projected_orderings { - // Check if any file groups are not sorted - if base_config.file_groups.iter().any(|group| { - if group.len() <= 1 { - // File groups with <= 1 files are always sorted - return false; - } - - let Some(indices) = base_config - .file_source - .projection() - .as_ref() - .map(|p| ordered_column_indices_from_projection(p)) - else { - // Can't determine if ordered without a simple projection - return true; - }; - - let statistics = match MinMaxStatistics::new_from_files( - &new_ordering, + let indices = base_config + .file_source + .projection() + .as_ref() + .map(|p| ordered_column_indices_from_projection(p)); + + match indices { + Some(Some(indices)) => { + // Simple column projection — validate with statistics + validate_orderings( + &projected_orderings, projected_schema, - indices.as_deref(), - group.iter(), - ) { - Ok(statistics) => statistics, - Err(e) => { - log::trace!("Error fetching statistics for file group: {e}"); - // we can't prove that it's ordered, so we have to reject it - return true; - } - }; - - !statistics.is_sorted() - }) { - debug!( - "Skipping specified output ordering {:?}. \ - Some file groups couldn't be determined to be sorted: {:?}", - base_config.output_ordering[0], base_config.file_groups - ); - continue; + &base_config.file_groups, + Some(indices.as_slice()), + ) + } + None => { + // No projection — validate with statistics (no remapping needed) + validate_orderings( + &projected_orderings, + projected_schema, + &base_config.file_groups, + None, + ) + } + Some(None) => { + // Complex projection (expressions, not simple columns) — can't + // determine column indices for statistics. Still valid if all + // file groups have at most one file. + if base_config.file_groups.iter().all(|g| g.len() <= 1) { + projected_orderings + } else { + debug!( + "Skipping specified output orderings. \ + Some file groups couldn't be determined to be sorted: {:?}", + base_config.file_groups + ); + vec![] + } } - - all_orderings.push(new_ordering); } - all_orderings } /// Convert type to a type suitable for use as a `ListingTable` diff --git a/datafusion/datasource/src/statistics.rs b/datafusion/datasource/src/statistics.rs index 2f34ca032e13..b1a56e096c22 100644 --- a/datafusion/datasource/src/statistics.rs +++ b/datafusion/datasource/src/statistics.rs @@ -266,11 +266,12 @@ impl MinMaxStatistics { } /// Check if the min/max statistics are in order and non-overlapping + /// (or touching at boundaries) pub fn is_sorted(&self) -> bool { self.max_by_sort_order .iter() .zip(self.min_by_sort_order.iter().skip(1)) - .all(|(max, next_min)| max < next_min) + .all(|(max, next_min)| max <= next_min) } } diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index f5f49c643c28..32a88ab8cf31 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -260,7 +260,18 @@ fn coerce_exprs_for_schema( } #[expect(deprecated)] Expr::Wildcard { .. } => Ok(expr), - _ => expr.cast_to(new_type, src_schema), + _ => { + match expr { + // maintain the original name when casting a column, to avoid the + // tablename being added to it when not explicitly set by the query + // (see: https://github.com/apache/datafusion/issues/18818) + Expr::Column(ref column) => { + let name = column.name().to_owned(); + Ok(expr.cast_to(new_type, src_schema)?.alias(name)) + } + _ => Ok(expr.cast_to(new_type, src_schema)?), + } + } } } else { Ok(expr) diff --git a/datafusion/ffi/src/session/mod.rs b/datafusion/ffi/src/session/mod.rs index aa910abb9149..6b8664a43749 100644 --- a/datafusion/ffi/src/session/mod.rs +++ b/datafusion/ffi/src/session/mod.rs @@ -26,7 +26,7 @@ use arrow_schema::SchemaRef; use arrow_schema::ffi::FFI_ArrowSchema; use async_ffi::{FfiFuture, FutureExt}; use async_trait::async_trait; -use datafusion_common::config::{ConfigOptions, TableOptions}; +use datafusion_common::config::{ConfigFileType, ConfigOptions, TableOptions}; use datafusion_common::{DFSchema, DataFusionError}; use datafusion_execution::TaskContext; use datafusion_execution::config::SessionConfig; @@ -240,12 +240,30 @@ unsafe extern "C" fn window_functions_fn_wrapper( .collect() } -fn table_options_to_rhash(options: &TableOptions) -> RHashMap { - options +fn table_options_to_rhash(mut options: TableOptions) -> RHashMap { + // It is important that we mutate options here and set current format + // to None so that when we call `entries()` we get ALL format entries. + // We will pass current_format as a special case and strip it on the + // other side of the boundary. + let current_format = options.current_format.take(); + let mut options: HashMap = options .entries() .into_iter() .filter_map(|entry| entry.value.map(|v| (entry.key.into(), v.into()))) - .collect() + .collect(); + if let Some(current_format) = current_format { + options.insert( + "datafusion_ffi.table_current_format".into(), + match current_format { + ConfigFileType::JSON => "json", + ConfigFileType::PARQUET => "parquet", + ConfigFileType::CSV => "csv", + } + .into(), + ); + } + + options.into() } unsafe extern "C" fn table_options_fn_wrapper( @@ -253,7 +271,7 @@ unsafe extern "C" fn table_options_fn_wrapper( ) -> RHashMap { let session = session.inner(); let table_options = session.table_options(); - table_options_to_rhash(table_options) + table_options_to_rhash(table_options.clone()) } unsafe extern "C" fn default_table_options_fn_wrapper( @@ -262,7 +280,7 @@ unsafe extern "C" fn default_table_options_fn_wrapper( let session = session.inner(); let table_options = session.default_table_options(); - table_options_to_rhash(&table_options) + table_options_to_rhash(table_options) } unsafe extern "C" fn task_ctx_fn_wrapper(session: &FFI_SessionRef) -> FFI_TaskContext { @@ -438,15 +456,70 @@ impl Clone for FFI_SessionRef { } fn table_options_from_rhashmap(options: RHashMap) -> TableOptions { - let options = options + let mut options: HashMap = options .into_iter() .map(|kv_pair| (kv_pair.0.into_string(), kv_pair.1.into_string())) .collect(); + let current_format = options.remove("datafusion_ffi.table_current_format"); + + let mut table_options = TableOptions::default(); + let formats = [ + ConfigFileType::CSV, + ConfigFileType::JSON, + ConfigFileType::PARQUET, + ]; + for format in formats { + // It is imperative that if new enum variants are added below that they be + // included in the formats list above and in the extension check below. + let format_name = match &format { + ConfigFileType::CSV => "csv", + ConfigFileType::PARQUET => "parquet", + ConfigFileType::JSON => "json", + }; + let format_options: HashMap = options + .iter() + .filter_map(|(k, v)| { + let (prefix, key) = k.split_once(".")?; + if prefix == format_name { + Some((format!("format.{key}"), v.to_owned())) + } else { + None + } + }) + .collect(); + if !format_options.is_empty() { + table_options.current_format = Some(format.clone()); + table_options + .alter_with_string_hash_map(&format_options) + .unwrap_or_else(|err| log::warn!("Error parsing table options: {err}")); + } + } + + let extension_options: HashMap = options + .iter() + .filter_map(|(k, v)| { + let (prefix, _) = k.split_once(".")?; + if !["json", "parquet", "csv"].contains(&prefix) { + Some((k.to_owned(), v.to_owned())) + } else { + None + } + }) + .collect(); + if !extension_options.is_empty() { + table_options + .alter_with_string_hash_map(&extension_options) + .unwrap_or_else(|err| log::warn!("Error parsing table options: {err}")); + } - TableOptions::from_string_hash_map(&options).unwrap_or_else(|err| { - log::warn!("Error parsing default table options: {err}"); - TableOptions::default() - }) + table_options.current_format = + current_format.and_then(|format| match format.as_str() { + "csv" => Some(ConfigFileType::CSV), + "parquet" => Some(ConfigFileType::PARQUET), + "json" => Some(ConfigFileType::JSON), + _ => None, + }); + table_options } #[async_trait] @@ -556,6 +629,7 @@ mod tests { use std::sync::Arc; use arrow_schema::{DataType, Field, Schema}; + use datafusion::execution::SessionStateBuilder; use datafusion_common::DataFusionError; use datafusion_expr::col; use datafusion_expr::registry::FunctionRegistry; @@ -566,7 +640,16 @@ mod tests { #[tokio::test] async fn test_ffi_session() -> Result<(), DataFusionError> { let (ctx, task_ctx_provider) = crate::util::tests::test_session_and_ctx(); - let state = ctx.state(); + let mut table_options = TableOptions::default(); + table_options.csv.has_header = Some(true); + table_options.json.schema_infer_max_rec = Some(10); + table_options.parquet.global.coerce_int96 = Some("123456789".into()); + table_options.current_format = Some(ConfigFileType::JSON); + + let state = SessionStateBuilder::new_from_existing(ctx.state()) + .with_table_options(table_options) + .build(); + let logical_codec = FFI_LogicalExtensionCodec::new( Arc::new(DefaultLogicalExtensionCodec {}), None, diff --git a/datafusion/functions-nested/src/sort.rs b/datafusion/functions-nested/src/sort.rs index ba2da0f760ee..cbe101f111b2 100644 --- a/datafusion/functions-nested/src/sort.rs +++ b/datafusion/functions-nested/src/sort.rs @@ -18,16 +18,14 @@ //! [`ScalarUDFImpl`] definitions for array_sort function. use crate::utils::make_scalar_function; -use arrow::array::{ - Array, ArrayRef, GenericListArray, NullBufferBuilder, OffsetSizeTrait, new_null_array, -}; +use arrow::array::{Array, ArrayRef, GenericListArray, OffsetSizeTrait, new_null_array}; use arrow::buffer::OffsetBuffer; use arrow::compute::SortColumn; use arrow::datatypes::{DataType, FieldRef}; use arrow::{compute, compute::SortOptions}; use datafusion_common::cast::{as_large_list_array, as_list_array, as_string_array}; use datafusion_common::utils::ListCoercion; -use datafusion_common::{Result, exec_err, plan_err}; +use datafusion_common::{Result, exec_err}; use datafusion_expr::{ ArrayFunctionArgument, ArrayFunctionSignature, ColumnarValue, Documentation, ScalarUDFImpl, Signature, TypeSignature, Volatility, @@ -134,18 +132,7 @@ impl ScalarUDFImpl for ArraySort { } fn return_type(&self, arg_types: &[DataType]) -> Result { - match &arg_types[0] { - DataType::Null => Ok(DataType::Null), - DataType::List(field) => { - Ok(DataType::new_list(field.data_type().clone(), true)) - } - DataType::LargeList(field) => { - Ok(DataType::new_large_list(field.data_type().clone(), true)) - } - arg_type => { - plan_err!("{} does not support type {arg_type}", self.name()) - } - } + Ok(arg_types[0].clone()) } fn invoke_with_args( @@ -206,11 +193,11 @@ fn array_sort_inner(args: &[ArrayRef]) -> Result { } DataType::List(field) => { let array = as_list_array(&args[0])?; - array_sort_generic(array, field, sort_options) + array_sort_generic(array, Arc::clone(field), sort_options) } DataType::LargeList(field) => { let array = as_large_list_array(&args[0])?; - array_sort_generic(array, field, sort_options) + array_sort_generic(array, Arc::clone(field), sort_options) } // Signature should prevent this arm ever occurring _ => exec_err!("array_sort expects list for first argument"), @@ -219,18 +206,16 @@ fn array_sort_inner(args: &[ArrayRef]) -> Result { fn array_sort_generic( list_array: &GenericListArray, - field: &FieldRef, + field: FieldRef, sort_options: Option, ) -> Result { let row_count = list_array.len(); let mut array_lengths = vec![]; let mut arrays = vec![]; - let mut valid = NullBufferBuilder::new(row_count); for i in 0..row_count { if list_array.is_null(i) { array_lengths.push(0); - valid.append_null(); } else { let arr_ref = list_array.value(i); @@ -253,25 +238,22 @@ fn array_sort_generic( }; array_lengths.push(sorted_array.len()); arrays.push(sorted_array); - valid.append_non_null(); } } - let buffer = valid.finish(); - let elements = arrays .iter() .map(|a| a.as_ref()) .collect::>(); let list_arr = if elements.is_empty() { - GenericListArray::::new_null(Arc::clone(field), row_count) + GenericListArray::::new_null(field, row_count) } else { GenericListArray::::new( - Arc::clone(field), + field, OffsetBuffer::from_lengths(array_lengths), Arc::new(compute::concat(elements.as_slice())?), - buffer, + list_array.nulls().cloned(), ) }; Ok(Arc::new(list_arr)) diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 0e3d0c7b0f23..02395c76bdd9 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -1441,7 +1441,7 @@ mod test { true, plan.clone(), @r" - Projection: CAST(a AS LargeUtf8) + Projection: CAST(a AS LargeUtf8) AS a EmptyRelation: rows=0 " )?; @@ -1477,7 +1477,7 @@ mod test { true, plan.clone(), @r" - Projection: CAST(a AS LargeUtf8) + Projection: CAST(a AS LargeUtf8) AS a EmptyRelation: rows=0 " )?; @@ -1507,7 +1507,7 @@ mod test { true, sort_plan.clone(), @r" - Projection: CAST(a AS LargeUtf8) + Projection: CAST(a AS LargeUtf8) AS a Sort: a ASC NULLS FIRST Projection: a EmptyRelation: rows=0 @@ -1536,7 +1536,7 @@ mod test { true, plan.clone(), @r" - Projection: CAST(a AS LargeUtf8) + Projection: CAST(a AS LargeUtf8) AS a Sort: a ASC NULLS FIRST Projection: a EmptyRelation: rows=0 @@ -1572,7 +1572,7 @@ mod test { true, plan.clone(), @r" - Projection: CAST(a AS LargeBinary) + Projection: CAST(a AS LargeBinary) AS a EmptyRelation: rows=0 " )?; @@ -1629,7 +1629,7 @@ mod test { true, sort_plan.clone(), @r" - Projection: CAST(a AS LargeBinary) + Projection: CAST(a AS LargeBinary) AS a Sort: a ASC NULLS FIRST Projection: a EmptyRelation: rows=0 @@ -1660,7 +1660,7 @@ mod test { true, plan.clone(), @r" - Projection: CAST(a AS LargeBinary) + Projection: CAST(a AS LargeBinary) AS a Sort: a ASC NULLS FIRST Projection: a EmptyRelation: rows=0 diff --git a/datafusion/optimizer/tests/optimizer_integration.rs b/datafusion/optimizer/tests/optimizer_integration.rs index c8dceed737f3..fd4991c24413 100644 --- a/datafusion/optimizer/tests/optimizer_integration.rs +++ b/datafusion/optimizer/tests/optimizer_integration.rs @@ -538,14 +538,15 @@ fn recursive_cte_projection_pushdown() -> Result<()> { // columns from the base table and recursive table, eliminating unused columns assert_snapshot!( format!("{plan}"), - @r#"SubqueryAlias: nodes - RecursiveQuery: is_distinct=false - Projection: test.col_int32 AS id - TableScan: test projection=[col_int32] - Projection: CAST(CAST(nodes.id AS Int64) + Int64(1) AS Int32) - Filter: nodes.id < Int32(3) - TableScan: nodes projection=[id] -"# + @r" + SubqueryAlias: nodes + RecursiveQuery: is_distinct=false + Projection: test.col_int32 AS id + TableScan: test projection=[col_int32] + Projection: CAST(CAST(nodes.id AS Int64) + Int64(1) AS Int32) + Filter: nodes.id < Int32(3) + TableScan: nodes projection=[id] + " ); Ok(()) } @@ -561,14 +562,16 @@ fn recursive_cte_with_aliased_self_reference() -> Result<()> { assert_snapshot!( format!("{plan}"), - @r#"SubqueryAlias: nodes - RecursiveQuery: is_distinct=false - Projection: test.col_int32 AS id - TableScan: test projection=[col_int32] - Projection: CAST(CAST(child.id AS Int64) + Int64(1) AS Int32) - SubqueryAlias: child - Filter: nodes.id < Int32(3) - TableScan: nodes projection=[id]"#, + @r" + SubqueryAlias: nodes + RecursiveQuery: is_distinct=false + Projection: test.col_int32 AS id + TableScan: test projection=[col_int32] + Projection: CAST(CAST(child.id AS Int64) + Int64(1) AS Int32) + SubqueryAlias: child + Filter: nodes.id < Int32(3) + TableScan: nodes projection=[id] + ", ); Ok(()) } @@ -621,15 +624,16 @@ fn recursive_cte_projection_pushdown_baseline() -> Result<()> { // and only the needed column is selected from the recursive table assert_snapshot!( format!("{plan}"), - @r#"SubqueryAlias: countdown - RecursiveQuery: is_distinct=false - Projection: test.col_int32 AS n - Filter: test.col_int32 = Int32(5) - TableScan: test projection=[col_int32] - Projection: CAST(CAST(countdown.n AS Int64) - Int64(1) AS Int32) - Filter: countdown.n > Int32(1) - TableScan: countdown projection=[n] -"# + @r" + SubqueryAlias: countdown + RecursiveQuery: is_distinct=false + Projection: test.col_int32 AS n + Filter: test.col_int32 = Int32(5) + TableScan: test projection=[col_int32] + Projection: CAST(CAST(countdown.n AS Int64) - Int64(1) AS Int32) + Filter: countdown.n > Int32(1) + TableScan: countdown projection=[n] + " ); Ok(()) } diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index 5c2f1adcd0cf..379bd7edf7e6 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -98,11 +98,18 @@ impl StaticFilter for ArrayStaticFilter { )); } + // Unwrap dictionary-encoded needles when the value type matches + // in_array, evaluating against the dictionary values and mapping + // back via keys. downcast_dictionary_array! { v => { - let values_contains = self.contains(v.values().as_ref(), negated)?; - let result = take(&values_contains, v.keys(), None)?; - return Ok(downcast_array(result.as_ref())) + // Only unwrap when the haystack (in_array) type matches + // the dictionary value type + if v.values().data_type() == self.in_array.data_type() { + let values_contains = self.contains(v.values().as_ref(), negated)?; + let result = take(&values_contains, v.keys(), None)?; + return Ok(downcast_array(result.as_ref())); + } } _ => {} } @@ -3507,4 +3514,536 @@ mod tests { Ok(()) } + /// Helper: creates an InListExpr with `static_filter = None` + /// to force the column-reference evaluation path. + fn make_in_list_with_columns( + expr: Arc, + list: Vec>, + negated: bool, + ) -> Arc { + Arc::new(InListExpr::new(expr, list, negated, None)) + } + + #[test] + fn test_in_list_with_columns_int32_scalars() -> Result<()> { + // Column-reference path with scalar literals (bypassing static filter) + let schema = Schema::new(vec![Field::new("a", DataType::Int32, true)]); + let col_a = col("a", &schema)?; + let batch = RecordBatch::try_new( + Arc::new(schema), + vec![Arc::new(Int32Array::from(vec![ + Some(1), + Some(2), + Some(3), + None, + ]))], + )?; + + let list = vec![ + lit(ScalarValue::Int32(Some(1))), + lit(ScalarValue::Int32(Some(3))), + ]; + let expr = make_in_list_with_columns(col_a, list, false); + + let result = expr.evaluate(&batch)?.into_array(batch.num_rows())?; + let result = as_boolean_array(&result); + assert_eq!( + result, + &BooleanArray::from(vec![Some(true), Some(false), Some(true), None,]) + ); + Ok(()) + } + + #[test] + fn test_in_list_with_columns_int32_column_refs() -> Result<()> { + // IN list with column references + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ]); + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![ + Arc::new(Int32Array::from(vec![Some(1), Some(2), Some(3), None])), + Arc::new(Int32Array::from(vec![ + Some(1), + Some(99), + Some(99), + Some(99), + ])), + Arc::new(Int32Array::from(vec![Some(99), Some(99), Some(3), None])), + ], + )?; + + let col_a = col("a", &schema)?; + let list = vec![col("b", &schema)?, col("c", &schema)?]; + let expr = make_in_list_with_columns(col_a, list, false); + + let result = expr.evaluate(&batch)?.into_array(batch.num_rows())?; + let result = as_boolean_array(&result); + // row 0: 1 IN (1, 99) → true + // row 1: 2 IN (99, 99) → false + // row 2: 3 IN (99, 3) → true + // row 3: NULL IN (99, NULL) → NULL + assert_eq!( + result, + &BooleanArray::from(vec![Some(true), Some(false), Some(true), None,]) + ); + Ok(()) + } + + #[test] + fn test_in_list_with_columns_utf8_column_refs() -> Result<()> { + // IN list with Utf8 column references + let schema = Schema::new(vec![ + Field::new("a", DataType::Utf8, false), + Field::new("b", DataType::Utf8, false), + ]); + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![ + Arc::new(StringArray::from(vec!["x", "y", "z"])), + Arc::new(StringArray::from(vec!["x", "x", "z"])), + ], + )?; + + let col_a = col("a", &schema)?; + let list = vec![col("b", &schema)?]; + let expr = make_in_list_with_columns(col_a, list, false); + + let result = expr.evaluate(&batch)?.into_array(batch.num_rows())?; + let result = as_boolean_array(&result); + // row 0: "x" IN ("x") → true + // row 1: "y" IN ("x") → false + // row 2: "z" IN ("z") → true + assert_eq!(result, &BooleanArray::from(vec![true, false, true])); + Ok(()) + } + + #[test] + fn test_in_list_with_columns_negated() -> Result<()> { + // NOT IN with column references + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + ]); + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3])), + Arc::new(Int32Array::from(vec![1, 99, 3])), + ], + )?; + + let col_a = col("a", &schema)?; + let list = vec![col("b", &schema)?]; + let expr = make_in_list_with_columns(col_a, list, true); + + let result = expr.evaluate(&batch)?.into_array(batch.num_rows())?; + let result = as_boolean_array(&result); + // row 0: 1 NOT IN (1) → false + // row 1: 2 NOT IN (99) → true + // row 2: 3 NOT IN (3) → false + assert_eq!(result, &BooleanArray::from(vec![false, true, false])); + Ok(()) + } + + #[test] + fn test_in_list_with_columns_null_in_list() -> Result<()> { + // IN list with NULL scalar (column-reference path) + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + let col_a = col("a", &schema)?; + let batch = RecordBatch::try_new( + Arc::new(schema), + vec![Arc::new(Int32Array::from(vec![1, 2]))], + )?; + + let list = vec![ + lit(ScalarValue::Int32(None)), + lit(ScalarValue::Int32(Some(1))), + ]; + let expr = make_in_list_with_columns(col_a, list, false); + + let result = expr.evaluate(&batch)?.into_array(batch.num_rows())?; + let result = as_boolean_array(&result); + // row 0: 1 IN (NULL, 1) → true (true OR null = true) + // row 1: 2 IN (NULL, 1) → NULL (false OR null = null) + assert_eq!(result, &BooleanArray::from(vec![Some(true), None])); + Ok(()) + } + + #[test] + fn test_in_list_with_columns_float_nan() -> Result<()> { + // Verify NaN == NaN is true in the column-reference path + // (consistent with Arrow's totalOrder semantics) + let schema = Schema::new(vec![ + Field::new("a", DataType::Float64, false), + Field::new("b", DataType::Float64, false), + ]); + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![ + Arc::new(Float64Array::from(vec![f64::NAN, 1.0, f64::NAN])), + Arc::new(Float64Array::from(vec![f64::NAN, 2.0, 0.0])), + ], + )?; + + let col_a = col("a", &schema)?; + let list = vec![col("b", &schema)?]; + let expr = make_in_list_with_columns(col_a, list, false); + + let result = expr.evaluate(&batch)?.into_array(batch.num_rows())?; + let result = as_boolean_array(&result); + // row 0: NaN IN (NaN) → true + // row 1: 1.0 IN (2.0) → false + // row 2: NaN IN (0.0) → false + assert_eq!(result, &BooleanArray::from(vec![true, false, false])); + Ok(()) + } + + /// Tests that short-circuit evaluation produces correct results. + /// When all rows match after the first list item, remaining items + /// should be skipped without affecting correctness. + #[test] + fn test_in_list_with_columns_short_circuit() -> Result<()> { + // a IN (b, c) where b already matches every row of a + // The short-circuit should skip evaluating c + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + Field::new("c", DataType::Int32, false), + ]); + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3])), + Arc::new(Int32Array::from(vec![1, 2, 3])), // b == a for all rows + Arc::new(Int32Array::from(vec![99, 99, 99])), + ], + )?; + + let col_a = col("a", &schema)?; + let list = vec![col("b", &schema)?, col("c", &schema)?]; + let expr = make_in_list_with_columns(col_a, list, false); + + let result = expr.evaluate(&batch)?.into_array(batch.num_rows())?; + let result = as_boolean_array(&result); + assert_eq!(result, &BooleanArray::from(vec![true, true, true])); + Ok(()) + } + + /// Short-circuit must NOT skip when nulls are present (three-valued logic). + /// Even if all non-null values are true, null rows keep the result as null. + #[test] + fn test_in_list_with_columns_short_circuit_with_nulls() -> Result<()> { + // a IN (b, c) where a has nulls + // Even if b matches all non-null rows, result should preserve nulls + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, false), + Field::new("c", DataType::Int32, false), + ]); + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![ + Arc::new(Int32Array::from(vec![Some(1), None, Some(3)])), + Arc::new(Int32Array::from(vec![1, 2, 3])), // matches non-null rows + Arc::new(Int32Array::from(vec![99, 99, 99])), + ], + )?; + + let col_a = col("a", &schema)?; + let list = vec![col("b", &schema)?, col("c", &schema)?]; + let expr = make_in_list_with_columns(col_a, list, false); + + let result = expr.evaluate(&batch)?.into_array(batch.num_rows())?; + let result = as_boolean_array(&result); + // row 0: 1 IN (1, 99) → true + // row 1: NULL IN (2, 99) → NULL + // row 2: 3 IN (3, 99) → true + assert_eq!( + result, + &BooleanArray::from(vec![Some(true), None, Some(true)]) + ); + Ok(()) + } + + /// Tests the make_comparator + collect_bool fallback path using + /// struct column references (nested types don't support arrow_eq). + #[test] + fn test_in_list_with_columns_struct() -> Result<()> { + let struct_fields = Fields::from(vec![ + Field::new("x", DataType::Int32, false), + Field::new("y", DataType::Utf8, false), + ]); + let struct_dt = DataType::Struct(struct_fields.clone()); + + let schema = Schema::new(vec![ + Field::new("a", struct_dt.clone(), true), + Field::new("b", struct_dt.clone(), false), + Field::new("c", struct_dt.clone(), false), + ]); + + // a: [{1,"a"}, {2,"b"}, NULL, {4,"d"}] + // b: [{1,"a"}, {9,"z"}, {3,"c"}, {4,"d"}] + // c: [{9,"z"}, {2,"b"}, {9,"z"}, {9,"z"}] + let a = Arc::new(StructArray::new( + struct_fields.clone(), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3, 4])), + Arc::new(StringArray::from(vec!["a", "b", "c", "d"])), + ], + Some(vec![true, true, false, true].into()), + )); + let b = Arc::new(StructArray::new( + struct_fields.clone(), + vec![ + Arc::new(Int32Array::from(vec![1, 9, 3, 4])), + Arc::new(StringArray::from(vec!["a", "z", "c", "d"])), + ], + None, + )); + let c = Arc::new(StructArray::new( + struct_fields.clone(), + vec![ + Arc::new(Int32Array::from(vec![9, 2, 9, 9])), + Arc::new(StringArray::from(vec!["z", "b", "z", "z"])), + ], + None, + )); + + let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![a, b, c])?; + + let col_a = col("a", &schema)?; + let list = vec![col("b", &schema)?, col("c", &schema)?]; + let expr = make_in_list_with_columns(col_a, list, false); + + let result = expr.evaluate(&batch)?.into_array(batch.num_rows())?; + let result = as_boolean_array(&result); + // row 0: {1,"a"} IN ({1,"a"}, {9,"z"}) → true (matches b) + // row 1: {2,"b"} IN ({9,"z"}, {2,"b"}) → true (matches c) + // row 2: NULL IN ({3,"c"}, {9,"z"}) → NULL + // row 3: {4,"d"} IN ({4,"d"}, {9,"z"}) → true (matches b) + assert_eq!( + result, + &BooleanArray::from(vec![Some(true), Some(true), None, Some(true)]) + ); + + // Also test NOT IN + let col_a = col("a", &schema)?; + let list = vec![col("b", &schema)?, col("c", &schema)?]; + let expr = make_in_list_with_columns(col_a, list, true); + + let result = expr.evaluate(&batch)?.into_array(batch.num_rows())?; + let result = as_boolean_array(&result); + // row 0: {1,"a"} NOT IN ({1,"a"}, {9,"z"}) → false + // row 1: {2,"b"} NOT IN ({9,"z"}, {2,"b"}) → false + // row 2: NULL NOT IN ({3,"c"}, {9,"z"}) → NULL + // row 3: {4,"d"} NOT IN ({4,"d"}, {9,"z"}) → false + assert_eq!( + result, + &BooleanArray::from(vec![Some(false), Some(false), None, Some(false)]) + ); + Ok(()) + } + + // ----------------------------------------------------------------------- + // Tests for try_new_from_array: evaluates `needle IN in_array`. + // + // This exercises the code path used by HashJoin dynamic filter pushdown, + // where in_array is built directly from the join's build-side arrays. + // Unlike try_new (used by SQL IN expressions), which always produces a + // non-Dictionary in_array because evaluate_list() flattens Dictionary + // scalars, try_new_from_array passes the array directly and can produce + // a Dictionary in_array. + // ----------------------------------------------------------------------- + + fn wrap_in_dict(array: ArrayRef) -> ArrayRef { + let keys = Int32Array::from((0..array.len() as i32).collect::>()); + Arc::new(DictionaryArray::new(keys, array)) + } + + /// Evaluates `needle IN in_array` via try_new_from_array, the same + /// path used by HashJoin dynamic filter pushdown (not the SQL literal + /// IN path which goes through try_new). + fn eval_in_list_from_array( + needle: ArrayRef, + in_array: ArrayRef, + ) -> Result { + let schema = + Schema::new(vec![Field::new("a", needle.data_type().clone(), false)]); + let col_a = col("a", &schema)?; + let expr = Arc::new(InListExpr::try_new_from_array(col_a, in_array, false)?) + as Arc; + let batch = RecordBatch::try_new(Arc::new(schema), vec![needle])?; + let result = expr.evaluate(&batch)?.into_array(batch.num_rows())?; + Ok(as_boolean_array(&result).clone()) + } + + #[test] + fn test_in_list_from_array_type_combinations() -> Result<()> { + use arrow::compute::cast; + + // All cases: needle[0] and needle[2] match, needle[1] does not. + let expected = BooleanArray::from(vec![Some(true), Some(false), Some(true)]); + + // Base arrays cast to each target type + let base_in = Arc::new(Int64Array::from(vec![1i64, 2, 3])) as ArrayRef; + let base_needle = Arc::new(Int64Array::from(vec![1i64, 4, 2])) as ArrayRef; + + // Test all specializations in instantiate_static_filter + let primitive_types = vec![ + DataType::Int8, + DataType::Int16, + DataType::Int32, + DataType::Int64, + DataType::UInt8, + DataType::UInt16, + DataType::UInt32, + DataType::UInt64, + DataType::Float32, + DataType::Float64, + ]; + + for dt in &primitive_types { + let in_array = cast(&base_in, dt)?; + let needle = cast(&base_needle, dt)?; + + // T in_array, T needle + assert_eq!( + expected, + eval_in_list_from_array(Arc::clone(&needle), Arc::clone(&in_array))?, + "same-type failed for {dt:?}" + ); + + // T in_array, Dict(Int32, T) needle + assert_eq!( + expected, + eval_in_list_from_array(wrap_in_dict(needle), in_array)?, + "dict-needle failed for {dt:?}" + ); + } + + // Utf8 (falls through to ArrayStaticFilter) + let utf8_in = Arc::new(StringArray::from(vec!["a", "b", "c"])) as ArrayRef; + let utf8_needle = Arc::new(StringArray::from(vec!["a", "d", "b"])) as ArrayRef; + + // Utf8 in_array, Utf8 needle + assert_eq!( + expected, + eval_in_list_from_array(Arc::clone(&utf8_needle), Arc::clone(&utf8_in),)? + ); + + // Utf8 in_array, Dict(Utf8) needle + assert_eq!( + expected, + eval_in_list_from_array( + wrap_in_dict(Arc::clone(&utf8_needle)), + Arc::clone(&utf8_in), + )? + ); + + // Dict(Utf8) in_array, Dict(Utf8) needle: the #20937 bug + assert_eq!( + expected, + eval_in_list_from_array( + wrap_in_dict(Arc::clone(&utf8_needle)), + wrap_in_dict(Arc::clone(&utf8_in)), + )? + ); + + // Struct in_array, Struct needle: multi-column join + let struct_fields = Fields::from(vec![ + Field::new("c0", DataType::Utf8, true), + Field::new("c1", DataType::Int64, true), + ]); + let make_struct = |c0: ArrayRef, c1: ArrayRef| -> ArrayRef { + let pairs: Vec<(FieldRef, ArrayRef)> = + struct_fields.iter().cloned().zip([c0, c1]).collect(); + Arc::new(StructArray::from(pairs)) + }; + assert_eq!( + expected, + eval_in_list_from_array( + make_struct( + Arc::clone(&utf8_needle), + Arc::new(Int64Array::from(vec![1, 4, 2])), + ), + make_struct( + Arc::clone(&utf8_in), + Arc::new(Int64Array::from(vec![1, 2, 3])), + ), + )? + ); + + // Struct with Dict fields: multi-column Dict join + let dict_struct_fields = Fields::from(vec![ + Field::new( + "c0", + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + true, + ), + Field::new("c1", DataType::Int64, true), + ]); + let make_dict_struct = |c0: ArrayRef, c1: ArrayRef| -> ArrayRef { + let pairs: Vec<(FieldRef, ArrayRef)> = + dict_struct_fields.iter().cloned().zip([c0, c1]).collect(); + Arc::new(StructArray::from(pairs)) + }; + assert_eq!( + expected, + eval_in_list_from_array( + make_dict_struct( + wrap_in_dict(Arc::clone(&utf8_needle)), + Arc::new(Int64Array::from(vec![1, 4, 2])), + ), + make_dict_struct( + wrap_in_dict(Arc::clone(&utf8_in)), + Arc::new(Int64Array::from(vec![1, 2, 3])), + ), + )? + ); + + Ok(()) + } + + #[test] + fn test_in_list_from_array_type_mismatch_errors() -> Result<()> { + // Utf8 needle, Dict(Utf8) in_array + let err = eval_in_list_from_array( + Arc::new(StringArray::from(vec!["a", "d", "b"])), + wrap_in_dict(Arc::new(StringArray::from(vec!["a", "b", "c"]))), + ) + .unwrap_err() + .to_string(); + assert!( + err.contains("Can't compare arrays of different types"), + "{err}" + ); + + // Dict(Utf8) needle, Int64 in_array: specialized Int64StaticFilter + // rejects the Utf8 dictionary values at construction time + let err = eval_in_list_from_array( + wrap_in_dict(Arc::new(StringArray::from(vec!["a", "d", "b"]))), + Arc::new(Int64Array::from(vec![1, 2, 3])), + ) + .unwrap_err() + .to_string(); + assert!(err.contains("Failed to downcast"), "{err}"); + + // Dict(Int64) needle, Dict(Utf8) in_array: both Dict but different + // value types, make_comparator rejects the comparison + let err = eval_in_list_from_array( + wrap_in_dict(Arc::new(Int64Array::from(vec![1, 4, 2]))), + wrap_in_dict(Arc::new(StringArray::from(vec!["a", "b", "c"]))), + ) + .unwrap_err() + .to_string(); + assert!( + err.contains("Can't compare arrays of different types"), + "{err}" + ); + Ok(()) + } } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 698fdea8e766..267faeda0c1b 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -35,6 +35,7 @@ use datafusion_physical_expr_common::sort_expr::{ LexOrdering, LexRequirement, OrderingRequirements, PhysicalSortExpr, PhysicalSortRequirement, }; +use datafusion_physical_plan::aggregates::AggregateExec; use datafusion_physical_plan::execution_plan::CardinalityEffect; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::{ @@ -353,6 +354,8 @@ fn pushdown_requirement_to_children( Ok(None) } } + } else if let Some(aggregate_exec) = plan.as_any().downcast_ref::() { + handle_aggregate_pushdown(aggregate_exec, parent_required) } else if maintains_input_order.is_empty() || !maintains_input_order.iter().any(|o| *o) || plan.as_any().is::() @@ -388,6 +391,77 @@ fn pushdown_requirement_to_children( // TODO: Add support for Projection push down } +/// Try to push sorting through [`AggregateExec`] +/// +/// `AggregateExec` only preserves the input order of its group by columns +/// (not aggregates in general, which are formed from arbitrary expressions over +/// input) +/// +/// Thus function rewrites the parent required ordering in terms of the +/// aggregate input if possible. This rewritten requirement represents the +/// ordering of the `AggregateExec`'s **input** that would also satisfy the +/// **parent** ordering. +/// +/// If no such mapping is possible (e.g. because the sort references aggregate +/// columns), returns None. +fn handle_aggregate_pushdown( + aggregate_exec: &AggregateExec, + parent_required: OrderingRequirements, +) -> Result>>> { + if !aggregate_exec + .maintains_input_order() + .into_iter() + .any(|o| o) + { + return Ok(None); + } + + let group_expr = aggregate_exec.group_expr(); + // GROUPING SETS introduce additional output columns and NULL substitutions; + // skip pushdown until we can map those cases safely. + if group_expr.has_grouping_set() { + return Ok(None); + } + + let group_input_exprs = group_expr.input_exprs(); + let parent_requirement = parent_required.into_single(); + let mut child_requirement = Vec::with_capacity(parent_requirement.len()); + + for req in parent_requirement { + // Sort above AggregateExec should reference its output columns. Map each + // output group-by column to its original input expression. + let Some(column) = req.expr.as_any().downcast_ref::() else { + return Ok(None); + }; + if column.index() >= group_input_exprs.len() { + // AggregateExec does not produce output that is sorted on aggregate + // columns so those can not be pushed through. + return Ok(None); + } + child_requirement.push(PhysicalSortRequirement::new( + Arc::clone(&group_input_exprs[column.index()]), + req.options, + )); + } + + let Some(child_requirement) = LexRequirement::new(child_requirement) else { + return Ok(None); + }; + + // Keep sort above aggregate unless input ordering already satisfies the + // mapped requirement. + if aggregate_exec + .input() + .equivalence_properties() + .ordering_satisfy_requirement(child_requirement.iter().cloned())? + { + let child_requirements = OrderingRequirements::new(child_requirement); + Ok(Some(vec![Some(child_requirements)])) + } else { + Ok(None) + } +} + /// Return true if pushing the sort requirements through a node would violate /// the input sorting requirements for the plan fn pushdown_would_violate_requirements( diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 06f12a90195d..c030507ea3df 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -980,7 +980,7 @@ impl AggregateExec { } else if fun_name.eq_ignore_ascii_case("max") { DynamicFilterAggregateType::Max } else { - continue; + return; }; // 2. arg should be only 1 column reference diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 1ae720271111..a6fc27572370 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -1045,7 +1045,19 @@ impl GroupedHashAggregateStream { self.group_values.len() }; - if let Some(batch) = self.emit(EmitTo::First(n), false)? { + // Clamp to the sort boundary when using partial group ordering, + // otherwise remove_groups panics (#20445). + let n = match &self.group_ordering { + GroupOrdering::None => n, + _ => match self.group_ordering.emit_to() { + Some(EmitTo::First(max)) => n.min(max), + _ => 0, + }, + }; + + if n > 0 + && let Some(batch) = self.emit(EmitTo::First(n), false)? + { Ok(Some(ExecutionState::ProducingOutput(batch))) } else { Err(oom) @@ -1221,6 +1233,18 @@ impl GroupedHashAggregateStream { // on the grouping columns. self.group_ordering = GroupOrdering::Full(GroupOrderingFull::new()); + // Recreate group_values to use streaming mode (GroupValuesColumn + // with scalarized_intern) which preserves input row order, as required + // by GroupOrderingFull. This is only needed for multi-column group by, + // since single-column uses GroupValuesPrimitive which is always safe. + let group_schema = self + .spill_state + .merging_group_by + .group_schema(&self.spill_state.spill_schema)?; + if group_schema.fields().len() > 1 { + self.group_values = new_group_values(group_schema, &self.group_ordering)?; + } + // Use `OutOfMemoryMode::ReportError` from this point on // to ensure we don't spill the spilled data to disk again. self.oom_mode = OutOfMemoryMode::ReportError; @@ -1305,6 +1329,7 @@ impl GroupedHashAggregateStream { #[cfg(test)] mod tests { use super::*; + use crate::InputOrderMode; use crate::execution_plan::ExecutionPlan; use crate::test::TestMemoryExec; use arrow::array::{Int32Array, Int64Array}; @@ -1567,4 +1592,88 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_emit_early_with_partially_sorted() -> Result<()> { + // Reproducer for #20445: EmitEarly with PartiallySorted panics in + // remove_groups because it emits more groups than the sort boundary. + let schema = Arc::new(Schema::new(vec![ + Field::new("sort_col", DataType::Int32, false), + Field::new("group_col", DataType::Int32, false), + Field::new("value_col", DataType::Int64, false), + ])); + + // All rows share sort_col=1 (no sort boundary), with unique group_col + // values to create many groups and trigger memory pressure. + let n = 256; + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![1; n])), + Arc::new(Int32Array::from((0..n as i32).collect::>())), + Arc::new(Int64Array::from(vec![1; n])), + ], + )?; + + let runtime = RuntimeEnvBuilder::default() + .with_memory_limit(4096, 1.0) + .build_arc()?; + let mut task_ctx = TaskContext::default().with_runtime(runtime); + let mut cfg = task_ctx.session_config().clone(); + cfg = cfg.set( + "datafusion.execution.batch_size", + &datafusion_common::ScalarValue::UInt64(Some(128)), + ); + cfg = cfg.set( + "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", + &datafusion_common::ScalarValue::UInt64(Some(u64::MAX)), + ); + task_ctx = task_ctx.with_session_config(cfg); + let task_ctx = Arc::new(task_ctx); + + let ordering = LexOrdering::new(vec![PhysicalSortExpr::new_default(Arc::new( + Column::new("sort_col", 0), + ) + as _)]) + .unwrap(); + let exec = TestMemoryExec::try_new(&[vec![batch]], Arc::clone(&schema), None)? + .try_with_sort_information(vec![ordering])?; + let exec = Arc::new(TestMemoryExec::update_cache(&Arc::new(exec))); + + // GROUP BY sort_col, group_col with input sorted on sort_col + // gives PartiallySorted([0]) + let aggregate_exec = AggregateExec::try_new( + AggregateMode::Partial, + PhysicalGroupBy::new_single(vec![ + (col("sort_col", &schema)?, "sort_col".to_string()), + (col("group_col", &schema)?, "group_col".to_string()), + ]), + vec![Arc::new( + AggregateExprBuilder::new(count_udaf(), vec![col("value_col", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("count_value") + .build()?, + )], + vec![None], + exec, + Arc::clone(&schema), + )?; + assert!(matches!( + aggregate_exec.input_order_mode(), + InputOrderMode::PartiallySorted(_) + )); + + // Must not panic with "assertion failed: *current_sort >= n" + let mut stream = GroupedHashAggregateStream::new(&aggregate_exec, &task_ctx, 0)?; + while let Some(result) = stream.next().await { + if let Err(e) = result { + if e.to_string().contains("Resources exhausted") { + break; + } + return Err(e); + } + } + + Ok(()) + } } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index e724cdad6484..9e849759ca65 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -31,7 +31,7 @@ use crate::common::can_project; use crate::execution_plan::CardinalityEffect; use crate::filter_pushdown::{ ChildFilterDescription, ChildPushdownResult, FilterDescription, FilterPushdownPhase, - FilterPushdownPropagation, PushedDown, PushedDownPredicate, + FilterPushdownPropagation, PushedDown, }; use crate::metrics::{MetricBuilder, MetricType}; use crate::projection::{ @@ -55,12 +55,12 @@ use datafusion_common::{ use datafusion_execution::TaskContext; use datafusion_expr::Operator; use datafusion_physical_expr::equivalence::ProjectionMapping; -use datafusion_physical_expr::expressions::{BinaryExpr, Column, lit}; +use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal, lit}; use datafusion_physical_expr::intervals::utils::check_support; -use datafusion_physical_expr::utils::collect_columns; +use datafusion_physical_expr::utils::{collect_columns, reassign_expr_columns}; use datafusion_physical_expr::{ - AcrossPartitions, AnalysisContext, ConstExpr, ExprBoundaries, PhysicalExpr, analyze, - conjunction, split_conjunction, + AcrossPartitions, AnalysisContext, ConstExpr, EquivalenceProperties, ExprBoundaries, + PhysicalExpr, analyze, conjunction, split_conjunction, }; use datafusion_physical_expr_common::physical_expr::fmt_sql; @@ -232,6 +232,7 @@ impl FilterExec { let total_byte_size = total_byte_size.with_estimated_selectivity(selectivity); let column_statistics = collect_new_statistics( + schema, &input_stats.column_statistics, analysis_ctx.boundaries, ); @@ -242,6 +243,20 @@ impl FilterExec { }) } + /// Returns the `AcrossPartitions` value for `expr` if it is constant: + /// either already known constant in `input_eqs`, or a `Literal` + /// (which is inherently constant across all partitions). + fn expr_constant_or_literal( + expr: &Arc, + input_eqs: &EquivalenceProperties, + ) -> Option { + input_eqs.is_expr_constant(expr).or_else(|| { + expr.as_any() + .downcast_ref::() + .map(|l| AcrossPartitions::Uniform(Some(l.value().clone()))) + }) + } + fn extend_constants( input: &Arc, predicate: &Arc, @@ -254,18 +269,24 @@ impl FilterExec { if let Some(binary) = conjunction.as_any().downcast_ref::() && binary.op() == &Operator::Eq { - // Filter evaluates to single value for all partitions - if input_eqs.is_expr_constant(binary.left()).is_some() { - let across = input_eqs - .is_expr_constant(binary.right()) - .unwrap_or_default(); + // Check if either side is constant — either already known + // constant from the input equivalence properties, or a literal + // value (which is inherently constant across all partitions). + let left_const = Self::expr_constant_or_literal(binary.left(), input_eqs); + let right_const = + Self::expr_constant_or_literal(binary.right(), input_eqs); + + if let Some(left_across) = left_const { + // LEFT is constant, so RIGHT must also be constant. + // Use RIGHT's known across value if available, otherwise + // propagate LEFT's (e.g. Uniform from a literal). + let across = right_const.unwrap_or(left_across); res_constants .push(ConstExpr::new(Arc::clone(binary.right()), across)); - } else if input_eqs.is_expr_constant(binary.right()).is_some() { - let across = input_eqs - .is_expr_constant(binary.left()) - .unwrap_or_default(); - res_constants.push(ConstExpr::new(Arc::clone(binary.left()), across)); + } else if let Some(right_across) = right_const { + // RIGHT is constant, so LEFT must also be constant. + res_constants + .push(ConstExpr::new(Arc::clone(binary.left()), right_across)); } } } @@ -494,15 +515,9 @@ impl ExecutionPlan for FilterExec { ) -> Result { if !matches!(phase, FilterPushdownPhase::Pre) { // For non-pre phase, filters pass through unchanged - let filter_supports = parent_filters - .into_iter() - .map(PushedDownPredicate::supported) - .collect(); - - return Ok(FilterDescription::new().with_child(ChildFilterDescription { - parent_filters: filter_supports, - self_filters: vec![], - })); + let child = + ChildFilterDescription::from_child(&parent_filters, self.input())?; + return Ok(FilterDescription::new().with_child(child)); } let child = ChildFilterDescription::from_child(&parent_filters, self.input())? @@ -526,10 +541,26 @@ impl ExecutionPlan for FilterExec { return Ok(FilterPushdownPropagation::if_all(child_pushdown_result)); } // We absorb any parent filters that were not handled by our children - let unsupported_parent_filters = - child_pushdown_result.parent_filters.iter().filter_map(|f| { - matches!(f.all(), PushedDown::No).then_some(Arc::clone(&f.filter)) - }); + let mut unsupported_parent_filters: Vec> = + child_pushdown_result + .parent_filters + .iter() + .filter_map(|f| { + matches!(f.all(), PushedDown::No).then_some(Arc::clone(&f.filter)) + }) + .collect(); + + // If this FilterExec has a projection, the unsupported parent filters + // are in the output schema (after projection) coordinates. We need to + // remap them to the input schema coordinates before combining with self filters. + if self.projection.is_some() { + let input_schema = self.input().schema(); + unsupported_parent_filters = unsupported_parent_filters + .into_iter() + .map(|expr| reassign_expr_columns(expr, &input_schema)) + .collect::>>()?; + } + let unsupported_self_filters = child_pushdown_result .self_filters .first() @@ -577,7 +608,7 @@ impl ExecutionPlan for FilterExec { // The new predicate is the same as our current predicate None } else { - // Create a new FilterExec with the new predicate + // Create a new FilterExec with the new predicate, preserving the projection let new = FilterExec { predicate: Arc::clone(&new_predicate), input: Arc::clone(&filter_input), @@ -589,7 +620,7 @@ impl ExecutionPlan for FilterExec { self.default_selectivity, self.projection.as_ref(), )?, - projection: None, + projection: self.projection.clone(), batch_size: self.batch_size, fetch: self.fetch, }; @@ -627,6 +658,7 @@ impl EmbeddedProjection for FilterExec { /// is adjusted by using the next/previous value for its data type to convert /// it into a closed bound. fn collect_new_statistics( + schema: &SchemaRef, input_column_stats: &[ColumnStatistics], analysis_boundaries: Vec, ) -> Vec { @@ -643,12 +675,17 @@ fn collect_new_statistics( }, )| { let Some(interval) = interval else { - // If the interval is `None`, we can say that there are no rows: + // If the interval is `None`, we can say that there are no rows. + // Use a typed null to preserve the column's data type, so that + // downstream interval analysis can still intersect intervals + // of the same type. + let typed_null = ScalarValue::try_from(schema.field(idx).data_type()) + .unwrap_or(ScalarValue::Null); return ColumnStatistics { null_count: Precision::Exact(0), - max_value: Precision::Exact(ScalarValue::Null), - min_value: Precision::Exact(ScalarValue::Null), - sum_value: Precision::Exact(ScalarValue::Null), + max_value: Precision::Exact(typed_null.clone()), + min_value: Precision::Exact(typed_null.clone()), + sum_value: Precision::Exact(typed_null), distinct_count: Precision::Exact(0), byte_size: input_column_stats[idx].byte_size, }; @@ -849,6 +886,19 @@ fn collect_columns_from_predicate_inner( let predicates = split_conjunction(predicate); predicates.into_iter().for_each(|p| { if let Some(binary) = p.as_any().downcast_ref::() { + // Only extract pairs where at least one side is a Column reference. + // Pairs like `complex_expr = literal` should not create equivalence + // classes — the literal could appear in many unrelated expressions + // (e.g. sort keys), and normalize_expr's deep traversal would + // replace those occurrences with the complex expression, corrupting + // sort orderings. Constant propagation for such pairs is handled + // separately by `extend_constants`. + let has_direct_column_operand = + binary.left().as_any().downcast_ref::().is_some() + || binary.right().as_any().downcast_ref::().is_some(); + if !has_direct_column_operand { + return; + } match binary.op() { Operator::Eq => { eq_predicate_columns.push((binary.left(), binary.right())) @@ -1341,17 +1391,17 @@ mod tests { statistics.column_statistics, vec![ ColumnStatistics { - min_value: Precision::Exact(ScalarValue::Null), - max_value: Precision::Exact(ScalarValue::Null), - sum_value: Precision::Exact(ScalarValue::Null), + min_value: Precision::Exact(ScalarValue::Int32(None)), + max_value: Precision::Exact(ScalarValue::Int32(None)), + sum_value: Precision::Exact(ScalarValue::Int32(None)), distinct_count: Precision::Exact(0), null_count: Precision::Exact(0), byte_size: Precision::Absent, }, ColumnStatistics { - min_value: Precision::Exact(ScalarValue::Null), - max_value: Precision::Exact(ScalarValue::Null), - sum_value: Precision::Exact(ScalarValue::Null), + min_value: Precision::Exact(ScalarValue::Int32(None)), + max_value: Precision::Exact(ScalarValue::Int32(None)), + sum_value: Precision::Exact(ScalarValue::Int32(None)), distinct_count: Precision::Exact(0), null_count: Precision::Exact(0), byte_size: Precision::Absent, @@ -1362,6 +1412,70 @@ mod tests { Ok(()) } + /// Regression test: stacking two FilterExecs where the inner filter + /// proves zero selectivity should not panic with a type mismatch + /// during interval intersection. + /// + /// Previously, when a filter proved no rows could match, the column + /// statistics used untyped `ScalarValue::Null` (data type `Null`). + /// If an outer FilterExec then tried to analyze its own predicate + /// against those statistics, `Interval::intersect` would fail with: + /// "Only intervals with the same data type are intersectable, lhs:Null, rhs:Int32" + #[tokio::test] + async fn test_nested_filter_with_zero_selectivity_inner() -> Result<()> { + // Inner table: a: [1, 100], b: [1, 3] + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + ]); + let input = Arc::new(StatisticsExec::new( + Statistics { + num_rows: Precision::Inexact(1000), + total_byte_size: Precision::Inexact(4000), + column_statistics: vec![ + ColumnStatistics { + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), + ..Default::default() + }, + ColumnStatistics { + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(3))), + ..Default::default() + }, + ], + }, + schema, + )); + + // Inner filter: a > 200 (impossible given a max=100 → zero selectivity) + let inner_predicate: Arc = Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Gt, + Arc::new(Literal::new(ScalarValue::Int32(Some(200)))), + )); + let inner_filter: Arc = + Arc::new(FilterExec::try_new(inner_predicate, input)?); + + // Outer filter: a = 50 + // Before the fix, this would panic because the inner filter's + // zero-selectivity statistics produced Null-typed intervals for + // column `a`, which couldn't intersect with the Int32 literal. + let outer_predicate: Arc = Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Eq, + Arc::new(Literal::new(ScalarValue::Int32(Some(50)))), + )); + let outer_filter: Arc = + Arc::new(FilterExec::try_new(outer_predicate, inner_filter)?); + + // Should succeed without error + let statistics = outer_filter.partition_statistics(None)?; + assert_eq!(statistics.num_rows, Precision::Inexact(0)); + + Ok(()) + } + #[tokio::test] async fn test_filter_statistics_more_inputs() -> Result<()> { let schema = Schema::new(vec![ @@ -1569,4 +1683,97 @@ mod tests { Ok(()) } + + #[test] + fn test_filter_with_projection_remaps_post_phase_parent_filters() -> Result<()> { + // Test that FilterExec with a projection must remap parent dynamic + // filter column indices from its output schema to the input schema + // before passing them to the child. + let input_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 input = Arc::new(EmptyExec::new(Arc::clone(&input_schema))); + + // FilterExec: a > 0, projection=[c@2] + let predicate = Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Gt, + Arc::new(Literal::new(ScalarValue::Int32(Some(0)))), + )); + let filter = + FilterExec::try_new(predicate, input)?.with_projection(Some(vec![2]))?; + + // Output schema should be [c:Float64] + let output_schema = filter.schema(); + assert_eq!(output_schema.fields().len(), 1); + assert_eq!(output_schema.field(0).name(), "c"); + + // Simulate a parent dynamic filter referencing output column c@0 + let parent_filter: Arc = Arc::new(Column::new("c", 0)); + + let config = ConfigOptions::new(); + let desc = filter.gather_filters_for_pushdown( + FilterPushdownPhase::Post, + vec![parent_filter], + &config, + )?; + + // The filter pushed to the child must reference c@2 (input schema), + // not c@0 (output schema). + let parent_filters = desc.parent_filters(); + assert_eq!(parent_filters.len(), 1); // one child + assert_eq!(parent_filters[0].len(), 1); // one filter + let remapped = &parent_filters[0][0].predicate; + let display = format!("{remapped}"); + assert_eq!( + display, "c@2", + "Post-phase parent filter column index must be remapped \ + from output schema (c@0) to input schema (c@2)" + ); + + Ok(()) + } + /// Regression test for https://github.com/apache/datafusion/issues/20194 + /// + /// `collect_columns_from_predicate_inner` should only extract equality + /// pairs where at least one side is a Column. Pairs like + /// `complex_expr = literal` must not create equivalence classes because + /// `normalize_expr`'s deep traversal would replace the literal inside + /// unrelated expressions (e.g. sort keys) with the complex expression. + #[test] + fn test_collect_columns_skips_non_column_pairs() -> Result<()> { + let schema = test::aggr_test_schema(); + + // Simulate: nvl(c2, 0) = 0 → (c2 IS DISTINCT FROM 0) = 0 + // Neither side is a Column, so this should NOT be extracted. + let complex_expr: Arc = binary( + col("c2", &schema)?, + Operator::IsDistinctFrom, + lit(0u32), + &schema, + )?; + let predicate: Arc = + binary(complex_expr, Operator::Eq, lit(0u32), &schema)?; + + let (equal_pairs, _) = collect_columns_from_predicate_inner(&predicate); + assert_eq!( + 0, + equal_pairs.len(), + "Should not extract equality pairs where neither side is a Column" + ); + + // But col = literal should still be extracted + let predicate: Arc = + binary(col("c2", &schema)?, Operator::Eq, lit(0u32), &schema)?; + let (equal_pairs, _) = collect_columns_from_predicate_inner(&predicate); + assert_eq!( + 1, + equal_pairs.len(), + "Should extract equality pairs where one side is a Column" + ); + + Ok(()) + } } diff --git a/datafusion/physical-plan/src/joins/hash_join/inlist_builder.rs b/datafusion/physical-plan/src/joins/hash_join/inlist_builder.rs index 7dccc5b0ba7c..0ca338265ecc 100644 --- a/datafusion/physical-plan/src/joins/hash_join/inlist_builder.rs +++ b/datafusion/physical-plan/src/joins/hash_join/inlist_builder.rs @@ -21,7 +21,6 @@ use std::sync::Arc; use arrow::array::{ArrayRef, StructArray}; use arrow::datatypes::{Field, FieldRef, Fields}; -use arrow::downcast_dictionary_array; use arrow_schema::DataType; use datafusion_common::Result; @@ -33,18 +32,6 @@ pub(super) fn build_struct_fields(data_types: &[DataType]) -> Result { .collect() } -/// Flattens dictionary-encoded arrays to their underlying value arrays. -/// Non-dictionary arrays are returned as-is. -fn flatten_dictionary_array(array: &ArrayRef) -> ArrayRef { - downcast_dictionary_array! { - array => { - // Recursively flatten in case of nested dictionaries - flatten_dictionary_array(array.values()) - } - _ => Arc::clone(array) - } -} - /// Builds InList values from join key column arrays. /// /// If `join_key_arrays` is: @@ -64,20 +51,14 @@ fn flatten_dictionary_array(array: &ArrayRef) -> ArrayRef { pub(super) fn build_struct_inlist_values( join_key_arrays: &[ArrayRef], ) -> Result> { - // Flatten any dictionary-encoded arrays - let flattened_arrays: Vec = join_key_arrays - .iter() - .map(flatten_dictionary_array) - .collect(); - // Build the source array/struct - let source_array: ArrayRef = if flattened_arrays.len() == 1 { + let source_array: ArrayRef = if join_key_arrays.len() == 1 { // Single column: use directly - Arc::clone(&flattened_arrays[0]) + Arc::clone(&join_key_arrays[0]) } else { // Multi-column: build StructArray once from all columns let fields = build_struct_fields( - &flattened_arrays + &join_key_arrays .iter() .map(|arr| arr.data_type().clone()) .collect::>(), @@ -87,7 +68,7 @@ pub(super) fn build_struct_inlist_values( let arrays_with_fields: Vec<(FieldRef, ArrayRef)> = fields .iter() .cloned() - .zip(flattened_arrays.iter().cloned()) + .zip(join_key_arrays.iter().cloned()) .collect(); Arc::new(StructArray::from(arrays_with_fields)) @@ -99,7 +80,9 @@ pub(super) fn build_struct_inlist_values( #[cfg(test)] mod tests { use super::*; - use arrow::array::{Int32Array, StringArray}; + use arrow::array::{ + DictionaryArray, Int8Array, Int32Array, StringArray, StringDictionaryBuilder, + }; use arrow_schema::DataType; use std::sync::Arc; @@ -130,4 +113,48 @@ mod tests { ) ); } + + #[test] + fn test_build_multi_column_inlist_with_dictionary() { + let mut builder = StringDictionaryBuilder::::new(); + builder.append_value("foo"); + builder.append_value("foo"); + builder.append_value("foo"); + let dict_array = Arc::new(builder.finish()) as ArrayRef; + + let int_array = Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef; + + let result = build_struct_inlist_values(&[dict_array, int_array]) + .unwrap() + .unwrap(); + + assert_eq!(result.len(), 3); + assert_eq!( + *result.data_type(), + DataType::Struct( + build_struct_fields(&[ + DataType::Dictionary( + Box::new(DataType::Int8), + Box::new(DataType::Utf8) + ), + DataType::Int32 + ]) + .unwrap() + ) + ); + } + + #[test] + fn test_build_single_column_dictionary_inlist() { + let keys = Int8Array::from(vec![0i8, 0, 0]); + let values = Arc::new(StringArray::from(vec!["foo"])); + let dict_array = Arc::new(DictionaryArray::new(keys, values)) as ArrayRef; + + let result = build_struct_inlist_values(std::slice::from_ref(&dict_array)) + .unwrap() + .unwrap(); + + assert_eq!(result.len(), 3); + assert_eq!(result.data_type(), dict_array.data_type()); + } } diff --git a/datafusion/physical-plan/src/joins/hash_join/stream.rs b/datafusion/physical-plan/src/joins/hash_join/stream.rs index e6735675125b..c5c794f5a8c6 100644 --- a/datafusion/physical-plan/src/joins/hash_join/stream.rs +++ b/datafusion/physical-plan/src/joins/hash_join/stream.rs @@ -639,6 +639,7 @@ impl HashJoinStream { filter, JoinSide::Left, None, + self.join_type, )? } else { (left_indices, right_indices) @@ -707,6 +708,7 @@ impl HashJoinStream { &right_indices, &self.column_indices, join_side, + self.join_type, )?; self.output_buffer.push_batch(batch)?; @@ -770,6 +772,7 @@ impl HashJoinStream { &right_side, &self.column_indices, JoinSide::Left, + self.join_type, )?; self.output_buffer.push_batch(batch)?; } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/filter.rs b/datafusion/physical-plan/src/joins/sort_merge_join/filter.rs new file mode 100644 index 000000000000..d598442b653e --- /dev/null +++ b/datafusion/physical-plan/src/joins/sort_merge_join/filter.rs @@ -0,0 +1,595 @@ +// 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. + +//! Filter handling for Sort-Merge Join +//! +//! This module encapsulates the complexity of join filter evaluation, including: +//! - Immediate filtering for INNER joins +//! - Deferred filtering for outer/semi/anti/mark joins +//! - Metadata tracking for grouping output rows by input row +//! - Correcting filter masks to handle multiple matches per input row + +use std::sync::Arc; + +use arrow::array::{ + Array, ArrayBuilder, ArrayRef, BooleanArray, BooleanBuilder, RecordBatch, + UInt64Array, UInt64Builder, +}; +use arrow::compute::{self, concat_batches, filter_record_batch}; +use arrow::datatypes::SchemaRef; +use datafusion_common::{JoinSide, JoinType, Result}; + +use crate::joins::utils::JoinFilter; + +/// Metadata for tracking filter results during deferred filtering +/// +/// When a join filter is present and we need to ensure each input row produces +/// at least one output (outer joins) or exactly one output (semi joins), we can't +/// filter immediately. Instead, we accumulate all joined rows with metadata, +/// then post-process to determine which rows to output. +#[derive(Debug)] +pub struct FilterMetadata { + /// Did each output row pass the join filter? + /// Used to detect if an input row found ANY match + pub filter_mask: BooleanBuilder, + + /// Which input row (within batch) produced each output row? + /// Used for grouping output rows by input row + pub row_indices: UInt64Builder, + + /// Which input batch did each output row come from? + /// Used to disambiguate row_indices across multiple batches + pub batch_ids: Vec, +} + +impl FilterMetadata { + /// Create new empty filter metadata + pub fn new() -> Self { + Self { + filter_mask: BooleanBuilder::new(), + row_indices: UInt64Builder::new(), + batch_ids: vec![], + } + } + + /// Returns (row_indices, filter_mask, batch_ids_ref) and clears builders + pub fn finish_metadata(&mut self) -> (UInt64Array, BooleanArray, &[usize]) { + let row_indices = self.row_indices.finish(); + let filter_mask = self.filter_mask.finish(); + (row_indices, filter_mask, &self.batch_ids) + } + + /// Add metadata for null-joined rows (no filter applied) + pub fn append_nulls(&mut self, num_rows: usize) { + self.filter_mask.append_nulls(num_rows); + self.row_indices.append_nulls(num_rows); + self.batch_ids.resize( + self.batch_ids.len() + num_rows, + 0, // batch_id = 0 for null-joined rows + ); + } + + /// Add metadata for filtered rows + pub fn append_filter_metadata( + &mut self, + row_indices: &UInt64Array, + filter_mask: &BooleanArray, + batch_id: usize, + ) { + debug_assert_eq!( + row_indices.len(), + filter_mask.len(), + "row_indices and filter_mask must have same length" + ); + + for i in 0..row_indices.len() { + if filter_mask.is_null(i) { + self.filter_mask.append_null(); + } else if filter_mask.value(i) { + self.filter_mask.append_value(true); + } else { + self.filter_mask.append_value(false); + } + + if row_indices.is_null(i) { + self.row_indices.append_null(); + } else { + self.row_indices.append_value(row_indices.value(i)); + } + + self.batch_ids.push(batch_id); + } + } + + /// Verify that metadata arrays are aligned (same length) + pub fn debug_assert_metadata_aligned(&self) { + if self.filter_mask.len() > 0 { + debug_assert_eq!( + self.filter_mask.len(), + self.row_indices.len(), + "filter_mask and row_indices must have same length when metadata is used" + ); + debug_assert_eq!( + self.filter_mask.len(), + self.batch_ids.len(), + "filter_mask and batch_ids must have same length when metadata is used" + ); + } else { + debug_assert_eq!( + self.filter_mask.len(), + 0, + "filter_mask should be empty when batches is empty" + ); + } + } +} + +impl Default for FilterMetadata { + fn default() -> Self { + Self::new() + } +} + +/// Determines if a join type needs deferred filtering +/// +/// Deferred filtering is required when: +/// - A filter exists AND +/// - The join type requires ensuring each input row produces at least one output +/// (or exactly one for semi joins) +pub fn needs_deferred_filtering( + filter: &Option, + join_type: JoinType, +) -> bool { + filter.is_some() + && matches!( + join_type, + JoinType::Left + | JoinType::LeftSemi + | JoinType::LeftMark + | JoinType::Right + | JoinType::RightSemi + | JoinType::RightMark + | JoinType::LeftAnti + | JoinType::RightAnti + | JoinType::Full + ) +} + +/// Gets the arrays which join filters are applied on +/// +/// Extracts the columns needed for filter evaluation from left and right batch columns +pub fn get_filter_columns( + join_filter: &Option, + left_columns: &[ArrayRef], + right_columns: &[ArrayRef], +) -> Vec { + let mut filter_columns = vec![]; + + if let Some(f) = join_filter { + let left_columns: Vec = f + .column_indices() + .iter() + .filter(|col_index| col_index.side == JoinSide::Left) + .map(|i| Arc::clone(&left_columns[i.index])) + .collect(); + let right_columns: Vec = f + .column_indices() + .iter() + .filter(|col_index| col_index.side == JoinSide::Right) + .map(|i| Arc::clone(&right_columns[i.index])) + .collect(); + + filter_columns.extend(left_columns); + filter_columns.extend(right_columns); + } + + filter_columns +} + +/// Determines if current index is the last occurrence of a row +/// +/// Used during filter mask correction to detect row boundaries when grouping +/// output rows by input row. +fn last_index_for_row( + row_index: usize, + indices: &UInt64Array, + batch_ids: &[usize], + indices_len: usize, +) -> bool { + debug_assert_eq!( + indices.len(), + indices_len, + "indices.len() should match indices_len parameter" + ); + debug_assert_eq!( + batch_ids.len(), + indices_len, + "batch_ids.len() should match indices_len" + ); + debug_assert!( + row_index < indices_len, + "row_index {row_index} should be < indices_len {indices_len}", + ); + + // If this is the last index overall, it's definitely the last for this row + if row_index == indices_len - 1 { + return true; + } + + // Check if next row has different (batch_id, index) pair + let current_batch_id = batch_ids[row_index]; + let next_batch_id = batch_ids[row_index + 1]; + + if current_batch_id != next_batch_id { + return true; + } + + // Same batch_id, check if row index is different + // Both current and next should be non-null (already joined rows) + if indices.is_null(row_index) || indices.is_null(row_index + 1) { + return true; + } + + indices.value(row_index) != indices.value(row_index + 1) +} + +/// Corrects the filter mask for joins with deferred filtering +/// +/// When an input row joins with multiple buffered rows, we get multiple output rows. +/// This function groups them by input row and applies join-type-specific logic: +/// +/// - **Outer joins**: Keep first matching row, convert rest to nulls, add null-joined for unmatched +/// - **Semi joins**: Keep first matching row, discard rest +/// - **Anti joins**: Keep row only if NO matches passed filter +/// - **Mark joins**: Like semi but first match only +/// +/// # Arguments +/// * `join_type` - The type of join being performed +/// * `row_indices` - Which input row produced each output row +/// * `batch_ids` - Which batch each output row came from +/// * `filter_mask` - Whether each output row passed the filter +/// * `expected_size` - Total number of input rows (for adding unmatched) +/// +/// # Returns +/// Corrected mask indicating which rows to include in final output: +/// - `true`: Include this row +/// - `false`: Convert to null-joined row (outer joins) or include as unmatched (anti joins) +/// - `null`: Discard this row +pub fn get_corrected_filter_mask( + join_type: JoinType, + row_indices: &UInt64Array, + batch_ids: &[usize], + filter_mask: &BooleanArray, + expected_size: usize, +) -> Option { + let row_indices_length = row_indices.len(); + let mut corrected_mask: BooleanBuilder = + BooleanBuilder::with_capacity(row_indices_length); + let mut seen_true = false; + + match join_type { + JoinType::Left | JoinType::Right => { + // For outer joins: Keep first matching row per input row, + // convert rest to nulls, add null-joined rows for unmatched + for i in 0..row_indices_length { + let last_index = + last_index_for_row(i, row_indices, batch_ids, row_indices_length); + if filter_mask.value(i) { + seen_true = true; + corrected_mask.append_value(true); + } else if seen_true || !filter_mask.value(i) && !last_index { + corrected_mask.append_null(); // to be ignored and not set to output + } else { + corrected_mask.append_value(false); // to be converted to null joined row + } + + if last_index { + seen_true = false; + } + } + + // Generate null joined rows for records which have no matching join key + corrected_mask.append_n(expected_size - corrected_mask.len(), false); + Some(corrected_mask.finish()) + } + JoinType::LeftMark | JoinType::RightMark => { + // For mark joins: Like outer but only keep first match, mark with boolean + for i in 0..row_indices_length { + let last_index = + last_index_for_row(i, row_indices, batch_ids, row_indices_length); + if filter_mask.value(i) && !seen_true { + seen_true = true; + corrected_mask.append_value(true); + } else if seen_true || !filter_mask.value(i) && !last_index { + corrected_mask.append_null(); // to be ignored and not set to output + } else { + corrected_mask.append_value(false); // to be converted to null joined row + } + + if last_index { + seen_true = false; + } + } + + // Generate null joined rows for records which have no matching join key + corrected_mask.append_n(expected_size - corrected_mask.len(), false); + Some(corrected_mask.finish()) + } + JoinType::LeftSemi | JoinType::RightSemi => { + // For semi joins: Keep only first matching row per input row, discard rest + for i in 0..row_indices_length { + let last_index = + last_index_for_row(i, row_indices, batch_ids, row_indices_length); + if filter_mask.value(i) && !seen_true { + seen_true = true; + corrected_mask.append_value(true); + } else { + corrected_mask.append_null(); // to be ignored and not set to output + } + + if last_index { + seen_true = false; + } + } + + Some(corrected_mask.finish()) + } + JoinType::LeftAnti | JoinType::RightAnti => { + // For anti joins: Keep row only if NO matches passed the filter + for i in 0..row_indices_length { + let last_index = + last_index_for_row(i, row_indices, batch_ids, row_indices_length); + + if filter_mask.value(i) { + seen_true = true; + } + + if last_index { + if !seen_true { + corrected_mask.append_value(true); + } else { + corrected_mask.append_null(); + } + + seen_true = false; + } else { + corrected_mask.append_null(); + } + } + // Generate null joined rows for records which have no matching join key, + // for LeftAnti non-matched considered as true + corrected_mask.append_n(expected_size - corrected_mask.len(), true); + Some(corrected_mask.finish()) + } + JoinType::Full => { + // For full joins: Similar to outer but handle both sides + for i in 0..row_indices_length { + let last_index = + last_index_for_row(i, row_indices, batch_ids, row_indices_length); + + if filter_mask.is_null(i) { + // null joined + corrected_mask.append_value(true); + } else if filter_mask.value(i) { + seen_true = true; + corrected_mask.append_value(true); + } else if seen_true || !filter_mask.value(i) && !last_index { + corrected_mask.append_null(); // to be ignored and not set to output + } else { + corrected_mask.append_value(false); // to be converted to null joined row + } + + if last_index { + seen_true = false; + } + } + // Generate null joined rows for records which have no matching join key + corrected_mask.append_n(expected_size - corrected_mask.len(), false); + Some(corrected_mask.finish()) + } + JoinType::Inner => { + // Inner joins don't need deferred filtering + None + } + } +} + +/// Applies corrected filter mask to record batch based on join type +/// +/// Different join types require different handling of filtered results: +/// - Outer joins: Add null-joined rows for false mask values +/// - Semi/Anti joins: May need projection to remove right columns +/// - Full joins: Add null-joined rows for both sides +pub fn filter_record_batch_by_join_type( + record_batch: &RecordBatch, + corrected_mask: &BooleanArray, + join_type: JoinType, + schema: &SchemaRef, + streamed_schema: &SchemaRef, + buffered_schema: &SchemaRef, +) -> Result { + let filtered_record_batch = filter_record_batch(record_batch, corrected_mask)?; + + match join_type { + JoinType::Left | JoinType::LeftMark => { + // For left joins, add null-joined rows where mask is false + let null_mask = compute::not(corrected_mask)?; + let null_joined_batch = filter_record_batch(record_batch, &null_mask)?; + + if null_joined_batch.num_rows() == 0 { + return Ok(filtered_record_batch); + } + + // Create null columns for right side + let null_joined_streamed_batch = create_null_joined_batch( + &null_joined_batch, + buffered_schema, + JoinSide::Left, + join_type, + schema, + )?; + + Ok(concat_batches( + schema, + &[filtered_record_batch, null_joined_streamed_batch], + )?) + } + JoinType::LeftSemi + | JoinType::LeftAnti + | JoinType::RightSemi + | JoinType::RightAnti => { + // For semi/anti joins, project to only include the outer side columns + // Both Left and Right semi/anti use streamed_schema.len() because: + // - For Left: columns are [left, right], so we take first streamed_schema.len() + // - For Right: columns are [right, left], and streamed side is right, so we take first streamed_schema.len() + let output_column_indices: Vec = + (0..streamed_schema.fields().len()).collect(); + Ok(filtered_record_batch.project(&output_column_indices)?) + } + JoinType::Right | JoinType::RightMark => { + // For right joins, add null-joined rows where mask is false + let null_mask = compute::not(corrected_mask)?; + let null_joined_batch = filter_record_batch(record_batch, &null_mask)?; + + if null_joined_batch.num_rows() == 0 { + return Ok(filtered_record_batch); + } + + // Create null columns for left side (buffered side for RIGHT join) + let null_joined_buffered_batch = create_null_joined_batch( + &null_joined_batch, + buffered_schema, // Pass buffered (left) schema to create nulls for it + JoinSide::Right, + join_type, + schema, + )?; + + Ok(concat_batches( + schema, + &[filtered_record_batch, null_joined_buffered_batch], + )?) + } + JoinType::Full => { + // For full joins, add null-joined rows for both sides + let joined_filter_not_matched_mask = compute::not(corrected_mask)?; + let joined_filter_not_matched_batch = + filter_record_batch(record_batch, &joined_filter_not_matched_mask)?; + + if joined_filter_not_matched_batch.num_rows() == 0 { + return Ok(filtered_record_batch); + } + + // Create null-joined batches for both sides + let left_null_joined_batch = create_null_joined_batch( + &joined_filter_not_matched_batch, + buffered_schema, + JoinSide::Left, + join_type, + schema, + )?; + + Ok(concat_batches( + schema, + &[filtered_record_batch, left_null_joined_batch], + )?) + } + JoinType::Inner => Ok(filtered_record_batch), + } +} + +/// Creates a batch with null columns for the non-joined side +/// +/// Note: The input `batch` is assumed to be a fully-joined batch that already contains +/// columns from both sides. We need to extract the data side columns and replace the +/// null side columns with actual nulls. +fn create_null_joined_batch( + batch: &RecordBatch, + null_schema: &SchemaRef, + join_side: JoinSide, + join_type: JoinType, + output_schema: &SchemaRef, +) -> Result { + let num_rows = batch.num_rows(); + + // The input batch is a fully-joined batch [left_cols..., right_cols...] + // We need to extract the appropriate side and replace the other with nulls (or mark column) + let columns = match (join_side, join_type) { + (JoinSide::Left, JoinType::LeftMark) => { + // For LEFT mark: output is [left_cols..., mark_col] + // Batch is [left_cols..., right_cols...], extract left from beginning + // Number of left columns = output columns - 1 (mark column) + let left_col_count = output_schema.fields().len() - 1; + let mut result: Vec = batch.columns()[..left_col_count].to_vec(); + result.push(Arc::new(BooleanArray::from(vec![false; num_rows])) as ArrayRef); + result + } + (JoinSide::Right, JoinType::RightMark) => { + // For RIGHT mark: output is [right_cols..., mark_col] + // For RIGHT joins, batch is [right_cols..., left_cols...] (right comes first!) + // Extract right columns from the beginning + let right_col_count = output_schema.fields().len() - 1; // -1 for mark column + let mut result: Vec = batch.columns()[..right_col_count].to_vec(); + result.push(Arc::new(BooleanArray::from(vec![false; num_rows])) as ArrayRef); + result + } + (JoinSide::Left, _) => { + // For LEFT join: output is [left_cols..., right_cols...] + // Extract left columns, then add null right columns + let null_columns: Vec = null_schema + .fields() + .iter() + .map(|field| arrow::array::new_null_array(field.data_type(), num_rows)) + .collect(); + let left_col_count = output_schema.fields().len() - null_columns.len(); + let mut result: Vec = batch.columns()[..left_col_count].to_vec(); + result.extend(null_columns); + result + } + (JoinSide::Right, _) => { + // For RIGHT join: batch is [left_cols..., right_cols...] (same as schema) + // We want: [null_left..., actual_right...] + // Extract left columns from beginning, replace with nulls, keep right columns + let null_columns: Vec = null_schema + .fields() + .iter() + .map(|field| arrow::array::new_null_array(field.data_type(), num_rows)) + .collect(); + let left_col_count = null_columns.len(); + let mut result = null_columns; + // Extract right columns starting after left columns + result.extend_from_slice(&batch.columns()[left_col_count..]); + result + } + (JoinSide::None, _) => { + // This should not happen in normal join operations + unreachable!( + "JoinSide::None should not be used in null-joined batch creation" + ) + } + }; + + // Create the batch - don't validate nullability since outer joins can have + // null values in columns that were originally non-nullable + use arrow::array::RecordBatchOptions; + let mut options = RecordBatchOptions::new(); + options = options.with_row_count(Some(num_rows)); + Ok(RecordBatch::try_new_with_options( + Arc::clone(output_schema), + columns, + &options, + )?) +} diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/mod.rs b/datafusion/physical-plan/src/joins/sort_merge_join/mod.rs index 82f18e741409..06290ec4d090 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/mod.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/mod.rs @@ -20,6 +20,7 @@ pub use exec::SortMergeJoinExec; mod exec; +mod filter; mod metrics; mod stream; diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index b36992caf4b4..dca55c720ef6 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -33,6 +33,10 @@ use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::Relaxed; use std::task::{Context, Poll}; +use crate::joins::sort_merge_join::filter::{ + FilterMetadata, filter_record_batch_by_join_type, get_corrected_filter_mask, + get_filter_columns, needs_deferred_filtering, +}; use crate::joins::sort_merge_join::metrics::SortMergeJoinMetrics; use crate::joins::utils::{JoinFilter, compare_join_arrays}; use crate::metrics::RecordOutput; @@ -42,15 +46,13 @@ use crate::{PhysicalExpr, RecordBatchStream, SendableRecordBatchStream}; use arrow::array::{types::UInt64Type, *}; use arrow::compute::{ self, BatchCoalescer, SortOptions, concat_batches, filter_record_batch, is_not_null, - take, + take, take_arrays, }; use arrow::datatypes::{DataType, SchemaRef, TimeUnit}; -use arrow::error::ArrowError; use arrow::ipc::reader::StreamReader; use datafusion_common::config::SpillCompression; use datafusion_common::{ - DataFusionError, HashSet, JoinSide, JoinType, NullEquality, Result, exec_err, - internal_err, not_impl_err, + HashSet, JoinType, NullEquality, Result, exec_err, internal_err, not_impl_err, }; use datafusion_execution::disk_manager::RefCountedTempFile; use datafusion_execution::memory_pool::MemoryReservation; @@ -68,6 +70,8 @@ pub(super) enum SortMergeJoinState { Polling, /// Joining polled data and making output JoinOutput, + /// Emit ready data if have any and then go back to [`Self::Init`] state + EmitReadyThenInit, /// No more output Exhausted, } @@ -124,6 +128,8 @@ pub(super) struct StreamedBatch { pub join_arrays: Vec, /// Chunks of indices from buffered side (may be nulls) joined to streamed pub output_indices: Vec, + /// Total number of output rows across all chunks in `output_indices` + pub num_output_rows: usize, /// Index of currently scanned batch from buffered data pub buffered_batch_idx: Option, /// Indices that found a match for the given join filter @@ -140,6 +146,7 @@ impl StreamedBatch { idx: 0, join_arrays, output_indices: vec![], + num_output_rows: 0, buffered_batch_idx: None, join_filter_matched_idxs: HashSet::new(), } @@ -151,6 +158,7 @@ impl StreamedBatch { idx: 0, join_arrays: vec![], output_indices: vec![], + num_output_rows: 0, buffered_batch_idx: None, join_filter_matched_idxs: HashSet::new(), } @@ -158,10 +166,7 @@ impl StreamedBatch { /// Number of unfrozen output pairs in this streamed batch fn num_output_rows(&self) -> usize { - self.output_indices - .iter() - .map(|chunk| chunk.streamed_indices.len()) - .sum() + self.num_output_rows } /// Appends new pair consisting of current streamed index and `buffered_idx` @@ -171,7 +176,6 @@ impl StreamedBatch { buffered_batch_idx: Option, buffered_idx: Option, batch_size: usize, - num_unfrozen_pairs: usize, ) { // If no current chunk exists or current chunk is not for current buffered batch, // create a new chunk @@ -179,12 +183,13 @@ impl StreamedBatch { { // Compute capacity only when creating a new chunk (infrequent operation). // The capacity is the remaining space to reach batch_size. - // This should always be >= 1 since we only call this when num_unfrozen_pairs < batch_size. + // This should always be >= 1 since we only call this when num_output_rows < batch_size. debug_assert!( - batch_size > num_unfrozen_pairs, - "batch_size ({batch_size}) must be > num_unfrozen_pairs ({num_unfrozen_pairs})" + batch_size > self.num_output_rows, + "batch_size ({batch_size}) must be > num_output_rows ({})", + self.num_output_rows ); - let capacity = batch_size - num_unfrozen_pairs; + let capacity = batch_size - self.num_output_rows; self.output_indices.push(StreamedJoinedChunk { buffered_batch_idx, streamed_indices: UInt64Builder::with_capacity(capacity), @@ -201,6 +206,7 @@ impl StreamedBatch { } else { current_chunk.buffered_indices.append_null(); } + self.num_output_rows += 1; } } @@ -370,12 +376,8 @@ pub(super) struct SortMergeJoinStream { pub(super) struct JoinedRecordBatches { /// Joined batches. Each batch is already joined columns from left and right sources pub(super) joined_batches: BatchCoalescer, - /// Did each output row pass the join filter? (detect if input row found any match) - pub(super) filter_mask: BooleanBuilder, - /// Which input row (within batch) produced each output row? (for grouping by input row) - pub(super) row_indices: UInt64Builder, - /// Which input batch did each output row come from? (disambiguate row_indices) - pub(super) batch_ids: Vec, + /// Filter metadata for deferred filtering + pub(super) filter_metadata: FilterMetadata, } impl JoinedRecordBatches { @@ -398,61 +400,28 @@ impl JoinedRecordBatches { } } - /// Finishes and returns the metadata arrays, clearing the builders - /// - /// Returns (row_indices, filter_mask, batch_ids_ref) - /// Note: batch_ids is returned as a reference since it's still needed in the struct - fn finish_metadata(&mut self) -> (UInt64Array, BooleanArray, &[usize]) { - let row_indices = self.row_indices.finish(); - let filter_mask = self.filter_mask.finish(); - (row_indices, filter_mask, &self.batch_ids) - } - /// Clears batches without touching metadata (for early return when no filtering needed) fn clear_batches(&mut self, schema: &SchemaRef, batch_size: usize) { self.joined_batches = BatchCoalescer::new(Arc::clone(schema), batch_size) .with_biggest_coalesce_batch_size(Option::from(batch_size / 2)); } - /// Asserts that internal metadata arrays are consistent with each other - /// Only checks if metadata is actually being used (i.e., not all empty) - #[inline] - fn debug_assert_metadata_aligned(&self) { - // Metadata arrays should be aligned IF they're being used - // (For non-filtered joins, they may all be empty) - if self.filter_mask.len() > 0 - || self.row_indices.len() > 0 - || !self.batch_ids.is_empty() - { - debug_assert_eq!( - self.filter_mask.len(), - self.row_indices.len(), - "filter_mask and row_indices must have same length when metadata is used" - ); - debug_assert_eq!( - self.filter_mask.len(), - self.batch_ids.len(), - "filter_mask and batch_ids must have same length when metadata is used" - ); - } - } - /// Asserts that if batches is empty, metadata is also empty #[inline] fn debug_assert_empty_consistency(&self) { if self.joined_batches.is_empty() { debug_assert_eq!( - self.filter_mask.len(), + self.filter_metadata.filter_mask.len(), 0, "filter_mask should be empty when batches is empty" ); debug_assert_eq!( - self.row_indices.len(), + self.filter_metadata.row_indices.len(), 0, "row_indices should be empty when batches is empty" ); debug_assert_eq!( - self.batch_ids.len(), + self.filter_metadata.batch_ids.len(), 0, "batch_ids should be empty when batches is empty" ); @@ -473,14 +442,9 @@ impl JoinedRecordBatches { let num_rows = batch.num_rows(); - self.filter_mask.append_nulls(num_rows); - self.row_indices.append_nulls(num_rows); - self.batch_ids.resize( - self.batch_ids.len() + num_rows, - 0, // batch_id = 0 for null-joined rows - ); + self.filter_metadata.append_nulls(num_rows); - self.debug_assert_metadata_aligned(); + self.filter_metadata.debug_assert_metadata_aligned(); self.joined_batches .push_batch(batch) .expect("Failed to push batch to BatchCoalescer"); @@ -525,13 +489,13 @@ impl JoinedRecordBatches { "row_indices and filter_mask must have same length" ); - // For Full joins, we keep the pre_mask (with nulls), for others we keep the cleaned mask - self.filter_mask.extend(filter_mask); - self.row_indices.extend(row_indices); - self.batch_ids - .resize(self.batch_ids.len() + row_indices.len(), streamed_batch_id); + self.filter_metadata.append_filter_metadata( + row_indices, + filter_mask, + streamed_batch_id, + ); - self.debug_assert_metadata_aligned(); + self.filter_metadata.debug_assert_metadata_aligned(); self.joined_batches .push_batch(batch) .expect("Failed to push batch to BatchCoalescer"); @@ -551,9 +515,7 @@ impl JoinedRecordBatches { fn clear(&mut self, schema: &SchemaRef, batch_size: usize) { self.joined_batches = BatchCoalescer::new(Arc::clone(schema), batch_size) .with_biggest_coalesce_batch_size(Option::from(batch_size / 2)); - self.batch_ids.clear(); - self.filter_mask = BooleanBuilder::new(); - self.row_indices = UInt64Builder::new(); + self.filter_metadata = FilterMetadata::new(); self.debug_assert_empty_consistency(); } } @@ -563,199 +525,6 @@ impl RecordBatchStream for SortMergeJoinStream { } } -/// True if next index refers to either: -/// - another batch id -/// - another row index within same batch id -/// - end of row indices -#[inline(always)] -fn last_index_for_row( - row_index: usize, - indices: &UInt64Array, - batch_ids: &[usize], - indices_len: usize, -) -> bool { - debug_assert_eq!( - indices.len(), - indices_len, - "indices.len() should match indices_len parameter" - ); - debug_assert_eq!( - batch_ids.len(), - indices_len, - "batch_ids.len() should match indices_len" - ); - debug_assert!( - row_index < indices_len, - "row_index {row_index} should be < indices_len {indices_len}", - ); - - row_index == indices_len - 1 - || batch_ids[row_index] != batch_ids[row_index + 1] - || indices.value(row_index) != indices.value(row_index + 1) -} - -// Returns a corrected boolean bitmask for the given join type -// Values in the corrected bitmask can be: true, false, null -// `true` - the row found its match and sent to the output -// `null` - the row ignored, no output -// `false` - the row sent as NULL joined row -pub(super) fn get_corrected_filter_mask( - join_type: JoinType, - row_indices: &UInt64Array, - batch_ids: &[usize], - filter_mask: &BooleanArray, - expected_size: usize, -) -> Option { - let row_indices_length = row_indices.len(); - let mut corrected_mask: BooleanBuilder = - BooleanBuilder::with_capacity(row_indices_length); - let mut seen_true = false; - - match join_type { - JoinType::Left | JoinType::Right => { - for i in 0..row_indices_length { - let last_index = - last_index_for_row(i, row_indices, batch_ids, row_indices_length); - if filter_mask.value(i) { - seen_true = true; - corrected_mask.append_value(true); - } else if seen_true || !filter_mask.value(i) && !last_index { - corrected_mask.append_null(); // to be ignored and not set to output - } else { - corrected_mask.append_value(false); // to be converted to null joined row - } - - if last_index { - seen_true = false; - } - } - - // Generate null joined rows for records which have no matching join key - corrected_mask.append_n(expected_size - corrected_mask.len(), false); - Some(corrected_mask.finish()) - } - JoinType::LeftMark | JoinType::RightMark => { - for i in 0..row_indices_length { - let last_index = - last_index_for_row(i, row_indices, batch_ids, row_indices_length); - if filter_mask.value(i) && !seen_true { - seen_true = true; - corrected_mask.append_value(true); - } else if seen_true || !filter_mask.value(i) && !last_index { - corrected_mask.append_null(); // to be ignored and not set to output - } else { - corrected_mask.append_value(false); // to be converted to null joined row - } - - if last_index { - seen_true = false; - } - } - - // Generate null joined rows for records which have no matching join key - corrected_mask.append_n(expected_size - corrected_mask.len(), false); - Some(corrected_mask.finish()) - } - JoinType::LeftSemi | JoinType::RightSemi => { - for i in 0..row_indices_length { - let last_index = - last_index_for_row(i, row_indices, batch_ids, row_indices_length); - if filter_mask.value(i) && !seen_true { - seen_true = true; - corrected_mask.append_value(true); - } else { - corrected_mask.append_null(); // to be ignored and not set to output - } - - if last_index { - seen_true = false; - } - } - - Some(corrected_mask.finish()) - } - JoinType::LeftAnti | JoinType::RightAnti => { - for i in 0..row_indices_length { - let last_index = - last_index_for_row(i, row_indices, batch_ids, row_indices_length); - - if filter_mask.value(i) { - seen_true = true; - } - - if last_index { - if !seen_true { - corrected_mask.append_value(true); - } else { - corrected_mask.append_null(); - } - - seen_true = false; - } else { - corrected_mask.append_null(); - } - } - // Generate null joined rows for records which have no matching join key, - // for LeftAnti non-matched considered as true - corrected_mask.append_n(expected_size - corrected_mask.len(), true); - Some(corrected_mask.finish()) - } - JoinType::Full => { - let mut mask: Vec> = vec![Some(true); row_indices_length]; - let mut last_true_idx = 0; - let mut first_row_idx = 0; - let mut seen_false = false; - - for i in 0..row_indices_length { - let last_index = - last_index_for_row(i, row_indices, batch_ids, row_indices_length); - let val = filter_mask.value(i); - let is_null = filter_mask.is_null(i); - - if val { - // memoize the first seen matched row - if !seen_true { - last_true_idx = i; - } - seen_true = true; - } - - if is_null || val { - mask[i] = Some(true); - } else if !is_null && !val && (seen_true || seen_false) { - mask[i] = None; - } else { - mask[i] = Some(false); - } - - if !is_null && !val { - seen_false = true; - } - - if last_index { - // If the left row seen as true its needed to output it once - // To do that we mark all other matches for same row as null to avoid the output - if seen_true { - #[expect(clippy::needless_range_loop)] - for j in first_row_idx..last_true_idx { - mask[j] = None; - } - } - - seen_true = false; - seen_false = false; - last_true_idx = 0; - first_row_idx = i + 1; - } - } - - Some(BooleanArray::from(mask)) - } - // Only outer joins needs to keep track of processed rows and apply corrected filter mask - _ => None, - } -} - impl Stream for SortMergeJoinStream { type Item = Result; @@ -778,7 +547,10 @@ impl Stream for SortMergeJoinStream { match self.current_ordering { Ordering::Less | Ordering::Equal => { if !streamed_exhausted { - if self.needs_deferred_filtering() { + if needs_deferred_filtering( + &self.filter, + self.join_type, + ) { match self.process_filtered_batches()? { Poll::Ready(Some(batch)) => { return Poll::Ready(Some(Ok(batch))); @@ -830,22 +602,56 @@ impl Stream for SortMergeJoinStream { self.current_ordering = self.compare_streamed_buffered()?; self.state = SortMergeJoinState::JoinOutput; } + SortMergeJoinState::EmitReadyThenInit => { + // If have data to emit, emit it and if no more, change to next + + // Verify metadata alignment before checking if we have batches to output + self.joined_record_batches + .filter_metadata + .debug_assert_metadata_aligned(); + + // For filtered joins, skip output and let Init state handle it + if needs_deferred_filtering(&self.filter, self.join_type) { + self.state = SortMergeJoinState::Init; + continue; + } + + // For non-filtered joins, only output if we have a completed batch + // (opportunistic output when target batch size is reached) + if self + .joined_record_batches + .joined_batches + .has_completed_batch() + { + let record_batch = self + .joined_record_batches + .joined_batches + .next_completed_batch() + .expect("has_completed_batch was true"); + (&record_batch) + .record_output(&self.join_metrics.baseline_metrics()); + return Poll::Ready(Some(Ok(record_batch))); + } + self.state = SortMergeJoinState::Init; + } SortMergeJoinState::JoinOutput => { self.join_partial()?; if self.num_unfrozen_pairs() < self.batch_size { if self.buffered_data.scanning_finished() { self.buffered_data.scanning_reset(); - self.state = SortMergeJoinState::Init; + self.state = SortMergeJoinState::EmitReadyThenInit; } } else { self.freeze_all()?; // Verify metadata alignment before checking if we have batches to output - self.joined_record_batches.debug_assert_metadata_aligned(); + self.joined_record_batches + .filter_metadata + .debug_assert_metadata_aligned(); // For filtered joins, skip output and let Init state handle it - if self.needs_deferred_filtering() { + if needs_deferred_filtering(&self.filter, self.join_type) { continue; } @@ -872,10 +678,12 @@ impl Stream for SortMergeJoinStream { self.freeze_all()?; // Verify metadata alignment before final output - self.joined_record_batches.debug_assert_metadata_aligned(); + self.joined_record_batches + .filter_metadata + .debug_assert_metadata_aligned(); // For filtered joins, must concat and filter ALL data at once - if self.needs_deferred_filtering() + if needs_deferred_filtering(&self.filter, self.join_type) && !self.joined_record_batches.joined_batches.is_empty() { let record_batch = self.filter_joined_batch()?; @@ -975,9 +783,7 @@ impl SortMergeJoinStream { joined_record_batches: JoinedRecordBatches { joined_batches: BatchCoalescer::new(Arc::clone(&schema), batch_size) .with_biggest_coalesce_batch_size(Option::from(batch_size / 2)), - filter_mask: BooleanBuilder::new(), - row_indices: UInt64Builder::new(), - batch_ids: vec![], + filter_metadata: FilterMetadata::new(), }, output: BatchCoalescer::new(schema, batch_size) .with_biggest_coalesce_batch_size(Option::from(batch_size / 2)), @@ -996,26 +802,6 @@ impl SortMergeJoinStream { self.streamed_batch.num_output_rows() } - /// Returns true if this join needs deferred filtering - /// - /// Deferred filtering is needed when a filter exists and the join type requires - /// ensuring each input row produces at least one output row (or exactly one for semi). - fn needs_deferred_filtering(&self) -> bool { - self.filter.is_some() - && matches!( - self.join_type, - JoinType::Left - | JoinType::LeftSemi - | JoinType::LeftMark - | JoinType::Right - | JoinType::RightSemi - | JoinType::RightMark - | JoinType::LeftAnti - | JoinType::RightAnti - | JoinType::Full - ) - } - /// Process accumulated batches for filtered joins /// /// Freezes unfrozen pairs, applies deferred filtering, and outputs if ready. @@ -1023,7 +809,9 @@ impl SortMergeJoinStream { fn process_filtered_batches(&mut self) -> Poll>> { self.freeze_all()?; - self.joined_record_batches.debug_assert_metadata_aligned(); + self.joined_record_batches + .filter_metadata + .debug_assert_metadata_aligned(); if !self.joined_record_batches.joined_batches.is_empty() { let out_filtered_batch = self.filter_joined_batch()?; @@ -1348,13 +1136,10 @@ impl SortMergeJoinStream { let scanning_idx = self.buffered_data.scanning_idx(); if join_streamed { // Join streamed row and buffered row - // Pass batch_size and num_unfrozen_pairs to compute capacity only when - // creating a new chunk (when buffered_batch_idx changes), not on every iteration. self.streamed_batch.append_output_pair( Some(self.buffered_data.scanning_batch_idx), Some(scanning_idx), self.batch_size, - self.num_unfrozen_pairs(), ); } else { // Join nulls and buffered row for FULL join @@ -1380,13 +1165,10 @@ impl SortMergeJoinStream { // For Mark join we store a dummy id to indicate the row has a match let scanning_idx = mark_row_as_match.then_some(0); - // Pass batch_size=1 and num_unfrozen_pairs=0 to get capacity of 1, - // since we only append a single null-joined pair here (not in a loop). self.streamed_batch.append_output_pair( scanning_batch_idx, scanning_idx, - 1, - 0, + self.batch_size, ); self.buffered_data.scanning_finish(); self.streamed_joined = true; @@ -1399,7 +1181,9 @@ impl SortMergeJoinStream { self.freeze_streamed()?; // After freezing, metadata should be aligned - self.joined_record_batches.debug_assert_metadata_aligned(); + self.joined_record_batches + .filter_metadata + .debug_assert_metadata_aligned(); Ok(()) } @@ -1414,7 +1198,9 @@ impl SortMergeJoinStream { self.freeze_buffered(1)?; // After freezing, metadata should be aligned - self.joined_record_batches.debug_assert_metadata_aligned(); + self.joined_record_batches + .filter_metadata + .debug_assert_metadata_aligned(); Ok(()) } @@ -1490,13 +1276,19 @@ impl SortMergeJoinStream { continue; } - let mut left_columns = self - .streamed_batch - .batch - .columns() - .iter() - .map(|column| take(column, &left_indices, None)) - .collect::, ArrowError>>()?; + let mut left_columns = if let Some(range) = is_contiguous_range(&left_indices) + { + // When indices form a contiguous range (common for the streamed + // side which advances sequentially), use zero-copy slice instead + // of the O(n) take kernel. + self.streamed_batch + .batch + .slice(range.start, range.len()) + .columns() + .to_vec() + } else { + take_arrays(self.streamed_batch.batch.columns(), &left_indices, None)? + }; // The row indices of joined buffered batch let right_indices: UInt64Array = chunk.buffered_indices.finish(); @@ -1541,7 +1333,7 @@ impl SortMergeJoinStream { &right_indices, )?; - get_filter_column(&self.filter, &left_columns, &right_cols) + get_filter_columns(&self.filter, &left_columns, &right_cols) } else if matches!( self.join_type, JoinType::RightAnti | JoinType::RightSemi | JoinType::RightMark @@ -1552,12 +1344,12 @@ impl SortMergeJoinStream { &right_indices, )?; - get_filter_column(&self.filter, &right_cols, &left_columns) + get_filter_columns(&self.filter, &right_cols, &left_columns) } else { - get_filter_column(&self.filter, &left_columns, &right_columns) + get_filter_columns(&self.filter, &left_columns, &right_columns) } } else { - get_filter_column(&self.filter, &right_columns, &left_columns) + get_filter_columns(&self.filter, &right_columns, &left_columns) } } else { // This chunk is totally for null joined rows (outer join), we don't need to apply join filter. @@ -1673,17 +1465,20 @@ impl SortMergeJoinStream { } self.streamed_batch.output_indices.clear(); + self.streamed_batch.num_output_rows = 0; Ok(()) } fn filter_joined_batch(&mut self) -> Result { // Metadata should be aligned before processing - self.joined_record_batches.debug_assert_metadata_aligned(); + self.joined_record_batches + .filter_metadata + .debug_assert_metadata_aligned(); let record_batch = self.joined_record_batches.concat_batches(&self.schema)?; let (mut out_indices, mut out_mask, mut batch_ids) = - self.joined_record_batches.finish_metadata(); + self.joined_record_batches.filter_metadata.finish_metadata(); let default_batch_ids = vec![0; record_batch.num_rows()]; // If only nulls come in and indices sizes doesn't match with expected record batch count @@ -1754,139 +1549,14 @@ impl SortMergeJoinStream { record_batch: &RecordBatch, corrected_mask: &BooleanArray, ) -> Result { - // Corrected mask should have length matching or exceeding record_batch rows - // (for outer joins it may be longer to include null-joined rows) - debug_assert!( - corrected_mask.len() >= record_batch.num_rows(), - "corrected_mask length ({}) should be >= record_batch rows ({})", - corrected_mask.len(), - record_batch.num_rows() - ); - - let mut filtered_record_batch = - filter_record_batch(record_batch, corrected_mask)?; - let left_columns_length = self.streamed_schema.fields.len(); - let right_columns_length = self.buffered_schema.fields.len(); - - if matches!( - self.join_type, - JoinType::Left | JoinType::LeftMark | JoinType::Right | JoinType::RightMark - ) { - let null_mask = compute::not(corrected_mask)?; - let null_joined_batch = filter_record_batch(record_batch, &null_mask)?; - - let mut right_columns = create_unmatched_columns( - self.join_type, - &self.buffered_schema, - null_joined_batch.num_rows(), - ); - - let columns = match self.join_type { - JoinType::Right => { - // The first columns are the right columns. - let left_columns = null_joined_batch - .columns() - .iter() - .skip(right_columns_length) - .cloned() - .collect::>(); - - right_columns.extend(left_columns); - right_columns - } - JoinType::Left | JoinType::LeftMark | JoinType::RightMark => { - // The first columns are the left columns. - let mut left_columns = null_joined_batch - .columns() - .iter() - .take(left_columns_length) - .cloned() - .collect::>(); - - left_columns.extend(right_columns); - left_columns - } - _ => exec_err!("Did not expect join type {}", self.join_type)?, - }; - - // Push the streamed/buffered batch joined nulls to the output - let null_joined_streamed_batch = - RecordBatch::try_new(Arc::clone(&self.schema), columns)?; - - filtered_record_batch = concat_batches( - &self.schema, - &[filtered_record_batch, null_joined_streamed_batch], - )?; - } else if matches!( + let filtered_record_batch = filter_record_batch_by_join_type( + record_batch, + corrected_mask, self.join_type, - JoinType::LeftSemi - | JoinType::LeftAnti - | JoinType::RightAnti - | JoinType::RightSemi - ) { - let output_column_indices = (0..left_columns_length).collect::>(); - filtered_record_batch = - filtered_record_batch.project(&output_column_indices)?; - } else if matches!(self.join_type, JoinType::Full) - && corrected_mask.false_count() > 0 - { - // Find rows which joined by key but Filter predicate evaluated as false - let joined_filter_not_matched_mask = compute::not(corrected_mask)?; - let joined_filter_not_matched_batch = - filter_record_batch(record_batch, &joined_filter_not_matched_mask)?; - - // Add left unmatched rows adding the right side as nulls - let right_null_columns = self - .buffered_schema - .fields() - .iter() - .map(|f| { - new_null_array( - f.data_type(), - joined_filter_not_matched_batch.num_rows(), - ) - }) - .collect::>(); - - let mut result_joined = joined_filter_not_matched_batch - .columns() - .iter() - .take(left_columns_length) - .cloned() - .collect::>(); - - result_joined.extend(right_null_columns); - - let left_null_joined_batch = - RecordBatch::try_new(Arc::clone(&self.schema), result_joined)?; - - // Add right unmatched rows adding the left side as nulls - let mut result_joined = self - .streamed_schema - .fields() - .iter() - .map(|f| { - new_null_array( - f.data_type(), - joined_filter_not_matched_batch.num_rows(), - ) - }) - .collect::>(); - - let right_data = joined_filter_not_matched_batch - .columns() - .iter() - .skip(left_columns_length) - .cloned() - .collect::>(); - - result_joined.extend(right_data); - - filtered_record_batch = concat_batches( - &self.schema, - &[filtered_record_batch, left_null_joined_batch], - )?; - } + &self.schema, + &self.streamed_schema, + &self.buffered_schema, + )?; self.joined_record_batches .clear(&self.schema, self.batch_size); @@ -1911,36 +1581,6 @@ fn create_unmatched_columns( } } -/// Gets the arrays which join filters are applied on. -fn get_filter_column( - join_filter: &Option, - streamed_columns: &[ArrayRef], - buffered_columns: &[ArrayRef], -) -> Vec { - let mut filter_columns = vec![]; - - if let Some(f) = join_filter { - let left_columns = f - .column_indices() - .iter() - .filter(|col_index| col_index.side == JoinSide::Left) - .map(|i| Arc::clone(&streamed_columns[i.index])) - .collect::>(); - - let right_columns = f - .column_indices() - .iter() - .filter(|col_index| col_index.side == JoinSide::Right) - .map(|i| Arc::clone(&buffered_columns[i.index])) - .collect::>(); - - filter_columns.extend(left_columns); - filter_columns.extend(right_columns); - } - - filter_columns -} - fn produce_buffered_null_batch( schema: &SchemaRef, streamed_schema: &SchemaRef, @@ -1970,6 +1610,30 @@ fn produce_buffered_null_batch( )?)) } +/// Checks if a `UInt64Array` contains a contiguous ascending range (e.g. \[3,4,5,6\]). +/// Returns `Some(start..start+len)` if so, `None` otherwise. +/// This allows replacing an O(n) `take` with an O(1) `slice`. +#[inline] +fn is_contiguous_range(indices: &UInt64Array) -> Option> { + if indices.is_empty() || indices.null_count() > 0 { + return None; + } + let values = indices.values(); + let start = values[0]; + let len = values.len() as u64; + // Quick rejection: if last element doesn't match expected, not contiguous + if values[values.len() - 1] != start + len - 1 { + return None; + } + // Verify every element is sequential (handles duplicates and gaps) + for i in 1..values.len() { + if values[i] != start + i as u64 { + return None; + } + } + Some(start as usize..(start + len) as usize) +} + /// Get `buffered_indices` rows for `buffered_data[buffered_batch_idx]` by specific column indices #[inline(always)] fn fetch_right_columns_by_idxs( @@ -1990,12 +1654,16 @@ fn fetch_right_columns_from_batch_by_idxs( ) -> Result> { match &buffered_batch.batch { // In memory batch - BufferedBatchState::InMemory(batch) => Ok(batch - .columns() - .iter() - .map(|column| take(column, &buffered_indices, None)) - .collect::, ArrowError>>() - .map_err(Into::::into)?), + // In memory batch + BufferedBatchState::InMemory(batch) => { + // When indices form a contiguous range (common in SMJ since the + // buffered side is scanned sequentially), use zero-copy slice. + if let Some(range) = is_contiguous_range(buffered_indices) { + Ok(batch.slice(range.start, range.len()).columns().to_vec()) + } else { + Ok(take_arrays(batch.columns(), buffered_indices, None)?) + } + } // If the batch was spilled to disk, less likely BufferedBatchState::Spilled(spill_file) => { let mut buffered_cols: Vec = diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs index d0bcc79636f7..4329abdd522d 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs @@ -24,42 +24,44 @@ //! //! Add relevant tests under the specified sections. -use std::sync::Arc; - +use crate::joins::utils::{ColumnIndex, JoinFilter, JoinOn}; +use crate::joins::{HashJoinExec, PartitionMode, SortMergeJoinExec}; +use crate::test::TestMemoryExec; +use crate::test::exec::BarrierExec; +use crate::test::{build_table_i32, build_table_i32_two_cols}; +use crate::{ExecutionPlan, common}; +use crate::{ + expressions::Column, joins::sort_merge_join::filter::get_corrected_filter_mask, + joins::sort_merge_join::stream::JoinedRecordBatches, +}; use arrow::array::{ BinaryArray, BooleanArray, Date32Array, Date64Array, FixedSizeBinaryArray, Int32Array, RecordBatch, UInt64Array, - builder::{BooleanBuilder, UInt64Builder}, }; use arrow::compute::{BatchCoalescer, SortOptions, filter_record_batch}; use arrow::datatypes::{DataType, Field, Schema}; - +use arrow_ord::sort::SortColumn; +use arrow_schema::SchemaRef; use datafusion_common::JoinType::*; use datafusion_common::{ - JoinSide, + JoinSide, internal_err, test_util::{batches_to_sort_string, batches_to_string}, }; use datafusion_common::{ JoinType, NullEquality, Result, assert_batches_eq, assert_contains, }; -use datafusion_execution::TaskContext; +use datafusion_common_runtime::JoinSet; use datafusion_execution::config::SessionConfig; use datafusion_execution::disk_manager::{DiskManagerBuilder, DiskManagerMode}; use datafusion_execution::runtime_env::RuntimeEnvBuilder; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::BinaryExpr; +use futures::StreamExt; use insta::{allow_duplicates, assert_snapshot}; - -use crate::{ - expressions::Column, - joins::sort_merge_join::stream::{JoinedRecordBatches, get_corrected_filter_mask}, -}; - -use crate::joins::SortMergeJoinExec; -use crate::joins::utils::{ColumnIndex, JoinFilter, JoinOn}; -use crate::test::TestMemoryExec; -use crate::test::{build_table_i32, build_table_i32_two_cols}; -use crate::{ExecutionPlan, common}; +use itertools::Itertools; +use std::sync::Arc; +use std::task::Poll; fn build_table( a: (&str, &Vec), @@ -2375,9 +2377,7 @@ fn build_joined_record_batches() -> Result { let mut batches = JoinedRecordBatches { joined_batches: BatchCoalescer::new(Arc::clone(&schema), 8192), - filter_mask: BooleanBuilder::new(), - row_indices: UInt64Builder::new(), - batch_ids: vec![], + filter_metadata: crate::joins::sort_merge_join::filter::FilterMetadata::new(), }; // Insert already prejoined non-filtered rows @@ -2432,44 +2432,73 @@ fn build_joined_record_batches() -> Result { )?)?; let streamed_indices = vec![0, 0]; - batches.batch_ids.extend(vec![0; streamed_indices.len()]); batches + .filter_metadata + .batch_ids + .extend(vec![0; streamed_indices.len()]); + batches + .filter_metadata .row_indices .extend(&UInt64Array::from(streamed_indices)); let streamed_indices = vec![1]; - batches.batch_ids.extend(vec![0; streamed_indices.len()]); batches + .filter_metadata + .batch_ids + .extend(vec![0; streamed_indices.len()]); + batches + .filter_metadata .row_indices .extend(&UInt64Array::from(streamed_indices)); let streamed_indices = vec![0, 0]; - batches.batch_ids.extend(vec![1; streamed_indices.len()]); batches + .filter_metadata + .batch_ids + .extend(vec![1; streamed_indices.len()]); + batches + .filter_metadata .row_indices .extend(&UInt64Array::from(streamed_indices)); let streamed_indices = vec![0]; - batches.batch_ids.extend(vec![2; streamed_indices.len()]); batches + .filter_metadata + .batch_ids + .extend(vec![2; streamed_indices.len()]); + batches + .filter_metadata .row_indices .extend(&UInt64Array::from(streamed_indices)); let streamed_indices = vec![0, 0]; - batches.batch_ids.extend(vec![3; streamed_indices.len()]); batches + .filter_metadata + .batch_ids + .extend(vec![3; streamed_indices.len()]); + batches + .filter_metadata .row_indices .extend(&UInt64Array::from(streamed_indices)); batches + .filter_metadata .filter_mask .extend(&BooleanArray::from(vec![true, false])); - batches.filter_mask.extend(&BooleanArray::from(vec![true])); batches + .filter_metadata + .filter_mask + .extend(&BooleanArray::from(vec![true])); + batches + .filter_metadata .filter_mask .extend(&BooleanArray::from(vec![false, true])); - batches.filter_mask.extend(&BooleanArray::from(vec![false])); batches + .filter_metadata + .filter_mask + .extend(&BooleanArray::from(vec![false])); + batches + .filter_metadata .filter_mask .extend(&BooleanArray::from(vec![false, false])); @@ -2482,8 +2511,8 @@ async fn test_left_outer_join_filtered_mask() -> Result<()> { let schema = joined_batches.joined_batches.schema(); let output = joined_batches.concat_batches(&schema)?; - let out_mask = joined_batches.filter_mask.finish(); - let out_indices = joined_batches.row_indices.finish(); + let out_mask = joined_batches.filter_metadata.filter_mask.finish(); + let out_indices = joined_batches.filter_metadata.row_indices.finish(); assert_eq!( get_corrected_filter_mask( @@ -2620,7 +2649,7 @@ async fn test_left_outer_join_filtered_mask() -> Result<()> { let corrected_mask = get_corrected_filter_mask( Left, &out_indices, - &joined_batches.batch_ids, + &joined_batches.filter_metadata.batch_ids, &out_mask, output.num_rows(), ) @@ -2689,8 +2718,8 @@ async fn test_semi_join_filtered_mask() -> Result<()> { let schema = joined_batches.joined_batches.schema(); let output = joined_batches.concat_batches(&schema)?; - let out_mask = joined_batches.filter_mask.finish(); - let out_indices = joined_batches.row_indices.finish(); + let out_mask = joined_batches.filter_metadata.filter_mask.finish(); + let out_indices = joined_batches.filter_metadata.row_indices.finish(); assert_eq!( get_corrected_filter_mask( @@ -2791,7 +2820,7 @@ async fn test_semi_join_filtered_mask() -> Result<()> { let corrected_mask = get_corrected_filter_mask( join_type, &out_indices, - &joined_batches.batch_ids, + &joined_batches.filter_metadata.batch_ids, &out_mask, output.num_rows(), ) @@ -2864,8 +2893,8 @@ async fn test_anti_join_filtered_mask() -> Result<()> { let schema = joined_batches.joined_batches.schema(); let output = joined_batches.concat_batches(&schema)?; - let out_mask = joined_batches.filter_mask.finish(); - let out_indices = joined_batches.row_indices.finish(); + let out_mask = joined_batches.filter_metadata.filter_mask.finish(); + let out_indices = joined_batches.filter_metadata.row_indices.finish(); assert_eq!( get_corrected_filter_mask( @@ -2966,7 +2995,7 @@ async fn test_anti_join_filtered_mask() -> Result<()> { let corrected_mask = get_corrected_filter_mask( join_type, &out_indices, - &joined_batches.batch_ids, + &joined_batches.filter_metadata.batch_ids, &out_mask, output.num_rows(), ) @@ -3104,6 +3133,419 @@ fn test_partition_statistics() -> Result<()> { Ok(()) } +fn build_batches( + a: (&str, &[Vec]), + b: (&str, &[Vec]), + c: (&str, &[Vec]), +) -> (Vec, SchemaRef) { + assert_eq!(a.1.len(), b.1.len()); + let mut batches = vec![]; + + let schema = Arc::new(Schema::new(vec![ + Field::new(a.0, DataType::Boolean, false), + Field::new(b.0, DataType::Int32, false), + Field::new(c.0, DataType::Int32, false), + ])); + + for i in 0..a.1.len() { + batches.push( + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(BooleanArray::from(a.1[i].clone())), + Arc::new(Int32Array::from(b.1[i].clone())), + Arc::new(Int32Array::from(c.1[i].clone())), + ], + ) + .unwrap(), + ); + } + let schema = batches[0].schema(); + (batches, schema) +} + +fn build_batched_finish_barrier_table( + a: (&str, &[Vec]), + b: (&str, &[Vec]), + c: (&str, &[Vec]), +) -> (Arc, Arc) { + let (batches, schema) = build_batches(a, b, c); + + let memory_exec = TestMemoryExec::try_new_exec( + std::slice::from_ref(&batches), + Arc::clone(&schema), + None, + ) + .unwrap(); + + let barrier_exec = Arc::new( + BarrierExec::new(vec![batches], schema) + .with_log(false) + .without_start_barrier() + .with_finish_barrier(), + ); + + (barrier_exec, memory_exec) +} + +/// Concat and sort batches by all the columns to make sure we can compare them with different join +fn prepare_record_batches_for_cmp(output: Vec) -> RecordBatch { + let output_batch = arrow::compute::concat_batches(output[0].schema_ref(), &output) + .expect("failed to concat batches"); + + // Sort on all columns to make sure we have a deterministic order for the assertion + let sort_columns = output_batch + .columns() + .iter() + .map(|c| SortColumn { + values: Arc::clone(c), + options: None, + }) + .collect::>(); + + let sorted_columns = + arrow::compute::lexsort(&sort_columns, None).expect("failed to sort"); + + RecordBatch::try_new(output_batch.schema(), sorted_columns) + .expect("failed to create batch") +} + +#[expect(clippy::too_many_arguments)] +async fn join_get_stream_and_get_expected( + left: Arc, + right: Arc, + oracle_left: Arc, + oracle_right: Arc, + on: JoinOn, + join_type: JoinType, + filter: Option, + batch_size: usize, +) -> Result<(SendableRecordBatchStream, RecordBatch)> { + let sort_options = vec![SortOptions::default(); on.len()]; + let null_equality = NullEquality::NullEqualsNothing; + let task_ctx = Arc::new( + TaskContext::default() + .with_session_config(SessionConfig::default().with_batch_size(batch_size)), + ); + + let expected_output = { + let oracle = HashJoinExec::try_new( + oracle_left, + oracle_right, + on.clone(), + filter.clone(), + &join_type, + None, + PartitionMode::Partitioned, + null_equality, + )?; + + let stream = oracle.execute(0, Arc::clone(&task_ctx))?; + + let batches = common::collect(stream).await?; + + prepare_record_batches_for_cmp(batches) + }; + + let join = SortMergeJoinExec::try_new( + left, + right, + on, + filter, + join_type, + sort_options, + null_equality, + )?; + + let stream = join.execute(0, task_ctx)?; + + Ok((stream, expected_output)) +} + +fn generate_data_for_emit_early_test( + batch_size: usize, + number_of_batches: usize, + join_type: JoinType, +) -> ( + Arc, + Arc, + Arc, + Arc, +) { + let number_of_rows_per_batch = number_of_batches * batch_size; + // Prepare data + let left_a1 = (0..number_of_rows_per_batch as i32) + .chunks(batch_size) + .into_iter() + .map(|chunk| chunk.collect::>()) + .collect::>(); + let left_b1 = (0..1000000) + .filter(|item| { + match join_type { + LeftAnti | RightAnti => { + let remainder = item % (batch_size as i32); + + // Make sure to have one that match and one that don't + remainder == 0 || remainder == 1 + } + // Have at least 1 that is not matching + _ => item % batch_size as i32 != 0, + } + }) + .take(number_of_rows_per_batch) + .chunks(batch_size) + .into_iter() + .map(|chunk| chunk.collect::>()) + .collect::>(); + + let left_bool_col1 = left_a1 + .clone() + .into_iter() + .map(|b| { + b.into_iter() + // Mostly true but have some false that not overlap with the right column + .map(|a| a % (batch_size as i32) != (batch_size as i32) - 2) + .collect::>() + }) + .collect::>(); + + let (left, left_memory) = build_batched_finish_barrier_table( + ("bool_col1", left_bool_col1.as_slice()), + ("b1", left_b1.as_slice()), + ("a1", left_a1.as_slice()), + ); + + let right_a2 = (0..number_of_rows_per_batch as i32) + .map(|item| item * 11) + .chunks(batch_size) + .into_iter() + .map(|chunk| chunk.collect::>()) + .collect::>(); + let right_b1 = (0..1000000) + .filter(|item| { + match join_type { + LeftAnti | RightAnti => { + let remainder = item % (batch_size as i32); + + // Make sure to have one that match and one that don't + remainder == 1 || remainder == 2 + } + // Have at least 1 that is not matching + _ => item % batch_size as i32 != 1, + } + }) + .take(number_of_rows_per_batch) + .chunks(batch_size) + .into_iter() + .map(|chunk| chunk.collect::>()) + .collect::>(); + let right_bool_col2 = right_a2 + .clone() + .into_iter() + .map(|b| { + b.into_iter() + // Mostly true but have some false that not overlap with the left column + .map(|a| a % (batch_size as i32) != (batch_size as i32) - 1) + .collect::>() + }) + .collect::>(); + + let (right, right_memory) = build_batched_finish_barrier_table( + ("bool_col2", right_bool_col2.as_slice()), + ("b1", right_b1.as_slice()), + ("a2", right_a2.as_slice()), + ); + + (left, right, left_memory, right_memory) +} + +#[tokio::test] +async fn test_should_emit_early_when_have_enough_data_to_emit() -> Result<()> { + for with_filtering in [false, true] { + let join_types = vec![ + Inner, Left, Right, RightSemi, Full, LeftSemi, LeftAnti, LeftMark, RightMark, + ]; + const BATCH_SIZE: usize = 10; + for join_type in join_types { + for output_batch_size in [ + BATCH_SIZE / 3, + BATCH_SIZE / 2, + BATCH_SIZE, + BATCH_SIZE * 2, + BATCH_SIZE * 3, + ] { + // Make sure the number of batches is enough for all join type to emit some output + let number_of_batches = if output_batch_size <= BATCH_SIZE { + 100 + } else { + // Have enough batches + (output_batch_size * 100) / BATCH_SIZE + }; + + let (left, right, left_memory, right_memory) = + generate_data_for_emit_early_test( + BATCH_SIZE, + number_of_batches, + join_type, + ); + + let on = vec![( + Arc::new(Column::new_with_schema("b1", &left.schema())?) as _, + Arc::new(Column::new_with_schema("b1", &right.schema())?) as _, + )]; + + let join_filter = if with_filtering { + let filter = JoinFilter::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("bool_col1", 0)), + Operator::And, + Arc::new(Column::new("bool_col2", 1)), + )), + vec![ + ColumnIndex { + index: 0, + side: JoinSide::Left, + }, + ColumnIndex { + index: 0, + side: JoinSide::Right, + }, + ], + Arc::new(Schema::new(vec![ + Field::new("bool_col1", DataType::Boolean, true), + Field::new("bool_col2", DataType::Boolean, true), + ])), + ); + Some(filter) + } else { + None + }; + + // select * + // from t1 + // right join t2 on t1.b1 = t2.b1 and t1.bool_col1 AND t2.bool_col2 + let (mut output_stream, expected) = join_get_stream_and_get_expected( + Arc::clone(&left) as Arc, + Arc::clone(&right) as Arc, + left_memory as Arc, + right_memory as Arc, + on, + join_type, + join_filter, + output_batch_size, + ) + .await?; + + let (output_batched, output_batches_after_finish) = + consume_stream_until_finish_barrier_reached(left, right, &mut output_stream).await.unwrap_or_else(|e| panic!("Failed to consume stream for join type: '{join_type}' and with filtering '{with_filtering}': {e:?}")); + + // It should emit more than that, but we are being generous + // and to make sure the test pass for all + const MINIMUM_OUTPUT_BATCHES: usize = 5; + assert!( + MINIMUM_OUTPUT_BATCHES <= number_of_batches / 5, + "Make sure that the minimum output batches is realistic" + ); + // Test to make sure that we are not waiting for input to be fully consumed to emit some output + assert!( + output_batched.len() >= MINIMUM_OUTPUT_BATCHES, + "[Sort Merge Join {join_type}] Stream must have at least emit {} batches, but only got {} batches", + MINIMUM_OUTPUT_BATCHES, + output_batched.len() + ); + + // Just sanity test to make sure we are still producing valid output + { + let output = [output_batched, output_batches_after_finish].concat(); + let actual_prepared = prepare_record_batches_for_cmp(output); + + assert_eq!(actual_prepared.columns(), expected.columns()); + } + } + } + } + Ok(()) +} + +/// Polls the stream until both barriers are reached, +/// collecting the emitted batches along the way. +/// +/// If the stream is pending for too long (5s) without emitting any batches, +/// it panics to avoid hanging the test indefinitely. +/// +/// Note: The left and right BarrierExec might be the input of the output stream +async fn consume_stream_until_finish_barrier_reached( + left: Arc, + right: Arc, + output_stream: &mut SendableRecordBatchStream, +) -> Result<(Vec, Vec)> { + let mut switch_to_finish_barrier = false; + let mut output_batched = vec![]; + let mut after_finish_barrier_reached = vec![]; + let mut background_task = JoinSet::new(); + + let mut start_time_since_last_ready = datafusion_common::instant::Instant::now(); + loop { + let next_item = output_stream.next(); + + // Manual polling + let poll_output = futures::poll!(next_item); + + // Wake up the stream to make sure it makes progress + tokio::task::yield_now().await; + + match poll_output { + Poll::Ready(Some(Ok(batch))) => { + if batch.num_rows() == 0 { + return internal_err!("join stream should not emit empty batch"); + } + if switch_to_finish_barrier { + after_finish_barrier_reached.push(batch); + } else { + output_batched.push(batch); + } + start_time_since_last_ready = datafusion_common::instant::Instant::now(); + } + Poll::Ready(Some(Err(e))) => return Err(e), + Poll::Ready(None) if !switch_to_finish_barrier => { + unreachable!("Stream should not end before manually finishing it") + } + Poll::Ready(None) => { + break; + } + Poll::Pending => { + if right.is_finish_barrier_reached() + && left.is_finish_barrier_reached() + && !switch_to_finish_barrier + { + switch_to_finish_barrier = true; + + let right = Arc::clone(&right); + background_task.spawn(async move { + right.wait_finish().await; + }); + let left = Arc::clone(&left); + background_task.spawn(async move { + left.wait_finish().await; + }); + } + + // Make sure the test doesn't run forever + if start_time_since_last_ready.elapsed() + > std::time::Duration::from_secs(5) + { + return internal_err!( + "Stream should have emitted data by now, but it's still pending. Output batches so far: {}", + output_batched.len() + ); + } + } + } + } + + Ok((output_batched, after_finish_barrier_reached)) +} + /// Returns the column names on the schema fn columns(schema: &Schema) -> Vec { schema.fields().iter().map(|f| f.name().clone()).collect() diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 1f6bc703a030..a75a9893e9f1 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -930,6 +930,7 @@ pub(crate) fn build_side_determined_results( &probe_indices, column_indices, build_hash_joiner.build_side, + join_type, ) .map(|batch| (batch.num_rows() > 0).then_some(batch)) } else { @@ -993,6 +994,7 @@ pub(crate) fn join_with_probe_batch( filter, build_hash_joiner.build_side, None, + join_type, )? } else { (build_indices, probe_indices) @@ -1031,6 +1033,7 @@ pub(crate) fn join_with_probe_batch( &probe_indices, column_indices, build_hash_joiner.build_side, + join_type, ) .map(|batch| (batch.num_rows() > 0).then_some(batch)) } diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index a9243fe04e28..53b4c4f80236 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -910,6 +910,7 @@ pub(crate) fn get_final_indices_from_bit_map( (left_indices, right_indices) } +#[expect(clippy::too_many_arguments)] pub(crate) fn apply_join_filter_to_indices( build_input_buffer: &RecordBatch, probe_batch: &RecordBatch, @@ -918,6 +919,7 @@ pub(crate) fn apply_join_filter_to_indices( filter: &JoinFilter, build_side: JoinSide, max_intermediate_size: Option, + join_type: JoinType, ) -> Result<(UInt64Array, UInt32Array)> { if build_indices.is_empty() && probe_indices.is_empty() { return Ok((build_indices, probe_indices)); @@ -938,6 +940,7 @@ pub(crate) fn apply_join_filter_to_indices( &probe_indices.slice(i, len), filter.column_indices(), build_side, + join_type, )?; let filter_result = filter .expression() @@ -959,6 +962,7 @@ pub(crate) fn apply_join_filter_to_indices( &probe_indices, filter.column_indices(), build_side, + join_type, )?; filter @@ -979,6 +983,7 @@ pub(crate) fn apply_join_filter_to_indices( /// Returns a new [RecordBatch] by combining the `left` and `right` according to `indices`. /// The resulting batch has [Schema] `schema`. +#[expect(clippy::too_many_arguments)] pub(crate) fn build_batch_from_indices( schema: &Schema, build_input_buffer: &RecordBatch, @@ -987,11 +992,19 @@ pub(crate) fn build_batch_from_indices( probe_indices: &UInt32Array, column_indices: &[ColumnIndex], build_side: JoinSide, + join_type: JoinType, ) -> Result { if schema.fields().is_empty() { + // For RightAnti and RightSemi joins, after `adjust_indices_by_join_type` + // the build_indices were untouched so only probe_indices hold the actual + // row count. + let row_count = match join_type { + JoinType::RightAnti | JoinType::RightSemi => probe_indices.len(), + _ => build_indices.len(), + }; let options = RecordBatchOptions::new() .with_match_field_names(true) - .with_row_count(Some(build_indices.len())); + .with_row_count(Some(row_count)); return Ok(RecordBatch::try_new_with_options( Arc::new(schema.clone()), diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 3e8fdf1f3ed7..475738cca3f0 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -728,37 +728,27 @@ impl ExternalSorter { let sorted_batches = sort_batch_chunked(&batch, &expressions, batch_size)?; drop(batch); - // Free the old reservation and grow it to match the actual sorted output size - reservation.free(); + // Resize the reservation to match the actual sorted output size. + // Using try_resize avoids a release-then-reacquire cycle, which + // matters for MemoryPool implementations where grow/shrink have + // non-trivial cost (e.g. JNI calls in Comet). + let total_sorted_size: usize = sorted_batches + .iter() + .map(get_record_batch_memory_size) + .sum(); + reservation + .try_resize(total_sorted_size) + .map_err(Self::err_with_oom_context)?; - Result::<_, DataFusionError>::Ok((schema, sorted_batches, reservation)) - }) - .then({ - move |batches| async move { - match batches { - Ok((schema, sorted_batches, mut reservation)) => { - // Calculate the total size of sorted batches - let total_sorted_size: usize = sorted_batches - .iter() - .map(get_record_batch_memory_size) - .sum(); - reservation - .try_grow(total_sorted_size) - .map_err(Self::err_with_oom_context)?; - - // Wrap in ReservationStream to hold the reservation - Ok(Box::pin(ReservationStream::new( - Arc::clone(&schema), - Box::pin(RecordBatchStreamAdapter::new( - schema, - futures::stream::iter(sorted_batches.into_iter().map(Ok)), - )), - reservation, - )) as SendableRecordBatchStream) - } - Err(e) => Err(e), - } - } + // Wrap in ReservationStream to hold the reservation + Result::<_, DataFusionError>::Ok(Box::pin(ReservationStream::new( + Arc::clone(&schema), + Box::pin(RecordBatchStreamAdapter::new( + schema, + futures::stream::iter(sorted_batches.into_iter().map(Ok)), + )), + reservation, + )) as SendableRecordBatchStream) }) .try_flatten() .map(move |batch| match batch { diff --git a/datafusion/physical-plan/src/spill/in_progress_spill_file.rs b/datafusion/physical-plan/src/spill/in_progress_spill_file.rs index d2acf4993b85..b9ff6b2f3b65 100644 --- a/datafusion/physical-plan/src/spill/in_progress_spill_file.rs +++ b/datafusion/physical-plan/src/spill/in_progress_spill_file.rs @@ -88,6 +88,13 @@ impl InProgressSpillFile { Ok(()) } + pub fn flush(&mut self) -> Result<()> { + if let Some(writer) = &mut self.writer { + writer.flush()?; + } + Ok(()) + } + /// Returns a reference to the in-progress file, if it exists. /// This can be used to get the file path for creating readers before the file is finished. pub fn file(&self) -> Option<&RefCountedTempFile> { diff --git a/datafusion/physical-plan/src/spill/mod.rs b/datafusion/physical-plan/src/spill/mod.rs index 78dea99ac820..3c4ee065c315 100644 --- a/datafusion/physical-plan/src/spill/mod.rs +++ b/datafusion/physical-plan/src/spill/mod.rs @@ -310,6 +310,11 @@ impl IPCStreamWriter { Ok((delta_num_rows, delta_num_bytes)) } + pub fn flush(&mut self) -> Result<()> { + self.writer.flush()?; + Ok(()) + } + /// Finish the writer pub fn finish(&mut self) -> Result<()> { self.writer.finish().map_err(Into::into) diff --git a/datafusion/physical-plan/src/spill/spill_manager.rs b/datafusion/physical-plan/src/spill/spill_manager.rs index 89b027620677..6d931112ad88 100644 --- a/datafusion/physical-plan/src/spill/spill_manager.rs +++ b/datafusion/physical-plan/src/spill/spill_manager.rs @@ -188,6 +188,19 @@ impl SpillManager { Ok(spawn_buffered(stream, self.batch_read_buffer_capacity)) } + + /// Same as `read_spill_as_stream`, but without buffering. + pub fn read_spill_as_stream_unbuffered( + &self, + spill_file_path: RefCountedTempFile, + max_record_batch_memory: Option, + ) -> Result { + Ok(Box::pin(cooperative(SpillReaderStream::new( + Arc::clone(&self.schema), + spill_file_path, + max_record_batch_memory, + )))) + } } pub(crate) trait GetSlicedSize { diff --git a/datafusion/physical-plan/src/spill/spill_pool.rs b/datafusion/physical-plan/src/spill/spill_pool.rs index e3b547b5731f..36920c7fd060 100644 --- a/datafusion/physical-plan/src/spill/spill_pool.rs +++ b/datafusion/physical-plan/src/spill/spill_pool.rs @@ -61,6 +61,10 @@ struct SpillPoolShared { /// Writer's reference to the current file (shared by all cloned writers). /// Has its own lock to allow I/O without blocking queue access. current_write_file: Option>>, + /// Number of active writer clones. Only when this reaches zero should + /// `writer_dropped` be set to true. This prevents premature EOF signaling + /// when one writer clone is dropped while others are still active. + active_writer_count: usize, } impl SpillPoolShared { @@ -72,6 +76,7 @@ impl SpillPoolShared { waker: None, writer_dropped: false, current_write_file: None, + active_writer_count: 1, } } @@ -97,7 +102,6 @@ impl SpillPoolShared { /// The writer automatically manages file rotation based on the `max_file_size_bytes` /// configured in [`channel`]. When the last writer clone is dropped, it finalizes the /// current file so readers can access all written data. -#[derive(Clone)] pub struct SpillPoolWriter { /// Maximum size in bytes before rotating to a new file. /// Typically set from configuration `datafusion.execution.max_spill_file_size_bytes`. @@ -106,6 +110,18 @@ pub struct SpillPoolWriter { shared: Arc>, } +impl Clone for SpillPoolWriter { + fn clone(&self) -> Self { + // Increment the active writer count so that `writer_dropped` is only + // set to true when the *last* clone is dropped. + self.shared.lock().active_writer_count += 1; + Self { + max_file_size_bytes: self.max_file_size_bytes, + shared: Arc::clone(&self.shared), + } + } +} + impl SpillPoolWriter { /// Spills a batch to the pool, rotating files when necessary. /// @@ -194,6 +210,8 @@ impl SpillPoolWriter { // Append the batch if let Some(ref mut writer) = file_shared.writer { writer.append_batch(batch)?; + // make sure we flush the writer for readers + writer.flush()?; file_shared.batches_written += 1; file_shared.estimated_size += batch_size; } @@ -231,6 +249,15 @@ impl Drop for SpillPoolWriter { fn drop(&mut self) { let mut shared = self.shared.lock(); + shared.active_writer_count -= 1; + let is_last_writer = shared.active_writer_count == 0; + + if !is_last_writer { + // Other writer clones are still active; do not finalize or + // signal EOF to readers. + return; + } + // Finalize the current file when the last writer is dropped if let Some(current_file) = shared.current_write_file.take() { // Release shared lock before locking file @@ -535,7 +562,11 @@ impl Stream for SpillFile { // Step 2: Lazy-create reader stream if needed if self.reader.is_none() && should_read { if let Some(file) = file { - match self.spill_manager.read_spill_as_stream(file, None) { + // we want this unbuffered because files are actively being written to + match self + .spill_manager + .read_spill_as_stream_unbuffered(file, None) + { Ok(stream) => { self.reader = Some(SpillFileReader { stream, @@ -1337,6 +1368,81 @@ mod tests { Ok(()) } + /// Verifies that the reader stays alive as long as any writer clone exists. + /// + /// `SpillPoolWriter` is `Clone`, and in non-preserve-order repartitioning + /// mode multiple input partition tasks share clones of the same writer. + /// The reader must not see EOF until **all** clones have been dropped, + /// even if the queue is temporarily empty between writes from different + /// clones. + /// + /// The test sequence is: + /// + /// 1. writer1 writes a batch, then is dropped. + /// 2. The reader consumes that batch (queue is now empty). + /// 3. writer2 (still alive) writes a batch. + /// 4. The reader must see that batch. + /// 5. EOF is only signalled after writer2 is also dropped. + #[tokio::test] + async fn test_clone_drop_does_not_signal_eof_prematurely() -> Result<()> { + let (writer1, mut reader) = create_spill_channel(1024 * 1024); + let writer2 = writer1.clone(); + + // Synchronization: tell writer2 when it may proceed. + let (proceed_tx, proceed_rx) = tokio::sync::oneshot::channel::<()>(); + + // Spawn writer2 — it waits for the signal before writing. + let writer2_handle = SpawnedTask::spawn(async move { + proceed_rx.await.unwrap(); + writer2.push_batch(&create_test_batch(10, 10)).unwrap(); + // writer2 is dropped here (last clone → true EOF) + }); + + // Writer1 writes one batch, then drops. + writer1.push_batch(&create_test_batch(0, 10))?; + drop(writer1); + + // Read writer1's batch. + let batch1 = reader.next().await.unwrap()?; + assert_eq!(batch1.num_rows(), 10); + let col = batch1 + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col.value(0), 0); + + // Signal writer2 to write its batch. It will execute when the + // current task yields (i.e. when reader.next() returns Pending). + proceed_tx.send(()).unwrap(); + + // The reader should wait (Pending) for writer2's data, not EOF. + let batch2 = + tokio::time::timeout(std::time::Duration::from_secs(5), reader.next()) + .await + .expect("Reader timed out — should not hang"); + + assert!( + batch2.is_some(), + "Reader must not return EOF while a writer clone is still alive" + ); + let batch2 = batch2.unwrap()?; + assert_eq!(batch2.num_rows(), 10); + let col = batch2 + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col.value(0), 10); + + writer2_handle.await.unwrap(); + + // All writers dropped — reader should see real EOF now. + assert!(reader.next().await.is_none()); + + Ok(()) + } + #[tokio::test] async fn test_disk_usage_decreases_as_files_consumed() -> Result<()> { use datafusion_execution::runtime_env::RuntimeEnvBuilder; diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index 4507cccba05a..c2293184151f 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -17,13 +17,6 @@ //! Simple iterator over batches for use in testing -use std::{ - any::Any, - pin::Pin, - sync::{Arc, Weak}, - task::{Context, Poll}, -}; - use crate::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, common, @@ -33,6 +26,13 @@ use crate::{ execution_plan::EmissionType, stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}, }; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::{ + any::Any, + pin::Pin, + sync::{Arc, Weak}, + task::{Context, Poll}, +}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; @@ -298,29 +298,91 @@ pub struct BarrierExec { schema: SchemaRef, /// all streams wait on this barrier to produce - barrier: Arc, + start_data_barrier: Option>, + + /// the stream wait for this to return Poll::Ready(None) + finish_barrier: Option>, + cache: PlanProperties, + + log: bool, } impl BarrierExec { /// Create a new exec with some number of partitions. pub fn new(data: Vec>, schema: SchemaRef) -> Self { // wait for all streams and the input - let barrier = Arc::new(Barrier::new(data.len() + 1)); + let barrier = Some(Arc::new(Barrier::new(data.len() + 1))); let cache = Self::compute_properties(Arc::clone(&schema), &data); Self { data, schema, - barrier, + start_data_barrier: barrier, cache, + finish_barrier: None, + log: true, } } + pub fn with_log(mut self, log: bool) -> Self { + self.log = log; + self + } + + pub fn without_start_barrier(mut self) -> Self { + self.start_data_barrier = None; + self + } + + pub fn with_finish_barrier(mut self) -> Self { + let barrier = Arc::new(( + // wait for all streams and the input + Barrier::new(self.data.len() + 1), + AtomicUsize::new(0), + )); + + self.finish_barrier = Some(barrier); + self + } + /// wait until all the input streams and this function is ready pub async fn wait(&self) { - println!("BarrierExec::wait waiting on barrier"); - self.barrier.wait().await; - println!("BarrierExec::wait done waiting"); + let barrier = &self + .start_data_barrier + .as_ref() + .expect("Must only be called when having a start barrier"); + if self.log { + println!("BarrierExec::wait waiting on barrier"); + } + barrier.wait().await; + if self.log { + println!("BarrierExec::wait done waiting"); + } + } + + pub async fn wait_finish(&self) { + let (barrier, _) = &self + .finish_barrier + .as_deref() + .expect("Must only be called when having a finish barrier"); + + if self.log { + println!("BarrierExec::wait_finish waiting on barrier"); + } + barrier.wait().await; + if self.log { + println!("BarrierExec::wait_finish done waiting"); + } + } + + /// Return true if the finish barrier has been reached in all partitions + pub fn is_finish_barrier_reached(&self) -> bool { + let (_, reached_finish) = self + .finish_barrier + .as_deref() + .expect("Must only be called when having finish barrier"); + + reached_finish.load(Ordering::Relaxed) == self.data.len() } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. @@ -391,17 +453,32 @@ impl ExecutionPlan for BarrierExec { // task simply sends data in order after barrier is reached let data = self.data[partition].clone(); - let b = Arc::clone(&self.barrier); + let start_barrier = self.start_data_barrier.as_ref().map(Arc::clone); + let finish_barrier = self.finish_barrier.as_ref().map(Arc::clone); + let log = self.log; let tx = builder.tx(); builder.spawn(async move { - println!("Partition {partition} waiting on barrier"); - b.wait().await; + if let Some(barrier) = start_barrier { + if log { + println!("Partition {partition} waiting on barrier"); + } + barrier.wait().await; + } for batch in data { - println!("Partition {partition} sending batch"); + if log { + println!("Partition {partition} sending batch"); + } if let Err(e) = tx.send(Ok(batch)).await { println!("ERROR batch via barrier stream stream: {e}"); } } + if let Some((barrier, reached_finish)) = finish_barrier.as_deref() { + if log { + println!("Partition {partition} waiting on finish barrier"); + } + reached_finish.fetch_add(1, Ordering::Relaxed); + barrier.wait().await; + } Ok(()) }); diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index db09cf60e2c2..ae687f872db5 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -2577,6 +2577,31 @@ NULL NULL NULL NULL NULL NULL +# maintains inner nullability +query ?T +select array_sort(column1), arrow_typeof(array_sort(column1)) +from values + (arrow_cast([], 'List(non-null Int32)')), + (arrow_cast(NULL, 'List(non-null Int32)')), + (arrow_cast([1, 3, 5, -5], 'List(non-null Int32)')) +; +---- +[] List(non-null Int32) +NULL List(non-null Int32) +[-5, 1, 3, 5] List(non-null Int32) + +query ?T +select column1, arrow_typeof(column1) +from values (array_sort(arrow_cast([1, 3, 5, -5], 'LargeList(non-null Int32)'))); +---- +[-5, 1, 3, 5] LargeList(non-null Int32) + +query ?T +select column1, arrow_typeof(column1) +from values (array_sort(arrow_cast([1, 3, 5, -5], 'FixedSizeList(4 x non-null Int32)'))); +---- +[-5, 1, 3, 5] List(non-null Int32) + query ? select array_sort([struct('foo', 3), struct('foo', 1), struct('bar', 1)]) ---- diff --git a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt index 3e403171e071..6ee6ba0695cb 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -257,6 +257,25 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet]]}, projection=[score], file_type=parquet, predicate=category@0 = alpha AND DynamicFilter [ empty ], pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] +# Test 4b: COUNT + MAX — DynamicFilter should NOT appear here in mixed aggregates + +query TT +EXPLAIN SELECT COUNT(*), MAX(score) FROM agg_parquet WHERE category = 'alpha'; +---- +logical_plan +01)Projection: count(Int64(1)) AS count(*), max(agg_parquet.score) +02)--Aggregate: groupBy=[[]], aggr=[[count(Int64(1)), max(agg_parquet.score)]] +03)----Projection: agg_parquet.score +04)------Filter: agg_parquet.category = Utf8View("alpha") +05)--------TableScan: agg_parquet projection=[category, score], partial_filters=[agg_parquet.category = Utf8View("alpha")] +physical_plan +01)ProjectionExec: expr=[count(Int64(1))@0 as count(*), max(agg_parquet.score)@1 as max(agg_parquet.score)] +02)--AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1)), max(agg_parquet.score)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1)), max(agg_parquet.score)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet]]}, projection=[score], file_type=parquet, predicate=category@0 = alpha, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] + # Disable aggregate dynamic filters only statement ok SET datafusion.optimizer.enable_aggregate_dynamic_filter_pushdown = false; diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 38037ede21db..1155bc4f3b2b 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -5198,3 +5198,74 @@ DROP TABLE t1_c; statement ok DROP TABLE t2_c; + +# Issue #20437: HashJoin panic with dictionary-encoded columns in multi-key joins +# https://github.com/apache/datafusion/issues/20437 + +statement ok +CREATE TABLE issue_20437_small AS +SELECT id, arrow_cast(region, 'Dictionary(Int32, Utf8)') AS region +FROM (VALUES (1, 'west'), (2, 'west')) AS t(id, region); + +statement ok +CREATE TABLE issue_20437_large AS +SELECT id, region, value +FROM (VALUES (1, 'west', 100), (2, 'west', 200), (3, 'east', 300)) AS t(id, region, value); + +query ITI +SELECT s.id, s.region, l.value +FROM issue_20437_small s +JOIN issue_20437_large l ON s.id = l.id AND s.region = l.region +ORDER BY s.id; +---- +1 west 100 +2 west 200 + +statement count 0 +DROP TABLE issue_20437_small; + +statement count 0 +DROP TABLE issue_20437_large; + +# Test count(*) with right semi/anti joins returns correct row counts +# issue: https://github.com/apache/datafusion/issues/20669 + +statement ok +CREATE TABLE t1 (k INT, v INT); + +statement ok +CREATE TABLE t2 (k INT, v INT); + +statement ok +INSERT INTO t1 SELECT i AS k, i AS v FROM generate_series(1, 100) t(i); + +statement ok +INSERT INTO t2 VALUES (1, 1); + +query I +WITH t AS ( + SELECT * + FROM t1 + LEFT ANTI JOIN t2 ON t1.k = t2.k +) +SELECT count(*) +FROM t; +---- +99 + +query I +WITH t AS ( + SELECT * + FROM t1 + LEFT SEMI JOIN t2 ON t1.k = t2.k +) +SELECT count(*) +FROM t; +---- +1 + +statement count 0 +DROP TABLE t1; + +statement count 0 +DROP TABLE t2; diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index 8bb79d576990..d306f94ae310 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -563,3 +563,202 @@ ORDER BY start_timestamp, trace_id LIMIT 1; ---- 2024-10-01T00:00:00 + + +statement ok +set datafusion.execution.parquet.pushdown_filters = false; + +# Regression test for https://github.com/apache/datafusion/issues/20696 +# Multi-column INNER JOIN with dictionary fails +# when parquet pushdown filters are enabled. + +statement ok +COPY ( + SELECT + to_timestamp_nanos(time_ns) AS time, + arrow_cast(state, 'Dictionary(Int32, Utf8)') AS state, + arrow_cast(city, 'Dictionary(Int32, Utf8)') AS city, + temp + FROM ( + VALUES + (200, 'CA', 'LA', 90.0), + (250, 'MA', 'Boston', 72.4), + (100, 'MA', 'Boston', 70.4), + (350, 'CA', 'LA', 90.0) + ) AS t(time_ns, state, city, temp) +) +TO 'test_files/scratch/parquet_filter_pushdown/issue_20696/h2o/data.parquet'; + +statement ok +COPY ( + SELECT + to_timestamp_nanos(time_ns) AS time, + arrow_cast(state, 'Dictionary(Int32, Utf8)') AS state, + arrow_cast(city, 'Dictionary(Int32, Utf8)') AS city, + temp, + reading + FROM ( + VALUES + (250, 'MA', 'Boston', 53.4, 51.0), + (100, 'MA', 'Boston', 50.4, 50.0) + ) AS t(time_ns, state, city, temp, reading) +) +TO 'test_files/scratch/parquet_filter_pushdown/issue_20696/o2/data.parquet'; + +statement ok +CREATE EXTERNAL TABLE h2o_parquet_20696 STORED AS PARQUET +LOCATION 'test_files/scratch/parquet_filter_pushdown/issue_20696/h2o/'; + +statement ok +CREATE EXTERNAL TABLE o2_parquet_20696 STORED AS PARQUET +LOCATION 'test_files/scratch/parquet_filter_pushdown/issue_20696/o2/'; + +# Query should work both with and without filters +statement ok +set datafusion.execution.parquet.pushdown_filters = false; + +query RRR +SELECT + h2o_parquet_20696.temp AS h2o_temp, + o2_parquet_20696.temp AS o2_temp, + o2_parquet_20696.reading +FROM h2o_parquet_20696 +INNER JOIN o2_parquet_20696 + ON h2o_parquet_20696.time = o2_parquet_20696.time + AND h2o_parquet_20696.state = o2_parquet_20696.state + AND h2o_parquet_20696.city = o2_parquet_20696.city +WHERE h2o_parquet_20696.time >= '1970-01-01T00:00:00.000000050Z' + AND h2o_parquet_20696.time <= '1970-01-01T00:00:00.000000300Z'; +---- +72.4 53.4 51 +70.4 50.4 50 + + +statement ok +set datafusion.execution.parquet.pushdown_filters = true; + +query RRR +SELECT + h2o_parquet_20696.temp AS h2o_temp, + o2_parquet_20696.temp AS o2_temp, + o2_parquet_20696.reading +FROM h2o_parquet_20696 +INNER JOIN o2_parquet_20696 + ON h2o_parquet_20696.time = o2_parquet_20696.time + AND h2o_parquet_20696.state = o2_parquet_20696.state + AND h2o_parquet_20696.city = o2_parquet_20696.city +WHERE h2o_parquet_20696.time >= '1970-01-01T00:00:00.000000050Z' + AND h2o_parquet_20696.time <= '1970-01-01T00:00:00.000000300Z'; +---- +72.4 53.4 51 +70.4 50.4 50 + +# Cleanup +statement ok +DROP TABLE h2o_parquet_20696; + +statement ok +DROP TABLE o2_parquet_20696; + +# Cleanup settings +statement ok +set datafusion.execution.parquet.pushdown_filters = false; + +########## +# Regression test: filter pushdown with Struct columns in schema +# +# When a schema has Struct columns, Arrow field indices diverge from Parquet +# leaf indices (Struct children become separate leaves). A filter on a +# primitive column *after* a Struct must use the correct Parquet leaf index. +# +# Schema: +# Arrow: col_a=0 struct_col=1 col_b=2 +# Parquet: col_a=0 struct_col.x=1 struct_col.y=2 col_b=3 +########## + +statement ok +set datafusion.execution.parquet.pushdown_filters = true; + +statement ok +COPY ( + SELECT + column1 as col_a, + column2 as struct_col, + column3 as col_b + FROM VALUES + (1, {x: 10, y: 100}, 'aaa'), + (2, {x: 20, y: 200}, 'target'), + (3, {x: 30, y: 300}, 'zzz') +) TO 'test_files/scratch/parquet_filter_pushdown/struct_filter.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE t_struct_filter +STORED AS PARQUET +LOCATION 'test_files/scratch/parquet_filter_pushdown/struct_filter.parquet'; + +# Filter on col_b (the primitive column after the struct). +# Before the fix, this returned 0 rows because the filter read struct_col.y +# (Parquet leaf 2) instead of col_b (Parquet leaf 3). +query IT +SELECT col_a, col_b FROM t_struct_filter WHERE col_b = 'target'; +---- +2 target + +# Clean up +statement ok +set datafusion.execution.parquet.pushdown_filters = false; + +statement ok +DROP TABLE t_struct_filter; + +########## +# Regression test for https://github.com/apache/datafusion/issues/20937 +# +# Dynamic filter pushdown fails when joining VALUES against +# Dictionary-encoded Parquet columns. The InListExpr's ArrayStaticFilter +# unwraps the needle Dictionary but not the stored in_array, causing a +# make_comparator(Utf8, Dictionary) type mismatch. +########## + +statement ok +set datafusion.execution.parquet.pushdown_filters = true; + +statement ok +set datafusion.execution.parquet.reorder_filters = true; + +statement ok +COPY ( + SELECT + arrow_cast(chr(65 + (row_num % 26)), 'Dictionary(Int32, Utf8)') as tag1, + row_num * 1.0 as value + FROM (SELECT unnest(range(0, 10000)) as row_num) +) TO 'test_files/scratch/parquet_filter_pushdown/dict_filter_bug.parquet'; + +statement ok +CREATE EXTERNAL TABLE dict_filter_bug +STORED AS PARQUET +LOCATION 'test_files/scratch/parquet_filter_pushdown/dict_filter_bug.parquet'; + +query TR +SELECT t.tag1, t.value +FROM dict_filter_bug t +JOIN (VALUES ('A'), ('B')) AS v(c1) +ON t.tag1 = v.c1 +ORDER BY t.tag1, t.value +LIMIT 4; +---- +A 0 +A 26 +A 52 +A 78 + +# Cleanup +statement ok +set datafusion.execution.parquet.pushdown_filters = false; + +statement ok +set datafusion.execution.parquet.reorder_filters = false; + +statement ok +DROP TABLE dict_filter_bug; diff --git a/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt b/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt index 5a559bdb9483..fd3a40ca1707 100644 --- a/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt +++ b/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt @@ -274,4 +274,4 @@ logical_plan 02)--TableScan: test_table projection=[constant_col] physical_plan 01)SortPreservingMergeExec: [constant_col@0 ASC NULLS LAST] -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[constant_col], file_type=parquet +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[constant_col], output_ordering=[constant_col@0 ASC NULLS LAST], file_type=parquet diff --git a/datafusion/sqllogictest/test_files/sort_pushdown.slt b/datafusion/sqllogictest/test_files/sort_pushdown.slt index 58d9915a24be..99f26b66d458 100644 --- a/datafusion/sqllogictest/test_files/sort_pushdown.slt +++ b/datafusion/sqllogictest/test_files/sort_pushdown.slt @@ -851,7 +851,749 @@ LIMIT 3; 5 4 2 -3 +# Test 3.7: Aggregate ORDER BY expression should keep SortExec +# Source pattern declared on parquet scan: [x ASC, y ASC]. +# Requested pattern in ORDER BY: [x ASC, CAST(y AS BIGINT) % 2 ASC]. +# Example for x=1 input y order 1,2,3 gives bucket order 1,0,1, which does not +# match requested bucket ASC order. SortExec is required above AggregateExec. +statement ok +SET datafusion.execution.target_partitions = 1; + +statement ok +CREATE TABLE agg_expr_data(x INT, y INT, v INT) AS VALUES +(1, 1, 10), +(1, 2, 20), +(1, 3, 30), +(2, 1, 40), +(2, 2, 50), +(2, 3, 60); + +query I +COPY (SELECT * FROM agg_expr_data ORDER BY x, y) +TO 'test_files/scratch/sort_pushdown/agg_expr_sorted.parquet'; +---- +6 + +statement ok +CREATE EXTERNAL TABLE agg_expr_parquet(x INT, y INT, v INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/agg_expr_sorted.parquet' +WITH ORDER (x ASC, y ASC); + +query TT +EXPLAIN SELECT + x, + CAST(y AS BIGINT) % 2, + SUM(v) +FROM agg_expr_parquet +GROUP BY x, CAST(y AS BIGINT) % 2 +ORDER BY x, CAST(y AS BIGINT) % 2; +---- +logical_plan +01)Sort: agg_expr_parquet.x ASC NULLS LAST, agg_expr_parquet.y % Int64(2) ASC NULLS LAST +02)--Aggregate: groupBy=[[agg_expr_parquet.x, CAST(agg_expr_parquet.y AS Int64) % Int64(2)]], aggr=[[sum(CAST(agg_expr_parquet.v AS Int64))]] +03)----TableScan: agg_expr_parquet projection=[x, y, v] +physical_plan +01)SortExec: expr=[x@0 ASC NULLS LAST, agg_expr_parquet.y % Int64(2)@1 ASC NULLS LAST], preserve_partitioning=[false] +02)--AggregateExec: mode=Single, gby=[x@0 as x, CAST(y@1 AS Int64) % 2 as agg_expr_parquet.y % Int64(2)], aggr=[sum(agg_expr_parquet.v)], ordering_mode=PartiallySorted([0]) +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/agg_expr_sorted.parquet]]}, projection=[x, y, v], output_ordering=[x@0 ASC NULLS LAST, y@1 ASC NULLS LAST], file_type=parquet + +# Expected output pattern from ORDER BY [x, bucket]: +# rows grouped by x, and within each x bucket appears as 0 then 1. +query III +SELECT + x, + CAST(y AS BIGINT) % 2, + SUM(v) +FROM agg_expr_parquet +GROUP BY x, CAST(y AS BIGINT) % 2 +ORDER BY x, CAST(y AS BIGINT) % 2; +---- +1 0 20 +1 1 40 +2 0 50 +2 1 100 + +# Test 3.8: Aggregate ORDER BY monotonic expression can push down (no SortExec) +query TT +EXPLAIN SELECT + x, + CAST(y AS BIGINT), + SUM(v) +FROM agg_expr_parquet +GROUP BY x, CAST(y AS BIGINT) +ORDER BY x, CAST(y AS BIGINT); +---- +logical_plan +01)Sort: agg_expr_parquet.x ASC NULLS LAST, agg_expr_parquet.y ASC NULLS LAST +02)--Aggregate: groupBy=[[agg_expr_parquet.x, CAST(agg_expr_parquet.y AS Int64)]], aggr=[[sum(CAST(agg_expr_parquet.v AS Int64))]] +03)----TableScan: agg_expr_parquet projection=[x, y, v] +physical_plan +01)AggregateExec: mode=Single, gby=[x@0 as x, CAST(y@1 AS Int64) as agg_expr_parquet.y], aggr=[sum(agg_expr_parquet.v)], ordering_mode=Sorted +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/agg_expr_sorted.parquet]]}, projection=[x, y, v], output_ordering=[x@0 ASC NULLS LAST, y@1 ASC NULLS LAST], file_type=parquet + +query III +SELECT + x, + CAST(y AS BIGINT), + SUM(v) +FROM agg_expr_parquet +GROUP BY x, CAST(y AS BIGINT) +ORDER BY x, CAST(y AS BIGINT); +---- +1 1 10 +1 2 20 +1 3 30 +2 1 40 +2 2 50 +2 3 60 + +# Test 3.9: Aggregate ORDER BY aggregate output should keep SortExec +query TT +EXPLAIN SELECT x, SUM(v) +FROM agg_expr_parquet +GROUP BY x +ORDER BY SUM(v); +---- +logical_plan +01)Sort: sum(agg_expr_parquet.v) ASC NULLS LAST +02)--Aggregate: groupBy=[[agg_expr_parquet.x]], aggr=[[sum(CAST(agg_expr_parquet.v AS Int64))]] +03)----TableScan: agg_expr_parquet projection=[x, v] +physical_plan +01)SortExec: expr=[sum(agg_expr_parquet.v)@1 ASC NULLS LAST], preserve_partitioning=[false] +02)--AggregateExec: mode=Single, gby=[x@0 as x], aggr=[sum(agg_expr_parquet.v)], ordering_mode=Sorted +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/agg_expr_sorted.parquet]]}, projection=[x, v], output_ordering=[x@0 ASC NULLS LAST], file_type=parquet + +query II +SELECT x, SUM(v) +FROM agg_expr_parquet +GROUP BY x +ORDER BY SUM(v); +---- +1 60 +2 150 + +# Test 3.10: Aggregate with non-preserved input order should keep SortExec +# v is not part of the order by +query TT +EXPLAIN SELECT v, SUM(y) +FROM agg_expr_parquet +GROUP BY v +ORDER BY v; +---- +logical_plan +01)Sort: agg_expr_parquet.v ASC NULLS LAST +02)--Aggregate: groupBy=[[agg_expr_parquet.v]], aggr=[[sum(CAST(agg_expr_parquet.y AS Int64))]] +03)----TableScan: agg_expr_parquet projection=[y, v] +physical_plan +01)SortExec: expr=[v@0 ASC NULLS LAST], preserve_partitioning=[false] +02)--AggregateExec: mode=Single, gby=[v@1 as v], aggr=[sum(agg_expr_parquet.y)] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/agg_expr_sorted.parquet]]}, projection=[y, v], file_type=parquet + +query II +SELECT v, SUM(y) +FROM agg_expr_parquet +GROUP BY v +ORDER BY v; +---- +10 1 +20 2 +30 3 +40 1 +50 2 +60 3 + +# Test 3.11: Aggregate ORDER BY non-column expression (unsatisfied) keeps SortExec +# (though note in theory DataFusion could figure out that data sorted by x will also be sorted by x+1) +query TT +EXPLAIN SELECT x, SUM(v) +FROM agg_expr_parquet +GROUP BY x +ORDER BY x + 1 DESC; +---- +logical_plan +01)Sort: CAST(agg_expr_parquet.x AS Int64) + Int64(1) DESC NULLS FIRST +02)--Aggregate: groupBy=[[agg_expr_parquet.x]], aggr=[[sum(CAST(agg_expr_parquet.v AS Int64))]] +03)----TableScan: agg_expr_parquet projection=[x, v] +physical_plan +01)SortExec: expr=[CAST(x@0 AS Int64) + 1 DESC], preserve_partitioning=[false] +02)--AggregateExec: mode=Single, gby=[x@0 as x], aggr=[sum(agg_expr_parquet.v)], ordering_mode=Sorted +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/agg_expr_sorted.parquet]]}, projection=[x, v], output_ordering=[x@0 ASC NULLS LAST], file_type=parquet + +query II +SELECT x, SUM(v) +FROM agg_expr_parquet +GROUP BY x +ORDER BY x + 1 DESC; +---- +2 150 +1 60 + +# Test 3.12: Aggregate ORDER BY non-column expression (unsatisfied) keeps SortExec +# (though note in theory DataFusion could figure out that data sorted by x will also be sorted by x+1) +query TT +EXPLAIN SELECT x, SUM(v) +FROM agg_expr_parquet +GROUP BY x +ORDER BY 2 * x ASC; +---- +logical_plan +01)Sort: Int64(2) * CAST(agg_expr_parquet.x AS Int64) ASC NULLS LAST +02)--Aggregate: groupBy=[[agg_expr_parquet.x]], aggr=[[sum(CAST(agg_expr_parquet.v AS Int64))]] +03)----TableScan: agg_expr_parquet projection=[x, v] +physical_plan +01)SortExec: expr=[2 * CAST(x@0 AS Int64) ASC NULLS LAST], preserve_partitioning=[false] +02)--AggregateExec: mode=Single, gby=[x@0 as x], aggr=[sum(agg_expr_parquet.v)], ordering_mode=Sorted +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/agg_expr_sorted.parquet]]}, projection=[x, v], output_ordering=[x@0 ASC NULLS LAST], file_type=parquet + +query II +SELECT x, SUM(v) +FROM agg_expr_parquet +GROUP BY x +ORDER BY 2 * x ASC; +---- +1 60 +2 150 + +# Test 4: Reversed filesystem order with inferred ordering +# Create 3 parquet files with non-overlapping id ranges, named so filesystem +# order is OPPOSITE to data order. Each file is internally sorted by id ASC. +# Force target_partitions=1 so all files end up in one file group, which is +# where the inter-file ordering bug manifests. +# Without inter-file validation, the optimizer would incorrectly trust the +# inferred ordering and remove SortExec. + +# Save current target_partitions and set to 1 to force single file group +statement ok +SET datafusion.execution.target_partitions = 1; + +statement ok +CREATE TABLE reversed_high(id INT, value INT) AS VALUES (7, 700), (8, 800), (9, 900); + +statement ok +CREATE TABLE reversed_mid(id INT, value INT) AS VALUES (4, 400), (5, 500), (6, 600); + +statement ok +CREATE TABLE reversed_low(id INT, value INT) AS VALUES (1, 100), (2, 200), (3, 300); + +query I +COPY (SELECT * FROM reversed_high ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/reversed/a_high.parquet'; +---- +3 + +query I +COPY (SELECT * FROM reversed_mid ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/reversed/b_mid.parquet'; +---- +3 + +query I +COPY (SELECT * FROM reversed_low ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/reversed/c_low.parquet'; +---- +3 + +# External table with NO "WITH ORDER" — relies on inferred ordering from parquet metadata +statement ok +CREATE EXTERNAL TABLE reversed_parquet(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/reversed/'; + +# Test 4.1: SortExec must be present because files are not in inter-file order +query TT +EXPLAIN SELECT * FROM reversed_parquet ORDER BY id ASC; +---- +logical_plan +01)Sort: reversed_parquet.id ASC NULLS LAST +02)--TableScan: reversed_parquet projection=[id, value] +physical_plan +01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/a_high.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/c_low.parquet]]}, projection=[id, value], file_type=parquet + +# Test 4.2: Results must be correct +query II +SELECT * FROM reversed_parquet ORDER BY id ASC; +---- +1 100 +2 200 +3 300 +4 400 +5 500 +6 600 +7 700 +8 800 +9 900 + +# Test 5: Overlapping files with inferred ordering +# Create files with overlapping id ranges + +statement ok +CREATE TABLE overlap_x(id INT, value INT) AS VALUES (1, 100), (3, 300), (5, 500); + +statement ok +CREATE TABLE overlap_y(id INT, value INT) AS VALUES (2, 200), (4, 400), (6, 600); + +query I +COPY (SELECT * FROM overlap_x ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/overlap/file_x.parquet'; +---- +3 + +query I +COPY (SELECT * FROM overlap_y ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/overlap/file_y.parquet'; +---- +3 + +statement ok +CREATE EXTERNAL TABLE overlap_parquet(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/overlap/'; + +# Test 5.1: SortExec must be present because files have overlapping ranges +query TT +EXPLAIN SELECT * FROM overlap_parquet ORDER BY id ASC; +---- +logical_plan +01)Sort: overlap_parquet.id ASC NULLS LAST +02)--TableScan: overlap_parquet projection=[id, value] +physical_plan +01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/overlap/file_x.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/overlap/file_y.parquet]]}, projection=[id, value], file_type=parquet + +# Test 5.2: Results must be correct +query II +SELECT * FROM overlap_parquet ORDER BY id ASC; +---- +1 100 +2 200 +3 300 +4 400 +5 500 +6 600 + +# Test 6: WITH ORDER + reversed filesystem order +# Same file setup as Test 4 but explicitly declaring ordering via WITH ORDER. +# Even with WITH ORDER, the optimizer should detect that inter-file order is wrong +# and keep SortExec. + +statement ok +CREATE EXTERNAL TABLE reversed_with_order_parquet(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/reversed/' +WITH ORDER (id ASC); + +# Test 6.1: SortExec must be present despite WITH ORDER +query TT +EXPLAIN SELECT * FROM reversed_with_order_parquet ORDER BY id ASC; +---- +logical_plan +01)Sort: reversed_with_order_parquet.id ASC NULLS LAST +02)--TableScan: reversed_with_order_parquet projection=[id, value] +physical_plan +01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/a_high.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/c_low.parquet]]}, projection=[id, value], file_type=parquet + +# Test 6.2: Results must be correct +query II +SELECT * FROM reversed_with_order_parquet ORDER BY id ASC; +---- +1 100 +2 200 +3 300 +4 400 +5 500 +6 600 +7 700 +8 800 +9 900 + +# Test 7: Correctly ordered multi-file single group (positive case) +# Files are in CORRECT inter-file order within a single group. +# The validation should PASS and SortExec should be eliminated. + +statement ok +CREATE TABLE correct_low(id INT, value INT) AS VALUES (1, 100), (2, 200), (3, 300); + +statement ok +CREATE TABLE correct_mid(id INT, value INT) AS VALUES (4, 400), (5, 500), (6, 600); + +statement ok +CREATE TABLE correct_high(id INT, value INT) AS VALUES (7, 700), (8, 800), (9, 900); + +query I +COPY (SELECT * FROM correct_low ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/correct/a_low.parquet'; +---- +3 + +query I +COPY (SELECT * FROM correct_mid ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/correct/b_mid.parquet'; +---- +3 + +query I +COPY (SELECT * FROM correct_high ORDER BY id ASC) +TO 'test_files/scratch/sort_pushdown/correct/c_high.parquet'; +---- +3 + +statement ok +CREATE EXTERNAL TABLE correct_parquet(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/correct/' +WITH ORDER (id ASC); + +# Test 7.1: SortExec should be ELIMINATED — files are in correct inter-file order +query TT +EXPLAIN SELECT * FROM correct_parquet ORDER BY id ASC; +---- +logical_plan +01)Sort: correct_parquet.id ASC NULLS LAST +02)--TableScan: correct_parquet projection=[id, value] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/a_low.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/c_high.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet + +# Test 7.2: Results must be correct +query II +SELECT * FROM correct_parquet ORDER BY id ASC; +---- +1 100 +2 200 +3 300 +4 400 +5 500 +6 600 +7 700 +8 800 +9 900 + +# Test 7.3: DESC query on correctly ordered ASC files should still use SortExec +# Note: reverse_row_groups=true reverses the file list in the plan display +query TT +EXPLAIN SELECT * FROM correct_parquet ORDER BY id DESC; +---- +logical_plan +01)Sort: correct_parquet.id DESC NULLS FIRST +02)--TableScan: correct_parquet projection=[id, value] +physical_plan +01)SortExec: expr=[id@0 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/c_high.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/a_low.parquet]]}, projection=[id, value], file_type=parquet, reverse_row_groups=true + +query II +SELECT * FROM correct_parquet ORDER BY id DESC; +---- +9 900 +8 800 +7 700 +6 600 +5 500 +4 400 +3 300 +2 200 +1 100 + +# Test 8: DESC ordering with files in wrong inter-file DESC order +# Create files internally sorted by id DESC, but named so filesystem order +# is WRONG for DESC ordering (low values first in filesystem order). + +statement ok +CREATE TABLE desc_low(id INT, value INT) AS VALUES (3, 300), (2, 200), (1, 100); + +statement ok +CREATE TABLE desc_high(id INT, value INT) AS VALUES (9, 900), (8, 800), (7, 700); + +query I +COPY (SELECT * FROM desc_low ORDER BY id DESC) +TO 'test_files/scratch/sort_pushdown/desc_reversed/a_low.parquet'; +---- +3 + +query I +COPY (SELECT * FROM desc_high ORDER BY id DESC) +TO 'test_files/scratch/sort_pushdown/desc_reversed/b_high.parquet'; +---- +3 + +statement ok +CREATE EXTERNAL TABLE desc_reversed_parquet(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/desc_reversed/' +WITH ORDER (id DESC); + +# Test 8.1: SortExec must be present — files are in wrong inter-file DESC order +# (a_low has 1-3, b_high has 7-9; for DESC, b_high should come first) +query TT +EXPLAIN SELECT * FROM desc_reversed_parquet ORDER BY id DESC; +---- +logical_plan +01)Sort: desc_reversed_parquet.id DESC NULLS FIRST +02)--TableScan: desc_reversed_parquet projection=[id, value] +physical_plan +01)SortExec: expr=[id@0 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/desc_reversed/a_low.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/desc_reversed/b_high.parquet]]}, projection=[id, value], file_type=parquet + +# Test 8.2: Results must be correct +query II +SELECT * FROM desc_reversed_parquet ORDER BY id DESC; +---- +9 900 +8 800 +7 700 +3 300 +2 200 +1 100 + +# Test 9: Multi-column sort key validation +# Files have (category, id) ordering. Files share a boundary value on category='B' +# so column-level min/max statistics overlap on the primary key column. +# The validation conservatively rejects this because column-level stats can't +# precisely represent row-level boundaries for multi-column keys. + +statement ok +CREATE TABLE multi_col_a(category VARCHAR, id INT, value INT) AS VALUES +('A', 1, 10), ('A', 2, 20), ('B', 1, 30); + +statement ok +CREATE TABLE multi_col_b(category VARCHAR, id INT, value INT) AS VALUES +('B', 2, 40), ('C', 1, 50), ('C', 2, 60); + +query I +COPY (SELECT * FROM multi_col_a ORDER BY category ASC, id ASC) +TO 'test_files/scratch/sort_pushdown/multi_col/a_first.parquet'; +---- +3 + +query I +COPY (SELECT * FROM multi_col_b ORDER BY category ASC, id ASC) +TO 'test_files/scratch/sort_pushdown/multi_col/b_second.parquet'; +---- +3 + +statement ok +CREATE EXTERNAL TABLE multi_col_parquet(category VARCHAR, id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/multi_col/' +WITH ORDER (category ASC, id ASC); + +# Test 9.1: SortExec is present — validation conservatively rejects because +# column-level stats overlap on category='B' across both files +query TT +EXPLAIN SELECT * FROM multi_col_parquet ORDER BY category ASC, id ASC; +---- +logical_plan +01)Sort: multi_col_parquet.category ASC NULLS LAST, multi_col_parquet.id ASC NULLS LAST +02)--TableScan: multi_col_parquet projection=[category, id, value] +physical_plan +01)SortExec: expr=[category@0 ASC NULLS LAST, id@1 ASC NULLS LAST], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_col/a_first.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_col/b_second.parquet]]}, projection=[category, id, value], file_type=parquet + +# Test 9.2: Results must be correct +query TII +SELECT * FROM multi_col_parquet ORDER BY category ASC, id ASC; +---- +A 1 10 +A 2 20 +B 1 30 +B 2 40 +C 1 50 +C 2 60 + +# Test 9.3: Multi-column sort with non-overlapping primary key across files +# When files don't overlap on the primary column, validation succeeds. + +statement ok +CREATE TABLE multi_col_x(category VARCHAR, id INT, value INT) AS VALUES +('A', 1, 10), ('A', 2, 20); + +statement ok +CREATE TABLE multi_col_y(category VARCHAR, id INT, value INT) AS VALUES +('B', 1, 30), ('B', 2, 40); + +query I +COPY (SELECT * FROM multi_col_x ORDER BY category ASC, id ASC) +TO 'test_files/scratch/sort_pushdown/multi_col_clean/x_first.parquet'; +---- +2 + +query I +COPY (SELECT * FROM multi_col_y ORDER BY category ASC, id ASC) +TO 'test_files/scratch/sort_pushdown/multi_col_clean/y_second.parquet'; +---- +2 + +statement ok +CREATE EXTERNAL TABLE multi_col_clean_parquet(category VARCHAR, id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/multi_col_clean/' +WITH ORDER (category ASC, id ASC); + +# Test 9.3a: SortExec should be eliminated — non-overlapping primary column +query TT +EXPLAIN SELECT * FROM multi_col_clean_parquet ORDER BY category ASC, id ASC; +---- +logical_plan +01)Sort: multi_col_clean_parquet.category ASC NULLS LAST, multi_col_clean_parquet.id ASC NULLS LAST +02)--TableScan: multi_col_clean_parquet projection=[category, id, value] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_col_clean/x_first.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_col_clean/y_second.parquet]]}, projection=[category, id, value], output_ordering=[category@0 ASC NULLS LAST, id@1 ASC NULLS LAST], file_type=parquet + +# Test 9.3b: Results must be correct +query TII +SELECT * FROM multi_col_clean_parquet ORDER BY category ASC, id ASC; +---- +A 1 10 +A 2 20 +B 1 30 +B 2 40 + +# Test 10: Correctly ordered files WITH ORDER (positive counterpart to Test 6) +# Files in correct_parquet are in correct ASC order — WITH ORDER should pass validation +# and SortExec should be eliminated. + +statement ok +CREATE EXTERNAL TABLE correct_with_order_parquet(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/correct/' +WITH ORDER (id ASC); + +# Test 10.1: SortExec should be ELIMINATED — files are in correct order +query TT +EXPLAIN SELECT * FROM correct_with_order_parquet ORDER BY id ASC; +---- +logical_plan +01)Sort: correct_with_order_parquet.id ASC NULLS LAST +02)--TableScan: correct_with_order_parquet projection=[id, value] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/a_low.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/b_mid.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/correct/c_high.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet + +# Test 10.2: Results must be correct +query II +SELECT * FROM correct_with_order_parquet ORDER BY id ASC; +---- +1 100 +2 200 +3 300 +4 400 +5 500 +6 600 +7 700 +8 800 +9 900 + +# Test 11: Multiple file groups (target_partitions > 1) — each group has one file +# When files are spread across separate partitions (one file per group), each +# partition is trivially sorted and SortPreservingMergeExec handles the merge. + +# Restore higher target_partitions so files go into separate groups +statement ok +SET datafusion.execution.target_partitions = 4; + +statement ok +CREATE EXTERNAL TABLE multi_partition_parquet(id INT, value INT) +STORED AS PARQUET +LOCATION 'test_files/scratch/sort_pushdown/reversed/' +WITH ORDER (id ASC); + +# Test 11.1: With separate partitions, each file is trivially sorted. +# SortPreservingMergeExec merges, no SortExec needed per-partition. +query TT +EXPLAIN SELECT * FROM multi_partition_parquet ORDER BY id ASC; +---- +logical_plan +01)Sort: multi_partition_parquet.id ASC NULLS LAST +02)--TableScan: multi_partition_parquet projection=[id, value] +physical_plan +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] +02)--DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/a_high.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/b_mid.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/reversed/c_low.parquet]]}, projection=[id, value], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet + +# Test 11.2: Results must be correct +query II +SELECT * FROM multi_partition_parquet ORDER BY id ASC; +---- +1 100 +2 200 +3 300 +4 400 +5 500 +6 600 +7 700 +8 800 +9 900 + +# Restore target_partitions to 1 for remaining cleanup +statement ok +SET datafusion.execution.target_partitions = 2; + # Cleanup +statement ok +DROP TABLE reversed_high; + +statement ok +DROP TABLE reversed_mid; + +statement ok +DROP TABLE reversed_low; + +statement ok +DROP TABLE reversed_parquet; + +statement ok +DROP TABLE overlap_x; + +statement ok +DROP TABLE overlap_y; + +statement ok +DROP TABLE overlap_parquet; + +statement ok +DROP TABLE reversed_with_order_parquet; + +statement ok +DROP TABLE correct_low; + +statement ok +DROP TABLE correct_mid; + +statement ok +DROP TABLE correct_high; + +statement ok +DROP TABLE correct_parquet; + +statement ok +DROP TABLE desc_low; + +statement ok +DROP TABLE desc_high; + +statement ok +DROP TABLE desc_reversed_parquet; + +statement ok +DROP TABLE multi_col_a; + +statement ok +DROP TABLE multi_col_b; + +statement ok +DROP TABLE multi_col_parquet; + +statement ok +DROP TABLE multi_col_x; + +statement ok +DROP TABLE multi_col_y; + +statement ok +DROP TABLE multi_col_clean_parquet; + +statement ok +DROP TABLE correct_with_order_parquet; + +statement ok +DROP TABLE multi_partition_parquet; + statement ok DROP TABLE timestamp_data; @@ -882,5 +1624,11 @@ DROP TABLE signed_data; statement ok DROP TABLE signed_parquet; +statement ok +DROP TABLE agg_expr_data; + +statement ok +DROP TABLE agg_expr_parquet; + statement ok SET datafusion.optimizer.enable_sort_pushdown = true; diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 8ac8724683a8..b1329bf55346 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -6081,3 +6081,49 @@ WHERE acctbal > ( ); ---- 1 + +# Regression test for https://github.com/apache/datafusion/issues/20194 +# Window function with CASE WHEN in ORDER BY combined with NVL filter +# should not trigger SanityCheckPlan error from equivalence normalization +# replacing literals in sort expressions with complex filter expressions. +statement ok +CREATE TABLE issue_20194_t1 ( + value_1_1 decimal(25) NULL, + value_1_2 int NULL, + value_1_3 bigint NULL +); + +statement ok +CREATE TABLE issue_20194_t2 ( + value_2_1 bigint NULL, + value_2_2 varchar(140) NULL, + value_2_3 varchar(140) NULL +); + +statement ok +INSERT INTO issue_20194_t1 (value_1_1, value_1_2, value_1_3) VALUES (6774502793, 10040029, 1120); + +statement ok +INSERT INTO issue_20194_t2 (value_2_1, value_2_2, value_2_3) VALUES (1120, '0', '0'); + +query RII +SELECT + t1.value_1_1, t1.value_1_2, + ROW_NUMBER() OVER ( + PARTITION BY t1.value_1_1, t1.value_1_2 + ORDER BY + CASE WHEN t2.value_2_2 = '0' THEN 1 ELSE 0 END ASC, + CASE WHEN t2.value_2_3 = '0' THEN 1 ELSE 0 END ASC + ) AS ord +FROM issue_20194_t1 t1 +INNER JOIN issue_20194_t2 t2 + ON t1.value_1_3 = t2.value_2_1 + AND nvl(t2.value_2_3, '0') = '0'; +---- +6774502793 10040029 1 + +statement ok +DROP TABLE issue_20194_t1; + +statement ok +DROP TABLE issue_20194_t2; diff --git a/datafusion/substrait/tests/cases/logical_plans.rs b/datafusion/substrait/tests/cases/logical_plans.rs index 6de1a26230f3..1bf5d2573e8e 100644 --- a/datafusion/substrait/tests/cases/logical_plans.rs +++ b/datafusion/substrait/tests/cases/logical_plans.rs @@ -224,7 +224,6 @@ mod tests { Ok(()) } - #[tokio::test] async fn duplicate_name_in_union() -> Result<()> { let proto_plan = @@ -232,6 +231,31 @@ mod tests { let ctx = add_plan_schemas_to_ctx(SessionContext::new(), &proto_plan)?; let plan = from_substrait_plan(&ctx.state(), &proto_plan).await?; + assert_snapshot!( + plan, + @r" + Projection: foo AS col1, bar AS col2 + Union + Projection: foo, bar + Values: (Int64(100), Int64(200)) + Projection: x, foo + Values: (Int32(300), Int64(400)) + " + ); + + // Trigger execution to ensure plan validity + DataFrame::new(ctx.state(), plan).show().await?; + + Ok(()) + } + + #[tokio::test] + async fn duplicate_name_in_union_dd() -> Result<()> { + let proto_plan = + read_json("tests/testdata/test_plans/duplicate_name_in_union.substrait.json"); + let ctx = add_plan_schemas_to_ctx(SessionContext::new(), &proto_plan)?; + let plan = from_substrait_plan(&ctx.state(), &proto_plan).await?; + assert_snapshot!( plan, @r" diff --git a/dev/changelog/52.2.0.md b/dev/changelog/52.2.0.md new file mode 100644 index 000000000000..0801ec5e6a7e --- /dev/null +++ b/dev/changelog/52.2.0.md @@ -0,0 +1,47 @@ + + +# Apache DataFusion 52.2.0 Changelog + +This release consists of 5 commits from 3 contributors. See credits at the end of this changelog for more information. + +See the [upgrade guide](https://datafusion.apache.org/library-user-guide/upgrading.html) for information on how to upgrade from previous versions. + +**Other:** + +- [branch-52] fix: filter pushdown when merge filter (#20110) [#20289](https://github.com/apache/datafusion/pull/20289) (haohuaijin) +- [branch-52] FilterExec should remap indices of parent dynamic filters (#20286) [#20347](https://github.com/apache/datafusion/pull/20347) (alamb) +- [branch-52] fix: validate inter-file ordering in eq_properties() (#20329) [#20509](https://github.com/apache/datafusion/pull/20509) (alamb) +- Fix name tracker (#19856) [#20539](https://github.com/apache/datafusion/pull/20539) (hareshkh) +- [branch-52] fix: HashJoin panic with dictionary-encoded columns in multi-key joins (#20441) [#20512](https://github.com/apache/datafusion/pull/20512) (alamb) +- [branch-52] Fix incorrect `SortExec` removal before `AggregateExec` (#20247) [#20507](https://github.com/apache/datafusion/pull/20507) (alamb) +- [branch-52] Update aws-smithy, bytes and time for security audits [#20546](https://github.com/apache/datafusion/pull/20546) (alamb) +- [branch-52] Clamp early aggregation emit to the sort boundary when using partial group ordering (#20446) [#20558](https://github.com/apache/datafusion/pull/20558) (alamb) + +## Credits + +Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) per contributor. + +``` + 3 Andrew Lamb + 1 Haresh Khanna + 1 Huaijin +``` + +Thank you also to everyone who contributed in other ways such as filing issues, reviewing PRs, and providing feedback on this release. diff --git a/dev/changelog/52.3.0.md b/dev/changelog/52.3.0.md new file mode 100644 index 000000000000..ed505b7fc2d0 --- /dev/null +++ b/dev/changelog/52.3.0.md @@ -0,0 +1,50 @@ + + +# Apache DataFusion 52.3.0 Changelog + +This release consists of 7 commits from 4 contributors. See credits at the end of this changelog for more information. + +See the [upgrade guide](https://datafusion.apache.org/library-user-guide/upgrading.html) for information on how to upgrade from previous versions. + +**Performance related:** + +- [branch-52] perf: sort replace free()->try_grow() pattern with try_resize() to reduce memory pool interactions [#20732](https://github.com/apache/datafusion/pull/20732) (mbutrovich) + +**Other:** + +- [branch-52] Backport fix: SortMergeJoin don't wait for all input before emitting #20482 [#20699](https://github.com/apache/datafusion/pull/20699) (mbutrovich) +- [branch-52] Fix Arrow Spill Underrun (#20159) [#20684](https://github.com/apache/datafusion/pull/20684) (hareshkh) +- [branch-52] Fix constant value from stats (#20042) [#20709](https://github.com/apache/datafusion/pull/20709) (alamb) +- [branch-52] fix: `HashJoin` panic with String dictionary keys (don't flatten keys) (#20505) [#20708](https://github.com/apache/datafusion/pull/20708) (alamb) +- [branch-52] FFI_TableOptions are using default values only [#20705](https://github.com/apache/datafusion/pull/20705) (timsaucer) +- [branch-52] Fix repartition from dropping data when spilling (#20672) [#20777](https://github.com/apache/datafusion/pull/20777) (alamb) + +## Credits + +Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) per contributor. + +``` + 3 Andrew Lamb + 2 Matt Butrovich + 1 Haresh Khanna + 1 Tim Saucer +``` + +Thank you also to everyone who contributed in other ways such as filing issues, reviewing PRs, and providing feedback on this release. diff --git a/dev/changelog/52.4.0.md b/dev/changelog/52.4.0.md new file mode 100644 index 000000000000..04fba07cde9e --- /dev/null +++ b/dev/changelog/52.4.0.md @@ -0,0 +1,57 @@ + + +# Apache DataFusion 52.4.0 Changelog + +This release consists of 11 commits from 10 contributors. See credits at the end of this changelog for more information. + +See the [upgrade guide](https://datafusion.apache.org/library-user-guide/upgrading.html) for information on how to upgrade from previous versions. + +**Other:** + +- [branch-52] fix: maintain inner list nullability for (#19948) [#20878](https://github.com/apache/datafusion/pull/20878) (Jefffrey) +- [branch-52] fix: Ensure columns are casted to the correct names with Unions (#20146) [#20879](https://github.com/apache/datafusion/pull/20879) (nuno-faria) +- [branch-52] fix: interval analysis error when have two filterexec that inner filter proves zero selectivity (#20743) [#20880](https://github.com/apache/datafusion/pull/20880) (haohuaijin) +- [branch-52] fix: Return `probe_side.len()` for RightMark/Anti count(\*) queries (#20710) [#20881](https://github.com/apache/datafusion/pull/20881) (jonathanc-n) +- [branch-52] fix: disable dynamic filter pushdown for non min/max aggregates (#20279) [#20877](https://github.com/apache/datafusion/pull/20877) (notashes) +- [branch-52] Fix duplicate group keys after hash aggregation spill (#20724) (#20858) [#20917](https://github.com/apache/datafusion/pull/20917) (gboucher90) +- [branch-52] perf: Cache num_output_rows in sort merge join to avoid O(n) recount (#20478) [#20936](https://github.com/apache/datafusion/pull/20936) (andygrove) +- [branch-52] fix: SanityCheckPlan error with window functions and NVL filter (#20231) [#20931](https://github.com/apache/datafusion/pull/20931) (EeshanBembi) +- [branch-52] chore: Ignore RUSTSEC-2024-0014 (#20862) [#21020](https://github.com/apache/datafusion/pull/21020) (comphead) +- [branch-52] fix: InList Dictionary filter pushdown type mismatch (#20962) [#20997](https://github.com/apache/datafusion/pull/20997) (alamb) +- [branch-52] Update to use lz4_flex 0.12.1 and quinn-proto 0.11.14 [#21009](https://github.com/apache/datafusion/pull/21009) (alamb) + +## Credits + +Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) per contributor. + +``` + 2 Andrew Lamb + 1 Andy Grove + 1 EeshanBembi + 1 Guillaume Boucher + 1 Huaijin + 1 Jeffrey Vo + 1 Jonathan Chen + 1 Nuno Faria + 1 Oleks V + 1 notashes +``` + +Thank you also to everyone who contributed in other ways such as filing issues, reviewing PRs, and providing feedback on this release.