From a2919b6f205962233546ea215efcbda137030c28 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 16 Jan 2025 15:21:51 +0300 Subject: [PATCH 01/62] add monotonic function definitions for aggregate expressions --- .../src/physical_optimizer/enforce_sorting.rs | 273 +++++++++++++++++- .../core/src/physical_optimizer/test_utils.rs | 107 ++++++- datafusion/expr/src/udaf.rs | 14 +- datafusion/expr/src/window_frame.rs | 7 + datafusion/functions-aggregate/src/count.rs | 4 + datafusion/functions-aggregate/src/min_max.rs | 8 + datafusion/physical-expr/src/aggregate.rs | 36 ++- .../src/equivalence/properties.rs | 24 ++ .../physical-expr/src/window/aggregate.rs | 22 +- .../src/window/sliding_aggregate.rs | 24 +- .../physical-expr/src/window/standard.rs | 22 +- .../physical-plan/src/aggregates/mod.rs | 10 +- datafusion/physical-plan/src/windows/mod.rs | 10 +- .../sqllogictest/test_files/aggregate.slt | 21 +- .../test_files/aggregates_topk.slt | 13 +- 15 files changed, 533 insertions(+), 62 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index dd8e9d900b7d..f7045789c1c8 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -659,7 +659,9 @@ mod tests { use super::*; use crate::physical_optimizer::enforce_distribution::EnforceDistribution; use crate::physical_optimizer::test_utils::{ - aggregate_exec, bounded_window_exec, check_integrity, coalesce_batches_exec, + aggregate_exec, aggregate_exec_monotonic, aggregate_exec_non_monotonic, + bounded_window_exec, bounded_window_exec_non_monotonic, + bounded_window_exec_with_partition, check_integrity, coalesce_batches_exec, coalesce_partitions_exec, filter_exec, global_limit_exec, hash_join_exec, limit_exec, local_limit_exec, memory_exec, parquet_exec, parquet_exec_sorted, repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_merge_join_exec, @@ -705,6 +707,17 @@ mod tests { Ok(schema) } + // Generate a schema which consists of 5 columns (a, b, c, d, e) of Uint64 + fn create_test_schema4() -> Result { + let a = Field::new("a", DataType::UInt64, true); + let b = Field::new("b", DataType::UInt64, false); + let c = Field::new("c", DataType::UInt64, true); + let d = Field::new("d", DataType::UInt64, false); + let e = Field::new("e", DataType::UInt64, false); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e])); + Ok(schema) + } + /// Runs the sort enforcement optimizer and asserts the plan /// against the original and expected plans /// @@ -1008,6 +1021,63 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_aggregate_monotonic() -> Result<()> { + let schema = create_test_schema4()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("a", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + + let aggregate = aggregate_exec_monotonic(sort); + let sort_exprs = LexOrdering::new(vec![sort_expr("count", &aggregate.schema())]); + let physical_plan: Arc = + Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _; + + let expected_input = [ + "SortExec: expr=[count@0 ASC], preserve_partitioning=[false]", + " AggregateExec: mode=Final, gby=[], aggr=[count]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + + let expected_optimized = [ + "AggregateExec: mode=Final, gby=[], aggr=[count]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) + } + + #[tokio::test] + async fn test_aggregate_non_monotonic() -> Result<()> { + let schema = create_test_schema4()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("a", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + + let aggregate = aggregate_exec_non_monotonic(sort); + let sort_exprs = LexOrdering::new(vec![sort_expr("avg", &aggregate.schema())]); + let physical_plan: Arc = + Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _; + + let expected_input = [ + "SortExec: expr=[avg@0 ASC], preserve_partitioning=[false]", + " AggregateExec: mode=Final, gby=[], aggr=[avg]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + + let expected_optimized = [ + "SortExec: expr=[avg@0 ASC], preserve_partitioning=[false]", + " AggregateExec: mode=Final, gby=[], aggr=[avg]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) + } + #[tokio::test] async fn test_remove_unnecessary_sort4() -> Result<()> { let schema = create_test_schema()?; @@ -1756,6 +1826,207 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_bounded_window_monotonic_sort() -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let source = parquet_exec_sorted(&schema, sort_exprs.clone()); + let sort = sort_exec(sort_exprs.clone(), source); + + let bounded_window = + bounded_window_exec("nullable_col", sort_exprs.clone(), sort); + let output_schema = bounded_window.schema(); + let sort_exprs2 = vec![sort_expr_options( + "count", + &output_schema, + SortOptions { + descending: false, + nulls_first: false, + }, + )]; + let physical_plan = sort_exec(sort_exprs2.clone(), bounded_window); + + let expected_input = [ + "SortExec: expr=[count@2 ASC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + ]; + let expected_optimized = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) + } + + #[tokio::test] + async fn test_bounded_plain_window_monotonic_sort_with_partitions() -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let source = parquet_exec_sorted(&schema, sort_exprs.clone()); + let sort = sort_exec(sort_exprs.clone(), source); + let partition_bys = &[col("nullable_col", &schema)?]; + + let bounded_window = bounded_window_exec_with_partition( + "nullable_col", + sort_exprs.clone(), + partition_bys, + sort, + false, + ); + let output_schema = bounded_window.schema(); + let sort_exprs2 = vec![ + sort_expr_options( + "nullable_col", + &output_schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + sort_expr_options( + "count", + &output_schema, + SortOptions { + descending: false, + nulls_first: false, + }, + ), + ]; + let physical_plan = sort_exec(sort_exprs2.clone(), bounded_window); + + let expected_input = [ + "SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + ]; + let expected_optimized = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) + } + + #[tokio::test] + async fn test_bounded_sliding_window_monotonic_sort_with_partitions() -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let source = parquet_exec_sorted(&schema, sort_exprs.clone()); + let sort = sort_exec(sort_exprs.clone(), source); + let partition_bys = &[col("nullable_col", &schema)?]; + + let bounded_window = bounded_window_exec_with_partition( + "nullable_col", + sort_exprs.clone(), + partition_bys, + sort, + true, + ); + let output_schema = bounded_window.schema(); + let sort_exprs2 = vec![ + sort_expr_options( + "nullable_col", + &output_schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + sort_expr_options( + "count", + &output_schema, + SortOptions { + descending: false, + nulls_first: false, + }, + ), + ]; + let physical_plan = sort_exec(sort_exprs2.clone(), bounded_window); + + let expected_input = [ + "SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + ]; + let expected_optimized = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }], mode=[Sorted]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) + } + + #[tokio::test] + async fn test_bounded_window_non_monotonic_sort() -> Result<()> { + let schema = create_test_schema4()?; + let sort_exprs = vec![sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let source = parquet_exec_sorted(&schema, sort_exprs.clone()); + let sort = sort_exec(sort_exprs.clone(), source); + + let bounded_window = + bounded_window_exec_non_monotonic("a", sort_exprs.clone(), sort); + let output_schema = bounded_window.schema(); + let sort_exprs2 = vec![sort_expr_options( + "avg", + &output_schema, + SortOptions { + descending: false, + nulls_first: false, + }, + )]; + let physical_plan = sort_exec(sort_exprs2.clone(), bounded_window); + + let expected_input = [ + "SortExec: expr=[avg@5 ASC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[avg: Ok(Field { name: \"avg\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST]", + ]; + let expected_optimized = [ + "SortExec: expr=[avg@5 ASC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[avg: Ok(Field { name: \"avg\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) + } + #[tokio::test] async fn test_window_multi_path_sort2() -> Result<()> { let schema = create_test_schema()?; diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 9156301393c0..542d496a7052 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -39,7 +39,7 @@ use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::union::UnionExec; -use crate::physical_plan::windows::create_window_expr; +use crate::physical_plan::windows::{create_window_expr, BoundedWindowAggExec}; use crate::physical_plan::{ExecutionPlan, InputOrderMode, Partitioning}; use crate::prelude::{CsvReadOptions, SessionContext}; @@ -47,18 +47,20 @@ use arrow_schema::{Schema, SchemaRef, SortOptions}; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::JoinType; use datafusion_execution::object_store::ObjectStoreUrl; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_expr::test::function_stub::avg_udaf; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; use datafusion_functions_aggregate::count::count_udaf; +use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::{ displayable, DisplayAs, DisplayFormatType, PlanProperties, }; use async_trait::async_trait; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; async fn register_current_csv( ctx: &SessionContext, @@ -242,15 +244,58 @@ pub fn bounded_window_exec( col_name: &str, sort_exprs: impl IntoIterator, input: Arc, +) -> Arc { + bounded_window_exec_with_partition(col_name, sort_exprs, &[], input, false) +} + +pub fn bounded_window_exec_with_partition( + col_name: &str, + sort_exprs: impl IntoIterator, + partition_by: &[Arc], + input: Arc, + should_reverse: bool, +) -> Arc { + let sort_exprs: LexOrdering = sort_exprs.into_iter().collect(); + let schema = input.schema(); + let mut window_expr = create_window_expr( + &WindowFunctionDefinition::AggregateUDF(count_udaf()), + "count".to_owned(), + &[col(col_name, &schema).unwrap()], + partition_by, + sort_exprs.as_ref(), + Arc::new(WindowFrame::new(Some(false))), + schema.as_ref(), + false, + ) + .unwrap(); + if should_reverse { + window_expr = window_expr.get_reverse_expr().unwrap(); + } + + Arc::new( + BoundedWindowAggExec::try_new( + vec![window_expr], + input.clone(), + vec![], + InputOrderMode::Sorted, + ) + .unwrap(), + ) +} + +pub fn bounded_window_exec_non_monotonic( + col_name: &str, + sort_exprs: impl IntoIterator, + input: Arc, ) -> Arc { let sort_exprs: LexOrdering = sort_exprs.into_iter().collect(); let schema = input.schema(); Arc::new( - crate::physical_plan::windows::BoundedWindowAggExec::try_new( + BoundedWindowAggExec::try_new( vec![create_window_expr( - &WindowFunctionDefinition::AggregateUDF(count_udaf()), - "count".to_owned(), + &WindowFunctionDefinition::AggregateUDF(avg_udaf()), + "avg".to_owned(), &[col(col_name, &schema).unwrap()], &[], sort_exprs.as_ref(), @@ -349,6 +394,56 @@ pub fn aggregate_exec(input: Arc) -> Arc { ) } +pub fn aggregate_exec_monotonic(input: Arc) -> Arc { + let schema = input.schema(); + let aggregate_expr = + vec![ + AggregateExprBuilder::new(count_udaf(), vec![col("d", &schema).unwrap()]) + .schema(Arc::clone(&schema)) + .alias("count") + .build() + .map(Arc::new) + .unwrap(), + ]; + Arc::new( + AggregateExec::try_new( + AggregateMode::Final, + PhysicalGroupBy::default(), + aggregate_expr, + vec![None], + input, + schema, + ) + .unwrap(), + ) +} + +pub fn aggregate_exec_non_monotonic( + input: Arc, +) -> Arc { + let schema = input.schema(); + let aggregate_expr = + vec![ + AggregateExprBuilder::new(avg_udaf(), vec![col("d", &schema).unwrap()]) + .schema(Arc::clone(&schema)) + .alias("avg") + .build() + .map(Arc::new) + .unwrap(), + ]; + Arc::new( + AggregateExec::try_new( + AggregateMode::Final, + PhysicalGroupBy::default(), + aggregate_expr, + vec![None], + input, + schema, + ) + .unwrap(), + ) +} + pub fn coalesce_batches_exec(input: Arc) -> Arc { Arc::new(CoalesceBatchesExec::new(input, 128)) } diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 56c9822495f8..ce9025f98bfd 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -389,7 +389,7 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// Whether the aggregate function is nullable. /// - /// Nullable means that that the function could return `null` for any inputs. + /// Nullable means that the function could return `null` for any inputs. /// For example, aggregate functions like `COUNT` always return a non null value /// but others like `MIN` will return `NULL` if there is nullable input. /// Note that if the function is declared as *not* nullable, make sure the [`AggregateUDFImpl::default_value`] is `non-null` @@ -635,6 +635,18 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { fn documentation(&self) -> Option<&Documentation> { None } + + /// Indicates whether the aggregation function is monotonic as a set function. A set + /// function is monotonically increasing if its value increases as its argument grows + /// (as a set). Formally, `f` is a monotonically increasing set function if `f(S) >= f(T)` + /// whenever `S` is a superset of `T`. + /// + /// Returns None if the function is not monotonic. + /// If the function is monotonically decreasing returns Some(false) e.g. Min + /// If the function is monotonically increasing returns Some(true) e.g. Max + fn is_monotonic(&self) -> Option { + None + } } impl PartialEq for dyn AggregateUDFImpl { diff --git a/datafusion/expr/src/window_frame.rs b/datafusion/expr/src/window_frame.rs index 815d5742afd2..82b33650523b 100644 --- a/datafusion/expr/src/window_frame.rs +++ b/datafusion/expr/src/window_frame.rs @@ -291,6 +291,13 @@ impl WindowFrame { && (self.end_bound.is_unbounded() || self.end_bound == WindowFrameBound::CurrentRow) } + + /// Is the window frame ever-expanding (it always grows in the superset sense). + /// Useful when understanding if set-monotonicity properties of functions can + /// be exploited. + pub fn is_ever_expanding(&self) -> bool { + self.start_bound.is_unbounded() + } } /// There are five ways to describe starting and ending frame boundaries: diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index f2621b48be5c..ea762d63aa25 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -351,6 +351,10 @@ impl AggregateUDFImpl for Count { fn documentation(&self) -> Option<&Documentation> { self.doc() } + + fn is_monotonic(&self) -> Option { + Some(true) + } } #[derive(Debug)] diff --git a/datafusion/functions-aggregate/src/min_max.rs b/datafusion/functions-aggregate/src/min_max.rs index c4e05bd57de6..991d115dfb87 100644 --- a/datafusion/functions-aggregate/src/min_max.rs +++ b/datafusion/functions-aggregate/src/min_max.rs @@ -361,6 +361,10 @@ impl AggregateUDFImpl for Max { fn documentation(&self) -> Option<&Documentation> { self.doc() } + + fn is_monotonic(&self) -> Option { + Some(true) + } } // Statically-typed version of min/max(array) -> ScalarValue for string types @@ -1183,6 +1187,10 @@ impl AggregateUDFImpl for Min { fn documentation(&self) -> Option<&Documentation> { self.doc() } + + fn is_monotonic(&self) -> Option { + Some(false) + } } /// An accumulator to compute the minimum value diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 4eaabace7257..821272e91612 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -35,24 +35,27 @@ pub mod utils { }; } +use std::fmt::Debug; +use std::sync::Arc; + +use crate::expressions::Column; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow_schema::SortOptions; use datafusion_common::ScalarValue; use datafusion_common::{internal_err, not_impl_err, Result}; use datafusion_expr::AggregateUDF; use datafusion_expr::ReversedUDAF; use datafusion_expr_common::accumulator::Accumulator; +use datafusion_expr_common::groups_accumulator::GroupsAccumulator; use datafusion_expr_common::type_coercion::aggregates::check_arg_count; use datafusion_functions_aggregate_common::accumulator::AccumulatorArgs; use datafusion_functions_aggregate_common::accumulator::StateFieldsArgs; use datafusion_functions_aggregate_common::order::AggregateOrderSensitivity; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_expr_common::utils::reverse_order_bys; -use datafusion_expr_common::groups_accumulator::GroupsAccumulator; -use std::fmt::Debug; -use std::sync::Arc; - /// Builder for physical [`AggregateFunctionExpr`] /// /// `AggregateFunctionExpr` contains the information necessary to call @@ -533,6 +536,29 @@ impl AggregateFunctionExpr { pub fn default_value(&self, data_type: &DataType) -> Result { self.fun.default_value(data_type) } + + /// Indicates whether the aggregation function is monotonic as a set function. A set + /// function is monotonically increasing if its value increases as its argument grows + /// (as a set). Formally, `f` is a monotonically increasing set function if `f(S) >= f(T)` + /// whenever `S` is a superset of `T`. + /// + /// Returns None if the function is not monotonic. + /// If the function is monotonically decreasing returns Some(false) e.g. Min + /// If the function is monotonically increasing returns Some(true) e.g. Max + pub fn is_monotonic(&self) -> Option { + self.fun.inner().is_monotonic() + } + + /// Returns PhysicalSortExpr based on monotonicity of the function + pub fn natural_sort_expr(&self, schema: &SchemaRef) -> Option { + // If the aggregate expressions are monotonic, the output data is naturally ordered with it. + let is_ascending = self.is_monotonic()?; + let idx = schema.index_of(self.name()).unwrap_or(0); + let expr = Arc::new(Column::new(self.name(), idx)); + + let options = SortOptions::new(!is_ascending, false); + Some(PhysicalSortExpr { expr, options }) + } } /// Stores the physical expressions used inside the `AggregateExpr`. diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index 2c7335649b28..beb118de1bbf 100755 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -239,6 +239,30 @@ impl EquivalenceProperties { self.oeq_class.add_new_orderings(orderings); } + /// Adds new ordering expression into the existing ordering equivalence class based on partition by information. + pub fn add_new_ordering_expr_with_partition_by( + &mut self, + expr: PhysicalSortExpr, + partition_by: &Vec>, + ) { + if partition_by.is_empty() { + // In the absence of a PARTITION BY, ordering of `self.expr` is global: + self.add_new_orderings([LexOrdering::new(vec![expr])]); + } else { + // If we have a PARTITION BY, standard functions can not introduce + // a global ordering unless the existing ordering is compatible + // with PARTITION BY expressions. To elaborate, when PARTITION BY + // expressions and existing ordering expressions are equal (w.r.t. + // set equality), we can prefix the ordering of `self.expr` with + // the existing ordering. + let (mut ordering, _) = self.find_longest_permutation(partition_by); + if ordering.len() == partition_by.len() { + ordering.push(expr); + self.add_new_orderings([ordering]); + } + } + } + /// Adds a single ordering to the existing ordering equivalence class. pub fn add_new_ordering(&mut self, ordering: LexOrdering) { self.add_new_orderings([ordering]); diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index 0c56bdc92985..3ce528adf3c0 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -21,16 +21,16 @@ use std::any::Any; use std::ops::Range; use std::sync::Arc; -use arrow::array::Array; -use arrow::record_batch::RecordBatch; -use arrow::{array::ArrayRef, datatypes::Field}; - use crate::aggregate::AggregateFunctionExpr; use crate::window::window_expr::AggregateWindowExpr; use crate::window::{ PartitionBatches, PartitionWindowAggStates, SlidingAggregateWindowExpr, WindowExpr, }; -use crate::{reverse_order_bys, PhysicalExpr}; +use crate::{reverse_order_bys, EquivalenceProperties, PhysicalExpr}; + +use arrow::array::Array; +use arrow::record_batch::RecordBatch; +use arrow::{array::ArrayRef, datatypes::Field}; use datafusion_common::ScalarValue; use datafusion_common::{DataFusionError, Result}; use datafusion_expr::{Accumulator, WindowFrame}; @@ -67,6 +67,16 @@ impl PlainAggregateWindowExpr { pub fn get_aggregate_expr(&self) -> &AggregateFunctionExpr { &self.aggregate } + + pub fn add_equal_orderings(&self, eq_properties: &mut EquivalenceProperties) { + let Some(expr) = self + .get_aggregate_expr() + .natural_sort_expr(eq_properties.schema()) + else { + return; + }; + eq_properties.add_new_ordering_expr_with_partition_by(expr, &self.partition_by); + } } /// peer based evaluation based on the fact that batch is pre-sorted given the sort columns @@ -135,7 +145,7 @@ impl WindowExpr for PlainAggregateWindowExpr { fn get_reverse_expr(&self) -> Option> { self.aggregate.reverse_expr().map(|reverse_expr| { let reverse_window_frame = self.window_frame.reverse(); - if reverse_window_frame.start_bound.is_unbounded() { + if reverse_window_frame.is_ever_expanding() { Arc::new(PlainAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index 572eb8866a44..77376ad61374 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -21,16 +21,18 @@ use std::any::Any; use std::ops::Range; use std::sync::Arc; -use arrow::array::{Array, ArrayRef}; -use arrow::datatypes::Field; -use arrow::record_batch::RecordBatch; - use crate::aggregate::AggregateFunctionExpr; use crate::window::window_expr::AggregateWindowExpr; use crate::window::{ PartitionBatches, PartitionWindowAggStates, PlainAggregateWindowExpr, WindowExpr, }; -use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; +use crate::{ + expressions::PhysicalSortExpr, reverse_order_bys, EquivalenceProperties, PhysicalExpr, +}; + +use arrow::array::{Array, ArrayRef}; +use arrow::datatypes::Field; +use arrow::record_batch::RecordBatch; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::{Accumulator, WindowFrame}; use datafusion_physical_expr_common::sort_expr::LexOrdering; @@ -67,6 +69,16 @@ impl SlidingAggregateWindowExpr { pub fn get_aggregate_expr(&self) -> &AggregateFunctionExpr { &self.aggregate } + + pub fn add_equal_orderings(&self, eq_properties: &mut EquivalenceProperties) { + let Some(expr) = self + .get_aggregate_expr() + .natural_sort_expr(eq_properties.schema()) + else { + return; + }; + eq_properties.add_new_ordering_expr_with_partition_by(expr, &self.partition_by); + } } /// Incrementally update window function using the fact that batch is @@ -119,7 +131,7 @@ impl WindowExpr for SlidingAggregateWindowExpr { fn get_reverse_expr(&self) -> Option> { self.aggregate.reverse_expr().map(|reverse_expr| { let reverse_window_frame = self.window_frame.reverse(); - if reverse_window_frame.start_bound.is_unbounded() { + if reverse_window_frame.is_ever_expanding() { Arc::new(PlainAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), diff --git a/datafusion/physical-expr/src/window/standard.rs b/datafusion/physical-expr/src/window/standard.rs index 82e48a5f9338..158dff507871 100644 --- a/datafusion/physical-expr/src/window/standard.rs +++ b/datafusion/physical-expr/src/window/standard.rs @@ -74,24 +74,10 @@ impl StandardWindowExpr { pub fn add_equal_orderings(&self, eq_properties: &mut EquivalenceProperties) { let schema = eq_properties.schema(); if let Some(fn_res_ordering) = self.expr.get_result_ordering(schema) { - if self.partition_by.is_empty() { - // In the absence of a PARTITION BY, ordering of `self.expr` is global: - eq_properties - .add_new_orderings([LexOrdering::new(vec![fn_res_ordering])]); - } else { - // If we have a PARTITION BY, standard functions can not introduce - // a global ordering unless the existing ordering is compatible - // with PARTITION BY expressions. To elaborate, when PARTITION BY - // expressions and existing ordering expressions are equal (w.r.t. - // set equality), we can prefix the ordering of `self.expr` with - // the existing ordering. - let (mut ordering, _) = - eq_properties.find_longest_permutation(&self.partition_by); - if ordering.len() == self.partition_by.len() { - ordering.push(fn_res_ordering); - eq_properties.add_new_orderings([ordering]); - } - } + eq_properties.add_new_ordering_expr_with_partition_by( + fn_res_ordering, + &self.partition_by, + ); } } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index cc8d6e74f4b9..cdfd947b1c50 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -512,6 +512,7 @@ impl AggregateExec { &projection_mapping, &mode, &input_order_mode, + aggr_expr.clone(), ); Ok(AggregateExec { @@ -648,9 +649,10 @@ impl AggregateExec { projection_mapping: &ProjectionMapping, mode: &AggregateMode, input_order_mode: &InputOrderMode, + aggr_exprs: Vec>, ) -> PlanProperties { // Construct equivalence properties: - let eq_properties = input + let mut eq_properties = input .equivalence_properties() .project(projection_mapping, schema); @@ -673,6 +675,12 @@ impl AggregateExec { input.pipeline_behavior() }; + for aggr_expr in aggr_exprs { + if let Some(expr) = aggr_expr.natural_sort_expr(eq_properties.schema()) { + eq_properties.add_new_ordering(LexOrdering::new(vec![expr])); + } + } + PlanProperties::new( eq_properties, output_partitioning, diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 510cbc248b63..4c9df003642f 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -133,7 +133,7 @@ fn window_expr_from_aggregate_expr( aggregate: Arc, ) -> Arc { // Is there a potentially unlimited sized window frame? - let unbounded_window = window_frame.start_bound.is_unbounded(); + let unbounded_window = window_frame.is_ever_expanding(); if !unbounded_window { Arc::new(SlidingAggregateWindowExpr::new( @@ -347,6 +347,14 @@ pub(crate) fn window_equivalence_properties( if let Some(udf_window_expr) = expr.as_any().downcast_ref::() { udf_window_expr.add_equal_orderings(&mut window_eq_properties); + } else if let Some(aggregate_udf_window_expr) = + expr.as_any().downcast_ref::() + { + aggregate_udf_window_expr.add_equal_orderings(&mut window_eq_properties); + } else if let Some(aggregate_udf_window_expr) = + expr.as_any().downcast_ref::() + { + aggregate_udf_window_expr.add_equal_orderings(&mut window_eq_properties); } } window_eq_properties diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index bd3b40089519..8aa132abc3a3 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -4978,16 +4978,17 @@ logical_plan physical_plan 01)GlobalLimitExec: skip=0, fetch=5 02)--CoalescePartitionsExec -03)----AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], lim=[5] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([c3@0, min(aggregate_test_100.c1)@1], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], lim=[5] -07)------------AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3], aggr=[min(aggregate_test_100.c1)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 -10)------------------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)] -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], has_header=true +03)----AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], ordering_mode=PartiallySorted([1]) +04)------SortExec: expr=[min(aggregate_test_100.c1)@1 ASC NULLS LAST], preserve_partitioning=[true] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([c3@0, min(aggregate_test_100.c1)@1], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], ordering_mode=PartiallySorted([1]) +08)--------------AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3], aggr=[min(aggregate_test_100.c1)] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 +11)--------------------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)] +12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], has_header=true # diff --git a/datafusion/sqllogictest/test_files/aggregates_topk.slt b/datafusion/sqllogictest/test_files/aggregates_topk.slt index a67fec695f6c..9fe17a45edc1 100644 --- a/datafusion/sqllogictest/test_files/aggregates_topk.slt +++ b/datafusion/sqllogictest/test_files/aggregates_topk.slt @@ -143,13 +143,12 @@ logical_plan 03)----TableScan: traces projection=[trace_id, timestamp] physical_plan 01)SortPreservingMergeExec: [max(traces.timestamp)@1 ASC NULLS LAST], fetch=4 -02)--SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 ASC NULLS LAST], preserve_partitioning=[true] -03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +02)--AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +07)------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by trace_id asc limit 4; From 14109e6b050467e441c8b867d5ccf489e1907e85 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 16 Jan 2025 15:42:15 +0300 Subject: [PATCH 02/62] fix benchmark results --- .../test_files/tpch/plans/q13.slt.part | 35 ++++++++++--------- 1 file changed, 18 insertions(+), 17 deletions(-) diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index 2667bb23a459..b47567eaa613 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -56,20 +56,21 @@ physical_plan 01)SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[c_count@0 as c_count, count(*)@1 as custdist] -04)------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(*)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(*)] -08)--------------ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] -09)----------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false -16)----------------------CoalesceBatchesExec: target_batch_size=8192 -17)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -18)--------------------------CoalesceBatchesExec: target_batch_size=8192 -19)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] -20)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false +04)------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(*)], ordering_mode=Sorted +05)--------SortExec: expr=[c_count@0 ASC NULLS LAST], preserve_partitioning=[true] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(*)], ordering_mode=Sorted +09)----------------ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] +10)------------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false +17)------------------------CoalesceBatchesExec: target_batch_size=8192 +18)--------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +19)----------------------------CoalesceBatchesExec: target_batch_size=8192 +20)------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] +21)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false \ No newline at end of file From b3d75ba95fd312f6805c5485c35e81dd4eed6993 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 17 Jan 2025 14:14:57 +0300 Subject: [PATCH 03/62] set prefer_existing_sort to true in sqllogictests --- .../sqllogictest/test_files/aggregate.slt | 24 +++++----- .../test_files/tpch/plans/q13.slt.part | 44 +++++++++++-------- 2 files changed, 39 insertions(+), 29 deletions(-) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 8aa132abc3a3..ed5eca8cc879 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -4963,6 +4963,9 @@ false true NULL +statement ok +set datafusion.optimizer.prefer_existing_sort = true; + # # Add valid distinct case as aggregation plan test # @@ -4979,17 +4982,18 @@ physical_plan 01)GlobalLimitExec: skip=0, fetch=5 02)--CoalescePartitionsExec 03)----AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], ordering_mode=PartiallySorted([1]) -04)------SortExec: expr=[min(aggregate_test_100.c1)@1 ASC NULLS LAST], preserve_partitioning=[true] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([c3@0, min(aggregate_test_100.c1)@1], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], ordering_mode=PartiallySorted([1]) -08)--------------AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3], aggr=[min(aggregate_test_100.c1)] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 -11)--------------------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)] -12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], has_header=true +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([c3@0, min(aggregate_test_100.c1)@1], 4), input_partitions=4, preserve_order=true, sort_exprs=min(aggregate_test_100.c1)@1 DESC NULLS LAST +06)----------AggregateExec: mode=Partial, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], ordering_mode=PartiallySorted([1]) +07)------------AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3], aggr=[min(aggregate_test_100.c1)] +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 +10)------------------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)] +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], has_header=true +statement ok +set datafusion.optimizer.prefer_existing_sort = false; # # Push limit into distinct group-by aggregation tests diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index b47567eaa613..d07bb1cb84fc 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -16,6 +16,9 @@ # specific language governing permissions and limitations # under the License. +statement ok +set datafusion.optimizer.prefer_existing_sort = true; + query TT explain select c_count, @@ -53,24 +56,27 @@ logical_plan 11)------------------Filter: orders.o_comment NOT LIKE Utf8("%special%requests%") 12)--------------------TableScan: orders projection=[o_orderkey, o_custkey, o_comment], partial_filters=[orders.o_comment NOT LIKE Utf8("%special%requests%")] physical_plan -01)SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC], fetch=10 -02)--SortExec: TopK(fetch=10), expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true] +01)SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC], fetch=10 +02)--SortExec: TopK(fetch=10), expr=[custdist@1 DESC,c_count@0 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[c_count@0 as c_count, count(*)@1 as custdist] -04)------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(*)], ordering_mode=Sorted -05)--------SortExec: expr=[c_count@0 ASC NULLS LAST], preserve_partitioning=[true] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4 -08)--------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(*)], ordering_mode=Sorted -09)----------------ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] -10)------------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false -17)------------------------CoalesceBatchesExec: target_batch_size=8192 -18)--------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +04)------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(*)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(*)] +08)--------------ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] +09)----------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false +16)----------------------CoalesceBatchesExec: target_batch_size=8192 +17)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +18)--------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] 19)----------------------------CoalesceBatchesExec: target_batch_size=8192 -20)------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] -21)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false \ No newline at end of file +20)------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% +21)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false + +statement ok +set datafusion.optimizer.prefer_existing_sort = false; \ No newline at end of file From 549502e80234c112c90379bf0187ac4de542b2a1 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 17 Jan 2025 14:33:34 +0300 Subject: [PATCH 04/62] set prefer_existing_sort to true in sqllogictests --- .../test_files/tpch/plans/q13.slt.part | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index d07bb1cb84fc..73fe2cc6b430 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -56,13 +56,13 @@ logical_plan 11)------------------Filter: orders.o_comment NOT LIKE Utf8("%special%requests%") 12)--------------------TableScan: orders projection=[o_orderkey, o_custkey, o_comment], partial_filters=[orders.o_comment NOT LIKE Utf8("%special%requests%")] physical_plan -01)SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC], fetch=10 -02)--SortExec: TopK(fetch=10), expr=[custdist@1 DESC,c_count@0 DESC], preserve_partitioning=[true] +01)SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC], fetch=10 +02)--SortExec: TopK(fetch=10), expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[c_count@0 as c_count, count(*)@1 as custdist] -04)------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(*)] +04)------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(*)], ordering_mode=Sorted 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(*)] +06)----------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4, preserve_order=true, sort_exprs=c_count@0 ASC NULLS LAST, count(*)@0 ASC NULLS LAST +07)------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(*)], ordering_mode=Sorted08)--------------ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] 08)--------------ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] 09)----------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] 10)------------------CoalesceBatchesExec: target_batch_size=8192 @@ -73,10 +73,9 @@ physical_plan 15)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false 16)----------------------CoalesceBatchesExec: target_batch_size=8192 17)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -18)--------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -19)----------------------------CoalesceBatchesExec: target_batch_size=8192 -20)------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% -21)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false +18)--------------------------CoalesceBatchesExec: target_batch_size=8192 +19)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] +20)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false statement ok set datafusion.optimizer.prefer_existing_sort = false; \ No newline at end of file From 623e0c5dd741865d8e3a89a936d591084e2c7061 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 17 Jan 2025 15:19:23 +0300 Subject: [PATCH 05/62] fix typo --- datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index 73fe2cc6b430..85ecb8f56bdf 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -62,7 +62,7 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(*)], ordering_mode=Sorted 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4, preserve_order=true, sort_exprs=c_count@0 ASC NULLS LAST, count(*)@0 ASC NULLS LAST -07)------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(*)], ordering_mode=Sorted08)--------------ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] +07)------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(*)], ordering_mode=Sorted 08)--------------ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] 09)----------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] 10)------------------CoalesceBatchesExec: target_batch_size=8192 From 53ee3de7f6ad985fa0c2b263d47f0caa95d823a4 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Mon, 20 Jan 2025 17:27:25 +0300 Subject: [PATCH 06/62] re-add test_utils.rs changes to the new file --- .../src/physical_optimizer/enforce_sorting.rs | 4 +- datafusion/expr/src/test/function_stub.rs | 10 ++ .../physical-optimizer/src/test_utils.rs | 100 +++++++++++++++++- 3 files changed, 110 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 9b1b451b14bc..00caeb5948fb 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -663,7 +663,9 @@ mod tests { use crate::prelude::{SessionConfig, SessionContext}; use crate::test::{csv_exec_ordered, csv_exec_sorted, stream_exec_ordered}; use datafusion_physical_optimizer::test_utils::{ - aggregate_exec, bounded_window_exec, check_integrity, coalesce_batches_exec, + aggregate_exec, aggregate_exec_monotonic, aggregate_exec_non_monotonic, + bounded_window_exec, bounded_window_exec_non_monotonic, + bounded_window_exec_with_partition, check_integrity, coalesce_batches_exec, coalesce_partitions_exec, filter_exec, global_limit_exec, hash_join_exec, limit_exec, local_limit_exec, memory_exec, repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, diff --git a/datafusion/expr/src/test/function_stub.rs b/datafusion/expr/src/test/function_stub.rs index 71ab1ad6ef9b..f856672a040f 100644 --- a/datafusion/expr/src/test/function_stub.rs +++ b/datafusion/expr/src/test/function_stub.rs @@ -278,6 +278,10 @@ impl AggregateUDFImpl for Count { fn reverse_expr(&self) -> ReversedUDAF { ReversedUDAF::Identical } + + fn is_monotonic(&self) -> Option { + Some(true) + } } create_func!(Min, min_udaf); @@ -363,6 +367,9 @@ impl AggregateUDFImpl for Min { fn is_descending(&self) -> Option { Some(false) } + fn is_monotonic(&self) -> Option { + Some(false) + } } create_func!(Max, max_udaf); @@ -448,6 +455,9 @@ impl AggregateUDFImpl for Max { fn is_descending(&self) -> Option { Some(true) } + fn is_monotonic(&self) -> Option { + Some(true) + } } /// Testing stub implementation of avg aggregate diff --git a/datafusion/physical-optimizer/src/test_utils.rs b/datafusion/physical-optimizer/src/test_utils.rs index 9f0b5ddf6f40..54895580da18 100644 --- a/datafusion/physical-optimizer/src/test_utils.rs +++ b/datafusion/physical-optimizer/src/test_utils.rs @@ -25,7 +25,7 @@ use std::fmt::Formatter; use arrow_schema::{Schema, SchemaRef, SortOptions}; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{JoinType, Result}; -use datafusion_expr::test::function_stub::count_udaf; +use datafusion_expr::test::function_stub::{avg_udaf, count_udaf}; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalExpr; @@ -48,6 +48,7 @@ use datafusion_physical_plan::windows::{create_window_expr, BoundedWindowAggExec use datafusion_physical_plan::{InputOrderMode, Partitioning}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_plan::ExecutionPlan; @@ -124,6 +125,49 @@ pub fn bounded_window_exec( col_name: &str, sort_exprs: impl IntoIterator, input: Arc, +) -> Arc { + bounded_window_exec_with_partition(col_name, sort_exprs, &[], input, false) +} + +pub fn bounded_window_exec_with_partition( + col_name: &str, + sort_exprs: impl IntoIterator, + partition_by: &[Arc], + input: Arc, + should_reverse: bool, +) -> Arc { + let sort_exprs: LexOrdering = sort_exprs.into_iter().collect(); + let schema = input.schema(); + let mut window_expr = create_window_expr( + &WindowFunctionDefinition::AggregateUDF(count_udaf()), + "count".to_owned(), + &[col(col_name, &schema).unwrap()], + partition_by, + sort_exprs.as_ref(), + Arc::new(WindowFrame::new(Some(false))), + schema.as_ref(), + false, + ) + .unwrap(); + if should_reverse { + window_expr = window_expr.get_reverse_expr().unwrap(); + } + + Arc::new( + BoundedWindowAggExec::try_new( + vec![window_expr], + input.clone(), + vec![], + InputOrderMode::Sorted, + ) + .unwrap(), + ) +} + +pub fn bounded_window_exec_non_monotonic( + col_name: &str, + sort_exprs: impl IntoIterator, + input: Arc, ) -> Arc { let sort_exprs: LexOrdering = sort_exprs.into_iter().collect(); let schema = input.schema(); @@ -131,8 +175,8 @@ pub fn bounded_window_exec( Arc::new( BoundedWindowAggExec::try_new( vec![create_window_expr( - &WindowFunctionDefinition::AggregateUDF(count_udaf()), - "count".to_owned(), + &WindowFunctionDefinition::AggregateUDF(avg_udaf()), + "avg".to_owned(), &[col(col_name, &schema).unwrap()], &[], sort_exprs.as_ref(), @@ -207,6 +251,56 @@ pub fn aggregate_exec(input: Arc) -> Arc { ) } +pub fn aggregate_exec_monotonic(input: Arc) -> Arc { + let schema = input.schema(); + let aggregate_expr = + vec![ + AggregateExprBuilder::new(count_udaf(), vec![col("d", &schema).unwrap()]) + .schema(Arc::clone(&schema)) + .alias("count") + .build() + .map(Arc::new) + .unwrap(), + ]; + Arc::new( + AggregateExec::try_new( + AggregateMode::Final, + PhysicalGroupBy::default(), + aggregate_expr, + vec![None], + input, + schema, + ) + .unwrap(), + ) +} + +pub fn aggregate_exec_non_monotonic( + input: Arc, +) -> Arc { + let schema = input.schema(); + let aggregate_expr = + vec![ + AggregateExprBuilder::new(avg_udaf(), vec![col("d", &schema).unwrap()]) + .schema(Arc::clone(&schema)) + .alias("avg") + .build() + .map(Arc::new) + .unwrap(), + ]; + Arc::new( + AggregateExec::try_new( + AggregateMode::Final, + PhysicalGroupBy::default(), + aggregate_expr, + vec![None], + input, + schema, + ) + .unwrap(), + ) +} + pub fn coalesce_batches_exec(input: Arc) -> Arc { Arc::new(CoalesceBatchesExec::new(input, 128)) } From 97d89510c7f0501d6b4bb315b59c853136f81be8 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Mon, 20 Jan 2025 17:40:08 +0300 Subject: [PATCH 07/62] clone input with Arc --- datafusion/physical-optimizer/src/test_utils.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-optimizer/src/test_utils.rs b/datafusion/physical-optimizer/src/test_utils.rs index 54895580da18..8b14104bd317 100644 --- a/datafusion/physical-optimizer/src/test_utils.rs +++ b/datafusion/physical-optimizer/src/test_utils.rs @@ -156,7 +156,7 @@ pub fn bounded_window_exec_with_partition( Arc::new( BoundedWindowAggExec::try_new( vec![window_expr], - input.clone(), + Arc::clone(&input), vec![], InputOrderMode::Sorted, ) From e988dcffd9f4d02fecea59db59d9b26c27bd6510 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 23 Jan 2025 17:39:26 +0300 Subject: [PATCH 08/62] inject aggr expr indices separate stubs and count_udafs --- datafusion/physical-expr/src/aggregate.rs | 8 ++++--- .../physical-expr/src/window/aggregate.rs | 8 +++++-- .../src/window/sliding_aggregate.rs | 8 +++++-- .../physical-optimizer/src/test_utils.rs | 23 +++++++++++-------- .../physical-plan/src/aggregates/mod.rs | 11 +++++++-- .../src/windows/bounded_window_agg_exec.rs | 17 +++++++++++--- datafusion/physical-plan/src/windows/mod.rs | 12 ++++++---- .../src/windows/window_agg_exec.rs | 23 ++++++++++++++++--- 8 files changed, 82 insertions(+), 28 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 821272e91612..5bf691a8de9c 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -550,11 +550,13 @@ impl AggregateFunctionExpr { } /// Returns PhysicalSortExpr based on monotonicity of the function - pub fn natural_sort_expr(&self, schema: &SchemaRef) -> Option { + pub fn natural_sort_expr( + &self, + window_expr_index: usize, + ) -> Option { // If the aggregate expressions are monotonic, the output data is naturally ordered with it. let is_ascending = self.is_monotonic()?; - let idx = schema.index_of(self.name()).unwrap_or(0); - let expr = Arc::new(Column::new(self.name(), idx)); + let expr = Arc::new(Column::new(self.name(), window_expr_index)); let options = SortOptions::new(!is_ascending, false); Some(PhysicalSortExpr { expr, options }) diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index 3ce528adf3c0..638073cca317 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -68,10 +68,14 @@ impl PlainAggregateWindowExpr { &self.aggregate } - pub fn add_equal_orderings(&self, eq_properties: &mut EquivalenceProperties) { + pub fn add_equal_orderings( + &self, + eq_properties: &mut EquivalenceProperties, + window_expr_index: usize, + ) { let Some(expr) = self .get_aggregate_expr() - .natural_sort_expr(eq_properties.schema()) + .natural_sort_expr(window_expr_index) else { return; }; diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index 77376ad61374..889ec280cde2 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -70,10 +70,14 @@ impl SlidingAggregateWindowExpr { &self.aggregate } - pub fn add_equal_orderings(&self, eq_properties: &mut EquivalenceProperties) { + pub fn add_equal_orderings( + &self, + eq_properties: &mut EquivalenceProperties, + window_expr_index: usize, + ) { let Some(expr) = self .get_aggregate_expr() - .natural_sort_expr(eq_properties.schema()) + .natural_sort_expr(window_expr_index) else { return; }; diff --git a/datafusion/physical-optimizer/src/test_utils.rs b/datafusion/physical-optimizer/src/test_utils.rs index d99a3275517c..f9e17746cbd8 100644 --- a/datafusion/physical-optimizer/src/test_utils.rs +++ b/datafusion/physical-optimizer/src/test_utils.rs @@ -17,8 +17,13 @@ //! Test utilities for physical optimizer tests +use std::any::Any; +use std::fmt::Formatter; +use std::sync::Arc; + use crate::limited_distinct_aggregation::LimitedDistinctAggregation; use crate::PhysicalOptimizerRule; + use arrow::array::Int32Array; use arrow::record_batch::RecordBatch; use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; @@ -27,8 +32,11 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; use datafusion_common::{JoinType, Result}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_expr::test::function_stub::{avg_udaf, count_udaf}; +use datafusion_expr::test::function_stub::{ + avg_udaf as avg_stub, count_udaf as count_stub, +}; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; +use datafusion_functions_aggregate::count::count_udaf; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{expressions, PhysicalExpr}; @@ -56,9 +64,6 @@ use datafusion_physical_plan::{ displayable, DisplayAs, DisplayFormatType, PlanProperties, }; use datafusion_physical_plan::{InputOrderMode, Partitioning}; -use std::any::Any; -use std::fmt::Formatter; -use std::sync::Arc; pub fn schema() -> SchemaRef { Arc::new(Schema::new(vec![ @@ -188,7 +193,7 @@ pub fn bounded_window_exec_with_partition( let sort_exprs: LexOrdering = sort_exprs.into_iter().collect(); let schema = input.schema(); let mut window_expr = create_window_expr( - &WindowFunctionDefinition::AggregateUDF(count_udaf()), + &WindowFunctionDefinition::AggregateUDF(count_stub()), "count".to_owned(), &[col(col_name, &schema).unwrap()], partition_by, @@ -224,7 +229,7 @@ pub fn bounded_window_exec_non_monotonic( Arc::new( BoundedWindowAggExec::try_new( vec![create_window_expr( - &WindowFunctionDefinition::AggregateUDF(avg_udaf()), + &WindowFunctionDefinition::AggregateUDF(avg_stub()), "avg".to_owned(), &[col(col_name, &schema).unwrap()], &[], @@ -313,7 +318,7 @@ pub fn aggregate_exec_monotonic(input: Arc) -> Arc>, + aggr_expr_indices: Vec, ) -> PlanProperties { // Construct equivalence properties: let mut eq_properties = input @@ -694,8 +700,9 @@ impl AggregateExec { input.pipeline_behavior() }; - for aggr_expr in aggr_exprs { - if let Some(expr) = aggr_expr.natural_sort_expr(eq_properties.schema()) { + for (i, aggr_expr) in aggr_exprs.iter().enumerate() { + let aggr_expr_index = aggr_expr_indices[i]; + if let Some(expr) = aggr_expr.natural_sort_expr(aggr_expr_index) { eq_properties.add_new_ordering(LexOrdering::new(vec![expr])); } } diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 2ac86da92e50..03951a4de1e9 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -67,6 +67,7 @@ use futures::stream::Stream; use futures::{ready, StreamExt}; use hashbrown::hash_table::HashTable; use indexmap::IndexMap; +use itertools::Itertools; use log::debug; /// Window execution plan @@ -103,8 +104,11 @@ impl BoundedWindowAggExec { partition_keys: Vec>, input_order_mode: InputOrderMode, ) -> Result { + let old_fields_latest_index = input.schema().fields.len().saturating_sub(1); let schema = create_schema(&input.schema(), &window_expr)?; let schema = Arc::new(schema); + let window_expr_indices = + (old_fields_latest_index..schema.fields.len()).collect_vec(); let partition_by_exprs = window_expr[0].partition_by(); let ordered_partition_by_indices = match &input_order_mode { InputOrderMode::Sorted => { @@ -123,7 +127,8 @@ impl BoundedWindowAggExec { vec![] } }; - let cache = Self::compute_properties(&input, &schema, &window_expr); + let cache = + Self::compute_properties(&input, &schema, &window_expr, window_expr_indices); Ok(Self { input, window_expr, @@ -190,10 +195,16 @@ impl BoundedWindowAggExec { fn compute_properties( input: &Arc, schema: &SchemaRef, - window_expr: &[Arc], + window_exprs: &[Arc], + window_expr_indices: Vec, ) -> PlanProperties { // Calculate equivalence properties: - let eq_properties = window_equivalence_properties(schema, input, window_expr); + let eq_properties = window_equivalence_properties( + schema, + input, + window_exprs, + window_expr_indices, + ); // As we can have repartitioning using the partition keys, this can // be either one or more than one, depending on the presence of diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 4c9df003642f..dcd2953acdab 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -336,25 +336,29 @@ pub(crate) fn get_partition_by_sort_exprs( pub(crate) fn window_equivalence_properties( schema: &SchemaRef, input: &Arc, - window_expr: &[Arc], + window_exprs: &[Arc], + window_expr_indices: Vec, ) -> EquivalenceProperties { // We need to update the schema, so we can not directly use // `input.equivalence_properties()`. let mut window_eq_properties = EquivalenceProperties::new(Arc::clone(schema)) .extend(input.equivalence_properties().clone()); - for expr in window_expr { + for (i, expr) in window_exprs.iter().enumerate() { + let window_expr_index = window_expr_indices[i]; if let Some(udf_window_expr) = expr.as_any().downcast_ref::() { udf_window_expr.add_equal_orderings(&mut window_eq_properties); } else if let Some(aggregate_udf_window_expr) = expr.as_any().downcast_ref::() { - aggregate_udf_window_expr.add_equal_orderings(&mut window_eq_properties); + aggregate_udf_window_expr + .add_equal_orderings(&mut window_eq_properties, window_expr_index); } else if let Some(aggregate_udf_window_expr) = expr.as_any().downcast_ref::() { - aggregate_udf_window_expr.add_equal_orderings(&mut window_eq_properties); + aggregate_udf_window_expr + .add_equal_orderings(&mut window_eq_properties, window_expr_index); } } window_eq_properties diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index b132c3247072..3f6ed44d10f8 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -34,6 +34,7 @@ use crate::{ ExecutionPlanProperties, PhysicalExpr, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, WindowExpr, }; + use arrow::array::ArrayRef; use arrow::compute::{concat, concat_batches}; use arrow::datatypes::SchemaRef; @@ -44,7 +45,9 @@ use datafusion_common::utils::{evaluate_partition_ranges, transpose}; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; + use futures::{ready, Stream, StreamExt}; +use itertools::Itertools; /// Window execution plan #[derive(Debug, Clone)] @@ -73,12 +76,20 @@ impl WindowAggExec { input: Arc, partition_keys: Vec>, ) -> Result { + let old_fields_latest_index = input.schema().fields.len().saturating_sub(1); let schema = create_schema(&input.schema(), &window_expr)?; let schema = Arc::new(schema); + let window_expr_indices = + (old_fields_latest_index..schema.fields.len()).collect_vec(); let ordered_partition_by_indices = get_ordered_partition_by_indices(window_expr[0].partition_by(), &input); - let cache = Self::compute_properties(Arc::clone(&schema), &input, &window_expr); + let cache = Self::compute_properties( + Arc::clone(&schema), + &input, + &window_expr, + window_expr_indices, + ); Ok(Self { input, window_expr, @@ -118,10 +129,16 @@ impl WindowAggExec { fn compute_properties( schema: SchemaRef, input: &Arc, - window_expr: &[Arc], + window_exprs: &[Arc], + window_expr_indices: Vec, ) -> PlanProperties { // Calculate equivalence properties: - let eq_properties = window_equivalence_properties(&schema, input, window_expr); + let eq_properties = window_equivalence_properties( + &schema, + input, + window_exprs, + window_expr_indices, + ); // Get output partitioning: // Because we can have repartitioning using the partition keys this From 906245e4e2f8ec38692b6aa3968e2b1e8a6cb56d Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 23 Jan 2025 17:42:56 +0300 Subject: [PATCH 09/62] remove redundant file --- datafusion/core/src/physical_optimizer/enforce_sorting.rs | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 datafusion/core/src/physical_optimizer/enforce_sorting.rs diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs deleted file mode 100644 index e69de29bb2d1..000000000000 From 475fe2da5b44f9c120b1142bfa731afac8daca13 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 24 Jan 2025 11:39:59 +0300 Subject: [PATCH 10/62] add Sum monotonicity change monotonicity to return an Enum rather than Option fix indices re-add monotonicity tests --- .../physical_optimizer/enforce_sorting.rs | 259 +++++++++++++++++- datafusion/expr/src/lib.rs | 3 +- datafusion/expr/src/test/function_stub.rs | 26 +- datafusion/expr/src/udaf.rs | 24 +- datafusion/functions-aggregate/src/count.rs | 9 +- datafusion/functions-aggregate/src/min_max.rs | 12 +- datafusion/functions-aggregate/src/sum.rs | 15 +- datafusion/physical-expr/src/aggregate.rs | 15 +- .../physical-optimizer/src/test_utils.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 32 ++- .../physical-plan/src/aggregates/row_hash.rs | 2 +- .../src/windows/bounded_window_agg_exec.rs | 5 +- .../src/windows/window_agg_exec.rs | 5 +- 13 files changed, 360 insertions(+), 49 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 4fe04753fd9c..c5da5944751e 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -37,11 +37,12 @@ use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeE use datafusion_physical_plan::{get_plan_string, ExecutionPlan}; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{TreeNode, TransformedResult}; -use datafusion_physical_optimizer::test_utils::{check_integrity,bounded_window_exec, coalesce_partitions_exec, create_test_schema, create_test_schema2, create_test_schema3, filter_exec, global_limit_exec, hash_join_exec, limit_exec, local_limit_exec, memory_exec, repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, spr_repartition_exec, stream_exec_ordered, union_exec}; +use datafusion_physical_optimizer::test_utils::{check_integrity, bounded_window_exec, coalesce_partitions_exec, create_test_schema, create_test_schema2, create_test_schema3, filter_exec, global_limit_exec, hash_join_exec, limit_exec, local_limit_exec, memory_exec, repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, spr_repartition_exec, stream_exec_ordered, union_exec, create_test_schema4, aggregate_exec_monotonic, aggregate_exec_non_monotonic, bounded_window_exec_with_partition, bounded_window_exec_non_monotonic,}; use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::file_format::file_compression_type::FileCompressionType; +use datafusion_physical_plan::sorts::sort::SortExec; use rstest::rstest; @@ -229,6 +230,262 @@ async fn test_remove_unnecessary_sort5() -> Result<()> { Ok(()) } +#[tokio::test] +async fn test_aggregate_monotonic() -> Result<()> { + let schema = create_test_schema4()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("a", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + + let aggregate = aggregate_exec_monotonic(sort); + let sort_exprs = LexOrdering::new(vec![sort_expr("count", &aggregate.schema())]); + let physical_plan: Arc = + Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _; + + let expected_input = [ + "SortExec: expr=[count@0 ASC], preserve_partitioning=[false]", + " AggregateExec: mode=Single, gby=[], aggr=[count]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + + let expected_optimized = [ + "AggregateExec: mode=Single, gby=[], aggr=[count]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_aggregate_non_monotonic() -> Result<()> { + let schema = create_test_schema4()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("a", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + + let aggregate = aggregate_exec_non_monotonic(sort); + let sort_exprs = LexOrdering::new(vec![sort_expr("avg", &aggregate.schema())]); + let physical_plan: Arc = + Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _; + + let expected_input = [ + "SortExec: expr=[avg@0 ASC], preserve_partitioning=[false]", + " AggregateExec: mode=Single, gby=[], aggr=[avg]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + + let expected_optimized = [ + "SortExec: expr=[avg@0 ASC], preserve_partitioning=[false]", + " AggregateExec: mode=Single, gby=[], aggr=[avg]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_bounded_window_monotonic_sort() -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let source = parquet_exec_sorted(&schema, sort_exprs.clone()); + let sort = sort_exec(sort_exprs.clone(), source); + + let bounded_window = bounded_window_exec("nullable_col", sort_exprs.clone(), sort); + let output_schema = bounded_window.schema(); + let sort_exprs2 = vec![sort_expr_options( + "count", + &output_schema, + SortOptions { + descending: false, + nulls_first: false, + }, + )]; + let physical_plan = sort_exec(sort_exprs2.clone(), bounded_window); + + let expected_input = [ + "SortExec: expr=[count@2 ASC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + ]; + let expected_optimized = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_bounded_plain_window_monotonic_sort_with_partitions() -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let source = parquet_exec_sorted(&schema, sort_exprs.clone()); + let sort = sort_exec(sort_exprs.clone(), source); + let partition_bys = &[col("nullable_col", &schema)?]; + + let bounded_window = bounded_window_exec_with_partition( + "nullable_col", + sort_exprs.clone(), + partition_bys, + sort, + false, + ); + let output_schema = bounded_window.schema(); + let sort_exprs2 = vec![ + sort_expr_options( + "nullable_col", + &output_schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + sort_expr_options( + "count", + &output_schema, + SortOptions { + descending: false, + nulls_first: false, + }, + ), + ]; + let physical_plan = sort_exec(sort_exprs2.clone(), bounded_window); + + let expected_input = [ + "SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + ]; + let expected_optimized = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_bounded_sliding_window_monotonic_sort_with_partitions() -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let source = parquet_exec_sorted(&schema, sort_exprs.clone()); + let sort = sort_exec(sort_exprs.clone(), source); + let partition_bys = &[col("nullable_col", &schema)?]; + + let bounded_window = bounded_window_exec_with_partition( + "nullable_col", + sort_exprs.clone(), + partition_bys, + sort, + true, + ); + let output_schema = bounded_window.schema(); + let sort_exprs2 = vec![ + sort_expr_options( + "nullable_col", + &output_schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + sort_expr_options( + "count", + &output_schema, + SortOptions { + descending: false, + nulls_first: false, + }, + ), + ]; + let physical_plan = sort_exec(sort_exprs2.clone(), bounded_window); + + let expected_input = [ + "SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + ]; + let expected_optimized = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }], mode=[Sorted]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_bounded_window_non_monotonic_sort() -> Result<()> { + let schema = create_test_schema4()?; + let sort_exprs = vec![sort_expr_options( + "a", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let source = parquet_exec_sorted(&schema, sort_exprs.clone()); + let sort = sort_exec(sort_exprs.clone(), source); + + let bounded_window = bounded_window_exec_non_monotonic("a", sort_exprs.clone(), sort); + let output_schema = bounded_window.schema(); + let sort_exprs2 = vec![sort_expr_options( + "avg", + &output_schema, + SortOptions { + descending: false, + nulls_first: false, + }, + )]; + let physical_plan = sort_exec(sort_exprs2.clone(), bounded_window); + + let expected_input = [ + "SortExec: expr=[avg@5 ASC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[avg: Ok(Field { name: \"avg\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST]", + ]; + let expected_optimized = [ + "SortExec: expr=[avg@5 ASC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[avg: Ok(Field { name: \"avg\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + #[tokio::test] async fn test_do_not_remove_sort_with_limit() -> Result<()> { let schema = create_test_schema()?; diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index 017415da8f23..bd190dc91cb1 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -91,7 +91,8 @@ pub use partition_evaluator::PartitionEvaluator; pub use sqlparser; pub use table_source::{TableProviderFilterPushDown, TableSource, TableType}; pub use udaf::{ - aggregate_doc_sections, AggregateUDF, AggregateUDFImpl, ReversedUDAF, StatisticsArgs, + aggregate_doc_sections, AggregateExprMonotonicity, AggregateUDF, AggregateUDFImpl, + ReversedUDAF, StatisticsArgs, }; pub use udf::{ scalar_doc_sections, ReturnInfo, ReturnTypeArgs, ScalarFunctionArgs, ScalarUDF, diff --git a/datafusion/expr/src/test/function_stub.rs b/datafusion/expr/src/test/function_stub.rs index f856672a040f..84e7da2697b5 100644 --- a/datafusion/expr/src/test/function_stub.rs +++ b/datafusion/expr/src/test/function_stub.rs @@ -33,7 +33,8 @@ use crate::{ expr::AggregateFunction, function::{AccumulatorArgs, StateFieldsArgs}, utils::AggregateOrderSensitivity, - Accumulator, AggregateUDFImpl, Expr, GroupsAccumulator, ReversedUDAF, Signature, + Accumulator, AggregateExprMonotonicity, AggregateUDFImpl, Expr, GroupsAccumulator, + ReversedUDAF, Signature, }; macro_rules! create_func { @@ -203,6 +204,17 @@ impl AggregateUDFImpl for Sum { fn order_sensitivity(&self) -> AggregateOrderSensitivity { AggregateOrderSensitivity::Insensitive } + + fn monotonicity(&self, data_type: &DataType) -> AggregateExprMonotonicity { + // Sum is only monotonic if its input is unsigned + match data_type { + DataType::UInt8 => AggregateExprMonotonicity::MonotonicallyAscending, + DataType::UInt16 => AggregateExprMonotonicity::MonotonicallyAscending, + DataType::UInt32 => AggregateExprMonotonicity::MonotonicallyAscending, + DataType::UInt64 => AggregateExprMonotonicity::MonotonicallyAscending, + _ => AggregateExprMonotonicity::NotMonotonic, + } + } } /// Testing stub implementation of COUNT aggregate @@ -279,8 +291,8 @@ impl AggregateUDFImpl for Count { ReversedUDAF::Identical } - fn is_monotonic(&self) -> Option { - Some(true) + fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity { + AggregateExprMonotonicity::MonotonicallyAscending } } @@ -367,8 +379,8 @@ impl AggregateUDFImpl for Min { fn is_descending(&self) -> Option { Some(false) } - fn is_monotonic(&self) -> Option { - Some(false) + fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity { + AggregateExprMonotonicity::MonotonicallyDescending } } @@ -455,8 +467,8 @@ impl AggregateUDFImpl for Max { fn is_descending(&self) -> Option { Some(true) } - fn is_monotonic(&self) -> Option { - Some(true) + fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity { + AggregateExprMonotonicity::MonotonicallyAscending } } diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index ce9025f98bfd..8aeeaf1ea5a7 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -39,6 +39,26 @@ use crate::utils::AggregateOrderSensitivity; use crate::{Accumulator, Expr}; use crate::{Documentation, Signature}; +/// Status of an Aggregate Expression's Monotonicity +#[derive(Debug, Clone)] +pub enum AggregateExprMonotonicity { + /// Ordering exists as ascending + MonotonicallyAscending, + /// Ordering exists as descending + MonotonicallyDescending, + /// No ordering + NotMonotonic, +} + +impl AggregateExprMonotonicity { + pub fn is_descending(&self) -> bool { + matches!(self, Self::MonotonicallyDescending) + } + pub fn is_monotonic(&self) -> bool { + !matches!(self, Self::NotMonotonic) + } +} + /// Logical representation of a user-defined [aggregate function] (UDAF). /// /// An aggregate function combines the values from multiple input rows @@ -644,8 +664,8 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// Returns None if the function is not monotonic. /// If the function is monotonically decreasing returns Some(false) e.g. Min /// If the function is monotonically increasing returns Some(true) e.g. Max - fn is_monotonic(&self) -> Option { - None + fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity { + AggregateExprMonotonicity::NotMonotonic } } diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index ea762d63aa25..5322e29082ab 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -48,8 +48,9 @@ use datafusion_common::{ }; use datafusion_expr::function::StateFieldsArgs; use datafusion_expr::{ - function::AccumulatorArgs, utils::format_state_name, Accumulator, AggregateUDFImpl, - Documentation, EmitTo, GroupsAccumulator, Signature, Volatility, + function::AccumulatorArgs, utils::format_state_name, Accumulator, + AggregateExprMonotonicity, AggregateUDFImpl, Documentation, EmitTo, + GroupsAccumulator, Signature, Volatility, }; use datafusion_expr::{Expr, ReversedUDAF, StatisticsArgs, TypeSignature}; use datafusion_functions_aggregate_common::aggregate::count_distinct::{ @@ -352,8 +353,8 @@ impl AggregateUDFImpl for Count { self.doc() } - fn is_monotonic(&self) -> Option { - Some(true) + fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity { + AggregateExprMonotonicity::MonotonicallyAscending } } diff --git a/datafusion/functions-aggregate/src/min_max.rs b/datafusion/functions-aggregate/src/min_max.rs index 991d115dfb87..f30cd0acc338 100644 --- a/datafusion/functions-aggregate/src/min_max.rs +++ b/datafusion/functions-aggregate/src/min_max.rs @@ -56,8 +56,8 @@ use arrow::datatypes::{ use crate::min_max::min_max_bytes::MinMaxBytesAccumulator; use datafusion_common::ScalarValue; use datafusion_expr::{ - function::AccumulatorArgs, Accumulator, AggregateUDFImpl, Documentation, Signature, - Volatility, + function::AccumulatorArgs, Accumulator, AggregateExprMonotonicity, AggregateUDFImpl, + Documentation, Signature, Volatility, }; use datafusion_expr::{GroupsAccumulator, StatisticsArgs}; use datafusion_macros::user_doc; @@ -362,8 +362,8 @@ impl AggregateUDFImpl for Max { self.doc() } - fn is_monotonic(&self) -> Option { - Some(true) + fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity { + AggregateExprMonotonicity::MonotonicallyAscending } } @@ -1188,8 +1188,8 @@ impl AggregateUDFImpl for Min { self.doc() } - fn is_monotonic(&self) -> Option { - Some(false) + fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity { + AggregateExprMonotonicity::MonotonicallyDescending } } diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index 6c2854f6bc24..d14d80f3d998 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -38,8 +38,8 @@ use datafusion_expr::function::AccumulatorArgs; use datafusion_expr::function::StateFieldsArgs; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ - Accumulator, AggregateUDFImpl, Documentation, GroupsAccumulator, ReversedUDAF, - Signature, Volatility, + Accumulator, AggregateExprMonotonicity, AggregateUDFImpl, Documentation, + GroupsAccumulator, ReversedUDAF, Signature, Volatility, }; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; use datafusion_functions_aggregate_common::utils::Hashable; @@ -254,6 +254,17 @@ impl AggregateUDFImpl for Sum { fn documentation(&self) -> Option<&Documentation> { self.doc() } + + fn monotonicity(&self, data_type: &DataType) -> AggregateExprMonotonicity { + // Sum is only monotonic if its input is unsigned + match data_type { + DataType::UInt8 => AggregateExprMonotonicity::MonotonicallyAscending, + DataType::UInt16 => AggregateExprMonotonicity::MonotonicallyAscending, + DataType::UInt32 => AggregateExprMonotonicity::MonotonicallyAscending, + DataType::UInt64 => AggregateExprMonotonicity::MonotonicallyAscending, + _ => AggregateExprMonotonicity::NotMonotonic, + } + } } /// This accumulator computes SUM incrementally diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 5bf691a8de9c..02474ea85b00 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -44,8 +44,8 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow_schema::SortOptions; use datafusion_common::ScalarValue; use datafusion_common::{internal_err, not_impl_err, Result}; -use datafusion_expr::AggregateUDF; use datafusion_expr::ReversedUDAF; +use datafusion_expr::{AggregateExprMonotonicity, AggregateUDF}; use datafusion_expr_common::accumulator::Accumulator; use datafusion_expr_common::groups_accumulator::GroupsAccumulator; use datafusion_expr_common::type_coercion::aggregates::check_arg_count; @@ -545,8 +545,10 @@ impl AggregateFunctionExpr { /// Returns None if the function is not monotonic. /// If the function is monotonically decreasing returns Some(false) e.g. Min /// If the function is monotonically increasing returns Some(true) e.g. Max - pub fn is_monotonic(&self) -> Option { - self.fun.inner().is_monotonic() + pub fn monotonicity(&self) -> AggregateExprMonotonicity { + let field = self.field(); + let data_type = field.data_type(); + self.fun.inner().monotonicity(data_type) } /// Returns PhysicalSortExpr based on monotonicity of the function @@ -555,10 +557,13 @@ impl AggregateFunctionExpr { window_expr_index: usize, ) -> Option { // If the aggregate expressions are monotonic, the output data is naturally ordered with it. - let is_ascending = self.is_monotonic()?; + let monotonicity = self.monotonicity(); + if !monotonicity.is_monotonic() { + return None; + } let expr = Arc::new(Column::new(self.name(), window_expr_index)); - let options = SortOptions::new(!is_ascending, false); + let options = SortOptions::new(monotonicity.is_descending(), false); Some(PhysicalSortExpr { expr, options }) } } diff --git a/datafusion/physical-optimizer/src/test_utils.rs b/datafusion/physical-optimizer/src/test_utils.rs index f9e17746cbd8..560a5e598733 100644 --- a/datafusion/physical-optimizer/src/test_utils.rs +++ b/datafusion/physical-optimizer/src/test_utils.rs @@ -106,7 +106,7 @@ pub fn create_test_schema4() -> Result { let b = Field::new("b", DataType::UInt64, false); let c = Field::new("c", DataType::UInt64, true); let d = Field::new("d", DataType::UInt64, false); - let e = Field::new("e", DataType::UInt64, false); + let e = Field::new("e", DataType::Int64, false); let schema = Arc::new(Schema::new(vec![a, b, c, d, e])); Ok(schema) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 25a3575f3746..dc62208d0b0c 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -375,6 +375,7 @@ pub struct AggregateExec { /// Describes how the input is ordered relative to the group by columns input_order_mode: InputOrderMode, cache: PlanProperties, + aggr_expr_indices: Vec, } impl AggregateExec { @@ -399,6 +400,7 @@ impl AggregateExec { input: Arc::clone(&self.input), schema: Arc::clone(&self.schema), input_schema: Arc::clone(&self.input_schema), + aggr_expr_indices: self.aggr_expr_indices.clone(), } } @@ -415,7 +417,8 @@ impl AggregateExec { input: Arc, input_schema: SchemaRef, ) -> Result { - let schema = create_schema(&input.schema(), &group_by, &aggr_expr, mode)?; + let (schema, aggr_exprs_indices) = + create_schema(&input.schema(), &group_by, &aggr_expr, mode)?; let schema = Arc::new(schema); AggregateExec::try_new_with_schema( @@ -426,6 +429,7 @@ impl AggregateExec { input, input_schema, schema, + aggr_exprs_indices, ) } @@ -446,6 +450,7 @@ impl AggregateExec { input: Arc, input_schema: SchemaRef, schema: SchemaRef, + aggr_expr_indices: Vec, ) -> Result { // Make sure arguments are consistent in size if aggr_expr.len() != filter_expr.len() { @@ -505,10 +510,6 @@ impl AggregateExec { let required_input_ordering = (!new_requirement.is_empty()).then_some(new_requirement); - let input_fields_latest_index = input_schema.fields.len().saturating_sub(1); - let aggr_expr_indices = (input_fields_latest_index - ..(input_fields_latest_index + aggr_expr.len())) - .collect_vec(); let cache = Self::compute_properties( &input, @@ -517,7 +518,7 @@ impl AggregateExec { &mode, &input_order_mode, aggr_expr.clone(), - aggr_expr_indices, + aggr_expr_indices.clone(), ); Ok(AggregateExec { @@ -533,6 +534,7 @@ impl AggregateExec { limit: None, input_order_mode, cache, + aggr_expr_indices, }) } @@ -858,6 +860,7 @@ impl ExecutionPlan for AggregateExec { Arc::clone(&children[0]), Arc::clone(&self.input_schema), Arc::clone(&self.schema), + self.aggr_expr_indices.clone(), )?; me.limit = self.limit; @@ -934,7 +937,8 @@ fn create_schema( group_by: &PhysicalGroupBy, aggr_expr: &[Arc], mode: AggregateMode, -) -> Result { +) -> Result<(Schema, Vec)> { + let mut aggr_exprs_indices = vec![]; let mut fields = Vec::with_capacity(group_by.num_output_exprs() + aggr_expr.len()); fields.extend(group_by.output_fields(input_schema)?); @@ -942,7 +946,8 @@ fn create_schema( AggregateMode::Partial => { // in partial mode, the fields of the accumulator's state for expr in aggr_expr { - fields.extend(expr.state_fields()?.iter().cloned()) + fields.extend(expr.state_fields()?.iter().cloned()); + aggr_exprs_indices.push(fields.len() - 1); } } AggregateMode::Final @@ -951,14 +956,15 @@ fn create_schema( | AggregateMode::SinglePartitioned => { // in final mode, the field with the final result of the accumulator for expr in aggr_expr { - fields.push(expr.field()) + fields.push(expr.field()); + aggr_exprs_indices.push(fields.len() - 1); } } } - Ok(Schema::new_with_metadata( - fields, - input_schema.metadata().clone(), + Ok(( + Schema::new_with_metadata(fields, input_schema.metadata().clone()), + aggr_exprs_indices, )) } @@ -2794,7 +2800,7 @@ mod tests { vec![false, false], // (a,b) ], ); - let aggr_schema = create_schema( + let (aggr_schema, _) = create_schema( &input_schema, &grouping_set, &aggr_expr, diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index cc95ce51c15b..948729bf4312 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -506,7 +506,7 @@ impl GroupedHashAggregateStream { // Therefore, when we spill these intermediate states or pass them to another // aggregation operator, we must use a schema that includes both the group // columns **and** the partial-state columns. - let partial_agg_schema = create_schema( + let (partial_agg_schema, _) = create_schema( &agg.input().schema(), &agg_group_by, &aggregate_exprs, diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 03951a4de1e9..b6840bccf78e 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -104,11 +104,10 @@ impl BoundedWindowAggExec { partition_keys: Vec>, input_order_mode: InputOrderMode, ) -> Result { - let old_fields_latest_index = input.schema().fields.len().saturating_sub(1); + let old_fields_len = input.schema().fields.len(); let schema = create_schema(&input.schema(), &window_expr)?; let schema = Arc::new(schema); - let window_expr_indices = - (old_fields_latest_index..schema.fields.len()).collect_vec(); + let window_expr_indices = (old_fields_len..schema.fields.len()).collect_vec(); let partition_by_exprs = window_expr[0].partition_by(); let ordered_partition_by_indices = match &input_order_mode { InputOrderMode::Sorted => { diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 3f6ed44d10f8..16354cb69742 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -76,11 +76,10 @@ impl WindowAggExec { input: Arc, partition_keys: Vec>, ) -> Result { - let old_fields_latest_index = input.schema().fields.len().saturating_sub(1); + let old_fields_len = input.schema().fields.len(); let schema = create_schema(&input.schema(), &window_expr)?; let schema = Arc::new(schema); - let window_expr_indices = - (old_fields_latest_index..schema.fields.len()).collect_vec(); + let window_expr_indices = (old_fields_len..schema.fields.len()).collect_vec(); let ordered_partition_by_indices = get_ordered_partition_by_indices(window_expr[0].partition_by(), &input); From 57e000e112526e0ce01934fb6badd590edc099ca Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 24 Jan 2025 12:19:48 +0300 Subject: [PATCH 11/62] fix sql logic tests --- datafusion/sqllogictest/test_files/window.slt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 889810611cf1..5b3f29abc5c3 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3144,8 +3144,8 @@ logical_plan 03)----WindowAggr: windowExpr=[[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 04)------TableScan: aggregate_test_100 projection=[c9] physical_plan -01)SortExec: TopK(fetch=5), expr=[sum1@1 ASC NULLS LAST, c9@0 DESC], preserve_partitioning=[false] -02)--ProjectionExec: expr=[c9@0 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum1] +01)ProjectionExec: expr=[c9@0 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum1] +02)--GlobalLimitExec: skip=0, fetch=5 03)----BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true From ca57f4632ad104009f9c2a4692c86ec3e89caad1 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 24 Jan 2025 12:40:04 +0300 Subject: [PATCH 12/62] fix sql logic tests --- datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index 85ecb8f56bdf..4dc77cf94239 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -61,7 +61,7 @@ physical_plan 03)----ProjectionExec: expr=[c_count@0 as c_count, count(*)@1 as custdist] 04)------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(*)], ordering_mode=Sorted 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4, preserve_order=true, sort_exprs=c_count@0 ASC NULLS LAST, count(*)@0 ASC NULLS LAST +06)----------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4, preserve_order=true, sort_exprs=c_count@0 ASC NULLS LAST, count(*)@1 ASC NULLS LAST 07)------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(*)], ordering_mode=Sorted 08)--------------ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] 09)----------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] From 072e6efaf78094250f9bfc62af063d0285da7006 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 24 Jan 2025 16:04:55 +0300 Subject: [PATCH 13/62] update docs --- datafusion/expr/src/udaf.rs | 4 ---- datafusion/physical-expr/src/aggregate.rs | 4 ---- 2 files changed, 8 deletions(-) diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 8aeeaf1ea5a7..9c5d40f8412f 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -660,10 +660,6 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// function is monotonically increasing if its value increases as its argument grows /// (as a set). Formally, `f` is a monotonically increasing set function if `f(S) >= f(T)` /// whenever `S` is a superset of `T`. - /// - /// Returns None if the function is not monotonic. - /// If the function is monotonically decreasing returns Some(false) e.g. Min - /// If the function is monotonically increasing returns Some(true) e.g. Max fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity { AggregateExprMonotonicity::NotMonotonic } diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 02474ea85b00..06143b9dea71 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -541,10 +541,6 @@ impl AggregateFunctionExpr { /// function is monotonically increasing if its value increases as its argument grows /// (as a set). Formally, `f` is a monotonically increasing set function if `f(S) >= f(T)` /// whenever `S` is a superset of `T`. - /// - /// Returns None if the function is not monotonic. - /// If the function is monotonically decreasing returns Some(false) e.g. Min - /// If the function is monotonically increasing returns Some(true) e.g. Max pub fn monotonicity(&self) -> AggregateExprMonotonicity { let field = self.field(); let data_type = field.data_type(); From 491aabe4eaefb1bc2aabba57aee86de2730b0fd9 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 28 Jan 2025 16:20:50 +0300 Subject: [PATCH 14/62] review part 1 --- .../physical_optimizer/enforce_sorting.rs | 77 ++++++++++++++----- .../tests/physical_optimizer/test_utils.rs | 21 ++--- datafusion/expr/src/lib.rs | 2 +- datafusion/expr/src/test/function_stub.rs | 24 +----- datafusion/expr/src/udaf.rs | 44 +++++------ datafusion/functions-aggregate/src/count.rs | 6 +- datafusion/functions-aggregate/src/min_max.rs | 12 +-- datafusion/functions-aggregate/src/sum.rs | 15 ++-- datafusion/physical-expr/src/aggregate.rs | 18 ++--- .../src/equivalence/properties.rs | 24 ------ .../physical-expr/src/window/aggregate.rs | 3 +- .../src/window/sliding_aggregate.rs | 3 +- .../physical-expr/src/window/standard.rs | 29 ++++++- datafusion/sqllogictest/test_files/window.slt | 6 +- 14 files changed, 152 insertions(+), 132 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 88b3fe0da3a5..f8e2323150e9 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -18,12 +18,15 @@ use std::sync::Arc; use crate::physical_optimizer::test_utils::{ - aggregate_exec, bounded_window_exec, check_integrity, coalesce_batches_exec, + aggregate_exec, aggregate_exec_non_set_monotonic, aggregate_exec_set_monotonic, + bounded_window_exec, bounded_window_exec_non_set_monotonic, + bounded_window_exec_with_partition, check_integrity, coalesce_batches_exec, coalesce_partitions_exec, create_test_schema, create_test_schema2, - create_test_schema3, filter_exec, global_limit_exec, hash_join_exec, limit_exec, - local_limit_exec, memory_exec, parquet_exec, repartition_exec, sort_exec, sort_expr, - sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, - spr_repartition_exec, stream_exec_ordered, union_exec, RequirementsTestExec, + create_test_schema3, create_test_schema4, filter_exec, global_limit_exec, + hash_join_exec, limit_exec, local_limit_exec, memory_exec, parquet_exec, + repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_merge_join_exec, + sort_preserving_merge_exec, spr_repartition_exec, stream_exec_ordered, union_exec, + RequirementsTestExec, }; use datafusion_physical_plan::displayable; @@ -238,14 +241,14 @@ async fn test_remove_unnecessary_sort5() -> Result<()> { Ok(()) } -#[tokio::test] -async fn test_aggregate_monotonic() -> Result<()> { +#[test] +fn test_aggregate_set_monotonic() -> Result<()> { let schema = create_test_schema4()?; let source = memory_exec(&schema); let sort_exprs = vec![sort_expr("a", &schema)]; let sort = sort_exec(sort_exprs.clone(), source); - let aggregate = aggregate_exec_monotonic(sort); + let aggregate = aggregate_exec_set_monotonic(sort, vec![]); let sort_exprs = LexOrdering::new(vec![sort_expr("count", &aggregate.schema())]); let physical_plan: Arc = Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _; @@ -266,14 +269,44 @@ async fn test_aggregate_monotonic() -> Result<()> { Ok(()) } -#[tokio::test] -async fn test_aggregate_non_monotonic() -> Result<()> { +#[test] +fn test_aggregate_set_monotonic_with_groupby() -> Result<()> { let schema = create_test_schema4()?; let source = memory_exec(&schema); let sort_exprs = vec![sort_expr("a", &schema)]; let sort = sort_exec(sort_exprs.clone(), source); - let aggregate = aggregate_exec_non_monotonic(sort); + let aggregate = + aggregate_exec_set_monotonic(sort, vec![(col("a", &schema)?, "a".to_string())]); + let sort_exprs = LexOrdering::new(vec![sort_expr("count", &aggregate.schema())]); + let physical_plan: Arc = + Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _; + + let expected_input = [ + "SortExec: expr=[count@1 ASC], preserve_partitioning=[false]", + " AggregateExec: mode=Single, gby=[a@0 as a], aggr=[count], ordering_mode=Sorted", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + + let expected_optimized = [ + "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[count], ordering_mode=Sorted", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[test] +fn test_aggregate_non_set_monotonic() -> Result<()> { + let schema = create_test_schema4()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("a", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + + let aggregate = aggregate_exec_non_set_monotonic(sort); let sort_exprs = LexOrdering::new(vec![sort_expr("avg", &aggregate.schema())]); let physical_plan: Arc = Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _; @@ -296,7 +329,7 @@ async fn test_aggregate_non_monotonic() -> Result<()> { } #[tokio::test] -async fn test_bounded_window_monotonic_sort() -> Result<()> { +async fn test_bounded_window_set_monotonic_sort() -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr_options( "nullable_col", @@ -307,9 +340,11 @@ async fn test_bounded_window_monotonic_sort() -> Result<()> { }, )]; let source = parquet_exec_sorted(&schema, sort_exprs.clone()); + let sort = sort_exec(sort_exprs.clone(), source); let bounded_window = bounded_window_exec("nullable_col", sort_exprs.clone(), sort); + let output_schema = bounded_window.schema(); let sort_exprs2 = vec![sort_expr_options( "count", @@ -337,7 +372,7 @@ async fn test_bounded_window_monotonic_sort() -> Result<()> { } #[tokio::test] -async fn test_bounded_plain_window_monotonic_sort_with_partitions() -> Result<()> { +async fn test_bounded_plain_window_set_monotonic_sort_with_partitions() -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr_options( "nullable_col", @@ -348,9 +383,10 @@ async fn test_bounded_plain_window_monotonic_sort_with_partitions() -> Result<() }, )]; let source = parquet_exec_sorted(&schema, sort_exprs.clone()); + let sort = sort_exec(sort_exprs.clone(), source); - let partition_bys = &[col("nullable_col", &schema)?]; + let partition_bys = &[col("nullable_col", &schema)?]; let bounded_window = bounded_window_exec_with_partition( "nullable_col", sort_exprs.clone(), @@ -358,6 +394,7 @@ async fn test_bounded_plain_window_monotonic_sort_with_partitions() -> Result<() sort, false, ); + let output_schema = bounded_window.schema(); let sort_exprs2 = vec![ sort_expr_options( @@ -395,7 +432,8 @@ async fn test_bounded_plain_window_monotonic_sort_with_partitions() -> Result<() } #[tokio::test] -async fn test_bounded_sliding_window_monotonic_sort_with_partitions() -> Result<()> { +async fn test_bounded_plain_window_reverse_set_monotonic_sort_with_partitions( +) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr_options( "nullable_col", @@ -406,9 +444,10 @@ async fn test_bounded_sliding_window_monotonic_sort_with_partitions() -> Result< }, )]; let source = parquet_exec_sorted(&schema, sort_exprs.clone()); + let sort = sort_exec(sort_exprs.clone(), source); - let partition_bys = &[col("nullable_col", &schema)?]; + let partition_bys = &[col("nullable_col", &schema)?]; let bounded_window = bounded_window_exec_with_partition( "nullable_col", sort_exprs.clone(), @@ -416,6 +455,7 @@ async fn test_bounded_sliding_window_monotonic_sort_with_partitions() -> Result< sort, true, ); + let output_schema = bounded_window.schema(); let sort_exprs2 = vec![ sort_expr_options( @@ -453,7 +493,7 @@ async fn test_bounded_sliding_window_monotonic_sort_with_partitions() -> Result< } #[tokio::test] -async fn test_bounded_window_non_monotonic_sort() -> Result<()> { +async fn test_bounded_window_non_set_monotonic_sort() -> Result<()> { let schema = create_test_schema4()?; let sort_exprs = vec![sort_expr_options( "a", @@ -466,7 +506,8 @@ async fn test_bounded_window_non_monotonic_sort() -> Result<()> { let source = parquet_exec_sorted(&schema, sort_exprs.clone()); let sort = sort_exec(sort_exprs.clone(), source); - let bounded_window = bounded_window_exec_non_monotonic("a", sort_exprs.clone(), sort); + let bounded_window = + bounded_window_exec_non_set_monotonic("a", sort_exprs.clone(), sort); let output_schema = bounded_window.schema(); let sort_exprs2 = vec![sort_expr_options( "avg", diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 920b8da575c7..fe7d73da2bfd 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -32,10 +32,8 @@ use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; use datafusion_common::{JoinType, Result}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_expr::test::function_stub::{ - avg_udaf as avg_stub, count_udaf as count_stub, -}; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; +use datafusion_functions_aggregate::average::avg_udaf; use datafusion_functions_aggregate::count::count_udaf; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::col; @@ -216,7 +214,7 @@ pub fn bounded_window_exec_with_partition( let sort_exprs: LexOrdering = sort_exprs.into_iter().collect(); let schema = input.schema(); let mut window_expr = create_window_expr( - &WindowFunctionDefinition::AggregateUDF(count_stub()), + &WindowFunctionDefinition::AggregateUDF(count_udaf()), "count".to_owned(), &[col(col_name, &schema).unwrap()], partition_by, @@ -241,7 +239,7 @@ pub fn bounded_window_exec_with_partition( ) } -pub fn bounded_window_exec_non_monotonic( +pub fn bounded_window_exec_non_set_monotonic( col_name: &str, sort_exprs: impl IntoIterator, input: Arc, @@ -252,7 +250,7 @@ pub fn bounded_window_exec_non_monotonic( Arc::new( BoundedWindowAggExec::try_new( vec![create_window_expr( - &WindowFunctionDefinition::AggregateUDF(avg_stub()), + &WindowFunctionDefinition::AggregateUDF(avg_udaf()), "avg".to_owned(), &[col(col_name, &schema).unwrap()], &[], @@ -328,7 +326,10 @@ pub fn aggregate_exec(input: Arc) -> Arc { ) } -pub fn aggregate_exec_monotonic(input: Arc) -> Arc { +pub fn aggregate_exec_set_monotonic( + input: Arc, + group_by: Vec<(Arc, String)>, +) -> Arc { let schema = input.schema(); let aggregate_expr = vec![ @@ -342,7 +343,7 @@ pub fn aggregate_exec_monotonic(input: Arc) -> Arc) -> Arc, ) -> Arc { let schema = input.schema(); let aggregate_expr = vec![ - AggregateExprBuilder::new(avg_stub(), vec![col("d", &schema).unwrap()]) + AggregateExprBuilder::new(avg_udaf(), vec![col("d", &schema).unwrap()]) .schema(Arc::clone(&schema)) .alias("avg") .build() diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index bd190dc91cb1..138e6d0617b5 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -91,7 +91,7 @@ pub use partition_evaluator::PartitionEvaluator; pub use sqlparser; pub use table_source::{TableProviderFilterPushDown, TableSource, TableType}; pub use udaf::{ - aggregate_doc_sections, AggregateExprMonotonicity, AggregateUDF, AggregateUDFImpl, + aggregate_doc_sections, AggregateExprSetMonotonicity, AggregateUDF, AggregateUDFImpl, ReversedUDAF, StatisticsArgs, }; pub use udf::{ diff --git a/datafusion/expr/src/test/function_stub.rs b/datafusion/expr/src/test/function_stub.rs index 84e7da2697b5..71ab1ad6ef9b 100644 --- a/datafusion/expr/src/test/function_stub.rs +++ b/datafusion/expr/src/test/function_stub.rs @@ -33,8 +33,7 @@ use crate::{ expr::AggregateFunction, function::{AccumulatorArgs, StateFieldsArgs}, utils::AggregateOrderSensitivity, - Accumulator, AggregateExprMonotonicity, AggregateUDFImpl, Expr, GroupsAccumulator, - ReversedUDAF, Signature, + Accumulator, AggregateUDFImpl, Expr, GroupsAccumulator, ReversedUDAF, Signature, }; macro_rules! create_func { @@ -204,17 +203,6 @@ impl AggregateUDFImpl for Sum { fn order_sensitivity(&self) -> AggregateOrderSensitivity { AggregateOrderSensitivity::Insensitive } - - fn monotonicity(&self, data_type: &DataType) -> AggregateExprMonotonicity { - // Sum is only monotonic if its input is unsigned - match data_type { - DataType::UInt8 => AggregateExprMonotonicity::MonotonicallyAscending, - DataType::UInt16 => AggregateExprMonotonicity::MonotonicallyAscending, - DataType::UInt32 => AggregateExprMonotonicity::MonotonicallyAscending, - DataType::UInt64 => AggregateExprMonotonicity::MonotonicallyAscending, - _ => AggregateExprMonotonicity::NotMonotonic, - } - } } /// Testing stub implementation of COUNT aggregate @@ -290,10 +278,6 @@ impl AggregateUDFImpl for Count { fn reverse_expr(&self) -> ReversedUDAF { ReversedUDAF::Identical } - - fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity { - AggregateExprMonotonicity::MonotonicallyAscending - } } create_func!(Min, min_udaf); @@ -379,9 +363,6 @@ impl AggregateUDFImpl for Min { fn is_descending(&self) -> Option { Some(false) } - fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity { - AggregateExprMonotonicity::MonotonicallyDescending - } } create_func!(Max, max_udaf); @@ -467,9 +448,6 @@ impl AggregateUDFImpl for Max { fn is_descending(&self) -> Option { Some(true) } - fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity { - AggregateExprMonotonicity::MonotonicallyAscending - } } /// Testing stub implementation of avg aggregate diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 9c5d40f8412f..8e172d29c989 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -39,26 +39,6 @@ use crate::utils::AggregateOrderSensitivity; use crate::{Accumulator, Expr}; use crate::{Documentation, Signature}; -/// Status of an Aggregate Expression's Monotonicity -#[derive(Debug, Clone)] -pub enum AggregateExprMonotonicity { - /// Ordering exists as ascending - MonotonicallyAscending, - /// Ordering exists as descending - MonotonicallyDescending, - /// No ordering - NotMonotonic, -} - -impl AggregateExprMonotonicity { - pub fn is_descending(&self) -> bool { - matches!(self, Self::MonotonicallyDescending) - } - pub fn is_monotonic(&self) -> bool { - !matches!(self, Self::NotMonotonic) - } -} - /// Logical representation of a user-defined [aggregate function] (UDAF). /// /// An aggregate function combines the values from multiple input rows @@ -660,8 +640,8 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// function is monotonically increasing if its value increases as its argument grows /// (as a set). Formally, `f` is a monotonically increasing set function if `f(S) >= f(T)` /// whenever `S` is a superset of `T`. - fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity { - AggregateExprMonotonicity::NotMonotonic + fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { + AggregateExprSetMonotonicity::NotMonotonic } } @@ -846,6 +826,26 @@ pub mod aggregate_doc_sections { }; } +/// Status of an Aggregate Expression's Set Monotonicity +#[derive(Debug, Clone)] +pub enum AggregateExprSetMonotonicity { + /// Ordering exists as ascending + MonotonicallyAscending, + /// Ordering exists as descending + MonotonicallyDescending, + /// No ordering + NotMonotonic, +} + +impl AggregateExprSetMonotonicity { + pub fn is_descending(&self) -> bool { + matches!(self, Self::MonotonicallyDescending) + } + pub fn is_monotonic(&self) -> bool { + !matches!(self, Self::NotMonotonic) + } +} + #[cfg(test)] mod test { use crate::{AggregateUDF, AggregateUDFImpl}; diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 5322e29082ab..bdef99cb0aa7 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -49,7 +49,7 @@ use datafusion_common::{ use datafusion_expr::function::StateFieldsArgs; use datafusion_expr::{ function::AccumulatorArgs, utils::format_state_name, Accumulator, - AggregateExprMonotonicity, AggregateUDFImpl, Documentation, EmitTo, + AggregateExprSetMonotonicity, AggregateUDFImpl, Documentation, EmitTo, GroupsAccumulator, Signature, Volatility, }; use datafusion_expr::{Expr, ReversedUDAF, StatisticsArgs, TypeSignature}; @@ -353,8 +353,8 @@ impl AggregateUDFImpl for Count { self.doc() } - fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity { - AggregateExprMonotonicity::MonotonicallyAscending + fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { + AggregateExprSetMonotonicity::MonotonicallyAscending } } diff --git a/datafusion/functions-aggregate/src/min_max.rs b/datafusion/functions-aggregate/src/min_max.rs index f30cd0acc338..7e4978407949 100644 --- a/datafusion/functions-aggregate/src/min_max.rs +++ b/datafusion/functions-aggregate/src/min_max.rs @@ -56,8 +56,8 @@ use arrow::datatypes::{ use crate::min_max::min_max_bytes::MinMaxBytesAccumulator; use datafusion_common::ScalarValue; use datafusion_expr::{ - function::AccumulatorArgs, Accumulator, AggregateExprMonotonicity, AggregateUDFImpl, - Documentation, Signature, Volatility, + function::AccumulatorArgs, Accumulator, AggregateExprSetMonotonicity, + AggregateUDFImpl, Documentation, Signature, Volatility, }; use datafusion_expr::{GroupsAccumulator, StatisticsArgs}; use datafusion_macros::user_doc; @@ -362,8 +362,8 @@ impl AggregateUDFImpl for Max { self.doc() } - fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity { - AggregateExprMonotonicity::MonotonicallyAscending + fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { + AggregateExprSetMonotonicity::MonotonicallyAscending } } @@ -1188,8 +1188,8 @@ impl AggregateUDFImpl for Min { self.doc() } - fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity { - AggregateExprMonotonicity::MonotonicallyDescending + fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { + AggregateExprSetMonotonicity::MonotonicallyDescending } } diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index d14d80f3d998..a3a641784180 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -38,7 +38,7 @@ use datafusion_expr::function::AccumulatorArgs; use datafusion_expr::function::StateFieldsArgs; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ - Accumulator, AggregateExprMonotonicity, AggregateUDFImpl, Documentation, + Accumulator, AggregateExprSetMonotonicity, AggregateUDFImpl, Documentation, GroupsAccumulator, ReversedUDAF, Signature, Volatility, }; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; @@ -255,14 +255,15 @@ impl AggregateUDFImpl for Sum { self.doc() } - fn monotonicity(&self, data_type: &DataType) -> AggregateExprMonotonicity { + fn set_monotonicity(&self, data_type: &DataType) -> AggregateExprSetMonotonicity { // Sum is only monotonic if its input is unsigned + // TODO: Expand these utilizing statistics match data_type { - DataType::UInt8 => AggregateExprMonotonicity::MonotonicallyAscending, - DataType::UInt16 => AggregateExprMonotonicity::MonotonicallyAscending, - DataType::UInt32 => AggregateExprMonotonicity::MonotonicallyAscending, - DataType::UInt64 => AggregateExprMonotonicity::MonotonicallyAscending, - _ => AggregateExprMonotonicity::NotMonotonic, + DataType::UInt8 => AggregateExprSetMonotonicity::MonotonicallyAscending, + DataType::UInt16 => AggregateExprSetMonotonicity::MonotonicallyAscending, + DataType::UInt32 => AggregateExprSetMonotonicity::MonotonicallyAscending, + DataType::UInt64 => AggregateExprSetMonotonicity::MonotonicallyAscending, + _ => AggregateExprSetMonotonicity::NotMonotonic, } } } diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 06143b9dea71..2e82ab2ab263 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -45,7 +45,7 @@ use arrow_schema::SortOptions; use datafusion_common::ScalarValue; use datafusion_common::{internal_err, not_impl_err, Result}; use datafusion_expr::ReversedUDAF; -use datafusion_expr::{AggregateExprMonotonicity, AggregateUDF}; +use datafusion_expr::{AggregateExprSetMonotonicity, AggregateUDF}; use datafusion_expr_common::accumulator::Accumulator; use datafusion_expr_common::groups_accumulator::GroupsAccumulator; use datafusion_expr_common::type_coercion::aggregates::check_arg_count; @@ -541,24 +541,20 @@ impl AggregateFunctionExpr { /// function is monotonically increasing if its value increases as its argument grows /// (as a set). Formally, `f` is a monotonically increasing set function if `f(S) >= f(T)` /// whenever `S` is a superset of `T`. - pub fn monotonicity(&self) -> AggregateExprMonotonicity { + pub fn set_monotonicity(&self) -> AggregateExprSetMonotonicity { let field = self.field(); let data_type = field.data_type(); - self.fun.inner().monotonicity(data_type) + self.fun.inner().set_monotonicity(data_type) } /// Returns PhysicalSortExpr based on monotonicity of the function - pub fn natural_sort_expr( - &self, - window_expr_index: usize, - ) -> Option { - // If the aggregate expressions are monotonic, the output data is naturally ordered with it. - let monotonicity = self.monotonicity(); + pub fn natural_sort_expr(&self, aggr_func_idx: usize) -> Option { + // If the aggregate expressions are set-monotonic, the output data is naturally ordered with it. + let monotonicity = self.set_monotonicity(); if !monotonicity.is_monotonic() { return None; } - let expr = Arc::new(Column::new(self.name(), window_expr_index)); - + let expr = Arc::new(Column::new(self.name(), aggr_func_idx)); let options = SortOptions::new(monotonicity.is_descending(), false); Some(PhysicalSortExpr { expr, options }) } diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index b0fdc357d332..a6417044a061 100755 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -254,30 +254,6 @@ impl EquivalenceProperties { self.oeq_class.add_new_orderings(orderings); } - /// Adds new ordering expression into the existing ordering equivalence class based on partition by information. - pub fn add_new_ordering_expr_with_partition_by( - &mut self, - expr: PhysicalSortExpr, - partition_by: &Vec>, - ) { - if partition_by.is_empty() { - // In the absence of a PARTITION BY, ordering of `self.expr` is global: - self.add_new_orderings([LexOrdering::new(vec![expr])]); - } else { - // If we have a PARTITION BY, standard functions can not introduce - // a global ordering unless the existing ordering is compatible - // with PARTITION BY expressions. To elaborate, when PARTITION BY - // expressions and existing ordering expressions are equal (w.r.t. - // set equality), we can prefix the ordering of `self.expr` with - // the existing ordering. - let (mut ordering, _) = self.find_longest_permutation(partition_by); - if ordering.len() == partition_by.len() { - ordering.push(expr); - self.add_new_orderings([ordering]); - } - } - } - /// Adds a single ordering to the existing ordering equivalence class. pub fn add_new_ordering(&mut self, ordering: LexOrdering) { self.add_new_orderings([ordering]); diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index 638073cca317..f22c693126bc 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -22,6 +22,7 @@ use std::ops::Range; use std::sync::Arc; use crate::aggregate::AggregateFunctionExpr; +use crate::window::standard::add_new_ordering_expr_with_partition_by; use crate::window::window_expr::AggregateWindowExpr; use crate::window::{ PartitionBatches, PartitionWindowAggStates, SlidingAggregateWindowExpr, WindowExpr, @@ -79,7 +80,7 @@ impl PlainAggregateWindowExpr { else { return; }; - eq_properties.add_new_ordering_expr_with_partition_by(expr, &self.partition_by); + add_new_ordering_expr_with_partition_by(eq_properties, expr, &self.partition_by); } } diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index 889ec280cde2..4752b264d752 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -22,6 +22,7 @@ use std::ops::Range; use std::sync::Arc; use crate::aggregate::AggregateFunctionExpr; +use crate::window::standard::add_new_ordering_expr_with_partition_by; use crate::window::window_expr::AggregateWindowExpr; use crate::window::{ PartitionBatches, PartitionWindowAggStates, PlainAggregateWindowExpr, WindowExpr, @@ -81,7 +82,7 @@ impl SlidingAggregateWindowExpr { else { return; }; - eq_properties.add_new_ordering_expr_with_partition_by(expr, &self.partition_by); + add_new_ordering_expr_with_partition_by(eq_properties, expr, &self.partition_by); } } diff --git a/datafusion/physical-expr/src/window/standard.rs b/datafusion/physical-expr/src/window/standard.rs index 158dff507871..81038ae86e48 100644 --- a/datafusion/physical-expr/src/window/standard.rs +++ b/datafusion/physical-expr/src/window/standard.rs @@ -33,7 +33,7 @@ use datafusion_common::utils::evaluate_partition_ranges; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::window_state::{WindowAggState, WindowFrameContext}; use datafusion_expr::WindowFrame; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; /// A window expr that takes the form of a [`StandardWindowFunctionExpr`]. #[derive(Debug)] @@ -74,7 +74,8 @@ impl StandardWindowExpr { pub fn add_equal_orderings(&self, eq_properties: &mut EquivalenceProperties) { let schema = eq_properties.schema(); if let Some(fn_res_ordering) = self.expr.get_result_ordering(schema) { - eq_properties.add_new_ordering_expr_with_partition_by( + add_new_ordering_expr_with_partition_by( + eq_properties, fn_res_ordering, &self.partition_by, ); @@ -269,3 +270,27 @@ impl WindowExpr for StandardWindowExpr { } } } + +/// Adds new ordering expression into the existing ordering equivalence class based on partition by information. +pub(crate) fn add_new_ordering_expr_with_partition_by( + eqp: &mut EquivalenceProperties, + expr: PhysicalSortExpr, + partition_by: &Vec>, +) { + if partition_by.is_empty() { + // In the absence of a PARTITION BY, ordering of `self.expr` is global: + eqp.add_new_orderings([LexOrdering::new(vec![expr])]); + } else { + // If we have a PARTITION BY, standard functions can not introduce + // a global ordering unless the existing ordering is compatible + // with PARTITION BY expressions. To elaborate, when PARTITION BY + // expressions and existing ordering expressions are equal (w.r.t. + // set equality), we can prefix the ordering of `self.expr` with + // the existing ordering. + let (mut ordering, _) = eqp.find_longest_permutation(partition_by); + if ordering.len() == partition_by.len() { + ordering.push(expr); + eqp.add_new_orderings([ordering]); + } + } +} diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index e362b5318cbe..1d6174e7c74e 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3127,9 +3127,9 @@ physical_plan 04)------BoundedWindowAggExec: wdw=[row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "row_number() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] -# this is a negative test for asserting that window functions (other than ROW_NUMBER) -# are not added to ordering equivalence -# physical plan should contain SortExec. +# Top level sort is pushed down through BoundedWindowAggExec as its SUM result does already satisfy the required +# global order. The existing sort is for the second-term lexicographical ordering requirement, which is being +# preserved also at lexicographical level during the BoundedWindowAggExec. query TT EXPLAIN SELECT c9, sum1 FROM (SELECT c9, SUM(c9) OVER(ORDER BY c9 DESC) as sum1 From 71996fba9bdc797ba142dc41d6e830ae60d8e1ac Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Burak=20=C5=9Een?= Date: Tue, 28 Jan 2025 17:38:13 +0300 Subject: [PATCH 15/62] make AnalysisContext aware of empty sets to represent certainly false bounds (#14279) * ready for review * fmt and lint * Apply suggestions from code review Co-authored-by: Mehmet Ozan Kabak * apply reviews * fix test * Update analysis.rs * Update analysis.rs --------- Co-authored-by: Mehmet Ozan Kabak Co-authored-by: berkaysynnada --- datafusion-examples/examples/expr_api.rs | 2 +- datafusion/physical-expr/src/analysis.rs | 185 ++++++++++++++++------- datafusion/physical-plan/src/filter.rs | 25 ++- 3 files changed, 150 insertions(+), 62 deletions(-) diff --git a/datafusion-examples/examples/expr_api.rs b/datafusion-examples/examples/expr_api.rs index 6bfde2ebbf52..2908edbb754d 100644 --- a/datafusion-examples/examples/expr_api.rs +++ b/datafusion-examples/examples/expr_api.rs @@ -270,7 +270,7 @@ fn range_analysis_demo() -> Result<()> { // In this case, we can see that, as expected, `analyze` has figured out // that in this case, `date` must be in the range `['2020-09-01', '2020-10-01']` let expected_range = Interval::try_new(september_1, october_1)?; - assert_eq!(analysis_result.boundaries[0].interval, expected_range); + assert_eq!(analysis_result.boundaries[0].interval, Some(expected_range)); Ok(()) } diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index b602a9cba4f4..ceec21c71171 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -81,8 +81,12 @@ impl AnalysisContext { #[derive(Clone, Debug, PartialEq)] pub struct ExprBoundaries { pub column: Column, - /// Minimum and maximum values this expression can have. - pub interval: Interval, + /// Minimum and maximum values this expression can have. A `None` value + /// indicates that evaluating the given column results in an empty set. + /// For example, if the column `a` has values in the range [10, 20], + /// and there is a filter asserting that `a > 50`, then the resulting interval + /// range of `a` will be `None`. + pub interval: Option, /// Maximum number of distinct values this expression can produce, if known. pub distinct_count: Precision, } @@ -118,7 +122,7 @@ impl ExprBoundaries { let column = Column::new(field.name(), col_index); Ok(ExprBoundaries { column, - interval, + interval: Some(interval), distinct_count: col_stats.distinct_count, }) } @@ -133,7 +137,7 @@ impl ExprBoundaries { .map(|(i, field)| { Ok(Self { column: Column::new(field.name(), i), - interval: Interval::make_unbounded(field.data_type())?, + interval: Some(Interval::make_unbounded(field.data_type())?), distinct_count: Precision::Absent, }) }) @@ -161,40 +165,71 @@ pub fn analyze( context: AnalysisContext, schema: &Schema, ) -> Result { - let target_boundaries = context.boundaries; - - let mut graph = ExprIntervalGraph::try_new(Arc::clone(expr), schema)?; - - let columns = collect_columns(expr) - .into_iter() - .map(|c| Arc::new(c) as _) - .collect::>(); - - let target_expr_and_indices = graph.gather_node_indices(columns.as_slice()); - - let mut target_indices_and_boundaries = target_expr_and_indices + let initial_boundaries = &context.boundaries; + if initial_boundaries .iter() - .filter_map(|(expr, i)| { - target_boundaries.iter().find_map(|bound| { - expr.as_any() - .downcast_ref::() - .filter(|expr_column| bound.column.eq(*expr_column)) - .map(|_| (*i, bound.interval.clone())) - }) - }) - .collect::>(); - - match graph - .update_ranges(&mut target_indices_and_boundaries, Interval::CERTAINLY_TRUE)? + .all(|bound| bound.interval.is_none()) { - PropagationResult::Success => { - shrink_boundaries(graph, target_boundaries, target_expr_and_indices) + if initial_boundaries + .iter() + .any(|bound| bound.distinct_count != Precision::Exact(0)) + { + return internal_err!( + "ExprBoundaries has a non-zero distinct count although it represents an empty table" + ); + } + if context.selectivity != Some(0.0) { + return internal_err!( + "AnalysisContext has a non-zero selectivity although it represents an empty table" + ); } - PropagationResult::Infeasible => { - Ok(AnalysisContext::new(target_boundaries).with_selectivity(0.0)) + Ok(context) + } else if initial_boundaries + .iter() + .any(|bound| bound.interval.is_none()) + { + internal_err!( + "AnalysisContext is an inconsistent state. Some columns represent empty table while others don't" + ) + } else { + let mut target_boundaries = context.boundaries; + let mut graph = ExprIntervalGraph::try_new(Arc::clone(expr), schema)?; + let columns = collect_columns(expr) + .into_iter() + .map(|c| Arc::new(c) as _) + .collect::>(); + + let mut target_indices_and_boundaries = vec![]; + let target_expr_and_indices = graph.gather_node_indices(columns.as_slice()); + + for (expr, index) in &target_expr_and_indices { + if let Some(column) = expr.as_any().downcast_ref::() { + if let Some(bound) = + target_boundaries.iter().find(|b| b.column == *column) + { + // Now, it's safe to unwrap + target_indices_and_boundaries + .push((*index, bound.interval.as_ref().unwrap().clone())); + } + } } - PropagationResult::CannotPropagate => { - Ok(AnalysisContext::new(target_boundaries).with_selectivity(1.0)) + + match graph + .update_ranges(&mut target_indices_and_boundaries, Interval::CERTAINLY_TRUE)? + { + PropagationResult::Success => { + shrink_boundaries(graph, target_boundaries, target_expr_and_indices) + } + PropagationResult::Infeasible => { + // If the propagation result is infeasible, set intervals to None + target_boundaries + .iter_mut() + .for_each(|bound| bound.interval = None); + Ok(AnalysisContext::new(target_boundaries).with_selectivity(0.0)) + } + PropagationResult::CannotPropagate => { + Ok(AnalysisContext::new(target_boundaries).with_selectivity(1.0)) + } } } } @@ -215,12 +250,12 @@ fn shrink_boundaries( .iter_mut() .find(|bound| bound.column.eq(column)) { - bound.interval = graph.get_interval(*i); + bound.interval = Some(graph.get_interval(*i)); }; } }); - let selectivity = calculate_selectivity(&target_boundaries, &initial_boundaries); + let selectivity = calculate_selectivity(&target_boundaries, &initial_boundaries)?; if !(0.0..=1.0).contains(&selectivity) { return internal_err!("Selectivity is out of limit: {}", selectivity); @@ -235,16 +270,31 @@ fn shrink_boundaries( fn calculate_selectivity( target_boundaries: &[ExprBoundaries], initial_boundaries: &[ExprBoundaries], -) -> f64 { +) -> Result { // Since the intervals are assumed uniform and the values // are not correlated, we need to multiply the selectivities // of multiple columns to get the overall selectivity. - initial_boundaries - .iter() - .zip(target_boundaries.iter()) - .fold(1.0, |acc, (initial, target)| { - acc * cardinality_ratio(&initial.interval, &target.interval) - }) + if target_boundaries.len() != initial_boundaries.len() { + return Err(internal_datafusion_err!( + "The number of columns in the initial and target boundaries should be the same" + )); + } + let mut acc: f64 = 1.0; + for (initial, target) in initial_boundaries.iter().zip(target_boundaries) { + match (initial.interval.as_ref(), target.interval.as_ref()) { + (Some(initial), Some(target)) => { + acc *= cardinality_ratio(initial, target); + } + (None, Some(_)) => { + return internal_err!( + "Initial boundary cannot be None while having a Some() target boundary" + ); + } + _ => return Ok(0.0), + } + } + + Ok(acc) } #[cfg(test)] @@ -313,16 +363,6 @@ mod tests { Some(16), Some(19), ), - // (a > 10 AND a < 20) AND (a > 20 AND a < 30) - ( - col("a") - .gt(lit(10)) - .and(col("a").lt(lit(20))) - .and(col("a").gt(lit(20))) - .and(col("a").lt(lit(30))), - None, - None, - ), ]; for (expr, lower, upper) in test_cases { let boundaries = ExprBoundaries::try_new_unbounded(&schema).unwrap(); @@ -335,7 +375,9 @@ mod tests { df_schema.as_ref(), ) .unwrap(); - let actual = &analysis_result.boundaries[0].interval; + let Some(actual) = &analysis_result.boundaries[0].interval else { + panic!("The analysis result should contain non-empty intervals for all columns"); + }; let expected = Interval::make(lower, upper).unwrap(); assert_eq!( &expected, actual, @@ -344,6 +386,41 @@ mod tests { } } + #[test] + fn test_analyze_empty_set_boundary_exprs() { + let schema = Arc::new(Schema::new(vec![make_field("a", DataType::Int32)])); + + let test_cases: Vec = vec![ + // a > 10 AND a < 10 + col("a").gt(lit(10)).and(col("a").lt(lit(10))), + // a > 5 AND (a < 20 OR a > 20) + // a > 10 AND a < 20 + // (a > 10 AND a < 20) AND (a > 20 AND a < 30) + col("a") + .gt(lit(10)) + .and(col("a").lt(lit(20))) + .and(col("a").gt(lit(20))) + .and(col("a").lt(lit(30))), + ]; + + for expr in test_cases { + let boundaries = ExprBoundaries::try_new_unbounded(&schema).unwrap(); + let df_schema = DFSchema::try_from(Arc::clone(&schema)).unwrap(); + let physical_expr = + create_physical_expr(&expr, &df_schema, &ExecutionProps::new()).unwrap(); + let analysis_result = analyze( + &physical_expr, + AnalysisContext::new(boundaries), + df_schema.as_ref(), + ) + .unwrap(); + + for boundary in analysis_result.boundaries { + assert!(boundary.interval.is_none()); + } + } + } + #[test] fn test_analyze_invalid_boundary_exprs() { let schema = Arc::new(Schema::new(vec![make_field("a", DataType::Int32)])); diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index ae4a15ba5249..ec860b3a9f61 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -41,7 +41,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::cast::as_boolean_array; use datafusion_common::stats::Precision; use datafusion_common::{ - internal_err, plan_err, project_schema, DataFusionError, Result, + internal_err, plan_err, project_schema, DataFusionError, Result, ScalarValue, }; use datafusion_execution::TaskContext; use datafusion_expr::Operator; @@ -457,6 +457,15 @@ fn collect_new_statistics( .. }, )| { + let Some(interval) = interval else { + // If the interval is `None`, we can say that there are no rows: + return ColumnStatistics { + null_count: Precision::Exact(0), + max_value: Precision::Exact(ScalarValue::Null), + min_value: Precision::Exact(ScalarValue::Null), + distinct_count: Precision::Exact(0), + }; + }; let (lower, upper) = interval.into_bounds(); let (min_value, max_value) = if lower.eq(&upper) { (Precision::Exact(lower), Precision::Exact(upper)) @@ -1078,14 +1087,16 @@ mod tests { statistics.column_statistics, vec![ ColumnStatistics { - min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), - max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), - ..Default::default() + min_value: Precision::Exact(ScalarValue::Null), + max_value: Precision::Exact(ScalarValue::Null), + distinct_count: Precision::Exact(0), + null_count: Precision::Exact(0), }, ColumnStatistics { - min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), - max_value: Precision::Inexact(ScalarValue::Int32(Some(3))), - ..Default::default() + min_value: Precision::Exact(ScalarValue::Null), + max_value: Precision::Exact(ScalarValue::Null), + distinct_count: Precision::Exact(0), + null_count: Precision::Exact(0), }, ] ); From 1920771d8019f9490180571b7db4902a39311cba Mon Sep 17 00:00:00 2001 From: phisn <31885661+phisn@users.noreply.github.com> Date: Tue, 28 Jan 2025 17:30:30 +0100 Subject: [PATCH 16/62] Add relation to alias expr in schema display (#14311) * Add relation to alias expr in schema display * Add edge case for alias in physical_name --- datafusion/expr/src/expr.rs | 37 +++++++++++++++++++++++++++++++++---- 1 file changed, 33 insertions(+), 4 deletions(-) diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index b8e495ee7ae9..7070761f6383 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -2281,6 +2281,11 @@ impl Display for SchemaDisplay<'_> { Ok(()) } // Expr is not shown since it is aliased + Expr::Alias(Alias { + name, + relation: Some(relation), + .. + }) => write!(f, "{relation}.{name}"), Expr::Alias(Alias { name, .. }) => write!(f, "{name}"), Expr::Between(Between { expr, @@ -2769,10 +2774,10 @@ fn fmt_function( /// The name of the column (field) that this `Expr` will produce in the physical plan. /// The difference from [Expr::schema_name] is that top-level columns are unqualified. pub fn physical_name(expr: &Expr) -> Result { - if let Expr::Column(col) = expr { - Ok(col.name.clone()) - } else { - Ok(expr.schema_name().to_string()) + match expr { + Expr::Column(col) => Ok(col.name.clone()), + Expr::Alias(alias) => Ok(alias.name.clone()), + _ => Ok(expr.schema_name().to_string()), } } @@ -3023,6 +3028,30 @@ mod test { ) } + #[test] + fn test_schema_display_alias_with_relation() { + assert_eq!( + format!( + "{}", + SchemaDisplay( + &lit(1).alias_qualified("table_name".into(), "column_name") + ) + ), + "table_name.column_name" + ); + } + + #[test] + fn test_schema_display_alias_without_relation() { + assert_eq!( + format!( + "{}", + SchemaDisplay(&lit(1).alias_qualified(None::<&str>, "column_name")) + ), + "column_name" + ); + } + fn wildcard_options( opt_ilike: Option, opt_exclude: Option, From 8d542ecc13ddbb2be20448e61734ebc37134dfcf Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 28 Jan 2025 11:32:15 -0500 Subject: [PATCH 17/62] Improve deprecation message for MemoryExec (#14322) * Improve deprecation message for `MemoryExec` * clippy --- datafusion/physical-plan/src/values.rs | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index a30b8981fdd8..b000f9335a3e 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -34,7 +34,16 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; /// Execution plan for values list based relation (produces constant rows) -#[deprecated(since = "45.0.0", note = "Use `MemoryExec::try_new_as_values` instead")] +/// +/// Note this structure is the same as [`MemoryExec`] and is deprecated. +/// Please see the following for alternatives +/// * [`MemoryExec::try_new`] +/// * [`MemoryExec::try_new_from_batches`] +/// +/// [`MemoryExec`]: crate::memory::MemoryExec +/// [`MemoryExec::try_new`]: crate::memory::MemoryExec::try_new +/// [`MemoryExec::try_new_from_batches`]: crate::memory::MemoryExec::try_new_from_batches +#[deprecated(since = "45.0.0", note = "Use `MemoryExec` instead")] #[derive(Debug, Clone)] pub struct ValuesExec { /// The schema @@ -48,6 +57,7 @@ pub struct ValuesExec { #[allow(deprecated)] impl ValuesExec { /// Create a new values exec from data as expr + #[deprecated(since = "45.0.0", note = "Use `MemoryExec::try_new` instead")] pub fn try_new( schema: SchemaRef, data: Vec>>, @@ -101,6 +111,10 @@ impl ValuesExec { /// /// Errors if any of the batches don't match the provided schema, or if no /// batches are provided. + #[deprecated( + since = "45.0.0", + note = "Use `MemoryExec::try_new_from_batches` instead" + )] pub fn try_new_from_batches( schema: SchemaRef, batches: Vec, From dc445a1509dff4ec78df3a93dd765652abaaa753 Mon Sep 17 00:00:00 2001 From: Daniel Hegberg Date: Tue, 28 Jan 2025 08:34:58 -0800 Subject: [PATCH 18/62] fix: LogicalPlan::get_parameter_types fails to return all placeholders (#14312) --- datafusion/expr/src/logical_plan/plan.rs | 25 +++++++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 24fb0609b0fe..7e9c0cb75ec8 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -1497,7 +1497,9 @@ impl LogicalPlan { (_, Some(dt)) => { param_types.insert(id.clone(), Some(dt.clone())); } - _ => {} + _ => { + param_types.insert(id.clone(), None); + } } } Ok(TreeNodeRecursion::Continue) @@ -4347,4 +4349,25 @@ digraph { plan.rewrite_with_subqueries(&mut rewriter).unwrap(); assert!(!rewriter.filter_found); } + + #[test] + fn test_with_unresolved_placeholders() { + let field_name = "id"; + let placeholder_value = "$1"; + let schema = Schema::new(vec![Field::new(field_name, DataType::Int32, false)]); + + let plan = table_scan(TableReference::none(), &schema, None) + .unwrap() + .filter(col(field_name).eq(placeholder(placeholder_value))) + .unwrap() + .build() + .unwrap(); + + // Check that the placeholder parameters have not received a DataType. + let params = plan.get_parameter_types().unwrap(); + assert_eq!(params.len(), 1); + + let parameter_type = params.clone().get(placeholder_value).unwrap().clone(); + assert_eq!(parameter_type, None); + } } From ecc5694840249e1c18e9132a9833d00819749a45 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Wed, 29 Jan 2025 00:43:12 +0800 Subject: [PATCH 19/62] fix: FULL OUTER JOIN and LIMIT produces wrong results (#14338) * fix: FULL OUTER JOIN and LIMIT produces wrong results * Fix minor slt testing * fix test --- datafusion/optimizer/src/push_down_limit.rs | 1 - datafusion/sqllogictest/test_files/joins.slt | 159 +++++++++++++++++-- 2 files changed, 147 insertions(+), 13 deletions(-) diff --git a/datafusion/optimizer/src/push_down_limit.rs b/datafusion/optimizer/src/push_down_limit.rs index 8a3aa4bb8459..4da112d5153a 100644 --- a/datafusion/optimizer/src/push_down_limit.rs +++ b/datafusion/optimizer/src/push_down_limit.rs @@ -255,7 +255,6 @@ fn push_down_join(mut join: Join, limit: usize) -> Transformed { match join.join_type { Left => (Some(limit), None), Right => (None, Some(limit)), - Full => (Some(limit), Some(limit)), _ => (None, None), } }; diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 496c6c609e45..ac02aeb6fea4 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -4240,10 +4240,8 @@ EXPLAIN SELECT * FROM t0 FULL JOIN t1 ON t0.c1 = t1.c1 LIMIT 2; logical_plan 01)Limit: skip=0, fetch=2 02)--Full Join: t0.c1 = t1.c1 -03)----Limit: skip=0, fetch=2 -04)------TableScan: t0 projection=[c1, c2], fetch=2 -05)----Limit: skip=0, fetch=2 -06)------TableScan: t1 projection=[c1, c2, c3], fetch=2 +03)----TableScan: t0 projection=[c1, c2] +04)----TableScan: t1 projection=[c1, c2, c3] physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(c1@0, c1@0)] @@ -4257,10 +4255,8 @@ EXPLAIN SELECT * FROM t0 FULL JOIN t1 ON t0.c2 >= t1.c2 LIMIT 2; logical_plan 01)Limit: skip=0, fetch=2 02)--Full Join: Filter: t0.c2 >= t1.c2 -03)----Limit: skip=0, fetch=2 -04)------TableScan: t0 projection=[c1, c2], fetch=2 -05)----Limit: skip=0, fetch=2 -06)------TableScan: t1 projection=[c1, c2, c3], fetch=2 +03)----TableScan: t0 projection=[c1, c2] +04)----TableScan: t1 projection=[c1, c2, c3] physical_plan 01)GlobalLimitExec: skip=0, fetch=2 02)--NestedLoopJoinExec: join_type=Full, filter=c2@0 >= c2@1 @@ -4274,16 +4270,155 @@ EXPLAIN SELECT * FROM t0 FULL JOIN t1 ON t0.c1 = t1.c1 AND t0.c2 >= t1.c2 LIMIT logical_plan 01)Limit: skip=0, fetch=2 02)--Full Join: t0.c1 = t1.c1 Filter: t0.c2 >= t1.c2 -03)----Limit: skip=0, fetch=2 -04)------TableScan: t0 projection=[c1, c2], fetch=2 -05)----Limit: skip=0, fetch=2 -06)------TableScan: t1 projection=[c1, c2, c3], fetch=2 +03)----TableScan: t0 projection=[c1, c2] +04)----TableScan: t1 projection=[c1, c2, c3] physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(c1@0, c1@0)], filter=c2@0 >= c2@1 03)----MemoryExec: partitions=1, partition_sizes=[1] 04)----MemoryExec: partitions=1, partition_sizes=[1] +## Add more test cases for join limit pushdown +statement ok +drop table t1 + +## Test limit pushdown through OUTER JOIN including left/right and full outer join cases +statement ok +set datafusion.execution.target_partitions = 1; + +### Limit pushdown through join + +# Note we use csv as MemoryExec does not support limit push down (so doesn't manifest +# bugs if limits are improperly pushed down) +query I +COPY (values (1), (2), (3), (4), (5)) TO 'test_files/scratch/limit/t1.csv' +STORED AS CSV +---- +5 + +# store t2 in different order so the top N rows are not the same as the top N rows of t1 +query I +COPY (values (5), (4), (3), (2), (1)) TO 'test_files/scratch/limit/t2.csv' +STORED AS CSV +---- +5 + +statement ok +create external table t1(a int) stored as CSV location 'test_files/scratch/limit/t1.csv'; + +statement ok +create external table t2(b int) stored as CSV location 'test_files/scratch/limit/t2.csv'; + +###### +## LEFT JOIN w/ LIMIT +###### +query II +select * from t1 LEFT JOIN t2 ON t1.a = t2.b LIMIT 2; +---- +2 2 +1 1 + +# the output of this query should be two rows from the previous query +# there should be no nulls +query II +select * from t1 LEFT JOIN t2 ON t1.a = t2.b LIMIT 2; +---- +2 2 +1 1 + +# can only push down to t1 (preserved side) +query TT +explain select * from t1 LEFT JOIN t2 ON t1.a = t2.b LIMIT 2; +---- +logical_plan +01)Limit: skip=0, fetch=2 +02)--Left Join: t1.a = t2.b +03)----Limit: skip=0, fetch=2 +04)------TableScan: t1 projection=[a], fetch=2 +05)----TableScan: t2 projection=[b] +physical_plan +01)CoalesceBatchesExec: target_batch_size=3, fetch=2 +02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, b@0)] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], limit=2, has_header=true +04)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], has_header=true + +###### +## RIGHT JOIN w/ LIMIT +###### + +query II +select * from t1 RIGHT JOIN t2 ON t1.a = t2.b LIMIT 2; +---- +5 5 +4 4 + +# the output of this query should be two rows from the previous query +# there should be no nulls +query II +select * from t1 RIGHT JOIN t2 ON t1.a = t2.b LIMIT 2; +---- +5 5 +4 4 + +# can only push down to t2 (preserved side) +query TT +explain select * from t1 RIGHT JOIN t2 ON t1.a = t2.b LIMIT 2; +---- +logical_plan +01)Limit: skip=0, fetch=2 +02)--Right Join: t1.a = t2.b +03)----TableScan: t1 projection=[a] +04)----Limit: skip=0, fetch=2 +05)------TableScan: t2 projection=[b], fetch=2 +physical_plan +01)CoalesceBatchesExec: target_batch_size=3, fetch=2 +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@0, b@0)] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], has_header=true +04)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], limit=2, has_header=true + +###### +## FULL JOIN w/ LIMIT +###### +query II rowsort +select * from t1 FULL JOIN t2 ON t1.a = t2.b; +---- +1 1 +2 2 +3 3 +4 4 +5 5 + +# the output of this query should be two rows from the previous query +# there should be no nulls +# Reproducer for https://github.com/apache/datafusion/issues/14335 +query II +select * from t1 FULL JOIN t2 ON t1.a = t2.b LIMIT 2; +---- +5 5 +4 4 + + +# can't push limit for full outer join +query TT +explain select * from t1 FULL JOIN t2 ON t1.a = t2.b LIMIT 2; +---- +logical_plan +01)Limit: skip=0, fetch=2 +02)--Full Join: t1.a = t2.b +03)----TableScan: t1 projection=[a] +04)----TableScan: t2 projection=[b] +physical_plan +01)CoalesceBatchesExec: target_batch_size=3, fetch=2 +02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(a@0, b@0)] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], has_header=true +04)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], has_header=true + +statement ok +drop table t1; + +statement ok +drop table t2; + # Test Utf8View as Join Key # Issue: https://github.com/apache/datafusion/issues/12468 statement ok From 7f0c71b129b78e38067f86592c7f5a176bd645d5 Mon Sep 17 00:00:00 2001 From: Qianqian <130200611+Sevenannn@users.noreply.github.com> Date: Tue, 28 Jan 2025 09:11:37 -0800 Subject: [PATCH 20/62] Customize window frame support for dialect (#14288) * Customize window frame support for dialect (#70) * Customize window frame support for dialect * fix: ignore frame only when frame implies no frame * Add comments. move the window frame determine logic to dialect method * Update datafusion/sql/src/unparser/dialect.rs * Update test case * fix --------- Co-authored-by: Phillip LeBlanc * fix clippy --------- Co-authored-by: Phillip LeBlanc --- datafusion/sql/src/unparser/dialect.rs | 38 +++++++++++++++++- datafusion/sql/src/unparser/expr.rs | 55 +++++++++++++++++++++++--- 2 files changed, 87 insertions(+), 6 deletions(-) diff --git a/datafusion/sql/src/unparser/dialect.rs b/datafusion/sql/src/unparser/dialect.rs index 5c318a96ef6c..830435fd013c 100644 --- a/datafusion/sql/src/unparser/dialect.rs +++ b/datafusion/sql/src/unparser/dialect.rs @@ -23,7 +23,9 @@ use datafusion_expr::Expr; use regex::Regex; use sqlparser::tokenizer::Span; use sqlparser::{ - ast::{self, BinaryOperator, Function, Ident, ObjectName, TimezoneInfo}, + ast::{ + self, BinaryOperator, Function, Ident, ObjectName, TimezoneInfo, WindowFrameBound, + }, keywords::ALL_KEYWORDS, }; @@ -153,6 +155,18 @@ pub trait Dialect: Send + Sync { Ok(None) } + /// Allows the dialect to choose to omit window frame in unparsing + /// based on function name and window frame bound + /// Returns false if specific function name / window frame bound indicates no window frame is needed in unparsing + fn window_func_support_window_frame( + &self, + _func_name: &str, + _start_bound: &WindowFrameBound, + _end_bound: &WindowFrameBound, + ) -> bool { + true + } + /// Extends the dialect's default rules for unparsing scalar functions. /// This is useful for supporting application-specific UDFs or custom engine extensions. fn with_custom_scalar_overrides( @@ -500,6 +514,7 @@ pub struct CustomDialect { supports_column_alias_in_table_alias: bool, requires_derived_table_alias: bool, division_operator: BinaryOperator, + window_func_support_window_frame: bool, full_qualified_col: bool, unnest_as_table_factor: bool, } @@ -527,6 +542,7 @@ impl Default for CustomDialect { supports_column_alias_in_table_alias: true, requires_derived_table_alias: false, division_operator: BinaryOperator::Divide, + window_func_support_window_frame: true, full_qualified_col: false, unnest_as_table_factor: false, } @@ -634,6 +650,15 @@ impl Dialect for CustomDialect { self.division_operator.clone() } + fn window_func_support_window_frame( + &self, + _func_name: &str, + _start_bound: &WindowFrameBound, + _end_bound: &WindowFrameBound, + ) -> bool { + self.window_func_support_window_frame + } + fn full_qualified_col(&self) -> bool { self.full_qualified_col } @@ -675,6 +700,7 @@ pub struct CustomDialectBuilder { supports_column_alias_in_table_alias: bool, requires_derived_table_alias: bool, division_operator: BinaryOperator, + window_func_support_window_frame: bool, full_qualified_col: bool, unnest_as_table_factor: bool, } @@ -708,6 +734,7 @@ impl CustomDialectBuilder { supports_column_alias_in_table_alias: true, requires_derived_table_alias: false, division_operator: BinaryOperator::Divide, + window_func_support_window_frame: true, full_qualified_col: false, unnest_as_table_factor: false, } @@ -733,6 +760,7 @@ impl CustomDialectBuilder { .supports_column_alias_in_table_alias, requires_derived_table_alias: self.requires_derived_table_alias, division_operator: self.division_operator, + window_func_support_window_frame: self.window_func_support_window_frame, full_qualified_col: self.full_qualified_col, unnest_as_table_factor: self.unnest_as_table_factor, } @@ -857,6 +885,14 @@ impl CustomDialectBuilder { self } + pub fn with_window_func_support_window_frame( + mut self, + window_func_support_window_frame: bool, + ) -> Self { + self.window_func_support_window_frame = window_func_support_window_frame; + self + } + /// Customize the dialect to allow full qualified column names pub fn with_full_qualified_col(mut self, full_qualified_col: bool) -> Self { self.full_qualified_col = full_qualified_col; diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 7a110fd0785c..96e1cec001cb 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -217,6 +217,21 @@ impl Unparser<'_> { let start_bound = self.convert_bound(&window_frame.start_bound)?; let end_bound = self.convert_bound(&window_frame.end_bound)?; + + let window_frame = if self.dialect.window_func_support_window_frame( + func_name, + &start_bound, + &end_bound, + ) { + Some(ast::WindowFrame { + units, + start_bound, + end_bound: Some(end_bound), + }) + } else { + None + }; + let over = Some(ast::WindowType::WindowSpec(ast::WindowSpec { window_name: None, partition_by: partition_by @@ -224,11 +239,7 @@ impl Unparser<'_> { .map(|e| self.expr_to_sql_inner(e)) .collect::>>()?, order_by, - window_frame: Some(ast::WindowFrame { - units, - start_bound, - end_bound: Option::from(end_bound), - }), + window_frame, })); Ok(ast::Expr::Function(Function { @@ -1632,6 +1643,7 @@ mod tests { use datafusion_functions_aggregate::expr_fn::sum; use datafusion_functions_nested::expr_fn::{array_element, make_array}; use datafusion_functions_nested::map::map; + use datafusion_functions_window::rank::rank_udwf; use datafusion_functions_window::row_number::row_number_udwf; use crate::unparser::dialect::{ @@ -2677,6 +2689,39 @@ mod tests { Ok(()) } + #[test] + fn test_window_func_support_window_frame() -> Result<()> { + let default_dialect: Arc = + Arc::new(CustomDialectBuilder::new().build()); + + let test_dialect: Arc = Arc::new( + CustomDialectBuilder::new() + .with_window_func_support_window_frame(false) + .build(), + ); + + for (dialect, expected) in [ + ( + default_dialect, + "rank() OVER (ORDER BY a ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)", + ), + (test_dialect, "rank() OVER (ORDER BY a ASC NULLS FIRST)"), + ] { + let unparser = Unparser::new(dialect.as_ref()); + let func = WindowFunctionDefinition::WindowUDF(rank_udwf()); + let mut window_func = WindowFunction::new(func, vec![]); + window_func.order_by = vec![Sort::new(col("a"), true, true)]; + let expr = Expr::WindowFunction(window_func); + let ast = unparser.expr_to_sql(&expr)?; + + let actual = ast.to_string(); + let expected = expected.to_string(); + + assert_eq!(actual, expected); + } + Ok(()) + } + #[test] fn test_utf8_view_to_sql() -> Result<()> { let dialect = CustomDialectBuilder::new() From d05173118128f8993138a4b7d21b4a76de380756 Mon Sep 17 00:00:00 2001 From: Ian Lai <108986288+Chen-Yuan-Lai@users.noreply.github.com> Date: Wed, 29 Jan 2025 01:21:57 +0800 Subject: [PATCH 21/62] refactor: switch BooleanBufferBuilder to NullBufferBuilder in a unit test of common_scalar (#14339) Co-authored-by: Cheng-Yuan-Lai --- datafusion/common/src/scalar/mod.rs | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 14243d95a666..9d3429b67796 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -3963,7 +3963,7 @@ mod tests { use arrow::error::ArrowError; use arrow::util::pretty::pretty_format_columns; use arrow_array::types::Float64Type; - use arrow_buffer::{Buffer, NullBuffer}; + use arrow_buffer::{Buffer, NullBufferBuilder}; use arrow_schema::Fields; use chrono::NaiveDate; use rand::Rng; @@ -6912,12 +6912,11 @@ mod tests { let array_b = Arc::new(Int32Array::from_iter_values([2])); let arrays: Vec = vec![array_a, array_b]; - let mut not_nulls = BooleanBufferBuilder::new(1); - not_nulls.append(true); - let not_nulls = not_nulls.finish(); - let not_nulls = Some(NullBuffer::new(not_nulls)); + let mut not_nulls = NullBufferBuilder::new(1); - let ar = StructArray::new(fields, arrays, not_nulls); + not_nulls.append_non_null(); + + let ar = StructArray::new(fields, arrays, not_nulls.finish()); let s = ScalarValue::Struct(Arc::new(ar)); assert_eq!(s.to_string(), "{a:1,b:2}"); From f8063e840ecd80c36a0d9d8f32f3d1e38e79de1f Mon Sep 17 00:00:00 2001 From: Nicholas Gates Date: Tue, 28 Jan 2025 20:19:44 +0000 Subject: [PATCH 22/62] Add `ColumnStatistics::Sum` (#14074) * Add sum statistic * Add sum statistic * Add sum statistic * Add sum statistic * Add sum statistic * Add sum statistic * Add tests and Cargo fmt * fix up --------- Co-authored-by: Andrew Lamb --- datafusion/common/src/stats.rs | 173 ++++++++++++++++-- datafusion/core/src/datasource/statistics.rs | 6 +- .../tests/custom_sources_cases/statistics.rs | 2 + datafusion/physical-plan/src/common.rs | 3 + datafusion/physical-plan/src/filter.rs | 2 + .../physical-plan/src/joins/cross_join.rs | 52 +++++- datafusion/physical-plan/src/joins/utils.rs | 1 + datafusion/physical-plan/src/memory.rs | 1 + datafusion/physical-plan/src/projection.rs | 7 + datafusion/physical-plan/src/union.rs | 10 + datafusion/physical-plan/src/values.rs | 1 + .../proto/datafusion_common.proto | 1 + datafusion/proto-common/src/from_proto/mod.rs | 5 + .../proto-common/src/generated/pbjson.rs | 18 ++ .../proto-common/src/generated/prost.rs | 2 + datafusion/proto-common/src/to_proto/mod.rs | 1 + .../src/generated/datafusion_proto_common.rs | 2 + 17 files changed, 269 insertions(+), 18 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index d2ce965c5c49..dd8848d24923 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -21,7 +21,7 @@ use std::fmt::{self, Debug, Display}; use crate::{Result, ScalarValue}; -use arrow_schema::{Schema, SchemaRef}; +use arrow_schema::{DataType, Schema, SchemaRef}; /// Represents a value with a degree of certainty. `Precision` is used to /// propagate information the precision of statistical values. @@ -170,24 +170,63 @@ impl Precision { pub fn add(&self, other: &Precision) -> Precision { match (self, other) { (Precision::Exact(a), Precision::Exact(b)) => { - if let Ok(result) = a.add(b) { - Precision::Exact(result) - } else { - Precision::Absent - } + a.add(b).map(Precision::Exact).unwrap_or(Precision::Absent) } (Precision::Inexact(a), Precision::Exact(b)) | (Precision::Exact(a), Precision::Inexact(b)) - | (Precision::Inexact(a), Precision::Inexact(b)) => { - if let Ok(result) = a.add(b) { - Precision::Inexact(result) - } else { - Precision::Absent - } + | (Precision::Inexact(a), Precision::Inexact(b)) => a + .add(b) + .map(Precision::Inexact) + .unwrap_or(Precision::Absent), + (_, _) => Precision::Absent, + } + } + + /// Calculates the difference of two (possibly inexact) [`ScalarValue`] values, + /// conservatively propagating exactness information. If one of the input + /// values is [`Precision::Absent`], the result is `Absent` too. + pub fn sub(&self, other: &Precision) -> Precision { + match (self, other) { + (Precision::Exact(a), Precision::Exact(b)) => { + a.sub(b).map(Precision::Exact).unwrap_or(Precision::Absent) } + (Precision::Inexact(a), Precision::Exact(b)) + | (Precision::Exact(a), Precision::Inexact(b)) + | (Precision::Inexact(a), Precision::Inexact(b)) => a + .sub(b) + .map(Precision::Inexact) + .unwrap_or(Precision::Absent), + (_, _) => Precision::Absent, + } + } + + /// Calculates the multiplication of two (possibly inexact) [`ScalarValue`] values, + /// conservatively propagating exactness information. If one of the input + /// values is [`Precision::Absent`], the result is `Absent` too. + pub fn multiply(&self, other: &Precision) -> Precision { + match (self, other) { + (Precision::Exact(a), Precision::Exact(b)) => a + .mul_checked(b) + .map(Precision::Exact) + .unwrap_or(Precision::Absent), + (Precision::Inexact(a), Precision::Exact(b)) + | (Precision::Exact(a), Precision::Inexact(b)) + | (Precision::Inexact(a), Precision::Inexact(b)) => a + .mul_checked(b) + .map(Precision::Inexact) + .unwrap_or(Precision::Absent), (_, _) => Precision::Absent, } } + + /// Casts the value to the given data type, propagating exactness information. + pub fn cast_to(&self, data_type: &DataType) -> Result> { + match self { + Precision::Exact(value) => value.cast_to(data_type).map(Precision::Exact), + Precision::Inexact(value) => value.cast_to(data_type).map(Precision::Inexact), + Precision::Absent => Ok(Precision::Absent), + } + } } impl Debug for Precision { @@ -210,6 +249,18 @@ impl Display for Precision { } } +impl From> for Precision { + fn from(value: Precision) -> Self { + match value { + Precision::Exact(v) => Precision::Exact(ScalarValue::UInt64(Some(v as u64))), + Precision::Inexact(v) => { + Precision::Inexact(ScalarValue::UInt64(Some(v as u64))) + } + Precision::Absent => Precision::Absent, + } + } +} + /// Statistics for a relation /// Fields are optional and can be inexact because the sources /// sometimes provide approximate estimates for performance reasons @@ -401,6 +452,11 @@ impl Display for Statistics { } else { s }; + let s = if cs.sum_value != Precision::Absent { + format!("{} Sum={}", s, cs.sum_value) + } else { + s + }; let s = if cs.null_count != Precision::Absent { format!("{} Null={}", s, cs.null_count) } else { @@ -436,6 +492,8 @@ pub struct ColumnStatistics { pub max_value: Precision, /// Minimum value of column pub min_value: Precision, + /// Sum value of a column + pub sum_value: Precision, /// Number of distinct values pub distinct_count: Precision, } @@ -458,6 +516,7 @@ impl ColumnStatistics { null_count: Precision::Absent, max_value: Precision::Absent, min_value: Precision::Absent, + sum_value: Precision::Absent, distinct_count: Precision::Absent, } } @@ -469,6 +528,7 @@ impl ColumnStatistics { self.null_count = self.null_count.to_inexact(); self.max_value = self.max_value.to_inexact(); self.min_value = self.min_value.to_inexact(); + self.sum_value = self.sum_value.to_inexact(); self.distinct_count = self.distinct_count.to_inexact(); self } @@ -563,6 +623,26 @@ mod tests { assert_eq!(precision1.add(&absent_precision), Precision::Absent); } + #[test] + fn test_add_scalar() { + let precision = Precision::Exact(ScalarValue::Int32(Some(42))); + + assert_eq!( + precision.add(&Precision::Exact(ScalarValue::Int32(Some(23)))), + Precision::Exact(ScalarValue::Int32(Some(65))), + ); + assert_eq!( + precision.add(&Precision::Inexact(ScalarValue::Int32(Some(23)))), + Precision::Inexact(ScalarValue::Int32(Some(65))), + ); + assert_eq!( + precision.add(&Precision::Exact(ScalarValue::Int32(None))), + // As per behavior of ScalarValue::add + Precision::Exact(ScalarValue::Int32(None)), + ); + assert_eq!(precision.add(&Precision::Absent), Precision::Absent); + } + #[test] fn test_sub() { let precision1 = Precision::Exact(42); @@ -575,6 +655,26 @@ mod tests { assert_eq!(precision1.sub(&absent_precision), Precision::Absent); } + #[test] + fn test_sub_scalar() { + let precision = Precision::Exact(ScalarValue::Int32(Some(42))); + + assert_eq!( + precision.sub(&Precision::Exact(ScalarValue::Int32(Some(23)))), + Precision::Exact(ScalarValue::Int32(Some(19))), + ); + assert_eq!( + precision.sub(&Precision::Inexact(ScalarValue::Int32(Some(23)))), + Precision::Inexact(ScalarValue::Int32(Some(19))), + ); + assert_eq!( + precision.sub(&Precision::Exact(ScalarValue::Int32(None))), + // As per behavior of ScalarValue::sub + Precision::Exact(ScalarValue::Int32(None)), + ); + assert_eq!(precision.sub(&Precision::Absent), Precision::Absent); + } + #[test] fn test_multiply() { let precision1 = Precision::Exact(6); @@ -588,6 +688,54 @@ mod tests { assert_eq!(precision1.multiply(&absent_precision), Precision::Absent); } + #[test] + fn test_multiply_scalar() { + let precision = Precision::Exact(ScalarValue::Int32(Some(6))); + + assert_eq!( + precision.multiply(&Precision::Exact(ScalarValue::Int32(Some(5)))), + Precision::Exact(ScalarValue::Int32(Some(30))), + ); + assert_eq!( + precision.multiply(&Precision::Inexact(ScalarValue::Int32(Some(5)))), + Precision::Inexact(ScalarValue::Int32(Some(30))), + ); + assert_eq!( + precision.multiply(&Precision::Exact(ScalarValue::Int32(None))), + // As per behavior of ScalarValue::mul_checked + Precision::Exact(ScalarValue::Int32(None)), + ); + assert_eq!(precision.multiply(&Precision::Absent), Precision::Absent); + } + + #[test] + fn test_cast_to() { + // Valid + assert_eq!( + Precision::Exact(ScalarValue::Int32(Some(42))) + .cast_to(&DataType::Int64) + .unwrap(), + Precision::Exact(ScalarValue::Int64(Some(42))), + ); + assert_eq!( + Precision::Inexact(ScalarValue::Int32(Some(42))) + .cast_to(&DataType::Int64) + .unwrap(), + Precision::Inexact(ScalarValue::Int64(Some(42))), + ); + // Null + assert_eq!( + Precision::Exact(ScalarValue::Int32(None)) + .cast_to(&DataType::Int64) + .unwrap(), + Precision::Exact(ScalarValue::Int64(None)), + ); + // Overflow returns error + assert!(Precision::Exact(ScalarValue::Int32(Some(256))) + .cast_to(&DataType::Int8) + .is_err()); + } + #[test] fn test_precision_cloning() { // Precision is copy @@ -646,6 +794,7 @@ mod tests { null_count: Precision::Exact(null_count), max_value: Precision::Exact(ScalarValue::Int64(Some(42))), min_value: Precision::Exact(ScalarValue::Int64(Some(64))), + sum_value: Precision::Exact(ScalarValue::Int64(Some(4600))), distinct_count: Precision::Exact(100), } } diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 201bbfd5c007..f02927619a7d 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -76,6 +76,7 @@ pub async fn get_statistics_with_limit( col_stats_set[index].null_count = file_column.null_count; col_stats_set[index].max_value = file_column.max_value; col_stats_set[index].min_value = file_column.min_value; + col_stats_set[index].sum_value = file_column.sum_value; } // If the number of rows exceeds the limit, we can stop processing @@ -113,12 +114,14 @@ pub async fn get_statistics_with_limit( null_count: file_nc, max_value: file_max, min_value: file_min, + sum_value: file_sum, distinct_count: _, } = file_col_stats; col_stats.null_count = add_row_stats(*file_nc, col_stats.null_count); set_max_if_greater(file_max, &mut col_stats.max_value); - set_min_if_lesser(file_min, &mut col_stats.min_value) + set_min_if_lesser(file_min, &mut col_stats.min_value); + col_stats.sum_value = file_sum.add(&col_stats.sum_value); } // If the number of rows exceeds the limit, we can stop processing @@ -204,6 +207,7 @@ pub(crate) fn get_col_stats( null_count: null_counts[i], max_value: max_value.map(Precision::Exact).unwrap_or(Precision::Absent), min_value: min_value.map(Precision::Exact).unwrap_or(Precision::Absent), + sum_value: Precision::Absent, distinct_count: Precision::Absent, } }) diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index 9d3bd594a929..1fd6dfec79fb 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -200,12 +200,14 @@ fn fully_defined() -> (Statistics, Schema) { distinct_count: Precision::Exact(2), max_value: Precision::Exact(ScalarValue::Int32(Some(1023))), min_value: Precision::Exact(ScalarValue::Int32(Some(-24))), + sum_value: Precision::Exact(ScalarValue::Int64(Some(10))), null_count: Precision::Exact(0), }, ColumnStatistics { distinct_count: Precision::Exact(13), max_value: Precision::Exact(ScalarValue::Int64(Some(5486))), min_value: Precision::Exact(ScalarValue::Int64(Some(-6783))), + sum_value: Precision::Exact(ScalarValue::Int64(Some(10))), null_count: Precision::Exact(5), }, ], diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index aefb90d1d1b7..20a4e89dba94 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -333,12 +333,14 @@ mod tests { distinct_count: Precision::Absent, max_value: Precision::Absent, min_value: Precision::Absent, + sum_value: Precision::Absent, null_count: Precision::Exact(0), }, ColumnStatistics { distinct_count: Precision::Absent, max_value: Precision::Absent, min_value: Precision::Absent, + sum_value: Precision::Absent, null_count: Precision::Exact(0), }, ], @@ -371,6 +373,7 @@ mod tests { distinct_count: Precision::Absent, max_value: Precision::Absent, min_value: Precision::Absent, + sum_value: Precision::Absent, null_count: Precision::Exact(3), }], }; diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index ec860b3a9f61..39f022b58cf3 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -476,6 +476,7 @@ fn collect_new_statistics( null_count: input_column_stats[idx].null_count.to_inexact(), max_value, min_value, + sum_value: Precision::Absent, distinct_count: distinct_count.to_inexact(), } }, @@ -1196,6 +1197,7 @@ mod tests { null_count: Precision::Absent, min_value: Precision::Inexact(ScalarValue::Int32(Some(5))), max_value: Precision::Inexact(ScalarValue::Int32(Some(10))), + sum_value: Precision::Absent, distinct_count: Precision::Absent, }], }; diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 87fd0f96586a..ab94c132a209 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -411,12 +411,36 @@ fn stats_cartesian_product( distinct_count: s.distinct_count, min_value: s.min_value, max_value: s.max_value, + sum_value: s + .sum_value + .get_value() + // Cast the row count into the same type as any existing sum value + .and_then(|v| { + Precision::::from(right_row_count) + .cast_to(&v.data_type()) + .ok() + }) + .map(|row_count| s.sum_value.multiply(&row_count)) + .unwrap_or(Precision::Absent), }) - .chain(right_col_stats.into_iter().map(|s| ColumnStatistics { - null_count: s.null_count.multiply(&left_row_count), - distinct_count: s.distinct_count, - min_value: s.min_value, - max_value: s.max_value, + .chain(right_col_stats.into_iter().map(|s| { + ColumnStatistics { + null_count: s.null_count.multiply(&left_row_count), + distinct_count: s.distinct_count, + min_value: s.min_value, + max_value: s.max_value, + sum_value: s + .sum_value + .get_value() + // Cast the row count into the same type as any existing sum value + .and_then(|v| { + Precision::::from(left_row_count) + .cast_to(&v.data_type()) + .ok() + }) + .map(|row_count| s.sum_value.multiply(&row_count)) + .unwrap_or(Precision::Absent), + } })) .collect(); @@ -650,12 +674,14 @@ mod tests { distinct_count: Precision::Exact(5), max_value: Precision::Exact(ScalarValue::Int64(Some(21))), min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + sum_value: Precision::Exact(ScalarValue::Int64(Some(42))), null_count: Precision::Exact(0), }, ColumnStatistics { distinct_count: Precision::Exact(1), max_value: Precision::Exact(ScalarValue::from("x")), min_value: Precision::Exact(ScalarValue::from("a")), + sum_value: Precision::Absent, null_count: Precision::Exact(3), }, ], @@ -668,6 +694,7 @@ mod tests { distinct_count: Precision::Exact(3), max_value: Precision::Exact(ScalarValue::Int64(Some(12))), min_value: Precision::Exact(ScalarValue::Int64(Some(0))), + sum_value: Precision::Exact(ScalarValue::Int64(Some(20))), null_count: Precision::Exact(2), }], }; @@ -682,18 +709,25 @@ mod tests { distinct_count: Precision::Exact(5), max_value: Precision::Exact(ScalarValue::Int64(Some(21))), min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + sum_value: Precision::Exact(ScalarValue::Int64(Some( + 42 * right_row_count as i64, + ))), null_count: Precision::Exact(0), }, ColumnStatistics { distinct_count: Precision::Exact(1), max_value: Precision::Exact(ScalarValue::from("x")), min_value: Precision::Exact(ScalarValue::from("a")), + sum_value: Precision::Absent, null_count: Precision::Exact(3 * right_row_count), }, ColumnStatistics { distinct_count: Precision::Exact(3), max_value: Precision::Exact(ScalarValue::Int64(Some(12))), min_value: Precision::Exact(ScalarValue::Int64(Some(0))), + sum_value: Precision::Exact(ScalarValue::Int64(Some( + 20 * left_row_count as i64, + ))), null_count: Precision::Exact(2 * left_row_count), }, ], @@ -714,12 +748,14 @@ mod tests { distinct_count: Precision::Exact(5), max_value: Precision::Exact(ScalarValue::Int64(Some(21))), min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + sum_value: Precision::Exact(ScalarValue::Int64(Some(42))), null_count: Precision::Exact(0), }, ColumnStatistics { distinct_count: Precision::Exact(1), max_value: Precision::Exact(ScalarValue::from("x")), min_value: Precision::Exact(ScalarValue::from("a")), + sum_value: Precision::Absent, null_count: Precision::Exact(3), }, ], @@ -732,6 +768,7 @@ mod tests { distinct_count: Precision::Exact(3), max_value: Precision::Exact(ScalarValue::Int64(Some(12))), min_value: Precision::Exact(ScalarValue::Int64(Some(0))), + sum_value: Precision::Exact(ScalarValue::Int64(Some(20))), null_count: Precision::Exact(2), }], }; @@ -746,18 +783,23 @@ mod tests { distinct_count: Precision::Exact(5), max_value: Precision::Exact(ScalarValue::Int64(Some(21))), min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + sum_value: Precision::Absent, // we don't know the row count on the right null_count: Precision::Absent, // we don't know the row count on the right }, ColumnStatistics { distinct_count: Precision::Exact(1), max_value: Precision::Exact(ScalarValue::from("x")), min_value: Precision::Exact(ScalarValue::from("a")), + sum_value: Precision::Absent, null_count: Precision::Absent, // we don't know the row count on the right }, ColumnStatistics { distinct_count: Precision::Exact(3), max_value: Precision::Exact(ScalarValue::Int64(Some(12))), min_value: Precision::Exact(ScalarValue::Int64(Some(0))), + sum_value: Precision::Exact(ScalarValue::Int64(Some( + 20 * left_row_count as i64, + ))), null_count: Precision::Exact(2 * left_row_count), }, ], diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index dea4305fa6a1..5327793d01e2 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -2026,6 +2026,7 @@ mod tests { distinct_count, min_value: min.map(ScalarValue::from), max_value: max.map(ScalarValue::from), + sum_value: Absent, null_count, } } diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 5ad3c4881b39..198b8ccd6992 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -937,6 +937,7 @@ mod tests { distinct_count: Precision::Absent, max_value: Precision::Absent, min_value: Precision::Absent, + sum_value: Precision::Absent, },], } ); diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index b364d4a870e3..3ebfd8f8ca80 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -1106,18 +1106,21 @@ mod tests { distinct_count: Precision::Exact(5), max_value: Precision::Exact(ScalarValue::Int64(Some(21))), min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + sum_value: Precision::Exact(ScalarValue::Int64(Some(42))), null_count: Precision::Exact(0), }, ColumnStatistics { distinct_count: Precision::Exact(1), max_value: Precision::Exact(ScalarValue::from("x")), min_value: Precision::Exact(ScalarValue::from("a")), + sum_value: Precision::Absent, null_count: Precision::Exact(3), }, ColumnStatistics { distinct_count: Precision::Absent, max_value: Precision::Exact(ScalarValue::Float32(Some(1.1))), min_value: Precision::Exact(ScalarValue::Float32(Some(0.1))), + sum_value: Precision::Exact(ScalarValue::Float32(Some(5.5))), null_count: Precision::Absent, }, ], @@ -1150,12 +1153,14 @@ mod tests { distinct_count: Precision::Exact(1), max_value: Precision::Exact(ScalarValue::from("x")), min_value: Precision::Exact(ScalarValue::from("a")), + sum_value: Precision::Absent, null_count: Precision::Exact(3), }, ColumnStatistics { distinct_count: Precision::Exact(5), max_value: Precision::Exact(ScalarValue::Int64(Some(21))), min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + sum_value: Precision::Exact(ScalarValue::Int64(Some(42))), null_count: Precision::Exact(0), }, ], @@ -1184,12 +1189,14 @@ mod tests { distinct_count: Precision::Absent, max_value: Precision::Exact(ScalarValue::Float32(Some(1.1))), min_value: Precision::Exact(ScalarValue::Float32(Some(0.1))), + sum_value: Precision::Exact(ScalarValue::Float32(Some(5.5))), null_count: Precision::Absent, }, ColumnStatistics { distinct_count: Precision::Exact(5), max_value: Precision::Exact(ScalarValue::Int64(Some(21))), min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + sum_value: Precision::Exact(ScalarValue::Int64(Some(42))), null_count: Precision::Exact(0), }, ], diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index bacd02398ec0..a41336ea6eb7 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -610,6 +610,7 @@ fn col_stats_union( left.distinct_count = Precision::Absent; left.min_value = left.min_value.min(&right.min_value); left.max_value = left.max_value.max(&right.max_value); + left.sum_value = left.sum_value.add(&right.sum_value); left.null_count = left.null_count.add(&right.null_count); left @@ -702,18 +703,21 @@ mod tests { distinct_count: Precision::Exact(5), max_value: Precision::Exact(ScalarValue::Int64(Some(21))), min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + sum_value: Precision::Exact(ScalarValue::Int64(Some(42))), null_count: Precision::Exact(0), }, ColumnStatistics { distinct_count: Precision::Exact(1), max_value: Precision::Exact(ScalarValue::from("x")), min_value: Precision::Exact(ScalarValue::from("a")), + sum_value: Precision::Absent, null_count: Precision::Exact(3), }, ColumnStatistics { distinct_count: Precision::Absent, max_value: Precision::Exact(ScalarValue::Float32(Some(1.1))), min_value: Precision::Exact(ScalarValue::Float32(Some(0.1))), + sum_value: Precision::Exact(ScalarValue::Float32(Some(42.0))), null_count: Precision::Absent, }, ], @@ -727,18 +731,21 @@ mod tests { distinct_count: Precision::Exact(3), max_value: Precision::Exact(ScalarValue::Int64(Some(34))), min_value: Precision::Exact(ScalarValue::Int64(Some(1))), + sum_value: Precision::Exact(ScalarValue::Int64(Some(42))), null_count: Precision::Exact(1), }, ColumnStatistics { distinct_count: Precision::Absent, max_value: Precision::Exact(ScalarValue::from("c")), min_value: Precision::Exact(ScalarValue::from("b")), + sum_value: Precision::Absent, null_count: Precision::Absent, }, ColumnStatistics { distinct_count: Precision::Absent, max_value: Precision::Absent, min_value: Precision::Absent, + sum_value: Precision::Absent, null_count: Precision::Absent, }, ], @@ -753,18 +760,21 @@ mod tests { distinct_count: Precision::Absent, max_value: Precision::Exact(ScalarValue::Int64(Some(34))), min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + sum_value: Precision::Exact(ScalarValue::Int64(Some(84))), null_count: Precision::Exact(1), }, ColumnStatistics { distinct_count: Precision::Absent, max_value: Precision::Exact(ScalarValue::from("x")), min_value: Precision::Exact(ScalarValue::from("a")), + sum_value: Precision::Absent, null_count: Precision::Absent, }, ColumnStatistics { distinct_count: Precision::Absent, max_value: Precision::Absent, min_value: Precision::Absent, + sum_value: Precision::Absent, null_count: Precision::Absent, }, ], diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index b000f9335a3e..960e3f544ee0 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -321,6 +321,7 @@ mod tests { distinct_count: Precision::Absent, max_value: Precision::Absent, min_value: Precision::Absent, + sum_value: Precision::Absent, },], } ); diff --git a/datafusion/proto-common/proto/datafusion_common.proto b/datafusion/proto-common/proto/datafusion_common.proto index 6a7dc1604b0a..1c2807f390bf 100644 --- a/datafusion/proto-common/proto/datafusion_common.proto +++ b/datafusion/proto-common/proto/datafusion_common.proto @@ -570,6 +570,7 @@ message Statistics { message ColumnStats { Precision min_value = 1; Precision max_value = 2; + Precision sum_value = 5; Precision null_count = 3; Precision distinct_count = 4; } diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index d88186fbf366..b022e52b6a6f 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -701,6 +701,11 @@ impl From<&protobuf::ColumnStats> for ColumnStatistics { } else { Precision::Absent }, + sum_value: if let Some(sum) = &cs.sum_value { + sum.clone().into() + } else { + Precision::Absent + }, distinct_count: if let Some(dc) = &cs.distinct_count { dc.clone().into() } else { diff --git a/datafusion/proto-common/src/generated/pbjson.rs b/datafusion/proto-common/src/generated/pbjson.rs index e9f9de09d4d1..40687de098c1 100644 --- a/datafusion/proto-common/src/generated/pbjson.rs +++ b/datafusion/proto-common/src/generated/pbjson.rs @@ -985,6 +985,9 @@ impl serde::Serialize for ColumnStats { if self.max_value.is_some() { len += 1; } + if self.sum_value.is_some() { + len += 1; + } if self.null_count.is_some() { len += 1; } @@ -998,6 +1001,9 @@ impl serde::Serialize for ColumnStats { if let Some(v) = self.max_value.as_ref() { struct_ser.serialize_field("maxValue", v)?; } + if let Some(v) = self.sum_value.as_ref() { + struct_ser.serialize_field("sumValue", v)?; + } if let Some(v) = self.null_count.as_ref() { struct_ser.serialize_field("nullCount", v)?; } @@ -1018,6 +1024,8 @@ impl<'de> serde::Deserialize<'de> for ColumnStats { "minValue", "max_value", "maxValue", + "sum_value", + "sumValue", "null_count", "nullCount", "distinct_count", @@ -1028,6 +1036,7 @@ impl<'de> serde::Deserialize<'de> for ColumnStats { enum GeneratedField { MinValue, MaxValue, + SumValue, NullCount, DistinctCount, } @@ -1053,6 +1062,7 @@ impl<'de> serde::Deserialize<'de> for ColumnStats { match value { "minValue" | "min_value" => Ok(GeneratedField::MinValue), "maxValue" | "max_value" => Ok(GeneratedField::MaxValue), + "sumValue" | "sum_value" => Ok(GeneratedField::SumValue), "nullCount" | "null_count" => Ok(GeneratedField::NullCount), "distinctCount" | "distinct_count" => Ok(GeneratedField::DistinctCount), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), @@ -1076,6 +1086,7 @@ impl<'de> serde::Deserialize<'de> for ColumnStats { { let mut min_value__ = None; let mut max_value__ = None; + let mut sum_value__ = None; let mut null_count__ = None; let mut distinct_count__ = None; while let Some(k) = map_.next_key()? { @@ -1092,6 +1103,12 @@ impl<'de> serde::Deserialize<'de> for ColumnStats { } max_value__ = map_.next_value()?; } + GeneratedField::SumValue => { + if sum_value__.is_some() { + return Err(serde::de::Error::duplicate_field("sumValue")); + } + sum_value__ = map_.next_value()?; + } GeneratedField::NullCount => { if null_count__.is_some() { return Err(serde::de::Error::duplicate_field("nullCount")); @@ -1109,6 +1126,7 @@ impl<'de> serde::Deserialize<'de> for ColumnStats { Ok(ColumnStats { min_value: min_value__, max_value: max_value__, + sum_value: sum_value__, null_count: null_count__, distinct_count: distinct_count__, }) diff --git a/datafusion/proto-common/src/generated/prost.rs b/datafusion/proto-common/src/generated/prost.rs index 3263c1c755af..9e4a1ecb6b09 100644 --- a/datafusion/proto-common/src/generated/prost.rs +++ b/datafusion/proto-common/src/generated/prost.rs @@ -873,6 +873,8 @@ pub struct ColumnStats { pub min_value: ::core::option::Option, #[prost(message, optional, tag = "2")] pub max_value: ::core::option::Option, + #[prost(message, optional, tag = "5")] + pub sum_value: ::core::option::Option, #[prost(message, optional, tag = "3")] pub null_count: ::core::option::Option, #[prost(message, optional, tag = "4")] diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index 88bbbfd60426..ced1865795aa 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -750,6 +750,7 @@ impl From<&ColumnStatistics> for protobuf::ColumnStats { protobuf::ColumnStats { min_value: Some(protobuf::Precision::from(&s.min_value)), max_value: Some(protobuf::Precision::from(&s.max_value)), + sum_value: Some(protobuf::Precision::from(&s.sum_value)), null_count: Some(protobuf::Precision::from(&s.null_count)), distinct_count: Some(protobuf::Precision::from(&s.distinct_count)), } diff --git a/datafusion/proto/src/generated/datafusion_proto_common.rs b/datafusion/proto/src/generated/datafusion_proto_common.rs index 3263c1c755af..9e4a1ecb6b09 100644 --- a/datafusion/proto/src/generated/datafusion_proto_common.rs +++ b/datafusion/proto/src/generated/datafusion_proto_common.rs @@ -873,6 +873,8 @@ pub struct ColumnStats { pub min_value: ::core::option::Option, #[prost(message, optional, tag = "2")] pub max_value: ::core::option::Option, + #[prost(message, optional, tag = "5")] + pub sum_value: ::core::option::Option, #[prost(message, optional, tag = "3")] pub null_count: ::core::option::Option, #[prost(message, optional, tag = "4")] From 3a991e65f1bc44982f60133282b1eea3ffafb8f7 Mon Sep 17 00:00:00 2001 From: Ian Lai <108986288+Chen-Yuan-Lai@users.noreply.github.com> Date: Wed, 29 Jan 2025 04:26:46 +0800 Subject: [PATCH 23/62] refactor: switch BooleanBufferBuilder to NullBufferBuilder in unit tests for unnest (#14321) Co-authored-by: Cheng-Yuan-Lai --- datafusion/physical-plan/src/unnest.rs | 34 +++++++++++++------------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 3e2b3fb26d45..ef6797c9b10d 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -953,7 +953,7 @@ mod tests { use super::*; use arrow::datatypes::{Field, Int32Type}; use arrow_array::{GenericListArray, OffsetSizeTrait, StringArray}; - use arrow_buffer::{BooleanBufferBuilder, NullBuffer, OffsetBuffer}; + use arrow_buffer::{NullBuffer, NullBufferBuilder, OffsetBuffer}; use datafusion_common::assert_batches_eq; // Create a GenericListArray with the following list values: @@ -964,43 +964,43 @@ mod tests { { let mut values = vec![]; let mut offsets: Vec = vec![OffsetSize::zero()]; - let mut valid = BooleanBufferBuilder::new(6); + let mut valid = NullBufferBuilder::new(6); // [A, B, C] values.extend_from_slice(&[Some("A"), Some("B"), Some("C")]); offsets.push(OffsetSize::from_usize(values.len()).unwrap()); - valid.append(true); + valid.append_non_null(); // [] offsets.push(OffsetSize::from_usize(values.len()).unwrap()); - valid.append(true); + valid.append_non_null(); // NULL with non-zero value length // Issue https://github.com/apache/datafusion/issues/9932 values.push(Some("?")); offsets.push(OffsetSize::from_usize(values.len()).unwrap()); - valid.append(false); + valid.append_null(); // [D] values.push(Some("D")); offsets.push(OffsetSize::from_usize(values.len()).unwrap()); - valid.append(true); + valid.append_non_null(); // Another NULL with zero value length offsets.push(OffsetSize::from_usize(values.len()).unwrap()); - valid.append(false); + valid.append_null(); // [NULL, F] values.extend_from_slice(&[None, Some("F")]); offsets.push(OffsetSize::from_usize(values.len()).unwrap()); - valid.append(true); + valid.append_non_null(); let field = Arc::new(Field::new_list_field(DataType::Utf8, true)); GenericListArray::::new( field, OffsetBuffer::new(offsets.into()), Arc::new(StringArray::from(values)), - Some(NullBuffer::new(valid.finish())), + valid.finish(), ) } @@ -1055,10 +1055,10 @@ mod tests { let list_arr1_ref = Arc::new(list_arr1) as ArrayRef; let offsets = OffsetBuffer::from_lengths([3, 3, 0]); - let mut nulls = BooleanBufferBuilder::new(3); - nulls.append(true); - nulls.append(true); - nulls.append(false); + let mut nulls = NullBufferBuilder::new(3); + nulls.append_non_null(); + nulls.append_non_null(); + nulls.append_null(); // list> let col1_field = Field::new_list_field( DataType::List(Arc::new(Field::new_list_field( @@ -1074,7 +1074,7 @@ mod tests { )), offsets, list_arr1_ref, - Some(NullBuffer::new(nulls.finish())), + nulls.finish(), ); let list_arr2 = StringArray::from(vec![ @@ -1086,8 +1086,8 @@ mod tests { ]); let offsets = OffsetBuffer::from_lengths([2, 2, 1]); - let mut nulls = BooleanBufferBuilder::new(3); - nulls.append_n(3, true); + let mut nulls = NullBufferBuilder::new(3); + nulls.append_n_non_nulls(3); let col2_field = Field::new( "col2", DataType::List(Arc::new(Field::new_list_field(DataType::Utf8, true))), @@ -1097,7 +1097,7 @@ mod tests { Arc::new(Field::new_list_field(DataType::Utf8, true)), OffsetBuffer::new(offsets.into()), Arc::new(list_arr2), - Some(NullBuffer::new(nulls.finish())), + nulls.finish(), ); // convert col1 and col2 to a record batch let schema = Arc::new(Schema::new(vec![col1_field, col2_field])); From a4917d44c8cc3b08c61383320285d239250dd94e Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 28 Jan 2025 15:33:33 -0500 Subject: [PATCH 24/62] Increase MSRV to 1.81.0 (#14330) * Increase MSRV to 1.81.0 * also update datafusion-cli --- Cargo.toml | 2 +- datafusion-cli/Cargo.toml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 9eb4d5b39af0..85b26f802f05 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -65,7 +65,7 @@ homepage = "https://datafusion.apache.org" license = "Apache-2.0" readme = "README.md" repository = "https://github.com/apache/datafusion" -rust-version = "1.80.1" +rust-version = "1.81.0" version = "44.0.0" [workspace.dependencies] diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 30060665934c..c418d17ce29b 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -25,7 +25,7 @@ keywords = ["arrow", "datafusion", "query", "sql"] license = "Apache-2.0" homepage = "https://datafusion.apache.org" repository = "https://github.com/apache/datafusion" -rust-version = "1.80.1" +rust-version = "1.81.0" readme = "README.md" [dependencies] From 66b4da2c17b4538c936a6a080b4187b77b8dcd35 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 29 Jan 2025 01:14:15 -0500 Subject: [PATCH 25/62] Fix build due to logical error (#14345) --- datafusion/physical-plan/src/filter.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 39f022b58cf3..91c44a4139d2 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -463,6 +463,7 @@ fn collect_new_statistics( null_count: Precision::Exact(0), max_value: Precision::Exact(ScalarValue::Null), min_value: Precision::Exact(ScalarValue::Null), + sum_value: Precision::Exact(ScalarValue::Null), distinct_count: Precision::Exact(0), }; }; @@ -1090,12 +1091,14 @@ mod tests { ColumnStatistics { min_value: Precision::Exact(ScalarValue::Null), max_value: Precision::Exact(ScalarValue::Null), + sum_value: Precision::Exact(ScalarValue::Null), distinct_count: Precision::Exact(0), null_count: Precision::Exact(0), }, ColumnStatistics { min_value: Precision::Exact(ScalarValue::Null), max_value: Precision::Exact(ScalarValue::Null), + sum_value: Precision::Exact(ScalarValue::Null), distinct_count: Precision::Exact(0), null_count: Precision::Exact(0), }, From 972c56f5da89f64130971467da8f6c9e0e251a95 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 29 Jan 2025 15:16:36 +0300 Subject: [PATCH 26/62] fix the tests --- .../physical_optimizer/enforce_sorting.rs | 130 +++++++++++------- .../physical-plan/src/aggregates/mod.rs | 24 +++- 2 files changed, 101 insertions(+), 53 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index f8e2323150e9..304dd3c659fe 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -242,13 +242,16 @@ async fn test_remove_unnecessary_sort5() -> Result<()> { } #[test] -fn test_aggregate_set_monotonic() -> Result<()> { +fn test_aggregate_set_monotonic_no_group() -> Result<()> { let schema = create_test_schema4()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("a", &schema)]; let sort = sort_exec(sort_exprs.clone(), source); let aggregate = aggregate_exec_set_monotonic(sort, vec![]); + let sort_exprs = LexOrdering::new(vec![sort_expr("count", &aggregate.schema())]); let physical_plan: Arc = Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _; @@ -270,14 +273,17 @@ fn test_aggregate_set_monotonic() -> Result<()> { } #[test] -fn test_aggregate_set_monotonic_with_groupby() -> Result<()> { +fn test_aggregate_set_monotonic_with_group() -> Result<()> { let schema = create_test_schema4()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("a", &schema)]; let sort = sort_exec(sort_exprs.clone(), source); let aggregate = aggregate_exec_set_monotonic(sort, vec![(col("a", &schema)?, "a".to_string())]); + let sort_exprs = LexOrdering::new(vec![sort_expr("count", &aggregate.schema())]); let physical_plan: Arc = Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _; @@ -289,6 +295,43 @@ fn test_aggregate_set_monotonic_with_groupby() -> Result<()> { " MemoryExec: partitions=1, partition_sizes=[0]", ]; + let expected_optimized = [ + "SortExec: expr=[count@1 ASC], preserve_partitioning=[false]", + " AggregateExec: mode=Single, gby=[a@0 as a], aggr=[count], ordering_mode=Sorted", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[test] +fn test_aggregate_set_monotonic_with_group_partial() -> Result<()> { + let schema = create_test_schema4()?; + + let source = memory_exec(&schema); + + let sort_exprs = vec![sort_expr("a", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + + let aggregate = + aggregate_exec_set_monotonic(sort, vec![(col("a", &schema)?, "a".to_string())]); + + let sort_exprs = LexOrdering::new(vec![ + sort_expr("a", &schema), + sort_expr("count", &aggregate.schema()), + ]); + let physical_plan: Arc = + Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _; + + let expected_input = [ + "SortExec: expr=[a@0 ASC, count@1 ASC], preserve_partitioning=[false]", + " AggregateExec: mode=Single, gby=[a@0 as a], aggr=[count], ordering_mode=Sorted", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[count], ordering_mode=Sorted", " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", @@ -329,8 +372,11 @@ fn test_aggregate_non_set_monotonic() -> Result<()> { } #[tokio::test] -async fn test_bounded_window_set_monotonic_sort() -> Result<()> { +async fn test_bounded_window_set_monotonic_no_partition() -> Result<()> { let schema = create_test_schema()?; + + let source = parquet_exec_sorted(&schema, vec![]); + let sort_exprs = vec![sort_expr_options( "nullable_col", &schema, @@ -339,11 +385,9 @@ async fn test_bounded_window_set_monotonic_sort() -> Result<()> { nulls_first: false, }, )]; - let source = parquet_exec_sorted(&schema, sort_exprs.clone()); - let sort = sort_exec(sort_exprs.clone(), source); - let bounded_window = bounded_window_exec("nullable_col", sort_exprs.clone(), sort); + let bounded_window = bounded_window_exec("nullable_col", vec![], sort); let output_schema = bounded_window.schema(); let sort_exprs2 = vec![sort_expr_options( @@ -360,11 +404,11 @@ async fn test_bounded_window_set_monotonic_sort() -> Result<()> { "SortExec: expr=[count@2 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; let expected_optimized = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -372,8 +416,11 @@ async fn test_bounded_window_set_monotonic_sort() -> Result<()> { } #[tokio::test] -async fn test_bounded_plain_window_set_monotonic_sort_with_partitions() -> Result<()> { +async fn test_bounded_plain_window_set_monotonic_with_partitions() -> Result<()> { let schema = create_test_schema()?; + + let source = parquet_exec_sorted(&schema, vec![]); + let sort_exprs = vec![sort_expr_options( "nullable_col", &schema, @@ -382,49 +429,39 @@ async fn test_bounded_plain_window_set_monotonic_sort_with_partitions() -> Resul nulls_first: false, }, )]; - let source = parquet_exec_sorted(&schema, sort_exprs.clone()); - let sort = sort_exec(sort_exprs.clone(), source); let partition_bys = &[col("nullable_col", &schema)?]; let bounded_window = bounded_window_exec_with_partition( - "nullable_col", - sort_exprs.clone(), + "non_nullable_col", + vec![], partition_bys, sort, false, ); let output_schema = bounded_window.schema(); - let sort_exprs2 = vec![ - sort_expr_options( - "nullable_col", - &output_schema, - SortOptions { - descending: true, - nulls_first: false, - }, - ), - sort_expr_options( - "count", - &output_schema, - SortOptions { - descending: false, - nulls_first: false, - }, - ), - ]; + let sort_exprs2 = vec![sort_expr_options( + "count", + &output_schema, + SortOptions { + descending: false, + nulls_first: false, + }, + )]; let physical_plan = sort_exec(sort_exprs2.clone(), bounded_window); let expected_input = [ - "SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]", + "SortExec: expr=[count@2 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; let expected_optimized = [ - "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + "SortExec: expr=[count@2 ASC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 ASC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -432,9 +469,11 @@ async fn test_bounded_plain_window_set_monotonic_sort_with_partitions() -> Resul } #[tokio::test] -async fn test_bounded_plain_window_reverse_set_monotonic_sort_with_partitions( -) -> Result<()> { +async fn test_bounded_plain_window_set_monotonic_with_partitions_partial() -> Result<()> { let schema = create_test_schema()?; + + let source = parquet_exec_sorted(&schema, vec![]); + let sort_exprs = vec![sort_expr_options( "nullable_col", &schema, @@ -443,17 +482,15 @@ async fn test_bounded_plain_window_reverse_set_monotonic_sort_with_partitions( nulls_first: false, }, )]; - let source = parquet_exec_sorted(&schema, sort_exprs.clone()); - let sort = sort_exec(sort_exprs.clone(), source); let partition_bys = &[col("nullable_col", &schema)?]; let bounded_window = bounded_window_exec_with_partition( - "nullable_col", - sort_exprs.clone(), + "non_nullable_col", + vec![], partition_bys, sort, - true, + false, ); let output_schema = bounded_window.schema(); @@ -479,13 +516,14 @@ async fn test_bounded_plain_window_reverse_set_monotonic_sort_with_partitions( let expected_input = [ "SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]", - " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }], mode=[Sorted]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; let expected_optimized = [ - "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }], mode=[Sorted]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]", + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index dc62208d0b0c..11463b1c27d3 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -664,6 +664,23 @@ impl AggregateExec { .equivalence_properties() .project(group_expr_mapping, schema); + // if the aggregate function is set monotonic, add it into equivalence properties + for (i, aggr_expr) in aggr_exprs.iter().enumerate() { + let aggr_expr_index = aggr_expr_indices[i]; + if let Some(expr) = aggr_expr.natural_sort_expr(aggr_expr_index) { + if group_expr_mapping.map.is_empty() { + eq_properties.add_new_ordering(LexOrdering::new(vec![expr])); + } else if *input_order_mode != InputOrderMode::Linear { + if let Some(ordering) = eq_properties.output_ordering() { + let mut existing_ordering = ordering.to_vec(); + existing_ordering.push(expr); + eq_properties + .add_new_ordering(LexOrdering::new(existing_ordering)); + } + } + } + } + // Group by expression will be a distinct value after the aggregation. // Add it into the constraint set. let mut constraints = eq_properties.constraints().to_vec(); @@ -702,13 +719,6 @@ impl AggregateExec { input.pipeline_behavior() }; - for (i, aggr_expr) in aggr_exprs.iter().enumerate() { - let aggr_expr_index = aggr_expr_indices[i]; - if let Some(expr) = aggr_expr.natural_sort_expr(aggr_expr_index) { - eq_properties.add_new_ordering(LexOrdering::new(vec![expr])); - } - } - PlanProperties::new( eq_properties, output_partitioning, From 4b946b343f1ebf049b7270b6fd77fe8bdebb806a Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 29 Jan 2025 15:19:54 +0300 Subject: [PATCH 27/62] revert slt's --- datafusion/sqllogictest/test_files/aggregate.slt | 6 +++--- .../sqllogictest/test_files/aggregates_topk.slt | 13 +++++++------ .../sqllogictest/test_files/tpch/plans/q13.slt.part | 8 ++++---- 3 files changed, 14 insertions(+), 13 deletions(-) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index ed5eca8cc879..920e12e2fd31 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -4981,10 +4981,10 @@ logical_plan physical_plan 01)GlobalLimitExec: skip=0, fetch=5 02)--CoalescePartitionsExec -03)----AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], ordering_mode=PartiallySorted([1]) +03)----AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], lim=[5] 04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([c3@0, min(aggregate_test_100.c1)@1], 4), input_partitions=4, preserve_order=true, sort_exprs=min(aggregate_test_100.c1)@1 DESC NULLS LAST -06)----------AggregateExec: mode=Partial, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], ordering_mode=PartiallySorted([1]) +05)--------RepartitionExec: partitioning=Hash([c3@0, min(aggregate_test_100.c1)@1], 4), input_partitions=4 +06)----------AggregateExec: mode=Partial, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], lim=[5] 07)------------AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3], aggr=[min(aggregate_test_100.c1)] 08)--------------CoalesceBatchesExec: target_batch_size=8192 09)----------------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/aggregates_topk.slt b/datafusion/sqllogictest/test_files/aggregates_topk.slt index 9fe17a45edc1..a67fec695f6c 100644 --- a/datafusion/sqllogictest/test_files/aggregates_topk.slt +++ b/datafusion/sqllogictest/test_files/aggregates_topk.slt @@ -143,12 +143,13 @@ logical_plan 03)----TableScan: traces projection=[trace_id, timestamp] physical_plan 01)SortPreservingMergeExec: [max(traces.timestamp)@1 ASC NULLS LAST], fetch=4 -02)--AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -07)------------MemoryExec: partitions=1, partition_sizes=[1] +02)--SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 ASC NULLS LAST], preserve_partitioning=[true] +03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 +06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +08)--------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by trace_id asc limit 4; diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index 4dc77cf94239..a2e1b8b559da 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -59,10 +59,10 @@ physical_plan 01)SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[c_count@0 as c_count, count(*)@1 as custdist] -04)------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(*)], ordering_mode=Sorted +04)------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(*)] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4, preserve_order=true, sort_exprs=c_count@0 ASC NULLS LAST, count(*)@1 ASC NULLS LAST -07)------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(*)], ordering_mode=Sorted +06)----------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(*)] 08)--------------ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] 09)----------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] 10)------------------CoalesceBatchesExec: target_batch_size=8192 @@ -78,4 +78,4 @@ physical_plan 20)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false statement ok -set datafusion.optimizer.prefer_existing_sort = false; \ No newline at end of file +set datafusion.optimizer.prefer_existing_sort = false; From 481b5b4c3dbd7a80e538989e996f93959b1dc331 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 29 Jan 2025 15:47:38 +0300 Subject: [PATCH 28/62] simplify terms --- datafusion/physical-expr/src/aggregate.rs | 5 +++-- .../physical-expr/src/window/aggregate.rs | 15 +++++++++------ .../src/window/sliding_aggregate.rs | 19 +------------------ .../physical-expr/src/window/standard.rs | 3 +-- .../physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 10 ++++++---- 6 files changed, 21 insertions(+), 33 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 2e82ab2ab263..0fe08729828c 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -548,8 +548,9 @@ impl AggregateFunctionExpr { } /// Returns PhysicalSortExpr based on monotonicity of the function - pub fn natural_sort_expr(&self, aggr_func_idx: usize) -> Option { - // If the aggregate expressions are set-monotonic, the output data is naturally ordered with it. + pub fn get_result_ordering(&self, aggr_func_idx: usize) -> Option { + // If the aggregate expressions are set-monotonic, the output data is + // naturally ordered with it per group or partition. let monotonicity = self.set_monotonicity(); if !monotonicity.is_monotonic() { return None; diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index f22c693126bc..d94cdc1490dd 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -74,13 +74,16 @@ impl PlainAggregateWindowExpr { eq_properties: &mut EquivalenceProperties, window_expr_index: usize, ) { - let Some(expr) = self + if let Some(expr) = self .get_aggregate_expr() - .natural_sort_expr(window_expr_index) - else { - return; - }; - add_new_ordering_expr_with_partition_by(eq_properties, expr, &self.partition_by); + .get_result_ordering(window_expr_index) + { + add_new_ordering_expr_with_partition_by( + eq_properties, + expr, + &self.partition_by, + ); + } } } diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index 4752b264d752..23967e78f07a 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -22,14 +22,11 @@ use std::ops::Range; use std::sync::Arc; use crate::aggregate::AggregateFunctionExpr; -use crate::window::standard::add_new_ordering_expr_with_partition_by; use crate::window::window_expr::AggregateWindowExpr; use crate::window::{ PartitionBatches, PartitionWindowAggStates, PlainAggregateWindowExpr, WindowExpr, }; -use crate::{ - expressions::PhysicalSortExpr, reverse_order_bys, EquivalenceProperties, PhysicalExpr, -}; +use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; use arrow::array::{Array, ArrayRef}; use arrow::datatypes::Field; @@ -70,20 +67,6 @@ impl SlidingAggregateWindowExpr { pub fn get_aggregate_expr(&self) -> &AggregateFunctionExpr { &self.aggregate } - - pub fn add_equal_orderings( - &self, - eq_properties: &mut EquivalenceProperties, - window_expr_index: usize, - ) { - let Some(expr) = self - .get_aggregate_expr() - .natural_sort_expr(window_expr_index) - else { - return; - }; - add_new_ordering_expr_with_partition_by(eq_properties, expr, &self.partition_by); - } } /// Incrementally update window function using the fact that batch is diff --git a/datafusion/physical-expr/src/window/standard.rs b/datafusion/physical-expr/src/window/standard.rs index 81038ae86e48..0c0bdf7e99cd 100644 --- a/datafusion/physical-expr/src/window/standard.rs +++ b/datafusion/physical-expr/src/window/standard.rs @@ -65,8 +65,7 @@ impl StandardWindowExpr { &self.expr } - /// Adds any equivalent orderings generated by the `self.expr` - /// to `builder`. + /// Adds any equivalent orderings generated by the `self.expr` to `builder`. /// /// If `self.expr` doesn't have an ordering, ordering equivalence properties /// are not updated. Otherwise, ordering equivalence properties are updated diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 11463b1c27d3..fc129fe241e5 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -667,7 +667,7 @@ impl AggregateExec { // if the aggregate function is set monotonic, add it into equivalence properties for (i, aggr_expr) in aggr_exprs.iter().enumerate() { let aggr_expr_index = aggr_expr_indices[i]; - if let Some(expr) = aggr_expr.natural_sort_expr(aggr_expr_index) { + if let Some(expr) = aggr_expr.get_result_ordering(aggr_expr_index) { if group_expr_mapping.map.is_empty() { eq_properties.add_new_ordering(LexOrdering::new(vec![expr])); } else if *input_order_mode != InputOrderMode::Linear { diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index dcd2953acdab..24715c497fc3 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -354,11 +354,13 @@ pub(crate) fn window_equivalence_properties( { aggregate_udf_window_expr .add_equal_orderings(&mut window_eq_properties, window_expr_index); - } else if let Some(aggregate_udf_window_expr) = - expr.as_any().downcast_ref::() + } else if let Some(_) = expr.as_any().downcast_ref::() { - aggregate_udf_window_expr - .add_equal_orderings(&mut window_eq_properties, window_expr_index); + // TODO: SlidingAggregateWindowExpr cannot introduce a new ordering yet + // because we cannot determine whether the window's incoming elements + // are greater than its outgoing elements. However, we do have + // the necessary tools to support this, and we can extend support + // for these cases in the future. } } window_eq_properties From 29af731d32ee05cfc282e8556e20c71be7434f09 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 29 Jan 2025 15:53:14 +0300 Subject: [PATCH 29/62] Update mod.rs --- datafusion/physical-plan/src/windows/mod.rs | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 24715c497fc3..ce775f155912 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -354,14 +354,12 @@ pub(crate) fn window_equivalence_properties( { aggregate_udf_window_expr .add_equal_orderings(&mut window_eq_properties, window_expr_index); - } else if let Some(_) = expr.as_any().downcast_ref::() - { - // TODO: SlidingAggregateWindowExpr cannot introduce a new ordering yet - // because we cannot determine whether the window's incoming elements - // are greater than its outgoing elements. However, we do have - // the necessary tools to support this, and we can extend support - // for these cases in the future. } + // TODO: SlidingAggregateWindowExpr cannot introduce a new ordering yet + // because we cannot determine whether the window's incoming elements + // are greater than its outgoing elements. However, we do have + // the necessary tools to support this, and we can extend support + // for these cases in the future. } window_eq_properties } From 1f02953d6fa3239b490e49f05a2d3d0b3290ed19 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 29 Jan 2025 16:03:44 +0300 Subject: [PATCH 30/62] remove unnecessary computations --- .../src/windows/bounded_window_agg_exec.rs | 14 ++------------ datafusion/physical-plan/src/windows/mod.rs | 8 ++++++-- .../src/windows/window_agg_exec.rs | 18 ++---------------- 3 files changed, 10 insertions(+), 30 deletions(-) diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index b6840bccf78e..cf19d0daf378 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -67,7 +67,6 @@ use futures::stream::Stream; use futures::{ready, StreamExt}; use hashbrown::hash_table::HashTable; use indexmap::IndexMap; -use itertools::Itertools; use log::debug; /// Window execution plan @@ -104,10 +103,8 @@ impl BoundedWindowAggExec { partition_keys: Vec>, input_order_mode: InputOrderMode, ) -> Result { - let old_fields_len = input.schema().fields.len(); let schema = create_schema(&input.schema(), &window_expr)?; let schema = Arc::new(schema); - let window_expr_indices = (old_fields_len..schema.fields.len()).collect_vec(); let partition_by_exprs = window_expr[0].partition_by(); let ordered_partition_by_indices = match &input_order_mode { InputOrderMode::Sorted => { @@ -126,8 +123,7 @@ impl BoundedWindowAggExec { vec![] } }; - let cache = - Self::compute_properties(&input, &schema, &window_expr, window_expr_indices); + let cache = Self::compute_properties(&input, &schema, &window_expr); Ok(Self { input, window_expr, @@ -195,15 +191,9 @@ impl BoundedWindowAggExec { input: &Arc, schema: &SchemaRef, window_exprs: &[Arc], - window_expr_indices: Vec, ) -> PlanProperties { // Calculate equivalence properties: - let eq_properties = window_equivalence_properties( - schema, - input, - window_exprs, - window_expr_indices, - ); + let eq_properties = window_equivalence_properties(schema, input, window_exprs); // As we can have repartitioning using the partition keys, this can // be either one or more than one, depending on the presence of diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index ce775f155912..cf0de68afea4 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -337,21 +337,25 @@ pub(crate) fn window_equivalence_properties( schema: &SchemaRef, input: &Arc, window_exprs: &[Arc], - window_expr_indices: Vec, ) -> EquivalenceProperties { // We need to update the schema, so we can not directly use // `input.equivalence_properties()`. let mut window_eq_properties = EquivalenceProperties::new(Arc::clone(schema)) .extend(input.equivalence_properties().clone()); + let schema_len = schema.fields.len(); + let window_expr_indices = (schema_len..(schema_len - window_exprs.len())) + .rev() + .collect::>(); + for (i, expr) in window_exprs.iter().enumerate() { - let window_expr_index = window_expr_indices[i]; if let Some(udf_window_expr) = expr.as_any().downcast_ref::() { udf_window_expr.add_equal_orderings(&mut window_eq_properties); } else if let Some(aggregate_udf_window_expr) = expr.as_any().downcast_ref::() { + let window_expr_index = window_expr_indices[i]; aggregate_udf_window_expr .add_equal_orderings(&mut window_eq_properties, window_expr_index); } diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 16354cb69742..f0c258a02576 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -47,7 +47,6 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use futures::{ready, Stream, StreamExt}; -use itertools::Itertools; /// Window execution plan #[derive(Debug, Clone)] @@ -76,19 +75,12 @@ impl WindowAggExec { input: Arc, partition_keys: Vec>, ) -> Result { - let old_fields_len = input.schema().fields.len(); let schema = create_schema(&input.schema(), &window_expr)?; let schema = Arc::new(schema); - let window_expr_indices = (old_fields_len..schema.fields.len()).collect_vec(); let ordered_partition_by_indices = get_ordered_partition_by_indices(window_expr[0].partition_by(), &input); - let cache = Self::compute_properties( - Arc::clone(&schema), - &input, - &window_expr, - window_expr_indices, - ); + let cache = Self::compute_properties(Arc::clone(&schema), &input, &window_expr); Ok(Self { input, window_expr, @@ -129,15 +121,9 @@ impl WindowAggExec { schema: SchemaRef, input: &Arc, window_exprs: &[Arc], - window_expr_indices: Vec, ) -> PlanProperties { // Calculate equivalence properties: - let eq_properties = window_equivalence_properties( - &schema, - input, - window_exprs, - window_expr_indices, - ); + let eq_properties = window_equivalence_properties(&schema, input, window_exprs); // Get output partitioning: // Because we can have repartitioning using the partition keys this From 79dd9422e5163e78c7e3e1941c6660f1213a2421 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 29 Jan 2025 16:29:14 +0300 Subject: [PATCH 31/62] remove index calc --- .../physical-plan/src/aggregates/mod.rs | 31 +++++++------------ .../physical-plan/src/aggregates/row_hash.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 6 ++-- .../sqllogictest/test_files/aggregate.slt | 5 --- .../test_files/tpch/plans/q13.slt.part | 8 +---- 5 files changed, 15 insertions(+), 37 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index fc129fe241e5..75fb519b3245 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -375,7 +375,6 @@ pub struct AggregateExec { /// Describes how the input is ordered relative to the group by columns input_order_mode: InputOrderMode, cache: PlanProperties, - aggr_expr_indices: Vec, } impl AggregateExec { @@ -400,7 +399,6 @@ impl AggregateExec { input: Arc::clone(&self.input), schema: Arc::clone(&self.schema), input_schema: Arc::clone(&self.input_schema), - aggr_expr_indices: self.aggr_expr_indices.clone(), } } @@ -417,8 +415,7 @@ impl AggregateExec { input: Arc, input_schema: SchemaRef, ) -> Result { - let (schema, aggr_exprs_indices) = - create_schema(&input.schema(), &group_by, &aggr_expr, mode)?; + let schema = create_schema(&input.schema(), &group_by, &aggr_expr, mode)?; let schema = Arc::new(schema); AggregateExec::try_new_with_schema( @@ -429,7 +426,6 @@ impl AggregateExec { input, input_schema, schema, - aggr_exprs_indices, ) } @@ -450,7 +446,6 @@ impl AggregateExec { input: Arc, input_schema: SchemaRef, schema: SchemaRef, - aggr_expr_indices: Vec, ) -> Result { // Make sure arguments are consistent in size if aggr_expr.len() != filter_expr.len() { @@ -518,7 +513,6 @@ impl AggregateExec { &mode, &input_order_mode, aggr_expr.clone(), - aggr_expr_indices.clone(), ); Ok(AggregateExec { @@ -534,7 +528,6 @@ impl AggregateExec { limit: None, input_order_mode, cache, - aggr_expr_indices, }) } @@ -657,13 +650,15 @@ impl AggregateExec { mode: &AggregateMode, input_order_mode: &InputOrderMode, aggr_exprs: Vec>, - aggr_expr_indices: Vec, ) -> PlanProperties { // Construct equivalence properties: let mut eq_properties = input .equivalence_properties() - .project(group_expr_mapping, schema); + .project(group_expr_mapping, Arc::clone(&schema)); + let schema_len = schema.fields.len(); + let aggr_expr_indices = + ((schema_len - aggr_exprs.len())..schema_len).collect::>(); // if the aggregate function is set monotonic, add it into equivalence properties for (i, aggr_expr) in aggr_exprs.iter().enumerate() { let aggr_expr_index = aggr_expr_indices[i]; @@ -870,7 +865,6 @@ impl ExecutionPlan for AggregateExec { Arc::clone(&children[0]), Arc::clone(&self.input_schema), Arc::clone(&self.schema), - self.aggr_expr_indices.clone(), )?; me.limit = self.limit; @@ -947,8 +941,7 @@ fn create_schema( group_by: &PhysicalGroupBy, aggr_expr: &[Arc], mode: AggregateMode, -) -> Result<(Schema, Vec)> { - let mut aggr_exprs_indices = vec![]; +) -> Result { let mut fields = Vec::with_capacity(group_by.num_output_exprs() + aggr_expr.len()); fields.extend(group_by.output_fields(input_schema)?); @@ -957,7 +950,6 @@ fn create_schema( // in partial mode, the fields of the accumulator's state for expr in aggr_expr { fields.extend(expr.state_fields()?.iter().cloned()); - aggr_exprs_indices.push(fields.len() - 1); } } AggregateMode::Final @@ -966,15 +958,14 @@ fn create_schema( | AggregateMode::SinglePartitioned => { // in final mode, the field with the final result of the accumulator for expr in aggr_expr { - fields.push(expr.field()); - aggr_exprs_indices.push(fields.len() - 1); + fields.extend(expr.state_fields()?.iter().cloned()) } } } - Ok(( - Schema::new_with_metadata(fields, input_schema.metadata().clone()), - aggr_exprs_indices, + Ok(Schema::new_with_metadata( + fields, + input_schema.metadata().clone(), )) } @@ -2810,7 +2801,7 @@ mod tests { vec![false, false], // (a,b) ], ); - let (aggr_schema, _) = create_schema( + let aggr_schema = create_schema( &input_schema, &grouping_set, &aggr_expr, diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 948729bf4312..cc95ce51c15b 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -506,7 +506,7 @@ impl GroupedHashAggregateStream { // Therefore, when we spill these intermediate states or pass them to another // aggregation operator, we must use a schema that includes both the group // columns **and** the partial-state columns. - let (partial_agg_schema, _) = create_schema( + let partial_agg_schema = create_schema( &agg.input().schema(), &agg_group_by, &aggregate_exprs, diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index cf0de68afea4..3785230c0e79 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -344,10 +344,8 @@ pub(crate) fn window_equivalence_properties( .extend(input.equivalence_properties().clone()); let schema_len = schema.fields.len(); - let window_expr_indices = (schema_len..(schema_len - window_exprs.len())) - .rev() - .collect::>(); - + let window_expr_indices = + ((schema_len - window_exprs.len())..schema_len).collect::>(); for (i, expr) in window_exprs.iter().enumerate() { if let Some(udf_window_expr) = expr.as_any().downcast_ref::() { diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 920e12e2fd31..bd3b40089519 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -4963,9 +4963,6 @@ false true NULL -statement ok -set datafusion.optimizer.prefer_existing_sort = true; - # # Add valid distinct case as aggregation plan test # @@ -4992,8 +4989,6 @@ physical_plan 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], has_header=true -statement ok -set datafusion.optimizer.prefer_existing_sort = false; # # Push limit into distinct group-by aggregation tests diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index a2e1b8b559da..f584892e8aa2 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -16,9 +16,6 @@ # specific language governing permissions and limitations # under the License. -statement ok -set datafusion.optimizer.prefer_existing_sort = true; - query TT explain select c_count, @@ -75,7 +72,4 @@ physical_plan 17)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 18)--------------------------CoalesceBatchesExec: target_batch_size=8192 19)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] -20)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false - -statement ok -set datafusion.optimizer.prefer_existing_sort = false; +20)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false \ No newline at end of file From 247d5fe61dfd5105117e7b6728d23cccd74b8e78 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 29 Jan 2025 16:42:33 +0300 Subject: [PATCH 32/62] Update mod.rs --- datafusion/physical-plan/src/aggregates/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 75fb519b3245..a6d4a9b089a3 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -958,7 +958,7 @@ fn create_schema( | AggregateMode::SinglePartitioned => { // in final mode, the field with the final result of the accumulator for expr in aggr_expr { - fields.extend(expr.state_fields()?.iter().cloned()) + fields.push(expr.field()) } } } From 16bdac436134d13c18105cffa2b3ff042ad58bb8 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 29 Jan 2025 17:26:54 +0300 Subject: [PATCH 33/62] Apply suggestions from code review --- datafusion/physical-expr/src/aggregate.rs | 8 +++----- datafusion/physical-expr/src/window/aggregate.rs | 3 +-- datafusion/physical-expr/src/window/standard.rs | 5 +++-- 3 files changed, 7 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 0fe08729828c..6988ce042387 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -42,10 +42,8 @@ use crate::expressions::Column; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow_schema::SortOptions; -use datafusion_common::ScalarValue; -use datafusion_common::{internal_err, not_impl_err, Result}; -use datafusion_expr::ReversedUDAF; -use datafusion_expr::{AggregateExprSetMonotonicity, AggregateUDF}; +use datafusion_common::{internal_err, not_impl_err, ScalarValue, Result}; +use datafusion_expr::{AggregateExprSetMonotonicity, AggregateUDF, ReversedUDAF}; use datafusion_expr_common::accumulator::Accumulator; use datafusion_expr_common::groups_accumulator::GroupsAccumulator; use datafusion_expr_common::type_coercion::aggregates::check_arg_count; @@ -547,7 +545,7 @@ impl AggregateFunctionExpr { self.fun.inner().set_monotonicity(data_type) } - /// Returns PhysicalSortExpr based on monotonicity of the function + /// Returns `PhysicalSortExpr` based on the set monotonicity of the function. pub fn get_result_ordering(&self, aggr_func_idx: usize) -> Option { // If the aggregate expressions are set-monotonic, the output data is // naturally ordered with it per group or partition. diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index d94cdc1490dd..513d6a97cffa 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -32,8 +32,7 @@ use crate::{reverse_order_bys, EquivalenceProperties, PhysicalExpr}; use arrow::array::Array; use arrow::record_batch::RecordBatch; use arrow::{array::ArrayRef, datatypes::Field}; -use datafusion_common::ScalarValue; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::{DataFusionError, ScalarValue, Result}; use datafusion_expr::{Accumulator, WindowFrame}; use datafusion_physical_expr_common::sort_expr::LexOrdering; diff --git a/datafusion/physical-expr/src/window/standard.rs b/datafusion/physical-expr/src/window/standard.rs index 0c0bdf7e99cd..2682a7467ab6 100644 --- a/datafusion/physical-expr/src/window/standard.rs +++ b/datafusion/physical-expr/src/window/standard.rs @@ -65,7 +65,7 @@ impl StandardWindowExpr { &self.expr } - /// Adds any equivalent orderings generated by the `self.expr` to `builder`. + /// Adds any equivalent orderings generated by `self.expr` to `builder`. /// /// If `self.expr` doesn't have an ordering, ordering equivalence properties /// are not updated. Otherwise, ordering equivalence properties are updated @@ -270,7 +270,8 @@ impl WindowExpr for StandardWindowExpr { } } -/// Adds new ordering expression into the existing ordering equivalence class based on partition by information. +/// Adds a new ordering expression into existing ordering equivalence class(es) based on +/// PARTITION BY information (if it exists). pub(crate) fn add_new_ordering_expr_with_partition_by( eqp: &mut EquivalenceProperties, expr: PhysicalSortExpr, From 187533641e62c857fa35b7b13bb748859b89dd4d Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 29 Jan 2025 17:29:10 +0300 Subject: [PATCH 34/62] add slt --- datafusion/physical-expr/src/aggregate.rs | 2 +- .../physical-expr/src/window/aggregate.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 4 +- .../sqllogictest/test_files/aggregate.slt | 94 +++++++++++++++++++ datafusion/sqllogictest/test_files/window.slt | 86 +++++++++++++++++ 5 files changed, 184 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 6988ce042387..5e771421829e 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -42,7 +42,7 @@ use crate::expressions::Column; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow_schema::SortOptions; -use datafusion_common::{internal_err, not_impl_err, ScalarValue, Result}; +use datafusion_common::{internal_err, not_impl_err, Result, ScalarValue}; use datafusion_expr::{AggregateExprSetMonotonicity, AggregateUDF, ReversedUDAF}; use datafusion_expr_common::accumulator::Accumulator; use datafusion_expr_common::groups_accumulator::GroupsAccumulator; diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index 513d6a97cffa..a94d5b1212f5 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -32,7 +32,7 @@ use crate::{reverse_order_bys, EquivalenceProperties, PhysicalExpr}; use arrow::array::Array; use arrow::record_batch::RecordBatch; use arrow::{array::ArrayRef, datatypes::Field}; -use datafusion_common::{DataFusionError, ScalarValue, Result}; +use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_expr::{Accumulator, WindowFrame}; use datafusion_physical_expr_common::sort_expr::LexOrdering; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index a6d4a9b089a3..daa2d1075718 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -512,7 +512,7 @@ impl AggregateExec { &group_expr_mapping, &mode, &input_order_mode, - aggr_expr.clone(), + aggr_expr.as_slice(), ); Ok(AggregateExec { @@ -649,7 +649,7 @@ impl AggregateExec { group_expr_mapping: &ProjectionMapping, mode: &AggregateMode, input_order_mode: &InputOrderMode, - aggr_exprs: Vec>, + aggr_exprs: &[Arc], ) -> PlanProperties { // Construct equivalence properties: let mut eq_properties = input diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index bd3b40089519..dc7d656c47bc 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -6203,3 +6203,97 @@ physical_plan 14)--------------PlaceholderRowExec 15)------------ProjectionExec: expr=[1 as id, 2 as foo] 16)--------------PlaceholderRowExec + + +# Set-Monotonic Aggregate functions can output results in order +statement ok +CREATE EXTERNAL TABLE aggregate_test_100_ordered ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 INT UNSIGNED NOT NULL, + c10 BIGINT UNSIGNED NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL +) +STORED AS CSV +LOCATION '../../testing/data/csv/aggregate_test_100.csv' +WITH ORDER (c1) +OPTIONS ('format.has_header' 'true'); + +statement ok +set datafusion.optimizer.prefer_existing_sort = true; + +query TT +EXPLAIN SELECT c1, SUM(c9) as sum_c9 FROM aggregate_test_100_ordered GROUP BY c1 ORDER BY c1, sum_c9; +---- +logical_plan +01)Sort: aggregate_test_100_ordered.c1 ASC NULLS LAST, sum_c9 ASC NULLS LAST +02)--Projection: aggregate_test_100_ordered.c1, sum(aggregate_test_100_ordered.c9) AS sum_c9 +03)----Aggregate: groupBy=[[aggregate_test_100_ordered.c1]], aggr=[[sum(CAST(aggregate_test_100_ordered.c9 AS UInt64))]] +04)------TableScan: aggregate_test_100_ordered projection=[c1, c9] +physical_plan +01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, sum_c9@1 ASC NULLS LAST] +02)--ProjectionExec: expr=[c1@0 as c1, sum(aggregate_test_100_ordered.c9)@1 as sum_c9] +03)----AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[sum(aggregate_test_100_ordered.c9)], ordering_mode=Sorted +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4, preserve_order=true, sort_exprs=c1@0 ASC NULLS LAST, sum(aggregate_test_100_ordered.c9)@1 ASC NULLS LAST +06)----------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[sum(aggregate_test_100_ordered.c9)], ordering_mode=Sorted +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], output_ordering=[c1@0 ASC NULLS LAST], has_header=true + +query TT +EXPLAIN SELECT SUM(c9) as sum_c9 FROM aggregate_test_100_ordered ORDER BY sum_c9; +---- +logical_plan +01)Sort: sum_c9 ASC NULLS LAST +02)--Projection: sum(aggregate_test_100_ordered.c9) AS sum_c9 +03)----Aggregate: groupBy=[[]], aggr=[[sum(CAST(aggregate_test_100_ordered.c9 AS UInt64))]] +04)------TableScan: aggregate_test_100_ordered projection=[c9] +physical_plan +01)ProjectionExec: expr=[sum(aggregate_test_100_ordered.c9)@0 as sum_c9] +02)--AggregateExec: mode=Final, gby=[], aggr=[sum(aggregate_test_100_ordered.c9)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(aggregate_test_100_ordered.c9)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true + +query TT +EXPLAIN SELECT c1, MIN(c5) as min_c5 FROM aggregate_test_100_ordered GROUP BY c1 ORDER BY c1, min_c5 DESC NULLS LAST; +---- +logical_plan +01)Sort: aggregate_test_100_ordered.c1 ASC NULLS LAST, min_c5 DESC NULLS LAST +02)--Projection: aggregate_test_100_ordered.c1, min(aggregate_test_100_ordered.c5) AS min_c5 +03)----Aggregate: groupBy=[[aggregate_test_100_ordered.c1]], aggr=[[min(aggregate_test_100_ordered.c5)]] +04)------TableScan: aggregate_test_100_ordered projection=[c1, c5] +physical_plan +01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, min_c5@1 DESC NULLS LAST] +02)--ProjectionExec: expr=[c1@0 as c1, min(aggregate_test_100_ordered.c5)@1 as min_c5] +03)----AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[min(aggregate_test_100_ordered.c5)], ordering_mode=Sorted +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4, preserve_order=true, sort_exprs=c1@0 ASC NULLS LAST, min(aggregate_test_100_ordered.c5)@1 DESC NULLS LAST +06)----------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[min(aggregate_test_100_ordered.c5)], ordering_mode=Sorted +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c5], output_ordering=[c1@0 ASC NULLS LAST], has_header=true + +query TT +EXPLAIN SELECT MAX(c5) as max_c5 FROM aggregate_test_100_ordered ORDER BY max_c5; +---- +logical_plan +01)Sort: max_c5 ASC NULLS LAST +02)--Projection: max(aggregate_test_100_ordered.c5) AS max_c5 +03)----Aggregate: groupBy=[[]], aggr=[[max(aggregate_test_100_ordered.c5)]] +04)------TableScan: aggregate_test_100_ordered projection=[c5] +physical_plan +01)ProjectionExec: expr=[max(aggregate_test_100_ordered.c5)@0 as max_c5] +02)--AggregateExec: mode=Final, gby=[], aggr=[max(aggregate_test_100_ordered.c5)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[max(aggregate_test_100_ordered.c5)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], has_header=true diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 1d6174e7c74e..679e0f637f4b 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -5452,3 +5452,89 @@ order by c1, c2, rank1, rank2; statement ok drop table t1; + + +# Set-Monotonic Window Aggregate functions can output results in order +statement ok +CREATE EXTERNAL TABLE aggregate_test_100_ordered ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 INT UNSIGNED NOT NULL, + c10 BIGINT UNSIGNED NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL +) +STORED AS CSV +LOCATION '../../testing/data/csv/aggregate_test_100.csv' +WITH ORDER (c1) +OPTIONS ('format.has_header' 'true'); + +statement ok +set datafusion.optimizer.prefer_existing_sort = true; + +query TT +EXPLAIN SELECT c1, SUM(c9) OVER(PARTITION BY c1) as sum_c9 FROM aggregate_test_100_ordered ORDER BY c1, sum_c9; +---- +logical_plan +01)Sort: aggregate_test_100_ordered.c1 ASC NULLS LAST, sum_c9 ASC NULLS LAST +02)--Projection: aggregate_test_100_ordered.c1, sum(aggregate_test_100_ordered.c9) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS sum_c9 +03)----WindowAggr: windowExpr=[[sum(CAST(aggregate_test_100_ordered.c9 AS UInt64)) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +04)------TableScan: aggregate_test_100_ordered projection=[c1, c9] +physical_plan +01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, sum_c9@1 ASC NULLS LAST] +02)--ProjectionExec: expr=[c1@0 as c1, sum(aggregate_test_100_ordered.c9) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as sum_c9] +03)----WindowAggExec: wdw=[sum(aggregate_test_100_ordered.c9) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100_ordered.c9) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +04)------CoalesceBatchesExec: target_batch_size=1 +05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2, preserve_order=true, sort_exprs=c1@0 ASC NULLS LAST +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], output_ordering=[c1@0 ASC NULLS LAST], has_header=true + +query TT +EXPLAIN SELECT SUM(c9) OVER() as sum_c9 FROM aggregate_test_100_ordered ORDER BY sum_c9; +---- +logical_plan +01)Sort: sum_c9 ASC NULLS LAST +02)--Projection: sum(aggregate_test_100_ordered.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS sum_c9 +03)----WindowAggr: windowExpr=[[sum(CAST(aggregate_test_100_ordered.c9 AS UInt64)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +04)------TableScan: aggregate_test_100_ordered projection=[c9] +physical_plan +01)ProjectionExec: expr=[sum(aggregate_test_100_ordered.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as sum_c9] +02)--WindowAggExec: wdw=[sum(aggregate_test_100_ordered.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100_ordered.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true + +query TT +EXPLAIN SELECT c1, MIN(c5) OVER(PARTITION BY c1) as min_c5 FROM aggregate_test_100_ordered ORDER BY c1, min_c5 DESC NULLS LAST; +---- +logical_plan +01)Sort: aggregate_test_100_ordered.c1 ASC NULLS LAST, min_c5 DESC NULLS LAST +02)--Projection: aggregate_test_100_ordered.c1, min(aggregate_test_100_ordered.c5) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS min_c5 +03)----WindowAggr: windowExpr=[[min(aggregate_test_100_ordered.c5) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +04)------TableScan: aggregate_test_100_ordered projection=[c1, c5] +physical_plan +01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, min_c5@1 DESC NULLS LAST] +02)--ProjectionExec: expr=[c1@0 as c1, min(aggregate_test_100_ordered.c5) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as min_c5] +03)----WindowAggExec: wdw=[min(aggregate_test_100_ordered.c5) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "min(aggregate_test_100_ordered.c5) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +04)------CoalesceBatchesExec: target_batch_size=1 +05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2, preserve_order=true, sort_exprs=c1@0 ASC NULLS LAST +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c5], output_ordering=[c1@0 ASC NULLS LAST], has_header=true + +query TT +EXPLAIN SELECT MAX(c5) OVER() as max_c5 FROM aggregate_test_100_ordered ORDER BY max_c5; +---- +logical_plan +01)Sort: max_c5 ASC NULLS LAST +02)--Projection: max(aggregate_test_100_ordered.c5) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max_c5 +03)----WindowAggr: windowExpr=[[max(aggregate_test_100_ordered.c5) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +04)------TableScan: aggregate_test_100_ordered projection=[c5] +physical_plan +01)ProjectionExec: expr=[max(aggregate_test_100_ordered.c5) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as max_c5] +02)--WindowAggExec: wdw=[max(aggregate_test_100_ordered.c5) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "max(aggregate_test_100_ordered.c5) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], has_header=true From 8464f0ae7fc9e978e3d3eac9f192508363d209ec Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 29 Jan 2025 09:53:57 -0500 Subject: [PATCH 35/62] chore(deps): bump serde_json from 1.0.137 to 1.0.138 in /datafusion-cli (#14351) Bumps [serde_json](https://github.com/serde-rs/json) from 1.0.137 to 1.0.138. - [Release notes](https://github.com/serde-rs/json/releases) - [Commits](https://github.com/serde-rs/json/compare/v1.0.137...v1.0.138) --- updated-dependencies: - dependency-name: serde_json dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion-cli/Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 84354c8c0e9a..fd09e7808807 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -3665,9 +3665,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.137" +version = "1.0.138" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "930cfb6e6abf99298aaad7d29abbef7a9999a9a8806a40088f55f0dcec03146b" +checksum = "d434192e7da787e94a6ea7e9670b26a036d0ca41e0b7efb2676dd32bae872949" dependencies = [ "itoa", "memchr", From 50ac43d6d28264e72aa37585c543ea1197ab1d13 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 29 Jan 2025 09:54:33 -0500 Subject: [PATCH 36/62] chore(deps): bump tempfile from 3.15.0 to 3.16.0 in /datafusion-cli (#14350) Bumps [tempfile](https://github.com/Stebalien/tempfile) from 3.15.0 to 3.16.0. - [Changelog](https://github.com/Stebalien/tempfile/blob/master/CHANGELOG.md) - [Commits](https://github.com/Stebalien/tempfile/compare/v3.15.0...v3.16.0) --- updated-dependencies: - dependency-name: tempfile dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion-cli/Cargo.lock | 66 ++++++++++++++++++++++++++++----------- 1 file changed, 48 insertions(+), 18 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index fd09e7808807..d32f1725f8a6 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -31,7 +31,7 @@ checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" dependencies = [ "cfg-if", "const-random", - "getrandom", + "getrandom 0.2.15", "once_cell", "version_check", "zerocopy", @@ -1085,7 +1085,7 @@ version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" dependencies = [ - "getrandom", + "getrandom 0.2.15", "once_cell", "tiny-keccak", ] @@ -1762,7 +1762,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "33d852cb9b869c2a9b3df2f71a3074817f01e1844f839a144f5fcef059a4eb5d" dependencies = [ "libc", - "windows-sys 0.59.0", + "windows-sys 0.52.0", ] [[package]] @@ -1952,10 +1952,22 @@ dependencies = [ "cfg-if", "js-sys", "libc", - "wasi", + "wasi 0.11.0+wasi-snapshot-preview1", "wasm-bindgen", ] +[[package]] +name = "getrandom" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43a49c392881ce6d5c3b8cb70f98717b7c07aabbdff06687b9030dbfbe2725f8" +dependencies = [ + "cfg-if", + "libc", + "wasi 0.13.3+wasi-0.2.2", + "windows-targets", +] + [[package]] name = "gimli" version = "0.31.1" @@ -2688,7 +2700,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2886843bf800fba2e3377cff24abf6379b4c4d5c6681eaf9ea5b0d15090450bd" dependencies = [ "libc", - "wasi", + "wasi 0.11.0+wasi-snapshot-preview1", "windows-sys 0.52.0", ] @@ -3132,7 +3144,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a2fe5ef3495d7d2e377ff17b1a8ce2ee2ec2a18cde8b6ad6619d65d0701c135d" dependencies = [ "bytes", - "getrandom", + "getrandom 0.2.15", "rand", "ring", "rustc-hash", @@ -3156,7 +3168,7 @@ dependencies = [ "once_cell", "socket2", "tracing", - "windows-sys 0.59.0", + "windows-sys 0.52.0", ] [[package]] @@ -3205,7 +3217,7 @@ version = "0.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" dependencies = [ - "getrandom", + "getrandom 0.2.15", ] [[package]] @@ -3243,7 +3255,7 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dd6f9d3d47bdd2ad6945c5015a226ec6155d0bcdfd8f7cd29f86b71f8de99d2b" dependencies = [ - "getrandom", + "getrandom 0.2.15", "libredox", "thiserror 2.0.11", ] @@ -3343,7 +3355,7 @@ checksum = "c17fa4cb658e3583423e915b9f3acc01cceaee1860e33d59ebae66adc3a2dc0d" dependencies = [ "cc", "cfg-if", - "getrandom", + "getrandom 0.2.15", "libc", "spin", "untrusted", @@ -3417,7 +3429,7 @@ dependencies = [ "errno", "libc", "linux-raw-sys", - "windows-sys 0.59.0", + "windows-sys 0.52.0", ] [[package]] @@ -3814,7 +3826,7 @@ dependencies = [ "cfg-if", "libc", "psm", - "windows-sys 0.59.0", + "windows-sys 0.52.0", ] [[package]] @@ -3887,16 +3899,16 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.15.0" +version = "3.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a8a559c81686f576e8cd0290cd2a24a2a9ad80c98b3478856500fcbd7acd704" +checksum = "38c246215d7d24f48ae091a2902398798e05d978b24315d6efbc00ede9a8bb91" dependencies = [ "cfg-if", "fastrand", - "getrandom", + "getrandom 0.3.1", "once_cell", "rustix", - "windows-sys 0.59.0", + "windows-sys 0.52.0", ] [[package]] @@ -4270,7 +4282,7 @@ version = "1.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "744018581f9a3454a9e15beb8a33b017183f1e7c0cd170232a2d1453b23a51c4" dependencies = [ - "getrandom", + "getrandom 0.2.15", "serde", ] @@ -4320,6 +4332,15 @@ version = "0.11.0+wasi-snapshot-preview1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" +[[package]] +name = "wasi" +version = "0.13.3+wasi-0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26816d2e1a4a36a2940b96c5296ce403917633dff8f3440e9b236ed6f6bacad2" +dependencies = [ + "wit-bindgen-rt", +] + [[package]] name = "wasm-bindgen" version = "0.2.100" @@ -4430,7 +4451,7 @@ version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb" dependencies = [ - "windows-sys 0.59.0", + "windows-sys 0.52.0", ] [[package]] @@ -4563,6 +4584,15 @@ dependencies = [ "memchr", ] +[[package]] +name = "wit-bindgen-rt" +version = "0.33.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3268f3d866458b787f390cf61f4bbb563b922d091359f9608842999eaee3943c" +dependencies = [ + "bitflags 2.8.0", +] + [[package]] name = "write16" version = "1.0.0" From 2a8b885be878ff9ed04331fef1052315fb9b4043 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 29 Jan 2025 10:03:20 -0500 Subject: [PATCH 37/62] Update version in `datafusion-cli/Dockerfile` to 1.81 (#14344) * Update version in dockerfile * Update datafusion-cli/Dockerfile Co-authored-by: Oleks V --------- Co-authored-by: Oleks V --- datafusion-cli/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion-cli/Dockerfile b/datafusion-cli/Dockerfile index faf345660dbe..f73b76b5dbdf 100644 --- a/datafusion-cli/Dockerfile +++ b/datafusion-cli/Dockerfile @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. -FROM rust:1.80-bookworm AS builder +FROM rust:bookworm AS builder COPY . /usr/src/datafusion COPY ./datafusion /usr/src/datafusion/datafusion From 62000b4e169fac85cd71715b59e0e796b8e694ec Mon Sep 17 00:00:00 2001 From: Raz Luvaton <16746759+rluvaton@users.noreply.github.com> Date: Wed, 29 Jan 2025 17:06:37 +0200 Subject: [PATCH 38/62] perf(array-agg): add fast path for array agg for `merge_batch` (#14299) * perf(array-agg): add fast path for array agg for `merge_batch` * update comment * fix slice length * fix: make sure we are not inserting empty lists --- .../functions-aggregate/src/array_agg.rs | 79 ++++++++++++++++++- 1 file changed, 76 insertions(+), 3 deletions(-) diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index b75de83f6ace..9fff05999122 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -17,7 +17,7 @@ //! `ARRAY_AGG` aggregate implementation: [`ArrayAgg`] -use arrow::array::{new_empty_array, Array, ArrayRef, AsArray, StructArray}; +use arrow::array::{new_empty_array, Array, ArrayRef, AsArray, ListArray, StructArray}; use arrow::datatypes::DataType; use arrow_schema::{Field, Fields}; @@ -177,6 +177,67 @@ impl ArrayAggAccumulator { datatype: datatype.clone(), }) } + + /// This function will return the underlying list array values if all valid values are consecutive without gaps (i.e. no null value point to a non empty list) + /// If there are gaps but only in the end of the list array, the function will return the values without the null values in the end + fn get_optional_values_to_merge_as_is(list_array: &ListArray) -> Option { + let offsets = list_array.value_offsets(); + // Offsets always have at least 1 value + let initial_offset = offsets[0]; + let null_count = list_array.null_count(); + + // If no nulls than just use the fast path + // This is ok as the state is a ListArray rather than a ListViewArray so all the values are consecutive + if null_count == 0 { + // According to Arrow specification, the first offset can be non-zero + let list_values = list_array.values().slice( + initial_offset as usize, + (offsets[offsets.len() - 1] - initial_offset) as usize, + ); + return Some(list_values); + } + + // If all the values are null than just return an empty values array + if list_array.null_count() == list_array.len() { + return Some(list_array.values().slice(0, 0)); + } + + // According to the Arrow spec, null values can point to non empty lists + // So this will check if all null values starting from the first valid value to the last one point to a 0 length list so we can just slice the underlying value + + // Unwrapping is safe as we just checked if there is a null value + let nulls = list_array.nulls().unwrap(); + + let mut valid_slices_iter = nulls.valid_slices(); + + // This is safe as we validated that that are at least 1 valid value in the array + let (start, end) = valid_slices_iter.next().unwrap(); + + let start_offset = offsets[start]; + + // End is exclusive, so it already point to the last offset value + // This is valid as the length of the array is always 1 less than the length of the offsets + let mut end_offset_of_last_valid_value = offsets[end]; + + for (start, end) in valid_slices_iter { + // If there is a null value that point to a non empty list than the start offset of the valid value + // will be different that the end offset of the last valid value + if offsets[start] != end_offset_of_last_valid_value { + return None; + } + + // End is exclusive, so it already point to the last offset value + // This is valid as the length of the array is always 1 less than the length of the offsets + end_offset_of_last_valid_value = offsets[end]; + } + + let consecutive_valid_values = list_array.values().slice( + start_offset as usize, + (end_offset_of_last_valid_value - start_offset) as usize, + ); + + Some(consecutive_valid_values) + } } impl Accumulator for ArrayAggAccumulator { @@ -208,9 +269,21 @@ impl Accumulator for ArrayAggAccumulator { } let list_arr = as_list_array(&states[0])?; - for arr in list_arr.iter().flatten() { - self.values.push(arr); + + match Self::get_optional_values_to_merge_as_is(list_arr) { + Some(values) => { + // Make sure we don't insert empty lists + if values.len() > 0 { + self.values.push(values); + } + } + None => { + for arr in list_arr.iter().flatten() { + self.values.push(arr); + } + } } + Ok(()) } From 1da5252c3ccee4f0bd2384444aefd636fd02cebb Mon Sep 17 00:00:00 2001 From: logan-keede <68557630+logan-keede@users.noreply.github.com> Date: Wed, 29 Jan 2025 21:39:01 +0530 Subject: [PATCH 39/62] moving memory.rs out of datafusion/core (#14332) * moving memory.rs out of datafusion/core * fix: forgotten license * fix: taplo + documentation * fix: cargo fmt * Cargo.toml update, removing unnecessary dependency --- datafusion-cli/Cargo.lock | 3 +- datafusion/catalog/Cargo.toml | 2 + datafusion/catalog/src/lib.rs | 10 ++ .../catalog_common => catalog/src}/memory.rs | 157 +--------------- datafusion/core/Cargo.toml | 1 - datafusion/core/src/catalog_common/mod.rs | 6 - datafusion/core/src/execution/context/mod.rs | 4 +- .../core/src/execution/session_state.rs | 4 +- .../src/execution/session_state_defaults.rs | 2 +- datafusion/core/tests/catalog/memory.rs | 170 ++++++++++++++++++ datafusion/core/tests/catalog/mod.rs | 18 ++ datafusion/core/tests/core_integration.rs | 2 + datafusion/sqllogictest/Cargo.toml | 1 + datafusion/sqllogictest/src/test_context.rs | 4 +- 14 files changed, 213 insertions(+), 171 deletions(-) rename datafusion/{core/src/catalog_common => catalog/src}/memory.rs (51%) create mode 100644 datafusion/core/tests/catalog/memory.rs create mode 100644 datafusion/core/tests/catalog/mod.rs diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index d32f1725f8a6..b47e5df97d0a 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1236,7 +1236,6 @@ dependencies = [ "bytes", "bzip2 0.5.0", "chrono", - "dashmap", "datafusion-catalog", "datafusion-common", "datafusion-common-runtime", @@ -1280,10 +1279,12 @@ version = "44.0.0" dependencies = [ "arrow-schema", "async-trait", + "dashmap", "datafusion-common", "datafusion-execution", "datafusion-expr", "datafusion-physical-plan", + "itertools 0.14.0", "parking_lot", ] diff --git a/datafusion/catalog/Cargo.toml b/datafusion/catalog/Cargo.toml index 32a87cc7611c..bcc258c7a7f4 100644 --- a/datafusion/catalog/Cargo.toml +++ b/datafusion/catalog/Cargo.toml @@ -30,10 +30,12 @@ version.workspace = true [dependencies] arrow-schema = { workspace = true } async-trait = { workspace = true } +dashmap = { workspace = true } datafusion-common = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } datafusion-physical-plan = { workspace = true } +itertools = { workspace = true } parking_lot = { workspace = true } [dev-dependencies] diff --git a/datafusion/catalog/src/lib.rs b/datafusion/catalog/src/lib.rs index 3cf2a3b3cd33..28410eb76fab 100644 --- a/datafusion/catalog/src/lib.rs +++ b/datafusion/catalog/src/lib.rs @@ -15,6 +15,16 @@ // specific language governing permissions and limitations // under the License. +//! Interfaces and default implementations of catalogs and schemas. +//! +//! Implementations +//! * Simple memory based catalog: [`MemoryCatalogProviderList`], [`MemoryCatalogProvider`], [`MemorySchemaProvider`] + +pub mod memory; +pub use memory::{ + MemoryCatalogProvider, MemoryCatalogProviderList, MemorySchemaProvider, +}; + mod r#async; mod catalog; mod dynamic_file; diff --git a/datafusion/core/src/catalog_common/memory.rs b/datafusion/catalog/src/memory.rs similarity index 51% rename from datafusion/core/src/catalog_common/memory.rs rename to datafusion/catalog/src/memory.rs index 6cdefc31f18c..d22a98d3d064 100644 --- a/datafusion/core/src/catalog_common/memory.rs +++ b/datafusion/catalog/src/memory.rs @@ -18,9 +18,7 @@ //! [`MemoryCatalogProvider`], [`MemoryCatalogProviderList`]: In-memory //! implementations of [`CatalogProviderList`] and [`CatalogProvider`]. -use crate::catalog::{ - CatalogProvider, CatalogProviderList, SchemaProvider, TableProvider, -}; +use crate::{CatalogProvider, CatalogProviderList, SchemaProvider, TableProvider}; use async_trait::async_trait; use dashmap::DashMap; use datafusion_common::{exec_err, DataFusionError}; @@ -200,156 +198,3 @@ impl SchemaProvider for MemorySchemaProvider { self.tables.contains_key(name) } } - -#[cfg(test)] -mod test { - use super::*; - use crate::catalog::CatalogProvider; - use crate::catalog_common::memory::MemorySchemaProvider; - use crate::datasource::empty::EmptyTable; - use crate::datasource::listing::{ListingTable, ListingTableConfig, ListingTableUrl}; - use crate::prelude::SessionContext; - use arrow_schema::Schema; - use datafusion_common::assert_batches_eq; - use std::any::Any; - use std::sync::Arc; - - #[test] - fn memory_catalog_dereg_nonempty_schema() { - let cat = Arc::new(MemoryCatalogProvider::new()) as Arc; - - let schema = Arc::new(MemorySchemaProvider::new()) as Arc; - let test_table = Arc::new(EmptyTable::new(Arc::new(Schema::empty()))) - as Arc; - schema.register_table("t".into(), test_table).unwrap(); - - cat.register_schema("foo", schema.clone()).unwrap(); - - assert!( - cat.deregister_schema("foo", false).is_err(), - "dropping empty schema without cascade should error" - ); - assert!(cat.deregister_schema("foo", true).unwrap().is_some()); - } - - #[test] - fn memory_catalog_dereg_empty_schema() { - let cat = Arc::new(MemoryCatalogProvider::new()) as Arc; - - let schema = Arc::new(MemorySchemaProvider::new()) as Arc; - cat.register_schema("foo", schema).unwrap(); - - assert!(cat.deregister_schema("foo", false).unwrap().is_some()); - } - - #[test] - fn memory_catalog_dereg_missing() { - let cat = Arc::new(MemoryCatalogProvider::new()) as Arc; - assert!(cat.deregister_schema("foo", false).unwrap().is_none()); - } - - #[test] - fn default_register_schema_not_supported() { - // mimic a new CatalogProvider and ensure it does not support registering schemas - #[derive(Debug)] - struct TestProvider {} - impl CatalogProvider for TestProvider { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema_names(&self) -> Vec { - unimplemented!() - } - - fn schema(&self, _name: &str) -> Option> { - unimplemented!() - } - } - - let schema = Arc::new(MemorySchemaProvider::new()) as Arc; - let catalog = Arc::new(TestProvider {}); - - match catalog.register_schema("foo", schema) { - Ok(_) => panic!("unexpected OK"), - Err(e) => assert_eq!(e.strip_backtrace(), "This feature is not implemented: Registering new schemas is not supported"), - }; - } - - #[tokio::test] - async fn test_mem_provider() { - let provider = MemorySchemaProvider::new(); - let table_name = "test_table_exist"; - assert!(!provider.table_exist(table_name)); - assert!(provider.deregister_table(table_name).unwrap().is_none()); - let test_table = EmptyTable::new(Arc::new(Schema::empty())); - // register table successfully - assert!(provider - .register_table(table_name.to_string(), Arc::new(test_table)) - .unwrap() - .is_none()); - assert!(provider.table_exist(table_name)); - let other_table = EmptyTable::new(Arc::new(Schema::empty())); - let result = - provider.register_table(table_name.to_string(), Arc::new(other_table)); - assert!(result.is_err()); - } - - #[tokio::test] - async fn test_schema_register_listing_table() { - let testdata = crate::test_util::parquet_test_data(); - let testdir = if testdata.starts_with('/') { - format!("file://{testdata}") - } else { - format!("file:///{testdata}") - }; - let filename = if testdir.ends_with('/') { - format!("{}{}", testdir, "alltypes_plain.parquet") - } else { - format!("{}/{}", testdir, "alltypes_plain.parquet") - }; - - let table_path = ListingTableUrl::parse(filename).unwrap(); - - let catalog = MemoryCatalogProvider::new(); - let schema = MemorySchemaProvider::new(); - - let ctx = SessionContext::new(); - - let config = ListingTableConfig::new(table_path) - .infer(&ctx.state()) - .await - .unwrap(); - let table = ListingTable::try_new(config).unwrap(); - - schema - .register_table("alltypes_plain".to_string(), Arc::new(table)) - .unwrap(); - - catalog.register_schema("active", Arc::new(schema)).unwrap(); - ctx.register_catalog("cat", Arc::new(catalog)); - - let df = ctx - .sql("SELECT id, bool_col FROM cat.active.alltypes_plain") - .await - .unwrap(); - - let actual = df.collect().await.unwrap(); - - let expected = [ - "+----+----------+", - "| id | bool_col |", - "+----+----------+", - "| 4 | true |", - "| 5 | false |", - "| 6 | true |", - "| 7 | false |", - "| 2 | true |", - "| 3 | false |", - "| 0 | true |", - "| 1 | false |", - "+----+----------+", - ]; - assert_batches_eq!(expected, &actual); - } -} diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 149bf8beb96e..c9b059ad0f40 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -99,7 +99,6 @@ async-trait = { workspace = true } bytes = { workspace = true } bzip2 = { version = "0.5.0", optional = true } chrono = { workspace = true } -dashmap = { workspace = true } datafusion-catalog = { workspace = true } datafusion-common = { workspace = true, features = ["object_store"] } datafusion-common-runtime = { workspace = true } diff --git a/datafusion/core/src/catalog_common/mod.rs b/datafusion/core/src/catalog_common/mod.rs index 68c78dda4899..45fb6ddae1d6 100644 --- a/datafusion/core/src/catalog_common/mod.rs +++ b/datafusion/core/src/catalog_common/mod.rs @@ -18,18 +18,12 @@ //! Interfaces and default implementations of catalogs and schemas. //! //! Implementations -//! * Simple memory based catalog: [`MemoryCatalogProviderList`], [`MemoryCatalogProvider`], [`MemorySchemaProvider`] //! * Information schema: [`information_schema`] //! * Listing schema: [`listing_schema`] pub mod information_schema; pub mod listing_schema; -pub mod memory; - pub use crate::catalog::{CatalogProvider, CatalogProviderList, SchemaProvider}; -pub use memory::{ - MemoryCatalogProvider, MemoryCatalogProviderList, MemorySchemaProvider, -}; pub use datafusion_sql::{ResolvedTableReference, TableReference}; diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index e377dd6297ce..a0aa6447871c 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -17,6 +17,8 @@ //! [`SessionContext`] API for registering data sources and executing queries +use datafusion_catalog::memory::MemorySchemaProvider; +use datafusion_catalog::MemoryCatalogProvider; use std::collections::HashSet; use std::fmt::Debug; use std::sync::{Arc, Weak}; @@ -27,8 +29,6 @@ use crate::{ CatalogProvider, CatalogProviderList, TableProvider, TableProviderFactory, }, catalog_common::listing_schema::ListingSchemaProvider, - catalog_common::memory::MemorySchemaProvider, - catalog_common::MemoryCatalogProvider, dataframe::DataFrame, datasource::listing::{ ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 54d505e1b4b9..6c3349625f04 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -27,13 +27,13 @@ use crate::catalog::{CatalogProviderList, SchemaProvider, TableProviderFactory}; use crate::catalog_common::information_schema::{ InformationSchemaProvider, INFORMATION_SCHEMA, }; -use crate::catalog_common::MemoryCatalogProviderList; use crate::datasource::cte_worktable::CteWorkTable; use crate::datasource::file_format::{format_as_file_type, FileFormatFactory}; use crate::datasource::provider_as_source; use crate::execution::context::{EmptySerializerRegistry, FunctionFactory, QueryPlanner}; use crate::execution::SessionStateDefaults; use crate::physical_planner::{DefaultPhysicalPlanner, PhysicalPlanner}; +use datafusion_catalog::MemoryCatalogProviderList; use arrow_schema::{DataType, SchemaRef}; use datafusion_catalog::{Session, TableFunction, TableFunctionImpl}; @@ -1987,11 +1987,11 @@ pub(crate) struct PreparedPlan { #[cfg(test)] mod tests { use super::{SessionContextProvider, SessionStateBuilder}; - use crate::catalog_common::MemoryCatalogProviderList; use crate::datasource::MemTable; use crate::execution::context::SessionState; use arrow_array::{ArrayRef, Int32Array, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema}; + use datafusion_catalog::MemoryCatalogProviderList; use datafusion_common::DFSchema; use datafusion_common::Result; use datafusion_execution::config::SessionConfig; diff --git a/datafusion/core/src/execution/session_state_defaults.rs b/datafusion/core/src/execution/session_state_defaults.rs index 106082bc7b3b..92f649781cfd 100644 --- a/datafusion/core/src/execution/session_state_defaults.rs +++ b/datafusion/core/src/execution/session_state_defaults.rs @@ -17,7 +17,6 @@ use crate::catalog::{CatalogProvider, TableProviderFactory}; use crate::catalog_common::listing_schema::ListingSchemaProvider; -use crate::catalog_common::{MemoryCatalogProvider, MemorySchemaProvider}; use crate::datasource::file_format::arrow::ArrowFormatFactory; use crate::datasource::file_format::avro::AvroFormatFactory; use crate::datasource::file_format::csv::CsvFormatFactory; @@ -31,6 +30,7 @@ use crate::execution::context::SessionState; use crate::functions_nested; use crate::{functions, functions_aggregate, functions_table, functions_window}; use datafusion_catalog::TableFunction; +use datafusion_catalog::{MemoryCatalogProvider, MemorySchemaProvider}; use datafusion_execution::config::SessionConfig; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_execution::runtime_env::RuntimeEnv; diff --git a/datafusion/core/tests/catalog/memory.rs b/datafusion/core/tests/catalog/memory.rs new file mode 100644 index 000000000000..bef23fff3e96 --- /dev/null +++ b/datafusion/core/tests/catalog/memory.rs @@ -0,0 +1,170 @@ +// 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. + +use arrow_schema::Schema; +use datafusion::catalog::CatalogProvider; +use datafusion::datasource::empty::EmptyTable; +use datafusion::datasource::listing::{ + ListingTable, ListingTableConfig, ListingTableUrl, +}; +use datafusion::prelude::SessionContext; +use datafusion_catalog::memory::*; +use datafusion_catalog::{SchemaProvider, TableProvider}; +use datafusion_common::assert_batches_eq; +use std::any::Any; +use std::sync::Arc; + +#[test] +fn memory_catalog_dereg_nonempty_schema() { + let cat = Arc::new(MemoryCatalogProvider::new()) as Arc; + + let schema = Arc::new(MemorySchemaProvider::new()) as Arc; + let test_table = + Arc::new(EmptyTable::new(Arc::new(Schema::empty()))) as Arc; + schema.register_table("t".into(), test_table).unwrap(); + + cat.register_schema("foo", schema.clone()).unwrap(); + + assert!( + cat.deregister_schema("foo", false).is_err(), + "dropping empty schema without cascade should error" + ); + assert!(cat.deregister_schema("foo", true).unwrap().is_some()); +} + +#[test] +fn memory_catalog_dereg_empty_schema() { + let cat = Arc::new(MemoryCatalogProvider::new()) as Arc; + + let schema = Arc::new(MemorySchemaProvider::new()) as Arc; + cat.register_schema("foo", schema).unwrap(); + + assert!(cat.deregister_schema("foo", false).unwrap().is_some()); +} + +#[test] +fn memory_catalog_dereg_missing() { + let cat = Arc::new(MemoryCatalogProvider::new()) as Arc; + assert!(cat.deregister_schema("foo", false).unwrap().is_none()); +} + +#[test] +fn default_register_schema_not_supported() { + // mimic a new CatalogProvider and ensure it does not support registering schemas + #[derive(Debug)] + struct TestProvider {} + impl CatalogProvider for TestProvider { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema_names(&self) -> Vec { + unimplemented!() + } + + fn schema(&self, _name: &str) -> Option> { + unimplemented!() + } + } + + let schema = Arc::new(MemorySchemaProvider::new()) as Arc; + let catalog = Arc::new(TestProvider {}); + + match catalog.register_schema("foo", schema) { + Ok(_) => panic!("unexpected OK"), + Err(e) => assert_eq!( + e.strip_backtrace(), + "This feature is not implemented: Registering new schemas is not supported" + ), + }; +} + +#[tokio::test] +async fn test_mem_provider() { + let provider = MemorySchemaProvider::new(); + let table_name = "test_table_exist"; + assert!(!provider.table_exist(table_name)); + assert!(provider.deregister_table(table_name).unwrap().is_none()); + let test_table = EmptyTable::new(Arc::new(Schema::empty())); + // register table successfully + assert!(provider + .register_table(table_name.to_string(), Arc::new(test_table)) + .unwrap() + .is_none()); + assert!(provider.table_exist(table_name)); + let other_table = EmptyTable::new(Arc::new(Schema::empty())); + let result = provider.register_table(table_name.to_string(), Arc::new(other_table)); + assert!(result.is_err()); +} + +#[tokio::test] +async fn test_schema_register_listing_table() { + let testdata = datafusion::test_util::parquet_test_data(); + let testdir = if testdata.starts_with('/') { + format!("file://{testdata}") + } else { + format!("file:///{testdata}") + }; + let filename = if testdir.ends_with('/') { + format!("{}{}", testdir, "alltypes_plain.parquet") + } else { + format!("{}/{}", testdir, "alltypes_plain.parquet") + }; + + let table_path = ListingTableUrl::parse(filename).unwrap(); + + let catalog = MemoryCatalogProvider::new(); + let schema = MemorySchemaProvider::new(); + + let ctx = SessionContext::new(); + + let config = ListingTableConfig::new(table_path) + .infer(&ctx.state()) + .await + .unwrap(); + let table = ListingTable::try_new(config).unwrap(); + + schema + .register_table("alltypes_plain".to_string(), Arc::new(table)) + .unwrap(); + + catalog.register_schema("active", Arc::new(schema)).unwrap(); + ctx.register_catalog("cat", Arc::new(catalog)); + + let df = ctx + .sql("SELECT id, bool_col FROM cat.active.alltypes_plain") + .await + .unwrap(); + + let actual = df.collect().await.unwrap(); + + let expected = [ + "+----+----------+", + "| id | bool_col |", + "+----+----------+", + "| 4 | true |", + "| 5 | false |", + "| 6 | true |", + "| 7 | false |", + "| 2 | true |", + "| 3 | false |", + "| 0 | true |", + "| 1 | false |", + "+----+----------+", + ]; + assert_batches_eq!(expected, &actual); +} diff --git a/datafusion/core/tests/catalog/mod.rs b/datafusion/core/tests/catalog/mod.rs new file mode 100644 index 000000000000..e86118afaecf --- /dev/null +++ b/datafusion/core/tests/catalog/mod.rs @@ -0,0 +1,18 @@ +// 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. + +mod memory; diff --git a/datafusion/core/tests/core_integration.rs b/datafusion/core/tests/core_integration.rs index 93f66282333d..66b4103160e7 100644 --- a/datafusion/core/tests/core_integration.rs +++ b/datafusion/core/tests/core_integration.rs @@ -44,6 +44,8 @@ mod optimizer; mod physical_optimizer; +mod catalog; + #[cfg(test)] #[ctor::ctor] fn init() { diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index 38fe14ab90b7..41b96e341074 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -42,6 +42,7 @@ bytes = { workspace = true, optional = true } chrono = { workspace = true, optional = true } clap = { version = "4.5.16", features = ["derive", "env"] } datafusion = { workspace = true, default-features = true, features = ["avro"] } +datafusion-catalog = { workspace = true, default-features = true } datafusion-common = { workspace = true, default-features = true } datafusion-common-runtime = { workspace = true, default-features = true } futures = { workspace = true } diff --git a/datafusion/sqllogictest/src/test_context.rs b/datafusion/sqllogictest/src/test_context.rs index 2466303c32a9..f7c9346a8983 100644 --- a/datafusion/sqllogictest/src/test_context.rs +++ b/datafusion/sqllogictest/src/test_context.rs @@ -31,11 +31,11 @@ use datafusion::logical_expr::{create_udf, ColumnarValue, Expr, ScalarUDF, Volat use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionConfig; use datafusion::{ - catalog::CatalogProvider, - catalog_common::{memory::MemoryCatalogProvider, memory::MemorySchemaProvider}, datasource::{MemTable, TableProvider, TableType}, prelude::{CsvReadOptions, SessionContext}, }; +use datafusion_catalog::CatalogProvider; +use datafusion_catalog::{memory::MemoryCatalogProvider, memory::MemorySchemaProvider}; use datafusion_common::cast::as_float64_array; use datafusion_common::DataFusionError; From d18a1d378e549505efccd9c02723e60423bd04f4 Mon Sep 17 00:00:00 2001 From: Ian Lai <108986288+Chen-Yuan-Lai@users.noreply.github.com> Date: Thu, 30 Jan 2025 00:09:58 +0800 Subject: [PATCH 40/62] refactor: switch `BooleanBufferBuilder` to `NullBufferBuilder` in binary_map (#14341) * refactor: switch BooleanBufferBuilder to NullBufferBuilder in binary_map * fix: check cargo.lock for datafusion-cli * add comment for .unwrap --------- Co-authored-by: Cheng-Yuan-Lai --- datafusion-cli/Cargo.lock | 73 ++++++++++--------- datafusion/physical-expr-common/Cargo.toml | 1 + .../physical-expr-common/src/binary_map.rs | 16 ++-- 3 files changed, 47 insertions(+), 43 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index b47e5df97d0a..9c0e209b6fbd 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -863,9 +863,9 @@ dependencies = [ [[package]] name = "brotli-decompressor" -version = "4.0.1" +version = "4.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a45bd2e4095a8b518033b128020dd4a55aab1c0a381ba4404a472630f4bc362" +checksum = "74fa05ad7d803d413eb8380983b092cbbaf9a85f151b871360e7b00cd7060b37" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", @@ -943,9 +943,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.2.9" +version = "1.2.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8293772165d9345bdaaa39b45b2109591e63fe5e6fbc23c6ff930a048aa310b" +checksum = "13208fcbb66eaeffe09b99fffbe1af420f00a7b35aa99ad683dfc1aa76145229" dependencies = [ "jobserver", "libc", @@ -985,9 +985,9 @@ dependencies = [ [[package]] name = "chrono-tz" -version = "0.10.0" +version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd6dd8046d00723a59a2f8c5f295c515b9bb9a331ee4f8f3d4dd49e428acd3b6" +checksum = "9c6ac4f2c0bf0f44e9161aec9675e1050aa4a530663c4a9e37e108fa948bca9f" dependencies = [ "chrono", "chrono-tz-build", @@ -1133,9 +1133,9 @@ dependencies = [ [[package]] name = "cpufeatures" -version = "0.2.16" +version = "0.2.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16b80225097f2e5ae4e7179dd2266824648f3e2f49d9134d584b76389d31c4c3" +checksum = "59ed5838eebb26a2bb2e58f6d5b5316989ae9d08bab10e0e6d103e656d1b0280" dependencies = [ "libc", ] @@ -1157,9 +1157,9 @@ checksum = "d0a5c400df2834b80a4c3327b3aad3a4c4cd4de0629063962b03235697506a28" [[package]] name = "crunchy" -version = "0.2.2" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" +checksum = "43da5946c66ffcc7745f48db692ffbb10a83bfe0afd96235c5c2a4fb23994929" [[package]] name = "crypto-common" @@ -1578,6 +1578,7 @@ version = "44.0.0" dependencies = [ "ahash", "arrow", + "arrow-buffer", "datafusion-common", "datafusion-expr-common", "hashbrown 0.14.5", @@ -2134,9 +2135,9 @@ dependencies = [ [[package]] name = "httparse" -version = "1.9.5" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d71d3574edd2771538b901e6549113b4006ece66150fb69c0fb6d9a2adae946" +checksum = "f2d708df4e7140240a16cd6ab0ab65c972d7433ab77819ea693fde9c43811e2a" [[package]] name = "httpdate" @@ -2176,9 +2177,9 @@ dependencies = [ [[package]] name = "hyper" -version = "1.5.2" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "256fb8d4bd6413123cc9d91832d78325c48ff41677595be797d90f42969beae0" +checksum = "cc2b571658e38e0c01b1fdca3bbbe93c00d3d71693ff2770043f8c29bc7d6f80" dependencies = [ "bytes", "futures-channel", @@ -2218,7 +2219,7 @@ checksum = "2d191583f3da1305256f22463b9bb0471acad48a4e534a5218b9963e9c1f59b2" dependencies = [ "futures-util", "http 1.2.0", - "hyper 1.5.2", + "hyper 1.6.0", "hyper-util", "rustls 0.23.21", "rustls-native-certs 0.8.1", @@ -2239,7 +2240,7 @@ dependencies = [ "futures-util", "http 1.2.0", "http-body 1.0.1", - "hyper 1.5.2", + "hyper 1.6.0", "pin-project-lite", "socket2", "tokio", @@ -2427,9 +2428,9 @@ checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" [[package]] name = "ipnet" -version = "2.10.1" +version = "2.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ddc24109865250148c2e0f3d25d4f0f479571723792d3802153c60922a4fb708" +checksum = "469fb0b9cefa57e3ef31275ee7cacb78f2fdca44e4765491884a2b119d4eb130" [[package]] name = "is_terminal_polyfill" @@ -2834,7 +2835,7 @@ dependencies = [ "chrono", "futures", "humantime", - "hyper 1.5.2", + "hyper 1.6.0", "itertools 0.13.0", "md-5", "parking_lot", @@ -2861,9 +2862,9 @@ checksum = "1261fe7e33c73b354eab43b1273a57c8f967d0391e80353e51f764ac02cf6775" [[package]] name = "openssl-probe" -version = "0.1.5" +version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" +checksum = "d05e27ee213611ffe7d6348b942e8f942b37114c00cc03cec254295a4a17852e" [[package]] name = "option-ext" @@ -3316,7 +3317,7 @@ dependencies = [ "http 1.2.0", "http-body 1.0.1", "http-body-util", - "hyper 1.5.2", + "hyper 1.6.0", "hyper-rustls 0.27.5", "hyper-util", "ipnet", @@ -3422,9 +3423,9 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.43" +version = "0.38.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a78891ee6bf2340288408954ac787aa063d8e8817e9f53abb37c695c6d834ef6" +checksum = "fdb5bc1ae2baa591800df16c9ca78619bf65c0488b41b96ccec5d11220d8c154" dependencies = [ "bitflags 2.8.0", "errno", @@ -3503,9 +3504,9 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.10.1" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2bf47e6ff922db3825eb750c4e2ff784c6ff8fb9e13046ef6a1d1c5401b0b37" +checksum = "917ce264624a4b4db1c364dcc35bfca9ded014d0a958cd47ad3e960e988ea51c" dependencies = [ "web-time", ] @@ -3561,9 +3562,9 @@ dependencies = [ [[package]] name = "ryu" -version = "1.0.18" +version = "1.0.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3cb5ba0dc43242ce17de99c180e96db90b235b8a9fdc9543c96d2209116bd9f" +checksum = "6ea1a2d0a644769cc99faa24c3ad26b379b786fe7c36fd3c546254801650e6dd" [[package]] name = "same-file" @@ -3637,9 +3638,9 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.24" +version = "1.0.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3cb6eb87a131f756572d7fb904f6e7b68633f09cca868c5df1c4b8d1a694bbba" +checksum = "f79dfe2d285b0488816f30e700a7438c5a73d816b5b7d3ac72fbc48b0d185e03" [[package]] name = "seq-macro" @@ -4214,9 +4215,9 @@ checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" [[package]] name = "unicode-ident" -version = "1.0.14" +version = "1.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "adb9e6ca4f869e1180728b7950e35922a7fc6397f7b641499e8f3ef06e50dc83" +checksum = "a210d160f08b701c8721ba1c726c11662f877ea6b7094007e1ca9a1041945034" [[package]] name = "unicode-segmentation" @@ -4279,9 +4280,9 @@ checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" [[package]] name = "uuid" -version = "1.12.0" +version = "1.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "744018581f9a3454a9e15beb8a33b017183f1e7c0cd170232a2d1453b23a51c4" +checksum = "b3758f5e68192bb96cc8f9b7e2c2cfdabb435499a28499a42f8f984092adad4b" dependencies = [ "getrandom 0.2.15", "serde", @@ -4578,9 +4579,9 @@ checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" [[package]] name = "winnow" -version = "0.6.24" +version = "0.6.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8d71a593cc5c42ad7876e2c1fda56f314f3754c084128833e64f1345ff8a03a" +checksum = "ad699df48212c6cc6eb4435f35500ac6fd3b9913324f938aea302022ce19d310" dependencies = [ "memchr", ] diff --git a/datafusion/physical-expr-common/Cargo.toml b/datafusion/physical-expr-common/Cargo.toml index 14d6ca64d15e..00ddb11cc42d 100644 --- a/datafusion/physical-expr-common/Cargo.toml +++ b/datafusion/physical-expr-common/Cargo.toml @@ -38,6 +38,7 @@ path = "src/lib.rs" [dependencies] ahash = { workspace = true } arrow = { workspace = true } +arrow-buffer = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-expr-common = { workspace = true } hashbrown = { workspace = true } diff --git a/datafusion/physical-expr-common/src/binary_map.rs b/datafusion/physical-expr-common/src/binary_map.rs index 8febbdd5b1f9..bdff494518da 100644 --- a/datafusion/physical-expr-common/src/binary_map.rs +++ b/datafusion/physical-expr-common/src/binary_map.rs @@ -22,11 +22,11 @@ use ahash::RandomState; use arrow::array::cast::AsArray; use arrow::array::types::{ByteArrayType, GenericBinaryType, GenericStringType}; use arrow::array::{ - Array, ArrayRef, BooleanBufferBuilder, BufferBuilder, GenericBinaryArray, - GenericStringArray, OffsetSizeTrait, + Array, ArrayRef, BufferBuilder, GenericBinaryArray, GenericStringArray, + OffsetSizeTrait, }; -use arrow::buffer::{NullBuffer, OffsetBuffer, ScalarBuffer}; use arrow::datatypes::DataType; +use arrow_buffer::{NullBuffer, NullBufferBuilder, OffsetBuffer, ScalarBuffer}; use datafusion_common::hash_utils::create_hashes; use datafusion_common::utils::proxy::{HashTableAllocExt, VecAllocExt}; use std::any::type_name; @@ -553,10 +553,12 @@ where /// Returns a `NullBuffer` with a single null value at the given index fn single_null_buffer(num_values: usize, null_index: usize) -> NullBuffer { - let mut bool_builder = BooleanBufferBuilder::new(num_values); - bool_builder.append_n(num_values, true); - bool_builder.set_bit(null_index, false); - NullBuffer::from(bool_builder.finish()) + let mut null_builder = NullBufferBuilder::new(num_values); + null_builder.append_n_non_nulls(null_index); + null_builder.append_null(); + null_builder.append_n_non_nulls(num_values - null_index - 1); + // SAFETY: inner builder must be constructed + null_builder.finish().unwrap() } impl Debug for ArrowBytesMap From 58974386a31563d9e33cdefa43bfebfddc2a4824 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 29 Jan 2025 17:10:37 +0100 Subject: [PATCH 41/62] Restore ability to run single SLT file (#14355) Make it possible again to run single SLT file, even if it's name is a substring of other file(s). For example, after the change, this command: cargo test --test sqllogictests -- test_files/union.slt runs `union.slt` file, but does not run `pg_compat_union.slt`. --- datafusion/sqllogictest/bin/sqllogictests.rs | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/datafusion/sqllogictest/bin/sqllogictests.rs b/datafusion/sqllogictest/bin/sqllogictests.rs index 2c518217d5ed..58e383237065 100644 --- a/datafusion/sqllogictest/bin/sqllogictests.rs +++ b/datafusion/sqllogictest/bin/sqllogictests.rs @@ -497,7 +497,7 @@ fn read_test_files<'a>( let mut paths = read_dir_recursive(TEST_DIRECTORY)? .into_iter() .map(TestFile::new) - .filter(|f| options.check_test_file(&f.relative_path)) + .filter(|f| options.check_test_file(&f.path)) .filter(|f| f.is_slt_file()) .filter(|f| f.check_tpch(options)) .filter(|f| f.check_sqlite(options)) @@ -507,7 +507,7 @@ fn read_test_files<'a>( let mut sqlite_paths = read_dir_recursive(DATAFUSION_TESTING_TEST_DIRECTORY)? .into_iter() .map(TestFile::new) - .filter(|f| options.check_test_file(&f.relative_path)) + .filter(|f| options.check_test_file(&f.path)) .filter(|f| f.is_slt_file()) .filter(|f| f.check_sqlite(options)) .filter(|f| options.check_pg_compat_file(f.path.as_path())) @@ -544,10 +544,7 @@ struct Options { #[clap(long, env = "INCLUDE_TPCH", help = "Include tpch files")] include_tpch: bool, - #[clap( - action, - help = "regex like arguments passed to the program which are treated as cargo test filter (substring match on filenames)" - )] + #[clap(action, help = "test filter (substring match on filenames)")] filters: Vec, #[clap( @@ -594,15 +591,16 @@ impl Options { /// To be compatible with this, treat the command line arguments as a /// filter and that does a substring match on each input. returns /// true f this path should be run - fn check_test_file(&self, relative_path: &Path) -> bool { + fn check_test_file(&self, path: &Path) -> bool { if self.filters.is_empty() { return true; } // otherwise check if any filter matches + let path_string = path.to_string_lossy(); self.filters .iter() - .any(|filter| relative_path.to_string_lossy().contains(filter)) + .any(|filter| path_string.contains(filter)) } /// Postgres runner executes only tests in files with specific names or in From 3f4852008a596ad57a3d2f9dff12f451f99b5d12 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 29 Jan 2025 11:11:45 -0500 Subject: [PATCH 42/62] chore(deps): bump home from 0.5.9 to 0.5.11 in /datafusion-cli (#14257) Bumps [home](https://github.com/rust-lang/cargo) from 0.5.9 to 0.5.11. - [Changelog](https://github.com/rust-lang/cargo/blob/master/CHANGELOG.md) - [Commits](https://github.com/rust-lang/cargo/compare/home-0.5.9...home-0.5.11) --- updated-dependencies: - dependency-name: home dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion-cli/Cargo.lock | 18 +++++++++--------- datafusion-cli/Cargo.toml | 2 +- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 9c0e209b6fbd..d08c414986cd 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1764,7 +1764,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "33d852cb9b869c2a9b3df2f71a3074817f01e1844f839a144f5fcef059a4eb5d" dependencies = [ "libc", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -2070,11 +2070,11 @@ dependencies = [ [[package]] name = "home" -version = "0.5.9" +version = "0.5.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3d1354bf6b7235cb4a0576c2619fd4ed18183f689b12b006a0ee7329eeff9a5" +checksum = "589533453244b0995c858700322199b2becb13b627df2851f64a2775d024abcf" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -3170,7 +3170,7 @@ dependencies = [ "once_cell", "socket2", "tracing", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -3431,7 +3431,7 @@ dependencies = [ "errno", "libc", "linux-raw-sys", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -3828,7 +3828,7 @@ dependencies = [ "cfg-if", "libc", "psm", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -3910,7 +3910,7 @@ dependencies = [ "getrandom 0.3.1", "once_cell", "rustix", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -4453,7 +4453,7 @@ version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index c418d17ce29b..e9e35c4b3734 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -58,7 +58,7 @@ dirs = "6.0.0" env_logger = "0.11" futures = "0.3" # pin as home 0.5.11 has MSRV 1.81. Can remove this once we bump MSRV to 1.81 -home = "=0.5.9" +home = "=0.5.11" mimalloc = { version = "0.1", default-features = false } object_store = { version = "0.11.0", features = ["aws", "gcp", "http"] } parking_lot = { version = "0.12" } From a93b4deac100dac70d79199a537e155f4b4811ff Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 29 Jan 2025 11:45:28 -0500 Subject: [PATCH 43/62] chore(deps): bump aws-sdk-ssooidc in /datafusion-cli (#14314) Bumps [aws-sdk-ssooidc](https://github.com/awslabs/aws-sdk-rust) from 1.51.0 to 1.57.1. - [Release notes](https://github.com/awslabs/aws-sdk-rust/releases) - [Commits](https://github.com/awslabs/aws-sdk-rust/commits) --- updated-dependencies: - dependency-name: aws-sdk-ssooidc dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion-cli/Cargo.lock | 4 ++-- datafusion-cli/Cargo.toml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index d08c414986cd..5666406fda55 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -538,9 +538,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.51.0" +version = "1.57.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "abaf490c2e48eed0bb8e2da2fb08405647bd7f253996e0f93b981958ea0f73b0" +checksum = "a120ade4a44691b3c5c2ff2fa61b14ed331fdc218397f61ab48d66593012ae2a" dependencies = [ "aws-credential-types", "aws-runtime", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index e9e35c4b3734..3e1776433bce 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -36,7 +36,7 @@ aws-config = "=1.5.10" ## 1.53.0 requires a higher MSRV 1.81 so lock until DataFusion MSRV catches up aws-sdk-sso = "=1.50.0" ## 1.54.0 requires a higher MSRV 1.81 so lock until DataFusion MSRV catches up -aws-sdk-ssooidc = "=1.51.0" +aws-sdk-ssooidc = "=1.57.1" ## 1.54.1 requires a higher MSRV 1.81 so lock until DataFusion MSRV catches up aws-sdk-sts = "=1.51.0" # end pin aws-sdk crates From 2510e34f1c9c068f25ebf24fb40950e4974c8a70 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Thu, 30 Jan 2025 04:48:23 +0800 Subject: [PATCH 44/62] fix: LimitPushdown rule uncorrect remove some GlobalLimitExec (#14245) * fix: LimitPushdown rule uncorrect remove some GlobalLimitExec * Fix some logic for maybe fetch * Fix test * Address comments * Address comments * Add comments * Address comments --- datafusion/core/tests/dataframe/mod.rs | 5 -- .../physical-optimizer/src/limit_pushdown.rs | 11 +++- datafusion/sqllogictest/test_files/limit.slt | 61 +++++++++++++++++++ 3 files changed, 70 insertions(+), 7 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 954c46ab27e4..1ebbf92c736e 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -2217,11 +2217,6 @@ async fn write_parquet_with_order() -> Result<()> { let df = ctx.sql("SELECT * FROM data").await?; let results = df.collect().await?; - let df_explain = ctx.sql("explain SELECT a FROM data").await?; - let explain_result = df_explain.collect().await?; - - println!("explain_result {:?}", explain_result); - assert_batches_eq!( &[ "+---+---+", diff --git a/datafusion/physical-optimizer/src/limit_pushdown.rs b/datafusion/physical-optimizer/src/limit_pushdown.rs index 1c7e4d3d4c3d..5887cb51a727 100644 --- a/datafusion/physical-optimizer/src/limit_pushdown.rs +++ b/datafusion/physical-optimizer/src/limit_pushdown.rs @@ -31,7 +31,6 @@ use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; - /// This rule inspects [`ExecutionPlan`]'s and pushes down the fetch limit from /// the parent to the child if applicable. #[derive(Default, Debug)] @@ -248,7 +247,15 @@ pub fn pushdown_limit_helper( } } else { // Add fetch or a `LimitExec`: - global_state.satisfied = true; + // If the plan's children have limit and the child's limit < parent's limit, we shouldn't change the global state to true, + // because the children limit will be overridden if the global state is changed. + if !pushdown_plan + .children() + .iter() + .any(|&child| extract_limit(child).is_some()) + { + global_state.satisfied = true; + } pushdown_plan = if let Some(plan_with_fetch) = maybe_fetchable { if global_skip > 0 { add_global_limit(plan_with_fetch, global_skip, Some(global_fetch)) diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 5b98392f1aa0..308e759fa9fa 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -711,3 +711,64 @@ OFFSET 3 LIMIT 2; statement ok drop table ordered_table; + +# Test issue: https://github.com/apache/datafusion/issues/14204 +# Test limit pushdown with subquery +statement ok +create table testSubQueryLimit (a int, b int) as values (1,2), (2,3), (3,4); + +query IIII +select * from testSubQueryLimit as t1 join (select * from testSubQueryLimit limit 1) limit 10; +---- +1 2 1 2 +2 3 1 2 +3 4 1 2 + +query TT +explain select * from testSubQueryLimit as t1 join (select * from testSubQueryLimit limit 1) limit 10; +---- +logical_plan +01)Limit: skip=0, fetch=10 +02)--Cross Join: +03)----SubqueryAlias: t1 +04)------Limit: skip=0, fetch=10 +05)--------TableScan: testsubquerylimit projection=[a, b], fetch=10 +06)----Limit: skip=0, fetch=1 +07)------TableScan: testsubquerylimit projection=[a, b], fetch=1 +physical_plan +01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] +02)--GlobalLimitExec: skip=0, fetch=10 +03)----CrossJoinExec +04)------GlobalLimitExec: skip=0, fetch=1 +05)--------MemoryExec: partitions=1, partition_sizes=[1] +06)------GlobalLimitExec: skip=0, fetch=10 +07)--------MemoryExec: partitions=1, partition_sizes=[1] + + +query IIII +select * from testSubQueryLimit as t1 join (select * from testSubQueryLimit limit 10) limit 2; +---- +1 2 1 2 +1 2 2 3 + +query TT +explain select * from testSubQueryLimit as t1 join (select * from testSubQueryLimit limit 10) limit 2; +---- +logical_plan +01)Limit: skip=0, fetch=2 +02)--Cross Join: +03)----SubqueryAlias: t1 +04)------Limit: skip=0, fetch=2 +05)--------TableScan: testsubquerylimit projection=[a, b], fetch=2 +06)----Limit: skip=0, fetch=2 +07)------TableScan: testsubquerylimit projection=[a, b], fetch=2 +physical_plan +01)GlobalLimitExec: skip=0, fetch=2 +02)--CrossJoinExec +03)----GlobalLimitExec: skip=0, fetch=2 +04)------MemoryExec: partitions=1, partition_sizes=[1] +05)----GlobalLimitExec: skip=0, fetch=2 +06)------MemoryExec: partitions=1, partition_sizes=[1] + +statement ok +drop table testSubQueryLimit; From 60c0fb887e8723c47c268e1b578f2d78aebaefde Mon Sep 17 00:00:00 2001 From: Ian Lai <108986288+Chen-Yuan-Lai@users.noreply.github.com> Date: Thu, 30 Jan 2025 04:48:57 +0800 Subject: [PATCH 45/62] refactor: switch `BooleanBufferBuilder` to `NullBufferBuilder` in single_group_by (#14360) * refactor: switch BooleanBufferBuilder to NullBufferBuilder in single_group_by * Update datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs --------- Co-authored-by: Cheng-Yuan-Lai Co-authored-by: Andrew Lamb --- .../group_values/single_group_by/primitive.rs | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs index 6b69c00bca74..78a5f619fcd6 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs @@ -17,12 +17,11 @@ use crate::aggregates::group_values::GroupValues; use ahash::RandomState; -use arrow::array::BooleanBufferBuilder; -use arrow::buffer::NullBuffer; use arrow::datatypes::i256; use arrow::record_batch::RecordBatch; use arrow_array::cast::AsArray; use arrow_array::{ArrayRef, ArrowNativeTypeOp, ArrowPrimitiveType, PrimitiveArray}; +use arrow_buffer::NullBufferBuilder; use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano}; use arrow_schema::DataType; use datafusion_common::Result; @@ -166,10 +165,12 @@ where null_idx: Option, ) -> PrimitiveArray { let nulls = null_idx.map(|null_idx| { - let mut buffer = BooleanBufferBuilder::new(values.len()); - buffer.append_n(values.len(), true); - buffer.set_bit(null_idx, false); - unsafe { NullBuffer::new_unchecked(buffer.finish(), 1) } + let mut buffer = NullBufferBuilder::new(values.len()); + buffer.append_n_non_nulls(null_idx); + buffer.append_null(); + buffer.append_n_non_nulls(values.len() - null_idx - 1); + // NOTE: The inner builder must be constructed as there is at least one null + buffer.finish().unwrap() }); PrimitiveArray::::new(values.into(), nulls) } From 707f67308be8f8491bf3366bb0d920c89fc10183 Mon Sep 17 00:00:00 2001 From: Eren Avsarogullari Date: Wed, 29 Jan 2025 13:55:09 -0800 Subject: [PATCH 46/62] Issue-14216 - Support arrays_overlap function (#14217) --- datafusion/functions-nested/src/array_has.rs | 2 +- datafusion/sqllogictest/test_files/array.slt | 9 +++++++++ docs/source/user-guide/sql/scalar_functions.md | 6 ++++++ 3 files changed, 16 insertions(+), 1 deletion(-) diff --git a/datafusion/functions-nested/src/array_has.rs b/datafusion/functions-nested/src/array_has.rs index 96996cc07f27..df007b5cd60d 100644 --- a/datafusion/functions-nested/src/array_has.rs +++ b/datafusion/functions-nested/src/array_has.rs @@ -385,7 +385,7 @@ impl ArrayHasAny { pub fn new() -> Self { Self { signature: Signature::any(2, Volatility::Immutable), - aliases: vec![String::from("list_has_any")], + aliases: vec![String::from("list_has_any"), String::from("arrays_overlap")], } } } diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index baf4ef7795e7..46dbb4816694 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -6029,6 +6029,15 @@ select list_has_all(make_array(1,2,3), make_array(4,5,6)), ---- false true false true +query BBBB +select arrays_overlap(make_array(1,2,3), make_array(4,5,6)), + arrays_overlap(make_array(1,2,3), make_array(1,2,4)), + arrays_overlap(make_array(['aa']), make_array(['aa'],['bb'])), + arrays_overlap(make_array('aa',NULL), make_array('bb',NULL)) +; +---- +false true true true + query ??? select range(column2), range(column1, column2), diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index a64ed597e007..b769b8b7bdb0 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -2545,6 +2545,7 @@ _Alias of [current_date](#current_date)._ - [array_sort](#array_sort) - [array_to_string](#array_to_string) - [array_union](#array_union) +- [arrays_overlap](#arrays_overlap) - [cardinality](#cardinality) - [empty](#empty) - [flatten](#flatten) @@ -2929,6 +2930,7 @@ array_has_any(array, sub-array) #### Aliases - list_has_any +- arrays_overlap ### `array_indexof` @@ -3573,6 +3575,10 @@ array_union(array1, array2) - list_union +### `arrays_overlap` + +_Alias of [array_has_any](#array_has_any)._ + ### `cardinality` Returns the total number of elements in the array. From 99f907b98c9788d2bb5219cda644dd585e451326 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 29 Jan 2025 16:56:11 -0500 Subject: [PATCH 47/62] chore(deps): bump rustyline from 14.0.0 to 15.0.0 in /datafusion-cli (#14265) * chore(deps): bump rustyline from 14.0.0 to 15.0.0 in /datafusion-cli Bumps [rustyline](https://github.com/kkawakam/rustyline) from 14.0.0 to 15.0.0. - [Release notes](https://github.com/kkawakam/rustyline/releases) - [Changelog](https://github.com/kkawakam/rustyline/blob/master/History.md) - [Commits](https://github.com/kkawakam/rustyline/compare/v14.0.0...v15.0.0) --- updated-dependencies: - dependency-name: rustyline dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] * fix build --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: jonahgao Co-authored-by: Andrew Lamb --- datafusion-cli/Cargo.lock | 30 +++++++++--------------------- datafusion-cli/Cargo.toml | 2 +- datafusion-cli/src/helper.rs | 6 +++--- datafusion-cli/src/highlighter.rs | 4 ++-- 4 files changed, 15 insertions(+), 27 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 5666406fda55..733a700ce189 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -958,12 +958,6 @@ version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" -[[package]] -name = "cfg_aliases" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd16c4719339c4530435d38e511904438d07cce7950afa3718a84ac36c10e89e" - [[package]] name = "cfg_aliases" version = "0.2.1" @@ -1067,7 +1061,7 @@ checksum = "24f165e7b643266ea80cb858aed492ad9280e3e05ce24d4a99d7d7b889b6a4d9" dependencies = [ "strum", "strum_macros", - "unicode-width 0.2.0", + "unicode-width", ] [[package]] @@ -2717,13 +2711,13 @@ dependencies = [ [[package]] name = "nix" -version = "0.28.0" +version = "0.29.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab2156c4fce2f8df6c499cc1c763e4394b7482525bf2a9701c9d79d215f519e4" +checksum = "71e2746dc3a24dd78b3cfcb7be93368c6de9963d30f43a6a73998a9cf4b17b46" dependencies = [ "bitflags 2.8.0", "cfg-if", - "cfg_aliases 0.1.1", + "cfg_aliases", "libc", ] @@ -3165,7 +3159,7 @@ version = "0.5.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1c40286217b4ba3a71d644d752e6a0b71f13f1b6a2c5311acfcbe0c2418ed904" dependencies = [ - "cfg_aliases 0.2.1", + "cfg_aliases", "libc", "once_cell", "socket2", @@ -3540,9 +3534,9 @@ checksum = "f7c45b9784283f1b2e7fb61b42047c2fd678ef0960d4f6f1eba131594cc369d4" [[package]] name = "rustyline" -version = "14.0.0" +version = "15.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7803e8936da37efd9b6d4478277f4b2b9bb5cdb37a113e8d63222e58da647e63" +checksum = "2ee1e066dc922e513bda599c6ccb5f3bb2b0ea5870a579448f2622993f0a9a2f" dependencies = [ "bitflags 2.8.0", "cfg-if", @@ -3555,9 +3549,9 @@ dependencies = [ "nix", "radix_trie", "unicode-segmentation", - "unicode-width 0.1.14", + "unicode-width", "utf8parse", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -4225,12 +4219,6 @@ version = "1.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f6ccf251212114b54433ec949fd6a7841275f9ada20dddd2f29e9ceea4501493" -[[package]] -name = "unicode-width" -version = "0.1.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7dd6e30e90baa6f72411720665d41d89b9a3d039dc45b8faea1ddd07f617f6af" - [[package]] name = "unicode-width" version = "0.2.0" diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 3e1776433bce..f93d70b1a579 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -64,7 +64,7 @@ object_store = { version = "0.11.0", features = ["aws", "gcp", "http"] } parking_lot = { version = "0.12" } parquet = { version = "54.0.0", default-features = false } regex = "1.8" -rustyline = "14.0" +rustyline = "15.0" tokio = { version = "1.24", features = ["macros", "rt", "rt-multi-thread", "sync", "parking_lot", "signal"] } url = "2.5.4" diff --git a/datafusion-cli/src/helper.rs b/datafusion-cli/src/helper.rs index 86a51f40a0a4..871e6de9d31f 100644 --- a/datafusion-cli/src/helper.rs +++ b/datafusion-cli/src/helper.rs @@ -30,7 +30,7 @@ use datafusion::sql::sqlparser::parser::ParserError; use rustyline::completion::{Completer, FilenameCompleter, Pair}; use rustyline::error::ReadlineError; -use rustyline::highlight::Highlighter; +use rustyline::highlight::{CmdKind, Highlighter}; use rustyline::hint::Hinter; use rustyline::validate::{ValidationContext, ValidationResult, Validator}; use rustyline::{Context, Helper, Result}; @@ -118,8 +118,8 @@ impl Highlighter for CliHelper { self.highlighter.highlight(line, pos) } - fn highlight_char(&self, line: &str, pos: usize, forced: bool) -> bool { - self.highlighter.highlight_char(line, pos, forced) + fn highlight_char(&self, line: &str, pos: usize, kind: CmdKind) -> bool { + self.highlighter.highlight_char(line, pos, kind) } } diff --git a/datafusion-cli/src/highlighter.rs b/datafusion-cli/src/highlighter.rs index 530b87af8732..7a886b94740b 100644 --- a/datafusion-cli/src/highlighter.rs +++ b/datafusion-cli/src/highlighter.rs @@ -27,7 +27,7 @@ use datafusion::sql::sqlparser::{ keywords::Keyword, tokenizer::{Token, Tokenizer}, }; -use rustyline::highlight::Highlighter; +use rustyline::highlight::{CmdKind, Highlighter}; /// The syntax highlighter. #[derive(Debug)] @@ -73,7 +73,7 @@ impl Highlighter for SyntaxHighlighter { } } - fn highlight_char(&self, line: &str, _pos: usize, _forced: bool) -> bool { + fn highlight_char(&self, line: &str, _pos: usize, _cmd: CmdKind) -> bool { !line.is_empty() } } From ba7b94f12be95ba0d27ddf483e50989cbcf7d5a2 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 30 Jan 2025 10:44:35 +0300 Subject: [PATCH 48/62] remove aggregate changes, tests already give expected results --- .../physical_optimizer/enforce_sorting.rs | 130 ------------------ datafusion/expr/src/udaf.rs | 14 +- datafusion/functions-aggregate/src/count.rs | 2 +- datafusion/functions-aggregate/src/min_max.rs | 8 +- datafusion/functions-aggregate/src/sum.rs | 8 +- datafusion/physical-expr/src/aggregate.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 24 +--- .../sqllogictest/test_files/aggregate.slt | 96 +------------ 8 files changed, 24 insertions(+), 260 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 304dd3c659fe..656034ef81cd 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -241,136 +241,6 @@ async fn test_remove_unnecessary_sort5() -> Result<()> { Ok(()) } -#[test] -fn test_aggregate_set_monotonic_no_group() -> Result<()> { - let schema = create_test_schema4()?; - - let source = memory_exec(&schema); - - let sort_exprs = vec![sort_expr("a", &schema)]; - let sort = sort_exec(sort_exprs.clone(), source); - - let aggregate = aggregate_exec_set_monotonic(sort, vec![]); - - let sort_exprs = LexOrdering::new(vec![sort_expr("count", &aggregate.schema())]); - let physical_plan: Arc = - Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _; - - let expected_input = [ - "SortExec: expr=[count@0 ASC], preserve_partitioning=[false]", - " AggregateExec: mode=Single, gby=[], aggr=[count]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - - let expected_optimized = [ - "AggregateExec: mode=Single, gby=[], aggr=[count]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[test] -fn test_aggregate_set_monotonic_with_group() -> Result<()> { - let schema = create_test_schema4()?; - - let source = memory_exec(&schema); - - let sort_exprs = vec![sort_expr("a", &schema)]; - let sort = sort_exec(sort_exprs.clone(), source); - - let aggregate = - aggregate_exec_set_monotonic(sort, vec![(col("a", &schema)?, "a".to_string())]); - - let sort_exprs = LexOrdering::new(vec![sort_expr("count", &aggregate.schema())]); - let physical_plan: Arc = - Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _; - - let expected_input = [ - "SortExec: expr=[count@1 ASC], preserve_partitioning=[false]", - " AggregateExec: mode=Single, gby=[a@0 as a], aggr=[count], ordering_mode=Sorted", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - - let expected_optimized = [ - "SortExec: expr=[count@1 ASC], preserve_partitioning=[false]", - " AggregateExec: mode=Single, gby=[a@0 as a], aggr=[count], ordering_mode=Sorted", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[test] -fn test_aggregate_set_monotonic_with_group_partial() -> Result<()> { - let schema = create_test_schema4()?; - - let source = memory_exec(&schema); - - let sort_exprs = vec![sort_expr("a", &schema)]; - let sort = sort_exec(sort_exprs.clone(), source); - - let aggregate = - aggregate_exec_set_monotonic(sort, vec![(col("a", &schema)?, "a".to_string())]); - - let sort_exprs = LexOrdering::new(vec![ - sort_expr("a", &schema), - sort_expr("count", &aggregate.schema()), - ]); - let physical_plan: Arc = - Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _; - - let expected_input = [ - "SortExec: expr=[a@0 ASC, count@1 ASC], preserve_partitioning=[false]", - " AggregateExec: mode=Single, gby=[a@0 as a], aggr=[count], ordering_mode=Sorted", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - - let expected_optimized = [ - "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[count], ordering_mode=Sorted", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[test] -fn test_aggregate_non_set_monotonic() -> Result<()> { - let schema = create_test_schema4()?; - let source = memory_exec(&schema); - let sort_exprs = vec![sort_expr("a", &schema)]; - let sort = sort_exec(sort_exprs.clone(), source); - - let aggregate = aggregate_exec_non_set_monotonic(sort); - let sort_exprs = LexOrdering::new(vec![sort_expr("avg", &aggregate.schema())]); - let physical_plan: Arc = - Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _; - - let expected_input = [ - "SortExec: expr=[avg@0 ASC], preserve_partitioning=[false]", - " AggregateExec: mode=Single, gby=[], aggr=[avg]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - - let expected_optimized = [ - "SortExec: expr=[avg@0 ASC], preserve_partitioning=[false]", - " AggregateExec: mode=Single, gby=[], aggr=[avg]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - #[tokio::test] async fn test_bounded_window_set_monotonic_no_partition() -> Result<()> { let schema = create_test_schema()?; diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 8e172d29c989..e9cdd754e136 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -640,6 +640,12 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// function is monotonically increasing if its value increases as its argument grows /// (as a set). Formally, `f` is a monotonically increasing set function if `f(S) >= f(T)` /// whenever `S` is a superset of `T`. + /// + /// For example `count` and `max` are monotonically increasing as their values always + /// increase (or stay the same) as new values are seen. + /// + /// `min` is monotonically decreasing as its value always decreases or stays + /// the same as new values are seen. fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { AggregateExprSetMonotonicity::NotMonotonic } @@ -830,16 +836,16 @@ pub mod aggregate_doc_sections { #[derive(Debug, Clone)] pub enum AggregateExprSetMonotonicity { /// Ordering exists as ascending - MonotonicallyAscending, + Increasing, /// Ordering exists as descending - MonotonicallyDescending, + Decreasing, /// No ordering NotMonotonic, } impl AggregateExprSetMonotonicity { - pub fn is_descending(&self) -> bool { - matches!(self, Self::MonotonicallyDescending) + pub fn is_decreasing(&self) -> bool { + matches!(self, Self::Decreasing) } pub fn is_monotonic(&self) -> bool { !matches!(self, Self::NotMonotonic) diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index bdef99cb0aa7..48eabe7b9cce 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -354,7 +354,7 @@ impl AggregateUDFImpl for Count { } fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { - AggregateExprSetMonotonicity::MonotonicallyAscending + AggregateExprSetMonotonicity::Increasing } } diff --git a/datafusion/functions-aggregate/src/min_max.rs b/datafusion/functions-aggregate/src/min_max.rs index 7e4978407949..a36a2f184593 100644 --- a/datafusion/functions-aggregate/src/min_max.rs +++ b/datafusion/functions-aggregate/src/min_max.rs @@ -363,7 +363,9 @@ impl AggregateUDFImpl for Max { } fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { - AggregateExprSetMonotonicity::MonotonicallyAscending + // max is monotonically increasing as it always increases or + // stays the same as new values are seen + AggregateExprSetMonotonicity::Increasing } } @@ -1189,7 +1191,9 @@ impl AggregateUDFImpl for Min { } fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { - AggregateExprSetMonotonicity::MonotonicallyDescending + // min is monotonically decreasing as it always decreases or + // stays the same as new values are seen + AggregateExprSetMonotonicity::Decreasing } } diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index a3a641784180..1bb54734fb69 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -259,10 +259,10 @@ impl AggregateUDFImpl for Sum { // Sum is only monotonic if its input is unsigned // TODO: Expand these utilizing statistics match data_type { - DataType::UInt8 => AggregateExprSetMonotonicity::MonotonicallyAscending, - DataType::UInt16 => AggregateExprSetMonotonicity::MonotonicallyAscending, - DataType::UInt32 => AggregateExprSetMonotonicity::MonotonicallyAscending, - DataType::UInt64 => AggregateExprSetMonotonicity::MonotonicallyAscending, + DataType::UInt8 => AggregateExprSetMonotonicity::Increasing, + DataType::UInt16 => AggregateExprSetMonotonicity::Increasing, + DataType::UInt32 => AggregateExprSetMonotonicity::Increasing, + DataType::UInt64 => AggregateExprSetMonotonicity::Increasing, _ => AggregateExprSetMonotonicity::NotMonotonic, } } diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 5e771421829e..ac86939f8cbb 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -554,7 +554,7 @@ impl AggregateFunctionExpr { return None; } let expr = Arc::new(Column::new(self.name(), aggr_func_idx)); - let options = SortOptions::new(monotonicity.is_descending(), false); + let options = SortOptions::new(monotonicity.is_decreasing(), false); Some(PhysicalSortExpr { expr, options }) } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index daa2d1075718..6fb1c0a3463c 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -512,7 +512,6 @@ impl AggregateExec { &group_expr_mapping, &mode, &input_order_mode, - aggr_expr.as_slice(), ); Ok(AggregateExec { @@ -649,32 +648,11 @@ impl AggregateExec { group_expr_mapping: &ProjectionMapping, mode: &AggregateMode, input_order_mode: &InputOrderMode, - aggr_exprs: &[Arc], ) -> PlanProperties { // Construct equivalence properties: let mut eq_properties = input .equivalence_properties() - .project(group_expr_mapping, Arc::clone(&schema)); - - let schema_len = schema.fields.len(); - let aggr_expr_indices = - ((schema_len - aggr_exprs.len())..schema_len).collect::>(); - // if the aggregate function is set monotonic, add it into equivalence properties - for (i, aggr_expr) in aggr_exprs.iter().enumerate() { - let aggr_expr_index = aggr_expr_indices[i]; - if let Some(expr) = aggr_expr.get_result_ordering(aggr_expr_index) { - if group_expr_mapping.map.is_empty() { - eq_properties.add_new_ordering(LexOrdering::new(vec![expr])); - } else if *input_order_mode != InputOrderMode::Linear { - if let Some(ordering) = eq_properties.output_ordering() { - let mut existing_ordering = ordering.to_vec(); - existing_ordering.push(expr); - eq_properties - .add_new_ordering(LexOrdering::new(existing_ordering)); - } - } - } - } + .project(group_expr_mapping, schema); // Group by expression will be a distinct value after the aggregation. // Add it into the constraint set. diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index dc7d656c47bc..da3d76124c73 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -6202,98 +6202,4 @@ physical_plan 13)------------ProjectionExec: expr=[1 as id, 3 as foo] 14)--------------PlaceholderRowExec 15)------------ProjectionExec: expr=[1 as id, 2 as foo] -16)--------------PlaceholderRowExec - - -# Set-Monotonic Aggregate functions can output results in order -statement ok -CREATE EXTERNAL TABLE aggregate_test_100_ordered ( - c1 VARCHAR NOT NULL, - c2 TINYINT NOT NULL, - c3 SMALLINT NOT NULL, - c4 SMALLINT, - c5 INT, - c6 BIGINT NOT NULL, - c7 SMALLINT NOT NULL, - c8 INT NOT NULL, - c9 INT UNSIGNED NOT NULL, - c10 BIGINT UNSIGNED NOT NULL, - c11 FLOAT NOT NULL, - c12 DOUBLE NOT NULL, - c13 VARCHAR NOT NULL -) -STORED AS CSV -LOCATION '../../testing/data/csv/aggregate_test_100.csv' -WITH ORDER (c1) -OPTIONS ('format.has_header' 'true'); - -statement ok -set datafusion.optimizer.prefer_existing_sort = true; - -query TT -EXPLAIN SELECT c1, SUM(c9) as sum_c9 FROM aggregate_test_100_ordered GROUP BY c1 ORDER BY c1, sum_c9; ----- -logical_plan -01)Sort: aggregate_test_100_ordered.c1 ASC NULLS LAST, sum_c9 ASC NULLS LAST -02)--Projection: aggregate_test_100_ordered.c1, sum(aggregate_test_100_ordered.c9) AS sum_c9 -03)----Aggregate: groupBy=[[aggregate_test_100_ordered.c1]], aggr=[[sum(CAST(aggregate_test_100_ordered.c9 AS UInt64))]] -04)------TableScan: aggregate_test_100_ordered projection=[c1, c9] -physical_plan -01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, sum_c9@1 ASC NULLS LAST] -02)--ProjectionExec: expr=[c1@0 as c1, sum(aggregate_test_100_ordered.c9)@1 as sum_c9] -03)----AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[sum(aggregate_test_100_ordered.c9)], ordering_mode=Sorted -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4, preserve_order=true, sort_exprs=c1@0 ASC NULLS LAST, sum(aggregate_test_100_ordered.c9)@1 ASC NULLS LAST -06)----------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[sum(aggregate_test_100_ordered.c9)], ordering_mode=Sorted -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], output_ordering=[c1@0 ASC NULLS LAST], has_header=true - -query TT -EXPLAIN SELECT SUM(c9) as sum_c9 FROM aggregate_test_100_ordered ORDER BY sum_c9; ----- -logical_plan -01)Sort: sum_c9 ASC NULLS LAST -02)--Projection: sum(aggregate_test_100_ordered.c9) AS sum_c9 -03)----Aggregate: groupBy=[[]], aggr=[[sum(CAST(aggregate_test_100_ordered.c9 AS UInt64))]] -04)------TableScan: aggregate_test_100_ordered projection=[c9] -physical_plan -01)ProjectionExec: expr=[sum(aggregate_test_100_ordered.c9)@0 as sum_c9] -02)--AggregateExec: mode=Final, gby=[], aggr=[sum(aggregate_test_100_ordered.c9)] -03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(aggregate_test_100_ordered.c9)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true - -query TT -EXPLAIN SELECT c1, MIN(c5) as min_c5 FROM aggregate_test_100_ordered GROUP BY c1 ORDER BY c1, min_c5 DESC NULLS LAST; ----- -logical_plan -01)Sort: aggregate_test_100_ordered.c1 ASC NULLS LAST, min_c5 DESC NULLS LAST -02)--Projection: aggregate_test_100_ordered.c1, min(aggregate_test_100_ordered.c5) AS min_c5 -03)----Aggregate: groupBy=[[aggregate_test_100_ordered.c1]], aggr=[[min(aggregate_test_100_ordered.c5)]] -04)------TableScan: aggregate_test_100_ordered projection=[c1, c5] -physical_plan -01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, min_c5@1 DESC NULLS LAST] -02)--ProjectionExec: expr=[c1@0 as c1, min(aggregate_test_100_ordered.c5)@1 as min_c5] -03)----AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[min(aggregate_test_100_ordered.c5)], ordering_mode=Sorted -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4, preserve_order=true, sort_exprs=c1@0 ASC NULLS LAST, min(aggregate_test_100_ordered.c5)@1 DESC NULLS LAST -06)----------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[min(aggregate_test_100_ordered.c5)], ordering_mode=Sorted -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c5], output_ordering=[c1@0 ASC NULLS LAST], has_header=true - -query TT -EXPLAIN SELECT MAX(c5) as max_c5 FROM aggregate_test_100_ordered ORDER BY max_c5; ----- -logical_plan -01)Sort: max_c5 ASC NULLS LAST -02)--Projection: max(aggregate_test_100_ordered.c5) AS max_c5 -03)----Aggregate: groupBy=[[]], aggr=[[max(aggregate_test_100_ordered.c5)]] -04)------TableScan: aggregate_test_100_ordered projection=[c5] -physical_plan -01)ProjectionExec: expr=[max(aggregate_test_100_ordered.c5)@0 as max_c5] -02)--AggregateExec: mode=Final, gby=[], aggr=[max(aggregate_test_100_ordered.c5)] -03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[], aggr=[max(aggregate_test_100_ordered.c5)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], has_header=true +16)--------------PlaceholderRowExec \ No newline at end of file From 2152b7f1d31b985e987ad5e73fd8932e72f806be Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 30 Jan 2025 10:45:53 +0300 Subject: [PATCH 49/62] fix clippy --- .../physical_optimizer/enforce_sorting.rs | 3 +- .../tests/physical_optimizer/test_utils.rs | 53 ------------------- 2 files changed, 1 insertion(+), 55 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 656034ef81cd..ab90eab74d3f 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -18,8 +18,7 @@ use std::sync::Arc; use crate::physical_optimizer::test_utils::{ - aggregate_exec, aggregate_exec_non_set_monotonic, aggregate_exec_set_monotonic, - bounded_window_exec, bounded_window_exec_non_set_monotonic, + aggregate_exec, bounded_window_exec, bounded_window_exec_non_set_monotonic, bounded_window_exec_with_partition, check_integrity, coalesce_batches_exec, coalesce_partitions_exec, create_test_schema, create_test_schema2, create_test_schema3, create_test_schema4, filter_exec, global_limit_exec, diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index fe7d73da2bfd..b26b1227ea43 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -326,59 +326,6 @@ pub fn aggregate_exec(input: Arc) -> Arc { ) } -pub fn aggregate_exec_set_monotonic( - input: Arc, - group_by: Vec<(Arc, String)>, -) -> Arc { - let schema = input.schema(); - let aggregate_expr = - vec![ - AggregateExprBuilder::new(count_udaf(), vec![col("d", &schema).unwrap()]) - .schema(Arc::clone(&schema)) - .alias("count") - .build() - .map(Arc::new) - .unwrap(), - ]; - Arc::new( - AggregateExec::try_new( - AggregateMode::Single, - PhysicalGroupBy::new_single(group_by), - aggregate_expr, - vec![None], - input, - schema, - ) - .unwrap(), - ) -} - -pub fn aggregate_exec_non_set_monotonic( - input: Arc, -) -> Arc { - let schema = input.schema(); - let aggregate_expr = - vec![ - AggregateExprBuilder::new(avg_udaf(), vec![col("d", &schema).unwrap()]) - .schema(Arc::clone(&schema)) - .alias("avg") - .build() - .map(Arc::new) - .unwrap(), - ]; - Arc::new( - AggregateExec::try_new( - AggregateMode::Single, - PhysicalGroupBy::default(), - aggregate_expr, - vec![None], - input, - schema, - ) - .unwrap(), - ) -} - pub fn coalesce_batches_exec(input: Arc) -> Arc { Arc::new(CoalesceBatchesExec::new(input, 128)) } From 0d367dd7ab8fe3212e8469fd383d508584559f3f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 30 Jan 2025 00:55:26 -0800 Subject: [PATCH 50/62] chore(deps): bump aws-sdk-sts from 1.51.0 to 1.57.0 in /datafusion-cli (#14263) Bumps [aws-sdk-sts](https://github.com/awslabs/aws-sdk-rust) from 1.51.0 to 1.57.0. - [Release notes](https://github.com/awslabs/aws-sdk-rust/releases) - [Commits](https://github.com/awslabs/aws-sdk-rust/commits) --- updated-dependencies: - dependency-name: aws-sdk-sts dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Andrew Lamb --- datafusion-cli/Cargo.lock | 4 ++-- datafusion-cli/Cargo.toml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 733a700ce189..750eb7878cf2 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -560,9 +560,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.51.0" +version = "1.57.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b68fde0d69c8bfdc1060ea7da21df3e39f6014da316783336deff0a9ec28f4bf" +checksum = "115fd4fb663817ed595a5ee4f1649d7aacd861d47462323cb37576ce89271b93" dependencies = [ "aws-credential-types", "aws-runtime", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index f93d70b1a579..7dbd61153db4 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -38,7 +38,7 @@ aws-sdk-sso = "=1.50.0" ## 1.54.0 requires a higher MSRV 1.81 so lock until DataFusion MSRV catches up aws-sdk-ssooidc = "=1.57.1" ## 1.54.1 requires a higher MSRV 1.81 so lock until DataFusion MSRV catches up -aws-sdk-sts = "=1.51.0" +aws-sdk-sts = "=1.57.0" # end pin aws-sdk crates aws-credential-types = "1.2.0" clap = { version = "4.5.27", features = ["derive", "cargo"] } From 51a23cfab8bb100cfcfaa398cb9e1fce7d680493 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 30 Jan 2025 00:55:44 -0800 Subject: [PATCH 51/62] chore(deps): bump aws-sdk-sso from 1.50.0 to 1.56.0 in /datafusion-cli (#14259) Bumps [aws-sdk-sso](https://github.com/awslabs/aws-sdk-rust) from 1.50.0 to 1.56.0. - [Release notes](https://github.com/awslabs/aws-sdk-rust/releases) - [Commits](https://github.com/awslabs/aws-sdk-rust/commits) --- updated-dependencies: - dependency-name: aws-sdk-sso dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Andrew Lamb --- datafusion-cli/Cargo.lock | 4 ++-- datafusion-cli/Cargo.toml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 750eb7878cf2..106b0b4c2fd7 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -516,9 +516,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.50.0" +version = "1.56.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05ca43a4ef210894f93096039ef1d6fa4ad3edfabb3be92b80908b9f2e4b4eab" +checksum = "12e057fdcb8842de9b83592a70f5b4da0ee10bc0ad278247da1425a742a444d7" dependencies = [ "aws-credential-types", "aws-runtime", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 7dbd61153db4..8ef002e9f29c 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -34,7 +34,7 @@ async-trait = "0.1.73" ## 1.5.13 requires a hiher MSRV 1.81 so lock until DataFusion MSRV catches up aws-config = "=1.5.10" ## 1.53.0 requires a higher MSRV 1.81 so lock until DataFusion MSRV catches up -aws-sdk-sso = "=1.50.0" +aws-sdk-sso = "=1.56.0" ## 1.54.0 requires a higher MSRV 1.81 so lock until DataFusion MSRV catches up aws-sdk-ssooidc = "=1.57.1" ## 1.54.1 requires a higher MSRV 1.81 so lock until DataFusion MSRV catches up From b79efc0fb29c93c6bcd22097dcd49a5a59e424ed Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 30 Jan 2025 00:56:13 -0800 Subject: [PATCH 52/62] chore(deps): bump korandoru/hawkeye from 5 to 6 (#14354) Bumps [korandoru/hawkeye](https://github.com/korandoru/hawkeye) from 5 to 6. - [Release notes](https://github.com/korandoru/hawkeye/releases) - [Changelog](https://github.com/korandoru/hawkeye/blob/main/CHANGELOG.md) - [Commits](https://github.com/korandoru/hawkeye/compare/v5...v6) --- updated-dependencies: - dependency-name: korandoru/hawkeye dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/dev.yml | 2 +- .github/workflows/rust.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/dev.yml b/.github/workflows/dev.yml index cf204b2cd6c1..aa4bd862e09e 100644 --- a/.github/workflows/dev.yml +++ b/.github/workflows/dev.yml @@ -28,7 +28,7 @@ jobs: name: Check License Header steps: - uses: actions/checkout@v4 - - uses: korandoru/hawkeye@v5 + - uses: korandoru/hawkeye@v6 prettier: name: Use prettier to check formatting of documents diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 7ac0dfa78215..d5d60b131c7e 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -45,7 +45,7 @@ jobs: name: Check License Header steps: - uses: actions/checkout@v4 - - uses: korandoru/hawkeye@v5 + - uses: korandoru/hawkeye@v6 # Check crate compiles and base cargo check passes linux-build-lib: From 7822613c7a3914bacb1eb172e16d567c973ed49f Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 30 Jan 2025 14:56:34 +0300 Subject: [PATCH 53/62] remove one row sorts --- .../physical-plan/src/aggregates/mod.rs | 15 ++++++++++++++- .../sqllogictest/test_files/aggregate.slt | 19 ++++++++++++++++++- 2 files changed, 32 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 6fb1c0a3463c..3aaa766f92f2 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -45,7 +45,7 @@ use datafusion_expr::{Accumulator, Aggregate}; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr::{ equivalence::ProjectionMapping, expressions::Column, physical_exprs_contains, - EquivalenceProperties, LexOrdering, LexRequirement, PhysicalExpr, + ConstExpr, EquivalenceProperties, LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortRequirement, }; @@ -512,6 +512,7 @@ impl AggregateExec { &group_expr_mapping, &mode, &input_order_mode, + aggr_expr.as_slice(), ); Ok(AggregateExec { @@ -648,12 +649,24 @@ impl AggregateExec { group_expr_mapping: &ProjectionMapping, mode: &AggregateMode, input_order_mode: &InputOrderMode, + aggr_exprs: &[Arc], ) -> PlanProperties { // Construct equivalence properties: let mut eq_properties = input .equivalence_properties() .project(group_expr_mapping, schema); + // If the group by is empty, then we ensure that the operator will produce + // only one row, and mark the generated result as a constant value. + if group_expr_mapping.map.is_empty() { + let mut constants = eq_properties.constants().to_vec(); + let new_constants = aggr_exprs.iter().enumerate().map(|(idx, func)| { + ConstExpr::new(Arc::new(Column::new(func.name(), idx))) + }); + constants.extend(new_constants); + eq_properties = eq_properties.with_constants(constants); + } + // Group by expression will be a distinct value after the aggregation. // Add it into the constraint set. let mut constraints = eq_properties.constraints().to_vec(); diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index da3d76124c73..a5cbdbb79db0 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -6202,4 +6202,21 @@ physical_plan 13)------------ProjectionExec: expr=[1 as id, 3 as foo] 14)--------------PlaceholderRowExec 15)------------ProjectionExec: expr=[1 as id, 2 as foo] -16)--------------PlaceholderRowExec \ No newline at end of file +16)--------------PlaceholderRowExec + +# SortExec is removed if it is coming after one-row producing AggregateExec's having an empty group by expression +query TT +EXPLAIN SELECT COUNT(c5) as count_c5 FROM aggregate_test_100 ORDER BY count_c5; +---- +logical_plan +01)Sort: count_c5 ASC NULLS LAST +02)--Projection: count(aggregate_test_100.c5) AS count_c5 +03)----Aggregate: groupBy=[[]], aggr=[[count(aggregate_test_100.c5)]] +04)------TableScan: aggregate_test_100 projection=[c5] +physical_plan +01)ProjectionExec: expr=[count(aggregate_test_100.c5)@0 as count_c5] +02)--AggregateExec: mode=Final, gby=[], aggr=[count(aggregate_test_100.c5)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[count(aggregate_test_100.c5)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], has_header=true From 5e9b2db16b9a2b012ce07aa86fb7e5ca9759710c Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 30 Jan 2025 15:54:32 +0300 Subject: [PATCH 54/62] Improve comments --- datafusion/expr/src/udaf.rs | 41 ++++++++----------- datafusion/functions-aggregate/src/count.rs | 2 + datafusion/functions-aggregate/src/min_max.rs | 8 ++-- datafusion/functions-aggregate/src/sum.rs | 4 +- datafusion/physical-expr/src/aggregate.rs | 18 ++++---- 5 files changed, 35 insertions(+), 38 deletions(-) diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index e9cdd754e136..31678ad165e3 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -636,16 +636,8 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { None } - /// Indicates whether the aggregation function is monotonic as a set function. A set - /// function is monotonically increasing if its value increases as its argument grows - /// (as a set). Formally, `f` is a monotonically increasing set function if `f(S) >= f(T)` - /// whenever `S` is a superset of `T`. - /// - /// For example `count` and `max` are monotonically increasing as their values always - /// increase (or stay the same) as new values are seen. - /// - /// `min` is monotonically decreasing as its value always decreases or stays - /// the same as new values are seen. + /// Indicates whether the aggregation function is monotonic as a set + /// function. See [`AggregateExprSetMonotonicity`] for details. fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { AggregateExprSetMonotonicity::NotMonotonic } @@ -832,26 +824,27 @@ pub mod aggregate_doc_sections { }; } -/// Status of an Aggregate Expression's Set Monotonicity -#[derive(Debug, Clone)] +/// Indicates whether an aggregation function is monotonic as a set +/// function. A set function is monotonically increasing if its value +/// increases as its argument grows (as a set). Formally, `f` is a +/// monotonically increasing set function if `f(S) >= f(T)` whenever `S` +/// is a superset of `T`. +/// +/// For example `COUNT` and `MAX` are monotonically increasing as their +/// values always increase (or stay the same) as new values are seen. On +/// the other hand, `MIN` is monotonically decreasing as its value always +/// decreases or stays the same as new values are seen. +#[derive(Debug, Clone, PartialEq)] pub enum AggregateExprSetMonotonicity { - /// Ordering exists as ascending + /// Aggregate value increases or stays the same as the input set grows. Increasing, - /// Ordering exists as descending + /// Aggregate value decreases or stays the same as the input set grows. Decreasing, - /// No ordering + /// Aggregate value may increase, decrease, or stay the same as the input + /// set grows. NotMonotonic, } -impl AggregateExprSetMonotonicity { - pub fn is_decreasing(&self) -> bool { - matches!(self, Self::Decreasing) - } - pub fn is_monotonic(&self) -> bool { - !matches!(self, Self::NotMonotonic) - } -} - #[cfg(test)] mod test { use crate::{AggregateUDF, AggregateUDFImpl}; diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 48eabe7b9cce..992004c66981 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -354,6 +354,8 @@ impl AggregateUDFImpl for Count { } fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { + // `COUNT` is monotonically increasing as it always increases or stays + // the same as new values are seen. AggregateExprSetMonotonicity::Increasing } } diff --git a/datafusion/functions-aggregate/src/min_max.rs b/datafusion/functions-aggregate/src/min_max.rs index a36a2f184593..6ef97f8d61a6 100644 --- a/datafusion/functions-aggregate/src/min_max.rs +++ b/datafusion/functions-aggregate/src/min_max.rs @@ -363,8 +363,8 @@ impl AggregateUDFImpl for Max { } fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { - // max is monotonically increasing as it always increases or - // stays the same as new values are seen + // `MAX` is monotonically increasing as it always increases or stays + // the same as new values are seen. AggregateExprSetMonotonicity::Increasing } } @@ -1191,8 +1191,8 @@ impl AggregateUDFImpl for Min { } fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { - // min is monotonically decreasing as it always decreases or - // stays the same as new values are seen + // `MIN` is monotonically decreasing as it always decreases or stays + // the same as new values are seen. AggregateExprSetMonotonicity::Decreasing } } diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index 1bb54734fb69..e5d7ef8555eb 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -256,8 +256,8 @@ impl AggregateUDFImpl for Sum { } fn set_monotonicity(&self, data_type: &DataType) -> AggregateExprSetMonotonicity { - // Sum is only monotonic if its input is unsigned - // TODO: Expand these utilizing statistics + // `SUM` is only monotonically increasing when its input is unsigned. + // TODO: Expand these utilizing statistics. match data_type { DataType::UInt8 => AggregateExprSetMonotonicity::Increasing, DataType::UInt16 => AggregateExprSetMonotonicity::Increasing, diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index ac86939f8cbb..3087bfbf9a4b 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -47,8 +47,9 @@ use datafusion_expr::{AggregateExprSetMonotonicity, AggregateUDF, ReversedUDAF}; use datafusion_expr_common::accumulator::Accumulator; use datafusion_expr_common::groups_accumulator::GroupsAccumulator; use datafusion_expr_common::type_coercion::aggregates::check_arg_count; -use datafusion_functions_aggregate_common::accumulator::AccumulatorArgs; -use datafusion_functions_aggregate_common::accumulator::StateFieldsArgs; +use datafusion_functions_aggregate_common::accumulator::{ + AccumulatorArgs, StateFieldsArgs, +}; use datafusion_functions_aggregate_common::order::AggregateOrderSensitivity; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; @@ -535,10 +536,8 @@ impl AggregateFunctionExpr { self.fun.default_value(data_type) } - /// Indicates whether the aggregation function is monotonic as a set function. A set - /// function is monotonically increasing if its value increases as its argument grows - /// (as a set). Formally, `f` is a monotonically increasing set function if `f(S) >= f(T)` - /// whenever `S` is a superset of `T`. + /// Indicates whether the aggregation function is monotonic as a set + /// function. See [`AggregateExprSetMonotonicity`] for details. pub fn set_monotonicity(&self) -> AggregateExprSetMonotonicity { let field = self.field(); let data_type = field.data_type(); @@ -550,11 +549,14 @@ impl AggregateFunctionExpr { // If the aggregate expressions are set-monotonic, the output data is // naturally ordered with it per group or partition. let monotonicity = self.set_monotonicity(); - if !monotonicity.is_monotonic() { + if monotonicity == AggregateExprSetMonotonicity::NotMonotonic { return None; } let expr = Arc::new(Column::new(self.name(), aggr_func_idx)); - let options = SortOptions::new(monotonicity.is_decreasing(), false); + let options = SortOptions::new( + monotonicity == AggregateExprSetMonotonicity::Decreasing, + false, + ); Some(PhysicalSortExpr { expr, options }) } } From 7efa2f338ca63eff7287cd37237548c28fa1ae31 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 30 Jan 2025 09:14:58 -0500 Subject: [PATCH 55/62] chore(deps): bump aws-sdk-ssooidc in /datafusion-cli (#14369) Bumps [aws-sdk-ssooidc](https://github.com/awslabs/aws-sdk-rust) from 1.57.1 to 1.58.0. - [Release notes](https://github.com/awslabs/aws-sdk-rust/releases) - [Commits](https://github.com/awslabs/aws-sdk-rust/commits) --- updated-dependencies: - dependency-name: aws-sdk-ssooidc dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion-cli/Cargo.lock | 4 ++-- datafusion-cli/Cargo.toml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 106b0b4c2fd7..09623c16aca2 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -538,9 +538,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.57.1" +version = "1.58.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a120ade4a44691b3c5c2ff2fa61b14ed331fdc218397f61ab48d66593012ae2a" +checksum = "8c8234fd024f7ac61c4e44ea008029bde934250f371efe7d4a39708397b1080c" dependencies = [ "aws-credential-types", "aws-runtime", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 8ef002e9f29c..4184455f19c7 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -36,7 +36,7 @@ aws-config = "=1.5.10" ## 1.53.0 requires a higher MSRV 1.81 so lock until DataFusion MSRV catches up aws-sdk-sso = "=1.56.0" ## 1.54.0 requires a higher MSRV 1.81 so lock until DataFusion MSRV catches up -aws-sdk-ssooidc = "=1.57.1" +aws-sdk-ssooidc = "=1.58.0" ## 1.54.1 requires a higher MSRV 1.81 so lock until DataFusion MSRV catches up aws-sdk-sts = "=1.57.0" # end pin aws-sdk crates From 0edc3d99ad63399696135d3bb3fc387b38803d1f Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 30 Jan 2025 09:15:15 -0500 Subject: [PATCH 56/62] Minor: include the number of files run in sqllogictest display (#14359) * Minor: include the number of files run in sqllogictest display * fmt * Update datafusion/sqllogictest/bin/sqllogictests.rs --- datafusion/sqllogictest/bin/sqllogictests.rs | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/datafusion/sqllogictest/bin/sqllogictests.rs b/datafusion/sqllogictest/bin/sqllogictests.rs index 58e383237065..c30aaf38ec9c 100644 --- a/datafusion/sqllogictest/bin/sqllogictests.rs +++ b/datafusion/sqllogictest/bin/sqllogictests.rs @@ -120,7 +120,9 @@ async fn run_tests() -> Result<()> { let start = Instant::now(); - let errors: Vec<_> = futures::stream::iter(read_test_files(&options)?) + let test_files = read_test_files(&options)?; + let num_tests = test_files.len(); + let errors: Vec<_> = futures::stream::iter(test_files) .map(|test_file| { let validator = if options.include_sqlite && test_file.relative_path.starts_with(SQLITE_PREFIX) @@ -184,7 +186,11 @@ async fn run_tests() -> Result<()> { .collect() .await; - m.println(format!("Completed in {}", HumanDuration(start.elapsed())))?; + m.println(format!( + "Completed {} test files in {}", + num_tests, + HumanDuration(start.elapsed()) + ))?; #[cfg(feature = "postgres")] terminate_postgres_container().await?; @@ -491,9 +497,7 @@ impl TestFile { } } -fn read_test_files<'a>( - options: &'a Options, -) -> Result + 'a>> { +fn read_test_files(options: &Options) -> Result> { let mut paths = read_dir_recursive(TEST_DIRECTORY)? .into_iter() .map(TestFile::new) @@ -516,7 +520,7 @@ fn read_test_files<'a>( paths.append(&mut sqlite_paths) } - Ok(Box::new(paths.into_iter())) + Ok(paths) } /// Parsed command line options From 54d62d6cacf5774d48fe804636b35d2d1ead0d5f Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 30 Jan 2025 18:58:48 +0300 Subject: [PATCH 57/62] Use a short name for set monotonicity --- datafusion/expr/src/lib.rs | 4 ++-- datafusion/expr/src/udaf.rs | 8 ++++---- datafusion/functions-aggregate/src/count.rs | 9 ++++----- datafusion/functions-aggregate/src/min_max.rs | 12 ++++++------ datafusion/functions-aggregate/src/sum.rs | 16 ++++++++-------- datafusion/physical-expr/src/aggregate.rs | 14 ++++++-------- 6 files changed, 30 insertions(+), 33 deletions(-) diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index 138e6d0617b5..aaa65c676a42 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -91,8 +91,8 @@ pub use partition_evaluator::PartitionEvaluator; pub use sqlparser; pub use table_source::{TableProviderFilterPushDown, TableSource, TableType}; pub use udaf::{ - aggregate_doc_sections, AggregateExprSetMonotonicity, AggregateUDF, AggregateUDFImpl, - ReversedUDAF, StatisticsArgs, + aggregate_doc_sections, AggregateUDF, AggregateUDFImpl, ReversedUDAF, + SetMonotonicity, StatisticsArgs, }; pub use udf::{ scalar_doc_sections, ReturnInfo, ReturnTypeArgs, ScalarFunctionArgs, ScalarUDF, diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 31678ad165e3..84a513b21b47 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -637,9 +637,9 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { } /// Indicates whether the aggregation function is monotonic as a set - /// function. See [`AggregateExprSetMonotonicity`] for details. - fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { - AggregateExprSetMonotonicity::NotMonotonic + /// function. See [`SetMonotonicity`] for details. + fn set_monotonicity(&self, _data_type: &DataType) -> SetMonotonicity { + SetMonotonicity::NotMonotonic } } @@ -835,7 +835,7 @@ pub mod aggregate_doc_sections { /// the other hand, `MIN` is monotonically decreasing as its value always /// decreases or stays the same as new values are seen. #[derive(Debug, Clone, PartialEq)] -pub enum AggregateExprSetMonotonicity { +pub enum SetMonotonicity { /// Aggregate value increases or stays the same as the input set grows. Increasing, /// Aggregate value decreases or stays the same as the input set grows. diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 992004c66981..fa04e1aca2c9 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -48,9 +48,8 @@ use datafusion_common::{ }; use datafusion_expr::function::StateFieldsArgs; use datafusion_expr::{ - function::AccumulatorArgs, utils::format_state_name, Accumulator, - AggregateExprSetMonotonicity, AggregateUDFImpl, Documentation, EmitTo, - GroupsAccumulator, Signature, Volatility, + function::AccumulatorArgs, utils::format_state_name, Accumulator, AggregateUDFImpl, + Documentation, EmitTo, GroupsAccumulator, SetMonotonicity, Signature, Volatility, }; use datafusion_expr::{Expr, ReversedUDAF, StatisticsArgs, TypeSignature}; use datafusion_functions_aggregate_common::aggregate::count_distinct::{ @@ -353,10 +352,10 @@ impl AggregateUDFImpl for Count { self.doc() } - fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { + fn set_monotonicity(&self, _data_type: &DataType) -> SetMonotonicity { // `COUNT` is monotonically increasing as it always increases or stays // the same as new values are seen. - AggregateExprSetMonotonicity::Increasing + SetMonotonicity::Increasing } } diff --git a/datafusion/functions-aggregate/src/min_max.rs b/datafusion/functions-aggregate/src/min_max.rs index 6ef97f8d61a6..da5ec739ad8d 100644 --- a/datafusion/functions-aggregate/src/min_max.rs +++ b/datafusion/functions-aggregate/src/min_max.rs @@ -56,8 +56,8 @@ use arrow::datatypes::{ use crate::min_max::min_max_bytes::MinMaxBytesAccumulator; use datafusion_common::ScalarValue; use datafusion_expr::{ - function::AccumulatorArgs, Accumulator, AggregateExprSetMonotonicity, - AggregateUDFImpl, Documentation, Signature, Volatility, + function::AccumulatorArgs, Accumulator, AggregateUDFImpl, Documentation, + SetMonotonicity, Signature, Volatility, }; use datafusion_expr::{GroupsAccumulator, StatisticsArgs}; use datafusion_macros::user_doc; @@ -362,10 +362,10 @@ impl AggregateUDFImpl for Max { self.doc() } - fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { + fn set_monotonicity(&self, _data_type: &DataType) -> SetMonotonicity { // `MAX` is monotonically increasing as it always increases or stays // the same as new values are seen. - AggregateExprSetMonotonicity::Increasing + SetMonotonicity::Increasing } } @@ -1190,10 +1190,10 @@ impl AggregateUDFImpl for Min { self.doc() } - fn set_monotonicity(&self, _data_type: &DataType) -> AggregateExprSetMonotonicity { + fn set_monotonicity(&self, _data_type: &DataType) -> SetMonotonicity { // `MIN` is monotonically decreasing as it always decreases or stays // the same as new values are seen. - AggregateExprSetMonotonicity::Decreasing + SetMonotonicity::Decreasing } } diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index e5d7ef8555eb..9615ca33a5f3 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -38,8 +38,8 @@ use datafusion_expr::function::AccumulatorArgs; use datafusion_expr::function::StateFieldsArgs; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ - Accumulator, AggregateExprSetMonotonicity, AggregateUDFImpl, Documentation, - GroupsAccumulator, ReversedUDAF, Signature, Volatility, + Accumulator, AggregateUDFImpl, Documentation, GroupsAccumulator, ReversedUDAF, + SetMonotonicity, Signature, Volatility, }; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; use datafusion_functions_aggregate_common::utils::Hashable; @@ -255,15 +255,15 @@ impl AggregateUDFImpl for Sum { self.doc() } - fn set_monotonicity(&self, data_type: &DataType) -> AggregateExprSetMonotonicity { + fn set_monotonicity(&self, data_type: &DataType) -> SetMonotonicity { // `SUM` is only monotonically increasing when its input is unsigned. // TODO: Expand these utilizing statistics. match data_type { - DataType::UInt8 => AggregateExprSetMonotonicity::Increasing, - DataType::UInt16 => AggregateExprSetMonotonicity::Increasing, - DataType::UInt32 => AggregateExprSetMonotonicity::Increasing, - DataType::UInt64 => AggregateExprSetMonotonicity::Increasing, - _ => AggregateExprSetMonotonicity::NotMonotonic, + DataType::UInt8 => SetMonotonicity::Increasing, + DataType::UInt16 => SetMonotonicity::Increasing, + DataType::UInt32 => SetMonotonicity::Increasing, + DataType::UInt64 => SetMonotonicity::Increasing, + _ => SetMonotonicity::NotMonotonic, } } } diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 3087bfbf9a4b..84406f50051f 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -43,7 +43,7 @@ use crate::expressions::Column; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow_schema::SortOptions; use datafusion_common::{internal_err, not_impl_err, Result, ScalarValue}; -use datafusion_expr::{AggregateExprSetMonotonicity, AggregateUDF, ReversedUDAF}; +use datafusion_expr::{AggregateUDF, ReversedUDAF, SetMonotonicity}; use datafusion_expr_common::accumulator::Accumulator; use datafusion_expr_common::groups_accumulator::GroupsAccumulator; use datafusion_expr_common::type_coercion::aggregates::check_arg_count; @@ -537,8 +537,8 @@ impl AggregateFunctionExpr { } /// Indicates whether the aggregation function is monotonic as a set - /// function. See [`AggregateExprSetMonotonicity`] for details. - pub fn set_monotonicity(&self) -> AggregateExprSetMonotonicity { + /// function. See [`SetMonotonicity`] for details. + pub fn set_monotonicity(&self) -> SetMonotonicity { let field = self.field(); let data_type = field.data_type(); self.fun.inner().set_monotonicity(data_type) @@ -549,14 +549,12 @@ impl AggregateFunctionExpr { // If the aggregate expressions are set-monotonic, the output data is // naturally ordered with it per group or partition. let monotonicity = self.set_monotonicity(); - if monotonicity == AggregateExprSetMonotonicity::NotMonotonic { + if monotonicity == SetMonotonicity::NotMonotonic { return None; } let expr = Arc::new(Column::new(self.name(), aggr_func_idx)); - let options = SortOptions::new( - monotonicity == AggregateExprSetMonotonicity::Decreasing, - false, - ); + let options = + SortOptions::new(monotonicity == SetMonotonicity::Decreasing, false); Some(PhysicalSortExpr { expr, options }) } } From 29e9a1c2703d9a43a4f3be4462d6e9ed4ad538ee Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Burak=20=C5=9Een?= Date: Thu, 30 Jan 2025 20:53:08 +0300 Subject: [PATCH 58/62] removed (#14370) --- datafusion/common/src/utils/mod.rs | 26 -------------------------- 1 file changed, 26 deletions(-) diff --git a/datafusion/common/src/utils/mod.rs b/datafusion/common/src/utils/mod.rs index 29d33fec14ab..c748f932395c 100644 --- a/datafusion/common/src/utils/mod.rs +++ b/datafusion/common/src/utils/mod.rs @@ -769,20 +769,6 @@ pub fn set_difference, S: Borrow>( .collect() } -/// Checks whether the given index sequence is monotonically non-decreasing. -pub fn is_sorted>(sequence: impl IntoIterator) -> bool { - // TODO: Remove this function when `is_sorted` graduates from Rust nightly. - let mut previous = 0; - for item in sequence.into_iter() { - let current = *item.borrow(); - if current < previous { - return false; - } - previous = current; - } - true -} - /// Find indices of each element in `targets` inside `items`. If one of the /// elements is absent in `items`, returns an error. pub fn find_indices>( @@ -1171,18 +1157,6 @@ mod tests { assert_eq!(set_difference([3, 4, 0], [4, 1, 2]), vec![3, 0]); } - #[test] - fn test_is_sorted() { - assert!(is_sorted::([])); - assert!(is_sorted([0])); - assert!(is_sorted([0, 3, 4])); - assert!(is_sorted([0, 1, 2])); - assert!(is_sorted([0, 1, 4])); - assert!(is_sorted([0usize; 0])); - assert!(is_sorted([1, 2])); - assert!(!is_sorted([3, 2])); - } - #[test] fn test_find_indices() -> Result<()> { assert_eq!(find_indices(&[0, 3, 4], [0, 3, 4])?, vec![0, 1, 2]); From c077ef5638c76cfca7af1967497aae5d2fd069a0 Mon Sep 17 00:00:00 2001 From: logan-keede <68557630+logan-keede@users.noreply.github.com> Date: Thu, 30 Jan 2025 23:23:58 +0530 Subject: [PATCH 59/62] move information_schema to datafusion-catalog (#14364) * move information_schema to datafusion-catalog * fix: formatting * fix: doctests import * Remove unecessary datafuson-catalog dependency * remove some more unecessary dependencies * Update datafusion-cli/Carglo.ock * fix: doctest dependency --------- Co-authored-by: Andrew Lamb --- datafusion-cli/Cargo.lock | 6 +- datafusion/catalog/Cargo.toml | 6 +- datafusion/catalog/src/async.rs | 2 +- .../src}/information_schema.rs | 22 +- datafusion/catalog/src/lib.rs | 243 +++++++++++++++++- .../datasource => catalog/src}/streaming.rs | 8 +- datafusion/catalog/src/table.rs | 4 +- datafusion/core/src/catalog_common/mod.rs | 243 ------------------ datafusion/core/src/datasource/mod.rs | 4 +- .../core/src/execution/session_state.rs | 12 +- datafusion/core/src/lib.rs | 2 +- datafusion/core/tests/memory_limit/mod.rs | 2 +- .../tests/cases/substrait_validations.rs | 3 +- datafusion/substrait/tests/utils.rs | 3 +- 14 files changed, 280 insertions(+), 280 deletions(-) rename datafusion/{core/src/catalog_common => catalog/src}/information_schema.rs (98%) rename datafusion/{core/src/datasource => catalog/src}/streaming.rs (93%) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 09623c16aca2..1db4f5fc789c 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1271,15 +1271,19 @@ dependencies = [ name = "datafusion-catalog" version = "44.0.0" dependencies = [ - "arrow-schema", + "arrow", "async-trait", "dashmap", "datafusion-common", "datafusion-execution", "datafusion-expr", "datafusion-physical-plan", + "datafusion-sql", + "futures", "itertools 0.14.0", + "log", "parking_lot", + "sqlparser", ] [[package]] diff --git a/datafusion/catalog/Cargo.toml b/datafusion/catalog/Cargo.toml index bcc258c7a7f4..749457855ca2 100644 --- a/datafusion/catalog/Cargo.toml +++ b/datafusion/catalog/Cargo.toml @@ -28,15 +28,19 @@ rust-version.workspace = true version.workspace = true [dependencies] -arrow-schema = { workspace = true } +arrow = { workspace = true } async-trait = { workspace = true } dashmap = { workspace = true } datafusion-common = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } datafusion-physical-plan = { workspace = true } +datafusion-sql = { workspace = true } +futures = { workspace = true } itertools = { workspace = true } +log = { workspace = true } parking_lot = { workspace = true } +sqlparser = { workspace = true } [dev-dependencies] tokio = { workspace = true } diff --git a/datafusion/catalog/src/async.rs b/datafusion/catalog/src/async.rs index a244261b91e2..5d7a51ad7123 100644 --- a/datafusion/catalog/src/async.rs +++ b/datafusion/catalog/src/async.rs @@ -430,7 +430,7 @@ mod tests { }, }; - use arrow_schema::SchemaRef; + use arrow::datatypes::SchemaRef; use async_trait::async_trait; use datafusion_common::{error::Result, Statistics, TableReference}; use datafusion_execution::config::SessionConfig; diff --git a/datafusion/core/src/catalog_common/information_schema.rs b/datafusion/catalog/src/information_schema.rs similarity index 98% rename from datafusion/core/src/catalog_common/information_schema.rs rename to datafusion/catalog/src/information_schema.rs index ce3092acfdf1..e68e636989f8 100644 --- a/datafusion/core/src/catalog_common/information_schema.rs +++ b/datafusion/catalog/src/information_schema.rs @@ -19,31 +19,29 @@ //! //! [Information Schema]: https://en.wikipedia.org/wiki/Information_schema -use crate::catalog::{CatalogProviderList, SchemaProvider, TableProvider}; -use crate::datasource::streaming::StreamingTable; -use crate::execution::context::TaskContext; -use crate::logical_expr::{TableType, Volatility}; -use crate::physical_plan::stream::RecordBatchStreamAdapter; -use crate::physical_plan::SendableRecordBatchStream; -use crate::{ - config::{ConfigEntry, ConfigOptions}, - physical_plan::streaming::PartitionStream, -}; +use crate::streaming::StreamingTable; +use crate::{CatalogProviderList, SchemaProvider, TableProvider}; +use arrow::array::builder::{BooleanBuilder, UInt8Builder}; use arrow::{ array::{StringBuilder, UInt64Builder}, datatypes::{DataType, Field, Schema, SchemaRef}, record_batch::RecordBatch, }; -use arrow_array::builder::{BooleanBuilder, UInt8Builder}; use async_trait::async_trait; +use datafusion_common::config::{ConfigEntry, ConfigOptions}; use datafusion_common::error::Result; use datafusion_common::DataFusionError; +use datafusion_execution::TaskContext; use datafusion_expr::{AggregateUDF, ScalarUDF, Signature, TypeSignature, WindowUDF}; +use datafusion_expr::{TableType, Volatility}; +use datafusion_physical_plan::stream::RecordBatchStreamAdapter; +use datafusion_physical_plan::streaming::PartitionStream; +use datafusion_physical_plan::SendableRecordBatchStream; use std::collections::{HashMap, HashSet}; use std::fmt::Debug; use std::{any::Any, sync::Arc}; -pub(crate) const INFORMATION_SCHEMA: &str = "information_schema"; +pub const INFORMATION_SCHEMA: &str = "information_schema"; pub(crate) const TABLES: &str = "tables"; pub(crate) const VIEWS: &str = "views"; pub(crate) const COLUMNS: &str = "columns"; diff --git a/datafusion/catalog/src/lib.rs b/datafusion/catalog/src/lib.rs index 28410eb76fab..63d75fa3ef0c 100644 --- a/datafusion/catalog/src/lib.rs +++ b/datafusion/catalog/src/lib.rs @@ -18,23 +18,264 @@ //! Interfaces and default implementations of catalogs and schemas. //! //! Implementations +//! * Information schema: [`information_schema`] //! * Simple memory based catalog: [`MemoryCatalogProviderList`], [`MemoryCatalogProvider`], [`MemorySchemaProvider`] pub mod memory; +pub use datafusion_sql::{ResolvedTableReference, TableReference}; pub use memory::{ MemoryCatalogProvider, MemoryCatalogProviderList, MemorySchemaProvider, }; +use std::collections::BTreeSet; +use std::ops::ControlFlow; mod r#async; mod catalog; mod dynamic_file; +pub mod information_schema; mod schema; mod session; mod table; - pub use catalog::*; pub use dynamic_file::catalog::*; pub use r#async::*; pub use schema::*; pub use session::*; pub use table::*; +pub mod streaming; + +/// Collects all tables and views referenced in the SQL statement. CTEs are collected separately. +/// This can be used to determine which tables need to be in the catalog for a query to be planned. +/// +/// # Returns +/// +/// A `(table_refs, ctes)` tuple, the first element contains table and view references and the second +/// element contains any CTE aliases that were defined and possibly referenced. +/// +/// ## Example +/// +/// ``` +/// # use datafusion_sql::parser::DFParser; +/// # use datafusion_catalog::resolve_table_references; +/// let query = "SELECT a FROM foo where x IN (SELECT y FROM bar)"; +/// let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); +/// let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); +/// assert_eq!(table_refs.len(), 2); +/// assert_eq!(table_refs[0].to_string(), "bar"); +/// assert_eq!(table_refs[1].to_string(), "foo"); +/// assert_eq!(ctes.len(), 0); +/// ``` +/// +/// ## Example with CTEs +/// +/// ``` +/// # use datafusion_sql::parser::DFParser; +/// # use datafusion_catalog::resolve_table_references; +/// let query = "with my_cte as (values (1), (2)) SELECT * from my_cte;"; +/// let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); +/// let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); +/// assert_eq!(table_refs.len(), 0); +/// assert_eq!(ctes.len(), 1); +/// assert_eq!(ctes[0].to_string(), "my_cte"); +/// ``` +pub fn resolve_table_references( + statement: &datafusion_sql::parser::Statement, + enable_ident_normalization: bool, +) -> datafusion_common::Result<(Vec, Vec)> { + use datafusion_sql::parser::{ + CopyToSource, CopyToStatement, Statement as DFStatement, + }; + use datafusion_sql::planner::object_name_to_table_reference; + use information_schema::INFORMATION_SCHEMA; + use information_schema::INFORMATION_SCHEMA_TABLES; + use sqlparser::ast::*; + + struct RelationVisitor { + relations: BTreeSet, + all_ctes: BTreeSet, + ctes_in_scope: Vec, + } + + impl RelationVisitor { + /// Record the reference to `relation`, if it's not a CTE reference. + fn insert_relation(&mut self, relation: &ObjectName) { + if !self.relations.contains(relation) + && !self.ctes_in_scope.contains(relation) + { + self.relations.insert(relation.clone()); + } + } + } + + impl Visitor for RelationVisitor { + type Break = (); + + fn pre_visit_relation(&mut self, relation: &ObjectName) -> ControlFlow<()> { + self.insert_relation(relation); + ControlFlow::Continue(()) + } + + fn pre_visit_query(&mut self, q: &Query) -> ControlFlow { + if let Some(with) = &q.with { + for cte in &with.cte_tables { + // The non-recursive CTE name is not in scope when evaluating the CTE itself, so this is valid: + // `WITH t AS (SELECT * FROM t) SELECT * FROM t` + // Where the first `t` refers to a predefined table. So we are careful here + // to visit the CTE first, before putting it in scope. + if !with.recursive { + // This is a bit hackish as the CTE will be visited again as part of visiting `q`, + // but thankfully `insert_relation` is idempotent. + cte.visit(self); + } + self.ctes_in_scope + .push(ObjectName(vec![cte.alias.name.clone()])); + } + } + ControlFlow::Continue(()) + } + + fn post_visit_query(&mut self, q: &Query) -> ControlFlow { + if let Some(with) = &q.with { + for _ in &with.cte_tables { + // Unwrap: We just pushed these in `pre_visit_query` + self.all_ctes.insert(self.ctes_in_scope.pop().unwrap()); + } + } + ControlFlow::Continue(()) + } + + fn pre_visit_statement(&mut self, statement: &Statement) -> ControlFlow<()> { + if let Statement::ShowCreate { + obj_type: ShowCreateObject::Table | ShowCreateObject::View, + obj_name, + } = statement + { + self.insert_relation(obj_name) + } + + // SHOW statements will later be rewritten into a SELECT from the information_schema + let requires_information_schema = matches!( + statement, + Statement::ShowFunctions { .. } + | Statement::ShowVariable { .. } + | Statement::ShowStatus { .. } + | Statement::ShowVariables { .. } + | Statement::ShowCreate { .. } + | Statement::ShowColumns { .. } + | Statement::ShowTables { .. } + | Statement::ShowCollation { .. } + ); + if requires_information_schema { + for s in INFORMATION_SCHEMA_TABLES { + self.relations.insert(ObjectName(vec![ + Ident::new(INFORMATION_SCHEMA), + Ident::new(*s), + ])); + } + } + ControlFlow::Continue(()) + } + } + + let mut visitor = RelationVisitor { + relations: BTreeSet::new(), + all_ctes: BTreeSet::new(), + ctes_in_scope: vec![], + }; + + fn visit_statement(statement: &DFStatement, visitor: &mut RelationVisitor) { + match statement { + DFStatement::Statement(s) => { + let _ = s.as_ref().visit(visitor); + } + DFStatement::CreateExternalTable(table) => { + visitor.relations.insert(table.name.clone()); + } + DFStatement::CopyTo(CopyToStatement { source, .. }) => match source { + CopyToSource::Relation(table_name) => { + visitor.insert_relation(table_name); + } + CopyToSource::Query(query) => { + query.visit(visitor); + } + }, + DFStatement::Explain(explain) => visit_statement(&explain.statement, visitor), + } + } + + visit_statement(statement, &mut visitor); + + let table_refs = visitor + .relations + .into_iter() + .map(|x| object_name_to_table_reference(x, enable_ident_normalization)) + .collect::>()?; + let ctes = visitor + .all_ctes + .into_iter() + .map(|x| object_name_to_table_reference(x, enable_ident_normalization)) + .collect::>()?; + Ok((table_refs, ctes)) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn resolve_table_references_shadowed_cte() { + use datafusion_sql::parser::DFParser; + + // An interesting edge case where the `t` name is used both as an ordinary table reference + // and as a CTE reference. + let query = "WITH t AS (SELECT * FROM t) SELECT * FROM t"; + let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); + let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); + assert_eq!(table_refs.len(), 1); + assert_eq!(ctes.len(), 1); + assert_eq!(ctes[0].to_string(), "t"); + assert_eq!(table_refs[0].to_string(), "t"); + + // UNION is a special case where the CTE is not in scope for the second branch. + let query = "(with t as (select 1) select * from t) union (select * from t)"; + let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); + let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); + assert_eq!(table_refs.len(), 1); + assert_eq!(ctes.len(), 1); + assert_eq!(ctes[0].to_string(), "t"); + assert_eq!(table_refs[0].to_string(), "t"); + + // Nested CTEs are also handled. + // Here the first `u` is a CTE, but the second `u` is a table reference. + // While `t` is always a CTE. + let query = "(with t as (with u as (select 1) select * from u) select * from u cross join t)"; + let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); + let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); + assert_eq!(table_refs.len(), 1); + assert_eq!(ctes.len(), 2); + assert_eq!(ctes[0].to_string(), "t"); + assert_eq!(ctes[1].to_string(), "u"); + assert_eq!(table_refs[0].to_string(), "u"); + } + + #[test] + fn resolve_table_references_recursive_cte() { + use datafusion_sql::parser::DFParser; + + let query = " + WITH RECURSIVE nodes AS ( + SELECT 1 as id + UNION ALL + SELECT id + 1 as id + FROM nodes + WHERE id < 10 + ) + SELECT * FROM nodes + "; + let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); + let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); + assert_eq!(table_refs.len(), 0); + assert_eq!(ctes.len(), 1); + assert_eq!(ctes[0].to_string(), "nodes"); + } +} diff --git a/datafusion/core/src/datasource/streaming.rs b/datafusion/catalog/src/streaming.rs similarity index 93% rename from datafusion/core/src/datasource/streaming.rs rename to datafusion/catalog/src/streaming.rs index 1da3c3da9c89..654e6755d7d4 100644 --- a/datafusion/core/src/datasource/streaming.rs +++ b/datafusion/catalog/src/streaming.rs @@ -23,12 +23,12 @@ use std::sync::Arc; use arrow::datatypes::SchemaRef; use async_trait::async_trait; -use crate::datasource::TableProvider; -use crate::physical_plan::streaming::{PartitionStream, StreamingTableExec}; -use crate::physical_plan::ExecutionPlan; -use datafusion_catalog::Session; +use crate::Session; +use crate::TableProvider; use datafusion_common::{plan_err, Result}; use datafusion_expr::{Expr, TableType}; +use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; +use datafusion_physical_plan::ExecutionPlan; use log::debug; /// A [`TableProvider`] that streams a set of [`PartitionStream`] diff --git a/datafusion/catalog/src/table.rs b/datafusion/catalog/src/table.rs index 3c8960495588..88d2d8bde51e 100644 --- a/datafusion/catalog/src/table.rs +++ b/datafusion/catalog/src/table.rs @@ -21,7 +21,7 @@ use std::fmt::Debug; use std::sync::Arc; use crate::session::Session; -use arrow_schema::SchemaRef; +use arrow::datatypes::SchemaRef; use async_trait::async_trait; use datafusion_common::Result; use datafusion_common::{not_impl_err, Constraints, Statistics}; @@ -202,7 +202,7 @@ pub trait TableProvider: Debug + Sync + Send { /// ```rust /// # use std::any::Any; /// # use std::sync::Arc; - /// # use arrow_schema::SchemaRef; + /// # use arrow::datatypes::SchemaRef; /// # use async_trait::async_trait; /// # use datafusion_catalog::{TableProvider, Session}; /// # use datafusion_common::Result; diff --git a/datafusion/core/src/catalog_common/mod.rs b/datafusion/core/src/catalog_common/mod.rs index 45fb6ddae1d6..213afb32405e 100644 --- a/datafusion/core/src/catalog_common/mod.rs +++ b/datafusion/core/src/catalog_common/mod.rs @@ -18,250 +18,7 @@ //! Interfaces and default implementations of catalogs and schemas. //! //! Implementations -//! * Information schema: [`information_schema`] //! * Listing schema: [`listing_schema`] -pub mod information_schema; pub mod listing_schema; pub use crate::catalog::{CatalogProvider, CatalogProviderList, SchemaProvider}; - -pub use datafusion_sql::{ResolvedTableReference, TableReference}; - -use std::collections::BTreeSet; -use std::ops::ControlFlow; - -/// Collects all tables and views referenced in the SQL statement. CTEs are collected separately. -/// This can be used to determine which tables need to be in the catalog for a query to be planned. -/// -/// # Returns -/// -/// A `(table_refs, ctes)` tuple, the first element contains table and view references and the second -/// element contains any CTE aliases that were defined and possibly referenced. -/// -/// ## Example -/// -/// ``` -/// # use datafusion_sql::parser::DFParser; -/// # use datafusion::catalog_common::resolve_table_references; -/// let query = "SELECT a FROM foo where x IN (SELECT y FROM bar)"; -/// let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); -/// let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); -/// assert_eq!(table_refs.len(), 2); -/// assert_eq!(table_refs[0].to_string(), "bar"); -/// assert_eq!(table_refs[1].to_string(), "foo"); -/// assert_eq!(ctes.len(), 0); -/// ``` -/// -/// ## Example with CTEs -/// -/// ``` -/// # use datafusion_sql::parser::DFParser; -/// # use datafusion::catalog_common::resolve_table_references; -/// let query = "with my_cte as (values (1), (2)) SELECT * from my_cte;"; -/// let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); -/// let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); -/// assert_eq!(table_refs.len(), 0); -/// assert_eq!(ctes.len(), 1); -/// assert_eq!(ctes[0].to_string(), "my_cte"); -/// ``` -pub fn resolve_table_references( - statement: &datafusion_sql::parser::Statement, - enable_ident_normalization: bool, -) -> datafusion_common::Result<(Vec, Vec)> { - use crate::sql::planner::object_name_to_table_reference; - use datafusion_sql::parser::{ - CopyToSource, CopyToStatement, Statement as DFStatement, - }; - use information_schema::INFORMATION_SCHEMA; - use information_schema::INFORMATION_SCHEMA_TABLES; - use sqlparser::ast::*; - - struct RelationVisitor { - relations: BTreeSet, - all_ctes: BTreeSet, - ctes_in_scope: Vec, - } - - impl RelationVisitor { - /// Record the reference to `relation`, if it's not a CTE reference. - fn insert_relation(&mut self, relation: &ObjectName) { - if !self.relations.contains(relation) - && !self.ctes_in_scope.contains(relation) - { - self.relations.insert(relation.clone()); - } - } - } - - impl Visitor for RelationVisitor { - type Break = (); - - fn pre_visit_relation(&mut self, relation: &ObjectName) -> ControlFlow<()> { - self.insert_relation(relation); - ControlFlow::Continue(()) - } - - fn pre_visit_query(&mut self, q: &Query) -> ControlFlow { - if let Some(with) = &q.with { - for cte in &with.cte_tables { - // The non-recursive CTE name is not in scope when evaluating the CTE itself, so this is valid: - // `WITH t AS (SELECT * FROM t) SELECT * FROM t` - // Where the first `t` refers to a predefined table. So we are careful here - // to visit the CTE first, before putting it in scope. - if !with.recursive { - // This is a bit hackish as the CTE will be visited again as part of visiting `q`, - // but thankfully `insert_relation` is idempotent. - cte.visit(self); - } - self.ctes_in_scope - .push(ObjectName(vec![cte.alias.name.clone()])); - } - } - ControlFlow::Continue(()) - } - - fn post_visit_query(&mut self, q: &Query) -> ControlFlow { - if let Some(with) = &q.with { - for _ in &with.cte_tables { - // Unwrap: We just pushed these in `pre_visit_query` - self.all_ctes.insert(self.ctes_in_scope.pop().unwrap()); - } - } - ControlFlow::Continue(()) - } - - fn pre_visit_statement(&mut self, statement: &Statement) -> ControlFlow<()> { - if let Statement::ShowCreate { - obj_type: ShowCreateObject::Table | ShowCreateObject::View, - obj_name, - } = statement - { - self.insert_relation(obj_name) - } - - // SHOW statements will later be rewritten into a SELECT from the information_schema - let requires_information_schema = matches!( - statement, - Statement::ShowFunctions { .. } - | Statement::ShowVariable { .. } - | Statement::ShowStatus { .. } - | Statement::ShowVariables { .. } - | Statement::ShowCreate { .. } - | Statement::ShowColumns { .. } - | Statement::ShowTables { .. } - | Statement::ShowCollation { .. } - ); - if requires_information_schema { - for s in INFORMATION_SCHEMA_TABLES { - self.relations.insert(ObjectName(vec![ - Ident::new(INFORMATION_SCHEMA), - Ident::new(*s), - ])); - } - } - ControlFlow::Continue(()) - } - } - - let mut visitor = RelationVisitor { - relations: BTreeSet::new(), - all_ctes: BTreeSet::new(), - ctes_in_scope: vec![], - }; - - fn visit_statement(statement: &DFStatement, visitor: &mut RelationVisitor) { - match statement { - DFStatement::Statement(s) => { - let _ = s.as_ref().visit(visitor); - } - DFStatement::CreateExternalTable(table) => { - visitor.relations.insert(table.name.clone()); - } - DFStatement::CopyTo(CopyToStatement { source, .. }) => match source { - CopyToSource::Relation(table_name) => { - visitor.insert_relation(table_name); - } - CopyToSource::Query(query) => { - query.visit(visitor); - } - }, - DFStatement::Explain(explain) => visit_statement(&explain.statement, visitor), - } - } - - visit_statement(statement, &mut visitor); - - let table_refs = visitor - .relations - .into_iter() - .map(|x| object_name_to_table_reference(x, enable_ident_normalization)) - .collect::>()?; - let ctes = visitor - .all_ctes - .into_iter() - .map(|x| object_name_to_table_reference(x, enable_ident_normalization)) - .collect::>()?; - Ok((table_refs, ctes)) -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn resolve_table_references_shadowed_cte() { - use datafusion_sql::parser::DFParser; - - // An interesting edge case where the `t` name is used both as an ordinary table reference - // and as a CTE reference. - let query = "WITH t AS (SELECT * FROM t) SELECT * FROM t"; - let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); - let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); - assert_eq!(table_refs.len(), 1); - assert_eq!(ctes.len(), 1); - assert_eq!(ctes[0].to_string(), "t"); - assert_eq!(table_refs[0].to_string(), "t"); - - // UNION is a special case where the CTE is not in scope for the second branch. - let query = "(with t as (select 1) select * from t) union (select * from t)"; - let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); - let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); - assert_eq!(table_refs.len(), 1); - assert_eq!(ctes.len(), 1); - assert_eq!(ctes[0].to_string(), "t"); - assert_eq!(table_refs[0].to_string(), "t"); - - // Nested CTEs are also handled. - // Here the first `u` is a CTE, but the second `u` is a table reference. - // While `t` is always a CTE. - let query = "(with t as (with u as (select 1) select * from u) select * from u cross join t)"; - let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); - let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); - assert_eq!(table_refs.len(), 1); - assert_eq!(ctes.len(), 2); - assert_eq!(ctes[0].to_string(), "t"); - assert_eq!(ctes[1].to_string(), "u"); - assert_eq!(table_refs[0].to_string(), "u"); - } - - #[test] - fn resolve_table_references_recursive_cte() { - use datafusion_sql::parser::DFParser; - - let query = " - WITH RECURSIVE nodes AS ( - SELECT 1 as id - UNION ALL - SELECT id + 1 as id - FROM nodes - WHERE id < 10 - ) - SELECT * FROM nodes - "; - let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); - let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); - assert_eq!(table_refs.len(), 0); - assert_eq!(ctes.len(), 1); - assert_eq!(ctes[0].to_string(), "nodes"); - } -} diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 7d3fe9ddd751..aeb7d6569de9 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -33,12 +33,9 @@ pub mod provider; pub mod schema_adapter; mod statistics; pub mod stream; -pub mod streaming; pub mod view; // backwards compatibility -pub use datafusion_execution::object_store; - pub use self::default_table_source::{ provider_as_source, source_as_provider, DefaultTableSource, }; @@ -46,6 +43,7 @@ pub use self::memory::MemTable; pub use self::view::ViewTable; pub use crate::catalog::TableProvider; pub use crate::logical_expr::TableType; +pub use datafusion_execution::object_store; pub use statistics::get_statistics_with_limit; use arrow_schema::{Schema, SortOptions}; diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 6c3349625f04..f83b8528d1e8 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -24,15 +24,15 @@ use std::fmt::Debug; use std::sync::Arc; use crate::catalog::{CatalogProviderList, SchemaProvider, TableProviderFactory}; -use crate::catalog_common::information_schema::{ - InformationSchemaProvider, INFORMATION_SCHEMA, -}; use crate::datasource::cte_worktable::CteWorkTable; use crate::datasource::file_format::{format_as_file_type, FileFormatFactory}; use crate::datasource::provider_as_source; use crate::execution::context::{EmptySerializerRegistry, FunctionFactory, QueryPlanner}; use crate::execution::SessionStateDefaults; use crate::physical_planner::{DefaultPhysicalPlanner, PhysicalPlanner}; +use datafusion_catalog::information_schema::{ + InformationSchemaProvider, INFORMATION_SCHEMA, +}; use datafusion_catalog::MemoryCatalogProviderList; use arrow_schema::{DataType, SchemaRef}; @@ -529,16 +529,16 @@ impl SessionState { /// Resolve all table references in the SQL statement. Does not include CTE references. /// - /// See [`catalog::resolve_table_references`] for more information. + /// See [`datafusion_catalog::resolve_table_references`] for more information. /// - /// [`catalog::resolve_table_references`]: crate::catalog_common::resolve_table_references + /// [`datafusion_catalog::resolve_table_references`]: datafusion_catalog::resolve_table_references pub fn resolve_table_references( &self, statement: &Statement, ) -> datafusion_common::Result> { let enable_ident_normalization = self.config.options().sql_parser.enable_ident_normalization; - let (table_refs, _) = crate::catalog_common::resolve_table_references( + let (table_refs, _) = datafusion_catalog::resolve_table_references( statement, enable_ident_normalization, )?; diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index 5d917e1673f1..780b22983393 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -309,7 +309,7 @@ //! //! [`ListingTable`]: crate::datasource::listing::ListingTable //! [`MemTable`]: crate::datasource::memory::MemTable -//! [`StreamingTable`]: crate::datasource::streaming::StreamingTable +//! [`StreamingTable`]: datafusion_catalog::streaming::StreamingTable //! //! ## Plan Representations //! diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index 77e4b491da6d..212ffdaaa2a5 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -28,7 +28,6 @@ use arrow::record_batch::RecordBatch; use arrow_array::{ArrayRef, DictionaryArray}; use arrow_schema::SortOptions; use datafusion::assert_batches_eq; -use datafusion::datasource::streaming::StreamingTable; use datafusion::datasource::{MemTable, TableProvider}; use datafusion::execution::disk_manager::DiskManagerConfig; use datafusion::execution::runtime_env::RuntimeEnvBuilder; @@ -38,6 +37,7 @@ use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::streaming::PartitionStream; use datafusion::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; use datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion_catalog::streaming::StreamingTable; use datafusion_catalog::Session; use datafusion_common::{assert_contains, Result}; use datafusion_execution::memory_pool::{ diff --git a/datafusion/substrait/tests/cases/substrait_validations.rs b/datafusion/substrait/tests/cases/substrait_validations.rs index 8357e0a8621d..a7d4d4aa82fc 100644 --- a/datafusion/substrait/tests/cases/substrait_validations.rs +++ b/datafusion/substrait/tests/cases/substrait_validations.rs @@ -22,8 +22,7 @@ mod tests { mod schema_compatibility { use crate::utils::test::read_json; use datafusion::arrow::datatypes::{DataType, Field}; - use datafusion::catalog_common::TableReference; - use datafusion::common::{DFSchema, Result}; + use datafusion::common::{DFSchema, Result, TableReference}; use datafusion::datasource::empty::EmptyTable; use datafusion::prelude::SessionContext; use datafusion_substrait::logical_plan::consumer::from_substrait_plan; diff --git a/datafusion/substrait/tests/utils.rs b/datafusion/substrait/tests/utils.rs index f8b2d87d317e..0034cc27bf6e 100644 --- a/datafusion/substrait/tests/utils.rs +++ b/datafusion/substrait/tests/utils.rs @@ -17,8 +17,7 @@ #[cfg(test)] pub mod test { - use datafusion::catalog_common::TableReference; - use datafusion::common::{substrait_datafusion_err, substrait_err}; + use datafusion::common::{substrait_datafusion_err, substrait_err, TableReference}; use datafusion::datasource::empty::EmptyTable; use datafusion::datasource::TableProvider; use datafusion::error::Result; From 07ee09aef19cc96905492adda84af597b1d3355c Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 30 Jan 2025 15:02:33 -0500 Subject: [PATCH 60/62] Unpin aws sdk dependencies (#14361) --- datafusion-cli/Cargo.lock | 37 ++++++++++++++----------------------- datafusion-cli/Cargo.toml | 15 +++++---------- 2 files changed, 19 insertions(+), 33 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 1db4f5fc789c..f4374af625e2 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -449,9 +449,9 @@ checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" [[package]] name = "aws-config" -version = "1.5.10" +version = "1.5.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b49afaa341e8dd8577e1a2200468f98956d6eda50bcf4a53246cc00174ba924" +checksum = "dc47e70fc35d054c8fcd296d47a61711f043ac80534a10b4f741904f81e73a90" dependencies = [ "aws-credential-types", "aws-runtime", @@ -460,7 +460,7 @@ dependencies = [ "aws-sdk-sts", "aws-smithy-async", "aws-smithy-http", - "aws-smithy-json 0.60.7", + "aws-smithy-json", "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", @@ -516,15 +516,15 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.56.0" +version = "1.57.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12e057fdcb8842de9b83592a70f5b4da0ee10bc0ad278247da1425a742a444d7" +checksum = "c54bab121fe1881a74c338c5f723d1592bf3b53167f80268a1274f404e1acc38" dependencies = [ "aws-credential-types", "aws-runtime", "aws-smithy-async", "aws-smithy-http", - "aws-smithy-json 0.61.2", + "aws-smithy-json", "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", @@ -546,7 +546,7 @@ dependencies = [ "aws-runtime", "aws-smithy-async", "aws-smithy-http", - "aws-smithy-json 0.61.2", + "aws-smithy-json", "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", @@ -560,15 +560,15 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.57.0" +version = "1.58.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "115fd4fb663817ed595a5ee4f1649d7aacd861d47462323cb37576ce89271b93" +checksum = "ba60e1d519d6f23a9df712c04fdeadd7872ac911c84b2f62a8bda92e129b7962" dependencies = [ "aws-credential-types", "aws-runtime", "aws-smithy-async", "aws-smithy-http", - "aws-smithy-json 0.61.2", + "aws-smithy-json", "aws-smithy-query", "aws-smithy-runtime", "aws-smithy-runtime-api", @@ -635,15 +635,6 @@ dependencies = [ "tracing", ] -[[package]] -name = "aws-smithy-json" -version = "0.60.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4683df9469ef09468dad3473d129960119a0d3593617542b7d52086c8486f2d6" -dependencies = [ - "aws-smithy-types", -] - [[package]] name = "aws-smithy-json" version = "0.61.2" @@ -884,9 +875,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.16.0" +version = "3.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" +checksum = "1628fb46dfa0b37568d12e5edd512553eccf6a22a78e8bde00bb4aed84d5bdbf" [[package]] name = "byteorder" @@ -4571,9 +4562,9 @@ checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" [[package]] name = "winnow" -version = "0.6.25" +version = "0.6.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad699df48212c6cc6eb4435f35500ac6fd3b9913324f938aea302022ce19d310" +checksum = "1e90edd2ac1aa278a5c4599b1d89cf03074b610800f866d4026dc199d7929a28" dependencies = [ "memchr", ] diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 4184455f19c7..89777ea1a4cb 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -30,17 +30,12 @@ readme = "README.md" [dependencies] arrow = { version = "54.0.0" } -async-trait = "0.1.73" -## 1.5.13 requires a hiher MSRV 1.81 so lock until DataFusion MSRV catches up -aws-config = "=1.5.10" -## 1.53.0 requires a higher MSRV 1.81 so lock until DataFusion MSRV catches up -aws-sdk-sso = "=1.56.0" -## 1.54.0 requires a higher MSRV 1.81 so lock until DataFusion MSRV catches up -aws-sdk-ssooidc = "=1.58.0" -## 1.54.1 requires a higher MSRV 1.81 so lock until DataFusion MSRV catches up -aws-sdk-sts = "=1.57.0" -# end pin aws-sdk crates +async-trait = "0.1.0" +aws-config = "1.5.0" aws-credential-types = "1.2.0" +aws-sdk-sso = "1.57.0" +aws-sdk-ssooidc = "1.57.0" +aws-sdk-sts = "1.57.0" clap = { version = "4.5.27", features = ["derive", "cargo"] } datafusion = { path = "../datafusion/core", version = "44.0.0", features = [ "avro", From 11435ded45434edd34ed0b7738b9b9b2194f1774 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 30 Jan 2025 21:59:49 +0100 Subject: [PATCH 61/62] Core: Fix incorrect searched CASE optimization (#14349) * Fix incorrect searched CASE optimization There is an optimization for searched CASE where values are of boolean type. It was converting the expression like CASE WHEN X THEN A WHEN Y THEN B .. [ ELSE D ] END into (X AND A) OR (Y AND NOT X AND B) [ OR (NOT (X OR Y) AND D) ] This had the following problems - does not work for nullable conditions. If X is nullable, we cannot use NOT (X) to compliment it. We need to use `X IS DISTINCT FROM true` - it does not work correctly when some conditions are nullable and other values are false. E.g. X=NULL, A=true, Y=NULL, B=true, D=false, the CASE should return false, but the boolean expression will simplify to `(NULL AND ..) OR (NULL AND ..) OR (false)` which is NULL, not false - thus we use `X` for truthness check of `X`, we need to test `X IS NOT DISTINCT FROM true` - it did not work correctly when default D is missing, but conditions do not evaluate to NULL. CASE's result should be NULL but was false. This commit fixes that optimization. * Fix complexity comment --- .../simplify_expressions/expr_simplifier.rs | 98 ++++++++++++++----- datafusion/sqllogictest/test_files/case.slt | 20 +++- 2 files changed, 89 insertions(+), 29 deletions(-) diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index 74d2ce0b6be9..29f3d7cbda39 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -1385,29 +1385,26 @@ impl TreeNodeRewriter for Simplifier<'_, S> { when_then_expr, else_expr, }) if !when_then_expr.is_empty() - && when_then_expr.len() < 3 // The rewrite is O(n!) so limit to small number + && when_then_expr.len() < 3 // The rewrite is O(n²) so limit to small number && info.is_boolean_type(&when_then_expr[0].1)? => { - // The disjunction of all the when predicates encountered so far + // String disjunction of all the when predicates encountered so far. Not nullable. let mut filter_expr = lit(false); // The disjunction of all the cases let mut out_expr = lit(false); for (when, then) in when_then_expr { - let case_expr = when - .as_ref() - .clone() - .and(filter_expr.clone().not()) - .and(*then); + let when = is_exactly_true(*when, info)?; + let case_expr = + when.clone().and(filter_expr.clone().not()).and(*then); out_expr = out_expr.or(case_expr); - filter_expr = filter_expr.or(*when); + filter_expr = filter_expr.or(when); } - if let Some(else_expr) = else_expr { - let case_expr = filter_expr.not().and(*else_expr); - out_expr = out_expr.or(case_expr); - } + let else_expr = else_expr.map(|b| *b).unwrap_or_else(lit_bool_null); + let case_expr = filter_expr.not().and(else_expr); + out_expr = out_expr.or(case_expr); // Do a first pass at simplification out_expr.rewrite(self)? @@ -1881,6 +1878,19 @@ fn inlist_except(mut l1: InList, l2: &InList) -> Result { Ok(Expr::InList(l1)) } +/// Returns expression testing a boolean `expr` for being exactly `true` (not `false` or NULL). +fn is_exactly_true(expr: Expr, info: &impl SimplifyInfo) -> Result { + if !info.nullable(&expr)? { + Ok(expr) + } else { + Ok(Expr::BinaryExpr(BinaryExpr { + left: Box::new(expr), + op: Operator::IsNotDistinctFrom, + right: Box::new(lit(true)), + })) + } +} + #[cfg(test)] mod tests { use crate::simplify_expressions::SimplifyContext; @@ -3272,12 +3282,12 @@ mod tests { simplify(Expr::Case(Case::new( None, vec![( - Box::new(col("c2").not_eq(lit(false))), + Box::new(col("c2_non_null").not_eq(lit(false))), Box::new(lit("ok").eq(lit("not_ok"))), )], - Some(Box::new(col("c2").eq(lit(true)))), + Some(Box::new(col("c2_non_null").eq(lit(true)))), ))), - col("c2").not().and(col("c2")) // #1716 + lit(false) // #1716 ); // CASE WHEN c2 != false THEN "ok" == "ok" ELSE c2 @@ -3292,12 +3302,12 @@ mod tests { simplify(simplify(Expr::Case(Case::new( None, vec![( - Box::new(col("c2").not_eq(lit(false))), + Box::new(col("c2_non_null").not_eq(lit(false))), Box::new(lit("ok").eq(lit("ok"))), )], - Some(Box::new(col("c2").eq(lit(true)))), + Some(Box::new(col("c2_non_null").eq(lit(true)))), )))), - col("c2") + col("c2_non_null") ); // CASE WHEN ISNULL(c2) THEN true ELSE c2 @@ -3328,12 +3338,12 @@ mod tests { simplify(simplify(Expr::Case(Case::new( None, vec![ - (Box::new(col("c1")), Box::new(lit(true)),), - (Box::new(col("c2")), Box::new(lit(false)),), + (Box::new(col("c1_non_null")), Box::new(lit(true)),), + (Box::new(col("c2_non_null")), Box::new(lit(false)),), ], Some(Box::new(lit(true))), )))), - col("c1").or(col("c1").not().and(col("c2").not())) + col("c1_non_null").or(col("c1_non_null").not().and(col("c2_non_null").not())) ); // CASE WHEN c1 then true WHEN c2 then true ELSE false @@ -3347,13 +3357,53 @@ mod tests { simplify(simplify(Expr::Case(Case::new( None, vec![ - (Box::new(col("c1")), Box::new(lit(true)),), - (Box::new(col("c2")), Box::new(lit(false)),), + (Box::new(col("c1_non_null")), Box::new(lit(true)),), + (Box::new(col("c2_non_null")), Box::new(lit(false)),), ], Some(Box::new(lit(true))), )))), - col("c1").or(col("c1").not().and(col("c2").not())) + col("c1_non_null").or(col("c1_non_null").not().and(col("c2_non_null").not())) + ); + + // CASE WHEN c > 0 THEN true END AS c1 + assert_eq!( + simplify(simplify(Expr::Case(Case::new( + None, + vec![(Box::new(col("c3").gt(lit(0_i64))), Box::new(lit(true)))], + None, + )))), + not_distinct_from(col("c3").gt(lit(0_i64)), lit(true)).or(distinct_from( + col("c3").gt(lit(0_i64)), + lit(true) + ) + .and(lit_bool_null())) ); + + // CASE WHEN c > 0 THEN true ELSE false END AS c1 + assert_eq!( + simplify(simplify(Expr::Case(Case::new( + None, + vec![(Box::new(col("c3").gt(lit(0_i64))), Box::new(lit(true)))], + Some(Box::new(lit(false))), + )))), + not_distinct_from(col("c3").gt(lit(0_i64)), lit(true)) + ); + } + + fn distinct_from(left: impl Into, right: impl Into) -> Expr { + Expr::BinaryExpr(BinaryExpr { + left: Box::new(left.into()), + op: Operator::IsDistinctFrom, + right: Box::new(right.into()), + }) + } + + fn not_distinct_from(left: impl Into, right: impl Into) -> Expr { + Expr::BinaryExpr(BinaryExpr { + left: Box::new(left.into()), + op: Operator::IsNotDistinctFrom, + right: Box::new(right.into()), + }) } #[test] diff --git a/datafusion/sqllogictest/test_files/case.slt b/datafusion/sqllogictest/test_files/case.slt index 157bfb8a02aa..a339c2aa037e 100644 --- a/datafusion/sqllogictest/test_files/case.slt +++ b/datafusion/sqllogictest/test_files/case.slt @@ -289,12 +289,22 @@ query B select case when a=1 then false end from foo; ---- false -false -false -false -false -false +NULL +NULL +NULL +NULL +NULL +query IBB +SELECT c, + CASE WHEN c > 0 THEN true END AS c1, + CASE WHEN c > 0 THEN true ELSE false END AS c2 +FROM (VALUES (1), (0), (-1), (NULL)) AS t(c) +---- +1 true true +0 NULL false +-1 NULL false +NULL NULL false statement ok drop table foo From 53728b30aa95be7865a0ad35e70ec574312a69bc Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 31 Jan 2025 09:47:16 +0800 Subject: [PATCH 62/62] Improve speed of `median` by implementing special `GroupsAccumulator` (#13681) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * draft of `MedianGroupAccumulator`. * impl `state`. * impl rest methods of `MedianGroupsAccumulator`. * improve comments. * use `MedianGroupsAccumulator`. * remove unused import. * add `group_median_table` to test group median. * complete group median test cases in aggregate slt. * fix type of state. * Clippy * Fmt * add fuzzy tests for median. * fix decimal. * fix clippy. * improve comments. * add median cases with nulls. * Update datafusion/functions-aggregate/src/median.rs Co-authored-by: Andrew Lamb * use `OffsetBuffer::new_unchecked` in `convert_to_state`. * add todo. * remove assert and switch to i32 try from. * return error when try from failed. --------- Co-authored-by: Daniël Heres Co-authored-by: Andrew Lamb --- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 20 ++ datafusion/functions-aggregate/src/median.rs | 262 +++++++++++++++++- .../sqllogictest/test_files/aggregate.slt | 261 +++++++++++++++++ 3 files changed, 541 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 09d0c8d5ca2e..bcd88bae739a 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -148,6 +148,26 @@ async fn test_count() { .await; } +#[tokio::test(flavor = "multi_thread")] +async fn test_median() { + let data_gen_config = baseline_config(); + + // Queries like SELECT median(a), median(distinct) FROM fuzz_table GROUP BY b + let query_builder = QueryBuilder::new() + .with_table_name("fuzz_table") + .with_aggregate_function("median") + .with_distinct_aggregate_function("median") + // median only works on numeric columns + .with_aggregate_arguments(data_gen_config.numeric_columns()) + .set_group_by_columns(data_gen_config.all_columns()); + + AggregationFuzzerBuilder::from(data_gen_config) + .add_query_builder(query_builder) + .build() + .run() + .await; +} + /// Return a standard set of columns for testing data generation /// /// Includes numeric and string types diff --git a/datafusion/functions-aggregate/src/median.rs b/datafusion/functions-aggregate/src/median.rs index 70f192c32ae1..defbbe737a9d 100644 --- a/datafusion/functions-aggregate/src/median.rs +++ b/datafusion/functions-aggregate/src/median.rs @@ -20,7 +20,11 @@ use std::fmt::{Debug, Formatter}; use std::mem::{size_of, size_of_val}; use std::sync::Arc; -use arrow::array::{downcast_integer, ArrowNumericType}; +use arrow::array::{ + downcast_integer, ArrowNumericType, BooleanArray, ListArray, PrimitiveArray, + PrimitiveBuilder, +}; +use arrow::buffer::{OffsetBuffer, ScalarBuffer}; use arrow::{ array::{ArrayRef, AsArray}, datatypes::{ @@ -33,12 +37,17 @@ use arrow::array::Array; use arrow::array::ArrowNativeTypeOp; use arrow::datatypes::{ArrowNativeType, ArrowPrimitiveType}; -use datafusion_common::{DataFusionError, HashSet, Result, ScalarValue}; +use datafusion_common::{ + internal_datafusion_err, internal_err, DataFusionError, HashSet, Result, ScalarValue, +}; use datafusion_expr::function::StateFieldsArgs; use datafusion_expr::{ function::AccumulatorArgs, utils::format_state_name, Accumulator, AggregateUDFImpl, Documentation, Signature, Volatility, }; +use datafusion_expr::{EmitTo, GroupsAccumulator}; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::accumulate; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::nulls::filtered_null_mask; use datafusion_functions_aggregate_common::utils::Hashable; use datafusion_macros::user_doc; @@ -165,6 +174,45 @@ impl AggregateUDFImpl for Median { } } + fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { + !args.is_distinct + } + + fn create_groups_accumulator( + &self, + args: AccumulatorArgs, + ) -> Result> { + let num_args = args.exprs.len(); + if num_args != 1 { + return internal_err!( + "median should only have 1 arg, but found num args:{}", + args.exprs.len() + ); + } + + let dt = args.exprs[0].data_type(args.schema)?; + + macro_rules! helper { + ($t:ty, $dt:expr) => { + Ok(Box::new(MedianGroupsAccumulator::<$t>::new($dt))) + }; + } + + downcast_integer! { + dt => (helper, dt), + DataType::Float16 => helper!(Float16Type, dt), + DataType::Float32 => helper!(Float32Type, dt), + DataType::Float64 => helper!(Float64Type, dt), + DataType::Decimal128(_, _) => helper!(Decimal128Type, dt), + DataType::Decimal256(_, _) => helper!(Decimal256Type, dt), + _ => Err(DataFusionError::NotImplemented(format!( + "MedianGroupsAccumulator not supported for {} with {}", + args.name, + dt, + ))), + } + } + fn aliases(&self) -> &[String] { &[] } @@ -230,6 +278,216 @@ impl Accumulator for MedianAccumulator { } } +/// The median groups accumulator accumulates the raw input values +/// +/// For calculating the accurate medians of groups, we need to store all values +/// of groups before final evaluation. +/// So values in each group will be stored in a `Vec`, and the total group values +/// will be actually organized as a `Vec>`. +/// +#[derive(Debug)] +struct MedianGroupsAccumulator { + data_type: DataType, + group_values: Vec>, +} + +impl MedianGroupsAccumulator { + pub fn new(data_type: DataType) -> Self { + Self { + data_type, + group_values: Vec::new(), + } + } +} + +impl GroupsAccumulator for MedianGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "single argument to update_batch"); + let values = values[0].as_primitive::(); + + // Push the `not nulls + not filtered` row into its group + self.group_values.resize(total_num_groups, Vec::new()); + accumulate( + group_indices, + values, + opt_filter, + |group_index, new_value| { + self.group_values[group_index].push(new_value); + }, + ); + + Ok(()) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + // Since aggregate filter should be applied in partial stage, in final stage there should be no filter + _opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "one argument to merge_batch"); + + // The merged values should be organized like as a `ListArray` which is nullable + // (input with nulls usually generated from `convert_to_state`), but `inner array` of + // `ListArray` is `non-nullable`. + // + // Following is the possible and impossible input `values`: + // + // # Possible values + // ```text + // group 0: [1, 2, 3] + // group 1: null (list array is nullable) + // group 2: [6, 7, 8] + // ... + // group n: [...] + // ``` + // + // # Impossible values + // ```text + // group x: [1, 2, null] (values in list array is non-nullable) + // ``` + // + let input_group_values = values[0].as_list::(); + + // Ensure group values big enough + self.group_values.resize(total_num_groups, Vec::new()); + + // Extend values to related groups + // TODO: avoid using iterator of the `ListArray`, this will lead to + // many calls of `slice` of its ``inner array`, and `slice` is not + // so efficient(due to the calculation of `null_count` for each `slice`). + group_indices + .iter() + .zip(input_group_values.iter()) + .for_each(|(&group_index, values_opt)| { + if let Some(values) = values_opt { + let values = values.as_primitive::(); + self.group_values[group_index].extend(values.values().iter()); + } + }); + + Ok(()) + } + + fn state(&mut self, emit_to: EmitTo) -> Result> { + // Emit values + let emit_group_values = emit_to.take_needed(&mut self.group_values); + + // Build offsets + let mut offsets = Vec::with_capacity(self.group_values.len() + 1); + offsets.push(0); + let mut cur_len = 0_i32; + for group_value in &emit_group_values { + cur_len += group_value.len() as i32; + offsets.push(cur_len); + } + // TODO: maybe we can use `OffsetBuffer::new_unchecked` like what in `convert_to_state`, + // but safety should be considered more carefully here(and I am not sure if it can get + // performance improvement when we introduce checks to keep the safety...). + // + // Can see more details in: + // https://github.com/apache/datafusion/pull/13681#discussion_r1931209791 + // + let offsets = OffsetBuffer::new(ScalarBuffer::from(offsets)); + + // Build inner array + let flatten_group_values = + emit_group_values.into_iter().flatten().collect::>(); + let group_values_array = + PrimitiveArray::::new(ScalarBuffer::from(flatten_group_values), None) + .with_data_type(self.data_type.clone()); + + // Build the result list array + let result_list_array = ListArray::new( + Arc::new(Field::new_list_field(self.data_type.clone(), true)), + offsets, + Arc::new(group_values_array), + None, + ); + + Ok(vec![Arc::new(result_list_array)]) + } + + fn evaluate(&mut self, emit_to: EmitTo) -> Result { + // Emit values + let emit_group_values = emit_to.take_needed(&mut self.group_values); + + // Calculate median for each group + let mut evaluate_result_builder = + PrimitiveBuilder::::new().with_data_type(self.data_type.clone()); + for values in emit_group_values { + let median = calculate_median::(values); + evaluate_result_builder.append_option(median); + } + + Ok(Arc::new(evaluate_result_builder.finish())) + } + + fn convert_to_state( + &self, + values: &[ArrayRef], + opt_filter: Option<&BooleanArray>, + ) -> Result> { + assert_eq!(values.len(), 1, "one argument to merge_batch"); + + let input_array = values[0].as_primitive::(); + + // Directly convert the input array to states, each row will be + // seen as a respective group. + // For detail, the `input_array` will be converted to a `ListArray`. + // And if row is `not null + not filtered`, it will be converted to a list + // with only one element; otherwise, this row in `ListArray` will be set + // to null. + + // Reuse values buffer in `input_array` to build `values` in `ListArray` + let values = PrimitiveArray::::new(input_array.values().clone(), None) + .with_data_type(self.data_type.clone()); + + // `offsets` in `ListArray`, each row as a list element + let offset_end = i32::try_from(input_array.len()).map_err(|e| { + internal_datafusion_err!( + "cast array_len to i32 failed in convert_to_state of group median, err:{e:?}" + ) + })?; + let offsets = (0..=offset_end).collect::>(); + // Safety: all checks in `OffsetBuffer::new` are ensured to pass + let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(offsets)) }; + + // `nulls` for converted `ListArray` + let nulls = filtered_null_mask(opt_filter, input_array); + + let converted_list_array = ListArray::new( + Arc::new(Field::new_list_field(self.data_type.clone(), true)), + offsets, + Arc::new(values), + nulls, + ); + + Ok(vec![Arc::new(converted_list_array)]) + } + + fn supports_convert_to_state(&self) -> bool { + true + } + + fn size(&self) -> usize { + self.group_values + .iter() + .map(|values| values.capacity() * size_of::()) + .sum::() + // account for size of self.grou_values too + + self.group_values.capacity() * size_of::>() + } +} + /// The distinct median accumulator accumulates the raw input values /// as `ScalarValue`s /// diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index bd3b40089519..4838911649bd 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -67,6 +67,62 @@ statement ok CREATE TABLE test (c1 BIGINT,c2 BIGINT) as values (0,null), (1,1), (null,1), (3,2), (3,2) +statement ok +CREATE TABLE group_median_table_non_nullable ( + col_group STRING NOT NULL, + col_i8 TINYINT NOT NULL, + col_i16 SMALLINT NOT NULL, + col_i32 INT NOT NULL, + col_i64 BIGINT NOT NULL, + col_u8 TINYINT UNSIGNED NOT NULL, + col_u16 SMALLINT UNSIGNED NOT NULL, + col_u32 INT UNSIGNED NOT NULL, + col_u64 BIGINT UNSIGNED NOT NULL, + col_f32 FLOAT NOT NULL, + col_f64 DOUBLE NOT NULL, + col_f64_nan DOUBLE NOT NULL, + col_decimal128 DECIMAL(10, 4) NOT NULL, + col_decimal256 NUMERIC(10, 4) NOT NULL +) as VALUES +( 'group0', -128, -32768, -2147483648, arrow_cast(-9223372036854775808,'Int64'), 0, 0, 0, arrow_cast(0,'UInt64'), 1.1, 1.1, 1.1, 0.0001, 0.0001 ), +( 'group0', -128, -32768, -2147483648, arrow_cast(-9223372036854775808,'Int64'), 0, 0, 0, arrow_cast(0,'UInt64'), 4.4, 4.4, 1.1, 0.0002, 0.0002 ), +( 'group0', 100, 100, 100, arrow_cast(100,'Int64'), 100, 100, 100, arrow_cast(100,'UInt64'), 3.3, 3.3, arrow_cast('NAN','Float64'), 0.0003, 0.0003 ), +( 'group0', 127, 32767, 2147483647, arrow_cast(9223372036854775807,'Int64'), 255, 65535, 4294967295, 18446744073709551615, 2.2, 2.2, arrow_cast('NAN','Float64'), 0.0004, 0.0004 ), +( 'group1', -128, -32768, -2147483648, arrow_cast(-9223372036854775808,'Int64'), 0, 0, 0, arrow_cast(0,'UInt64'), 1.1, 1.1, 1.1, 0.0001, 0.0001 ), +( 'group1', -128, -32768, -2147483648, arrow_cast(-9223372036854775808,'Int64'), 0, 0, 0, arrow_cast(0,'UInt64'), 4.4, 4.4, arrow_cast('NAN','Float64'), 0.0002, 0.0002 ), +( 'group1', 100, 100, 100, arrow_cast(100,'Int64'), 101, 100, 100, arrow_cast(100,'UInt64'), 3.3, 3.3, arrow_cast('NAN','Float64'), 0.0003, 0.0003 ), +( 'group1', 125, 32766, 2147483646, arrow_cast(9223372036854775806,'Int64'), 100, 101, 4294967294, arrow_cast(100,'UInt64'), 3.2, 5.5, arrow_cast('NAN','Float64'), 0.0004, 0.0004 ), +( 'group1', 127, 32767, 2147483647, arrow_cast(9223372036854775807,'Int64'), 255, 65535, 4294967295, 18446744073709551615, 2.2, 2.2, arrow_cast('NAN','Float64'), 0.0005, 0.0005 ) + +statement ok +CREATE TABLE group_median_table_nullable ( + col_group STRING NOT NULL, + col_i8 TINYINT, + col_i16 SMALLINT, + col_i32 INT, + col_i64 BIGINT, + col_u8 TINYINT UNSIGNED, + col_u16 SMALLINT UNSIGNED, + col_u32 INT UNSIGNED, + col_u64 BIGINT UNSIGNED, + col_f32 FLOAT, + col_f64 DOUBLE, + col_f64_nan DOUBLE, + col_decimal128 DECIMAL(10, 4), + col_decimal256 NUMERIC(10, 4) +) as VALUES +( 'group0', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL ), +( 'group0', -128, -32768, -2147483648, arrow_cast(-9223372036854775808,'Int64'), 0, 0, 0, arrow_cast(0,'UInt64'), 1.1, 1.1, 1.1, 0.0001, 0.0001 ), +( 'group0', -128, -32768, -2147483648, arrow_cast(-9223372036854775808,'Int64'), 0, 0, 0, arrow_cast(0,'UInt64'), 4.4, 4.4, 1.1, 0.0002, 0.0002 ), +( 'group0', 100, 100, 100, arrow_cast(100,'Int64'), 100, 100, 100, arrow_cast(100,'UInt64'), 3.3, 3.3, arrow_cast('NAN','Float64'), 0.0003, 0.0003 ), +( 'group0', 127, 32767, 2147483647, arrow_cast(9223372036854775807,'Int64'), 255, 65535, 4294967295, 18446744073709551615, 2.2, 2.2, arrow_cast('NAN','Float64'), 0.0004, 0.0004 ), +( 'group1', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL ), +( 'group1', -128, -32768, -2147483648, arrow_cast(-9223372036854775808,'Int64'), 0, 0, 0, arrow_cast(0,'UInt64'), 1.1, 1.1, 1.1, 0.0001, 0.0001 ), +( 'group1', -128, -32768, -2147483648, arrow_cast(-9223372036854775808,'Int64'), 0, 0, 0, arrow_cast(0,'UInt64'), 4.4, 4.4, arrow_cast('NAN','Float64'), 0.0002, 0.0002 ), +( 'group1', 100, 100, 100, arrow_cast(100,'Int64'), 101, 100, 100, arrow_cast(100,'UInt64'), 3.3, 3.3, arrow_cast('NAN','Float64'), 0.0003, 0.0003 ), +( 'group1', 125, 32766, 2147483646, arrow_cast(9223372036854775806,'Int64'), 100, 101, 4294967294, arrow_cast(100,'UInt64'), 3.2, 5.5, arrow_cast('NAN','Float64'), 0.0004, 0.0004 ), +( 'group1', 127, 32767, 2147483647, arrow_cast(9223372036854775807,'Int64'), 255, 65535, 4294967295, 18446744073709551615, 2.2, 2.2, arrow_cast('NAN','Float64'), 0.0005, 0.0005 ) + ####### # Error tests ####### @@ -6203,3 +6259,208 @@ physical_plan 14)--------------PlaceholderRowExec 15)------------ProjectionExec: expr=[1 as id, 2 as foo] 16)--------------PlaceholderRowExec + +####### +# Group median test +####### + +# group median i8 non-nullable +query TI rowsort +SELECT col_group, median(col_i8) FROM group_median_table_non_nullable GROUP BY col_group +---- +group0 -14 +group1 100 + +# group median i16 non-nullable +query TI +SELECT col_group, median(col_i16) FROM group_median_table_non_nullable GROUP BY col_group +---- +group0 -16334 +group1 100 + +# group median i32 non-nullable +query TI +SELECT col_group, median(col_i32) FROM group_median_table_non_nullable GROUP BY col_group +---- +group0 -1073741774 +group1 100 + +# group median i64 non-nullable +query TI +SELECT col_group, median(col_i64) FROM group_median_table_non_nullable GROUP BY col_group +---- +group0 -4611686018427387854 +group1 100 + +# group median u8 non-nullable +query TI rowsort +SELECT col_group, median(col_u8) FROM group_median_table_non_nullable GROUP BY col_group +---- +group0 50 +group1 100 + +# group median u16 non-nullable +query TI +SELECT col_group, median(col_u16) FROM group_median_table_non_nullable GROUP BY col_group +---- +group0 50 +group1 100 + +# group median u32 non-nullable +query TI +SELECT col_group, median(col_u32) FROM group_median_table_non_nullable GROUP BY col_group +---- +group0 50 +group1 100 + +# group median u64 non-nullable +query TI +SELECT col_group, median(col_u64) FROM group_median_table_non_nullable GROUP BY col_group +---- +group0 50 +group1 100 + +# group median f32 non-nullable +query TR +SELECT col_group, median(col_f32) FROM group_median_table_non_nullable GROUP BY col_group +---- +group0 2.75 +group1 3.2 + +# group median f64 non-nullable +query TR +SELECT col_group, median(col_f64) FROM group_median_table_non_nullable GROUP BY col_group +---- +group0 2.75 +group1 3.3 + +# group median f64_nan non-nullable +query TR +SELECT col_group, median(col_f64_nan) FROM group_median_table_non_nullable GROUP BY col_group +---- +group0 NaN +group1 NaN + +# group median decimal128 non-nullable +query TR +SELECT col_group, median(col_decimal128) FROM group_median_table_non_nullable GROUP BY col_group +---- +group0 0.0002 +group1 0.0003 + +# group median decimal256 non-nullable +query TR +SELECT col_group, median(col_decimal256) FROM group_median_table_non_nullable GROUP BY col_group +---- +group0 0.0002 +group1 0.0003 + +# group median i8 nullable +query TI rowsort +SELECT col_group, median(col_i8) FROM group_median_table_nullable GROUP BY col_group +---- +group0 -14 +group1 100 + +# group median i16 nullable +query TI rowsort +SELECT col_group, median(col_i16) FROM group_median_table_nullable GROUP BY col_group +---- +group0 -16334 +group1 100 + +# group median i32 nullable +query TI rowsort +SELECT col_group, median(col_i32) FROM group_median_table_nullable GROUP BY col_group +---- +group0 -1073741774 +group1 100 + +# group median i64 nullable +query TI rowsort +SELECT col_group, median(col_i64) FROM group_median_table_nullable GROUP BY col_group +---- +group0 -4611686018427387854 +group1 100 + +# group median u8 nullable +query TI rowsort +SELECT col_group, median(col_u8) FROM group_median_table_nullable GROUP BY col_group +---- +group0 50 +group1 100 + +# group median u16 nullable +query TI rowsort +SELECT col_group, median(col_u16) FROM group_median_table_nullable GROUP BY col_group +---- +group0 50 +group1 100 + +# group median u32 nullable +query TI rowsort +SELECT col_group, median(col_u32) FROM group_median_table_nullable GROUP BY col_group +---- +group0 50 +group1 100 + +# group median u64 nullable +query TI rowsort +SELECT col_group, median(col_u64) FROM group_median_table_nullable GROUP BY col_group +---- +group0 50 +group1 100 + +# group median f32 nullable +query TR rowsort +SELECT col_group, median(col_f32) FROM group_median_table_nullable GROUP BY col_group +---- +group0 2.75 +group1 3.2 + +# group median f64 nullable +query TR rowsort +SELECT col_group, median(col_f64) FROM group_median_table_nullable GROUP BY col_group +---- +group0 2.75 +group1 3.3 + +# group median f64_nan nullable +query TR rowsort +SELECT col_group, median(col_f64_nan) FROM group_median_table_nullable GROUP BY col_group +---- +group0 NaN +group1 NaN + +# group median decimal128 nullable +query TR rowsort +SELECT col_group, median(col_decimal128) FROM group_median_table_nullable GROUP BY col_group +---- +group0 0.0002 +group1 0.0003 + +# group median decimal256 nullable +query TR rowsort +SELECT col_group, median(col_decimal256) FROM group_median_table_nullable GROUP BY col_group +---- +group0 0.0002 +group1 0.0003 + +# median with all nulls +statement ok +create table group_median_all_nulls( + a STRING NOT NULL, + b INT +) AS VALUES +( 'group0', NULL), +( 'group0', NULL), +( 'group0', NULL), +( 'group1', NULL), +( 'group1', NULL), +( 'group1', NULL) + +query TIT rowsort +SELECT a, median(b), arrow_typeof(median(b)) FROM group_median_all_nulls GROUP BY a +---- +group0 NULL Int32 +group1 NULL Int32