From a2919b6f205962233546ea215efcbda137030c28 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 16 Jan 2025 15:21:51 +0300 Subject: [PATCH 01/28] 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/28] 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/28] 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/28] 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/28] 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/28] 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/28] 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/28] 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/28] 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/28] 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/28] 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/28] 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/28] 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/28] 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 972c56f5da89f64130971467da8f6c9e0e251a95 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 29 Jan 2025 15:16:36 +0300 Subject: [PATCH 15/28] 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 16/28] 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 17/28] 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 18/28] 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 19/28] 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 20/28] 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 21/28] 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 22/28] 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 23/28] 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 ba7b94f12be95ba0d27ddf483e50989cbcf7d5a2 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 30 Jan 2025 10:44:35 +0300 Subject: [PATCH 24/28] 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 25/28] 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 7822613c7a3914bacb1eb172e16d567c973ed49f Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 30 Jan 2025 14:56:34 +0300 Subject: [PATCH 26/28] 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 27/28] 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 54d62d6cacf5774d48fe804636b35d2d1ead0d5f Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 30 Jan 2025 18:58:48 +0300 Subject: [PATCH 28/28] 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 }) } }