From 192211f9ec67ca83f905b0c70625d94553a7718f Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sun, 5 May 2024 10:59:04 +0800 Subject: [PATCH 01/41] move accumulate Signed-off-by: jayzhan211 --- datafusion/physical-expr-common/Cargo.toml | 3 +++ .../aggregate/groups_accumulator/accumulate.rs | 8 ++++---- .../src/aggregate/groups_accumulator/mod.rs | 18 ++++++++++++++++++ .../physical-expr-common/src/aggregate/mod.rs | 1 + .../src/aggregate/groups_accumulator/mod.rs | 4 +++- 5 files changed, 29 insertions(+), 5 deletions(-) rename datafusion/{physical-expr => physical-expr-common}/src/aggregate/groups_accumulator/accumulate.rs (99%) create mode 100644 datafusion/physical-expr-common/src/aggregate/groups_accumulator/mod.rs diff --git a/datafusion/physical-expr-common/Cargo.toml b/datafusion/physical-expr-common/Cargo.toml index d1202c83d526..d1683c894d6d 100644 --- a/datafusion/physical-expr-common/Cargo.toml +++ b/datafusion/physical-expr-common/Cargo.toml @@ -39,3 +39,6 @@ path = "src/lib.rs" arrow = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-expr = { workspace = true } + +[dev-dependencies] +rand = { workspace = true } \ No newline at end of file diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/accumulate.rs b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/accumulate.rs similarity index 99% rename from datafusion/physical-expr/src/aggregate/groups_accumulator/accumulate.rs rename to datafusion/physical-expr-common/src/aggregate/groups_accumulator/accumulate.rs index 9850b002e40e..0326d6d87cf0 100644 --- a/datafusion/physical-expr/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/accumulate.rs @@ -20,8 +20,8 @@ //! [`GroupsAccumulator`]: datafusion_expr::GroupsAccumulator use arrow::datatypes::ArrowPrimitiveType; -use arrow_array::{Array, BooleanArray, PrimitiveArray}; -use arrow_buffer::{BooleanBuffer, BooleanBufferBuilder, NullBuffer}; +use arrow::array::{Array, BooleanArray, PrimitiveArray, BooleanBufferBuilder}; +use arrow::buffer::{BooleanBuffer, NullBuffer}; use datafusion_expr::EmitTo; /// Track the accumulator null state per row: if any values for that @@ -462,8 +462,8 @@ fn initialize_builder( mod test { use super::*; - use arrow_array::UInt32Array; - use hashbrown::HashSet; + use arrow::array::UInt32Array; + use std::collections::HashSet; use rand::{rngs::ThreadRng, Rng}; #[test] diff --git a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/mod.rs b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/mod.rs new file mode 100644 index 000000000000..471ffba69490 --- /dev/null +++ b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/mod.rs @@ -0,0 +1,18 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +pub mod accumulate; \ No newline at end of file diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 448af634176a..4e1b53dbe7c7 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +pub mod groups_accumulator; pub mod utils; use arrow::datatypes::{DataType, Field, Schema}; diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs b/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs index de090badd349..0581c3971e4a 100644 --- a/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs +++ b/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs @@ -15,7 +15,9 @@ // specific language governing permissions and limitations // under the License. -pub(crate) mod accumulate; +pub(crate) mod accumulate { + pub use datafusion_physical_expr_common::aggregate::groups_accumulator::accumulate::*; +} mod adapter; pub use accumulate::NullState; pub use adapter::GroupsAccumulatorAdapter; From 403ee1d2426c839842a17eb7b046fc0aa33ba75f Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sun, 5 May 2024 11:00:39 +0800 Subject: [PATCH 02/41] move prim_op Signed-off-by: jayzhan211 --- .../src/aggregate/groups_accumulator/mod.rs | 3 ++- .../src/aggregate/groups_accumulator/prim_op.rs | 4 ++-- .../physical-expr/src/aggregate/groups_accumulator/mod.rs | 4 +++- 3 files changed, 7 insertions(+), 4 deletions(-) rename datafusion/{physical-expr => physical-expr-common}/src/aggregate/groups_accumulator/prim_op.rs (97%) diff --git a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/mod.rs b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/mod.rs index 471ffba69490..fffdae11bec2 100644 --- a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/mod.rs @@ -15,4 +15,5 @@ // specific language governing permissions and limitations // under the License. -pub mod accumulate; \ No newline at end of file +pub mod accumulate; +pub mod prim_op; \ No newline at end of file diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/prim_op.rs b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs similarity index 97% rename from datafusion/physical-expr/src/aggregate/groups_accumulator/prim_op.rs rename to datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs index 994f5447d7c0..b282b461a185 100644 --- a/datafusion/physical-expr/src/aggregate/groups_accumulator/prim_op.rs +++ b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs @@ -18,8 +18,8 @@ use std::sync::Arc; use arrow::{array::AsArray, datatypes::ArrowPrimitiveType}; -use arrow_array::{ArrayRef, BooleanArray, PrimitiveArray}; -use arrow_schema::DataType; +use arrow::array::{ArrayRef, BooleanArray, PrimitiveArray}; +use arrow::datatypes::DataType; use datafusion_common::Result; use datafusion_expr::{EmitTo, GroupsAccumulator}; diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs b/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs index 0581c3971e4a..b41f04b681cc 100644 --- a/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs +++ b/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs @@ -23,4 +23,6 @@ pub use accumulate::NullState; pub use adapter::GroupsAccumulatorAdapter; pub(crate) mod bool_op; -pub(crate) mod prim_op; +pub(crate) mod prim_op { + pub use datafusion_physical_expr_common::aggregate::groups_accumulator::prim_op::*; +} From 5bcab35dc36bc1fd9bb3242c844232a05d4e300d Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sun, 5 May 2024 13:14:10 +0800 Subject: [PATCH 03/41] move test to slt Signed-off-by: jayzhan211 --- datafusion/physical-expr/src/aggregate/sum.rs | 111 ------------------ .../sqllogictest/test_files/aggregate.slt | 75 ++++++++++++ 2 files changed, 75 insertions(+), 111 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/sum.rs b/datafusion/physical-expr/src/aggregate/sum.rs index f19be62bbc95..2f34b02ed856 100644 --- a/datafusion/physical-expr/src/aggregate/sum.rs +++ b/datafusion/physical-expr/src/aggregate/sum.rs @@ -289,114 +289,3 @@ impl Accumulator for SlidingSumAccumulator { true } } - -#[cfg(test)] -mod tests { - use super::*; - use crate::expressions::tests::assert_aggregate; - use arrow_array::*; - use datafusion_expr::AggregateFunction; - - #[test] - fn sum_decimal() { - // test agg - let array: ArrayRef = Arc::new( - (1..6) - .map(Some) - .collect::() - .with_precision_and_scale(10, 0) - .unwrap(), - ); - - assert_aggregate( - array, - AggregateFunction::Sum, - false, - ScalarValue::Decimal128(Some(15), 20, 0), - ); - } - - #[test] - fn sum_decimal_with_nulls() { - // test agg - let array: ArrayRef = Arc::new( - (1..6) - .map(|i| if i == 2 { None } else { Some(i) }) - .collect::() - .with_precision_and_scale(35, 0) - .unwrap(), - ); - - assert_aggregate( - array, - AggregateFunction::Sum, - false, - ScalarValue::Decimal128(Some(13), 38, 0), - ); - } - - #[test] - fn sum_decimal_all_nulls() { - // test with batch - let array: ArrayRef = Arc::new( - std::iter::repeat::>(None) - .take(6) - .collect::() - .with_precision_and_scale(10, 0) - .unwrap(), - ); - - // test agg - assert_aggregate( - array, - AggregateFunction::Sum, - false, - ScalarValue::Decimal128(None, 20, 0), - ); - } - - #[test] - fn sum_i32() { - let a: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])); - assert_aggregate(a, AggregateFunction::Sum, false, ScalarValue::from(15i64)); - } - - #[test] - fn sum_i32_with_nulls() { - let a: ArrayRef = Arc::new(Int32Array::from(vec![ - Some(1), - None, - Some(3), - Some(4), - Some(5), - ])); - assert_aggregate(a, AggregateFunction::Sum, false, ScalarValue::from(13i64)); - } - - #[test] - fn sum_i32_all_nulls() { - let a: ArrayRef = Arc::new(Int32Array::from(vec![None, None])); - assert_aggregate(a, AggregateFunction::Sum, false, ScalarValue::Int64(None)); - } - - #[test] - fn sum_u32() { - let a: ArrayRef = - Arc::new(UInt32Array::from(vec![1_u32, 2_u32, 3_u32, 4_u32, 5_u32])); - assert_aggregate(a, AggregateFunction::Sum, false, ScalarValue::from(15u64)); - } - - #[test] - fn sum_f32() { - let a: ArrayRef = - Arc::new(Float32Array::from(vec![1_f32, 2_f32, 3_f32, 4_f32, 5_f32])); - assert_aggregate(a, AggregateFunction::Sum, false, ScalarValue::from(15_f64)); - } - - #[test] - fn sum_f64() { - let a: ArrayRef = - Arc::new(Float64Array::from(vec![1_f64, 2_f64, 3_f64, 4_f64, 5_f64])); - assert_aggregate(a, AggregateFunction::Sum, false, ScalarValue::from(15_f64)); - } -} diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 7e4826cd1040..8685303fcd65 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -2316,6 +2316,81 @@ select sum(c1), arrow_typeof(sum(c1)) from d_table; ---- 100 Decimal128(20, 3) +# aggregate sum with deciaml +statement ok +create table t (c decimal(35, 3)) as values (10), (null), (20); + +query RT +select sum(c), arrow_typeof(sum(c)) from t; +---- +30 Decimal128(38, 3) + +statement ok +drop table t; + +# aggregate sum with i32, sum coerced result to i64 +statement ok +create table t (c int) as values (1), (-1), (10), (null), (-11); + +query IT +select sum(c), arrow_typeof(sum(c)) from t; +---- +-1 Int64 + +statement ok +drop table t; + +# aggregate sum with all nulls +statement ok +create table t (c1 decimal(10, 0), c2 int) as values (null, null), (null, null), (null, null); + +query RTIT +select + sum(c1), arrow_typeof(sum(c1)), + sum(c2), arrow_typeof(sum(c2)) +from t; +---- +NULL Decimal128(20, 0) NULL Int64 + +statement ok +drop table t; + +# aggregate sum with u32, sum coerced result to u64 +statement ok +create table t (c int unsigned) as values (1), (0), (10), (null), (4); + +query IT +select sum(c), arrow_typeof(sum(c)) from t; +---- +15 UInt64 + +statement ok +drop table t; + +# aggregate sum with f32, sum coerced result to f64 +statement ok +create table t (c float) as values (1.2), (0.2), (-1.2), (null), (-1.0); + +query RT +select sum(c), arrow_typeof(sum(c)) from t; +---- +-0.79999999702 Float64 + +statement ok +drop table t; + +# aggregate sum with f64 +statement ok +create table t (c double) as values (1.2), (0.2), (-1.2), (null), (-1.0); + +query RT +select sum(c), arrow_typeof(sum(c)) from t; +---- +-0.8 Float64 + +statement ok +drop table t; + query TRT select c2, sum(c1), arrow_typeof(sum(c1)) from d_table GROUP BY c2 ORDER BY c2; ---- From 426f2abd2354075fc274fcb8400789964c433bf3 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sun, 5 May 2024 13:31:36 +0800 Subject: [PATCH 04/41] remove sum distinct Signed-off-by: jayzhan211 --- .../src/aggregate/sum_distinct.rs | 81 ------------------- .../sqllogictest/test_files/aggregate.slt | 46 +++++++++++ 2 files changed, 46 insertions(+), 81 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/sum_distinct.rs b/datafusion/physical-expr/src/aggregate/sum_distinct.rs index 09f3f9b498c1..a46c3d311e19 100644 --- a/datafusion/physical-expr/src/aggregate/sum_distinct.rs +++ b/datafusion/physical-expr/src/aggregate/sum_distinct.rs @@ -200,84 +200,3 @@ impl Accumulator for DistinctSumAccumulator { + self.values.capacity() * std::mem::size_of::() } } - -#[cfg(test)] -mod tests { - use super::*; - use crate::expressions::tests::assert_aggregate; - use arrow::array::*; - use datafusion_expr::AggregateFunction; - - fn run_update_batch( - return_type: DataType, - arrays: &[ArrayRef], - ) -> Result<(Vec, ScalarValue)> { - let agg = DistinctSum::new(vec![], String::from("__col_name__"), return_type); - - let mut accum = agg.create_accumulator()?; - accum.update_batch(arrays)?; - - Ok((accum.state()?, accum.evaluate()?)) - } - - #[test] - fn sum_distinct_update_batch() -> Result<()> { - let array_int64: ArrayRef = Arc::new(Int64Array::from(vec![1, 1, 3])); - let arrays = vec![array_int64]; - let (states, result) = run_update_batch(DataType::Int64, &arrays)?; - - assert_eq!(states.len(), 1); - assert_eq!(result, ScalarValue::Int64(Some(4))); - - Ok(()) - } - - #[test] - fn sum_distinct_i32_with_nulls() { - let array = Arc::new(Int32Array::from(vec![ - Some(1), - Some(1), - None, - Some(2), - Some(2), - Some(3), - ])); - assert_aggregate(array, AggregateFunction::Sum, true, 6_i64.into()); - } - - #[test] - fn sum_distinct_u32_with_nulls() { - let array: ArrayRef = Arc::new(UInt32Array::from(vec![ - Some(1_u32), - Some(1_u32), - Some(3_u32), - Some(3_u32), - None, - ])); - assert_aggregate(array, AggregateFunction::Sum, true, 4_u64.into()); - } - - #[test] - fn sum_distinct_f64() { - let array: ArrayRef = - Arc::new(Float64Array::from(vec![1_f64, 1_f64, 3_f64, 3_f64, 3_f64])); - assert_aggregate(array, AggregateFunction::Sum, true, 4_f64.into()); - } - - #[test] - fn sum_distinct_decimal_with_nulls() { - let array: ArrayRef = Arc::new( - (1..6) - .map(|i| if i == 2 { None } else { Some(i % 2) }) - .collect::() - .with_precision_and_scale(35, 0) - .unwrap(), - ); - assert_aggregate( - array, - AggregateFunction::Sum, - true, - ScalarValue::Decimal128(Some(1), 38, 0), - ); - } -} diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 8685303fcd65..933bbb4b6312 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -1625,7 +1625,53 @@ select mean(c1) from test ---- 1.75 +# aggregate sum distinct, coerced result from i32 to i64 +statement ok +create table t (c int) as values (1), (2), (1), (3), (null), (null), (-3), (-3); + +query IT +select sum(distinct c), arrow_typeof(sum(distinct c)) from t; +---- +3 Int64 + +statement ok +drop table t; + +# aggregate sum distinct, coerced result from u32 to u64 +statement ok +create table t (c int unsigned) as values (1), (2), (1), (3), (null), (null), (3); + +query IT +select sum(distinct c), arrow_typeof(sum(distinct c)) from t; +---- +6 UInt64 + +statement ok +drop table t; +# aggregate sum distinct, coerced result from f32 to f64 +statement ok +create table t (c float) as values (1.0), (2.2), (1.0), (3.3), (null), (null), (3.3), (-2.0); + +query RT +select sum(distinct c), arrow_typeof(sum(distinct c)) from t; +---- +4.5 Float64 + +statement ok +drop table t; + +# aggregate sum distinct with decimal +statement ok +create table t (c decimal(35, 0)) as values (1), (2), (1), (3), (null), (null), (3), (-2); + +query RT +select sum(distinct c), arrow_typeof(sum(distinct c)) from t; +---- +4 Decimal128(38, 0) + +statement ok +drop table t; # query_sum_distinct - 2 different aggregate functions: avg and sum(distinct) query RI From e080d373070ca8bba47238a28d17087aca789607 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sun, 5 May 2024 14:05:03 +0800 Subject: [PATCH 05/41] move sum aggregate Signed-off-by: jayzhan211 --- datafusion/expr/src/expr_fn.rs | 6 +- datafusion/expr/src/function.rs | 44 ++- datafusion/expr/src/lib.rs | 2 +- datafusion/expr/src/udaf.rs | 61 ++- .../functions-aggregate/src/first_last.rs | 15 +- datafusion/functions-aggregate/src/lib.rs | 1 + datafusion/functions-aggregate/src/macros.rs | 70 ++-- datafusion/functions-aggregate/src/sum.rs | 364 ++++++++++++++++++ datafusion/physical-expr-common/Cargo.toml | 2 +- .../src/aggregate/groups_accumulator/mod.rs | 2 +- .../physical-expr-common/src/aggregate/mod.rs | 48 ++- 11 files changed, 552 insertions(+), 63 deletions(-) create mode 100644 datafusion/functions-aggregate/src/sum.rs diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 1d976a12cc4f..17a1ee357c95 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -22,7 +22,7 @@ use crate::expr::{ Placeholder, TryCast, }; use crate::function::{ - AccumulatorArgs, AccumulatorFactoryFunction, PartitionEvaluatorFactory, + AccumulatorArgs, AccumulatorFactoryFunction, PartitionEvaluatorFactory, StateFieldsArgs, }; use crate::{ aggregate_function, conditional_expressions::CaseBuilder, logical_plan::Subquery, @@ -692,9 +692,7 @@ impl AggregateUDFImpl for SimpleAggregateUDF { fn state_fields( &self, - _name: &str, - _value_type: DataType, - _ordering_fields: Vec, + _args: StateFieldsArgs, ) -> Result> { Ok(self.state_fields.clone()) } diff --git a/datafusion/expr/src/function.rs b/datafusion/expr/src/function.rs index 7a92a50ae15d..ff1d73360787 100644 --- a/datafusion/expr/src/function.rs +++ b/datafusion/expr/src/function.rs @@ -19,7 +19,7 @@ use crate::ColumnarValue; use crate::{Accumulator, Expr, PartitionEvaluator}; -use arrow::datatypes::{DataType, Schema}; +use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::Result; use std::sync::Arc; @@ -38,6 +38,43 @@ pub type ScalarFunctionImplementation = pub type ReturnTypeFunction = Arc Result> + Send + Sync>; + +/// `StateFieldsArgs` encapsulates details regarding the required state fields for an aggregate function. +/// +/// - `name`: Name of the aggregate function. +/// - `input_type`: Input type of the state fields. +/// - `ordering_fields`: Fields utilized for functions sensitive to ordering. +/// - `nullable`: Indicates whether the state fields can be null. +pub struct StateFieldsArgs<'a> { + pub name: &'a str, + pub input_type: DataType, + pub ordering_fields: Vec, + pub nullable: bool, +} + +impl<'a> StateFieldsArgs<'a> { + pub fn new( + name: &'a str, + value_type: DataType, + ordering_fields: Vec, + nullable: bool, + ) -> Self { + Self { + name, + input_type: value_type, + ordering_fields, + nullable, + } + } +} + +pub struct GroupsAccumulatorArgs<'a> { + /// The return type of the aggregate function. + pub data_type: &'a DataType, + /// The name of the aggregate expression + pub name: &'a str, +} + /// [`AccumulatorArgs`] contains information about how an aggregate /// function was called, including the types of its arguments and any optional /// ordering expressions. @@ -66,6 +103,9 @@ pub struct AccumulatorArgs<'a> { /// /// If no `ORDER BY` is specified, `sort_exprs`` will be empty. pub sort_exprs: &'a [Expr], + + /// The name of the aggregate expression + pub name: &'a str, } impl<'a> AccumulatorArgs<'a> { @@ -74,12 +114,14 @@ impl<'a> AccumulatorArgs<'a> { schema: &'a Schema, ignore_nulls: bool, sort_exprs: &'a [Expr], + name: &'a str, ) -> Self { Self { data_type, schema, ignore_nulls, sort_exprs, + name, } } } diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index de4f31029293..340306cc057e 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -81,7 +81,7 @@ pub use signature::{ TIMEZONE_WILDCARD, }; pub use table_source::{TableProviderFilterPushDown, TableSource, TableType}; -pub use udaf::{AggregateUDF, AggregateUDFImpl}; +pub use udaf::{AggregateUDF, AggregateUDFImpl, ReversedUDAF}; pub use udf::{ScalarUDF, ScalarUDFImpl}; pub use udwf::{WindowUDF, WindowUDFImpl}; pub use window_frame::{WindowFrame, WindowFrameBound, WindowFrameUnits}; diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 67c3b51ca373..24f85addcb50 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -17,7 +17,7 @@ //! [`AggregateUDF`]: User Defined Aggregate Functions -use crate::function::AccumulatorArgs; +use crate::function::{AccumulatorArgs, GroupsAccumulatorArgs, StateFieldsArgs}; use crate::groups_accumulator::GroupsAccumulator; use crate::utils::format_state_name; use crate::{Accumulator, Expr}; @@ -179,11 +179,9 @@ impl AggregateUDF { /// This is used to support multi-phase aggregations pub fn state_fields( &self, - name: &str, - value_type: DataType, - ordering_fields: Vec, + args: StateFieldsArgs, ) -> Result> { - self.inner.state_fields(name, value_type, ordering_fields) + self.inner.state_fields(args) } /// See [`AggregateUDFImpl::groups_accumulator_supported`] for more details. @@ -192,8 +190,22 @@ impl AggregateUDF { } /// See [`AggregateUDFImpl::create_groups_accumulator`] for more details. - pub fn create_groups_accumulator(&self) -> Result> { - self.inner.create_groups_accumulator() + pub fn create_groups_accumulator( + &self, + args: GroupsAccumulatorArgs, + ) -> Result> { + self.inner.create_groups_accumulator(args) + } + + pub fn create_sliding_accumulator( + &self, + args: AccumulatorArgs, + ) -> Result> { + self.inner.create_sliding_accumulator(args) + } + + pub fn reverse_expr(&self) -> ReversedUDAF { + self.inner.reverse_expr() } } @@ -311,17 +323,15 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// to generate a unique name. fn state_fields( &self, - name: &str, - value_type: DataType, - ordering_fields: Vec, + args: StateFieldsArgs, ) -> Result> { let value_fields = vec![Field::new( - format_state_name(name, "value"), - value_type, + format_state_name(args.name, "value"), + args.input_type, true, )]; - Ok(value_fields.into_iter().chain(ordering_fields).collect()) + Ok(value_fields.into_iter().chain(args.ordering_fields).collect()) } /// If the aggregate expression has a specialized @@ -343,7 +353,10 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// /// For maximum performance, a [`GroupsAccumulator`] should be /// implemented in addition to [`Accumulator`]. - fn create_groups_accumulator(&self) -> Result> { + fn create_groups_accumulator( + &self, + _args: GroupsAccumulatorArgs, + ) -> Result> { not_impl_err!("GroupsAccumulator hasn't been implemented for {self:?} yet") } @@ -354,6 +367,26 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { fn aliases(&self) -> &[String] { &[] } + + fn create_sliding_accumulator( + &self, + args: AccumulatorArgs, + ) -> Result> { + self.accumulator(args) + } + + fn reverse_expr(&self) -> ReversedUDAF { + ReversedUDAF::NotSupported + } +} + +pub enum ReversedUDAF { + /// The expression is the same as the original expression, like SUM, COUNT + Identical, + /// The expression does not support reverse calculation, like ArrayAgg + NotSupported, + /// The expression is different from the original expression + Reversed(Arc), } /// AggregateUDF that adds an alias to the underlying function. It is better to diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 8dc4cee87a3b..1ea959bac046 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -24,7 +24,7 @@ use datafusion_common::utils::{compare_rows, get_arrayref_at_indices, get_row_at use datafusion_common::{ arrow_datafusion_err, internal_err, DataFusionError, Result, ScalarValue, }; -use datafusion_expr::function::AccumulatorArgs; +use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::NUMERICS; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ @@ -39,12 +39,11 @@ use datafusion_physical_expr_common::expressions; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_expr_common::utils::reverse_order_bys; -use sqlparser::ast::NullTreatment; use std::any::Any; use std::fmt::Debug; use std::sync::Arc; -make_udaf_function!( +make_udaf_expr_and_func!( FirstValue, first_value, "Returns the first value in a group of values.", @@ -149,16 +148,14 @@ impl AggregateUDFImpl for FirstValue { fn state_fields( &self, - name: &str, - value_type: DataType, - ordering_fields: Vec, + args: StateFieldsArgs, ) -> Result> { let mut fields = vec![Field::new( - format_state_name(name, "first_value"), - value_type, + format_state_name(args.name, "first_value"), + args.input_type, true, )]; - fields.extend(ordering_fields); + fields.extend(args.ordering_fields); fields.push(Field::new("is_set", DataType::Boolean, true)); Ok(fields) } diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index 8016b76889f7..69f82138f14f 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -56,6 +56,7 @@ pub mod macros; pub mod first_last; +pub mod sum; use datafusion_common::Result; use datafusion_execution::FunctionRegistry; diff --git a/datafusion/functions-aggregate/src/macros.rs b/datafusion/functions-aggregate/src/macros.rs index 04f9fecb8b19..751aec51247b 100644 --- a/datafusion/functions-aggregate/src/macros.rs +++ b/datafusion/functions-aggregate/src/macros.rs @@ -15,33 +15,59 @@ // specific language governing permissions and limitations // under the License. -macro_rules! make_udaf_function { +macro_rules! make_udaf_expr_and_func { + ($UDAF:ty, $EXPR_FN:ident, $($arg:ident)*, $DOC:expr, $AGGREGATE_UDF_FN:ident) => { + // "fluent expr_fn" style function + #[doc = $DOC] + pub fn $EXPR_FN( + $($arg: datafusion_expr::Expr,)* + distinct: bool, + filter: Option>, + order_by: Option>, + null_treatment: Option + ) -> datafusion_expr::Expr { + datafusion_expr::Expr::AggregateFunction(datafusion_expr::expr::AggregateFunction::new_udf( + $AGGREGATE_UDF_FN(), + vec![$($arg),*], + distinct, + filter, + order_by, + null_treatment, + )) + } + create_func!($UDAF, $AGGREGATE_UDF_FN); + }; ($UDAF:ty, $EXPR_FN:ident, $DOC:expr, $AGGREGATE_UDF_FN:ident) => { - paste::paste! { - // "fluent expr_fn" style function - #[doc = $DOC] - pub fn $EXPR_FN( - args: Vec, - distinct: bool, - filter: Option>, - order_by: Option>, - null_treatment: Option - ) -> Expr { - Expr::AggregateFunction(datafusion_expr::expr::AggregateFunction::new_udf( - $AGGREGATE_UDF_FN(), - args, - distinct, - filter, - order_by, - null_treatment, - )) - } + // "fluent expr_fn" style function + #[doc = $DOC] + pub fn $EXPR_FN( + args: Vec, + distinct: bool, + filter: Option>, + order_by: Option>, + null_treatment: Option + ) -> datafusion_expr::Expr { + datafusion_expr::Expr::AggregateFunction(datafusion_expr::expr::AggregateFunction::new_udf( + $AGGREGATE_UDF_FN(), + args, + distinct, + filter, + order_by, + null_treatment, + )) + } + create_func!($UDAF, $AGGREGATE_UDF_FN); + }; +} +macro_rules! create_func { + ($UDAF:ty, $AGGREGATE_UDF_FN:ident) => { + paste::paste! { /// Singleton instance of [$UDAF], ensures the UDAF is only created once /// named STATIC_$(UDAF). For example `STATIC_FirstValue` #[allow(non_upper_case_globals)] static [< STATIC_ $UDAF >]: std::sync::OnceLock> = - std::sync::OnceLock::new(); + std::sync::OnceLock::new(); /// AggregateFunction that returns a [AggregateUDF] for [$UDAF] /// @@ -55,4 +81,4 @@ macro_rules! make_udaf_function { } } } -} +} \ No newline at end of file diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs new file mode 100644 index 000000000000..14cbe155edc7 --- /dev/null +++ b/datafusion/functions-aggregate/src/sum.rs @@ -0,0 +1,364 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Defines `SUM` and `SUM DISTINCT` aggregate accumulators + +use std::any::Any; + +use arrow::array::{ArrowNumericType, AsArray}; +use arrow::array::ArrowNativeTypeOp; +use arrow::array::Array; +use datafusion_physical_expr_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; +use arrow::datatypes::{DataType, Decimal128Type, Decimal256Type, Float64Type, Int64Type, UInt64Type, DECIMAL128_MAX_PRECISION, DECIMAL256_MAX_PRECISION}; +use arrow::datatypes::ArrowNativeType; +use arrow::{array::ArrayRef, datatypes::Field}; +use datafusion_common::{exec_err, not_impl_err, Result, ScalarValue}; +use datafusion_expr::function::{ + AccumulatorArgs, GroupsAccumulatorArgs, StateFieldsArgs, +}; +use datafusion_expr::type_coercion::aggregates::NUMERICS; +use datafusion_expr::utils::format_state_name; +use datafusion_expr::{ + Accumulator, AggregateUDFImpl, GroupsAccumulator, ReversedUDAF, Signature, Volatility +}; + +make_udaf_expr_and_func!( + Sum, + sum, + "Returns the first value in a group of values.", + sum_udaf +); + +/// Sum only supports a subset of numeric types, instead relying on type coercion +/// +/// This macro is similar to [downcast_primitive](arrow::array::downcast_primitive) +/// +/// `args` is [AccumulatorArgs] +/// `helper` is a macro accepting (ArrowPrimitiveType, DataType) +macro_rules! downcast_sum { + ($args:ident, $helper:ident) => { + match $args.data_type { + DataType::UInt64 => $helper!(UInt64Type, $args.data_type), + DataType::Int64 => $helper!(Int64Type, $args.data_type), + DataType::Float64 => $helper!(Float64Type, $args.data_type), + DataType::Decimal128(_, _) => $helper!(Decimal128Type, $args.data_type), + DataType::Decimal256(_, _) => $helper!(Decimal256Type, $args.data_type), + _ => not_impl_err!("Sum not supported for {}: {}", $args.name, $args.data_type), + } + }; +} + +#[derive(Debug)] +pub struct Sum { + pub is_distinct: bool, + signature: Signature, + aliases: Vec, +} + +impl Sum { + pub fn new(is_distinct: bool) -> Self { + Self { + signature: Signature::uniform(1, NUMERICS.to_vec(), Volatility::Immutable), + aliases: vec![], + is_distinct, + } + } +} + +impl Default for Sum { + fn default() -> Self { + Self::new(false) + } +} + +impl AggregateUDFImpl for Sum { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "sum" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> Result { + // Refer to https://www.postgresql.org/docs/8.2/functions-aggregate.html doc + // smallint, int, bigint, real, double precision, decimal, or interval. + + fn coerced_types(data_type: &DataType) -> Result { + match data_type { + DataType::Dictionary(_, v) => coerced_types(v), + // in the spark, the result type is DECIMAL(min(38,precision+10), s) + // ref: https://github.com/apache/spark/blob/fcf636d9eb8d645c24be3db2d599aba2d7e2955a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala#L66 + DataType::Decimal128(precision, scale) => { + let new_precision = DECIMAL128_MAX_PRECISION.min(*precision + 10); + Ok(DataType::Decimal128(new_precision, *scale)) + } + DataType::Decimal256(precision, scale) => { + let new_precision = DECIMAL256_MAX_PRECISION.min(*precision + 10); + Ok(DataType::Decimal256(new_precision, *scale)) + } + dt if dt.is_signed_integer() => Ok(DataType::Int64), + dt if dt.is_unsigned_integer() => Ok(DataType::UInt64), + dt if dt.is_floating() => Ok(DataType::Float64), + _ => exec_err!("Sum not supported for {}", data_type), + } + } + + coerced_types(&arg_types[0]) + } + + fn accumulator(&self, args: AccumulatorArgs) -> Result> { + macro_rules! helper { + ($t:ty, $dt:expr) => { + Ok(Box::new(SumAccumulator::<$t>::new($dt.clone()))) + }; + } + downcast_sum!(args, helper) + } + + fn state_fields(&self, args: StateFieldsArgs) -> Result> { + Ok(vec![Field::new( + format_state_name(args.name, "sum"), + args.input_type, + args.nullable, + )]) + } + + fn aliases(&self) -> &[String] { + &self.aliases + } + + fn groups_accumulator_supported(&self) -> bool { + true + } + + fn create_groups_accumulator( + &self, + args: GroupsAccumulatorArgs, + ) -> Result> { + macro_rules! helper { + ($t:ty, $dt:expr) => { + Ok(Box::new(PrimitiveGroupsAccumulator::<$t, _>::new( + &$dt, + |x, y| *x = x.add_wrapping(y), + ))) + }; + } + downcast_sum!(args, helper) + + // let return_type = args.data_type; + + // match return_type { + // DataType::UInt64 => Ok(Box::new( + // PrimitiveGroupsAccumulator::::new(&return_type, |x, y| { + // *x = x.add_wrapping(y) + // }), + // )), + // DataType::Int64 => Ok(Box::new( + // PrimitiveGroupsAccumulator::::new(&return_type, |x, y| { + // *x = x.add_wrapping(y) + // }), + // )), + // DataType::Float64 => { + // Ok(Box::new(PrimitiveGroupsAccumulator::::new( + // &return_type, + // |x, y| *x = x.add_wrapping(y), + // ))) + // } + // DataType::Decimal128(_, _) => Ok(Box::new(PrimitiveGroupsAccumulator::< + // Decimal128Type, + // _, + // >::new( + // &return_type, + // |x, y| *x = x.add_wrapping(y), + // ))), + // DataType::Decimal256(_, _) => Ok(Box::new(PrimitiveGroupsAccumulator::< + // Decimal256Type, + // _, + // >::new( + // &return_type, + // |x, y| *x = x.add_wrapping(y), + // ))), + // _ => not_impl_err!("Sum not supported for {}: {}", args.name, return_type), + // } + } + + fn create_sliding_accumulator( + &self, + args: AccumulatorArgs, + ) -> Result> { + macro_rules! helper { + ($t:ty, $dt:expr) => { + Ok(Box::new(SlidingSumAccumulator::<$t>::new($dt.clone()))) + }; + } + downcast_sum!(args, helper) + // let return_type = args.data_type; + + // match return_type { + // DataType::UInt64 => Ok(Box::new(SlidingSumAccumulator::::new( + // return_type.clone(), + // ))), + // DataType::Int64 => Ok(Box::new(SlidingSumAccumulator::::new( + // return_type.clone(), + // ))), + // DataType::Float64 => Ok(Box::new(SlidingSumAccumulator::::new( + // return_type.clone(), + // ))), + // DataType::Decimal128(_, _) => { + // Ok(Box::new(SlidingSumAccumulator::::new( + // return_type.clone(), + // ))) + // } + // DataType::Decimal256(_, _) => { + // Ok(Box::new(SlidingSumAccumulator::::new( + // return_type.clone(), + // ))) + // } + // _ => not_impl_err!("Sum not supported for {}: {}", args.name, return_type), + // } + } + + fn reverse_expr(&self) -> ReversedUDAF { + ReversedUDAF::Identical + } +} + +/// This accumulator computes SUM incrementally +struct SumAccumulator { + sum: Option, + data_type: DataType, +} + +impl std::fmt::Debug for SumAccumulator { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "SumAccumulator({})", self.data_type) + } +} + +impl SumAccumulator { + fn new(data_type: DataType) -> Self { + Self { + sum: None, + data_type, + } + } +} + +impl Accumulator for SumAccumulator { + fn state(&mut self) -> Result> { + Ok(vec![self.evaluate()?]) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let values = values[0].as_primitive::(); + if let Some(x) = arrow::compute::sum(values) { + let v = self.sum.get_or_insert(T::Native::usize_as(0)); + *v = v.add_wrapping(x); + } + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + self.update_batch(states) + } + + fn evaluate(&mut self) -> Result { + ScalarValue::new_primitive::(self.sum, &self.data_type) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } +} + +/// This accumulator incrementally computes sums over a sliding window +/// +/// This is separate from [`SumAccumulator`] as requires additional state +struct SlidingSumAccumulator { + sum: T::Native, + count: u64, + data_type: DataType, +} + +impl std::fmt::Debug for SlidingSumAccumulator { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "SlidingSumAccumulator({})", self.data_type) + } +} + +impl SlidingSumAccumulator { + fn new(data_type: DataType) -> Self { + Self { + sum: T::Native::usize_as(0), + count: 0, + data_type, + } + } +} + +impl Accumulator for SlidingSumAccumulator { + fn state(&mut self) -> Result> { + Ok(vec![self.evaluate()?, self.count.into()]) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let values = values[0].as_primitive::(); + self.count += (values.len() - values.null_count()) as u64; + if let Some(x) = arrow::compute::sum(values) { + self.sum = self.sum.add_wrapping(x) + } + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + let values = states[0].as_primitive::(); + if let Some(x) = arrow::compute::sum(values) { + self.sum = self.sum.add_wrapping(x) + } + if let Some(x) = arrow::compute::sum(states[1].as_primitive::()) { + self.count += x; + } + Ok(()) + } + + fn evaluate(&mut self) -> Result { + let v = (self.count != 0).then_some(self.sum); + ScalarValue::new_primitive::(v, &self.data_type) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } + + fn retract_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let values = values[0].as_primitive::(); + if let Some(x) = arrow::compute::sum(values) { + self.sum = self.sum.sub_wrapping(x) + } + self.count -= (values.len() - values.null_count()) as u64; + Ok(()) + } + + fn supports_retract_batch(&self) -> bool { + true + } +} \ No newline at end of file diff --git a/datafusion/physical-expr-common/Cargo.toml b/datafusion/physical-expr-common/Cargo.toml index d1683c894d6d..8b26fae31737 100644 --- a/datafusion/physical-expr-common/Cargo.toml +++ b/datafusion/physical-expr-common/Cargo.toml @@ -41,4 +41,4 @@ datafusion-common = { workspace = true, default-features = true } datafusion-expr = { workspace = true } [dev-dependencies] -rand = { workspace = true } \ No newline at end of file +rand = { workspace = true } diff --git a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/mod.rs b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/mod.rs index fffdae11bec2..c54b38df8a22 100644 --- a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/mod.rs @@ -16,4 +16,4 @@ // under the License. pub mod accumulate; -pub mod prim_op; \ No newline at end of file +pub mod prim_op; diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 4e1b53dbe7c7..d85c4d181d5a 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -20,7 +20,9 @@ pub mod utils; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::{not_impl_err, Result}; +use datafusion_expr::function::{GroupsAccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::check_arg_count; +use datafusion_expr::ReversedUDAF; use datafusion_expr::{ function::AccumulatorArgs, Accumulator, AggregateUDF, Expr, GroupsAccumulator, }; @@ -148,7 +150,7 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { } /// Physical aggregate expression of a UDAF. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct AggregateFunctionExpr { fun: AggregateUDF, args: Vec>, @@ -160,7 +162,9 @@ pub struct AggregateFunctionExpr { sort_exprs: Vec, // The physical order by expressions ordering_req: LexOrdering, + // Whether to ignore null values ignore_nulls: bool, + // fields used for order sensitive aggregation functions ordering_fields: Vec, } @@ -182,11 +186,14 @@ impl AggregateExpr for AggregateFunctionExpr { } fn state_fields(&self) -> Result> { - self.fun.state_fields( - self.name(), - self.data_type.clone(), - self.ordering_fields.clone(), - ) + let args = StateFieldsArgs { + name: self.name(), + input_type: self.data_type.clone(), + ordering_fields: self.ordering_fields.clone(), + nullable: true, + }; + + self.fun.state_fields(args) } fn field(&self) -> Result { @@ -194,18 +201,27 @@ impl AggregateExpr for AggregateFunctionExpr { } fn create_accumulator(&self) -> Result> { - let acc_args = AccumulatorArgs::new( + let args = AccumulatorArgs::new( &self.data_type, &self.schema, self.ignore_nulls, &self.sort_exprs, + self.name(), ); - self.fun.accumulator(acc_args) + self.fun.accumulator(args) } fn create_sliding_accumulator(&self) -> Result> { - let accumulator = self.create_accumulator()?; + let args = AccumulatorArgs::new( + &self.data_type, + &self.schema, + self.ignore_nulls, + &self.sort_exprs, + self.name(), + ); + + let accumulator = self.fun().create_sliding_accumulator(args)?; // Accumulators that have window frame startings different // than `UNBOUNDED PRECEDING`, such as `1 PRECEEDING`, need to @@ -268,12 +284,24 @@ impl AggregateExpr for AggregateFunctionExpr { } fn create_groups_accumulator(&self) -> Result> { - self.fun.create_groups_accumulator() + let args = GroupsAccumulatorArgs { + data_type: &self.data_type, + name: self.name(), + }; + self.fun.create_groups_accumulator(args) } fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { (!self.ordering_req.is_empty()).then_some(&self.ordering_req) } + + fn reverse_expr(&self) -> Option> { + match self.fun.reverse_expr() { + ReversedUDAF::NotSupported => None, + ReversedUDAF::Identical => Some(Arc::new(self.clone())), + ReversedUDAF::Reversed(fun) => todo!("Reverse UDAF: {:?}", fun), + } + } } impl PartialEq for AggregateFunctionExpr { From 64db75c94497e8cd9807f2f571547d3991882aa2 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 8 May 2024 14:04:54 +0800 Subject: [PATCH 06/41] fix args Signed-off-by: jayzhan211 --- datafusion-examples/examples/advanced_udaf.rs | 9 +++-- .../user_defined/user_defined_aggregates.rs | 10 ++++-- datafusion/expr/src/expr_fn.rs | 6 ++-- datafusion/expr/src/function.rs | 32 +---------------- datafusion/expr/src/udaf.rs | 26 ++++++++------ .../functions-aggregate/src/first_last.rs | 12 ++++--- datafusion/functions-aggregate/src/macros.rs | 2 +- datafusion/functions-aggregate/src/sum.rs | 36 +++++++++++-------- .../groups_accumulator/accumulate.rs | 6 ++-- .../aggregate/groups_accumulator/prim_op.rs | 2 +- .../physical-expr-common/src/aggregate/mod.rs | 19 +++++----- 11 files changed, 76 insertions(+), 84 deletions(-) diff --git a/datafusion-examples/examples/advanced_udaf.rs b/datafusion-examples/examples/advanced_udaf.rs index 342a23b6e73d..f4c97adc6e07 100644 --- a/datafusion-examples/examples/advanced_udaf.rs +++ b/datafusion-examples/examples/advanced_udaf.rs @@ -31,8 +31,8 @@ use datafusion::error::Result; use datafusion::prelude::*; use datafusion_common::{cast::as_float64_array, ScalarValue}; use datafusion_expr::{ - function::AccumulatorArgs, Accumulator, AggregateUDF, AggregateUDFImpl, - GroupsAccumulator, Signature, + function::{AccumulatorArgs, GroupsAccumulatorArgs}, + Accumulator, AggregateUDF, AggregateUDFImpl, GroupsAccumulator, Signature, }; /// This example shows how to use the full AggregateUDFImpl API to implement a user @@ -110,7 +110,10 @@ impl AggregateUDFImpl for GeoMeanUdaf { true } - fn create_groups_accumulator(&self) -> Result> { + fn create_groups_accumulator( + &self, + _args: GroupsAccumulatorArgs, + ) -> Result> { Ok(Box::new(GeometricMeanGroupsAccumulator::new())) } } diff --git a/datafusion/core/tests/user_defined/user_defined_aggregates.rs b/datafusion/core/tests/user_defined/user_defined_aggregates.rs index 8f02fb30b013..3e7f59116935 100644 --- a/datafusion/core/tests/user_defined/user_defined_aggregates.rs +++ b/datafusion/core/tests/user_defined/user_defined_aggregates.rs @@ -45,8 +45,9 @@ use datafusion::{ }; use datafusion_common::{assert_contains, cast::as_primitive_array, exec_err}; use datafusion_expr::{ - create_udaf, function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator, - SimpleAggregateUDF, + create_udaf, + function::{AccumulatorArgs, GroupsAccumulatorArgs}, + AggregateUDFImpl, GroupsAccumulator, SimpleAggregateUDF, }; use datafusion_physical_expr::expressions::AvgAccumulator; @@ -729,7 +730,10 @@ impl AggregateUDFImpl for TestGroupsAccumulator { true } - fn create_groups_accumulator(&self) -> Result> { + fn create_groups_accumulator( + &self, + _args: GroupsAccumulatorArgs, + ) -> Result> { Ok(Box::new(self.clone())) } } diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 17a1ee357c95..1d976a12cc4f 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -22,7 +22,7 @@ use crate::expr::{ Placeholder, TryCast, }; use crate::function::{ - AccumulatorArgs, AccumulatorFactoryFunction, PartitionEvaluatorFactory, StateFieldsArgs, + AccumulatorArgs, AccumulatorFactoryFunction, PartitionEvaluatorFactory, }; use crate::{ aggregate_function, conditional_expressions::CaseBuilder, logical_plan::Subquery, @@ -692,7 +692,9 @@ impl AggregateUDFImpl for SimpleAggregateUDF { fn state_fields( &self, - _args: StateFieldsArgs, + _name: &str, + _value_type: DataType, + _ordering_fields: Vec, ) -> Result> { Ok(self.state_fields.clone()) } diff --git a/datafusion/expr/src/function.rs b/datafusion/expr/src/function.rs index ff1d73360787..5d2261891a72 100644 --- a/datafusion/expr/src/function.rs +++ b/datafusion/expr/src/function.rs @@ -19,7 +19,7 @@ use crate::ColumnarValue; use crate::{Accumulator, Expr, PartitionEvaluator}; -use arrow::datatypes::{DataType, Field, Schema}; +use arrow::datatypes::{DataType, Schema}; use datafusion_common::Result; use std::sync::Arc; @@ -38,36 +38,6 @@ pub type ScalarFunctionImplementation = pub type ReturnTypeFunction = Arc Result> + Send + Sync>; - -/// `StateFieldsArgs` encapsulates details regarding the required state fields for an aggregate function. -/// -/// - `name`: Name of the aggregate function. -/// - `input_type`: Input type of the state fields. -/// - `ordering_fields`: Fields utilized for functions sensitive to ordering. -/// - `nullable`: Indicates whether the state fields can be null. -pub struct StateFieldsArgs<'a> { - pub name: &'a str, - pub input_type: DataType, - pub ordering_fields: Vec, - pub nullable: bool, -} - -impl<'a> StateFieldsArgs<'a> { - pub fn new( - name: &'a str, - value_type: DataType, - ordering_fields: Vec, - nullable: bool, - ) -> Self { - Self { - name, - input_type: value_type, - ordering_fields, - nullable, - } - } -} - pub struct GroupsAccumulatorArgs<'a> { /// The return type of the aggregate function. pub data_type: &'a DataType, diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 24f85addcb50..93b7aefa18fb 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -17,7 +17,7 @@ //! [`AggregateUDF`]: User Defined Aggregate Functions -use crate::function::{AccumulatorArgs, GroupsAccumulatorArgs, StateFieldsArgs}; +use crate::function::{AccumulatorArgs, GroupsAccumulatorArgs}; use crate::groups_accumulator::GroupsAccumulator; use crate::utils::format_state_name; use crate::{Accumulator, Expr}; @@ -179,9 +179,11 @@ impl AggregateUDF { /// This is used to support multi-phase aggregations pub fn state_fields( &self, - args: StateFieldsArgs, + name: &str, + value_type: DataType, + ordering_fields: Vec, ) -> Result> { - self.inner.state_fields(args) + self.inner.state_fields(name, value_type, ordering_fields) } /// See [`AggregateUDFImpl::groups_accumulator_supported`] for more details. @@ -303,7 +305,8 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// /// # Arguments: /// 1. `name`: the name of the expression (e.g. AVG, SUM, etc) - /// 2. `value_type`: Aggregate's aggregate's output (returned by [`Self::return_type`]) + /// 2. `value_type`: Aggregate function output returned by [`Self::return_type`] if defined, otherwise + /// it is equivalent to the data type of the first arguments /// 3. `ordering_fields`: the fields used to order the input arguments, if any. /// Empty if no ordering expression is provided. /// @@ -323,15 +326,18 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// to generate a unique name. fn state_fields( &self, - args: StateFieldsArgs, + name: &str, + value_type: DataType, + ordering_fields: Vec, ) -> Result> { - let value_fields = vec![Field::new( - format_state_name(args.name, "value"), - args.input_type, + let mut fields = vec![Field::new( + format_state_name(name, "first_value"), + value_type, true, )]; - - Ok(value_fields.into_iter().chain(args.ordering_fields).collect()) + fields.extend(ordering_fields); + fields.push(Field::new("is_set", DataType::Boolean, true)); + Ok(fields) } /// If the aggregate expression has a specialized diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 1ea959bac046..0c393c2a398b 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -24,7 +24,7 @@ use datafusion_common::utils::{compare_rows, get_arrayref_at_indices, get_row_at use datafusion_common::{ arrow_datafusion_err, internal_err, DataFusionError, Result, ScalarValue, }; -use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; +use datafusion_expr::function::AccumulatorArgs; use datafusion_expr::type_coercion::aggregates::NUMERICS; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ @@ -148,14 +148,16 @@ impl AggregateUDFImpl for FirstValue { fn state_fields( &self, - args: StateFieldsArgs, + name: &str, + value_type: DataType, + ordering_fields: Vec, ) -> Result> { let mut fields = vec![Field::new( - format_state_name(args.name, "first_value"), - args.input_type, + format_state_name(name, "first_value"), + value_type, true, )]; - fields.extend(args.ordering_fields); + fields.extend(ordering_fields); fields.push(Field::new("is_set", DataType::Boolean, true)); Ok(fields) } diff --git a/datafusion/functions-aggregate/src/macros.rs b/datafusion/functions-aggregate/src/macros.rs index 751aec51247b..27fc623a182b 100644 --- a/datafusion/functions-aggregate/src/macros.rs +++ b/datafusion/functions-aggregate/src/macros.rs @@ -81,4 +81,4 @@ macro_rules! create_func { } } } -} \ No newline at end of file +} diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index 14cbe155edc7..63665723fd3f 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -19,22 +19,23 @@ use std::any::Any; -use arrow::array::{ArrowNumericType, AsArray}; -use arrow::array::ArrowNativeTypeOp; use arrow::array::Array; -use datafusion_physical_expr_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; -use arrow::datatypes::{DataType, Decimal128Type, Decimal256Type, Float64Type, Int64Type, UInt64Type, DECIMAL128_MAX_PRECISION, DECIMAL256_MAX_PRECISION}; +use arrow::array::ArrowNativeTypeOp; +use arrow::array::{ArrowNumericType, AsArray}; use arrow::datatypes::ArrowNativeType; +use arrow::datatypes::{ + DataType, Decimal128Type, Decimal256Type, Float64Type, Int64Type, UInt64Type, + DECIMAL128_MAX_PRECISION, DECIMAL256_MAX_PRECISION, +}; use arrow::{array::ArrayRef, datatypes::Field}; use datafusion_common::{exec_err, not_impl_err, Result, ScalarValue}; -use datafusion_expr::function::{ - AccumulatorArgs, GroupsAccumulatorArgs, StateFieldsArgs, -}; +use datafusion_expr::function::{AccumulatorArgs, GroupsAccumulatorArgs}; use datafusion_expr::type_coercion::aggregates::NUMERICS; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ - Accumulator, AggregateUDFImpl, GroupsAccumulator, ReversedUDAF, Signature, Volatility + Accumulator, AggregateUDFImpl, GroupsAccumulator, ReversedUDAF, Signature, Volatility, }; +use datafusion_physical_expr_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; make_udaf_expr_and_func!( Sum, @@ -57,7 +58,9 @@ macro_rules! downcast_sum { DataType::Float64 => $helper!(Float64Type, $args.data_type), DataType::Decimal128(_, _) => $helper!(Decimal128Type, $args.data_type), DataType::Decimal256(_, _) => $helper!(Decimal256Type, $args.data_type), - _ => not_impl_err!("Sum not supported for {}: {}", $args.name, $args.data_type), + _ => { + not_impl_err!("Sum not supported for {}: {}", $args.name, $args.data_type) + } } }; } @@ -134,11 +137,16 @@ impl AggregateUDFImpl for Sum { downcast_sum!(args, helper) } - fn state_fields(&self, args: StateFieldsArgs) -> Result> { + fn state_fields( + &self, + name: &str, + value_type: DataType, + _ordering_fields: Vec, + ) -> Result> { Ok(vec![Field::new( - format_state_name(args.name, "sum"), - args.input_type, - args.nullable, + format_state_name(name, "sum"), + value_type, + true, )]) } @@ -361,4 +369,4 @@ impl Accumulator for SlidingSumAccumulator { fn supports_retract_batch(&self) -> bool { true } -} \ No newline at end of file +} diff --git a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/accumulate.rs b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/accumulate.rs index 0326d6d87cf0..f109079f6a26 100644 --- a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/accumulate.rs @@ -19,9 +19,9 @@ //! //! [`GroupsAccumulator`]: datafusion_expr::GroupsAccumulator -use arrow::datatypes::ArrowPrimitiveType; -use arrow::array::{Array, BooleanArray, PrimitiveArray, BooleanBufferBuilder}; +use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; use arrow::buffer::{BooleanBuffer, NullBuffer}; +use arrow::datatypes::ArrowPrimitiveType; use datafusion_expr::EmitTo; /// Track the accumulator null state per row: if any values for that @@ -463,8 +463,8 @@ mod test { use super::*; use arrow::array::UInt32Array; - use std::collections::HashSet; use rand::{rngs::ThreadRng, Rng}; + use std::collections::HashSet; #[test] fn accumulate() { diff --git a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs index b282b461a185..0a404a89d5de 100644 --- a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs +++ b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs @@ -17,9 +17,9 @@ use std::sync::Arc; -use arrow::{array::AsArray, datatypes::ArrowPrimitiveType}; use arrow::array::{ArrayRef, BooleanArray, PrimitiveArray}; use arrow::datatypes::DataType; +use arrow::{array::AsArray, datatypes::ArrowPrimitiveType}; use datafusion_common::Result; use datafusion_expr::{EmitTo, GroupsAccumulator}; diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 3bd1a0cc3e70..3dd3d7eaf5ed 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -21,7 +21,7 @@ pub mod utils; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::{not_impl_err, Result}; -use datafusion_expr::function::{GroupsAccumulatorArgs, StateFieldsArgs}; +use datafusion_expr::function::GroupsAccumulatorArgs; use datafusion_expr::type_coercion::aggregates::check_arg_count; use datafusion_expr::ReversedUDAF; use datafusion_expr::{ @@ -187,14 +187,11 @@ impl AggregateExpr for AggregateFunctionExpr { } fn state_fields(&self) -> Result> { - let args = StateFieldsArgs { - name: self.name(), - input_type: self.data_type.clone(), - ordering_fields: self.ordering_fields.clone(), - nullable: true, - }; - - self.fun.state_fields(args) + self.fun.state_fields( + &self.name, + self.data_type.clone(), + self.ordering_fields.to_vec(), + ) } fn field(&self) -> Result { @@ -207,7 +204,7 @@ impl AggregateExpr for AggregateFunctionExpr { &self.schema, self.ignore_nulls, &self.sort_exprs, - self.name(), + &self.name, ); self.fun.accumulator(args) @@ -219,7 +216,7 @@ impl AggregateExpr for AggregateFunctionExpr { &self.schema, self.ignore_nulls, &self.sort_exprs, - self.name(), + &self.name, ); let accumulator = self.fun().create_sliding_accumulator(args)?; From a9a54233bd5ed412e300417bf6df438ad395d516 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 8 May 2024 14:51:04 +0800 Subject: [PATCH 07/41] add sum Signed-off-by: jayzhan211 --- datafusion/functions-aggregate/src/lib.rs | 1 + datafusion/functions-aggregate/src/sum.rs | 11 +++++------ 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index 3601c8821121..37124204a095 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -76,6 +76,7 @@ pub fn register_all(registry: &mut dyn FunctionRegistry) -> Result<()> { let functions: Vec> = vec![ first_last::first_value_udaf(), covariance::covar_samp_udaf(), + sum::sum_udaf(), ]; functions.into_iter().try_for_each(|udf| { diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index 63665723fd3f..059baee2fcaf 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -40,6 +40,7 @@ use datafusion_physical_expr_common::aggregate::groups_accumulator::prim_op::Pri make_udaf_expr_and_func!( Sum, sum, + expression, "Returns the first value in a group of values.", sum_udaf ); @@ -67,24 +68,22 @@ macro_rules! downcast_sum { #[derive(Debug)] pub struct Sum { - pub is_distinct: bool, signature: Signature, aliases: Vec, } impl Sum { - pub fn new(is_distinct: bool) -> Self { + pub fn new() -> Self { Self { signature: Signature::uniform(1, NUMERICS.to_vec(), Volatility::Immutable), - aliases: vec![], - is_distinct, + aliases: vec!["sum".to_string()], } } } impl Default for Sum { fn default() -> Self { - Self::new(false) + Self::new() } } @@ -94,7 +93,7 @@ impl AggregateUDFImpl for Sum { } fn name(&self) -> &str { - "sum" + "SUM" } fn signature(&self) -> &Signature { From 3e9e7e986cc4c75872957220496780911804aecb Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sun, 19 May 2024 10:14:55 +0800 Subject: [PATCH 08/41] merge fix Signed-off-by: jayzhan211 --- datafusion-examples/examples/advanced_udaf.rs | 2 +- .../user_defined/user_defined_aggregates.rs | 5 +- datafusion/expr/src/function.rs | 7 -- datafusion/expr/src/udaf.rs | 4 +- datafusion/functions-aggregate/src/sum.rs | 78 ++----------------- .../physical-expr-common/src/aggregate/mod.rs | 31 +++++--- 6 files changed, 31 insertions(+), 96 deletions(-) diff --git a/datafusion-examples/examples/advanced_udaf.rs b/datafusion-examples/examples/advanced_udaf.rs index c837a93f5690..2c672a18a738 100644 --- a/datafusion-examples/examples/advanced_udaf.rs +++ b/datafusion-examples/examples/advanced_udaf.rs @@ -107,7 +107,7 @@ impl AggregateUDFImpl for GeoMeanUdaf { fn create_groups_accumulator( &self, - _args: GroupsAccumulatorArgs, + _args: AccumulatorArgs, ) -> Result> { Ok(Box::new(GeometricMeanGroupsAccumulator::new())) } diff --git a/datafusion/core/tests/user_defined/user_defined_aggregates.rs b/datafusion/core/tests/user_defined/user_defined_aggregates.rs index 1f2cd31881da..64bfa3dd322a 100644 --- a/datafusion/core/tests/user_defined/user_defined_aggregates.rs +++ b/datafusion/core/tests/user_defined/user_defined_aggregates.rs @@ -730,10 +730,7 @@ impl AggregateUDFImpl for TestGroupsAccumulator { true } - fn create_groups_accumulator( - &self, - _args: GroupsAccumulatorArgs, - ) -> Result> { + fn create_groups_accumulator(&self) -> Result> { Ok(Box::new(self.clone())) } } diff --git a/datafusion/expr/src/function.rs b/datafusion/expr/src/function.rs index ae7ef1d4da1c..8e3ad23498d5 100644 --- a/datafusion/expr/src/function.rs +++ b/datafusion/expr/src/function.rs @@ -38,13 +38,6 @@ pub type ScalarFunctionImplementation = pub type ReturnTypeFunction = Arc Result> + Send + Sync>; -pub struct GroupsAccumulatorArgs<'a> { - /// The return type of the aggregate function. - pub data_type: &'a DataType, - /// The name of the aggregate expression - pub name: &'a str, -} - /// [`AccumulatorArgs`] contains information about how an aggregate /// function was called, including the types of its arguments and any optional /// ordering expressions. diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index b796e32a2d8e..2e21d00d75ec 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -191,7 +191,7 @@ impl AggregateUDF { /// See [`AggregateUDFImpl::create_groups_accumulator`] for more details. pub fn create_groups_accumulator( &self, - args: GroupsAccumulatorArgs, + args: AccumulatorArgs, ) -> Result> { self.inner.create_groups_accumulator(args) } @@ -365,7 +365,7 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// implemented in addition to [`Accumulator`]. fn create_groups_accumulator( &self, - _args: GroupsAccumulatorArgs, + _args: AccumulatorArgs, ) -> Result> { not_impl_err!("GroupsAccumulator hasn't been implemented for {self:?} yet") } diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index 059baee2fcaf..c8e6a1a70943 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -29,7 +29,8 @@ use arrow::datatypes::{ }; use arrow::{array::ArrayRef, datatypes::Field}; use datafusion_common::{exec_err, not_impl_err, Result, ScalarValue}; -use datafusion_expr::function::{AccumulatorArgs, GroupsAccumulatorArgs}; +use datafusion_expr::function::AccumulatorArgs; +use datafusion_expr::function::StateFieldsArgs; use datafusion_expr::type_coercion::aggregates::NUMERICS; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ @@ -136,15 +137,10 @@ impl AggregateUDFImpl for Sum { downcast_sum!(args, helper) } - fn state_fields( - &self, - name: &str, - value_type: DataType, - _ordering_fields: Vec, - ) -> Result> { + fn state_fields(&self, args: StateFieldsArgs) -> Result> { Ok(vec![Field::new( - format_state_name(name, "sum"), - value_type, + format_state_name(args.name, "sum"), + args.return_type.clone(), true, )]) } @@ -153,13 +149,13 @@ impl AggregateUDFImpl for Sum { &self.aliases } - fn groups_accumulator_supported(&self) -> bool { + fn groups_accumulator_supported(&self, _args: AccumulatorArgs) -> bool { true } fn create_groups_accumulator( &self, - args: GroupsAccumulatorArgs, + args: AccumulatorArgs, ) -> Result> { macro_rules! helper { ($t:ty, $dt:expr) => { @@ -170,42 +166,6 @@ impl AggregateUDFImpl for Sum { }; } downcast_sum!(args, helper) - - // let return_type = args.data_type; - - // match return_type { - // DataType::UInt64 => Ok(Box::new( - // PrimitiveGroupsAccumulator::::new(&return_type, |x, y| { - // *x = x.add_wrapping(y) - // }), - // )), - // DataType::Int64 => Ok(Box::new( - // PrimitiveGroupsAccumulator::::new(&return_type, |x, y| { - // *x = x.add_wrapping(y) - // }), - // )), - // DataType::Float64 => { - // Ok(Box::new(PrimitiveGroupsAccumulator::::new( - // &return_type, - // |x, y| *x = x.add_wrapping(y), - // ))) - // } - // DataType::Decimal128(_, _) => Ok(Box::new(PrimitiveGroupsAccumulator::< - // Decimal128Type, - // _, - // >::new( - // &return_type, - // |x, y| *x = x.add_wrapping(y), - // ))), - // DataType::Decimal256(_, _) => Ok(Box::new(PrimitiveGroupsAccumulator::< - // Decimal256Type, - // _, - // >::new( - // &return_type, - // |x, y| *x = x.add_wrapping(y), - // ))), - // _ => not_impl_err!("Sum not supported for {}: {}", args.name, return_type), - // } } fn create_sliding_accumulator( @@ -218,30 +178,6 @@ impl AggregateUDFImpl for Sum { }; } downcast_sum!(args, helper) - // let return_type = args.data_type; - - // match return_type { - // DataType::UInt64 => Ok(Box::new(SlidingSumAccumulator::::new( - // return_type.clone(), - // ))), - // DataType::Int64 => Ok(Box::new(SlidingSumAccumulator::::new( - // return_type.clone(), - // ))), - // DataType::Float64 => Ok(Box::new(SlidingSumAccumulator::::new( - // return_type.clone(), - // ))), - // DataType::Decimal128(_, _) => { - // Ok(Box::new(SlidingSumAccumulator::::new( - // return_type.clone(), - // ))) - // } - // DataType::Decimal256(_, _) => { - // Ok(Box::new(SlidingSumAccumulator::::new( - // return_type.clone(), - // ))) - // } - // _ => not_impl_err!("Sum not supported for {}: {}", args.name, return_type), - // } } fn reverse_expr(&self) -> ReversedUDAF { diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 3258b2a2cf8f..f7fc44bccd6e 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -225,19 +225,22 @@ impl AggregateExpr for AggregateFunctionExpr { name: &self.name, }; - self.fun.accumulator(args) + self.fun.accumulator(acc_args) } fn create_sliding_accumulator(&self) -> Result> { - let args = AccumulatorArgs::new( - &self.data_type, - &self.schema, - self.ignore_nulls, - &self.sort_exprs, - &self.name, - ); + let args = AccumulatorArgs { + data_type: &self.data_type, + schema: &self.schema, + ignore_nulls: self.ignore_nulls, + sort_exprs: &self.sort_exprs, + is_distinct: self.is_distinct, + input_type: &self.input_type, + args_num: self.args.len(), + name: &self.name, + }; - let accumulator = self.fun().create_sliding_accumulator(args)?; + let accumulator = self.fun.create_sliding_accumulator(args)?; // Accumulators that have window frame startings different // than `UNBOUNDED PRECEDING`, such as `1 PRECEEDING`, need to @@ -310,9 +313,15 @@ impl AggregateExpr for AggregateFunctionExpr { } fn create_groups_accumulator(&self) -> Result> { - let args = GroupsAccumulatorArgs { + let args = AccumulatorArgs { data_type: &self.data_type, - name: self.name(), + schema: &self.schema, + ignore_nulls: self.ignore_nulls, + sort_exprs: &self.sort_exprs, + is_distinct: self.is_distinct, + input_type: &self.input_type, + args_num: self.args.len(), + name: &self.name, }; self.fun.create_groups_accumulator(args) } From 22540b231c3ea3816e9b3a2652f9bc59f89bd194 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sun, 19 May 2024 11:12:21 +0800 Subject: [PATCH 09/41] fix sum sig Signed-off-by: jayzhan211 --- datafusion/expr/src/expr_schema.rs | 15 +++++- .../expr/src/type_coercion/aggregates.rs | 3 ++ datafusion/expr/src/udaf.rs | 28 ++++++++++- datafusion/functions-aggregate/src/sum.rs | 49 ++++++++++++++----- .../optimizer/src/analyzer/type_coercion.rs | 2 +- .../physical-expr-common/src/aggregate/mod.rs | 12 ++--- 6 files changed, 85 insertions(+), 24 deletions(-) diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 2c08dbe0429a..d032e2442013 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -23,7 +23,7 @@ use crate::expr::{ }; use crate::field_util::GetFieldAccessSchema; use crate::type_coercion::binary::get_result_type; -use crate::type_coercion::functions::data_types_with_scalar_udf; +use crate::type_coercion::functions::{data_types_with_aggregate_udf, data_types_with_scalar_udf}; use crate::{utils, LogicalPlan, Projection, Subquery}; use arrow::compute::can_cast_types; use arrow::datatypes::{DataType, Field}; @@ -172,7 +172,18 @@ impl ExprSchemable for Expr { fun.return_type(&data_types) } AggregateFunctionDefinition::UDF(fun) => { - Ok(fun.return_type(&data_types)?) + let new_types = data_types_with_aggregate_udf(&data_types, fun).map_err(|err| { + plan_datafusion_err!( + "{} and {}", + err, + utils::generate_signature_error_msg( + fun.name(), + fun.signature().clone(), + &data_types + ) + ) + })?; + Ok(fun.return_type(&new_types)?) } } } diff --git a/datafusion/expr/src/type_coercion/aggregates.rs b/datafusion/expr/src/type_coercion/aggregates.rs index 57c0b6f4edc5..e8cd6740be2c 100644 --- a/datafusion/expr/src/type_coercion/aggregates.rs +++ b/datafusion/expr/src/type_coercion/aggregates.rs @@ -355,6 +355,9 @@ pub fn check_arg_count( ); } } + TypeSignature::UserDefined => { + // User-defined functions are not validated here + } _ => { return internal_err!( "Aggregate functions do not support this {signature:?}" diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 2e21d00d75ec..d71e5197fc6b 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -207,9 +207,10 @@ impl AggregateUDF { self.inner.reverse_expr() } - pub fn coerce_types(&self, _args: &[DataType]) -> Result> { - not_impl_err!("coerce_types not implemented for {:?} yet", self.name()) + pub fn coerce_types(&self, arg_types: &[DataType]) -> Result> { + self.inner.coerce_types(arg_types) } + /// Do the function rewrite /// /// See [`AggregateUDFImpl::simplify`] for more details. @@ -414,6 +415,29 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { fn reverse_expr(&self) -> ReversedUDAF { ReversedUDAF::NotSupported } + + /// Coerce arguments of a function call to types that the function can evaluate. + /// + /// This function is only called if [`AggregateUDFImpl::signature`] returns [`crate::TypeSignature::UserDefined`]. Most + /// UDAFs should return one of the other variants of `TypeSignature` which handle common + /// cases + /// + /// See the [type coercion module](crate::type_coercion) + /// documentation for more details on type coercion + /// + /// For example, if your function requires a floating point arguments, but the user calls + /// it like `my_func(1::int)` (aka with `1` as an integer), coerce_types could return `[DataType::Float64]` + /// to ensure the argument was cast to `1::double` + /// + /// # Parameters + /// * `arg_types`: The argument types of the arguments this function with + /// + /// # Return value + /// A Vec the same length as `arg_types`. DataFusion will `CAST` the function call + /// arguments to these specific types. + fn coerce_types(&self, _arg_types: &[DataType]) -> Result> { + not_impl_err!("Function {} does not implement coerce_types", self.name()) + } } pub enum ReversedUDAF { diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index c8e6a1a70943..8438ceecc558 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -31,7 +31,6 @@ use arrow::{array::ArrayRef, datatypes::Field}; use datafusion_common::{exec_err, not_impl_err, Result, ScalarValue}; use datafusion_expr::function::AccumulatorArgs; use datafusion_expr::function::StateFieldsArgs; -use datafusion_expr::type_coercion::aggregates::NUMERICS; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ Accumulator, AggregateUDFImpl, GroupsAccumulator, ReversedUDAF, Signature, Volatility, @@ -76,7 +75,7 @@ pub struct Sum { impl Sum { pub fn new() -> Self { Self { - signature: Signature::uniform(1, NUMERICS.to_vec(), Volatility::Immutable), + signature: Signature::user_defined(Volatility::Immutable), aliases: vec!["sum".to_string()], } } @@ -101,22 +100,22 @@ impl AggregateUDFImpl for Sum { &self.signature } - fn return_type(&self, arg_types: &[DataType]) -> Result { + fn coerce_types(&self, arg_types: &[DataType]) -> Result> { + if arg_types.len() != 1 { + return exec_err!("SUM expects exactly one argument"); + } + // Refer to https://www.postgresql.org/docs/8.2/functions-aggregate.html doc // smallint, int, bigint, real, double precision, decimal, or interval. - fn coerced_types(data_type: &DataType) -> Result { + fn coerced_type(data_type: &DataType) -> Result { match data_type { - DataType::Dictionary(_, v) => coerced_types(v), + DataType::Dictionary(_, v) => coerced_type(v), // in the spark, the result type is DECIMAL(min(38,precision+10), s) // ref: https://github.com/apache/spark/blob/fcf636d9eb8d645c24be3db2d599aba2d7e2955a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala#L66 - DataType::Decimal128(precision, scale) => { - let new_precision = DECIMAL128_MAX_PRECISION.min(*precision + 10); - Ok(DataType::Decimal128(new_precision, *scale)) - } - DataType::Decimal256(precision, scale) => { - let new_precision = DECIMAL256_MAX_PRECISION.min(*precision + 10); - Ok(DataType::Decimal256(new_precision, *scale)) + DataType::Decimal128(_, _) | + DataType::Decimal256(_, _) => { + Ok(data_type.clone()) } dt if dt.is_signed_integer() => Ok(DataType::Int64), dt if dt.is_unsigned_integer() => Ok(DataType::UInt64), @@ -125,7 +124,31 @@ impl AggregateUDFImpl for Sum { } } - coerced_types(&arg_types[0]) + Ok(vec![coerced_type(&arg_types[0])?]) + } + + fn return_type(&self, arg_types: &[DataType]) -> Result { + match &arg_types[0] { + DataType::Int64 => Ok(DataType::Int64), + DataType::UInt64 => Ok(DataType::UInt64), + DataType::Float64 => Ok(DataType::Float64), + DataType::Decimal128(precision, scale) => { + // in the spark, the result type is DECIMAL(min(38,precision+10), s) + // ref: https://github.com/apache/spark/blob/fcf636d9eb8d645c24be3db2d599aba2d7e2955a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala#L66 + let new_precision = DECIMAL128_MAX_PRECISION.min(*precision + 10); + Ok(DataType::Decimal128(new_precision, *scale)) + } + DataType::Decimal256(precision, scale) => { + // in the spark, the result type is DECIMAL(min(38,precision+10), s) + // ref: https://github.com/apache/spark/blob/fcf636d9eb8d645c24be3db2d599aba2d7e2955a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala#L66 + let new_precision = DECIMAL256_MAX_PRECISION.min(*precision + 10); + Ok(DataType::Decimal256(new_precision, *scale)) + } + other => { + panic!("asdf"); + exec_err!("[return_type] SUM not supported for {}", other) + }, + } } fn accumulator(&self, args: AccumulatorArgs) -> Result> { diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 0f1f3ba7e729..e366e663e0ba 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -402,7 +402,7 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { expr::AggregateFunction::new_udf( fun, new_expr, - false, + distinct, filter, order_by, null_treatment, diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index f7fc44bccd6e..2552a4e87ad4 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -22,7 +22,7 @@ pub mod utils; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::{not_impl_err, Result}; use datafusion_expr::function::StateFieldsArgs; -use datafusion_expr::type_coercion::aggregates::check_arg_count; +// use datafusion_expr::type_coercion::aggregates::check_arg_count; use datafusion_expr::ReversedUDAF; use datafusion_expr::{ function::AccumulatorArgs, Accumulator, AggregateUDF, Expr, GroupsAccumulator, @@ -53,11 +53,11 @@ pub fn create_aggregate_expr( .map(|arg| arg.data_type(schema)) .collect::>>()?; - check_arg_count( - fun.name(), - &input_exprs_types, - &fun.signature().type_signature, - )?; + // check_arg_count( + // fun.name(), + // &input_exprs_types, + // &fun.signature().type_signature, + // )?; let ordering_types = ordering_req .iter() From d9229dbc03b7d31e3c9e8b056743224fb3548281 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sun, 19 May 2024 11:21:38 +0800 Subject: [PATCH 10/41] todo: wait ahash merge Signed-off-by: jayzhan211 --- datafusion/expr/src/expr_schema.rs | 4 +- datafusion/functions-aggregate/src/sum.rs | 129 ++++++++++++++++-- .../src/aggregate/utils.rs | 21 ++- 3 files changed, 137 insertions(+), 17 deletions(-) diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index d032e2442013..bf6bea9d1429 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -23,7 +23,9 @@ use crate::expr::{ }; use crate::field_util::GetFieldAccessSchema; use crate::type_coercion::binary::get_result_type; -use crate::type_coercion::functions::{data_types_with_aggregate_udf, data_types_with_scalar_udf}; +use crate::type_coercion::functions::{ + data_types_with_aggregate_udf, data_types_with_scalar_udf, +}; use crate::{utils, LogicalPlan, Projection, Subquery}; use arrow::compute::can_cast_types; use arrow::datatypes::{DataType, Field}; diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index 8438ceecc558..baaa4f9e6cad 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -18,11 +18,14 @@ //! Defines `SUM` and `SUM DISTINCT` aggregate accumulators use std::any::Any; +use std::collections::HashSet; +use ahash::RandomState; use arrow::array::Array; use arrow::array::ArrowNativeTypeOp; use arrow::array::{ArrowNumericType, AsArray}; use arrow::datatypes::ArrowNativeType; +use arrow::datatypes::ArrowPrimitiveType; use arrow::datatypes::{ DataType, Decimal128Type, Decimal256Type, Float64Type, Int64Type, UInt64Type, DECIMAL128_MAX_PRECISION, DECIMAL256_MAX_PRECISION, @@ -36,6 +39,7 @@ use datafusion_expr::{ Accumulator, AggregateUDFImpl, GroupsAccumulator, ReversedUDAF, Signature, Volatility, }; use datafusion_physical_expr_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; +use datafusion_physical_expr_common::aggregate::utils::Hashable; make_udaf_expr_and_func!( Sum, @@ -113,8 +117,7 @@ impl AggregateUDFImpl for Sum { DataType::Dictionary(_, v) => coerced_type(v), // in the spark, the result type is DECIMAL(min(38,precision+10), s) // ref: https://github.com/apache/spark/blob/fcf636d9eb8d645c24be3db2d599aba2d7e2955a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala#L66 - DataType::Decimal128(_, _) | - DataType::Decimal256(_, _) => { + DataType::Decimal128(_, _) | DataType::Decimal256(_, _) => { Ok(data_type.clone()) } dt if dt.is_signed_integer() => Ok(DataType::Int64), @@ -145,27 +148,43 @@ impl AggregateUDFImpl for Sum { Ok(DataType::Decimal256(new_precision, *scale)) } other => { - panic!("asdf"); exec_err!("[return_type] SUM not supported for {}", other) - }, + } } } fn accumulator(&self, args: AccumulatorArgs) -> Result> { - macro_rules! helper { - ($t:ty, $dt:expr) => { - Ok(Box::new(SumAccumulator::<$t>::new($dt.clone()))) - }; + if args.is_distinct { + macro_rules! helper { + ($t:ty, $dt:expr) => { + Ok(Box::new(DistinctSumAccumulator::<$t>::new($dt.clone()))) + }; + } + downcast_sum!(args, helper) + } else { + macro_rules! helper { + ($t:ty, $dt:expr) => { + Ok(Box::new(SumAccumulator::<$t>::new($dt.clone()))) + }; + } + downcast_sum!(args, helper) } - downcast_sum!(args, helper) } fn state_fields(&self, args: StateFieldsArgs) -> Result> { - Ok(vec![Field::new( - format_state_name(args.name, "sum"), - args.return_type.clone(), - true, - )]) + if args.is_distinct { + Ok(vec![Field::new_list( + format_state_name(args.name, "sum distinct"), + Field::new("item", args.return_type.clone(), true), + false, + )]) + } else { + Ok(vec![Field::new( + format_state_name(args.name, "sum"), + args.return_type.clone(), + true, + )]) + } } fn aliases(&self) -> &[String] { @@ -328,3 +347,85 @@ impl Accumulator for SlidingSumAccumulator { true } } + +struct DistinctSumAccumulator { + values: HashSet, RandomState>, + data_type: DataType, +} + +impl std::fmt::Debug for DistinctSumAccumulator { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "DistinctSumAccumulator({})", self.data_type) + } +} + +impl DistinctSumAccumulator { + pub fn try_new(data_type: &DataType) -> Result { + Ok(Self { + values: HashSet::default(), + data_type: data_type.clone(), + }) + } +} + +impl Accumulator for DistinctSumAccumulator { + fn state(&mut self) -> Result> { + // 1. Stores aggregate state in `ScalarValue::List` + // 2. Constructs `ScalarValue::List` state from distinct numeric stored in hash set + let state_out = { + let distinct_values = self + .values + .iter() + .map(|value| { + ScalarValue::new_primitive::(Some(value.0), &self.data_type) + }) + .collect::>>()?; + + vec![ScalarValue::List(ScalarValue::new_list( + &distinct_values, + &self.data_type, + ))] + }; + Ok(state_out) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + if values.is_empty() { + return Ok(()); + } + + let array = values[0].as_primitive::(); + match array.nulls().filter(|x| x.null_count() > 0) { + Some(n) => { + for idx in n.valid_indices() { + self.values.insert(Hashable(array.value(idx))); + } + } + None => array.values().iter().for_each(|x| { + self.values.insert(Hashable(*x)); + }), + } + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + for x in states[0].as_list::().iter().flatten() { + self.update_batch(&[x])? + } + Ok(()) + } + + fn evaluate(&mut self) -> Result { + let mut acc = T::Native::usize_as(0); + for distinct_value in self.values.iter() { + acc = acc.add_wrapping(distinct_value.0) + } + let v = (!self.values.is_empty()).then_some(acc); + ScalarValue::new_primitive::(v, &self.data_type) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + + self.values.capacity() * std::mem::size_of::() + } +} \ No newline at end of file diff --git a/datafusion/physical-expr-common/src/aggregate/utils.rs b/datafusion/physical-expr-common/src/aggregate/utils.rs index 9821ba626b18..c52b1f54baa3 100644 --- a/datafusion/physical-expr-common/src/aggregate/utils.rs +++ b/datafusion/physical-expr-common/src/aggregate/utils.rs @@ -18,8 +18,7 @@ use std::{any::Any, sync::Arc}; use arrow::{ - compute::SortOptions, - datatypes::{DataType, Field}, + array::ArrowNativeTypeOp, compute::SortOptions, datatypes::{DataType, Field, ToByteSlice} }; use crate::sort_expr::PhysicalSortExpr; @@ -67,3 +66,21 @@ pub fn ordering_fields( pub fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { ordering_req.iter().map(|item| item.options).collect() } + +/// A wrapper around a type to provide hash for floats +#[derive(Copy, Clone, Debug)] +pub struct Hashable(pub T); + +impl std::hash::Hash for Hashable { + fn hash(&self, state: &mut H) { + self.0.to_byte_slice().hash(state) + } +} + +impl PartialEq for Hashable { + fn eq(&self, other: &Self) -> bool { + self.0.is_eq(other.0) + } +} + +impl Eq for Hashable {} \ No newline at end of file From ee068db34705ceba8c4caef5ccc3ede277bafa2e Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 25 May 2024 09:43:19 +0800 Subject: [PATCH 11/41] rebase Signed-off-by: jayzhan211 --- datafusion/expr/src/tree_node.rs | 2 +- datafusion/functions-aggregate/Cargo.toml | 1 + datafusion/functions-aggregate/src/sum.rs | 10 ++-- .../src/single_distinct_to_groupby.rs | 49 +++++++++++++++++++ .../src/aggregate/utils.rs | 4 +- .../src/aggregate/groups_accumulator/mod.rs | 2 +- .../sqllogictest/test_files/aggregate.slt | 32 +++++++++--- 7 files changed, 85 insertions(+), 15 deletions(-) diff --git a/datafusion/expr/src/tree_node.rs b/datafusion/expr/src/tree_node.rs index 31ca4c40942b..c5f1694c1138 100644 --- a/datafusion/expr/src/tree_node.rs +++ b/datafusion/expr/src/tree_node.rs @@ -332,7 +332,7 @@ impl TreeNode for Expr { Ok(Expr::AggregateFunction(AggregateFunction::new_udf( fun, new_args, - false, + distinct, new_filter, new_order_by, null_treatment, diff --git a/datafusion/functions-aggregate/Cargo.toml b/datafusion/functions-aggregate/Cargo.toml index f97647565364..4184b3a484d9 100644 --- a/datafusion/functions-aggregate/Cargo.toml +++ b/datafusion/functions-aggregate/Cargo.toml @@ -38,6 +38,7 @@ path = "src/lib.rs" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] +ahash = { workspace = true } arrow = { workspace = true } datafusion-common = { workspace = true } datafusion-execution = { workspace = true } diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index baaa4f9e6cad..8d6b17a67d10 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -17,9 +17,9 @@ //! Defines `SUM` and `SUM DISTINCT` aggregate accumulators +use ahash::RandomState; use std::any::Any; use std::collections::HashSet; -use ahash::RandomState; use arrow::array::Array; use arrow::array::ArrowNativeTypeOp; @@ -157,7 +157,7 @@ impl AggregateUDFImpl for Sum { if args.is_distinct { macro_rules! helper { ($t:ty, $dt:expr) => { - Ok(Box::new(DistinctSumAccumulator::<$t>::new($dt.clone()))) + Ok(Box::new(DistinctSumAccumulator::<$t>::try_new(&$dt)?)) }; } downcast_sum!(args, helper) @@ -191,8 +191,8 @@ impl AggregateUDFImpl for Sum { &self.aliases } - fn groups_accumulator_supported(&self, _args: AccumulatorArgs) -> bool { - true + fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { + !args.is_distinct } fn create_groups_accumulator( @@ -428,4 +428,4 @@ impl Accumulator for DistinctSumAccumulator { std::mem::size_of_val(self) + self.values.capacity() * std::mem::size_of::() } -} \ No newline at end of file +} diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index 4b1f9a0d1401..06d0dee27099 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -257,6 +257,55 @@ impl OptimizerRule for SingleDistinctToGroupBy { ))) } } + Expr::AggregateFunction(AggregateFunction { + func_def: AggregateFunctionDefinition::UDF(udf), + mut args, + distinct, + .. + }) => { + if distinct { + if args.len() != 1 { + return internal_err!("DISTINCT aggregate should have exactly one argument"); + } + let arg = args.swap_remove(0); + + if group_fields_set.insert(arg.display_name()?) { + inner_group_exprs + .push(arg.alias(SINGLE_DISTINCT_ALIAS)); + } + Ok(Expr::AggregateFunction(AggregateFunction::new_udf( + udf, + vec![col(SINGLE_DISTINCT_ALIAS)], + false, // intentional to remove distinct here + None, + None, + None, + ))) + // if the aggregate function is not distinct, we need to rewrite it like two phase aggregation + } else { + index += 1; + let alias_str = format!("alias{}", index); + inner_aggr_exprs.push( + Expr::AggregateFunction(AggregateFunction::new_udf( + udf.clone(), + args, + false, + None, + None, + None, + )) + .alias(&alias_str), + ); + Ok(Expr::AggregateFunction(AggregateFunction::new_udf( + udf, + vec![col(&alias_str)], + false, + None, + None, + None, + ))) + } + } _ => Ok(aggr_expr), }) .collect::>>()?; diff --git a/datafusion/physical-expr-common/src/aggregate/utils.rs b/datafusion/physical-expr-common/src/aggregate/utils.rs index 253d5d0acdc0..bcd0d05be054 100644 --- a/datafusion/physical-expr-common/src/aggregate/utils.rs +++ b/datafusion/physical-expr-common/src/aggregate/utils.rs @@ -17,10 +17,10 @@ use std::{any::Any, sync::Arc}; +use arrow::array::{ArrayRef, AsArray}; use arrow::datatypes::ArrowNativeType; use arrow::{ - array::ArrowNativeTypeOp, compute::SortOptions, datatypes::{DataType, Field, ToByteSlice} - array::{ArrayRef, ArrowNativeTypeOp, AsArray}, + array::ArrowNativeTypeOp, compute::SortOptions, datatypes::{ DataType, Decimal128Type, DecimalType, Field, TimeUnit, TimestampMicrosecondType, diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs b/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs index 50daba511b7b..65227b727be7 100644 --- a/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs +++ b/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs @@ -30,4 +30,4 @@ pub(crate) mod bool_op { } pub(crate) mod prim_op { pub use datafusion_physical_expr_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; -} \ No newline at end of file +} diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index c2478e543735..9adebea1b4fa 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -3292,10 +3292,10 @@ NULL NULL NULL NULL NULL NULL NULL NULL Row 2 Y # aggregate_timestamps_sum -statement error DataFusion error: Error during planning: No function matches the given name and argument types 'SUM\(Timestamp\(Nanosecond, None\)\)'\. You might need to add explicit type casts\. +query error SELECT sum(nanos), sum(micros), sum(millis), sum(secs) FROM t; -statement error DataFusion error: Error during planning: No function matches the given name and argument types 'SUM\(Timestamp\(Nanosecond, None\)\)'\. You might need to add explicit type casts\. +query error SELECT tag, sum(nanos), sum(micros), sum(millis), sum(secs) FROM t GROUP BY tag ORDER BY tag; # aggregate_timestamps_count @@ -3403,11 +3403,21 @@ NULL NULL Row 2 Y # aggregate_timestamps_sum -statement error DataFusion error: Error during planning: No function matches the given name and argument types 'SUM\(Date32\)'\. You might need to add explicit type casts\. +statement error SELECT sum(date32), sum(date64) FROM t; +---- +DataFusion error: Error during planning: Execution error: User-defined coercion failed with Execution("Sum not supported for Date32") and No function matches the given name and argument types 'SUM(Date32)'. You might need to add explicit type casts. + Candidate functions: + SUM(UserDefined) + -statement error DataFusion error: Error during planning: No function matches the given name and argument types 'SUM\(Date32\)'\. You might need to add explicit type casts\. +statement error SELECT tag, sum(date32), sum(date64) FROM t GROUP BY tag ORDER BY tag; +---- +DataFusion error: Error during planning: Execution error: User-defined coercion failed with Execution("Sum not supported for Date32") and No function matches the given name and argument types 'SUM(Date32)'. You might need to add explicit type casts. + Candidate functions: + SUM(UserDefined) + # aggregate_timestamps_count query II @@ -3500,11 +3510,21 @@ select * from t; 21:06:28.247821084 21:06:28.247821 21:06:28.247 21:06:28 Row 3 B # aggregate_times_sum -statement error DataFusion error: Error during planning: No function matches the given name and argument types 'SUM\(Time64\(Nanosecond\)\)'\. You might need to add explicit type casts\. +statement error SELECT sum(nanos), sum(micros), sum(millis), sum(secs) FROM t +---- +DataFusion error: Error during planning: Execution error: User-defined coercion failed with Execution("Sum not supported for Time64(Nanosecond)") and No function matches the given name and argument types 'SUM(Time64(Nanosecond))'. You might need to add explicit type casts. + Candidate functions: + SUM(UserDefined) + -statement error DataFusion error: Error during planning: No function matches the given name and argument types 'SUM\(Time64\(Nanosecond\)\)'\. You might need to add explicit type casts\. +statement error SELECT tag, sum(nanos), sum(micros), sum(millis), sum(secs) FROM t GROUP BY tag ORDER BY tag +---- +DataFusion error: Error during planning: Execution error: User-defined coercion failed with Execution("Sum not supported for Time64(Nanosecond)") and No function matches the given name and argument types 'SUM(Time64(Nanosecond))'. You might need to add explicit type casts. + Candidate functions: + SUM(UserDefined) + # aggregate_times_count query IIII From 622433359d7f87aeb57d10024e49a00ce137df61 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 25 May 2024 10:26:40 +0800 Subject: [PATCH 12/41] disable ordering req by default Signed-off-by: jayzhan211 --- datafusion/expr/src/udaf.rs | 9 +++++++++ datafusion/physical-expr-common/src/aggregate/mod.rs | 5 ++++- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index d71e5197fc6b..751c684d5704 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -217,6 +217,10 @@ impl AggregateUDF { pub fn simplify(&self) -> Option { self.inner.simplify() } + + pub fn has_ordering_requirements(&self) -> bool { + self.inner.has_ordering_requirements() + } } impl From for AggregateUDF @@ -438,6 +442,11 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { fn coerce_types(&self, _arg_types: &[DataType]) -> Result> { not_impl_err!("Function {} does not implement coerce_types", self.name()) } + + /// Returns true if the function has ordering requirements. + fn has_ordering_requirements(&self) -> bool { + false + } } pub enum ReversedUDAF { diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 2552a4e87ad4..a3f328641e43 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -327,7 +327,10 @@ impl AggregateExpr for AggregateFunctionExpr { } fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { - (!self.ordering_req.is_empty()).then_some(&self.ordering_req) + if self.fun.has_ordering_requirements() && !self.ordering_req.is_empty() { + return Some(&self.ordering_req); + } + None } fn reverse_expr(&self) -> Option> { From 47ae11f818158b4b68ef07d2dd48f8bc082a5bec Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 25 May 2024 10:46:38 +0800 Subject: [PATCH 13/41] check arg count Signed-off-by: jayzhan211 --- datafusion/physical-expr-common/src/aggregate/mod.rs | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index a3f328641e43..0575487f1212 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -22,6 +22,7 @@ pub mod utils; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::{not_impl_err, Result}; use datafusion_expr::function::StateFieldsArgs; +use datafusion_expr::type_coercion::aggregates::check_arg_count; // use datafusion_expr::type_coercion::aggregates::check_arg_count; use datafusion_expr::ReversedUDAF; use datafusion_expr::{ @@ -53,11 +54,11 @@ pub fn create_aggregate_expr( .map(|arg| arg.data_type(schema)) .collect::>>()?; - // check_arg_count( - // fun.name(), - // &input_exprs_types, - // &fun.signature().type_signature, - // )?; + check_arg_count( + fun.name(), + &input_exprs_types, + &fun.signature().type_signature, + )?; let ordering_types = ordering_req .iter() From 25dcb646be4f142de84af5a1cf64d87665f26254 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 25 May 2024 12:22:39 +0800 Subject: [PATCH 14/41] rm old workflow Signed-off-by: jayzhan211 --- datafusion-cli/Cargo.lock | 1 + .../examples/simplify_udaf_expression.rs | 6 ++++- .../user_defined/user_defined_aggregates.rs | 10 ++++--- datafusion/expr/src/expr.rs | 15 ++++++++--- datafusion/expr/src/expr_schema.rs | 26 +++++++++++++++++-- datafusion/expr/src/udaf.rs | 6 +++++ .../optimizer/src/analyzer/type_coercion.rs | 7 +++++ .../simplify_expressions/expr_simplifier.rs | 5 +++- .../physical-expr/src/aggregate/build_in.rs | 15 +++-------- datafusion/physical-plan/src/windows/mod.rs | 3 +-- 10 files changed, 69 insertions(+), 25 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 99af80bf9df2..062db8958629 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1286,6 +1286,7 @@ dependencies = [ name = "datafusion-functions-aggregate" version = "38.0.0" dependencies = [ + "ahash", "arrow", "datafusion-common", "datafusion-execution", diff --git a/datafusion-examples/examples/simplify_udaf_expression.rs b/datafusion-examples/examples/simplify_udaf_expression.rs index 08b6bcab0190..5b6ea0f4ba6c 100644 --- a/datafusion-examples/examples/simplify_udaf_expression.rs +++ b/datafusion-examples/examples/simplify_udaf_expression.rs @@ -78,9 +78,13 @@ impl AggregateUDFImpl for BetterAvgUdaf { true } - fn create_groups_accumulator(&self) -> Result> { + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { unimplemented!("should not get here"); } + // we override method, to return new expression which would substitute // user defined function call fn simplify(&self) -> Option { diff --git a/datafusion/core/tests/user_defined/user_defined_aggregates.rs b/datafusion/core/tests/user_defined/user_defined_aggregates.rs index 64bfa3dd322a..5d2149dc4884 100644 --- a/datafusion/core/tests/user_defined/user_defined_aggregates.rs +++ b/datafusion/core/tests/user_defined/user_defined_aggregates.rs @@ -45,9 +45,8 @@ use datafusion::{ }; use datafusion_common::{assert_contains, cast::as_primitive_array, exec_err}; use datafusion_expr::{ - create_udaf, - function::{AccumulatorArgs, GroupsAccumulatorArgs}, - AggregateUDFImpl, GroupsAccumulator, SimpleAggregateUDF, + create_udaf, function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator, + SimpleAggregateUDF, }; use datafusion_physical_expr::expressions::AvgAccumulator; @@ -730,7 +729,10 @@ impl AggregateUDFImpl for TestGroupsAccumulator { true } - fn create_groups_accumulator(&self) -> Result> { + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { Ok(Box::new(self.clone())) } } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 5e43c160ba0a..bc06eeb8ae73 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -634,10 +634,14 @@ impl WindowFunctionDefinition { impl fmt::Display for WindowFunctionDefinition { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { match self { - WindowFunctionDefinition::AggregateFunction(fun) => fun.fmt(f), - WindowFunctionDefinition::BuiltInWindowFunction(fun) => fun.fmt(f), - WindowFunctionDefinition::AggregateUDF(fun) => std::fmt::Debug::fmt(fun, f), - WindowFunctionDefinition::WindowUDF(fun) => fun.fmt(f), + WindowFunctionDefinition::AggregateFunction(fun) => { + std::fmt::Display::fmt(fun, f) + } + WindowFunctionDefinition::BuiltInWindowFunction(fun) => { + std::fmt::Display::fmt(fun, f) + } + WindowFunctionDefinition::AggregateUDF(fun) => std::fmt::Display::fmt(fun, f), + WindowFunctionDefinition::WindowUDF(fun) => std::fmt::Display::fmt(fun, f), } } } @@ -694,6 +698,9 @@ pub fn find_df_window_func(name: &str) -> Option { Some(WindowFunctionDefinition::BuiltInWindowFunction( built_in_function, )) + // filter out aggregate function that is udaf + } else if name.as_str() == "sum" { + None } else if let Ok(aggregate) = aggregate_function::AggregateFunction::from_str(name.as_str()) { diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 0f3ff87171fe..35d99c0f3099 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -24,7 +24,7 @@ use crate::type_coercion::binary::get_result_type; use crate::type_coercion::functions::{ data_types_with_aggregate_udf, data_types_with_scalar_udf, }; -use crate::{utils, LogicalPlan, Projection, Subquery}; +use crate::{utils, LogicalPlan, Projection, Subquery, WindowFunctionDefinition}; use arrow::compute::can_cast_types; use arrow::datatypes::{DataType, Field}; use datafusion_common::{ @@ -160,7 +160,29 @@ impl ExprSchemable for Expr { .iter() .map(|e| e.get_type(schema)) .collect::>>()?; - fun.return_type(&data_types) + match fun { + WindowFunctionDefinition::AggregateUDF(udf) => { + let new_types = data_types_with_aggregate_udf(&data_types, udf).map_err(|err| { + plan_datafusion_err!( + "{} and {}", + err, + utils::generate_signature_error_msg( + fun.name(), + fun.signature().clone(), + &data_types + ) + ) + })?; + Ok(fun.return_type(&new_types)?) + } + _ => { + let data_types = args + .iter() + .map(|e| e.get_type(schema)) + .collect::>>()?; + fun.return_type(&data_types) + } + } } Expr::AggregateFunction(AggregateFunction { func_def, args, .. }) => { let data_types = args diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 751c684d5704..6df8fca3318f 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -82,6 +82,12 @@ impl std::hash::Hash for AggregateUDF { } } +impl std::fmt::Display for AggregateUDF { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + write!(f, "{}", self.name()) + } +} + impl AggregateUDF { /// Create a new AggregateUDF /// diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 69be344cb753..a397628b9f67 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -430,6 +430,13 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { &fun.signature(), )? } + expr::WindowFunctionDefinition::AggregateUDF(udf) => { + coerce_arguments_for_signature_with_aggregate_udf( + args, + self.schema, + udf, + )? + } _ => args, }; diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index 25504e5c78e7..ee8be2b27256 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -3788,7 +3788,10 @@ mod tests { unimplemented!("not needed for testing") } - fn create_groups_accumulator(&self) -> Result> { + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { unimplemented!("not needed for testing") } diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index 145e7feadf8c..66cff984255f 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -30,7 +30,7 @@ use std::sync::Arc; use arrow::datatypes::Schema; -use datafusion_common::{exec_err, not_impl_err, Result}; +use datafusion_common::{exec_err, internal_err, not_impl_err, Result}; use datafusion_expr::AggregateFunction; use crate::aggregate::regr::RegrType; @@ -104,16 +104,9 @@ pub fn create_aggregate_expr( name, data_type, )), - (AggregateFunction::Sum, false) => Arc::new(expressions::Sum::new( - input_phy_exprs[0].clone(), - name, - input_phy_types[0].clone(), - )), - (AggregateFunction::Sum, true) => Arc::new(expressions::DistinctSum::new( - vec![input_phy_exprs[0].clone()], - name, - data_type, - )), + (AggregateFunction::Sum, _) => { + return internal_err!("Builtin Sum will be removed"); + } (AggregateFunction::ApproxDistinct, _) => Arc::new( expressions::ApproxDistinct::new(input_phy_exprs[0].clone(), name, data_type), ), diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 42c630741cc9..58d677d293dd 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -119,13 +119,12 @@ pub fn create_window_expr( WindowFunctionDefinition::AggregateUDF(fun) => { // TODO: Ordering not supported for Window UDFs yet let sort_exprs = &[]; - let ordering_req = &[]; let aggregate = udaf::create_aggregate_expr( fun.as_ref(), args, sort_exprs, - ordering_req, + order_by, input_schema, name, ignore_nulls, From d16f1b1e705e890140c2cb4ed2f8e82bd07ccd9f Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 25 May 2024 12:27:31 +0800 Subject: [PATCH 15/41] fmt Signed-off-by: jayzhan211 --- datafusion-examples/examples/simplify_udaf_expression.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion-examples/examples/simplify_udaf_expression.rs b/datafusion-examples/examples/simplify_udaf_expression.rs index 5b6ea0f4ba6c..d2c8c6a86c7c 100644 --- a/datafusion-examples/examples/simplify_udaf_expression.rs +++ b/datafusion-examples/examples/simplify_udaf_expression.rs @@ -84,7 +84,7 @@ impl AggregateUDFImpl for BetterAvgUdaf { ) -> Result> { unimplemented!("should not get here"); } - + // we override method, to return new expression which would substitute // user defined function call fn simplify(&self) -> Option { From 5381f2dff0c8fe8f9e3d87b507bdc147b5540401 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 25 May 2024 13:33:56 +0800 Subject: [PATCH 16/41] fix failed test Signed-off-by: jayzhan211 --- datafusion/core/src/dataframe/mod.rs | 5 +++-- datafusion/core/src/physical_planner.rs | 3 ++- datafusion/functions-aggregate/src/lib.rs | 1 + 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index d4626134acbf..a85853b27fcc 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -53,7 +53,8 @@ use datafusion_expr::{ avg, count, max, median, min, stddev, utils::COUNT_STAR_EXPANSION, TableProviderFilterPushDown, UNNAMED_TABLE, }; -use datafusion_expr::{case, is_null, sum}; +use datafusion_expr::{case, is_null}; +use datafusion_functions_aggregate::expr_fn::sum; use async_trait::async_trait; @@ -1587,7 +1588,7 @@ mod tests { use datafusion_common::{Constraint, Constraints}; use datafusion_common_runtime::SpawnedTask; use datafusion_expr::{ - array_agg, cast, count_distinct, create_udf, expr, lit, sum, + array_agg, cast, count_distinct, create_udf, expr, lit, BuiltInWindowFunction, ScalarFunctionImplementation, Volatility, WindowFrame, WindowFunctionDefinition, }; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index bc5818361b7d..c2bdbfcad3ff 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2258,8 +2258,9 @@ mod tests { use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_expr::{ - col, lit, sum, LogicalPlanBuilder, UserDefinedLogicalNodeCore, + col, lit, LogicalPlanBuilder, UserDefinedLogicalNodeCore, }; + use datafusion_functions_aggregate::expr_fn::sum; use datafusion_physical_expr::EquivalenceProperties; fn make_session_state() -> SessionState { diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index 3d35c84893c4..18c9eb10fd87 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -69,6 +69,7 @@ use std::sync::Arc; pub mod expr_fn { pub use super::covariance::covar_samp; pub use super::first_last::first_value; + pub use super::sum::sum; } /// Registers all enabled packages with a [`FunctionRegistry`] From b403331057f1c7f2c1fa1f49f750da3aa7956108 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 25 May 2024 13:37:02 +0800 Subject: [PATCH 17/41] doc and fmt Signed-off-by: jayzhan211 --- datafusion/core/src/dataframe/mod.rs | 5 ++--- datafusion/core/src/physical_planner.rs | 4 +--- datafusion/expr/src/expr_fn.rs | 2 ++ 3 files changed, 5 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index a85853b27fcc..c595279d3ece 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1588,9 +1588,8 @@ mod tests { use datafusion_common::{Constraint, Constraints}; use datafusion_common_runtime::SpawnedTask; use datafusion_expr::{ - array_agg, cast, count_distinct, create_udf, expr, lit, - BuiltInWindowFunction, ScalarFunctionImplementation, Volatility, WindowFrame, - WindowFunctionDefinition, + array_agg, cast, count_distinct, create_udf, expr, lit, BuiltInWindowFunction, + ScalarFunctionImplementation, Volatility, WindowFrame, WindowFunctionDefinition, }; use datafusion_physical_expr::expressions::Column; use datafusion_physical_plan::{get_plan_string, ExecutionPlanProperties}; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index c2bdbfcad3ff..8d5ae1882683 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2257,9 +2257,7 @@ mod tests { use datafusion_common::{assert_contains, DFSchemaRef, TableReference}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; - use datafusion_expr::{ - col, lit, LogicalPlanBuilder, UserDefinedLogicalNodeCore, - }; + use datafusion_expr::{col, lit, LogicalPlanBuilder, UserDefinedLogicalNodeCore}; use datafusion_functions_aggregate::expr_fn::sum; use datafusion_physical_expr::EquivalenceProperties; diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 2a2bb75f1884..0428c1df2501 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -169,6 +169,8 @@ pub fn max(expr: Expr) -> Expr { } /// Create an expression to represent the sum() aggregate function +/// +/// TODO: Remove this function and use `sum` from `datafusion_functions_aggregate::expr_fn` instead pub fn sum(expr: Expr) -> Expr { Expr::AggregateFunction(AggregateFunction::new( aggregate_function::AggregateFunction::Sum, From 78a70b3a3c159350d935755e5f5111d202d5c4e5 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 25 May 2024 13:49:32 +0800 Subject: [PATCH 18/41] check udaf first Signed-off-by: jayzhan211 --- datafusion/core/tests/dataframe/mod.rs | 3 ++- datafusion/expr/src/expr.rs | 3 --- datafusion/functions-aggregate/src/first_last.rs | 3 ++- .../physical-expr-common/src/aggregate/mod.rs | 1 - .../proto/tests/cases/roundtrip_logical_plan.rs | 2 ++ datafusion/sql/src/expr/function.rs | 13 +++++-------- 6 files changed, 11 insertions(+), 14 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 60e60bb1e3b1..3a6dd3bb264a 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -52,10 +52,11 @@ use datafusion_expr::expr::{GroupingSet, Sort}; use datafusion_expr::var_provider::{VarProvider, VarType}; use datafusion_expr::{ array_agg, avg, cast, col, count, exists, expr, in_subquery, lit, max, out_ref_col, - placeholder, scalar_subquery, sum, when, wildcard, AggregateFunction, Expr, + placeholder, scalar_subquery, when, wildcard, AggregateFunction, Expr, ExprSchemable, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; +use datafusion_functions_aggregate::expr_fn::sum; #[tokio::test] async fn test_count_wildcard_on_sort() -> Result<()> { diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index bc06eeb8ae73..d3f01b5a0894 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -698,9 +698,6 @@ pub fn find_df_window_func(name: &str) -> Option { Some(WindowFunctionDefinition::BuiltInWindowFunction( built_in_function, )) - // filter out aggregate function that is udaf - } else if name.as_str() == "sum" { - None } else if let Ok(aggregate) = aggregate_function::AggregateFunction::from_str(name.as_str()) { diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 94b065d9aede..b2abf8e2cdf1 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -39,6 +39,7 @@ use datafusion_physical_expr_common::expressions; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_expr_common::utils::reverse_order_bys; + use std::any::Any; use std::fmt::Debug; use std::sync::Arc; @@ -1029,4 +1030,4 @@ mod tests { Ok(()) } -} +} \ No newline at end of file diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 0575487f1212..8fe8757c7862 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -23,7 +23,6 @@ use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::{not_impl_err, Result}; use datafusion_expr::function::StateFieldsArgs; use datafusion_expr::type_coercion::aggregates::check_arg_count; -// use datafusion_expr::type_coercion::aggregates::check_arg_count; use datafusion_expr::ReversedUDAF; use datafusion_expr::{ function::AccumulatorArgs, Accumulator, AggregateUDF, Expr, GroupsAccumulator, diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 6e819ef5bf46..04a725567ff7 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -33,6 +33,7 @@ use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion::execution::FunctionRegistry; use datafusion::functions_aggregate::covariance::{covar_pop, covar_samp}; use datafusion::functions_aggregate::expr_fn::first_value; +use datafusion::functions_aggregate::expr_fn::sum; use datafusion::prelude::*; use datafusion::test_util::{TestTableFactory, TestTableProvider}; use datafusion_common::config::{FormatOptions, TableOptions}; @@ -624,6 +625,7 @@ async fn roundtrip_expr_api() -> Result<()> { first_value(vec![lit(1)], false, None, None, None), covar_samp(lit(1.5), lit(2.2)), covar_pop(lit(1.5), lit(2.2)), + sum(lit(1)), ]; // ensure expressions created with the expr api can be round tripped diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 1f8492b9ba47..aa54b9605bc2 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -297,14 +297,11 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { &self, name: &str, ) -> Result { - expr::find_df_window_func(name) - // next check user defined aggregates - .or_else(|| { - self.context_provider - .get_aggregate_meta(name) - .map(WindowFunctionDefinition::AggregateUDF) - }) - // next check user defined window functions + // check udaf first + self.context_provider + .get_aggregate_meta(name) + .map(WindowFunctionDefinition::AggregateUDF) + .or_else(|| expr::find_df_window_func(name)) .or_else(|| { self.context_provider .get_window_meta(name) From 79019fefbe2b8a4516a48fafa0460678698e8058 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 25 May 2024 13:52:55 +0800 Subject: [PATCH 19/41] fmt Signed-off-by: jayzhan211 --- datafusion/core/tests/dataframe/mod.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 3a6dd3bb264a..befd98d04302 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -52,9 +52,8 @@ use datafusion_expr::expr::{GroupingSet, Sort}; use datafusion_expr::var_provider::{VarProvider, VarType}; use datafusion_expr::{ array_agg, avg, cast, col, count, exists, expr, in_subquery, lit, max, out_ref_col, - placeholder, scalar_subquery, when, wildcard, AggregateFunction, Expr, - ExprSchemable, WindowFrame, WindowFrameBound, WindowFrameUnits, - WindowFunctionDefinition, + placeholder, scalar_subquery, when, wildcard, AggregateFunction, Expr, ExprSchemable, + WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; use datafusion_functions_aggregate::expr_fn::sum; From 20e9f794837bdfbd0beea1baa037b0769466d98f Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 25 May 2024 15:27:26 +0800 Subject: [PATCH 20/41] fix ci Signed-off-by: jayzhan211 --- datafusion/core/tests/fuzz_cases/window_fuzz.rs | 12 +++++++++++- datafusion/functions-aggregate/src/first_last.rs | 2 +- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index fe0c408dc114..b85f6376c3f2 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -33,10 +33,12 @@ use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_common::{Result, ScalarValue}; use datafusion_common_runtime::SpawnedTask; use datafusion_expr::type_coercion::aggregates::coerce_types; +use datafusion_expr::type_coercion::functions::data_types_with_aggregate_udf; use datafusion_expr::{ AggregateFunction, BuiltInWindowFunction, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; +use datafusion_functions_aggregate::sum::sum_udaf; use datafusion_physical_expr::expressions::{cast, col, lit}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use test_utils::add_empty_batches; @@ -341,7 +343,7 @@ fn get_random_function( window_fn_map.insert( "sum", ( - WindowFunctionDefinition::AggregateFunction(AggregateFunction::Sum), + WindowFunctionDefinition::AggregateUDF(sum_udaf()), vec![arg.clone()], ), ); @@ -468,6 +470,14 @@ fn get_random_function( let coerced = coerce_types(f, &[dt], &sig).unwrap(); args[0] = cast(a, schema, coerced[0].clone()).unwrap(); } + } else if let WindowFunctionDefinition::AggregateUDF(udf) = window_fn { + if !args.is_empty() { + // Do type coercion first argument + let a = args[0].clone(); + let dt = a.data_type(schema.as_ref()).unwrap(); + let coerced = data_types_with_aggregate_udf(&[dt], udf).unwrap(); + args[0] = cast(a, schema, coerced[0].clone()).unwrap(); + } } (window_fn.clone(), args, fn_name.to_string()) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index b2abf8e2cdf1..5d3d48344014 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -1030,4 +1030,4 @@ mod tests { Ok(()) } -} \ No newline at end of file +} From 4f2f0ac5001f48984561f7091f1c3000ed4ebcba Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 25 May 2024 16:00:49 +0800 Subject: [PATCH 21/41] fix ci Signed-off-by: jayzhan211 --- datafusion/core/tests/user_defined/user_defined_aggregates.rs | 2 +- .../core/tests/user_defined/user_defined_scalar_functions.rs | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/core/tests/user_defined/user_defined_aggregates.rs b/datafusion/core/tests/user_defined/user_defined_aggregates.rs index 5d2149dc4884..071db5adf06a 100644 --- a/datafusion/core/tests/user_defined/user_defined_aggregates.rs +++ b/datafusion/core/tests/user_defined/user_defined_aggregates.rs @@ -142,7 +142,7 @@ async fn test_udaf_as_window_with_frame_without_retract_batch() { let sql = "SELECT time_sum(time) OVER(ORDER BY time ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) as time_sum from t"; // Note if this query ever does start working let err = execute(&ctx, sql).await.unwrap_err(); - assert_contains!(err.to_string(), "This feature is not implemented: Aggregate can not be used as a sliding accumulator because `retract_batch` is not implemented: AggregateUDF { inner: AggregateUDF { name: \"time_sum\", signature: Signature { type_signature: Exact([Timestamp(Nanosecond, None)]), volatility: Immutable }, fun: \"\" } }(t.time) ORDER BY [t.time ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING"); + assert_contains!(err.to_string(), "This feature is not implemented: Aggregate can not be used as a sliding accumulator because `retract_batch` is not implemented: time_sum(t.time) ORDER BY [t.time ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING"); } /// Basic query for with a udaf returning a structure diff --git a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs index df41cab7bf02..2d98b7f80fc5 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -378,8 +378,8 @@ async fn udaf_as_window_func() -> Result<()> { context.register_udaf(my_acc); let sql = "SELECT a, MY_ACC(b) OVER(PARTITION BY a) FROM my_table"; - let expected = r#"Projection: my_table.a, AggregateUDF { inner: AggregateUDF { name: "my_acc", signature: Signature { type_signature: Exact([Int32]), volatility: Immutable }, fun: "" } }(my_table.b) PARTITION BY [my_table.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING - WindowAggr: windowExpr=[[AggregateUDF { inner: AggregateUDF { name: "my_acc", signature: Signature { type_signature: Exact([Int32]), volatility: Immutable }, fun: "" } }(my_table.b) PARTITION BY [my_table.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] + let expected = r#"Projection: my_table.a, my_acc(my_table.b) PARTITION BY [my_table.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING + WindowAggr: windowExpr=[[my_acc(my_table.b) PARTITION BY [my_table.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] TableScan: my_table"#; let dataframe = context.sql(sql).await.unwrap(); From ca4b5284fe4f37efa14026a84ac96f36c84b0800 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 25 May 2024 16:21:22 +0800 Subject: [PATCH 22/41] fix ci Signed-off-by: jayzhan211 --- datafusion/optimizer/src/analyzer/type_coercion.rs | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index a397628b9f67..4e6e8ce23944 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -992,13 +992,12 @@ mod test { None, None, )); - let plan = LogicalPlan::Projection(Projection::try_new(vec![udaf], empty)?); - let err = assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, "") - .err() - .unwrap(); + + let err = Projection::try_new(vec![udaf], empty).err().unwrap(); + assert_eq!( - "type_coercion\ncaused by\nError during planning: [data_types_with_aggregate_udf] Coercion from [Utf8] to the signature Uniform(1, [Float64]) failed.", - err.strip_backtrace() + err.strip_backtrace(), + "Error during planning: Error during planning: [data_types_with_aggregate_udf] Coercion from [Utf8] to the signature Uniform(1, [Float64]) failed. and No function matches the given name and argument types 'MY_AVG(Utf8)'. You might need to add explicit type casts.\n\tCandidate functions:\n\tMY_AVG(Float64)" ); Ok(()) } From e6b021e46cf78d986cce34df1780dffcb41d7960 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 25 May 2024 16:39:46 +0800 Subject: [PATCH 23/41] fix err msg AGAIN Signed-off-by: jayzhan211 --- datafusion/optimizer/src/analyzer/type_coercion.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 4e6e8ce23944..3b22780d180e 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -995,9 +995,8 @@ mod test { let err = Projection::try_new(vec![udaf], empty).err().unwrap(); - assert_eq!( - err.strip_backtrace(), - "Error during planning: Error during planning: [data_types_with_aggregate_udf] Coercion from [Utf8] to the signature Uniform(1, [Float64]) failed. and No function matches the given name and argument types 'MY_AVG(Utf8)'. You might need to add explicit type casts.\n\tCandidate functions:\n\tMY_AVG(Float64)" + assert!( + err.strip_backtrace().starts_with("Error during planning: Error during planning: [data_types_with_aggregate_udf] Coercion from [Utf8] to the signature Uniform(1, [Float64]) failed") ); Ok(()) } From 81dd68f0839ec44d82316fc8f1628f4a79b7c165 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 25 May 2024 17:00:42 +0800 Subject: [PATCH 24/41] rm sum in builtin test which covered in sql Signed-off-by: jayzhan211 --- .../physical-expr/src/aggregate/build_in.rs | 80 +++---------------- 1 file changed, 9 insertions(+), 71 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index 66cff984255f..8010fdd03064 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -402,7 +402,7 @@ mod tests { use crate::expressions::{ try_cast, ApproxDistinct, ApproxMedian, ApproxPercentileCont, ArrayAgg, Avg, BitAnd, BitOr, BitXor, BoolAnd, BoolOr, Count, DistinctArrayAgg, DistinctCount, - Max, Min, Stddev, Sum, Variance, + Max, Min, Stddev, Variance, }; use super::*; @@ -711,7 +711,7 @@ mod tests { #[test] fn test_sum_avg_expr() -> Result<()> { - let funcs = vec![AggregateFunction::Sum, AggregateFunction::Avg]; + let funcs = vec![AggregateFunction::Avg]; let data_types = vec![ DataType::UInt32, DataType::UInt64, @@ -734,37 +734,13 @@ mod tests { &input_schema, "c1", )?; - match fun { - AggregateFunction::Sum => { - assert!(result_agg_phy_exprs.as_any().is::()); - assert_eq!("c1", result_agg_phy_exprs.name()); - let expect_type = match data_type { - DataType::UInt8 - | DataType::UInt16 - | DataType::UInt32 - | DataType::UInt64 => DataType::UInt64, - DataType::Int8 - | DataType::Int16 - | DataType::Int32 - | DataType::Int64 => DataType::Int64, - DataType::Float32 | DataType::Float64 => DataType::Float64, - _ => data_type.clone(), - }; - - assert_eq!( - Field::new("c1", expect_type.clone(), true), - result_agg_phy_exprs.field().unwrap() - ); - } - AggregateFunction::Avg => { - assert!(result_agg_phy_exprs.as_any().is::()); - assert_eq!("c1", result_agg_phy_exprs.name()); - assert_eq!( - Field::new("c1", DataType::Float64, true), - result_agg_phy_exprs.field().unwrap() - ); - } - _ => {} + if fun == AggregateFunction::Avg { + assert!(result_agg_phy_exprs.as_any().is::()); + assert_eq!("c1", result_agg_phy_exprs.name()); + assert_eq!( + Field::new("c1", DataType::Float64, true), + result_agg_phy_exprs.field().unwrap() + ); }; } } @@ -998,44 +974,6 @@ mod tests { Ok(()) } - #[test] - fn test_sum_return_type() -> Result<()> { - let observed = AggregateFunction::Sum.return_type(&[DataType::Int32])?; - assert_eq!(DataType::Int64, observed); - - let observed = AggregateFunction::Sum.return_type(&[DataType::UInt8])?; - assert_eq!(DataType::UInt64, observed); - - let observed = AggregateFunction::Sum.return_type(&[DataType::Float32])?; - assert_eq!(DataType::Float64, observed); - - let observed = AggregateFunction::Sum.return_type(&[DataType::Float64])?; - assert_eq!(DataType::Float64, observed); - - let observed = - AggregateFunction::Sum.return_type(&[DataType::Decimal128(10, 5)])?; - assert_eq!(DataType::Decimal128(20, 5), observed); - - let observed = - AggregateFunction::Sum.return_type(&[DataType::Decimal128(35, 5)])?; - assert_eq!(DataType::Decimal128(38, 5), observed); - - Ok(()) - } - - #[test] - fn test_sum_no_utf8() { - let observed = AggregateFunction::Sum.return_type(&[DataType::Utf8]); - assert!(observed.is_err()); - } - - #[test] - fn test_sum_upcasts() -> Result<()> { - let observed = AggregateFunction::Sum.return_type(&[DataType::UInt32])?; - assert_eq!(DataType::UInt64, observed); - Ok(()) - } - #[test] fn test_count_return_type() -> Result<()> { let observed = AggregateFunction::Count.return_type(&[DataType::Utf8])?; From ffb0a98ca44e7d55afa06c6b72222eb85605a731 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 25 May 2024 20:28:57 +0800 Subject: [PATCH 25/41] proto for window with udaf Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/windows/mod.rs | 17 +++-- datafusion/proto/proto/datafusion.proto | 2 +- datafusion/proto/src/generated/pbjson.rs | 13 ++++ datafusion/proto/src/generated/prost.rs | 5 +- .../proto/src/physical_plan/from_proto.rs | 62 +++++++++---------- .../proto/src/physical_plan/to_proto.rs | 32 ++++++---- .../tests/cases/roundtrip_physical_plan.rs | 21 +++++-- 7 files changed, 93 insertions(+), 59 deletions(-) diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 58d677d293dd..acdac59b93ee 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -70,12 +70,17 @@ pub fn schema_add_window_field( .iter() .map(|f| f.as_ref().clone()) .collect_vec(); - window_fields.extend_from_slice(&[Field::new( - fn_name, - window_expr_return_type, - false, - )]); - Ok(Arc::new(Schema::new(window_fields))) + // Skip extending schema for UDAF + if let WindowFunctionDefinition::AggregateUDF(_) = window_fn { + Ok(Arc::new(Schema::new(window_fields))) + } else { + window_fields.extend_from_slice(&[Field::new( + fn_name, + window_expr_return_type, + false, + )]); + Ok(Arc::new(Schema::new(window_fields))) + } } /// Create a physical expression for window function diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 73e751c616ac..ed1c71bda6d0 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1346,7 +1346,7 @@ message PhysicalWindowExprNode { oneof window_function { AggregateFunction aggr_function = 1; BuiltInWindowFunction built_in_function = 2; - // udaf = 3 + string user_defined_aggr_function = 3; } repeated PhysicalExprNode args = 4; repeated PhysicalExprNode partition_by = 5; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 77ba0808fb77..fa35d96c2a44 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -20683,6 +20683,9 @@ impl serde::Serialize for PhysicalWindowExprNode { .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; struct_ser.serialize_field("builtInFunction", &v)?; } + physical_window_expr_node::WindowFunction::UserDefinedAggrFunction(v) => { + struct_ser.serialize_field("userDefinedAggrFunction", v)?; + } } } struct_ser.end() @@ -20707,6 +20710,8 @@ impl<'de> serde::Deserialize<'de> for PhysicalWindowExprNode { "aggrFunction", "built_in_function", "builtInFunction", + "user_defined_aggr_function", + "userDefinedAggrFunction", ]; #[allow(clippy::enum_variant_names)] @@ -20718,6 +20723,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalWindowExprNode { Name, AggrFunction, BuiltInFunction, + UserDefinedAggrFunction, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -20746,6 +20752,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalWindowExprNode { "name" => Ok(GeneratedField::Name), "aggrFunction" | "aggr_function" => Ok(GeneratedField::AggrFunction), "builtInFunction" | "built_in_function" => Ok(GeneratedField::BuiltInFunction), + "userDefinedAggrFunction" | "user_defined_aggr_function" => Ok(GeneratedField::UserDefinedAggrFunction), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -20815,6 +20822,12 @@ impl<'de> serde::Deserialize<'de> for PhysicalWindowExprNode { } window_function__ = map_.next_value::<::std::option::Option>()?.map(|x| physical_window_expr_node::WindowFunction::BuiltInFunction(x as i32)); } + GeneratedField::UserDefinedAggrFunction => { + if window_function__.is_some() { + return Err(serde::de::Error::duplicate_field("userDefinedAggrFunction")); + } + window_function__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_window_expr_node::WindowFunction::UserDefinedAggrFunction); + } } } Ok(PhysicalWindowExprNode { diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index a175987f1994..9082a4583996 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2161,7 +2161,7 @@ pub struct PhysicalWindowExprNode { pub window_frame: ::core::option::Option, #[prost(string, tag = "8")] pub name: ::prost::alloc::string::String, - #[prost(oneof = "physical_window_expr_node::WindowFunction", tags = "1, 2")] + #[prost(oneof = "physical_window_expr_node::WindowFunction", tags = "1, 2, 3")] pub window_function: ::core::option::Option< physical_window_expr_node::WindowFunction, >, @@ -2173,9 +2173,10 @@ pub mod physical_window_expr_node { pub enum WindowFunction { #[prost(enumeration = "super::AggregateFunction", tag = "1")] AggrFunction(i32), - /// udaf = 3 #[prost(enumeration = "super::BuiltInWindowFunction", tag = "2")] BuiltInFunction(i32), + #[prost(string, tag = "3")] + UserDefinedAggrFunction(::prost::alloc::string::String), } } #[allow(clippy::derive_partial_eq_without_eq)] diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index b7bc60a0486c..cc2026376c1c 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -155,8 +155,37 @@ pub fn parse_physical_window_expr( ) })?; - let fun: WindowFunctionDefinition = convert_required!(proto.window_function)?; + let fun = if let Some(window_func) = proto.window_function.as_ref() { + match window_func { + protobuf::physical_window_expr_node::WindowFunction::AggrFunction(n) => { + let f = protobuf::AggregateFunction::try_from(*n).map_err(|_| { + proto_error(format!( + "Received an unknown window aggregate function: {n}" + )) + })?; + + WindowFunctionDefinition::AggregateFunction(f.into()) + } + protobuf::physical_window_expr_node::WindowFunction::BuiltInFunction(n) => { + let f = protobuf::BuiltInWindowFunction::try_from(*n).map_err(|_| { + proto_error(format!( + "Received an unknown window builtin function: {n}" + )) + })?; + + WindowFunctionDefinition::BuiltInWindowFunction(f.into()) + } + protobuf::physical_window_expr_node::WindowFunction::UserDefinedAggrFunction(udaf_name) => { + let agg_udf = registry.udaf(udaf_name)?; + WindowFunctionDefinition::AggregateUDF(agg_udf) + } + } + } else { + return Err(proto_error("Missing required field in protobuf")); + }; + let name = proto.name.clone(); + // TODO: Remove extended_schema if functions are all UDAF let extended_schema = schema_add_window_field(&window_node_expr, input_schema, &fun, &name)?; create_window_expr( @@ -393,37 +422,6 @@ fn parse_required_physical_expr( }) } -impl TryFrom<&protobuf::physical_window_expr_node::WindowFunction> - for WindowFunctionDefinition -{ - type Error = DataFusionError; - - fn try_from( - expr: &protobuf::physical_window_expr_node::WindowFunction, - ) -> Result { - match expr { - protobuf::physical_window_expr_node::WindowFunction::AggrFunction(n) => { - let f = protobuf::AggregateFunction::try_from(*n).map_err(|_| { - proto_error(format!( - "Received an unknown window aggregate function: {n}" - )) - })?; - - Ok(WindowFunctionDefinition::AggregateFunction(f.into())) - } - protobuf::physical_window_expr_node::WindowFunction::BuiltInFunction(n) => { - let f = protobuf::BuiltInWindowFunction::try_from(*n).map_err(|_| { - proto_error(format!( - "Received an unknown window builtin function: {n}" - )) - })?; - - Ok(WindowFunctionDefinition::BuiltInWindowFunction(f.into())) - } - } - } -} - pub fn parse_protobuf_hash_partitioning( partitioning: Option<&protobuf::PhysicalHashRepartition>, registry: &dyn FunctionRegistry, diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index c6b94a934f23..879dbe3e5ff5 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -200,21 +200,29 @@ pub fn serialize_physical_window_expr( } else if let Some(sliding_aggr_window_expr) = expr.downcast_ref::() { - let AggrFn { inner, distinct } = - aggr_expr_to_aggr_fn(sliding_aggr_window_expr.get_aggregate_expr().as_ref())?; + let aggr_expr = sliding_aggr_window_expr.get_aggregate_expr(); + if let Some(a) = aggr_expr.as_any().downcast_ref::() { + physical_window_expr_node::WindowFunction::UserDefinedAggrFunction( + a.fun().name().to_string(), + ) + } else { + let AggrFn { inner, distinct } = aggr_expr_to_aggr_fn( + sliding_aggr_window_expr.get_aggregate_expr().as_ref(), + )?; + + if distinct { + // TODO + return not_impl_err!( + "Distinct aggregate functions not supported in window expressions" + ); + } - if distinct { - // TODO - return not_impl_err!( - "Distinct aggregate functions not supported in window expressions" - ); - } + if window_frame.start_bound.is_unbounded() { + return Err(DataFusionError::Internal(format!("Invalid SlidingAggregateWindowExpr = {window_expr:?} with WindowFrame = {window_frame:?}"))); + } - if window_frame.start_bound.is_unbounded() { - return Err(DataFusionError::Internal(format!("Invalid SlidingAggregateWindowExpr = {window_expr:?} with WindowFrame = {window_frame:?}"))); + physical_window_expr_node::WindowFunction::AggrFunction(inner as i32) } - - physical_window_expr_node::WindowFunction::AggrFunction(inner as i32) } else { return not_impl_err!("WindowExpr not supported: {window_expr:?}"); }; diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 79abecf556da..dfe7cf04225d 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -21,6 +21,7 @@ use std::sync::Arc; use std::vec; use arrow::csv::WriterBuilder; +use datafusion::functions_aggregate::sum::sum_udaf; use prost::Message; use datafusion::arrow::array::ArrayRef; @@ -47,7 +48,7 @@ use datafusion::physical_plan::analyze::AnalyzeExec; use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::expressions::{ binary, cast, col, in_list, like, lit, Avg, BinaryExpr, Column, DistinctCount, - NotExpr, NthValue, PhysicalSortExpr, StringAgg, Sum, + NotExpr, NthValue, PhysicalSortExpr, StringAgg, }; use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::insert::DataSinkExec; @@ -296,12 +297,20 @@ fn roundtrip_window() -> Result<()> { WindowFrameBound::Preceding(ScalarValue::Int64(None)), ); + let args = vec![cast(col("a", &schema)?, &schema, DataType::Float64)?]; + let sum_expr = udaf::create_aggregate_expr( + &sum_udaf(), + &args, + &[], + &[], + &schema, + "SUM(a) RANGE BETWEEN CURRENT ROW AND UNBOUNDED PRECEEDING", + false, + false, + )?; + let sliding_aggr_window_expr = Arc::new(SlidingAggregateWindowExpr::new( - Arc::new(Sum::new( - cast(col("a", &schema)?, &schema, DataType::Float64)?, - "SUM(a) RANGE BETWEEN CURRENT ROW AND UNBOUNDED PRECEEDING", - DataType::Float64, - )), + sum_expr, &[], &[], Arc::new(window_frame), From dafd1aa560b1037a5211820f0d57be94d7c4dcd3 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sun, 26 May 2024 07:59:40 +0800 Subject: [PATCH 26/41] fix slt Signed-off-by: jayzhan211 --- .../sqllogictest/test_files/aggregate.slt | 28 +++---------------- 1 file changed, 4 insertions(+), 24 deletions(-) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 9adebea1b4fa..065de6892d62 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -3403,21 +3403,11 @@ NULL NULL Row 2 Y # aggregate_timestamps_sum -statement error +query error SELECT sum(date32), sum(date64) FROM t; ----- -DataFusion error: Error during planning: Execution error: User-defined coercion failed with Execution("Sum not supported for Date32") and No function matches the given name and argument types 'SUM(Date32)'. You might need to add explicit type casts. - Candidate functions: - SUM(UserDefined) - -statement error +query error SELECT tag, sum(date32), sum(date64) FROM t GROUP BY tag ORDER BY tag; ----- -DataFusion error: Error during planning: Execution error: User-defined coercion failed with Execution("Sum not supported for Date32") and No function matches the given name and argument types 'SUM(Date32)'. You might need to add explicit type casts. - Candidate functions: - SUM(UserDefined) - # aggregate_timestamps_count query II @@ -3510,21 +3500,11 @@ select * from t; 21:06:28.247821084 21:06:28.247821 21:06:28.247 21:06:28 Row 3 B # aggregate_times_sum -statement error +query error SELECT sum(nanos), sum(micros), sum(millis), sum(secs) FROM t ----- -DataFusion error: Error during planning: Execution error: User-defined coercion failed with Execution("Sum not supported for Time64(Nanosecond)") and No function matches the given name and argument types 'SUM(Time64(Nanosecond))'. You might need to add explicit type casts. - Candidate functions: - SUM(UserDefined) - -statement error +query error SELECT tag, sum(nanos), sum(micros), sum(millis), sum(secs) FROM t GROUP BY tag ORDER BY tag ----- -DataFusion error: Error during planning: Execution error: User-defined coercion failed with Execution("Sum not supported for Time64(Nanosecond)") and No function matches the given name and argument types 'SUM(Time64(Nanosecond))'. You might need to add explicit type casts. - Candidate functions: - SUM(UserDefined) - # aggregate_times_count query IIII From f6d37bf651fdb3b776cea11eb9846909cd894b1f Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sun, 26 May 2024 21:17:43 +0800 Subject: [PATCH 27/41] fmt Signed-off-by: jayzhan211 --- datafusion/core/src/dataframe/mod.rs | 2 +- datafusion/expr/src/function.rs | 3 --- datafusion/functions-aggregate/src/lib.rs | 4 ++-- 3 files changed, 3 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 2fc717b8f415..2da1a14b0d8d 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -54,8 +54,8 @@ use datafusion_expr::{ TableProviderFilterPushDown, UNNAMED_TABLE, }; use datafusion_expr::{case, is_null}; -use datafusion_functions_aggregate::expr_fn::sum; use datafusion_functions_aggregate::expr_fn::median; +use datafusion_functions_aggregate::expr_fn::sum; use async_trait::async_trait; diff --git a/datafusion/expr/src/function.rs b/datafusion/expr/src/function.rs index 050d957ebab1..8e3ad23498d5 100644 --- a/datafusion/expr/src/function.rs +++ b/datafusion/expr/src/function.rs @@ -85,9 +85,6 @@ pub struct AccumulatorArgs<'a> { /// The number of arguments the aggregate function takes. pub args_num: usize, - - /// The name of the expression - pub name: &'a str, } /// [`StateFieldsArgs`] contains information about the fields that an diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index d446e519183a..0ed9d01573cc 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -57,8 +57,8 @@ pub mod macros; pub mod covariance; pub mod first_last; -pub mod sum; pub mod median; +pub mod sum; use datafusion_common::Result; use datafusion_execution::FunctionRegistry; @@ -70,8 +70,8 @@ use std::sync::Arc; pub mod expr_fn { pub use super::covariance::covar_samp; pub use super::first_last::first_value; - pub use super::sum::sum; pub use super::median::median; + pub use super::sum::sum; } /// Registers all enabled packages with a [`FunctionRegistry`] From 921dc00e60e9bf5f9a048bcaea855c0841af9efc Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sun, 26 May 2024 21:40:57 +0800 Subject: [PATCH 28/41] fix err msg Signed-off-by: jayzhan211 --- datafusion/optimizer/src/analyzer/type_coercion.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 3b22780d180e..31dc9028b915 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -994,9 +994,8 @@ mod test { )); let err = Projection::try_new(vec![udaf], empty).err().unwrap(); - assert!( - err.strip_backtrace().starts_with("Error during planning: Error during planning: [data_types_with_aggregate_udf] Coercion from [Utf8] to the signature Uniform(1, [Float64]) failed") + err.strip_backtrace().starts_with("Error during planning: Error during planning: Coercion from [Utf8] to the signature Uniform(1, [Float64]) failed") ); Ok(()) } From f684f5d74b3a989dd437f471403790e09ee62be8 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 27 May 2024 20:25:13 +0800 Subject: [PATCH 29/41] fix exprfn Signed-off-by: jayzhan211 --- datafusion/core/src/prelude.rs | 1 - datafusion/expr/src/udaf.rs | 2 +- datafusion/functions-aggregate/src/lib.rs | 1 + .../physical-expr-common/src/aggregate/mod.rs | 17 +---------------- .../proto/tests/cases/roundtrip_logical_plan.rs | 2 ++ 5 files changed, 5 insertions(+), 18 deletions(-) diff --git a/datafusion/core/src/prelude.rs b/datafusion/core/src/prelude.rs index 0d8d06f49bc3..d82a5a2cc1a1 100644 --- a/datafusion/core/src/prelude.rs +++ b/datafusion/core/src/prelude.rs @@ -39,7 +39,6 @@ pub use datafusion_expr::{ Expr, }; pub use datafusion_functions::expr_fn::*; -pub use datafusion_functions_aggregate::expr_fn::*; #[cfg(feature = "array_expressions")] pub use datafusion_functions_array::expr_fn::*; diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index fbe4c172467b..ff1e45037bef 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -420,7 +420,7 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { ) -> Result> { self.accumulator(args) } - + /// Sets the indicator whether ordering requirements of the AggregateUDFImpl is /// satisfied by its input. If this is not the case, UDFs with order /// sensitivity `AggregateOrderSensitivity::Beneficial` can still produce diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index 834115a95572..0a2de362e36a 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -68,6 +68,7 @@ use std::sync::Arc; /// Fluent-style API for creating `Expr`s pub mod expr_fn { + pub use super::covariance::covar_pop; pub use super::covariance::covar_samp; pub use super::first_last::first_value; pub use super::median::median; diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index e67b22be81e0..2db9a6762f35 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -23,7 +23,6 @@ use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::{not_impl_err, Result}; use datafusion_expr::function::StateFieldsArgs; use datafusion_expr::type_coercion::aggregates::check_arg_count; -use datafusion_expr::ReversedUDAF; use datafusion_expr::{ function::AccumulatorArgs, Accumulator, AggregateUDF, Expr, GroupsAccumulator, }; @@ -35,14 +34,8 @@ use crate::physical_expr::PhysicalExpr; use crate::sort_expr::{LexOrdering, PhysicalSortExpr}; use crate::utils::reverse_order_bys; -use arrow::datatypes::{DataType, Field, Schema}; -use datafusion_common::{exec_err, not_impl_err, Result}; -use datafusion_expr::function::StateFieldsArgs; -use datafusion_expr::type_coercion::aggregates::check_arg_count; +use datafusion_common::exec_err; use datafusion_expr::utils::AggregateOrderSensitivity; -use datafusion_expr::{ - function::AccumulatorArgs, Accumulator, AggregateUDF, Expr, GroupsAccumulator, -}; /// Creates a physical expression of the UDAF, that includes all necessary type coercion. /// This function errors when `args`' can't be coerced to a valid argument type of the UDAF. @@ -374,14 +367,6 @@ impl AggregateExpr for AggregateFunctionExpr { None } - fn reverse_expr(&self) -> Option> { - match self.fun.reverse_expr() { - ReversedUDAF::NotSupported => None, - ReversedUDAF::Identical => Some(Arc::new(self.clone())), - ReversedUDAF::Reversed(fun) => todo!("Reverse UDAF: {:?}", fun), - } - } - fn order_sensitivity(&self) -> AggregateOrderSensitivity { if !self.ordering_req.is_empty() { // If there is requirement, use the sensitivity of the implementation diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index ae56a373c820..e08c71f6515b 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -31,6 +31,8 @@ use datafusion::datasource::TableProvider; use datafusion::execution::context::SessionState; use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion::execution::FunctionRegistry; +use datafusion::functions_aggregate::expr_fn::{covar_pop, covar_samp, first_value}; +use datafusion::functions_aggregate::median::median; use datafusion::prelude::*; use datafusion::test_util::{TestTableFactory, TestTableProvider}; use datafusion_common::config::{FormatOptions, TableOptions}; From c1e74f72c110834009a2fb455a85b005e50fa410 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 27 May 2024 21:09:16 +0800 Subject: [PATCH 30/41] fix ciy Signed-off-by: jayzhan211 --- datafusion/core/src/dataframe/mod.rs | 7 +++---- datafusion/functions-aggregate/src/first_last.rs | 5 +++-- datafusion/physical-plan/src/windows/mod.rs | 1 + 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 2da1a14b0d8d..86d258338076 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1589,9 +1589,10 @@ mod tests { use datafusion_common::{Constraint, Constraints}; use datafusion_common_runtime::SpawnedTask; use datafusion_expr::{ - array_agg, cast, count_distinct, create_udf, expr, lit, BuiltInWindowFunction, + array_agg, cast, count_distinct, create_udf, expr, lit, ScalarFunctionImplementation, Volatility, WindowFrame, WindowFunctionDefinition, }; + use datafusion_functions_aggregate::first_last::first_value_udaf; use datafusion_physical_expr::expressions::Column; use datafusion_physical_plan::{get_plan_string, ExecutionPlanProperties}; @@ -1755,9 +1756,7 @@ mod tests { // build plan using Table API let t = test_table().await?; let first_row = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunctionDefinition::BuiltInWindowFunction( - BuiltInWindowFunction::FirstValue, - ), + WindowFunctionDefinition::AggregateUDF(first_value_udaf()), vec![col("aggregate_test_100.c1")], vec![col("aggregate_test_100.c2")], vec![], diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index fd4e21971028..3bc987c90d96 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -72,12 +72,13 @@ impl Default for FirstValue { impl FirstValue { pub fn new() -> Self { Self { - aliases: vec![String::from("FIRST_VALUE"), String::from("first_value")], + aliases: vec![String::from("first_value")], signature: Signature::one_of( vec![ // TODO: we can introduce more strict signature that only numeric of array types are allowed TypeSignature::ArraySignature(ArrayFunctionSignature::Array), - TypeSignature::Uniform(1, NUMERICS.to_vec()), + TypeSignature::Numeric(1), + TypeSignature::Uniform(1, vec![DataType::Utf8]), ], Volatility::Immutable, ), diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index acdac59b93ee..e775a7497d15 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -124,6 +124,7 @@ pub fn create_window_expr( WindowFunctionDefinition::AggregateUDF(fun) => { // TODO: Ordering not supported for Window UDFs yet let sort_exprs = &[]; + let order_by = &[]; let aggregate = udaf::create_aggregate_expr( fun.as_ref(), From 6d3ef5875c62e2faf62f6d17874b2676775c20c4 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 27 May 2024 23:22:49 +0800 Subject: [PATCH 31/41] fix ci Signed-off-by: jayzhan211 --- datafusion/core/src/dataframe/mod.rs | 7 +- datafusion/expr/src/udaf.rs | 10 +-- .../functions-aggregate/src/first_last.rs | 12 +++- .../physical-expr-common/src/aggregate/mod.rs | 70 +++++++++++-------- datafusion/physical-plan/src/windows/mod.rs | 1 - datafusion/sql/src/expr/function.rs | 29 ++++---- 6 files changed, 73 insertions(+), 56 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 86d258338076..2da1a14b0d8d 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1589,10 +1589,9 @@ mod tests { use datafusion_common::{Constraint, Constraints}; use datafusion_common_runtime::SpawnedTask; use datafusion_expr::{ - array_agg, cast, count_distinct, create_udf, expr, lit, + array_agg, cast, count_distinct, create_udf, expr, lit, BuiltInWindowFunction, ScalarFunctionImplementation, Volatility, WindowFrame, WindowFunctionDefinition, }; - use datafusion_functions_aggregate::first_last::first_value_udaf; use datafusion_physical_expr::expressions::Column; use datafusion_physical_plan::{get_plan_string, ExecutionPlanProperties}; @@ -1756,7 +1755,9 @@ mod tests { // build plan using Table API let t = test_table().await?; let first_row = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunctionDefinition::AggregateUDF(first_value_udaf()), + WindowFunctionDefinition::BuiltInWindowFunction( + BuiltInWindowFunction::FirstValue, + ), vec![col("aggregate_test_100.c1")], vec![col("aggregate_test_100.c2")], vec![], diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index ff1e45037bef..443896a5efd6 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -233,12 +233,8 @@ impl AggregateUDF { /// Reserves the `AggregateUDF` (e.g. returns the `AggregateUDF` that will /// generate same result with this `AggregateUDF` when iterated in reverse /// order, and `None` if there is no such `AggregateUDF`). - pub fn reverse_udf(&self) -> Option { - match self.inner.reverse_expr() { - ReversedUDAF::NotSupported => None, - ReversedUDAF::Identical => Some(self.clone()), - ReversedUDAF::Reversed(reverse) => Some(Self { inner: reverse }), - } + pub fn reverse_udf(&self) -> ReversedUDAF { + self.inner.reverse_expr() } /// Do the function rewrite @@ -519,7 +515,7 @@ pub enum ReversedUDAF { /// The expression does not support reverse calculation, like ArrayAgg NotSupported, /// The expression is different from the original expression - Reversed(Arc), + Reversed(Arc), } /// AggregateUDF that adds an alias to the underlying function. It is better to diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 3bc987c90d96..6acb71932034 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -162,7 +162,11 @@ impl AggregateUDFImpl for FirstValue { } fn reverse_expr(&self) -> datafusion_expr::ReversedUDAF { - datafusion_expr::ReversedUDAF::Reversed(last_value_udaf().inner()) + datafusion_expr::ReversedUDAF::Reversed(last_value_udaf()) + } + + fn has_ordering_requirements(&self) -> bool { + true } } @@ -467,7 +471,11 @@ impl AggregateUDFImpl for LastValue { } fn reverse_expr(&self) -> datafusion_expr::ReversedUDAF { - datafusion_expr::ReversedUDAF::Reversed(first_value_udaf().inner()) + datafusion_expr::ReversedUDAF::Reversed(first_value_udaf()) + } + + fn has_ordering_requirements(&self) -> bool { + true } } diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 2db9a6762f35..006501a4adce 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -23,6 +23,7 @@ use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::{not_impl_err, Result}; use datafusion_expr::function::StateFieldsArgs; use datafusion_expr::type_coercion::aggregates::check_arg_count; +use datafusion_expr::ReversedUDAF; use datafusion_expr::{ function::AccumulatorArgs, Accumulator, AggregateUDF, Expr, GroupsAccumulator, }; @@ -50,7 +51,10 @@ pub fn create_aggregate_expr( ignore_nulls: bool, is_distinct: bool, ) -> Result> { - debug_assert_eq!(sort_exprs.len(), ordering_req.len()); + // sort_exprs is used for udaf accumulator only, so it is not neccessary + // if the aggregate function does not care about accumulator + // debug_assert_eq!(sort_exprs.len(), ordering_req.len()); + let input_exprs_types = input_phy_exprs .iter() .map(|arg| arg.data_type(schema)) @@ -402,37 +406,41 @@ impl AggregateExpr for AggregateFunctionExpr { } fn reverse_expr(&self) -> Option> { - if let Some(reverse_udf) = self.fun.reverse_udf() { - let reverse_ordering_req = reverse_order_bys(&self.ordering_req); - let reverse_sort_exprs = self - .sort_exprs - .iter() - .map(|e| { - if let Expr::Sort(s) = e { - Expr::Sort(s.reverse()) - } else { - // Expects to receive `Expr::Sort`. - unreachable!() - } - }) - .collect::>(); - let mut name = self.name().to_string(); - replace_order_by_clause(&mut name); - replace_fn_name_clause(&mut name, self.fun.name(), reverse_udf.name()); - let reverse_aggr = create_aggregate_expr( - &reverse_udf, - &self.args, - &reverse_sort_exprs, - &reverse_ordering_req, - &self.schema, - name, - self.ignore_nulls, - self.is_distinct, - ) - .unwrap(); - return Some(reverse_aggr); + match self.fun.reverse_udf() { + ReversedUDAF::NotSupported => None, + ReversedUDAF::Identical => Some(Arc::new(self.clone())), + ReversedUDAF::Reversed(reverse_udf) => { + let reverse_ordering_req = reverse_order_bys(&self.ordering_req); + let reverse_sort_exprs = self + .sort_exprs + .iter() + .map(|e| { + if let Expr::Sort(s) = e { + Expr::Sort(s.reverse()) + } else { + // Expects to receive `Expr::Sort`. + unreachable!() + } + }) + .collect::>(); + let mut name = self.name().to_string(); + replace_order_by_clause(&mut name); + replace_fn_name_clause(&mut name, self.fun.name(), reverse_udf.name()); + let reverse_aggr = create_aggregate_expr( + &reverse_udf, + &self.args, + &reverse_sort_exprs, + &reverse_ordering_req, + &self.schema, + name, + self.ignore_nulls, + self.is_distinct, + ) + .unwrap(); + + Some(reverse_aggr) + } } - None } } diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index e775a7497d15..acdac59b93ee 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -124,7 +124,6 @@ pub fn create_window_expr( WindowFunctionDefinition::AggregateUDF(fun) => { // TODO: Ordering not supported for Window UDFs yet let sort_exprs = &[]; - let order_by = &[]; let aggregate = udaf::create_aggregate_expr( fun.as_ref(), diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index aa54b9605bc2..8d731470434c 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -298,18 +298,23 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { name: &str, ) -> Result { // check udaf first - self.context_provider - .get_aggregate_meta(name) - .map(WindowFunctionDefinition::AggregateUDF) - .or_else(|| expr::find_df_window_func(name)) - .or_else(|| { - self.context_provider - .get_window_meta(name) - .map(WindowFunctionDefinition::WindowUDF) - }) - .ok_or_else(|| { - plan_datafusion_err!("There is no window function named {name}") - }) + let udaf = self.context_provider.get_aggregate_meta(name); + // Skip first value and last value, since we expect window builtin first/last value not udaf version + if udaf.as_ref().is_some_and(|udaf| { + udaf.name() != "FIRST_VALUE" && udaf.name() != "LAST_VALUE" + }) { + Ok(WindowFunctionDefinition::AggregateUDF(udaf.unwrap())) + } else { + expr::find_df_window_func(name) + .or_else(|| { + self.context_provider + .get_window_meta(name) + .map(WindowFunctionDefinition::WindowUDF) + }) + .ok_or_else(|| { + plan_datafusion_err!("There is no window function named {name}") + }) + } } fn sql_fn_arg_to_logical_expr( From 02fd8a5544f87728e6d52cd1cb4c40b2becb4eee Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 30 May 2024 09:30:39 +0800 Subject: [PATCH 32/41] rename first/last to lowercase Signed-off-by: jayzhan211 --- datafusion/expr/src/built_in_window_function.rs | 4 ++-- datafusion/expr/src/expr.rs | 6 +++++- datafusion/proto-common/src/to_proto/mod.rs | 2 -- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/datafusion/expr/src/built_in_window_function.rs b/datafusion/expr/src/built_in_window_function.rs index 18a888ae8b2a..3885d70049f3 100644 --- a/datafusion/expr/src/built_in_window_function.rs +++ b/datafusion/expr/src/built_in_window_function.rs @@ -82,8 +82,8 @@ impl BuiltInWindowFunction { Ntile => "NTILE", Lag => "LAG", Lead => "LEAD", - FirstValue => "FIRST_VALUE", - LastValue => "LAST_VALUE", + FirstValue => "first_value", + LastValue => "last_value", NthValue => "NTH_VALUE", } } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 3a9991f7b866..14c64ef8f89d 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -2267,7 +2267,11 @@ mod test { let fun = find_df_window_func(name).unwrap(); let fun2 = find_df_window_func(name.to_uppercase().as_str()).unwrap(); assert_eq!(fun, fun2); - assert_eq!(fun.to_string(), name.to_uppercase()); + if fun.to_string() == "first_value" || fun.to_string() == "last_value" { + assert_eq!(fun.to_string(), name); + } else { + assert_eq!(fun.to_string(), name.to_uppercase()); + } } Ok(()) } diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index e53604fc748c..f160bc40af39 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -289,8 +289,6 @@ impl TryFrom<&ScalarValue> for protobuf::ScalarValue { type Error = Error; fn try_from(val: &ScalarValue) -> Result { - use protobuf::scalar_value::Value; - let data_type = val.data_type(); match val { ScalarValue::Boolean(val) => { From 6c7ce0467c1654d60ad23a2ce878c1321504c81c Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 30 May 2024 17:42:17 +0800 Subject: [PATCH 33/41] skip sum Signed-off-by: jayzhan211 --- datafusion/functions-aggregate/src/lib.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index 7da38e770f9c..35e1c8a33719 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -113,6 +113,11 @@ mod tests { fn test_no_duplicate_name() -> Result<()> { let mut names = HashSet::new(); for func in all_default_aggregate_functions() { + // TODO: remove this + // sum is in intermidiate migration state, skip this + if func.name() == "sum" { + continue; + } assert!( names.insert(func.name().to_string().to_lowercase()), "duplicate function name: {}", From 5490bcf9bf107a3f9cffede415257912819a7fbd Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 30 May 2024 20:11:55 +0800 Subject: [PATCH 34/41] fix firstvalue Signed-off-by: jayzhan211 --- datafusion/functions-aggregate/src/lib.rs | 2 +- datafusion/sql/src/expr/function.rs | 2 +- datafusion/sqllogictest/test_files/order.slt | 2 +- .../test_files/sort_merge_join.slt | 1 + datafusion/sqllogictest/test_files/unnest.slt | 4 +-- datafusion/sqllogictest/test_files/window.slt | 36 +++++++++---------- 6 files changed, 24 insertions(+), 23 deletions(-) diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index 35e1c8a33719..cb8ef65420c2 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -115,7 +115,7 @@ mod tests { for func in all_default_aggregate_functions() { // TODO: remove this // sum is in intermidiate migration state, skip this - if func.name() == "sum" { + if func.name().to_lowercase() == "sum" { continue; } assert!( diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 8d731470434c..81a9b4b772d0 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -301,7 +301,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let udaf = self.context_provider.get_aggregate_meta(name); // Skip first value and last value, since we expect window builtin first/last value not udaf version if udaf.as_ref().is_some_and(|udaf| { - udaf.name() != "FIRST_VALUE" && udaf.name() != "LAST_VALUE" + udaf.name() != "first_value" && udaf.name() != "last_value" }) { Ok(WindowFunctionDefinition::AggregateUDF(udaf.unwrap())) } else { diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index d7f10537d02a..2678e8cbd1ba 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -1131,4 +1131,4 @@ physical_plan 01)SortPreservingMergeExec: [c@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[CAST(inc_col@0 > desc_col@1 AS Int32) as c] 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[inc_col, desc_col], output_orderings=[[inc_col@0 ASC NULLS LAST], [desc_col@1 DESC]], has_header=true \ No newline at end of file +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[inc_col, desc_col], output_orderings=[[inc_col@0 ASC NULLS LAST], [desc_col@1 DESC]], has_header=true diff --git a/datafusion/sqllogictest/test_files/sort_merge_join.slt b/datafusion/sqllogictest/test_files/sort_merge_join.slt index 3a27d9693d00..0cdcfabce4bd 100644 --- a/datafusion/sqllogictest/test_files/sort_merge_join.slt +++ b/datafusion/sqllogictest/test_files/sort_merge_join.slt @@ -344,6 +344,7 @@ t1 as ( select 11 a, 13 b) select t1.* from t1 where exists (select 1 from t1 t2 where t2.a = t1.a and t2.b != t1.b) ) order by 1, 2; +---- query II select * from ( diff --git a/datafusion/sqllogictest/test_files/unnest.slt b/datafusion/sqllogictest/test_files/unnest.slt index bdd7e6631c16..8866cd009c32 100644 --- a/datafusion/sqllogictest/test_files/unnest.slt +++ b/datafusion/sqllogictest/test_files/unnest.slt @@ -65,7 +65,7 @@ select * from unnest(struct(1,2,3)); ---- 1 2 3 -## Multiple unnest expression in from clause +## Multiple unnest expression in from clause query IIII select * from unnest(struct(1,2,3)),unnest([4,5,6]); ---- @@ -446,7 +446,7 @@ query error DataFusion error: type_coercion\ncaused by\nThis feature is not impl select sum(unnest(generate_series(1,10))); ## TODO: support unnest as a child expr -query error DataFusion error: Internal error: unnest on struct can ony be applied at the root level of select expression +query error DataFusion error: Internal error: unnest on struct can ony be applied at the root level of select expression select arrow_typeof(unnest(column5)) from unnest_table; diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index be1517aa75c1..2d5dd439d76d 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1344,16 +1344,16 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -01)Projection: aggregate_test_100.c9, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS fv1, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS fv2, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS lag1, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS lead1, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2 +01)Projection: aggregate_test_100.c9, first_value(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS fv1, first_value(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS fv2, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS lag1, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS lead1, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2 02)--Limit: skip=0, fetch=5 -03)----WindowAggr: windowExpr=[[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, LAG(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, LEAD(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -04)------WindowAggr: windowExpr=[[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, LAG(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] +03)----WindowAggr: windowExpr=[[first_value(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, LAG(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, LEAD(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +04)------WindowAggr: windowExpr=[[first_value(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, LAG(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] 05)--------TableScan: aggregate_test_100 projection=[c9] physical_plan -01)ProjectionExec: expr=[c9@0 as c9, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as fv1, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as fv2, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as lag1, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as lag2, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as lead1, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as lead2] +01)ProjectionExec: expr=[c9@0 as c9, first_value(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as fv1, first_value(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as fv2, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as lag1, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as lag2, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as lead1, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as lead2] 02)--GlobalLimitExec: skip=0, fetch=5 -03)----BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] 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: CurrentRow, end_bound: Following(UInt64(NULL)), is_causal: false }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] 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: CurrentRow, end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -04)------BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +03)----BoundedWindowAggExec: wdw=[first_value(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "first_value(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] 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: CurrentRow, end_bound: Following(UInt64(NULL)), is_causal: false }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] 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: CurrentRow, end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +04)------BoundedWindowAggExec: wdw=[first_value(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "first_value(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true @@ -2634,16 +2634,16 @@ EXPLAIN SELECT logical_plan 01)Limit: skip=0, fetch=5 02)--Sort: annotated_data_finite.ts DESC NULLS FIRST, fetch=5 -03)----Projection: annotated_data_finite.ts, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS leadr2 -04)------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] -05)--------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] +03)----Projection: annotated_data_finite.ts, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS leadr2 +04)------WindowAggr: windowExpr=[[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] +05)--------WindowAggr: windowExpr=[[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] 06)----------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan 01)GlobalLimitExec: skip=0, fetch=5 02)--SortExec: TopK(fetch=5), expr=[ts@0 DESC], preserve_partitioning=[false] -03)----ProjectionExec: expr=[ts@0 as ts, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2] -04)------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -05)--------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }], mode=[Sorted] +03)----ProjectionExec: expr=[ts@0 as ts, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2] +04)------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +05)--------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }], mode=[Sorted] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true query IIIIIIIIIIIIIIIIIIIIIIIII @@ -2761,17 +2761,17 @@ logical_plan 01)Projection: first_value1, first_value2, last_value1, last_value2, nth_value1 02)--Limit: skip=0, fetch=5 03)----Sort: annotated_data_finite.inc_col ASC NULLS LAST, fetch=5 -04)------Projection: FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS first_value1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS first_value2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS last_value1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS last_value2, NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS nth_value1, annotated_data_finite.inc_col -05)--------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] -06)----------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] +04)------Projection: first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS first_value1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS first_value2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS last_value1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS last_value2, NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS nth_value1, annotated_data_finite.inc_col +05)--------WindowAggr: windowExpr=[[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] 07)------------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan 01)ProjectionExec: expr=[first_value1@0 as first_value1, first_value2@1 as first_value2, last_value1@2 as last_value1, last_value2@3 as last_value2, nth_value1@4 as nth_value1] 02)--GlobalLimitExec: skip=0, fetch=5 03)----SortExec: TopK(fetch=5), expr=[inc_col@5 ASC NULLS LAST], preserve_partitioning=[false] -04)------ProjectionExec: expr=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@4 as first_value1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as first_value2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as last_value1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as last_value2, NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as nth_value1, inc_col@1 as inc_col] -05)--------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 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(1)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 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(1)), is_causal: false }, NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 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(1)), is_causal: false }], mode=[Sorted] -06)----------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 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(3)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 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(3)), is_causal: false }], mode=[Sorted] +04)------ProjectionExec: expr=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@4 as first_value1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as first_value2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as last_value1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as last_value2, NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as nth_value1, inc_col@1 as inc_col] +05)--------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 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(1)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 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(1)), is_causal: false }, NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 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(1)), is_causal: false }], mode=[Sorted] +06)----------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 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(3)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 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(3)), is_causal: false }], mode=[Sorted] 07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true query IIIII From ff947bb3eb1ad9ebea665f08b6d0598a960869f3 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 30 May 2024 20:55:16 +0800 Subject: [PATCH 35/41] clippy Signed-off-by: jayzhan211 --- datafusion/core/src/datasource/listing/helpers.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index b531cf8369cf..822a66783819 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -786,7 +786,7 @@ mod tests { assert_eq!( evaluate_partition_prefix( partitions, - &[col("a").eq(lit("foo")).and((col("b").eq(lit("bar"))))], + &[col("a").eq(lit("foo")).and(col("b").eq(lit("bar")))], ), Some(Path::from("a=foo/b=bar")), ); From 2492ba78173b479380c7a0b2d956850d222674e7 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 31 May 2024 08:06:30 +0800 Subject: [PATCH 36/41] add doc Signed-off-by: jayzhan211 --- datafusion/expr/src/udaf.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 443896a5efd6..7235fc3c2524 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -410,6 +410,12 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { &[] } + /// Sliding accumulator is an alternative accumulator that can be used for + /// window functions. It has retract method to revert the previous update. + /// + /// See [retract_batch] for more details. + /// + /// [retract_batch]: crate::accumulator::Accumulator::retract_batch fn create_sliding_accumulator( &self, args: AccumulatorArgs, From 73573be44a54fdb0ab94bdcbd952993f1dd9d274 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 31 May 2024 21:17:03 +0800 Subject: [PATCH 37/41] rm has_ordering_req Signed-off-by: jayzhan211 --- datafusion/expr/src/udaf.rs | 17 +++-------------- .../functions-aggregate/src/first_last.rs | 8 -------- .../physical-expr-common/src/aggregate/mod.rs | 7 ++++--- 3 files changed, 7 insertions(+), 25 deletions(-) diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 7235fc3c2524..9aadb18eb8e4 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -243,11 +243,6 @@ impl AggregateUDF { pub fn simplify(&self) -> Option { self.inner.simplify() } - - /// See [`AggregateUDFImpl::has_ordering_requirements`] for more details. - pub fn has_ordering_requirements(&self) -> bool { - self.inner.has_ordering_requirements() - } } impl From for AggregateUDF @@ -450,10 +445,9 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// Gets the order sensitivity of the UDF. See [`AggregateOrderSensitivity`] /// for possible options. fn order_sensitivity(&self) -> AggregateOrderSensitivity { - // We have hard ordering requirements by default, meaning that order - // sensitive UDFs need their input orderings to satisfy their ordering - // requirements to generate correct results. - AggregateOrderSensitivity::HardRequirement + // We have Insensitive requirements by default, they don't care about ordering + // requirements + AggregateOrderSensitivity::Insensitive } /// Optionally apply per-UDaF simplification / rewrite rules. @@ -508,11 +502,6 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { fn coerce_types(&self, _arg_types: &[DataType]) -> Result> { not_impl_err!("Function {} does not implement coerce_types", self.name()) } - - /// Returns true if the function has ordering requirements. - fn has_ordering_requirements(&self) -> bool { - false - } } pub enum ReversedUDAF { diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 77842a73deef..fe4501c14948 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -162,10 +162,6 @@ impl AggregateUDFImpl for FirstValue { fn reverse_expr(&self) -> datafusion_expr::ReversedUDAF { datafusion_expr::ReversedUDAF::Reversed(last_value_udaf()) } - - fn has_ordering_requirements(&self) -> bool { - true - } } #[derive(Debug)] @@ -469,10 +465,6 @@ impl AggregateUDFImpl for LastValue { fn reverse_expr(&self) -> datafusion_expr::ReversedUDAF { datafusion_expr::ReversedUDAF::Reversed(first_value_udaf()) } - - fn has_ordering_requirements(&self) -> bool { - true - } } #[derive(Debug)] diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 006501a4adce..f679f94135dd 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -365,10 +365,11 @@ impl AggregateExpr for AggregateFunctionExpr { } fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { - if self.fun.has_ordering_requirements() && !self.ordering_req.is_empty() { - return Some(&self.ordering_req); + if self.order_sensitivity().is_insensitive() { + return None } - None + + Some(&self.ordering_req) } fn order_sensitivity(&self) -> AggregateOrderSensitivity { From f2b373241e68d1a94fcff66c0bee7a0b7a114c66 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 31 May 2024 22:09:06 +0800 Subject: [PATCH 38/41] default hard req Signed-off-by: jayzhan211 --- datafusion/expr/src/udaf.rs | 7 ++++--- datafusion/physical-expr-common/src/aggregate/mod.rs | 10 +++++++--- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 9aadb18eb8e4..d778203207c9 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -445,9 +445,10 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// Gets the order sensitivity of the UDF. See [`AggregateOrderSensitivity`] /// for possible options. fn order_sensitivity(&self) -> AggregateOrderSensitivity { - // We have Insensitive requirements by default, they don't care about ordering - // requirements - AggregateOrderSensitivity::Insensitive + // We have hard ordering requirements by default, meaning that order + // sensitive UDFs need their input orderings to satisfy their ordering + // requirements to generate correct results. + AggregateOrderSensitivity::HardRequirement } /// Optionally apply per-UDaF simplification / rewrite rules. diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index f679f94135dd..e0e07d6343bc 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -365,11 +365,15 @@ impl AggregateExpr for AggregateFunctionExpr { } fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { - if self.order_sensitivity().is_insensitive() { - return None + if self.ordering_req.is_empty() { + return None; } - Some(&self.ordering_req) + if !self.order_sensitivity().is_insensitive() { + return Some(&self.ordering_req); + } + + None } fn order_sensitivity(&self) -> AggregateOrderSensitivity { From 2c0c52c61f7fa47b8f1d175eb764c0ba0ee37aaf Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 31 May 2024 22:22:37 +0800 Subject: [PATCH 39/41] insensitive for sum Signed-off-by: jayzhan211 --- datafusion/functions-aggregate/src/sum.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index 8d6b17a67d10..b3127726cbbf 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -18,6 +18,7 @@ //! Defines `SUM` and `SUM DISTINCT` aggregate accumulators use ahash::RandomState; +use datafusion_expr::utils::AggregateOrderSensitivity; use std::any::Any; use std::collections::HashSet; @@ -225,6 +226,10 @@ impl AggregateUDFImpl for Sum { fn reverse_expr(&self) -> ReversedUDAF { ReversedUDAF::Identical } + + fn order_sensitivity(&self) -> AggregateOrderSensitivity { + AggregateOrderSensitivity::Insensitive + } } /// This accumulator computes SUM incrementally From 62346ddd730ec388777c8d6e4ca190fb43b32431 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 3 Jun 2024 09:31:11 +0800 Subject: [PATCH 40/41] cleanup duplicate code Signed-off-by: jayzhan211 --- datafusion/expr/src/expr_schema.rs | 4 ---- 1 file changed, 4 deletions(-) diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 35d99c0f3099..57470db2e0d9 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -176,10 +176,6 @@ impl ExprSchemable for Expr { Ok(fun.return_type(&new_types)?) } _ => { - let data_types = args - .iter() - .map(|e| e.get_type(schema)) - .collect::>>()?; fun.return_type(&data_types) } } From a41fcc54b60a0bfea5b6f184f163c168ede47b03 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 3 Jun 2024 08:50:21 +0300 Subject: [PATCH 41/41] Re-introduce check --- datafusion/expr/src/lib.rs | 2 +- .../physical-expr-common/src/aggregate/mod.rs | 4 +-- datafusion/physical-plan/src/windows/mod.rs | 26 +++++++++++++++---- 3 files changed, 23 insertions(+), 9 deletions(-) diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index d0114a472541..5cb07e2f8e0d 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -63,7 +63,7 @@ pub use built_in_window_function::BuiltInWindowFunction; pub use columnar_value::ColumnarValue; pub use expr::{ Between, BinaryExpr, Case, Cast, Expr, GetFieldAccess, GetIndexedField, GroupingSet, - Like, TryCast, WindowFunctionDefinition, + Like, Sort as SortExpr, TryCast, WindowFunctionDefinition, }; pub use expr_fn::*; pub use expr_schema::ExprSchemable; diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index e0e07d6343bc..b82ff706fb80 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -51,9 +51,7 @@ pub fn create_aggregate_expr( ignore_nulls: bool, is_distinct: bool, ) -> Result> { - // sort_exprs is used for udaf accumulator only, so it is not neccessary - // if the aggregate function does not care about accumulator - // debug_assert_eq!(sort_exprs.len(), ordering_req.len()); + debug_assert_eq!(sort_exprs.len(), ordering_req.len()); let input_exprs_types = input_phy_exprs .iter() diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index acdac59b93ee..9b392d941ef4 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -31,10 +31,11 @@ use crate::{ use arrow::datatypes::Schema; use arrow_schema::{DataType, Field, SchemaRef}; -use datafusion_common::{exec_err, DataFusionError, Result, ScalarValue}; +use datafusion_common::{exec_err, Column, DataFusionError, Result, ScalarValue}; +use datafusion_expr::Expr; use datafusion_expr::{ - BuiltInWindowFunction, PartitionEvaluator, WindowFrame, WindowFunctionDefinition, - WindowUDF, + BuiltInWindowFunction, PartitionEvaluator, SortExpr, WindowFrame, + WindowFunctionDefinition, WindowUDF, }; use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ @@ -123,12 +124,27 @@ pub fn create_window_expr( } WindowFunctionDefinition::AggregateUDF(fun) => { // TODO: Ordering not supported for Window UDFs yet - let sort_exprs = &[]; + // Convert `Vec` into `Vec` + let sort_exprs = order_by + .iter() + .map(|PhysicalSortExpr { expr, options }| { + let field_name = expr.to_string(); + let field_name = field_name.split('@').next().unwrap_or(&field_name); + Expr::Sort(SortExpr { + expr: Box::new(Expr::Column(Column::new( + None::, + field_name, + ))), + asc: !options.descending, + nulls_first: options.nulls_first, + }) + }) + .collect::>(); let aggregate = udaf::create_aggregate_expr( fun.as_ref(), args, - sort_exprs, + &sort_exprs, order_by, input_schema, name,