From f9c20873ed9f32e35738569b1759342820296d79 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 6 Aug 2024 13:41:09 +0800 Subject: [PATCH 01/26] init draft Signed-off-by: jayzhan211 --- Cargo.toml | 6 + datafusion/core/Cargo.toml | 1 + datafusion/core/src/lib.rs | 5 + .../aggregate_statistics.rs | 2 +- .../combine_partial_final_agg.rs | 2 +- .../src/physical_optimizer/limit_pushdown.rs | 4 +- datafusion/core/src/physical_planner.rs | 2 - .../core/tests/fuzz_cases/aggregate_fuzz.rs | 2 +- datafusion/expr-common/Cargo.toml | 43 +++ .../{expr => expr-common}/src/accumulator.rs | 0 .../src/columnar_value.rs | 0 .../src/groups_accumulator.rs | 2 +- .../src/interval_arithmetic.rs | 12 +- datafusion/expr-common/src/lib.rs | 25 ++ datafusion/expr-common/src/operator.rs | 282 ++++++++++++++++++ .../{expr => expr-common}/src/signature.rs | 7 +- .../src/sort_properties.rs | 0 datafusion/expr-common/src/type_coercion.rs | 19 ++ .../src/type_coercion/aggregates.rs | 2 +- .../src/type_coercion/binary.rs | 4 +- .../expr-functions-aggregate/Cargo.toml | 50 ++++ .../src/aggregate.rs} | 261 ++++------------ .../src/lib.rs} | 7 +- datafusion/expr/Cargo.toml | 5 +- datafusion/expr/src/function.rs | 8 +- datafusion/expr/src/lib.rs | 31 +- datafusion/expr/src/operator.rs | 266 +---------------- .../expr/src/type_coercion/functions.rs | 6 +- datafusion/expr/src/type_coercion/mod.rs | 7 +- datafusion/expr/src/udf.rs | 2 +- datafusion/expr/src/utils.rs | 88 ++++-- .../functions-aggregate-common/Cargo.toml | 49 +++ .../src/aggregate.rs | 187 ++++++++++++ .../src/aggregate/count_distinct.rs} | 0 .../src/aggregate/count_distinct/bytes.rs | 6 +- .../src/aggregate/count_distinct/native.rs | 4 +- .../src/aggregate/groups_accumulator.rs} | 0 .../groups_accumulator/accumulate.rs | 2 +- .../aggregate/groups_accumulator/bool_op.rs | 2 +- .../aggregate/groups_accumulator/prim_op.rs | 2 +- .../functions-aggregate-common/src/lib.rs | 25 ++ .../src}/merge_arrays.rs | 0 .../functions-aggregate-common/src/order.rs | 30 ++ .../src}/stats.rs | 0 .../src}/tdigest.rs | 0 .../src}/utils.rs | 7 +- datafusion/functions-aggregate/Cargo.toml | 2 + .../functions-aggregate/benches/count.rs | 5 +- datafusion/functions-aggregate/benches/sum.rs | 5 +- .../src/approx_percentile_cont.rs | 64 +++- .../src/approx_percentile_cont_with_weight.rs | 11 +- .../functions-aggregate/src/array_agg.rs | 21 +- datafusion/functions-aggregate/src/average.rs | 4 +- .../functions-aggregate/src/bit_and_or_xor.rs | 2 +- .../functions-aggregate/src/bool_and_or.rs | 2 +- .../functions-aggregate/src/correlation.rs | 2 +- datafusion/functions-aggregate/src/count.rs | 18 +- .../functions-aggregate/src/covariance.rs | 2 +- .../functions-aggregate/src/first_last.rs | 33 +- datafusion/functions-aggregate/src/median.rs | 2 +- datafusion/functions-aggregate/src/min_max.rs | 2 +- .../functions-aggregate/src/nth_value.rs | 70 +++-- datafusion/functions-aggregate/src/stddev.rs | 17 +- .../functions-aggregate/src/string_agg.rs | 49 ++- datafusion/functions-aggregate/src/sum.rs | 4 +- .../functions-aggregate/src/variance.rs | 2 +- datafusion/physical-expr-common/Cargo.toml | 2 +- .../physical-expr-common/src/binary_map.rs | 2 +- datafusion/physical-expr-common/src/datum.rs | 3 +- datafusion/physical-expr-common/src/lib.rs | 2 - .../physical-expr-common/src/physical_expr.rs | 39 +-- .../physical-expr-common/src/sort_expr.rs | 31 +- datafusion/physical-expr-common/src/utils.rs | 43 +-- datafusion/physical-expr/Cargo.toml | 2 + datafusion/physical-expr/benches/case_when.rs | 3 +- datafusion/physical-expr/benches/is_null.rs | 2 +- .../src/aggregate/groups_accumulator/mod.rs | 4 +- datafusion/physical-expr/src/aggregate/mod.rs | 4 +- .../physical-expr/src/aggregate/stats.rs | 2 +- .../src/equivalence/properties.rs | 5 +- .../physical-expr/src/expressions/binary.rs | 2 +- .../physical-expr/src/expressions/case.rs | 6 +- .../src/expressions/cast.rs | 6 +- .../src/expressions/column.rs | 33 +- .../src/expressions/literal.rs | 7 +- .../physical-expr/src/expressions/mod.rs | 9 +- datafusion/physical-expr/src/lib.rs | 2 +- datafusion/physical-expr/src/physical_expr.rs | 2 +- datafusion/physical-plan/Cargo.toml | 2 + .../physical-plan/src/aggregates/mod.rs | 16 +- datafusion/physical-plan/src/lib.rs | 2 +- datafusion/physical-plan/src/union.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 6 +- datafusion/proto/src/physical_plan/mod.rs | 2 +- .../tests/cases/roundtrip_physical_plan.rs | 2 +- 95 files changed, 1182 insertions(+), 843 deletions(-) create mode 100644 datafusion/expr-common/Cargo.toml rename datafusion/{expr => expr-common}/src/accumulator.rs (100%) rename datafusion/{expr => expr-common}/src/columnar_value.rs (100%) rename datafusion/{expr => expr-common}/src/groups_accumulator.rs (99%) rename datafusion/{expr => expr-common}/src/interval_arithmetic.rs (99%) create mode 100644 datafusion/expr-common/src/lib.rs create mode 100644 datafusion/expr-common/src/operator.rs rename datafusion/{expr => expr-common}/src/signature.rs (98%) rename datafusion/{expr => expr-common}/src/sort_properties.rs (100%) create mode 100644 datafusion/expr-common/src/type_coercion.rs rename datafusion/{expr => expr-common}/src/type_coercion/aggregates.rs (99%) rename datafusion/{expr => expr-common}/src/type_coercion/binary.rs (99%) create mode 100644 datafusion/expr-functions-aggregate/Cargo.toml rename datafusion/{physical-expr-common/src/aggregate/mod.rs => expr-functions-aggregate/src/aggregate.rs} (69%) rename datafusion/{physical-expr-common/src/expressions/mod.rs => expr-functions-aggregate/src/lib.rs} (86%) create mode 100644 datafusion/functions-aggregate-common/Cargo.toml create mode 100644 datafusion/functions-aggregate-common/src/aggregate.rs rename datafusion/{physical-expr-common/src/aggregate/count_distinct/mod.rs => functions-aggregate-common/src/aggregate/count_distinct.rs} (100%) rename datafusion/{physical-expr-common => functions-aggregate-common}/src/aggregate/count_distinct/bytes.rs (95%) rename datafusion/{physical-expr-common => functions-aggregate-common}/src/aggregate/count_distinct/native.rs (98%) rename datafusion/{physical-expr-common/src/aggregate/groups_accumulator/mod.rs => functions-aggregate-common/src/aggregate/groups_accumulator.rs} (100%) rename datafusion/{physical-expr-common => functions-aggregate-common}/src/aggregate/groups_accumulator/accumulate.rs (99%) rename datafusion/{physical-expr-common => functions-aggregate-common}/src/aggregate/groups_accumulator/bool_op.rs (98%) rename datafusion/{physical-expr-common => functions-aggregate-common}/src/aggregate/groups_accumulator/prim_op.rs (98%) create mode 100644 datafusion/functions-aggregate-common/src/lib.rs rename datafusion/{physical-expr-common/src/aggregate => functions-aggregate-common/src}/merge_arrays.rs (100%) create mode 100644 datafusion/functions-aggregate-common/src/order.rs rename datafusion/{physical-expr-common/src/aggregate => functions-aggregate-common/src}/stats.rs (100%) rename datafusion/{physical-expr-common/src/aggregate => functions-aggregate-common/src}/tdigest.rs (100%) rename datafusion/{physical-expr-common/src/aggregate => functions-aggregate-common/src}/utils.rs (98%) rename datafusion/{physical-expr-common => physical-expr}/src/expressions/cast.rs (99%) rename datafusion/{physical-expr-common => physical-expr}/src/expressions/column.rs (83%) rename datafusion/{physical-expr-common => physical-expr}/src/expressions/literal.rs (95%) diff --git a/Cargo.toml b/Cargo.toml index 90aff3f715ca..d0ac37a892dc 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -23,8 +23,11 @@ members = [ "datafusion/catalog", "datafusion/core", "datafusion/expr", + "datafusion/expr-common", + "datafusion/expr-functions-aggregate", "datafusion/execution", "datafusion/functions-aggregate", + "datafusion/functions-aggregate-common", "datafusion/functions", "datafusion/functions-nested", "datafusion/optimizer", @@ -94,8 +97,11 @@ datafusion-common = { path = "datafusion/common", version = "40.0.0", default-fe datafusion-common-runtime = { path = "datafusion/common-runtime", version = "40.0.0" } datafusion-execution = { path = "datafusion/execution", version = "40.0.0" } datafusion-expr = { path = "datafusion/expr", version = "40.0.0" } +datafusion-expr-common = { path = "datafusion/expr-common", version = "40.0.0" } +datafusion-expr-functions-aggregate = { path = "datafusion/expr-functions-aggregate", version = "40.0.0" } datafusion-functions = { path = "datafusion/functions", version = "40.0.0" } datafusion-functions-aggregate = { path = "datafusion/functions-aggregate", version = "40.0.0" } +datafusion-functions-aggregate-common = { path = "datafusion/functions-aggregate-common", version = "40.0.0" } datafusion-functions-nested = { path = "datafusion/functions-nested", version = "40.0.0" } datafusion-optimizer = { path = "datafusion/optimizer", version = "40.0.0", default-features = false } datafusion-physical-expr = { path = "datafusion/physical-expr", version = "40.0.0", default-features = false } diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 0415c3164b38..7a5ef953c5ad 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -103,6 +103,7 @@ datafusion-common = { workspace = true, features = ["object_store"] } datafusion-common-runtime = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } +datafusion-expr-functions-aggregate = { workspace = true } datafusion-functions = { workspace = true } datafusion-functions-aggregate = { workspace = true } datafusion-functions-nested = { workspace = true, optional = true } diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index 3bb0636652c0..12b5cd13aba7 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -556,6 +556,11 @@ pub mod physical_expr_common { pub use datafusion_physical_expr_common::*; } +/// re-export of [`datafusion_expr_functions_aggregate`] crate +pub mod expr_functions_aggregate { + pub use datafusion_expr_functions_aggregate::*; +} + /// re-export of [`datafusion_physical_expr`] crate pub mod physical_expr { pub use datafusion_physical_expr::*; diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index a0f6f6a65b1f..07942b6eddae 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -313,10 +313,10 @@ pub(crate) mod tests { use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; use datafusion_common::cast::as_int64_array; + use datafusion_expr_functions_aggregate::aggregate::AggregateExprBuilder; use datafusion_functions_aggregate::count::count_udaf; use datafusion_physical_expr::expressions::cast; use datafusion_physical_expr::PhysicalExpr; - use datafusion_physical_expr_common::aggregate::AggregateExprBuilder; use datafusion_physical_plan::aggregates::AggregateMode; /// Mock data using a MemoryExec which has an exact count statistic diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 6f3274820c8c..873e1d9f6173 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -174,10 +174,10 @@ mod tests { use crate::physical_plan::{displayable, Partitioning}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; + use datafusion_expr_functions_aggregate::aggregate::AggregateExprBuilder; use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::sum::sum_udaf; use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr_common::aggregate::AggregateExprBuilder; /// Runs the CombinePartialFinalAggregate optimizer and asserts the plan against the expected macro_rules! assert_optimized { diff --git a/datafusion/core/src/physical_optimizer/limit_pushdown.rs b/datafusion/core/src/physical_optimizer/limit_pushdown.rs index 4379a34a9426..732acc520498 100644 --- a/datafusion/core/src/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/limit_pushdown.rs @@ -257,10 +257,10 @@ mod tests { use arrow_schema::{DataType, Field, Schema, SchemaRef}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::Operator; + use datafusion_physical_expr::expressions::col; + use datafusion_physical_expr::expressions::lit; use datafusion_physical_expr::expressions::BinaryExpr; use datafusion_physical_expr::Partitioning; - use datafusion_physical_expr_common::expressions::column::col; - use datafusion_physical_expr_common::expressions::lit; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::empty::EmptyExec; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 378a892111c5..fb1c29d2d633 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1853,8 +1853,6 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( let agg_expr = udaf::create_aggregate_expr_with_dfschema( func, &physical_args, - args, - &sort_exprs, &ordering_reqs, logical_input_schema, name, diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 6f286c9aeba1..6b4e0c29a5cd 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -25,6 +25,7 @@ use arrow::util::pretty::pretty_format_batches; use arrow_array::types::Int64Type; use datafusion::common::Result; use datafusion::datasource::MemTable; +use datafusion::expr_functions_aggregate::aggregate::AggregateExprBuilder; use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; @@ -35,7 +36,6 @@ use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor} use datafusion_functions_aggregate::sum::sum_udaf; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; -use datafusion_physical_expr_common::aggregate::AggregateExprBuilder; use datafusion_physical_plan::InputOrderMode; use test_utils::{add_empty_batches, StringBatchGenerator}; diff --git a/datafusion/expr-common/Cargo.toml b/datafusion/expr-common/Cargo.toml new file mode 100644 index 000000000000..7e477efc4ebc --- /dev/null +++ b/datafusion/expr-common/Cargo.toml @@ -0,0 +1,43 @@ +# 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. + +[package] +name = "datafusion-expr-common" +description = "Logical plan and expression representation for DataFusion query engine" +keywords = ["datafusion", "logical", "plan", "expressions"] +readme = "README.md" +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +repository = { workspace = true } +license = { workspace = true } +authors = { workspace = true } +rust-version = { workspace = true } + +[lints] +workspace = true + +[lib] +name = "datafusion_expr_common" +path = "src/lib.rs" + +[features] + +[dependencies] +arrow = { workspace = true } +datafusion-common = { workspace = true } +paste = "^1.0" diff --git a/datafusion/expr/src/accumulator.rs b/datafusion/expr-common/src/accumulator.rs similarity index 100% rename from datafusion/expr/src/accumulator.rs rename to datafusion/expr-common/src/accumulator.rs diff --git a/datafusion/expr/src/columnar_value.rs b/datafusion/expr-common/src/columnar_value.rs similarity index 100% rename from datafusion/expr/src/columnar_value.rs rename to datafusion/expr-common/src/columnar_value.rs diff --git a/datafusion/expr/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs similarity index 99% rename from datafusion/expr/src/groups_accumulator.rs rename to datafusion/expr-common/src/groups_accumulator.rs index 886bd8443e4d..055d731b114c 100644 --- a/datafusion/expr/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -17,7 +17,7 @@ //! Vectorized [`GroupsAccumulator`] -use arrow_array::{ArrayRef, BooleanArray}; +use arrow::array::{ArrayRef, BooleanArray}; use datafusion_common::{not_impl_err, Result}; /// Describes how many rows should be emitted during grouping. diff --git a/datafusion/expr/src/interval_arithmetic.rs b/datafusion/expr-common/src/interval_arithmetic.rs similarity index 99% rename from datafusion/expr/src/interval_arithmetic.rs rename to datafusion/expr-common/src/interval_arithmetic.rs index 553cdd8c8709..d872b27d3c86 100644 --- a/datafusion/expr/src/interval_arithmetic.rs +++ b/datafusion/expr-common/src/interval_arithmetic.rs @@ -17,9 +17,9 @@ //! Interval arithmetic library +use crate::operator::Operator; use crate::type_coercion::binary::get_result_type; -use crate::Operator; -use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano}; +use arrow::datatypes::{IntervalDayTime, IntervalMonthDayNano}; use std::borrow::Borrow; use std::fmt::{self, Display, Formatter}; use std::ops::{AddAssign, SubAssign}; @@ -120,12 +120,12 @@ macro_rules! value_transition { IntervalYearMonth(None) } IntervalDayTime(Some(value)) - if value == arrow_buffer::IntervalDayTime::$bound => + if value == arrow::datatypes::IntervalDayTime::$bound => { IntervalDayTime(None) } IntervalMonthDayNano(Some(value)) - if value == arrow_buffer::IntervalMonthDayNano::$bound => + if value == arrow::datatypes::IntervalMonthDayNano::$bound => { IntervalMonthDayNano(None) } @@ -1135,12 +1135,12 @@ fn next_value_helper(value: ScalarValue) -> ScalarValue { } IntervalDayTime(Some(val)) => IntervalDayTime(Some(increment_decrement::< INC, - arrow_buffer::IntervalDayTime, + arrow::datatypes::IntervalDayTime, >(val))), IntervalMonthDayNano(Some(val)) => { IntervalMonthDayNano(Some(increment_decrement::< INC, - arrow_buffer::IntervalMonthDayNano, + arrow::datatypes::IntervalMonthDayNano, >(val))) } _ => value, // Unbounded values return without change. diff --git a/datafusion/expr-common/src/lib.rs b/datafusion/expr-common/src/lib.rs new file mode 100644 index 000000000000..2436b25271cf --- /dev/null +++ b/datafusion/expr-common/src/lib.rs @@ -0,0 +1,25 @@ +// 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 accumulator; +pub mod columnar_value; +pub mod groups_accumulator; +pub mod interval_arithmetic; +pub mod operator; +pub mod signature; +pub mod sort_properties; +pub mod type_coercion; diff --git a/datafusion/expr-common/src/operator.rs b/datafusion/expr-common/src/operator.rs new file mode 100644 index 000000000000..e013b6fafa22 --- /dev/null +++ b/datafusion/expr-common/src/operator.rs @@ -0,0 +1,282 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::fmt; + +/// Operators applied to expressions +#[derive(Debug, Copy, Clone, PartialEq, Eq, PartialOrd, Hash)] +pub enum Operator { + /// Expressions are equal + Eq, + /// Expressions are not equal + NotEq, + /// Left side is smaller than right side + Lt, + /// Left side is smaller or equal to right side + LtEq, + /// Left side is greater than right side + Gt, + /// Left side is greater or equal to right side + GtEq, + /// Addition + Plus, + /// Subtraction + Minus, + /// Multiplication operator, like `*` + Multiply, + /// Division operator, like `/` + Divide, + /// Remainder operator, like `%` + Modulo, + /// Logical AND, like `&&` + And, + /// Logical OR, like `||` + Or, + /// `IS DISTINCT FROM` (see [`distinct`]) + /// + /// [`distinct`]: arrow::compute::kernels::cmp::distinct + IsDistinctFrom, + /// `IS NOT DISTINCT FROM` (see [`not_distinct`]) + /// + /// [`not_distinct`]: arrow::compute::kernels::cmp::not_distinct + IsNotDistinctFrom, + /// Case sensitive regex match + RegexMatch, + /// Case insensitive regex match + RegexIMatch, + /// Case sensitive regex not match + RegexNotMatch, + /// Case insensitive regex not match + RegexNotIMatch, + /// Case sensitive pattern match + LikeMatch, + /// Case insensitive pattern match + ILikeMatch, + /// Case sensitive pattern not match + NotLikeMatch, + /// Case insensitive pattern not match + NotILikeMatch, + /// Bitwise and, like `&` + BitwiseAnd, + /// Bitwise or, like `|` + BitwiseOr, + /// Bitwise xor, such as `^` in MySQL or `#` in PostgreSQL + BitwiseXor, + /// Bitwise right, like `>>` + BitwiseShiftRight, + /// Bitwise left, like `<<` + BitwiseShiftLeft, + /// String concat + StringConcat, + /// At arrow, like `@>` + AtArrow, + /// Arrow at, like `<@` + ArrowAt, +} + +impl Operator { + /// If the operator can be negated, return the negated operator + /// otherwise return None + pub fn negate(&self) -> Option { + match self { + Operator::Eq => Some(Operator::NotEq), + Operator::NotEq => Some(Operator::Eq), + Operator::Lt => Some(Operator::GtEq), + Operator::LtEq => Some(Operator::Gt), + Operator::Gt => Some(Operator::LtEq), + Operator::GtEq => Some(Operator::Lt), + Operator::IsDistinctFrom => Some(Operator::IsNotDistinctFrom), + Operator::IsNotDistinctFrom => Some(Operator::IsDistinctFrom), + Operator::LikeMatch => Some(Operator::NotLikeMatch), + Operator::ILikeMatch => Some(Operator::NotILikeMatch), + Operator::NotLikeMatch => Some(Operator::LikeMatch), + Operator::NotILikeMatch => Some(Operator::ILikeMatch), + Operator::Plus + | Operator::Minus + | Operator::Multiply + | Operator::Divide + | Operator::Modulo + | Operator::And + | Operator::Or + | Operator::RegexMatch + | Operator::RegexIMatch + | Operator::RegexNotMatch + | Operator::RegexNotIMatch + | Operator::BitwiseAnd + | Operator::BitwiseOr + | Operator::BitwiseXor + | Operator::BitwiseShiftRight + | Operator::BitwiseShiftLeft + | Operator::StringConcat + | Operator::AtArrow + | Operator::ArrowAt => None, + } + } + + /// Return true if the operator is a numerical operator. + /// + /// For example, 'Binary(a, +, b)' would be a numerical expression. + /// PostgresSQL concept: + pub fn is_numerical_operators(&self) -> bool { + matches!( + self, + Operator::Plus + | Operator::Minus + | Operator::Multiply + | Operator::Divide + | Operator::Modulo + ) + } + + /// Return true if the operator is a comparison operator. + /// + /// For example, 'Binary(a, >, b)' would be a comparison expression. + pub fn is_comparison_operator(&self) -> bool { + matches!( + self, + Operator::Eq + | Operator::NotEq + | Operator::Lt + | Operator::LtEq + | Operator::Gt + | Operator::GtEq + | Operator::IsDistinctFrom + | Operator::IsNotDistinctFrom + | Operator::RegexMatch + | Operator::RegexIMatch + | Operator::RegexNotMatch + | Operator::RegexNotIMatch + ) + } + + /// Return true if the operator is a logic operator. + /// + /// For example, 'Binary(Binary(a, >, b), AND, Binary(a, <, b + 3))' would + /// be a logical expression. + pub fn is_logic_operator(&self) -> bool { + matches!(self, Operator::And | Operator::Or) + } + + /// Return the operator where swapping lhs and rhs wouldn't change the result. + /// + /// For example `Binary(50, >=, a)` could also be represented as `Binary(a, <=, 50)`. + pub fn swap(&self) -> Option { + match self { + Operator::Eq => Some(Operator::Eq), + Operator::NotEq => Some(Operator::NotEq), + Operator::Lt => Some(Operator::Gt), + Operator::LtEq => Some(Operator::GtEq), + Operator::Gt => Some(Operator::Lt), + Operator::GtEq => Some(Operator::LtEq), + Operator::AtArrow => Some(Operator::ArrowAt), + Operator::ArrowAt => Some(Operator::AtArrow), + Operator::IsDistinctFrom + | Operator::IsNotDistinctFrom + | Operator::Plus + | Operator::Minus + | Operator::Multiply + | Operator::Divide + | Operator::Modulo + | Operator::And + | Operator::Or + | Operator::RegexMatch + | Operator::RegexIMatch + | Operator::RegexNotMatch + | Operator::RegexNotIMatch + | Operator::LikeMatch + | Operator::ILikeMatch + | Operator::NotLikeMatch + | Operator::NotILikeMatch + | Operator::BitwiseAnd + | Operator::BitwiseOr + | Operator::BitwiseXor + | Operator::BitwiseShiftRight + | Operator::BitwiseShiftLeft + | Operator::StringConcat => None, + } + } + + /// Get the operator precedence + /// use as a reference + pub fn precedence(&self) -> u8 { + match self { + Operator::Or => 5, + Operator::And => 10, + Operator::Eq | Operator::NotEq | Operator::LtEq | Operator::GtEq => 15, + Operator::Lt | Operator::Gt => 20, + Operator::LikeMatch + | Operator::NotLikeMatch + | Operator::ILikeMatch + | Operator::NotILikeMatch => 25, + Operator::IsDistinctFrom + | Operator::IsNotDistinctFrom + | Operator::RegexMatch + | Operator::RegexNotMatch + | Operator::RegexIMatch + | Operator::RegexNotIMatch + | Operator::BitwiseAnd + | Operator::BitwiseOr + | Operator::BitwiseShiftLeft + | Operator::BitwiseShiftRight + | Operator::BitwiseXor + | Operator::StringConcat + | Operator::AtArrow + | Operator::ArrowAt => 30, + Operator::Plus | Operator::Minus => 40, + Operator::Multiply | Operator::Divide | Operator::Modulo => 45, + } + } +} + +impl fmt::Display for Operator { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + let display = match &self { + Operator::Eq => "=", + Operator::NotEq => "!=", + Operator::Lt => "<", + Operator::LtEq => "<=", + Operator::Gt => ">", + Operator::GtEq => ">=", + Operator::Plus => "+", + Operator::Minus => "-", + Operator::Multiply => "*", + Operator::Divide => "/", + Operator::Modulo => "%", + Operator::And => "AND", + Operator::Or => "OR", + Operator::RegexMatch => "~", + Operator::RegexIMatch => "~*", + Operator::RegexNotMatch => "!~", + Operator::RegexNotIMatch => "!~*", + Operator::LikeMatch => "~~", + Operator::ILikeMatch => "~~*", + Operator::NotLikeMatch => "!~~", + Operator::NotILikeMatch => "!~~*", + Operator::IsDistinctFrom => "IS DISTINCT FROM", + Operator::IsNotDistinctFrom => "IS NOT DISTINCT FROM", + Operator::BitwiseAnd => "&", + Operator::BitwiseOr => "|", + Operator::BitwiseXor => "BIT_XOR", + Operator::BitwiseShiftRight => ">>", + Operator::BitwiseShiftLeft => "<<", + Operator::StringConcat => "||", + Operator::AtArrow => "@>", + Operator::ArrowAt => "<@", + }; + write!(f, "{display}") + } +} diff --git a/datafusion/expr/src/signature.rs b/datafusion/expr-common/src/signature.rs similarity index 98% rename from datafusion/expr/src/signature.rs rename to datafusion/expr-common/src/signature.rs index b1cec3bad774..30758523105b 100644 --- a/datafusion/expr/src/signature.rs +++ b/datafusion/expr-common/src/signature.rs @@ -170,7 +170,7 @@ impl std::fmt::Display for ArrayFunctionSignature { } impl TypeSignature { - pub(crate) fn to_string_repr(&self) -> Vec { + pub fn to_string_repr(&self) -> Vec { match self { TypeSignature::Variadic(types) => { vec![format!("{}, ..", Self::join_types(types, "/"))] @@ -207,10 +207,7 @@ impl TypeSignature { } /// Helper function to join types with specified delimiter. - pub(crate) fn join_types( - types: &[T], - delimiter: &str, - ) -> String { + pub fn join_types(types: &[T], delimiter: &str) -> String { types .iter() .map(|t| t.to_string()) diff --git a/datafusion/expr/src/sort_properties.rs b/datafusion/expr-common/src/sort_properties.rs similarity index 100% rename from datafusion/expr/src/sort_properties.rs rename to datafusion/expr-common/src/sort_properties.rs diff --git a/datafusion/expr-common/src/type_coercion.rs b/datafusion/expr-common/src/type_coercion.rs new file mode 100644 index 000000000000..e934c6eaf35b --- /dev/null +++ b/datafusion/expr-common/src/type_coercion.rs @@ -0,0 +1,19 @@ +// 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 aggregates; +pub mod binary; diff --git a/datafusion/expr/src/type_coercion/aggregates.rs b/datafusion/expr-common/src/type_coercion/aggregates.rs similarity index 99% rename from datafusion/expr/src/type_coercion/aggregates.rs rename to datafusion/expr-common/src/type_coercion/aggregates.rs index e7e58bf84362..40ee596eee05 100644 --- a/datafusion/expr/src/type_coercion/aggregates.rs +++ b/datafusion/expr-common/src/type_coercion/aggregates.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::TypeSignature; +use crate::signature::TypeSignature; use arrow::datatypes::{ DataType, TimeUnit, DECIMAL128_MAX_PRECISION, DECIMAL128_MAX_SCALE, DECIMAL256_MAX_PRECISION, DECIMAL256_MAX_SCALE, diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr-common/src/type_coercion/binary.rs similarity index 99% rename from datafusion/expr/src/type_coercion/binary.rs rename to datafusion/expr-common/src/type_coercion/binary.rs index 17280289ed1b..f59b8c085819 100644 --- a/datafusion/expr/src/type_coercion/binary.rs +++ b/datafusion/expr-common/src/type_coercion/binary.rs @@ -20,7 +20,7 @@ use std::collections::HashSet; use std::sync::Arc; -use crate::Operator; +use crate::operator::Operator; use arrow::array::{new_empty_array, Array}; use arrow::compute::can_cast_types; @@ -569,7 +569,7 @@ fn string_temporal_coercion( } /// Coerce `lhs_type` and `rhs_type` to a common type where both are numeric -pub(crate) fn binary_numeric_coercion( +pub fn binary_numeric_coercion( lhs_type: &DataType, rhs_type: &DataType, ) -> Option { diff --git a/datafusion/expr-functions-aggregate/Cargo.toml b/datafusion/expr-functions-aggregate/Cargo.toml new file mode 100644 index 000000000000..5e20ac651f58 --- /dev/null +++ b/datafusion/expr-functions-aggregate/Cargo.toml @@ -0,0 +1,50 @@ +# 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. + +[package] +name = "datafusion-expr-functions-aggregate" +description = "Logical plan and expression representation for DataFusion query engine" +keywords = ["datafusion", "logical", "plan", "expressions"] +readme = "README.md" +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +repository = { workspace = true } +license = { workspace = true } +authors = { workspace = true } +rust-version = { workspace = true } + +[lints] +workspace = true + +[lib] +name = "datafusion_expr_functions_aggregate" +path = "src/lib.rs" + +[features] + +[dependencies] +ahash = { workspace = true } +arrow = { workspace = true } +datafusion-common = { workspace = true } +datafusion-expr-common = { workspace = true } +datafusion-expr = { workspace = true } +datafusion-physical-expr-common = { workspace = true } +datafusion-functions-aggregate-common = { workspace = true } +rand = { workspace = true } +# strum = { version = "0.26.1", features = ["derive"] } +# strum_macros = "0.26.0" diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/expr-functions-aggregate/src/aggregate.rs similarity index 69% rename from datafusion/physical-expr-common/src/aggregate/mod.rs rename to datafusion/expr-functions-aggregate/src/aggregate.rs index 665cdd708329..e657c6e64c67 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/expr-functions-aggregate/src/aggregate.rs @@ -15,33 +15,25 @@ // specific language governing permissions and limitations // under the License. -use std::fmt::Debug; -use std::{any::Any, sync::Arc}; - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - -use datafusion_common::exec_err; use datafusion_common::{internal_err, not_impl_err, DFSchema, Result}; use datafusion_expr::function::StateFieldsArgs; -use datafusion_expr::type_coercion::aggregates::check_arg_count; use datafusion_expr::utils::AggregateOrderSensitivity; use datafusion_expr::ReversedUDAF; -use datafusion_expr::{ - function::AccumulatorArgs, Accumulator, AggregateUDF, Expr, GroupsAccumulator, -}; +use datafusion_expr::{function::AccumulatorArgs, Accumulator, AggregateUDF, Expr}; -use crate::physical_expr::PhysicalExpr; -use crate::sort_expr::{LexOrdering, PhysicalSortExpr}; -use crate::utils::reverse_order_bys; +use datafusion_expr_common::type_coercion::aggregates::check_arg_count; +use datafusion_functions_aggregate_common::aggregate::AggregateExpr; -use self::utils::down_cast_any_ref; +use datafusion_expr_common::groups_accumulator::GroupsAccumulator; -pub mod count_distinct; -pub mod groups_accumulator; -pub mod merge_arrays; -pub mod stats; -pub mod tdigest; -pub mod utils; +use datafusion_functions_aggregate_common::utils::{self, down_cast_any_ref}; +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::fmt::Debug; +use std::{any::Any, sync::Arc}; /// 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. @@ -73,7 +65,7 @@ pub fn create_aggregate_expr( ) -> Result> { let mut builder = AggregateExprBuilder::new(Arc::new(fun.clone()), input_phy_exprs.to_vec()); - builder = builder.sort_exprs(sort_exprs.to_vec()); + // builder = builder.sort_exprs(sort_exprs.to_vec()); builder = builder.order_by(ordering_req.to_vec()); builder = builder.logical_exprs(input_exprs.to_vec()); builder = builder.schema(Arc::new(schema.clone())); @@ -94,8 +86,8 @@ pub fn create_aggregate_expr( pub fn create_aggregate_expr_with_dfschema( fun: &AggregateUDF, input_phy_exprs: &[Arc], - input_exprs: &[Expr], - sort_exprs: &[Expr], + // input_exprs: &[Expr], + // sort_exprs: &[Expr], ordering_req: &[PhysicalSortExpr], dfschema: &DFSchema, name: impl Into, @@ -105,9 +97,9 @@ pub fn create_aggregate_expr_with_dfschema( ) -> Result> { let mut builder = AggregateExprBuilder::new(Arc::new(fun.clone()), input_phy_exprs.to_vec()); - builder = builder.sort_exprs(sort_exprs.to_vec()); + // builder = builder.sort_exprs(sort_exprs.to_vec()); builder = builder.order_by(ordering_req.to_vec()); - builder = builder.logical_exprs(input_exprs.to_vec()); + // builder = builder.logical_exprs(input_exprs.to_vec()); builder = builder.dfschema(dfschema.clone()); let schema: Schema = dfschema.into(); builder = builder.schema(Arc::new(schema)); @@ -191,7 +183,6 @@ impl AggregateExprBuilder { let mut ordering_fields = vec![]; - debug_assert_eq!(sort_exprs.len(), ordering_req.len()); if !ordering_req.is_empty() { let ordering_types = ordering_req .iter() @@ -217,12 +208,12 @@ impl AggregateExprBuilder { Ok(Arc::new(AggregateFunctionExpr { fun: Arc::unwrap_or_clone(fun), args, - logical_args, + // logical_args, data_type, name, schema: Arc::unwrap_or_clone(schema), dfschema, - sort_exprs, + // sort_exprs, ordering_req, ignore_nulls, ordering_fields, @@ -283,10 +274,10 @@ impl AggregateExprBuilder { } /// This method will be deprecated in - pub fn sort_exprs(mut self, sort_exprs: Vec) -> Self { - self.sort_exprs = sort_exprs; - self - } + // pub fn sort_exprs(mut self, sort_exprs: Vec) -> Self { + // self.sort_exprs = sort_exprs; + // self + // } /// This method will be deprecated in pub fn logical_exprs(mut self, logical_args: Vec) -> Self { @@ -295,171 +286,16 @@ impl AggregateExprBuilder { } } -/// An aggregate expression that: -/// * knows its resulting field -/// * knows how to create its accumulator -/// * knows its accumulator's state's field -/// * knows the expressions from whose its accumulator will receive values -/// -/// Any implementation of this trait also needs to implement the -/// `PartialEq` to allows comparing equality between the -/// trait objects. -pub trait AggregateExpr: Send + Sync + Debug + PartialEq { - /// Returns the aggregate expression as [`Any`] so that it can be - /// downcast to a specific implementation. - fn as_any(&self) -> &dyn Any; - - /// the field of the final result of this aggregation. - fn field(&self) -> Result; - - /// the accumulator used to accumulate values from the expressions. - /// the accumulator expects the same number of arguments as `expressions` and must - /// return states with the same description as `state_fields` - fn create_accumulator(&self) -> Result>; - - /// the fields that encapsulate the Accumulator's state - /// the number of fields here equals the number of states that the accumulator contains - fn state_fields(&self) -> Result>; - - /// expressions that are passed to the Accumulator. - /// Single-column aggregations such as `sum` return a single value, others (e.g. `cov`) return many. - fn expressions(&self) -> Vec>; - - /// Order by requirements for the aggregate function - /// By default it is `None` (there is no requirement) - /// Order-sensitive aggregators, such as `FIRST_VALUE(x ORDER BY y)` should implement this - fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { - None - } - - /// Indicates whether aggregator can produce the correct result with any - /// arbitrary input ordering. By default, we assume that aggregate expressions - /// are order insensitive. - fn order_sensitivity(&self) -> AggregateOrderSensitivity { - AggregateOrderSensitivity::Insensitive - } - - /// Sets the indicator whether ordering requirements of the aggregator is - /// satisfied by its input. If this is not the case, aggregators with order - /// sensitivity `AggregateOrderSensitivity::Beneficial` can still produce - /// the correct result with possibly more work internally. - /// - /// # Returns - /// - /// Returns `Ok(Some(updated_expr))` if the process completes successfully. - /// If the expression can benefit from existing input ordering, but does - /// not implement the method, returns an error. Order insensitive and hard - /// requirement aggregators return `Ok(None)`. - fn with_beneficial_ordering( - self: Arc, - _requirement_satisfied: bool, - ) -> Result>> { - if self.order_bys().is_some() && self.order_sensitivity().is_beneficial() { - return exec_err!( - "Should implement with satisfied for aggregator :{:?}", - self.name() - ); - } - Ok(None) - } - - /// Human readable name such as `"MIN(c2)"`. The default - /// implementation returns placeholder text. - fn name(&self) -> &str { - "AggregateExpr: default name" - } - - /// If the aggregate expression has a specialized - /// [`GroupsAccumulator`] implementation. If this returns true, - /// `[Self::create_groups_accumulator`] will be called. - fn groups_accumulator_supported(&self) -> bool { - false - } - - /// Return a specialized [`GroupsAccumulator`] that manages state - /// for all groups. - /// - /// For maximum performance, a [`GroupsAccumulator`] should be - /// implemented in addition to [`Accumulator`]. - fn create_groups_accumulator(&self) -> Result> { - not_impl_err!("GroupsAccumulator hasn't been implemented for {self:?} yet") - } - - /// Construct an expression that calculates the aggregate in reverse. - /// Typically the "reverse" expression is itself (e.g. SUM, COUNT). - /// For aggregates that do not support calculation in reverse, - /// returns None (which is the default value). - fn reverse_expr(&self) -> Option> { - None - } - - /// Creates accumulator implementation that supports retract - fn create_sliding_accumulator(&self) -> Result> { - not_impl_err!("Retractable Accumulator hasn't been implemented for {self:?} yet") - } - - /// Returns all expressions used in the [`AggregateExpr`]. - /// These expressions are (1)function arguments, (2) order by expressions. - fn all_expressions(&self) -> AggregatePhysicalExpressions { - let args = self.expressions(); - let order_bys = self.order_bys().unwrap_or(&[]); - let order_by_exprs = order_bys - .iter() - .map(|sort_expr| sort_expr.expr.clone()) - .collect::>(); - AggregatePhysicalExpressions { - args, - order_by_exprs, - } - } - - /// Rewrites [`AggregateExpr`], with new expressions given. The argument should be consistent - /// with the return value of the [`AggregateExpr::all_expressions`] method. - /// Returns `Some(Arc)` if re-write is supported, otherwise returns `None`. - /// TODO: This method only rewrites the [`PhysicalExpr`]s and does not handle [`Expr`]s. - /// This can cause silent bugs and should be fixed in the future (possibly with physical-to-logical - /// conversions). - fn with_new_expressions( - &self, - _args: Vec>, - _order_by_exprs: Vec>, - ) -> Option> { - None - } - - /// If this function is max, return (output_field, true) - /// if the function is min, return (output_field, false) - /// otherwise return None (the default) - /// - /// output_field is the name of the column produced by this aggregate - /// - /// Note: this is used to use special aggregate implementations in certain conditions - fn get_minmax_desc(&self) -> Option<(Field, bool)> { - None - } -} - -/// Stores the physical expressions used inside the `AggregateExpr`. -pub struct AggregatePhysicalExpressions { - /// Aggregate function arguments - pub args: Vec>, - /// Order by expressions - pub order_by_exprs: Vec>, -} - /// Physical aggregate expression of a UDAF. #[derive(Debug, Clone)] pub struct AggregateFunctionExpr { fun: AggregateUDF, args: Vec>, - logical_args: Vec, /// Output / return type of this aggregate data_type: DataType, name: String, schema: Schema, dfschema: DFSchema, - // The logical order by expressions - sort_exprs: Vec, // The physical order by expressions ordering_req: LexOrdering, // Whether to ignore null values @@ -525,12 +361,14 @@ impl AggregateExpr for AggregateFunctionExpr { schema: &self.schema, dfschema: &self.dfschema, ignore_nulls: self.ignore_nulls, - sort_exprs: &self.sort_exprs, + // sort_exprs: &self.sort_exprs, + ordering_req: &self.ordering_req, is_distinct: self.is_distinct, input_types: &self.input_types, - input_exprs: &self.logical_args, + // input_exprs: &self.logical_args, name: &self.name, is_reversed: self.is_reversed, + physical_exprs: &self.args, }; self.fun.accumulator(acc_args) @@ -542,12 +380,13 @@ impl AggregateExpr for AggregateFunctionExpr { schema: &self.schema, dfschema: &self.dfschema, ignore_nulls: self.ignore_nulls, - sort_exprs: &self.sort_exprs, + ordering_req: &self.ordering_req, is_distinct: self.is_distinct, input_types: &self.input_types, - input_exprs: &self.logical_args, + // input_exprs: &self.logical_args, name: &self.name, is_reversed: self.is_reversed, + physical_exprs: &self.args, }; let accumulator = self.fun.create_sliding_accumulator(args)?; @@ -614,12 +453,13 @@ impl AggregateExpr for AggregateFunctionExpr { schema: &self.schema, dfschema: &self.dfschema, ignore_nulls: self.ignore_nulls, - sort_exprs: &self.sort_exprs, + ordering_req: &self.ordering_req, is_distinct: self.is_distinct, input_types: &self.input_types, - input_exprs: &self.logical_args, + // input_exprs: &self.logical_args, name: &self.name, is_reversed: self.is_reversed, + physical_exprs: &self.args, }; self.fun.groups_accumulator_supported(args) } @@ -630,12 +470,12 @@ impl AggregateExpr for AggregateFunctionExpr { schema: &self.schema, dfschema: &self.dfschema, ignore_nulls: self.ignore_nulls, - sort_exprs: &self.sort_exprs, + ordering_req: &self.ordering_req, is_distinct: self.is_distinct, input_types: &self.input_types, - input_exprs: &self.logical_args, name: &self.name, is_reversed: self.is_reversed, + physical_exprs: &self.args, }; self.fun.create_groups_accumulator(args) } @@ -676,14 +516,15 @@ impl AggregateExpr for AggregateFunctionExpr { create_aggregate_expr_with_dfschema( &updated_fn, &self.args, - &self.logical_args, - &self.sort_exprs, + // &self.logical_args, + // &self.sort_exprs, &self.ordering_req, &self.dfschema, self.name(), self.ignore_nulls, self.is_distinct, self.is_reversed, + // physical_exprs: &self.args, ) .map(Some) } @@ -694,18 +535,18 @@ impl AggregateExpr for AggregateFunctionExpr { 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 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(); // If the function is changed, we need to reverse order_by clause as well // i.e. First(a order by b asc null first) -> Last(a order by b desc null last) @@ -717,8 +558,8 @@ impl AggregateExpr for AggregateFunctionExpr { let reverse_aggr = create_aggregate_expr_with_dfschema( &reverse_udf, &self.args, - &self.logical_args, - &reverse_sort_exprs, + // &self.logical_args, + // &reverse_sort_exprs, &reverse_ordering_req, &self.dfschema, name, diff --git a/datafusion/physical-expr-common/src/expressions/mod.rs b/datafusion/expr-functions-aggregate/src/lib.rs similarity index 86% rename from datafusion/physical-expr-common/src/expressions/mod.rs rename to datafusion/expr-functions-aggregate/src/lib.rs index dd534cc07d20..1e14a97af3b6 100644 --- a/datafusion/physical-expr-common/src/expressions/mod.rs +++ b/datafusion/expr-functions-aggregate/src/lib.rs @@ -15,9 +15,6 @@ // specific language governing permissions and limitations // under the License. -mod cast; -pub mod column; -pub mod literal; +//! Technically, those aggregate functions releated things that has depedency on `expr` should be here -pub use cast::{cast, cast_with_options, CastExpr}; -pub use literal::{lit, Literal}; +pub mod aggregate; diff --git a/datafusion/expr/Cargo.toml b/datafusion/expr/Cargo.toml index 1b6878b6f49e..b5d34d9a3834 100644 --- a/datafusion/expr/Cargo.toml +++ b/datafusion/expr/Cargo.toml @@ -43,7 +43,10 @@ arrow = { workspace = true } arrow-array = { workspace = true } arrow-buffer = { workspace = true } chrono = { workspace = true } -datafusion-common = { workspace = true, default-features = true } +datafusion-common = { workspace = true } +datafusion-expr-common = { workspace = true } +datafusion-functions-aggregate-common = { workspace = true } +datafusion-physical-expr-common = { workspace = true } paste = "^1.0" serde_json = { workspace = true } sqlparser = { workspace = true } diff --git a/datafusion/expr/src/function.rs b/datafusion/expr/src/function.rs index d8be2b434732..79aed2f0aaf2 100644 --- a/datafusion/expr/src/function.rs +++ b/datafusion/expr/src/function.rs @@ -21,6 +21,8 @@ use crate::ColumnarValue; use crate::{Accumulator, Expr, PartitionEvaluator}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::{DFSchema, Result}; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use std::sync::Arc; #[derive(Debug, Clone, Copy)] @@ -79,7 +81,8 @@ pub struct AccumulatorArgs<'a> { /// ``` /// /// If no `ORDER BY` is specified, `sort_exprs`` will be empty. - pub sort_exprs: &'a [Expr], + // pub sort_exprs: &'a [Expr], + pub ordering_req: &'a [PhysicalSortExpr], /// Whether the aggregation is running in reverse order pub is_reversed: bool, @@ -98,7 +101,8 @@ pub struct AccumulatorArgs<'a> { pub input_types: &'a [DataType], /// The logical expression of arguments the aggregate function takes. - pub input_exprs: &'a [Expr], + // pub input_exprs: &'a [Expr], + pub physical_exprs: &'a [Arc], } /// [`StateFieldsArgs`] contains information about the fields that an diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index f5460918fa70..69fb9b61784b 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -27,13 +27,10 @@ //! //! The [expr_fn] module contains functions for creating expressions. -mod accumulator; mod built_in_window_function; -mod columnar_value; mod literal; mod operator; mod partition_evaluator; -mod signature; mod table_source; mod udaf; mod udf; @@ -46,13 +43,20 @@ pub mod expr_fn; pub mod expr_rewriter; pub mod expr_schema; pub mod function; -pub mod groups_accumulator; -pub mod interval_arithmetic; +pub mod groups_accumulator { + pub use datafusion_expr_common::groups_accumulator::*; +} + +pub mod interval_arithmetic { + pub use datafusion_expr_common::interval_arithmetic::*; +} pub mod logical_plan; pub mod planner; pub mod registry; pub mod simplify; -pub mod sort_properties; +pub mod sort_properties { + pub use datafusion_expr_common::sort_properties::*; +} pub mod test; pub mod tree_node; pub mod type_coercion; @@ -62,9 +66,15 @@ pub mod window_frame; pub mod window_function; pub mod window_state; -pub use accumulator::Accumulator; pub use built_in_window_function::BuiltInWindowFunction; -pub use columnar_value::ColumnarValue; +pub use datafusion_expr_common::accumulator::Accumulator; +pub use datafusion_expr_common::columnar_value::ColumnarValue; +pub use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; +pub use datafusion_expr_common::operator::Operator; +pub use datafusion_expr_common::signature::{ + ArrayFunctionSignature, Signature, TypeSignature, Volatility, TIMEZONE_WILDCARD, +}; +pub use datafusion_expr_common::type_coercion::binary; pub use expr::{ Between, BinaryExpr, Case, Cast, Expr, GetFieldAccess, GroupingSet, Like, Sort as SortExpr, TryCast, WindowFunctionDefinition, @@ -75,14 +85,9 @@ pub use function::{ AccumulatorFactoryFunction, PartitionEvaluatorFactory, ReturnTypeFunction, ScalarFunctionImplementation, StateTypeFunction, }; -pub use groups_accumulator::{EmitTo, GroupsAccumulator}; pub use literal::{lit, lit_timestamp_nano, Literal, TimestampLiteral}; pub use logical_plan::*; -pub use operator::Operator; pub use partition_evaluator::PartitionEvaluator; -pub use signature::{ - ArrayFunctionSignature, Signature, TypeSignature, Volatility, TIMEZONE_WILDCARD, -}; pub use sqlparser; pub use table_source::{TableProviderFilterPushDown, TableSource, TableType}; pub use udaf::{AggregateUDF, AggregateUDFImpl, ReversedUDAF}; diff --git a/datafusion/expr/src/operator.rs b/datafusion/expr/src/operator.rs index 9bb8c48d6c71..cdf550dae6c0 100644 --- a/datafusion/expr/src/operator.rs +++ b/datafusion/expr/src/operator.rs @@ -20,274 +20,10 @@ use crate::expr_fn::binary_expr; use crate::Expr; use crate::Like; -use std::fmt; +use datafusion_expr_common::operator::Operator; use std::ops; use std::ops::Not; -/// Operators applied to expressions -#[derive(Debug, Copy, Clone, PartialEq, Eq, PartialOrd, Hash)] -pub enum Operator { - /// Expressions are equal - Eq, - /// Expressions are not equal - NotEq, - /// Left side is smaller than right side - Lt, - /// Left side is smaller or equal to right side - LtEq, - /// Left side is greater than right side - Gt, - /// Left side is greater or equal to right side - GtEq, - /// Addition - Plus, - /// Subtraction - Minus, - /// Multiplication operator, like `*` - Multiply, - /// Division operator, like `/` - Divide, - /// Remainder operator, like `%` - Modulo, - /// Logical AND, like `&&` - And, - /// Logical OR, like `||` - Or, - /// `IS DISTINCT FROM` (see [`distinct`]) - /// - /// [`distinct`]: arrow::compute::kernels::cmp::distinct - IsDistinctFrom, - /// `IS NOT DISTINCT FROM` (see [`not_distinct`]) - /// - /// [`not_distinct`]: arrow::compute::kernels::cmp::not_distinct - IsNotDistinctFrom, - /// Case sensitive regex match - RegexMatch, - /// Case insensitive regex match - RegexIMatch, - /// Case sensitive regex not match - RegexNotMatch, - /// Case insensitive regex not match - RegexNotIMatch, - /// Case sensitive pattern match - LikeMatch, - /// Case insensitive pattern match - ILikeMatch, - /// Case sensitive pattern not match - NotLikeMatch, - /// Case insensitive pattern not match - NotILikeMatch, - /// Bitwise and, like `&` - BitwiseAnd, - /// Bitwise or, like `|` - BitwiseOr, - /// Bitwise xor, such as `^` in MySQL or `#` in PostgreSQL - BitwiseXor, - /// Bitwise right, like `>>` - BitwiseShiftRight, - /// Bitwise left, like `<<` - BitwiseShiftLeft, - /// String concat - StringConcat, - /// At arrow, like `@>` - AtArrow, - /// Arrow at, like `<@` - ArrowAt, -} - -impl Operator { - /// If the operator can be negated, return the negated operator - /// otherwise return None - pub fn negate(&self) -> Option { - match self { - Operator::Eq => Some(Operator::NotEq), - Operator::NotEq => Some(Operator::Eq), - Operator::Lt => Some(Operator::GtEq), - Operator::LtEq => Some(Operator::Gt), - Operator::Gt => Some(Operator::LtEq), - Operator::GtEq => Some(Operator::Lt), - Operator::IsDistinctFrom => Some(Operator::IsNotDistinctFrom), - Operator::IsNotDistinctFrom => Some(Operator::IsDistinctFrom), - Operator::LikeMatch => Some(Operator::NotLikeMatch), - Operator::ILikeMatch => Some(Operator::NotILikeMatch), - Operator::NotLikeMatch => Some(Operator::LikeMatch), - Operator::NotILikeMatch => Some(Operator::ILikeMatch), - Operator::Plus - | Operator::Minus - | Operator::Multiply - | Operator::Divide - | Operator::Modulo - | Operator::And - | Operator::Or - | Operator::RegexMatch - | Operator::RegexIMatch - | Operator::RegexNotMatch - | Operator::RegexNotIMatch - | Operator::BitwiseAnd - | Operator::BitwiseOr - | Operator::BitwiseXor - | Operator::BitwiseShiftRight - | Operator::BitwiseShiftLeft - | Operator::StringConcat - | Operator::AtArrow - | Operator::ArrowAt => None, - } - } - - /// Return true if the operator is a numerical operator. - /// - /// For example, 'Binary(a, +, b)' would be a numerical expression. - /// PostgresSQL concept: - pub fn is_numerical_operators(&self) -> bool { - matches!( - self, - Operator::Plus - | Operator::Minus - | Operator::Multiply - | Operator::Divide - | Operator::Modulo - ) - } - - /// Return true if the operator is a comparison operator. - /// - /// For example, 'Binary(a, >, b)' would be a comparison expression. - pub fn is_comparison_operator(&self) -> bool { - matches!( - self, - Operator::Eq - | Operator::NotEq - | Operator::Lt - | Operator::LtEq - | Operator::Gt - | Operator::GtEq - | Operator::IsDistinctFrom - | Operator::IsNotDistinctFrom - | Operator::RegexMatch - | Operator::RegexIMatch - | Operator::RegexNotMatch - | Operator::RegexNotIMatch - ) - } - - /// Return true if the operator is a logic operator. - /// - /// For example, 'Binary(Binary(a, >, b), AND, Binary(a, <, b + 3))' would - /// be a logical expression. - pub fn is_logic_operator(&self) -> bool { - matches!(self, Operator::And | Operator::Or) - } - - /// Return the operator where swapping lhs and rhs wouldn't change the result. - /// - /// For example `Binary(50, >=, a)` could also be represented as `Binary(a, <=, 50)`. - pub fn swap(&self) -> Option { - match self { - Operator::Eq => Some(Operator::Eq), - Operator::NotEq => Some(Operator::NotEq), - Operator::Lt => Some(Operator::Gt), - Operator::LtEq => Some(Operator::GtEq), - Operator::Gt => Some(Operator::Lt), - Operator::GtEq => Some(Operator::LtEq), - Operator::AtArrow => Some(Operator::ArrowAt), - Operator::ArrowAt => Some(Operator::AtArrow), - Operator::IsDistinctFrom - | Operator::IsNotDistinctFrom - | Operator::Plus - | Operator::Minus - | Operator::Multiply - | Operator::Divide - | Operator::Modulo - | Operator::And - | Operator::Or - | Operator::RegexMatch - | Operator::RegexIMatch - | Operator::RegexNotMatch - | Operator::RegexNotIMatch - | Operator::LikeMatch - | Operator::ILikeMatch - | Operator::NotLikeMatch - | Operator::NotILikeMatch - | Operator::BitwiseAnd - | Operator::BitwiseOr - | Operator::BitwiseXor - | Operator::BitwiseShiftRight - | Operator::BitwiseShiftLeft - | Operator::StringConcat => None, - } - } - - /// Get the operator precedence - /// use as a reference - pub fn precedence(&self) -> u8 { - match self { - Operator::Or => 5, - Operator::And => 10, - Operator::Eq | Operator::NotEq | Operator::LtEq | Operator::GtEq => 15, - Operator::Lt | Operator::Gt => 20, - Operator::LikeMatch - | Operator::NotLikeMatch - | Operator::ILikeMatch - | Operator::NotILikeMatch => 25, - Operator::IsDistinctFrom - | Operator::IsNotDistinctFrom - | Operator::RegexMatch - | Operator::RegexNotMatch - | Operator::RegexIMatch - | Operator::RegexNotIMatch - | Operator::BitwiseAnd - | Operator::BitwiseOr - | Operator::BitwiseShiftLeft - | Operator::BitwiseShiftRight - | Operator::BitwiseXor - | Operator::StringConcat - | Operator::AtArrow - | Operator::ArrowAt => 30, - Operator::Plus | Operator::Minus => 40, - Operator::Multiply | Operator::Divide | Operator::Modulo => 45, - } - } -} - -impl fmt::Display for Operator { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - let display = match &self { - Operator::Eq => "=", - Operator::NotEq => "!=", - Operator::Lt => "<", - Operator::LtEq => "<=", - Operator::Gt => ">", - Operator::GtEq => ">=", - Operator::Plus => "+", - Operator::Minus => "-", - Operator::Multiply => "*", - Operator::Divide => "/", - Operator::Modulo => "%", - Operator::And => "AND", - Operator::Or => "OR", - Operator::RegexMatch => "~", - Operator::RegexIMatch => "~*", - Operator::RegexNotMatch => "!~", - Operator::RegexNotIMatch => "!~*", - Operator::LikeMatch => "~~", - Operator::ILikeMatch => "~~*", - Operator::NotLikeMatch => "!~~", - Operator::NotILikeMatch => "!~~*", - Operator::IsDistinctFrom => "IS DISTINCT FROM", - Operator::IsNotDistinctFrom => "IS NOT DISTINCT FROM", - Operator::BitwiseAnd => "&", - Operator::BitwiseOr => "|", - Operator::BitwiseXor => "BIT_XOR", - Operator::BitwiseShiftRight => ">>", - Operator::BitwiseShiftLeft => "<<", - Operator::StringConcat => "||", - Operator::AtArrow => "@>", - Operator::ArrowAt => "<@", - }; - write!(f, "{display}") - } -} - /// Support ` + ` fluent style impl ops::Add for Expr { type Output = Self; diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index ef52a01e0598..8f7d658ba3e0 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -17,9 +17,6 @@ use std::sync::Arc; -use crate::signature::{ - ArrayFunctionSignature, FIXED_SIZE_LIST_WILDCARD, TIMEZONE_WILDCARD, -}; use crate::{AggregateUDF, ScalarUDF, Signature, TypeSignature}; use arrow::{ compute::can_cast_types, @@ -29,6 +26,9 @@ use datafusion_common::utils::{coerced_fixed_size_list_to_list, list_ndims}; use datafusion_common::{ exec_err, internal_datafusion_err, internal_err, plan_err, Result, }; +use datafusion_expr_common::signature::{ + ArrayFunctionSignature, FIXED_SIZE_LIST_WILDCARD, TIMEZONE_WILDCARD, +}; use super::binary::{binary_numeric_coercion, comparison_coercion}; diff --git a/datafusion/expr/src/type_coercion/mod.rs b/datafusion/expr/src/type_coercion/mod.rs index e0d1236aac2d..3a5c65fb46ee 100644 --- a/datafusion/expr/src/type_coercion/mod.rs +++ b/datafusion/expr/src/type_coercion/mod.rs @@ -31,11 +31,14 @@ //! i64. However, i64 -> i32 is never performed as there are i64 //! values which can not be represented by i32 values. -pub mod aggregates; -pub mod binary; +pub mod aggregates { + pub use datafusion_expr_common::type_coercion::aggregates::*; +} pub mod functions; pub mod other; +pub use datafusion_expr_common::type_coercion::binary; + use arrow::datatypes::DataType; /// Determine whether the given data type `dt` represents signed numeric values. pub fn is_signed_numeric(dt: &DataType) -> bool { diff --git a/datafusion/expr/src/udf.rs b/datafusion/expr/src/udf.rs index 5ba6e3007ead..8b7cdc8f5a73 100644 --- a/datafusion/expr/src/udf.rs +++ b/datafusion/expr/src/udf.rs @@ -27,12 +27,12 @@ use arrow::datatypes::DataType; use datafusion_common::{not_impl_err, ExprSchema, Result}; use crate::expr::create_name; -use crate::interval_arithmetic::Interval; use crate::simplify::{ExprSimplifyResult, SimplifyInfo}; use crate::sort_properties::{ExprProperties, SortProperties}; use crate::{ ColumnarValue, Expr, ReturnTypeFunction, ScalarFunctionImplementation, Signature, }; +use datafusion_expr_common::interval_arithmetic::Interval; /// Logical representation of a Scalar User Defined Function. /// diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 683a8e170ed4..4044b0e8352e 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -23,10 +23,10 @@ use std::sync::Arc; use crate::expr::{Alias, Sort, WindowFunction}; use crate::expr_rewriter::strip_outer_reference; -use crate::signature::{Signature, TypeSignature}; use crate::{ and, BinaryExpr, Expr, ExprSchemable, Filter, GroupingSet, LogicalPlan, Operator, }; +use datafusion_expr_common::signature::{Signature, TypeSignature}; use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; use datafusion_common::tree_node::{ @@ -40,6 +40,8 @@ use datafusion_common::{ use sqlparser::ast::{ExceptSelectItem, ExcludeSelectItem, WildcardAdditionalOptions}; +pub use datafusion_functions_aggregate_common::order::AggregateOrderSensitivity; + /// The value to which `COUNT(*)` is expanded to in /// `COUNT()` expressions pub const COUNT_STAR_EXPANSION: ScalarValue = ScalarValue::Int64(Some(1)); @@ -1217,36 +1219,60 @@ pub fn format_state_name(name: &str, state_name: &str) -> String { format!("{name}[{state_name}]") } -/// Represents the sensitivity of an aggregate expression to ordering. -#[derive(Debug, PartialEq, Eq, Clone, Copy)] -pub enum AggregateOrderSensitivity { - /// Indicates that the aggregate expression is insensitive to ordering. - /// Ordering at the input is not important for the result of the aggregator. - Insensitive, - /// Indicates that the aggregate expression has a hard requirement on ordering. - /// The aggregator can not produce a correct result unless its ordering - /// requirement is satisfied. - HardRequirement, - /// Indicates that ordering is beneficial for the aggregate expression in terms - /// of evaluation efficiency. The aggregator can produce its result efficiently - /// when its required ordering is satisfied; however, it can still produce the - /// correct result (albeit less efficiently) when its required ordering is not met. - Beneficial, -} - -impl AggregateOrderSensitivity { - pub fn is_insensitive(&self) -> bool { - self.eq(&AggregateOrderSensitivity::Insensitive) - } - - pub fn is_beneficial(&self) -> bool { - self.eq(&AggregateOrderSensitivity::Beneficial) - } - - pub fn hard_requires(&self) -> bool { - self.eq(&AggregateOrderSensitivity::HardRequirement) - } -} +// /// Converts `datafusion_expr::Expr` into corresponding `Arc`. +// /// If conversion is not supported yet, returns Error. +// pub fn limited_convert_logical_expr_to_physical_expr_with_dfschema( +// expr: &Expr, +// dfschema: &DFSchema, +// ) -> Result> { +// match expr { +// Expr::Alias(Alias { expr, .. }) => Ok( +// limited_convert_logical_expr_to_physical_expr_with_dfschema(expr, dfschema)?, +// ), +// Expr::Column(col) => { +// let idx = dfschema.index_of_column(col)?; +// Ok(Arc::new(Column::new(&col.name, idx))) +// } +// Expr::Cast(cast_expr) => Ok(Arc::new(CastExpr::new( +// limited_convert_logical_expr_to_physical_expr_with_dfschema( +// cast_expr.expr.as_ref(), +// dfschema, +// )?, +// cast_expr.data_type.clone(), +// None, +// ))), +// Expr::Literal(value) => Ok(Arc::new(Literal::new(value.clone()))), +// _ => exec_err!( +// "Unsupported expression: {expr} for conversion to Arc" +// ), +// } +// } + +// /// Converts each [`Expr::Sort`] into a corresponding [`PhysicalSortExpr`]. +// /// Returns an error if the given logical expression is not a [`Expr::Sort`]. +// pub fn limited_convert_logical_sort_exprs_to_physical_with_dfschema( +// exprs: &[Expr], +// dfschema: &DFSchema, +// ) -> Result> { +// // Construct PhysicalSortExpr objects from Expr objects: +// let mut sort_exprs = vec![]; +// for expr in exprs { +// let Expr::Sort(sort) = expr else { +// return exec_err!("Expects to receive sort expression"); +// }; +// sort_exprs.push(PhysicalSortExpr::new( +// limited_convert_logical_expr_to_physical_expr_with_dfschema( +// sort.expr.as_ref(), +// dfschema, +// )?, +// SortOptions { +// descending: !sort.asc, +// nulls_first: sort.nulls_first, +// }, +// )) +// } +// Ok(sort_exprs) +// } #[cfg(test)] mod tests { diff --git a/datafusion/functions-aggregate-common/Cargo.toml b/datafusion/functions-aggregate-common/Cargo.toml new file mode 100644 index 000000000000..2c1119b82a09 --- /dev/null +++ b/datafusion/functions-aggregate-common/Cargo.toml @@ -0,0 +1,49 @@ +# 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. + +[package] +name = "datafusion-functions-aggregate-common" +description = "Common aggregate function packages for the DataFusion query engine" +keywords = ["datafusion", "logical", "plan", "expressions"] +readme = "README.md" +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +repository = { workspace = true } +license = { workspace = true } +authors = { workspace = true } +rust-version = { workspace = true } + +[lints] +workspace = true + +[lib] +name = "datafusion_functions_aggregate_common" +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-expr-common = { workspace = true } +# datafusion-expr = { workspace = true } +datafusion-physical-expr-common = { workspace = true } +rand = { workspace = true } +# strum = { version = "0.26.1", features = ["derive"] } +# strum_macros = "0.26.0" \ No newline at end of file diff --git a/datafusion/functions-aggregate-common/src/aggregate.rs b/datafusion/functions-aggregate-common/src/aggregate.rs new file mode 100644 index 000000000000..771e86969ce6 --- /dev/null +++ b/datafusion/functions-aggregate-common/src/aggregate.rs @@ -0,0 +1,187 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::fmt::Debug; +use std::{any::Any, sync::Arc}; + +use arrow::datatypes::Field; + +use datafusion_common::exec_err; +use datafusion_common::{not_impl_err, Result}; +// use datafusion_expr::utils::AggregateOrderSensitivity; +use crate::order::AggregateOrderSensitivity; +use datafusion_expr_common::accumulator::Accumulator; + +use datafusion_expr_common::groups_accumulator::GroupsAccumulator; + +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; + +pub mod count_distinct; +pub mod groups_accumulator; + +/// An aggregate expression that: +/// * knows its resulting field +/// * knows how to create its accumulator +/// * knows its accumulator's state's field +/// * knows the expressions from whose its accumulator will receive values +/// +/// Any implementation of this trait also needs to implement the +/// `PartialEq` to allows comparing equality between the +/// trait objects. +pub trait AggregateExpr: Send + Sync + Debug + PartialEq { + /// Returns the aggregate expression as [`Any`] so that it can be + /// downcast to a specific implementation. + fn as_any(&self) -> &dyn Any; + + /// the field of the final result of this aggregation. + fn field(&self) -> Result; + + /// the accumulator used to accumulate values from the expressions. + /// the accumulator expects the same number of arguments as `expressions` and must + /// return states with the same description as `state_fields` + fn create_accumulator(&self) -> Result>; + + /// the fields that encapsulate the Accumulator's state + /// the number of fields here equals the number of states that the accumulator contains + fn state_fields(&self) -> Result>; + + /// expressions that are passed to the Accumulator. + /// Single-column aggregations such as `sum` return a single value, others (e.g. `cov`) return many. + fn expressions(&self) -> Vec>; + + /// Order by requirements for the aggregate function + /// By default it is `None` (there is no requirement) + /// Order-sensitive aggregators, such as `FIRST_VALUE(x ORDER BY y)` should implement this + fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { + None + } + + /// Indicates whether aggregator can produce the correct result with any + /// arbitrary input ordering. By default, we assume that aggregate expressions + /// are order insensitive. + fn order_sensitivity(&self) -> AggregateOrderSensitivity { + AggregateOrderSensitivity::Insensitive + } + + /// Sets the indicator whether ordering requirements of the aggregator is + /// satisfied by its input. If this is not the case, aggregators with order + /// sensitivity `AggregateOrderSensitivity::Beneficial` can still produce + /// the correct result with possibly more work internally. + /// + /// # Returns + /// + /// Returns `Ok(Some(updated_expr))` if the process completes successfully. + /// If the expression can benefit from existing input ordering, but does + /// not implement the method, returns an error. Order insensitive and hard + /// requirement aggregators return `Ok(None)`. + fn with_beneficial_ordering( + self: Arc, + _requirement_satisfied: bool, + ) -> Result>> { + if self.order_bys().is_some() && self.order_sensitivity().is_beneficial() { + return exec_err!( + "Should implement with satisfied for aggregator :{:?}", + self.name() + ); + } + Ok(None) + } + + /// Human readable name such as `"MIN(c2)"`. The default + /// implementation returns placeholder text. + fn name(&self) -> &str { + "AggregateExpr: default name" + } + + /// If the aggregate expression has a specialized + /// [`GroupsAccumulator`] implementation. If this returns true, + /// `[Self::create_groups_accumulator`] will be called. + fn groups_accumulator_supported(&self) -> bool { + false + } + + /// Return a specialized [`GroupsAccumulator`] that manages state + /// for all groups. + /// + /// For maximum performance, a [`GroupsAccumulator`] should be + /// implemented in addition to [`Accumulator`]. + fn create_groups_accumulator(&self) -> Result> { + not_impl_err!("GroupsAccumulator hasn't been implemented for {self:?} yet") + } + + /// Construct an expression that calculates the aggregate in reverse. + /// Typically the "reverse" expression is itself (e.g. SUM, COUNT). + /// For aggregates that do not support calculation in reverse, + /// returns None (which is the default value). + fn reverse_expr(&self) -> Option> { + None + } + + /// Creates accumulator implementation that supports retract + fn create_sliding_accumulator(&self) -> Result> { + not_impl_err!("Retractable Accumulator hasn't been implemented for {self:?} yet") + } + + /// Returns all expressions used in the [`AggregateExpr`]. + /// These expressions are (1)function arguments, (2) order by expressions. + fn all_expressions(&self) -> AggregatePhysicalExpressions { + let args = self.expressions(); + let order_bys = self.order_bys().unwrap_or(&[]); + let order_by_exprs = order_bys + .iter() + .map(|sort_expr| sort_expr.expr.clone()) + .collect::>(); + AggregatePhysicalExpressions { + args, + order_by_exprs, + } + } + + /// Rewrites [`AggregateExpr`], with new expressions given. The argument should be consistent + /// with the return value of the [`AggregateExpr::all_expressions`] method. + /// Returns `Some(Arc)` if re-write is supported, otherwise returns `None`. + /// TODO: This method only rewrites the [`PhysicalExpr`]s and does not handle [`Expr`]s. + /// This can cause silent bugs and should be fixed in the future (possibly with physical-to-logical + /// conversions). + fn with_new_expressions( + &self, + _args: Vec>, + _order_by_exprs: Vec>, + ) -> Option> { + None + } + + /// If this function is max, return (output_field, true) + /// if the function is min, return (output_field, false) + /// otherwise return None (the default) + /// + /// output_field is the name of the column produced by this aggregate + /// + /// Note: this is used to use special aggregate implementations in certain conditions + fn get_minmax_desc(&self) -> Option<(Field, bool)> { + None + } +} + +/// Stores the physical expressions used inside the `AggregateExpr`. +pub struct AggregatePhysicalExpressions { + /// Aggregate function arguments + pub args: Vec>, + /// Order by expressions + pub order_by_exprs: Vec>, +} diff --git a/datafusion/physical-expr-common/src/aggregate/count_distinct/mod.rs b/datafusion/functions-aggregate-common/src/aggregate/count_distinct.rs similarity index 100% rename from datafusion/physical-expr-common/src/aggregate/count_distinct/mod.rs rename to datafusion/functions-aggregate-common/src/aggregate/count_distinct.rs diff --git a/datafusion/physical-expr-common/src/aggregate/count_distinct/bytes.rs b/datafusion/functions-aggregate-common/src/aggregate/count_distinct/bytes.rs similarity index 95% rename from datafusion/physical-expr-common/src/aggregate/count_distinct/bytes.rs rename to datafusion/functions-aggregate-common/src/aggregate/count_distinct/bytes.rs index 360d64ce0141..ee61128979e1 100644 --- a/datafusion/physical-expr-common/src/aggregate/count_distinct/bytes.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/count_distinct/bytes.rs @@ -17,13 +17,13 @@ //! [`BytesDistinctCountAccumulator`] for Utf8/LargeUtf8/Binary/LargeBinary values -use crate::binary_map::{ArrowBytesSet, OutputType}; -use crate::binary_view_map::ArrowBytesViewSet; use arrow::array::{ArrayRef, OffsetSizeTrait}; use datafusion_common::cast::as_list_array; use datafusion_common::utils::array_into_list_array_nullable; use datafusion_common::ScalarValue; -use datafusion_expr::Accumulator; +use datafusion_expr_common::accumulator::Accumulator; +use datafusion_physical_expr_common::binary_map::{ArrowBytesSet, OutputType}; +use datafusion_physical_expr_common::binary_view_map::ArrowBytesViewSet; use std::fmt::Debug; use std::sync::Arc; diff --git a/datafusion/physical-expr-common/src/aggregate/count_distinct/native.rs b/datafusion/functions-aggregate-common/src/aggregate/count_distinct/native.rs similarity index 98% rename from datafusion/physical-expr-common/src/aggregate/count_distinct/native.rs rename to datafusion/functions-aggregate-common/src/aggregate/count_distinct/native.rs index e525118b9a17..d128a8af58ee 100644 --- a/datafusion/physical-expr-common/src/aggregate/count_distinct/native.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/count_distinct/native.rs @@ -35,9 +35,9 @@ use datafusion_common::cast::{as_list_array, as_primitive_array}; use datafusion_common::utils::array_into_list_array_nullable; use datafusion_common::utils::memory::estimate_memory_size; use datafusion_common::ScalarValue; -use datafusion_expr::Accumulator; +use datafusion_expr_common::accumulator::Accumulator; -use crate::aggregate::utils::Hashable; +use crate::utils::Hashable; #[derive(Debug)] pub struct PrimitiveDistinctCountAccumulator diff --git a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/mod.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs similarity index 100% rename from datafusion/physical-expr-common/src/aggregate/groups_accumulator/mod.rs rename to datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs diff --git a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/accumulate.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs similarity index 99% rename from datafusion/physical-expr-common/src/aggregate/groups_accumulator/accumulate.rs rename to datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs index 3fcd570f514e..18e497c63eb0 100644 --- a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -23,7 +23,7 @@ use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; use arrow::buffer::{BooleanBuffer, NullBuffer}; use arrow::datatypes::ArrowPrimitiveType; -use datafusion_expr::EmitTo; +use datafusion_expr_common::groups_accumulator::EmitTo; /// Track the accumulator null state per row: if any values for that /// group were null and if any values have been seen at all for that group. /// diff --git a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/bool_op.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs similarity index 98% rename from datafusion/physical-expr-common/src/aggregate/groups_accumulator/bool_op.rs rename to datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs index 8498d69dd333..be2b5e48a8db 100644 --- a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/bool_op.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use arrow::array::{ArrayRef, AsArray, BooleanArray, BooleanBufferBuilder}; use arrow::buffer::BooleanBuffer; use datafusion_common::Result; -use datafusion_expr::{EmitTo, GroupsAccumulator}; +use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; use super::accumulate::NullState; diff --git a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs similarity index 98% rename from datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs rename to datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs index 8d69646bd422..77828a59c3e7 100644 --- a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs @@ -23,7 +23,7 @@ use arrow::compute; use arrow::datatypes::ArrowPrimitiveType; use arrow::datatypes::DataType; use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; -use datafusion_expr::{EmitTo, GroupsAccumulator}; +use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; use super::accumulate::NullState; diff --git a/datafusion/functions-aggregate-common/src/lib.rs b/datafusion/functions-aggregate-common/src/lib.rs new file mode 100644 index 000000000000..43bc156f803e --- /dev/null +++ b/datafusion/functions-aggregate-common/src/lib.rs @@ -0,0 +1,25 @@ +// 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. + +//! [`AggregateUDF`]: User Defined Aggregate Functions + +pub mod aggregate; +pub mod merge_arrays; +pub mod order; +pub mod stats; +pub mod tdigest; +pub mod utils; diff --git a/datafusion/physical-expr-common/src/aggregate/merge_arrays.rs b/datafusion/functions-aggregate-common/src/merge_arrays.rs similarity index 100% rename from datafusion/physical-expr-common/src/aggregate/merge_arrays.rs rename to datafusion/functions-aggregate-common/src/merge_arrays.rs diff --git a/datafusion/functions-aggregate-common/src/order.rs b/datafusion/functions-aggregate-common/src/order.rs new file mode 100644 index 000000000000..4d7e604687b8 --- /dev/null +++ b/datafusion/functions-aggregate-common/src/order.rs @@ -0,0 +1,30 @@ +/// Represents the sensitivity of an aggregate expression to ordering. +#[derive(Debug, PartialEq, Eq, Clone, Copy)] +pub enum AggregateOrderSensitivity { + /// Indicates that the aggregate expression is insensitive to ordering. + /// Ordering at the input is not important for the result of the aggregator. + Insensitive, + /// Indicates that the aggregate expression has a hard requirement on ordering. + /// The aggregator can not produce a correct result unless its ordering + /// requirement is satisfied. + HardRequirement, + /// Indicates that ordering is beneficial for the aggregate expression in terms + /// of evaluation efficiency. The aggregator can produce its result efficiently + /// when its required ordering is satisfied; however, it can still produce the + /// correct result (albeit less efficiently) when its required ordering is not met. + Beneficial, +} + +impl AggregateOrderSensitivity { + pub fn is_insensitive(&self) -> bool { + self.eq(&AggregateOrderSensitivity::Insensitive) + } + + pub fn is_beneficial(&self) -> bool { + self.eq(&AggregateOrderSensitivity::Beneficial) + } + + pub fn hard_requires(&self) -> bool { + self.eq(&AggregateOrderSensitivity::HardRequirement) + } +} diff --git a/datafusion/physical-expr-common/src/aggregate/stats.rs b/datafusion/functions-aggregate-common/src/stats.rs similarity index 100% rename from datafusion/physical-expr-common/src/aggregate/stats.rs rename to datafusion/functions-aggregate-common/src/stats.rs diff --git a/datafusion/physical-expr-common/src/aggregate/tdigest.rs b/datafusion/functions-aggregate-common/src/tdigest.rs similarity index 100% rename from datafusion/physical-expr-common/src/aggregate/tdigest.rs rename to datafusion/functions-aggregate-common/src/tdigest.rs diff --git a/datafusion/physical-expr-common/src/aggregate/utils.rs b/datafusion/functions-aggregate-common/src/utils.rs similarity index 98% rename from datafusion/physical-expr-common/src/aggregate/utils.rs rename to datafusion/functions-aggregate-common/src/utils.rs index 9e380bd820ff..7b8ce0397af8 100644 --- a/datafusion/physical-expr-common/src/aggregate/utils.rs +++ b/datafusion/functions-aggregate-common/src/utils.rs @@ -29,11 +29,10 @@ use arrow::{ }, }; use datafusion_common::{exec_err, DataFusionError, Result}; -use datafusion_expr::Accumulator; +use datafusion_expr_common::accumulator::Accumulator; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; -use crate::sort_expr::PhysicalSortExpr; - -use super::AggregateExpr; +use crate::aggregate::AggregateExpr; /// Downcast a `Box` or `Arc` /// and return the inner trait object as [`Any`] so diff --git a/datafusion/functions-aggregate/Cargo.toml b/datafusion/functions-aggregate/Cargo.toml index 4f2bd864832e..9e7476e81d05 100644 --- a/datafusion/functions-aggregate/Cargo.toml +++ b/datafusion/functions-aggregate/Cargo.toml @@ -45,6 +45,8 @@ datafusion-common = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } datafusion-physical-expr-common = { workspace = true } +datafusion-physical-expr = { workspace = true } +datafusion-functions-aggregate-common = { workspace = true } log = { workspace = true } paste = "1.0.14" sqlparser = { workspace = true } diff --git a/datafusion/functions-aggregate/benches/count.rs b/datafusion/functions-aggregate/benches/count.rs index 875112ca8d47..fa1051e945b1 100644 --- a/datafusion/functions-aggregate/benches/count.rs +++ b/datafusion/functions-aggregate/benches/count.rs @@ -23,6 +23,7 @@ use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_common::DFSchema; use datafusion_expr::{function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator}; use datafusion_functions_aggregate::count::Count; +use datafusion_physical_expr::expressions::col; use std::sync::Arc; fn prepare_accumulator() -> Box { @@ -33,12 +34,12 @@ fn prepare_accumulator() -> Box { schema: &schema, dfschema: &df_schema, ignore_nulls: false, - sort_exprs: &[], + ordering_req: &[], is_reversed: false, name: "COUNT(f)", is_distinct: false, input_types: &[DataType::Int32], - input_exprs: &[datafusion_expr::col("f")], + physical_exprs: &[col("f", &schema).unwrap()], }; let count_fn = Count::new(); diff --git a/datafusion/functions-aggregate/benches/sum.rs b/datafusion/functions-aggregate/benches/sum.rs index dfaa93cdeff7..c800e09aa99b 100644 --- a/datafusion/functions-aggregate/benches/sum.rs +++ b/datafusion/functions-aggregate/benches/sum.rs @@ -23,6 +23,7 @@ use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_common::DFSchema; use datafusion_expr::{function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator}; use datafusion_functions_aggregate::sum::Sum; +use datafusion_physical_expr::expressions::col; use std::sync::Arc; fn prepare_accumulator(data_type: &DataType) -> Box { @@ -33,12 +34,12 @@ fn prepare_accumulator(data_type: &DataType) -> Box { schema: &schema, dfschema: &df_schema, ignore_nulls: false, - sort_exprs: &[], + ordering_req: &[], is_reversed: false, name: "SUM(f)", is_distinct: false, input_types: &[data_type.clone()], - input_exprs: &[datafusion_expr::col("f")], + physical_exprs: &[col("f", &schema).unwrap()], }; let sum_fn = Sum::new(); diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index af2a26fd05ec..1ce79df3aa78 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -31,9 +31,10 @@ use arrow::{ use arrow_schema::{Field, Schema}; use datafusion_common::{ - downcast_value, internal_err, not_impl_err, plan_err, DFSchema, DataFusionError, - ScalarValue, + downcast_value, exec_err, internal_err, not_impl_err, plan_err, DFSchema, + DataFusionError, Result, ScalarValue, }; +use datafusion_expr::expr::Alias; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::{INTEGERS, NUMERICS}; use datafusion_expr::utils::format_state_name; @@ -41,10 +42,11 @@ use datafusion_expr::{ Accumulator, AggregateUDFImpl, ColumnarValue, Expr, Signature, TypeSignature, Volatility, }; -use datafusion_physical_expr_common::aggregate::tdigest::{ +use datafusion_functions_aggregate_common::tdigest::{ TDigest, TryIntoF64, DEFAULT_MAX_SIZE, }; -use datafusion_physical_expr_common::utils::limited_convert_logical_expr_to_physical_expr_with_dfschema; +use datafusion_physical_expr::expressions::{CastExpr, Column, Literal}; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; make_udaf_expr_and_func!( ApproxPercentileCont, @@ -98,9 +100,9 @@ impl ApproxPercentileCont { &self, args: AccumulatorArgs, ) -> datafusion_common::Result { - let percentile = validate_input_percentile_expr(&args.input_exprs[1])?; - let tdigest_max_size = if args.input_exprs.len() == 3 { - Some(validate_input_max_size_expr(&args.input_exprs[2])?) + let percentile = validate_input_percentile_expr(&args.physical_exprs[1])?; + let tdigest_max_size = if args.physical_exprs.len() == 3 { + Some(validate_input_max_size_expr(&args.physical_exprs[2])?) } else { None }; @@ -134,12 +136,42 @@ impl ApproxPercentileCont { } } -fn get_lit_value(expr: &Expr) -> datafusion_common::Result { +/// Converts `datafusion_expr::Expr` into corresponding `Arc`. +/// If conversion is not supported yet, returns Error. +fn limited_convert_logical_expr_to_physical_expr_with_dfschema( + expr: &Expr, + dfschema: &DFSchema, +) -> Result> { + match expr { + Expr::Alias(Alias { expr, .. }) => Ok( + limited_convert_logical_expr_to_physical_expr_with_dfschema(expr, dfschema)?, + ), + Expr::Column(col) => { + let idx = dfschema.index_of_column(col)?; + Ok(Arc::new(Column::new(&col.name, idx))) + } + Expr::Cast(cast_expr) => Ok(Arc::new(CastExpr::new( + limited_convert_logical_expr_to_physical_expr_with_dfschema( + cast_expr.expr.as_ref(), + dfschema, + )?, + cast_expr.data_type.clone(), + None, + ))), + Expr::Literal(value) => Ok(Arc::new(Literal::new(value.clone()))), + _ => exec_err!( + "Unsupported expression: {expr} for conversion to Arc" + ), + } +} + +fn get_lit_value(expr: &Arc) -> datafusion_common::Result { + // TODO: use real schema let empty_schema = Arc::new(Schema::empty()); let empty_batch = RecordBatch::new_empty(Arc::clone(&empty_schema)); - let dfschema = DFSchema::empty(); - let expr = - limited_convert_logical_expr_to_physical_expr_with_dfschema(expr, &dfschema)?; + // let dfschema = DFSchema::empty(); + // let expr = + // limited_convert_logical_expr_to_physical_expr_with_dfschema(expr, &dfschema)?; let result = expr.evaluate(&empty_batch)?; match result { ColumnarValue::Array(_) => Err(DataFusionError::Internal(format!( @@ -150,7 +182,9 @@ fn get_lit_value(expr: &Expr) -> datafusion_common::Result { } } -fn validate_input_percentile_expr(expr: &Expr) -> datafusion_common::Result { +fn validate_input_percentile_expr( + expr: &Arc, +) -> datafusion_common::Result { let lit = get_lit_value(expr)?; let percentile = match &lit { ScalarValue::Float32(Some(q)) => *q as f64, @@ -170,7 +204,9 @@ fn validate_input_percentile_expr(expr: &Expr) -> datafusion_common::Result Ok(percentile) } -fn validate_input_max_size_expr(expr: &Expr) -> datafusion_common::Result { +fn validate_input_max_size_expr( + expr: &Arc, +) -> datafusion_common::Result { let lit = get_lit_value(expr)?; let max_size = match &lit { ScalarValue::UInt8(Some(q)) => *q as usize, @@ -464,7 +500,7 @@ impl Accumulator for ApproxPercentileAccumulator { mod tests { use arrow_schema::DataType; - use datafusion_physical_expr_common::aggregate::tdigest::TDigest; + use datafusion_functions_aggregate_common::tdigest::TDigest; use crate::approx_percentile_cont::ApproxPercentileAccumulator; diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs b/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs index 0dbea1fb1ff7..917da47ed669 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs @@ -17,6 +17,7 @@ use std::any::Any; use std::fmt::{Debug, Formatter}; +use std::sync::Arc; use arrow::{ array::ArrayRef, @@ -29,7 +30,7 @@ use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::NUMERICS; use datafusion_expr::Volatility::Immutable; use datafusion_expr::{Accumulator, AggregateUDFImpl, Signature, TypeSignature}; -use datafusion_physical_expr_common::aggregate::tdigest::{ +use datafusion_functions_aggregate_common::tdigest::{ Centroid, TDigest, DEFAULT_MAX_SIZE, }; @@ -123,16 +124,16 @@ impl AggregateUDFImpl for ApproxPercentileContWithWeight { ); } - if acc_args.input_exprs.len() != 3 { + if acc_args.physical_exprs.len() != 3 { return plan_err!( "approx_percentile_cont_with_weight requires three arguments: value, weight, percentile" ); } let sub_args = AccumulatorArgs { - input_exprs: &[ - acc_args.input_exprs[0].clone(), - acc_args.input_exprs[2].clone(), + physical_exprs: &[ + Arc::clone(&acc_args.physical_exprs[0]), + Arc::clone(&acc_args.physical_exprs[2]), ], ..acc_args }; diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index 36c9d6a0d7c8..b451b87ab0ea 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -29,12 +29,9 @@ use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::format_state_name; use datafusion_expr::AggregateUDFImpl; use datafusion_expr::{Accumulator, Signature, Volatility}; -use datafusion_physical_expr_common::aggregate::merge_arrays::merge_ordered_arrays; -use datafusion_physical_expr_common::aggregate::utils::ordering_fields; -use datafusion_physical_expr_common::sort_expr::{ - limited_convert_logical_sort_exprs_to_physical_with_dfschema, LexOrdering, - PhysicalSortExpr, -}; +use datafusion_functions_aggregate_common::merge_arrays::merge_ordered_arrays; +use datafusion_functions_aggregate_common::utils::ordering_fields; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use std::collections::{HashSet, VecDeque}; use std::sync::Arc; @@ -123,18 +120,14 @@ impl AggregateUDFImpl for ArrayAgg { )?)); } - if acc_args.sort_exprs.is_empty() { + if acc_args.ordering_req.is_empty() { return Ok(Box::new(ArrayAggAccumulator::try_new( &acc_args.input_types[0], )?)); } - let ordering_req = limited_convert_logical_sort_exprs_to_physical_with_dfschema( - acc_args.sort_exprs, - acc_args.dfschema, - )?; - - let ordering_dtypes = ordering_req + let ordering_dtypes = acc_args + .ordering_req .iter() .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; @@ -142,7 +135,7 @@ impl AggregateUDFImpl for ArrayAgg { OrderSensitiveArrayAggAccumulator::try_new( &acc_args.input_types[0], &ordering_dtypes, - ordering_req, + acc_args.ordering_req.to_vec(), acc_args.is_reversed, ) .map(|acc| Box::new(acc) as _) diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 228bce1979a3..259520c0dfa7 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -34,8 +34,8 @@ use datafusion_expr::Volatility::Immutable; use datafusion_expr::{ Accumulator, AggregateUDFImpl, EmitTo, GroupsAccumulator, ReversedUDAF, Signature, }; -use datafusion_physical_expr_common::aggregate::groups_accumulator::accumulate::NullState; -use datafusion_physical_expr_common::aggregate::utils::DecimalAverager; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::NullState; +use datafusion_functions_aggregate_common::utils::DecimalAverager; use log::debug; use std::any::Any; use std::fmt::Debug; diff --git a/datafusion/functions-aggregate/src/bit_and_or_xor.rs b/datafusion/functions-aggregate/src/bit_and_or_xor.rs index f6dd0bc20a83..b6b1862d281b 100644 --- a/datafusion/functions-aggregate/src/bit_and_or_xor.rs +++ b/datafusion/functions-aggregate/src/bit_and_or_xor.rs @@ -38,7 +38,7 @@ use datafusion_expr::{ Accumulator, AggregateUDFImpl, GroupsAccumulator, ReversedUDAF, Signature, Volatility, }; -use datafusion_physical_expr_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; use std::ops::{BitAndAssign, BitOrAssign, BitXorAssign}; /// This macro helps create group accumulators based on bitwise operations typically used internally diff --git a/datafusion/functions-aggregate/src/bool_and_or.rs b/datafusion/functions-aggregate/src/bool_and_or.rs index d0028672743e..c9d3d1d8c6b4 100644 --- a/datafusion/functions-aggregate/src/bool_and_or.rs +++ b/datafusion/functions-aggregate/src/bool_and_or.rs @@ -35,7 +35,7 @@ use datafusion_expr::{ Accumulator, AggregateUDFImpl, GroupsAccumulator, ReversedUDAF, Signature, Volatility, }; -use datafusion_physical_expr_common::aggregate::groups_accumulator::bool_op::BooleanGroupsAccumulator; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::bool_op::BooleanGroupsAccumulator; // returns the new value after bool_and/bool_or with the new values, taking nullability into account macro_rules! typed_bool_and_or_batch { diff --git a/datafusion/functions-aggregate/src/correlation.rs b/datafusion/functions-aggregate/src/correlation.rs index c2d7a89081d6..88f01b06d2d9 100644 --- a/datafusion/functions-aggregate/src/correlation.rs +++ b/datafusion/functions-aggregate/src/correlation.rs @@ -36,7 +36,7 @@ use datafusion_expr::{ utils::format_state_name, Accumulator, AggregateUDFImpl, Signature, Volatility, }; -use datafusion_physical_expr_common::aggregate::stats::StatsType; +use datafusion_functions_aggregate_common::stats::StatsType; make_udaf_expr_and_func!( Correlation, diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index aea05442536e..f41404eef880 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -16,7 +16,7 @@ // under the License. use ahash::RandomState; -use datafusion_physical_expr_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; +use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; use std::collections::HashSet; use std::ops::BitAnd; use std::{fmt::Debug, sync::Arc}; @@ -47,14 +47,12 @@ use datafusion_expr::{ EmitTo, GroupsAccumulator, Signature, Volatility, }; use datafusion_expr::{Expr, ReversedUDAF, TypeSignature}; -use datafusion_physical_expr_common::aggregate::groups_accumulator::accumulate::accumulate_indices; -use datafusion_physical_expr_common::{ - aggregate::count_distinct::{ - BytesDistinctCountAccumulator, FloatDistinctCountAccumulator, - PrimitiveDistinctCountAccumulator, - }, - binary_map::OutputType, +use datafusion_functions_aggregate_common::aggregate::count_distinct::{ + BytesDistinctCountAccumulator, FloatDistinctCountAccumulator, + PrimitiveDistinctCountAccumulator, }; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::accumulate_indices; +use datafusion_physical_expr_common::binary_map::OutputType; make_udaf_expr_and_func!( Count, @@ -145,7 +143,7 @@ impl AggregateUDFImpl for Count { return Ok(Box::new(CountAccumulator::new())); } - if acc_args.input_exprs.len() > 1 { + if acc_args.physical_exprs.len() > 1 { return not_impl_err!("COUNT DISTINCT with multiple arguments"); } @@ -271,7 +269,7 @@ impl AggregateUDFImpl for Count { if args.is_distinct { return false; } - args.input_exprs.len() == 1 + args.physical_exprs.len() == 1 } fn create_groups_accumulator( diff --git a/datafusion/functions-aggregate/src/covariance.rs b/datafusion/functions-aggregate/src/covariance.rs index 6f03b256fd9f..d0abb079ef15 100644 --- a/datafusion/functions-aggregate/src/covariance.rs +++ b/datafusion/functions-aggregate/src/covariance.rs @@ -35,7 +35,7 @@ use datafusion_expr::{ utils::format_state_name, Accumulator, AggregateUDFImpl, Signature, Volatility, }; -use datafusion_physical_expr_common::aggregate::stats::StatsType; +use datafusion_functions_aggregate_common::stats::StatsType; make_udaf_expr_and_func!( CovarianceSample, diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 587767b8e356..28007503f754 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -34,11 +34,8 @@ use datafusion_expr::{ Accumulator, AggregateUDFImpl, ArrayFunctionSignature, Expr, ExprFunctionExt, Signature, TypeSignature, Volatility, }; -use datafusion_physical_expr_common::aggregate::utils::get_sort_options; -use datafusion_physical_expr_common::sort_expr::{ - limited_convert_logical_sort_exprs_to_physical_with_dfschema, LexOrdering, - PhysicalSortExpr, -}; +use datafusion_functions_aggregate_common::utils::get_sort_options; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; create_func!(FirstValue, first_value_udaf); @@ -117,24 +114,21 @@ impl AggregateUDFImpl for FirstValue { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - let ordering_req = limited_convert_logical_sort_exprs_to_physical_with_dfschema( - acc_args.sort_exprs, - acc_args.dfschema, - )?; - - let ordering_dtypes = ordering_req + let ordering_dtypes = acc_args + .ordering_req .iter() .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; // When requirement is empty, or it is signalled by outside caller that // the ordering requirement is/will be satisfied. - let requirement_satisfied = ordering_req.is_empty() || self.requirement_satisfied; + let requirement_satisfied = + acc_args.ordering_req.is_empty() || self.requirement_satisfied; FirstValueAccumulator::try_new( acc_args.data_type, &ordering_dtypes, - ordering_req, + acc_args.ordering_req.to_vec(), acc_args.ignore_nulls, ) .map(|acc| Box::new(acc.with_requirement_satisfied(requirement_satisfied)) as _) @@ -416,22 +410,19 @@ impl AggregateUDFImpl for LastValue { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - let ordering_req = limited_convert_logical_sort_exprs_to_physical_with_dfschema( - acc_args.sort_exprs, - acc_args.dfschema, - )?; - - let ordering_dtypes = ordering_req + let ordering_dtypes = acc_args + .ordering_req .iter() .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; - let requirement_satisfied = ordering_req.is_empty() || self.requirement_satisfied; + let requirement_satisfied = + acc_args.ordering_req.is_empty() || self.requirement_satisfied; LastValueAccumulator::try_new( acc_args.data_type, &ordering_dtypes, - ordering_req, + acc_args.ordering_req.to_vec(), acc_args.ignore_nulls, ) .map(|acc| Box::new(acc.with_requirement_satisfied(requirement_satisfied)) as _) diff --git a/datafusion/functions-aggregate/src/median.rs b/datafusion/functions-aggregate/src/median.rs index febf1fcd2fef..9a70591e18cf 100644 --- a/datafusion/functions-aggregate/src/median.rs +++ b/datafusion/functions-aggregate/src/median.rs @@ -38,7 +38,7 @@ use datafusion_expr::{ function::AccumulatorArgs, utils::format_state_name, Accumulator, AggregateUDFImpl, Signature, Volatility, }; -use datafusion_physical_expr_common::aggregate::utils::Hashable; +use datafusion_functions_aggregate_common::utils::Hashable; make_udaf_expr_and_func!( Median, diff --git a/datafusion/functions-aggregate/src/min_max.rs b/datafusion/functions-aggregate/src/min_max.rs index 4d743983411d..41513cf24c68 100644 --- a/datafusion/functions-aggregate/src/min_max.rs +++ b/datafusion/functions-aggregate/src/min_max.rs @@ -49,7 +49,7 @@ use arrow::datatypes::{ }; use arrow_schema::IntervalUnit; use datafusion_common::{downcast_value, internal_err, DataFusionError, Result}; -use datafusion_physical_expr_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; use std::fmt::Debug; use arrow::datatypes::i256; diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index dc7c6c86f213..60b8e818df15 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -30,14 +30,12 @@ use datafusion_common::{exec_err, internal_err, not_impl_err, Result, ScalarValu use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ - Accumulator, AggregateUDFImpl, Expr, ReversedUDAF, Signature, Volatility, -}; -use datafusion_physical_expr_common::aggregate::merge_arrays::merge_ordered_arrays; -use datafusion_physical_expr_common::aggregate::utils::ordering_fields; -use datafusion_physical_expr_common::sort_expr::{ - limited_convert_logical_sort_exprs_to_physical_with_dfschema, LexOrdering, - PhysicalSortExpr, + Accumulator, AggregateUDFImpl, ReversedUDAF, Signature, Volatility, }; +use datafusion_functions_aggregate_common::merge_arrays::merge_ordered_arrays; +use datafusion_functions_aggregate_common::utils::ordering_fields; +use datafusion_physical_expr::expressions::Literal; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; make_udaf_expr_and_func!( NthValueAgg, @@ -87,27 +85,55 @@ impl AggregateUDFImpl for NthValueAgg { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - let n = match acc_args.input_exprs[1] { - Expr::Literal(ScalarValue::Int64(Some(value))) => { + // TODO: simplfiy this + let n = if let Some(lit) = acc_args.physical_exprs[1] + .as_any() + .downcast_ref::() + { + if let ScalarValue::Int64(Some(value)) = lit.value() { if acc_args.is_reversed { - Ok(-value) + -*value } else { - Ok(value) + *value } + } else { + return not_impl_err!( + "{} not supported for n: {}", + self.name(), + &acc_args.physical_exprs[1] + ); } - _ => not_impl_err!( + } else { + return not_impl_err!( "{} not supported for n: {}", self.name(), - &acc_args.input_exprs[1] - ), - }?; - - let ordering_req = limited_convert_logical_sort_exprs_to_physical_with_dfschema( - acc_args.sort_exprs, - acc_args.dfschema, - )?; + &acc_args.physical_exprs[1] + ); + }; - let ordering_dtypes = ordering_req + // let n = match acc_args.physical_exprs[1] { + + // Expr::Literal(ScalarValue::Int64(Some(value))) => { + // if acc_args.is_reversed { + // Ok(-value) + // } else { + // Ok(value) + // } + // } + // _ => not_impl_err!( + // "{} not supported for n: {}", + // self.name(), + // &acc_args.physical_exprs[1] + // ), + // }?; + + // let ordering_req = limited_convert_logical_sort_exprs_to_physical_with_dfschema( + // acc_args.sort_exprs, + // acc_args.dfschema, + // )?; + + let ordering_dtypes = acc_args + .ordering_req .iter() .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; @@ -116,7 +142,7 @@ impl AggregateUDFImpl for NthValueAgg { n, &acc_args.input_types[0], &ordering_dtypes, - ordering_req, + acc_args.ordering_req.to_vec(), ) .map(|acc| Box::new(acc) as _) } diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index df757ddc0422..a04e191d3f0e 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -27,7 +27,7 @@ use datafusion_common::{plan_err, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::format_state_name; use datafusion_expr::{Accumulator, AggregateUDFImpl, Signature, Volatility}; -use datafusion_physical_expr_common::aggregate::stats::StatsType; +use datafusion_functions_aggregate_common::stats::StatsType; use crate::variance::VarianceAccumulator; @@ -275,8 +275,8 @@ mod tests { use datafusion_common::DFSchema; use datafusion_expr::AggregateUDF; - use datafusion_physical_expr_common::aggregate::utils::get_accum_scalar_values_as_arrays; - use datafusion_physical_expr_common::expressions::column::col; + use datafusion_functions_aggregate_common::utils::get_accum_scalar_values_as_arrays; + use datafusion_physical_expr::expressions::col; use super::*; @@ -331,12 +331,13 @@ mod tests { schema, dfschema: &dfschema, ignore_nulls: false, - sort_exprs: &[], + ordering_req: &[], name: "a", is_distinct: false, is_reversed: false, input_types: &[DataType::Float64], - input_exprs: &[datafusion_expr::col("a")], + physical_exprs: &[col("a", schema)?], + // input_exprs: &[datafusion_expr::col("a")], }; let args2 = AccumulatorArgs { @@ -344,12 +345,14 @@ mod tests { schema, dfschema: &dfschema, ignore_nulls: false, - sort_exprs: &[], + // sort_exprs: &[], + ordering_req: &[], name: "a", is_distinct: false, is_reversed: false, input_types: &[DataType::Float64], - input_exprs: &[datafusion_expr::col("a")], + physical_exprs: &[col("a", schema)?], + // input_exprs: &[datafusion_expr::col("a")], }; let mut accum1 = agg1.accumulator(args1)?; diff --git a/datafusion/functions-aggregate/src/string_agg.rs b/datafusion/functions-aggregate/src/string_agg.rs index 5d91a52bc4c6..fa83337f217b 100644 --- a/datafusion/functions-aggregate/src/string_agg.rs +++ b/datafusion/functions-aggregate/src/string_agg.rs @@ -24,8 +24,9 @@ use datafusion_common::Result; use datafusion_common::{not_impl_err, ScalarValue}; use datafusion_expr::function::AccumulatorArgs; use datafusion_expr::{ - Accumulator, AggregateUDFImpl, Expr, Signature, TypeSignature, Volatility, + Accumulator, AggregateUDFImpl, Signature, TypeSignature, Volatility, }; +use datafusion_physical_expr::expressions::Literal; use std::any::Any; make_udaf_expr_and_func!( @@ -82,21 +83,39 @@ impl AggregateUDFImpl for StringAgg { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - match &acc_args.input_exprs[1] { - Expr::Literal(ScalarValue::Utf8(Some(delimiter))) - | Expr::Literal(ScalarValue::LargeUtf8(Some(delimiter))) => { - Ok(Box::new(StringAggAccumulator::new(delimiter))) - } - Expr::Literal(ScalarValue::Utf8(None)) - | Expr::Literal(ScalarValue::LargeUtf8(None)) - | Expr::Literal(ScalarValue::Null) => { - Ok(Box::new(StringAggAccumulator::new(""))) - } - _ => not_impl_err!( - "StringAgg not supported for delimiter {}", - &acc_args.input_exprs[1] - ), + if let Some(lit) = acc_args.physical_exprs[1] + .as_any() + .downcast_ref::() + { + return match lit.value() { + ScalarValue::Utf8(Some(delimiter)) + | ScalarValue::LargeUtf8(Some(delimiter)) => { + Ok(Box::new(StringAggAccumulator::new(delimiter.as_str()))) + } + ScalarValue::Utf8(None) + | ScalarValue::LargeUtf8(None) + | ScalarValue::Null => Ok(Box::new(StringAggAccumulator::new(""))), + e => not_impl_err!("StringAgg not supported for delimiter {}", e), + }; } + + not_impl_err!("expect literal") + + // match &acc_args.physical_exprs[1] { + // Expr::Literal(ScalarValue::Utf8(Some(delimiter))) + // | Expr::Literal(ScalarValue::LargeUtf8(Some(delimiter))) => { + // Ok(Box::new(StringAggAccumulator::new(delimiter))) + // } + // Expr::Literal(ScalarValue::Utf8(None)) + // | Expr::Literal(ScalarValue::LargeUtf8(None)) + // | Expr::Literal(ScalarValue::Null) => { + // Ok(Box::new(StringAggAccumulator::new(""))) + // } + // _ => not_impl_err!( + // "StringAgg not supported for delimiter {}", + // &acc_args.input_exprs[1] + // ), + // } } } diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index 08e3908a5829..40df5bf386ec 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -39,8 +39,8 @@ use datafusion_expr::utils::format_state_name; 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; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; +use datafusion_functions_aggregate_common::utils::Hashable; make_udaf_expr_and_func!( Sum, diff --git a/datafusion/functions-aggregate/src/variance.rs b/datafusion/functions-aggregate/src/variance.rs index c772608cb376..4c78a42ea494 100644 --- a/datafusion/functions-aggregate/src/variance.rs +++ b/datafusion/functions-aggregate/src/variance.rs @@ -34,7 +34,7 @@ use datafusion_expr::{ utils::format_state_name, Accumulator, AggregateUDFImpl, Signature, Volatility, }; -use datafusion_physical_expr_common::aggregate::stats::StatsType; +use datafusion_functions_aggregate_common::stats::StatsType; make_udaf_expr_and_func!( VarianceSample, diff --git a/datafusion/physical-expr-common/Cargo.toml b/datafusion/physical-expr-common/Cargo.toml index 3ef2d5345533..45ccb08e52e9 100644 --- a/datafusion/physical-expr-common/Cargo.toml +++ b/datafusion/physical-expr-common/Cargo.toml @@ -39,6 +39,6 @@ path = "src/lib.rs" ahash = { workspace = true } arrow = { workspace = true } datafusion-common = { workspace = true, default-features = true } -datafusion-expr = { workspace = true } +datafusion-expr-common = { workspace = true } hashbrown = { workspace = true } rand = { workspace = true } diff --git a/datafusion/physical-expr-common/src/binary_map.rs b/datafusion/physical-expr-common/src/binary_map.rs index edf608a2054f..d21bdb3434c4 100644 --- a/datafusion/physical-expr-common/src/binary_map.rs +++ b/datafusion/physical-expr-common/src/binary_map.rs @@ -60,7 +60,7 @@ impl ArrowBytesSet { /// Return the contents of this set and replace it with a new empty /// set with the same output type - pub(super) fn take(&mut self) -> Self { + pub fn take(&mut self) -> Self { Self(self.0.take()) } diff --git a/datafusion/physical-expr-common/src/datum.rs b/datafusion/physical-expr-common/src/datum.rs index d0ba5f113b6f..96c08d0d3a5b 100644 --- a/datafusion/physical-expr-common/src/datum.rs +++ b/datafusion/physical-expr-common/src/datum.rs @@ -22,7 +22,8 @@ use arrow::compute::SortOptions; use arrow::error::ArrowError; use datafusion_common::internal_err; use datafusion_common::{Result, ScalarValue}; -use datafusion_expr::{ColumnarValue, Operator}; +use datafusion_expr_common::columnar_value::ColumnarValue; +use datafusion_expr_common::operator::Operator; use std::sync::Arc; /// Applies a binary [`Datum`] kernel `f` to `lhs` and `rhs` diff --git a/datafusion/physical-expr-common/src/lib.rs b/datafusion/physical-expr-common/src/lib.rs index f03eedd4cf65..c186d16aa1ea 100644 --- a/datafusion/physical-expr-common/src/lib.rs +++ b/datafusion/physical-expr-common/src/lib.rs @@ -15,11 +15,9 @@ // specific language governing permissions and limitations // under the License. -pub mod aggregate; pub mod binary_map; pub mod binary_view_map; pub mod datum; -pub mod expressions; pub mod physical_expr; pub mod sort_expr; pub mod tree_node; diff --git a/datafusion/physical-expr-common/src/physical_expr.rs b/datafusion/physical-expr-common/src/physical_expr.rs index e62606a42e6f..75d300dd0107 100644 --- a/datafusion/physical-expr-common/src/physical_expr.rs +++ b/datafusion/physical-expr-common/src/physical_expr.rs @@ -20,18 +20,16 @@ use std::fmt::{Debug, Display}; use std::hash::{Hash, Hasher}; use std::sync::Arc; -use crate::expressions::column::Column; use crate::utils::scatter; use arrow::array::BooleanArray; use arrow::compute::filter_record_batch; -use arrow::datatypes::{DataType, Schema, SchemaRef}; +use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; -use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{internal_err, not_impl_err, plan_err, Result}; -use datafusion_expr::interval_arithmetic::Interval; -use datafusion_expr::sort_properties::ExprProperties; -use datafusion_expr::ColumnarValue; +use datafusion_common::{internal_err, not_impl_err, Result}; +use datafusion_expr_common::columnar_value::ColumnarValue; +use datafusion_expr_common::interval_arithmetic::Interval; +use datafusion_expr_common::sort_properties::ExprProperties; /// See [create_physical_expr](https://docs.rs/datafusion/latest/datafusion/physical_expr/fn.create_physical_expr.html) /// for examples of creating `PhysicalExpr` from `Expr` @@ -193,33 +191,6 @@ pub fn with_new_children_if_necessary( } } -/// Rewrites an expression according to new schema; i.e. changes the columns it -/// refers to with the column at corresponding index in the new schema. Returns -/// an error if the given schema has fewer columns than the original schema. -/// Note that the resulting expression may not be valid if data types in the -/// new schema is incompatible with expression nodes. -pub fn with_new_schema( - expr: Arc, - schema: &SchemaRef, -) -> Result> { - Ok(expr - .transform_up(|expr| { - if let Some(col) = expr.as_any().downcast_ref::() { - let idx = col.index(); - let Some(field) = schema.fields().get(idx) else { - return plan_err!( - "New schema has fewer columns than original schema" - ); - }; - let new_col = Column::new(field.name(), idx); - Ok(Transformed::yes(Arc::new(new_col) as _)) - } else { - Ok(Transformed::no(expr)) - } - })? - .data) -} - pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { if any.is::>() { any.downcast_ref::>() diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 2b506b74216f..9dc54d2eb2d0 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -22,13 +22,12 @@ use std::hash::{Hash, Hasher}; use std::sync::Arc; use crate::physical_expr::PhysicalExpr; -use crate::utils::limited_convert_logical_expr_to_physical_expr_with_dfschema; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; -use datafusion_common::{exec_err, DFSchema, Result}; -use datafusion_expr::{ColumnarValue, Expr}; +use datafusion_common::Result; +use datafusion_expr_common::columnar_value::ColumnarValue; /// Represents Sort operation for a column in a RecordBatch #[derive(Clone, Debug)] @@ -272,29 +271,3 @@ pub type LexRequirement = Vec; ///`LexRequirementRef` is an alias for the type &`[PhysicalSortRequirement]`, which /// represents a reference to a lexicographical ordering requirement. pub type LexRequirementRef<'a> = &'a [PhysicalSortRequirement]; - -/// Converts each [`Expr::Sort`] into a corresponding [`PhysicalSortExpr`]. -/// Returns an error if the given logical expression is not a [`Expr::Sort`]. -pub fn limited_convert_logical_sort_exprs_to_physical_with_dfschema( - exprs: &[Expr], - dfschema: &DFSchema, -) -> Result> { - // Construct PhysicalSortExpr objects from Expr objects: - let mut sort_exprs = vec![]; - for expr in exprs { - let Expr::Sort(sort) = expr else { - return exec_err!("Expects to receive sort expression"); - }; - sort_exprs.push(PhysicalSortExpr::new( - limited_convert_logical_expr_to_physical_expr_with_dfschema( - sort.expr.as_ref(), - dfschema, - )?, - SortOptions { - descending: !sort.asc, - nulls_first: sort.nulls_first, - }, - )) - } - Ok(sort_exprs) -} diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index 0978a906a5dc..86cbb3ea8e05 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -20,14 +20,12 @@ use std::sync::Arc; use arrow::array::{make_array, Array, ArrayRef, BooleanArray, MutableArrayData}; use arrow::compute::{and_kleene, is_not_null, SlicesIterator}; -use datafusion_common::{exec_err, DFSchema, Result}; -use datafusion_expr::expr::Alias; -use datafusion_expr::sort_properties::ExprProperties; -use datafusion_expr::Expr; - -use crate::expressions::column::Column; -use crate::expressions::literal::Literal; -use crate::expressions::CastExpr; +use datafusion_common::Result; +// use datafusion_expr::expr::Alias; +// use datafusion_expr::sort_properties::ExprProperties; +// use datafusion_expr::Expr; +use datafusion_expr_common::sort_properties::ExprProperties; + use crate::physical_expr::PhysicalExpr; use crate::sort_expr::PhysicalSortExpr; use crate::tree_node::ExprContext; @@ -108,35 +106,6 @@ pub fn reverse_order_bys(order_bys: &[PhysicalSortExpr]) -> Vec`. -/// If conversion is not supported yet, returns Error. -pub fn limited_convert_logical_expr_to_physical_expr_with_dfschema( - expr: &Expr, - dfschema: &DFSchema, -) -> Result> { - match expr { - Expr::Alias(Alias { expr, .. }) => Ok( - limited_convert_logical_expr_to_physical_expr_with_dfschema(expr, dfschema)?, - ), - Expr::Column(col) => { - let idx = dfschema.index_of_column(col)?; - Ok(Arc::new(Column::new(&col.name, idx))) - } - Expr::Cast(cast_expr) => Ok(Arc::new(CastExpr::new( - limited_convert_logical_expr_to_physical_expr_with_dfschema( - cast_expr.expr.as_ref(), - dfschema, - )?, - cast_expr.data_type.clone(), - None, - ))), - Expr::Literal(value) => Ok(Arc::new(Literal::new(value.clone()))), - _ => exec_err!( - "Unsupported expression: {expr} for conversion to Arc" - ), - } -} - #[cfg(test)] mod tests { use std::sync::Arc; diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index 8436b5279bd7..088cd5441789 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -56,7 +56,9 @@ chrono = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } +datafusion-expr-common = { workspace = true } datafusion-physical-expr-common = { workspace = true } +datafusion-functions-aggregate-common = { workspace = true } half = { workspace = true } hashbrown = { workspace = true } hex = { version = "0.4", optional = true } diff --git a/datafusion/physical-expr/benches/case_when.rs b/datafusion/physical-expr/benches/case_when.rs index 862edd9c1fac..b863b79b6a5c 100644 --- a/datafusion/physical-expr/benches/case_when.rs +++ b/datafusion/physical-expr/benches/case_when.rs @@ -23,8 +23,7 @@ use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_common::ScalarValue; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, CaseExpr}; -use datafusion_physical_expr_common::expressions::column::Column; -use datafusion_physical_expr_common::expressions::Literal; +use datafusion_physical_expr::expressions::{Column, Literal}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use std::sync::Arc; diff --git a/datafusion/physical-expr/benches/is_null.rs b/datafusion/physical-expr/benches/is_null.rs index 3dad8e9b456a..2880a3a62fb9 100644 --- a/datafusion/physical-expr/benches/is_null.rs +++ b/datafusion/physical-expr/benches/is_null.rs @@ -20,8 +20,8 @@ use arrow::record_batch::RecordBatch; use arrow_array::builder::Int32Builder; use arrow_schema::DataType; use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::expressions::{IsNotNullExpr, IsNullExpr}; -use datafusion_physical_expr_common::expressions::column::Column; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use std::sync::Arc; diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs b/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs index 3c0f3a28fedb..d2ea90ec3007 100644 --- a/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs +++ b/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs @@ -21,7 +21,7 @@ pub use adapter::GroupsAccumulatorAdapter; // Backward compatibility #[allow(unused_imports)] pub(crate) mod accumulate { - pub use datafusion_physical_expr_common::aggregate::groups_accumulator::accumulate::NullState; + pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::NullState; } -pub use datafusion_physical_expr_common::aggregate::groups_accumulator::accumulate::NullState; +pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::NullState; diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index 0760986a87c6..12f496eaadec 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -20,10 +20,10 @@ pub(crate) mod stats; pub mod moving_min_max; pub mod utils { - pub use datafusion_physical_expr_common::aggregate::utils::{ + pub use datafusion_functions_aggregate_common::utils::{ adjust_output_array, down_cast_any_ref, get_accum_scalar_values_as_arrays, get_sort_options, ordering_fields, DecimalAverager, Hashable, }; } -pub use datafusion_physical_expr_common::aggregate::AggregateExpr; +pub use datafusion_functions_aggregate_common::aggregate::AggregateExpr; diff --git a/datafusion/physical-expr/src/aggregate/stats.rs b/datafusion/physical-expr/src/aggregate/stats.rs index d9338f5a962f..db1934cd1ad9 100644 --- a/datafusion/physical-expr/src/aggregate/stats.rs +++ b/datafusion/physical-expr/src/aggregate/stats.rs @@ -15,4 +15,4 @@ // specific language governing permissions and limitations // under the License. -pub use datafusion_physical_expr_common::aggregate::stats::StatsType; +pub use datafusion_functions_aggregate_common::stats::StatsType; diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index a6e9fba28167..a5d54ee56cff 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -24,7 +24,7 @@ use crate::equivalence::{ collapse_lex_req, EquivalenceClass, EquivalenceGroup, OrderingEquivalenceClass, ProjectionMapping, }; -use crate::expressions::Literal; +use crate::expressions::{with_new_schema, CastExpr, Column, Literal}; use crate::{ physical_exprs_contains, ConstExpr, LexOrdering, LexOrderingRef, LexRequirement, LexRequirementRef, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, @@ -36,9 +36,6 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{plan_err, JoinSide, JoinType, Result}; use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; -use datafusion_physical_expr_common::expressions::column::Column; -use datafusion_physical_expr_common::expressions::CastExpr; -use datafusion_physical_expr_common::physical_expr::with_new_schema; use datafusion_physical_expr_common::utils::ExprPropertiesNode; use indexmap::{IndexMap, IndexSet}; diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index c34dcdfb7598..3896f89393ea 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -683,9 +683,9 @@ mod tests { use super::*; use crate::expressions::{col, lit, try_cast, Literal}; + use crate::expressions::Column; use datafusion_common::plan_datafusion_err; use datafusion_expr::type_coercion::binary::get_input_types; - use datafusion_physical_expr_common::expressions::column::Column; /// Performs a binary operation, applying any type coercion necessary fn binary_op( diff --git a/datafusion/physical-expr/src/expressions/case.rs b/datafusion/physical-expr/src/expressions/case.rs index b428d562bd1b..4a3fe7178a77 100644 --- a/datafusion/physical-expr/src/expressions/case.rs +++ b/datafusion/physical-expr/src/expressions/case.rs @@ -32,8 +32,8 @@ use datafusion_common::cast::as_boolean_array; use datafusion_common::{exec_err, internal_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::ColumnarValue; -use datafusion_physical_expr_common::expressions::column::Column; -use datafusion_physical_expr_common::expressions::Literal; +use super::Column; +use super::Literal; use itertools::Itertools; type WhenThen = (Arc, Arc); @@ -550,6 +550,7 @@ mod tests { use super::*; use crate::expressions::{binary, cast, col, lit, BinaryExpr}; + use super::Literal; use arrow::buffer::Buffer; use arrow::datatypes::DataType::Float64; use arrow::datatypes::*; @@ -558,7 +559,6 @@ mod tests { use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_expr::type_coercion::binary::comparison_coercion; use datafusion_expr::Operator; - use datafusion_physical_expr_common::expressions::Literal; #[test] fn case_with_expr() -> Result<()> { diff --git a/datafusion/physical-expr-common/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs similarity index 99% rename from datafusion/physical-expr-common/src/expressions/cast.rs rename to datafusion/physical-expr/src/expressions/cast.rs index dd6131ad65c3..343f210bcc67 100644 --- a/datafusion/physical-expr-common/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -27,9 +27,9 @@ use arrow::datatypes::{DataType, DataType::*, Schema}; use arrow::record_batch::RecordBatch; use datafusion_common::format::DEFAULT_FORMAT_OPTIONS; use datafusion_common::{not_impl_err, Result}; -use datafusion_expr::interval_arithmetic::Interval; -use datafusion_expr::sort_properties::ExprProperties; -use datafusion_expr::ColumnarValue; +use datafusion_expr_common::columnar_value::ColumnarValue; +use datafusion_expr_common::interval_arithmetic::Interval; +use datafusion_expr_common::sort_properties::ExprProperties; const DEFAULT_CAST_OPTIONS: CastOptions<'static> = CastOptions { safe: false, diff --git a/datafusion/physical-expr-common/src/expressions/column.rs b/datafusion/physical-expr/src/expressions/column.rs similarity index 83% rename from datafusion/physical-expr-common/src/expressions/column.rs rename to datafusion/physical-expr/src/expressions/column.rs index 5397599ea2dc..5a5fec499375 100644 --- a/datafusion/physical-expr-common/src/expressions/column.rs +++ b/datafusion/physical-expr/src/expressions/column.rs @@ -25,7 +25,9 @@ use arrow::{ datatypes::{DataType, Schema}, record_batch::RecordBatch, }; -use datafusion_common::{internal_err, Result}; +use arrow_schema::SchemaRef; +use datafusion_common::tree_node::{Transformed, TreeNode}; +use datafusion_common::{internal_err, plan_err, Result}; use datafusion_expr::ColumnarValue; use crate::physical_expr::{down_cast_any_ref, PhysicalExpr}; @@ -136,6 +138,35 @@ pub fn col(name: &str, schema: &Schema) -> Result> { Ok(Arc::new(Column::new_with_schema(name, schema)?)) } +// TODO: Move expressions out of physical-expr? + +/// Rewrites an expression according to new schema; i.e. changes the columns it +/// refers to with the column at corresponding index in the new schema. Returns +/// an error if the given schema has fewer columns than the original schema. +/// Note that the resulting expression may not be valid if data types in the +/// new schema is incompatible with expression nodes. +pub fn with_new_schema( + expr: Arc, + schema: &SchemaRef, +) -> Result> { + Ok(expr + .transform_up(|expr| { + if let Some(col) = expr.as_any().downcast_ref::() { + let idx = col.index(); + let Some(field) = schema.fields().get(idx) else { + return plan_err!( + "New schema has fewer columns than original schema" + ); + }; + let new_col = Column::new(field.name(), idx); + Ok(Transformed::yes(Arc::new(new_col) as _)) + } else { + Ok(Transformed::no(expr)) + } + })? + .data) +} + #[cfg(test)] mod test { use super::Column; diff --git a/datafusion/physical-expr-common/src/expressions/literal.rs b/datafusion/physical-expr/src/expressions/literal.rs similarity index 95% rename from datafusion/physical-expr-common/src/expressions/literal.rs rename to datafusion/physical-expr/src/expressions/literal.rs index b3cff1ef69ba..ed24e9028153 100644 --- a/datafusion/physical-expr-common/src/expressions/literal.rs +++ b/datafusion/physical-expr/src/expressions/literal.rs @@ -28,9 +28,10 @@ use arrow::{ record_batch::RecordBatch, }; use datafusion_common::{Result, ScalarValue}; -use datafusion_expr::interval_arithmetic::Interval; -use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; -use datafusion_expr::{ColumnarValue, Expr}; +use datafusion_expr::Expr; +use datafusion_expr_common::columnar_value::ColumnarValue; +use datafusion_expr_common::interval_arithmetic::Interval; +use datafusion_expr_common::sort_properties::{ExprProperties, SortProperties}; /// Represents a literal value #[derive(Debug, PartialEq, Eq, Hash)] diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index cbb697b5f304..9e65889d8758 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -20,10 +20,13 @@ #[macro_use] mod binary; mod case; +mod cast; +mod column; mod in_list; mod is_not_null; mod is_null; mod like; +mod literal; mod negative; mod no_op; mod not; @@ -42,14 +45,14 @@ pub use crate::PhysicalSortExpr; pub use binary::{binary, BinaryExpr}; pub use case::{case, CaseExpr}; +pub use cast::{cast, CastExpr}; +pub use column::{col, with_new_schema, Column}; pub use datafusion_expr::utils::format_state_name; -pub use datafusion_physical_expr_common::expressions::column::{col, Column}; -pub use datafusion_physical_expr_common::expressions::literal::{lit, Literal}; -pub use datafusion_physical_expr_common::expressions::{cast, CastExpr}; pub use in_list::{in_list, InListExpr}; pub use is_not_null::{is_not_null, IsNotNullExpr}; pub use is_null::{is_null, IsNullExpr}; pub use like::{like, LikeExpr}; +pub use literal::{lit, Literal}; pub use negative::{negative, NegativeExpr}; pub use no_op::NoOp; pub use not::{not, NotExpr}; diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 2e78119eba46..3d2ae2651bfc 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -45,7 +45,7 @@ pub mod execution_props { pub use aggregate::groups_accumulator::{GroupsAccumulatorAdapter, NullState}; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; -pub use datafusion_physical_expr_common::aggregate::{ +pub use datafusion_functions_aggregate_common::aggregate::{ AggregateExpr, AggregatePhysicalExpressions, }; pub use equivalence::{calculate_union, ConstExpr, EquivalenceProperties}; diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index c60a772b9ce2..c718e6b054ef 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -17,7 +17,7 @@ use std::sync::Arc; -use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +pub(crate) use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use itertools::izip; pub use datafusion_physical_expr_common::physical_expr::down_cast_any_ref; diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index d3f66bdea93d..102c66ab99bd 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -51,7 +51,9 @@ datafusion-common = { workspace = true, default-features = true } datafusion-common-runtime = { workspace = true, default-features = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } +datafusion-expr-functions-aggregate = { workspace = true } datafusion-functions-aggregate = { workspace = true } +datafusion-functions-aggregate-common = { workspace = true } datafusion-physical-expr = { workspace = true, default-features = true } datafusion-physical-expr-common = { workspace = true } futures = { workspace = true } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 8941418c12e1..bbc139460074 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1219,10 +1219,10 @@ mod tests { use datafusion_physical_expr::PhysicalSortExpr; use crate::common::collect; - use datafusion_physical_expr_common::aggregate::{ + use datafusion_expr_functions_aggregate::aggregate::{ create_aggregate_expr_with_dfschema, AggregateExprBuilder, }; - use datafusion_physical_expr_common::expressions::Literal; + use datafusion_physical_expr::expressions::Literal; use futures::{FutureExt, Stream}; // Generate a schema which consists of 5 columns (a, b, c, d, e) @@ -1995,11 +1995,9 @@ mod tests { let args = vec![col("b", schema)?]; let logical_args = vec![datafusion_expr::col("b")]; let func = datafusion_expr::AggregateUDF::new_from_impl(FirstValue::new()); - datafusion_physical_expr_common::aggregate::create_aggregate_expr_with_dfschema( + datafusion_expr_functions_aggregate::aggregate::create_aggregate_expr_with_dfschema( &func, &args, - &logical_args, - &sort_exprs, &ordering_req, dfschema, "FIRST_VALUE(b)", @@ -2030,8 +2028,6 @@ mod tests { create_aggregate_expr_with_dfschema( &func, &args, - &logical_args, - &sort_exprs, &ordering_req, dfschema, "LAST_VALUE(b)", @@ -2263,8 +2259,6 @@ mod tests { create_aggregate_expr_with_dfschema( &array_agg_udaf(), &[Arc::clone(col_a)], - &[], - &sort_exprs, &ordering_req, &test_df_schema, "array_agg", @@ -2423,8 +2417,6 @@ mod tests { vec![create_aggregate_expr_with_dfschema( &count_udaf(), &[col("val", &schema)?], - &[datafusion_expr::col("val")], - &[], &[], &df_schema, "COUNT(val)", @@ -2511,8 +2503,6 @@ mod tests { vec![create_aggregate_expr_with_dfschema( &count_udaf(), &[col("val", &schema)?], - &[datafusion_expr::col("val")], - &[], &[], &df_schema, "COUNT(val)", diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index eeecc017c2af..4c11b8d1a3dd 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -82,7 +82,7 @@ pub mod windows; pub mod work_table; pub mod udaf { - pub use datafusion_physical_expr_common::aggregate::{ + pub use datafusion_expr_functions_aggregate::aggregate::{ create_aggregate_expr, create_aggregate_expr_with_dfschema, AggregateFunctionExpr, }; } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 9321fdb2cadf..9ef29c833dcc 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -586,8 +586,8 @@ mod tests { use arrow_schema::{DataType, SortOptions}; use datafusion_common::ScalarValue; + use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; - use datafusion_physical_expr_common::expressions::column::col; // Generate a schema which consists of 7 columns (a, b, c, d, e, f, g) fn create_test_schema() -> Result { diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 65cef28efc45..c6a0af09738c 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -36,6 +36,7 @@ use datafusion_expr::{ BuiltInWindowFunction, PartitionEvaluator, WindowFrame, WindowFunctionDefinition, WindowUDF, }; +use datafusion_expr_functions_aggregate::aggregate::AggregateExprBuilder; use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ reverse_order_bys, @@ -43,7 +44,6 @@ use datafusion_physical_expr::{ AggregateExpr, ConstExpr, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, }; -use datafusion_physical_expr_common::aggregate::AggregateExprBuilder; use itertools::Itertools; mod bounded_window_agg_exec; @@ -130,7 +130,7 @@ pub fn create_window_expr( .schema(Arc::new(input_schema.clone())) .name(name) .order_by(order_by.to_vec()) - .sort_exprs(sort_exprs) + // .sort_exprs(sort_exprs) .with_ignore_nulls(ignore_nulls) .build()?; window_expr_from_aggregate_expr( @@ -412,7 +412,7 @@ pub(crate) fn calc_requirements< let PhysicalSortExpr { expr, options } = element.borrow(); if !sort_reqs.iter().any(|e| e.expr.eq(expr)) { sort_reqs.push(PhysicalSortRequirement::new( - Arc::clone(expr), + Arc::clone(&expr), Some(*options), )); } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index aefa1d87a278..9ce4eb5bfaca 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -18,7 +18,7 @@ use std::fmt::Debug; use std::sync::Arc; -use datafusion::physical_expr_common::aggregate::AggregateExprBuilder; +use datafusion::expr_functions_aggregate::aggregate::AggregateExprBuilder; use prost::bytes::BufMut; use prost::Message; diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 0e2bc9cbb3e2..4177391d26c7 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -24,7 +24,7 @@ use std::vec; use arrow::array::RecordBatch; use arrow::csv::WriterBuilder; -use datafusion::physical_expr_common::aggregate::AggregateExprBuilder; +use datafusion::expr_functions_aggregate::aggregate::AggregateExprBuilder; use datafusion_functions_aggregate::min_max::max_udaf; use prost::Message; From 51350d4ee5b3fd4205adfa3384962ae83c7f6b2f Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 6 Aug 2024 16:33:45 +0800 Subject: [PATCH 02/26] production ready Signed-off-by: jayzhan211 --- datafusion-cli/Cargo.lock | 147 +++++++++++++----- datafusion/core/src/physical_planner.rs | 1 - .../expr-functions-aggregate/Cargo.toml | 4 +- .../expr-functions-aggregate/src/aggregate.rs | 61 +------- .../expr-functions-aggregate/src/lib.rs | 2 +- .../functions-aggregate-common/Cargo.toml | 2 +- .../functions-aggregate-common/src/order.rs | 17 ++ datafusion/functions-aggregate/Cargo.toml | 4 +- .../src/approx_percentile_cont.rs | 40 +---- datafusion/physical-expr/Cargo.toml | 2 +- .../physical-expr/src/expressions/cast.rs | 4 +- .../physical-expr/src/expressions/column.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 59 +------ datafusion/physical-plan/src/windows/mod.rs | 22 +-- .../tests/cases/roundtrip_logical_plan.rs | 4 +- 15 files changed, 146 insertions(+), 225 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 5884e424c781..c28644842bf3 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -272,7 +272,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap 2.2.6", + "indexmap 2.3.0", "lexical-core", "num", "serde", @@ -375,7 +375,7 @@ dependencies = [ "tokio", "xz2", "zstd 0.13.2", - "zstd-safe 7.2.0", + "zstd-safe 7.2.1", ] [[package]] @@ -837,9 +837,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.6.1" +version = "1.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a12916984aab3fa6e39d655a33e09c0071eb36d6ab3aea5c2d78551f1df6d952" +checksum = "8318a53db07bb3f8dca91a600466bdb3f2eaadeedfdbcf02e1accbad9271ba50" [[package]] name = "bytes-utils" @@ -874,9 +874,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.1.6" +version = "1.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2aba8f4e9906c7ce3c73463f62a7f0c65183ada1a2d47e397cc8810827f9694f" +checksum = "26a5c3fd7bfa1ce3897a3a3501d362b2d87b7f2583ebcb4a949ec25911025cbc" dependencies = [ "jobserver", "libc", @@ -1147,6 +1147,7 @@ dependencies = [ "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", + "datafusion-expr-functions-aggregate", "datafusion-functions", "datafusion-functions-aggregate", "datafusion-functions-nested", @@ -1161,7 +1162,7 @@ dependencies = [ "glob", "half", "hashbrown 0.14.5", - "indexmap 2.2.6", + "indexmap 2.3.0", "itertools 0.12.1", "log", "num-traits", @@ -1278,6 +1279,9 @@ dependencies = [ "arrow-buffer", "chrono", "datafusion-common", + "datafusion-expr-common", + "datafusion-functions-aggregate-common", + "datafusion-physical-expr-common", "paste", "serde_json", "sqlparser", @@ -1285,6 +1289,29 @@ dependencies = [ "strum_macros 0.26.4", ] +[[package]] +name = "datafusion-expr-common" +version = "40.0.0" +dependencies = [ + "arrow", + "datafusion-common", + "paste", +] + +[[package]] +name = "datafusion-expr-functions-aggregate" +version = "40.0.0" +dependencies = [ + "ahash", + "arrow", + "datafusion-common", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions-aggregate-common", + "datafusion-physical-expr-common", + "rand", +] + [[package]] name = "datafusion-functions" version = "40.0.0" @@ -1320,12 +1347,26 @@ dependencies = [ "datafusion-common", "datafusion-execution", "datafusion-expr", + "datafusion-functions-aggregate-common", + "datafusion-physical-expr", "datafusion-physical-expr-common", "log", "paste", "sqlparser", ] +[[package]] +name = "datafusion-functions-aggregate-common" +version = "40.0.0" +dependencies = [ + "ahash", + "arrow", + "datafusion-common", + "datafusion-expr-common", + "datafusion-physical-expr-common", + "rand", +] + [[package]] name = "datafusion-functions-nested" version = "40.0.0" @@ -1357,7 +1398,7 @@ dependencies = [ "datafusion-expr", "datafusion-physical-expr", "hashbrown 0.14.5", - "indexmap 2.2.6", + "indexmap 2.3.0", "itertools 0.12.1", "log", "paste", @@ -1380,11 +1421,13 @@ dependencies = [ "datafusion-common", "datafusion-execution", "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions-aggregate-common", "datafusion-physical-expr-common", "half", "hashbrown 0.14.5", "hex", - "indexmap 2.2.6", + "indexmap 2.3.0", "itertools 0.12.1", "log", "paste", @@ -1399,7 +1442,7 @@ dependencies = [ "ahash", "arrow", "datafusion-common", - "datafusion-expr", + "datafusion-expr-common", "hashbrown 0.14.5", "rand", ] @@ -1430,13 +1473,15 @@ dependencies = [ "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", + "datafusion-expr-functions-aggregate", "datafusion-functions-aggregate", + "datafusion-functions-aggregate-common", "datafusion-physical-expr", "datafusion-physical-expr-common", "futures", "half", "hashbrown 0.14.5", - "indexmap 2.2.6", + "indexmap 2.3.0", "itertools 0.12.1", "log", "once_cell", @@ -1629,9 +1674,9 @@ dependencies = [ [[package]] name = "flate2" -version = "1.0.30" +version = "1.0.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f54427cfd1c7829e2a139fcefea601bf088ebca651d2bf53ebc600eac295dae" +checksum = "7f211bbe8e69bbd0cfdea405084f128ae8b4aaa6b0b522fc8f2b009084797920" dependencies = [ "crc32fast", "miniz_oxide", @@ -1801,7 +1846,7 @@ dependencies = [ "futures-sink", "futures-util", "http 0.2.12", - "indexmap 2.2.6", + "indexmap 2.3.0", "slab", "tokio", "tokio-util", @@ -1820,7 +1865,7 @@ dependencies = [ "futures-core", "futures-sink", "http 1.1.0", - "indexmap 2.2.6", + "indexmap 2.3.0", "slab", "tokio", "tokio-util", @@ -2112,9 +2157,9 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.2.6" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "168fb715dda47215e360912c096649d23d58bf392ac62f73919e831745e40f26" +checksum = "de3fc2e30ba82dd1b3911c8de1ffc143c74a914a14e99514d7637e3099df5ea0" dependencies = [ "equivalent", "hashbrown 0.14.5", @@ -2552,7 +2597,7 @@ dependencies = [ "rand", "reqwest", "ring 0.17.8", - "rustls-pemfile 2.1.2", + "rustls-pemfile 2.1.3", "serde", "serde_json", "snafu", @@ -2682,7 +2727,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b4c5cc86750666a3ed20bdaf5ca2a0344f9c67674cae0515bec2da16fbaa47db" dependencies = [ "fixedbitset", - "indexmap 2.2.6", + "indexmap 2.3.0", ] [[package]] @@ -2769,9 +2814,12 @@ checksum = "439ee305def115ba05938db6eb1644ff94165c5ab5e9420d1c1bcedbba909391" [[package]] name = "ppv-lite86" -version = "0.2.17" +version = "0.2.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" +checksum = "77957b295656769bb8ad2b6a6b09d897d94f05c41b069aede1fcdaa675eaea04" +dependencies = [ + "zerocopy", +] [[package]] name = "predicates" @@ -2854,9 +2902,9 @@ dependencies = [ [[package]] name = "quinn" -version = "0.11.2" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4ceeeeabace7857413798eb1ffa1e9c905a9946a57d81fb69b4b71c4d8eb3ad" +checksum = "b22d8e7369034b9a7132bc2008cac12f2013c8132b45e0554e6e20e2617f2156" dependencies = [ "bytes", "pin-project-lite", @@ -2864,6 +2912,7 @@ dependencies = [ "quinn-udp", "rustc-hash", "rustls 0.23.12", + "socket2", "thiserror", "tokio", "tracing", @@ -2871,9 +2920,9 @@ dependencies = [ [[package]] name = "quinn-proto" -version = "0.11.3" +version = "0.11.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ddf517c03a109db8100448a4be38d498df8a210a99fe0e1b9eaf39e78c640efe" +checksum = "ba92fb39ec7ad06ca2582c0ca834dfeadcaf06ddfc8e635c80aa7e1c05315fdd" dependencies = [ "bytes", "rand", @@ -2895,6 +2944,7 @@ dependencies = [ "libc", "once_cell", "socket2", + "tracing", "windows-sys 0.52.0", ] @@ -2969,9 +3019,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.10.5" +version = "1.10.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b91213439dad192326a0d7c6ee3955910425f441d7038e0d6933b0aec5c4517f" +checksum = "4219d74c6b67a3654a9fbebc4b419e22126d13d2f3c4a07ee0cb61ff79a79619" dependencies = [ "aho-corasick", "memchr", @@ -3029,7 +3079,7 @@ dependencies = [ "quinn", "rustls 0.23.12", "rustls-native-certs 0.7.1", - "rustls-pemfile 2.1.2", + "rustls-pemfile 2.1.3", "rustls-pki-types", "serde", "serde_json", @@ -3117,9 +3167,9 @@ checksum = "719b953e2095829ee67db738b3bfa9fa368c94900df327b3f07fe6e794d2fe1f" [[package]] name = "rustc-hash" -version = "1.1.0" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" +checksum = "583034fd73374156e66797ed8e5b0d5690409c9226b22d87cb7f19821c05d152" [[package]] name = "rustc_version" @@ -3188,7 +3238,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a88d6d420651b496bdd98684116959239430022a115c1240e6c3993be0b15fba" dependencies = [ "openssl-probe", - "rustls-pemfile 2.1.2", + "rustls-pemfile 2.1.3", "rustls-pki-types", "schannel", "security-framework", @@ -3205,9 +3255,9 @@ dependencies = [ [[package]] name = "rustls-pemfile" -version = "2.1.2" +version = "2.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29993a25686778eb88d4189742cd713c9bce943bc54251a33509dc63cbacf73d" +checksum = "196fe16b00e106300d3e45ecfcb764fa292a535d7326a29a5875c579c7417425" dependencies = [ "base64 0.22.1", "rustls-pki-types", @@ -3356,9 +3406,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.121" +version = "1.0.122" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ab380d7d9f22ef3f21ad3e6c1ebe8e4fc7a2000ccba2e4d71fc96f15b2cb609" +checksum = "784b6203951c57ff748476b126ccb5e8e2959a5c19e5c617ab1956be3dbc68da" dependencies = [ "itoa", "memchr", @@ -3585,12 +3635,13 @@ checksum = "a7065abeca94b6a8a577f9bd45aa0867a2238b74e8eb67cf10d492bc39351394" [[package]] name = "tempfile" -version = "3.10.1" +version = "3.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85b77fafb263dd9d05cbeac119526425676db3784113aa9295c88498cbf8bff1" +checksum = "b8fcd239983515c23a32fb82099f97d0b11b8c72f654ed659363a95c3dad7a53" dependencies = [ "cfg-if", "fastrand 2.1.0", + "once_cell", "rustix", "windows-sys 0.52.0", ] @@ -4119,11 +4170,11 @@ checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" [[package]] name = "winapi-util" -version = "0.1.8" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d4cc384e1e73b93bafa6fb4f1df8c41695c8a91cf9c4c64358067d15a7b6c6b" +checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -4159,6 +4210,15 @@ dependencies = [ "windows-targets 0.52.6", ] +[[package]] +name = "windows-sys" +version = "0.59.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e38bc4d79ed67fd075bcc251a1c39b32a1776bbe92e5bef1f0bf1f8c531853b" +dependencies = [ + "windows-targets 0.52.6", +] + [[package]] name = "windows-targets" version = "0.48.5" @@ -4311,6 +4371,7 @@ version = "0.7.35" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1b9b4fd18abc82b8136838da5d50bae7bdea537c574d8dc1a34ed098d6c166f0" dependencies = [ + "byteorder", "zerocopy-derive", ] @@ -4346,7 +4407,7 @@ version = "0.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fcf2b778a664581e31e389454a7072dab1647606d44f7feea22cd5abb9c9f3f9" dependencies = [ - "zstd-safe 7.2.0", + "zstd-safe 7.2.1", ] [[package]] @@ -4361,9 +4422,9 @@ dependencies = [ [[package]] name = "zstd-safe" -version = "7.2.0" +version = "7.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa556e971e7b568dc775c136fc9de8c779b1c2fc3a63defaafadffdbd3181afa" +checksum = "54a3ab4db68cea366acc5c897c7b4d4d1b8994a9cd6e6f841f8964566a419059" dependencies = [ "zstd-sys", ] diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index fb1c29d2d633..16e413021291 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1837,7 +1837,6 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( == NullTreatment::IgnoreNulls; let (agg_expr, filter, order_by) = { - let sort_exprs = order_by.clone().unwrap_or(vec![]); let physical_sort_exprs = match order_by { Some(exprs) => Some(create_physical_sort_exprs( exprs, diff --git a/datafusion/expr-functions-aggregate/Cargo.toml b/datafusion/expr-functions-aggregate/Cargo.toml index 5e20ac651f58..2792989d9afe 100644 --- a/datafusion/expr-functions-aggregate/Cargo.toml +++ b/datafusion/expr-functions-aggregate/Cargo.toml @@ -41,10 +41,10 @@ path = "src/lib.rs" ahash = { workspace = true } arrow = { workspace = true } datafusion-common = { workspace = true } -datafusion-expr-common = { workspace = true } datafusion-expr = { workspace = true } -datafusion-physical-expr-common = { workspace = true } +datafusion-expr-common = { workspace = true } datafusion-functions-aggregate-common = { workspace = true } +datafusion-physical-expr-common = { workspace = true } rand = { workspace = true } # strum = { version = "0.26.1", features = ["derive"] } # strum_macros = "0.26.0" diff --git a/datafusion/expr-functions-aggregate/src/aggregate.rs b/datafusion/expr-functions-aggregate/src/aggregate.rs index e657c6e64c67..7e107ad85ff7 100644 --- a/datafusion/expr-functions-aggregate/src/aggregate.rs +++ b/datafusion/expr-functions-aggregate/src/aggregate.rs @@ -18,15 +18,13 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::{internal_err, not_impl_err, DFSchema, Result}; use datafusion_expr::function::StateFieldsArgs; -use datafusion_expr::utils::AggregateOrderSensitivity; use datafusion_expr::ReversedUDAF; -use datafusion_expr::{function::AccumulatorArgs, Accumulator, AggregateUDF, Expr}; - +use datafusion_expr::{function::AccumulatorArgs, AggregateUDF}; +use datafusion_expr_common::accumulator::Accumulator; +use datafusion_expr_common::groups_accumulator::GroupsAccumulator; use datafusion_expr_common::type_coercion::aggregates::check_arg_count; use datafusion_functions_aggregate_common::aggregate::AggregateExpr; - -use datafusion_expr_common::groups_accumulator::GroupsAccumulator; - +use datafusion_functions_aggregate_common::order::AggregateOrderSensitivity; use datafusion_functions_aggregate_common::utils::{self, down_cast_any_ref}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; @@ -55,8 +53,6 @@ use std::{any::Any, sync::Arc}; pub fn create_aggregate_expr( fun: &AggregateUDF, input_phy_exprs: &[Arc], - input_exprs: &[Expr], - sort_exprs: &[Expr], ordering_req: &[PhysicalSortExpr], schema: &Schema, name: impl Into, @@ -65,9 +61,7 @@ pub fn create_aggregate_expr( ) -> Result> { let mut builder = AggregateExprBuilder::new(Arc::new(fun.clone()), input_phy_exprs.to_vec()); - // builder = builder.sort_exprs(sort_exprs.to_vec()); builder = builder.order_by(ordering_req.to_vec()); - builder = builder.logical_exprs(input_exprs.to_vec()); builder = builder.schema(Arc::new(schema.clone())); builder = builder.name(name); @@ -86,8 +80,6 @@ pub fn create_aggregate_expr( pub fn create_aggregate_expr_with_dfschema( fun: &AggregateUDF, input_phy_exprs: &[Arc], - // input_exprs: &[Expr], - // sort_exprs: &[Expr], ordering_req: &[PhysicalSortExpr], dfschema: &DFSchema, name: impl Into, @@ -97,9 +89,7 @@ pub fn create_aggregate_expr_with_dfschema( ) -> Result> { let mut builder = AggregateExprBuilder::new(Arc::new(fun.clone()), input_phy_exprs.to_vec()); - // builder = builder.sort_exprs(sort_exprs.to_vec()); builder = builder.order_by(ordering_req.to_vec()); - // builder = builder.logical_exprs(input_exprs.to_vec()); builder = builder.dfschema(dfschema.clone()); let schema: Schema = dfschema.into(); builder = builder.schema(Arc::new(schema)); @@ -127,15 +117,11 @@ pub struct AggregateExprBuilder { fun: Arc, /// Physical expressions of the aggregate function args: Vec>, - /// Logical expressions of the aggregate function, it will be deprecated in - logical_args: Vec, name: String, /// Arrow Schema for the aggregate function schema: SchemaRef, /// Datafusion Schema for the aggregate function dfschema: DFSchema, - /// The logical order by expressions, it will be deprecated in - sort_exprs: Vec, /// The physical order by expressions ordering_req: LexOrdering, /// Whether to ignore null values @@ -151,11 +137,9 @@ impl AggregateExprBuilder { Self { fun, args, - logical_args: vec![], name: String::new(), schema: Arc::new(Schema::empty()), dfschema: DFSchema::empty(), - sort_exprs: vec![], ordering_req: vec![], ignore_nulls: false, is_distinct: false, @@ -167,11 +151,9 @@ impl AggregateExprBuilder { let Self { fun, args, - logical_args, name, schema, dfschema, - sort_exprs, ordering_req, ignore_nulls, is_distinct, @@ -208,12 +190,10 @@ impl AggregateExprBuilder { Ok(Arc::new(AggregateFunctionExpr { fun: Arc::unwrap_or_clone(fun), args, - // logical_args, data_type, name, schema: Arc::unwrap_or_clone(schema), dfschema, - // sort_exprs, ordering_req, ignore_nulls, ordering_fields, @@ -272,18 +252,6 @@ impl AggregateExprBuilder { self.ignore_nulls = ignore_nulls; self } - - /// This method will be deprecated in - // pub fn sort_exprs(mut self, sort_exprs: Vec) -> Self { - // self.sort_exprs = sort_exprs; - // self - // } - - /// This method will be deprecated in - pub fn logical_exprs(mut self, logical_args: Vec) -> Self { - self.logical_args = logical_args; - self - } } /// Physical aggregate expression of a UDAF. @@ -361,11 +329,9 @@ impl AggregateExpr for AggregateFunctionExpr { schema: &self.schema, dfschema: &self.dfschema, ignore_nulls: self.ignore_nulls, - // sort_exprs: &self.sort_exprs, ordering_req: &self.ordering_req, is_distinct: self.is_distinct, input_types: &self.input_types, - // input_exprs: &self.logical_args, name: &self.name, is_reversed: self.is_reversed, physical_exprs: &self.args, @@ -383,7 +349,6 @@ impl AggregateExpr for AggregateFunctionExpr { ordering_req: &self.ordering_req, is_distinct: self.is_distinct, input_types: &self.input_types, - // input_exprs: &self.logical_args, name: &self.name, is_reversed: self.is_reversed, physical_exprs: &self.args, @@ -456,7 +421,6 @@ impl AggregateExpr for AggregateFunctionExpr { ordering_req: &self.ordering_req, is_distinct: self.is_distinct, input_types: &self.input_types, - // input_exprs: &self.logical_args, name: &self.name, is_reversed: self.is_reversed, physical_exprs: &self.args, @@ -516,15 +480,12 @@ impl AggregateExpr for AggregateFunctionExpr { create_aggregate_expr_with_dfschema( &updated_fn, &self.args, - // &self.logical_args, - // &self.sort_exprs, &self.ordering_req, &self.dfschema, self.name(), self.ignore_nulls, self.is_distinct, self.is_reversed, - // physical_exprs: &self.args, ) .map(Some) } @@ -535,18 +496,6 @@ impl AggregateExpr for AggregateFunctionExpr { 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(); // If the function is changed, we need to reverse order_by clause as well // i.e. First(a order by b asc null first) -> Last(a order by b desc null last) @@ -558,8 +507,6 @@ impl AggregateExpr for AggregateFunctionExpr { let reverse_aggr = create_aggregate_expr_with_dfschema( &reverse_udf, &self.args, - // &self.logical_args, - // &reverse_sort_exprs, &reverse_ordering_req, &self.dfschema, name, diff --git a/datafusion/expr-functions-aggregate/src/lib.rs b/datafusion/expr-functions-aggregate/src/lib.rs index 1e14a97af3b6..2ff7ff5777ec 100644 --- a/datafusion/expr-functions-aggregate/src/lib.rs +++ b/datafusion/expr-functions-aggregate/src/lib.rs @@ -15,6 +15,6 @@ // specific language governing permissions and limitations // under the License. -//! Technically, those aggregate functions releated things that has depedency on `expr` should be here +//! Technically, all aggregate functions that depend on `expr` crate should be included here. pub mod aggregate; diff --git a/datafusion/functions-aggregate-common/Cargo.toml b/datafusion/functions-aggregate-common/Cargo.toml index 2c1119b82a09..8473bc14c55c 100644 --- a/datafusion/functions-aggregate-common/Cargo.toml +++ b/datafusion/functions-aggregate-common/Cargo.toml @@ -46,4 +46,4 @@ datafusion-expr-common = { workspace = true } datafusion-physical-expr-common = { workspace = true } rand = { workspace = true } # strum = { version = "0.26.1", features = ["derive"] } -# strum_macros = "0.26.0" \ No newline at end of file +# strum_macros = "0.26.0" diff --git a/datafusion/functions-aggregate-common/src/order.rs b/datafusion/functions-aggregate-common/src/order.rs index 4d7e604687b8..bfa6e39138f9 100644 --- a/datafusion/functions-aggregate-common/src/order.rs +++ b/datafusion/functions-aggregate-common/src/order.rs @@ -1,3 +1,20 @@ +// 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. + /// Represents the sensitivity of an aggregate expression to ordering. #[derive(Debug, PartialEq, Eq, Clone, Copy)] pub enum AggregateOrderSensitivity { diff --git a/datafusion/functions-aggregate/Cargo.toml b/datafusion/functions-aggregate/Cargo.toml index 9e7476e81d05..0006f2578145 100644 --- a/datafusion/functions-aggregate/Cargo.toml +++ b/datafusion/functions-aggregate/Cargo.toml @@ -44,9 +44,9 @@ arrow-schema = { workspace = true } datafusion-common = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } -datafusion-physical-expr-common = { workspace = true } -datafusion-physical-expr = { workspace = true } datafusion-functions-aggregate-common = { workspace = true } +datafusion-physical-expr = { workspace = true } +datafusion-physical-expr-common = { workspace = true } log = { workspace = true } paste = "1.0.14" sqlparser = { workspace = true } diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index 1ce79df3aa78..5c7d06d2bccd 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -31,21 +31,17 @@ use arrow::{ use arrow_schema::{Field, Schema}; use datafusion_common::{ - downcast_value, exec_err, internal_err, not_impl_err, plan_err, DFSchema, - DataFusionError, Result, ScalarValue, + downcast_value, internal_err, not_impl_err, plan_err, DataFusionError, ScalarValue, }; -use datafusion_expr::expr::Alias; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::{INTEGERS, NUMERICS}; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ - Accumulator, AggregateUDFImpl, ColumnarValue, Expr, Signature, TypeSignature, - Volatility, + Accumulator, AggregateUDFImpl, ColumnarValue, Signature, TypeSignature, Volatility, }; use datafusion_functions_aggregate_common::tdigest::{ TDigest, TryIntoF64, DEFAULT_MAX_SIZE, }; -use datafusion_physical_expr::expressions::{CastExpr, Column, Literal}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; make_udaf_expr_and_func!( @@ -136,42 +132,10 @@ impl ApproxPercentileCont { } } -/// Converts `datafusion_expr::Expr` into corresponding `Arc`. -/// If conversion is not supported yet, returns Error. -fn limited_convert_logical_expr_to_physical_expr_with_dfschema( - expr: &Expr, - dfschema: &DFSchema, -) -> Result> { - match expr { - Expr::Alias(Alias { expr, .. }) => Ok( - limited_convert_logical_expr_to_physical_expr_with_dfschema(expr, dfschema)?, - ), - Expr::Column(col) => { - let idx = dfschema.index_of_column(col)?; - Ok(Arc::new(Column::new(&col.name, idx))) - } - Expr::Cast(cast_expr) => Ok(Arc::new(CastExpr::new( - limited_convert_logical_expr_to_physical_expr_with_dfschema( - cast_expr.expr.as_ref(), - dfschema, - )?, - cast_expr.data_type.clone(), - None, - ))), - Expr::Literal(value) => Ok(Arc::new(Literal::new(value.clone()))), - _ => exec_err!( - "Unsupported expression: {expr} for conversion to Arc" - ), - } -} - fn get_lit_value(expr: &Arc) -> datafusion_common::Result { // TODO: use real schema let empty_schema = Arc::new(Schema::empty()); let empty_batch = RecordBatch::new_empty(Arc::clone(&empty_schema)); - // let dfschema = DFSchema::empty(); - // let expr = - // limited_convert_logical_expr_to_physical_expr_with_dfschema(expr, &dfschema)?; let result = expr.evaluate(&empty_batch)?; match result { ColumnarValue::Array(_) => Err(DataFusionError::Internal(format!( diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index 088cd5441789..c53f7a6c4771 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -57,8 +57,8 @@ datafusion-common = { workspace = true, default-features = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } datafusion-expr-common = { workspace = true } -datafusion-physical-expr-common = { workspace = true } datafusion-functions-aggregate-common = { workspace = true } +datafusion-physical-expr-common = { workspace = true } half = { workspace = true } hashbrown = { workspace = true } hex = { version = "0.4", optional = true } diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index 343f210bcc67..5621473c4fdb 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -136,7 +136,7 @@ impl PhysicalExpr for CastExpr { children: Vec>, ) -> Result> { Ok(Arc::new(CastExpr::new( - children[0].clone(), + Arc::clone(&children[0]), self.cast_type.clone(), Some(self.cast_options.clone()), ))) @@ -211,7 +211,7 @@ pub fn cast_with_options( ) -> Result> { let expr_type = expr.data_type(input_schema)?; if expr_type == cast_type { - Ok(expr.clone()) + Ok(Arc::clone(&expr)) } else if can_cast_types(&expr_type, &cast_type) { Ok(Arc::new(CastExpr::new(expr, cast_type, cast_options))) } else { diff --git a/datafusion/physical-expr/src/expressions/column.rs b/datafusion/physical-expr/src/expressions/column.rs index 5a5fec499375..adf43ee545b4 100644 --- a/datafusion/physical-expr/src/expressions/column.rs +++ b/datafusion/physical-expr/src/expressions/column.rs @@ -91,7 +91,7 @@ impl PhysicalExpr for Column { /// Evaluate the expression fn evaluate(&self, batch: &RecordBatch) -> Result { self.bounds_check(batch.schema().as_ref())?; - Ok(ColumnarValue::Array(batch.column(self.index).clone())) + Ok(ColumnarValue::Array(Arc::clone(batch.column(self.index)))) } fn children(&self) -> Vec<&Arc> { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index bbc139460074..40598ccddcfe 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1209,7 +1209,6 @@ mod tests { use datafusion_execution::config::SessionConfig; use datafusion_execution::memory_pool::FairSpillPool; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; - use datafusion_expr::expr::Sort; use datafusion_functions_aggregate::array_agg::array_agg_udaf; use datafusion_functions_aggregate::average::avg_udaf; use datafusion_functions_aggregate::count::count_udaf; @@ -1363,7 +1362,6 @@ mod tests { let aggregates = vec![AggregateExprBuilder::new(count_udaf(), vec![lit(1i8)]) .schema(Arc::clone(&input_schema)) .name("COUNT(1)") - .logical_exprs(vec![datafusion_expr::lit(1i8)]) .build()?]; let task_ctx = if spill { @@ -1983,17 +1981,11 @@ mod tests { dfschema: &DFSchema, sort_options: SortOptions, ) -> Result> { - let sort_exprs = vec![datafusion_expr::Expr::Sort(Sort { - expr: Box::new(datafusion_expr::col("b")), - asc: !sort_options.descending, - nulls_first: sort_options.nulls_first, - })]; let ordering_req = vec![PhysicalSortExpr { expr: col("b", schema)?, options: sort_options, }]; let args = vec![col("b", schema)?]; - let logical_args = vec![datafusion_expr::col("b")]; let func = datafusion_expr::AggregateUDF::new_from_impl(FirstValue::new()); datafusion_expr_functions_aggregate::aggregate::create_aggregate_expr_with_dfschema( &func, @@ -2013,17 +2005,11 @@ mod tests { dfschema: &DFSchema, sort_options: SortOptions, ) -> Result> { - let sort_exprs = vec![datafusion_expr::Expr::Sort(Sort { - expr: Box::new(datafusion_expr::col("b")), - asc: !sort_options.descending, - nulls_first: sort_options.nulls_first, - })]; let ordering_req = vec![PhysicalSortExpr { expr: col("b", schema)?, options: sort_options, }]; let args = vec![col("b", schema)?]; - let logical_args = vec![datafusion_expr::col("b")]; let func = datafusion_expr::AggregateUDF::new_from_impl(LastValue::new()); create_aggregate_expr_with_dfschema( &func, @@ -2200,46 +2186,7 @@ mod tests { }, ]), ]; - let col_expr_a = Box::new(datafusion_expr::col("a")); - let col_expr_b = Box::new(datafusion_expr::col("b")); - let col_expr_c = Box::new(datafusion_expr::col("c")); - let sort_exprs = vec![ - None, - Some(vec![datafusion_expr::Expr::Sort(Sort::new( - col_expr_a.clone(), - options1.descending, - options1.nulls_first, - ))]), - Some(vec![ - datafusion_expr::Expr::Sort(Sort::new( - col_expr_a.clone(), - options1.descending, - options1.nulls_first, - )), - datafusion_expr::Expr::Sort(Sort::new( - col_expr_b.clone(), - options1.descending, - options1.nulls_first, - )), - datafusion_expr::Expr::Sort(Sort::new( - col_expr_c, - options1.descending, - options1.nulls_first, - )), - ]), - Some(vec![ - datafusion_expr::Expr::Sort(Sort::new( - col_expr_a, - options1.descending, - options1.nulls_first, - )), - datafusion_expr::Expr::Sort(Sort::new( - col_expr_b, - options1.descending, - options1.nulls_first, - )), - ]), - ]; + let common_requirement = vec![ PhysicalSortExpr { expr: Arc::clone(col_a), @@ -2252,10 +2199,8 @@ mod tests { ]; let mut aggr_exprs = order_by_exprs .into_iter() - .zip(sort_exprs.into_iter()) - .map(|(order_by_expr, sort_exprs)| { + .map(|order_by_expr| { let ordering_req = order_by_expr.unwrap_or_default(); - let sort_exprs = sort_exprs.unwrap_or_default(); create_aggregate_expr_with_dfschema( &array_agg_udaf(), &[Arc::clone(col_a)], diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index c6a0af09738c..940f336217f4 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -31,10 +31,9 @@ use crate::{ use arrow::datatypes::Schema; use arrow_schema::{DataType, Field, SchemaRef}; use datafusion_common::{exec_err, DataFusionError, Result, ScalarValue}; -use datafusion_expr::{col, Expr, SortExpr}; use datafusion_expr::{ - BuiltInWindowFunction, PartitionEvaluator, WindowFrame, WindowFunctionDefinition, - WindowUDF, + BuiltInWindowFunction, Expr, PartitionEvaluator, WindowFrame, + WindowFunctionDefinition, WindowUDF, }; use datafusion_expr_functions_aggregate::aggregate::AggregateExprBuilder; use datafusion_physical_expr::equivalence::collapse_lex_req; @@ -112,25 +111,10 @@ pub fn create_window_expr( )) } WindowFunctionDefinition::AggregateUDF(fun) => { - // 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(col(field_name)), - asc: !options.descending, - nulls_first: options.nulls_first, - }) - }) - .collect::>(); - let aggregate = AggregateExprBuilder::new(Arc::clone(fun), args.to_vec()) .schema(Arc::new(input_schema.clone())) .name(name) .order_by(order_by.to_vec()) - // .sort_exprs(sort_exprs) .with_ignore_nulls(ignore_nulls) .build()?; window_expr_from_aggregate_expr( @@ -395,6 +379,8 @@ impl BuiltInWindowFunctionExpr for WindowUDFExpr { } } +// TODO: Find a way to make clippy happy +#[allow(clippy::needless_borrow)] pub(crate) fn calc_requirements< T: Borrow>, S: Borrow, diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index b96398ef217f..e5c226418441 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -592,7 +592,9 @@ async fn roundtrip_logical_plan_copy_to_parquet() -> Result<()> { // Set specific Parquet format options let mut key_value_metadata = HashMap::new(); key_value_metadata.insert("test".to_string(), Some("test".to_string())); - parquet_format.key_value_metadata = key_value_metadata.clone(); + parquet_format + .key_value_metadata + .clone_from(&key_value_metadata); parquet_format.global.allow_single_file_parallelism = false; parquet_format.global.created_by = "test".to_string(); From 1bf175bcdb483460a2d52ae9f9cb65b66a71cb1b Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 6 Aug 2024 17:44:11 +0800 Subject: [PATCH 03/26] cleanup Signed-off-by: jayzhan211 --- .../expr-functions-aggregate/Cargo.toml | 2 - datafusion/expr/src/function.rs | 6 +- datafusion/expr/src/utils.rs | 55 ------------- .../functions-aggregate-common/Cargo.toml | 3 - .../src/approx_percentile_cont.rs | 82 ++++++++++--------- .../functions-aggregate/src/nth_value.rs | 52 ++++-------- datafusion/functions-aggregate/src/stddev.rs | 2 - .../functions-aggregate/src/string_agg.rs | 16 ---- 8 files changed, 62 insertions(+), 156 deletions(-) diff --git a/datafusion/expr-functions-aggregate/Cargo.toml b/datafusion/expr-functions-aggregate/Cargo.toml index 2792989d9afe..c6c78fc7c6ce 100644 --- a/datafusion/expr-functions-aggregate/Cargo.toml +++ b/datafusion/expr-functions-aggregate/Cargo.toml @@ -46,5 +46,3 @@ datafusion-expr-common = { workspace = true } datafusion-functions-aggregate-common = { workspace = true } datafusion-physical-expr-common = { workspace = true } rand = { workspace = true } -# strum = { version = "0.26.1", features = ["derive"] } -# strum_macros = "0.26.0" diff --git a/datafusion/expr/src/function.rs b/datafusion/expr/src/function.rs index 79aed2f0aaf2..888916981321 100644 --- a/datafusion/expr/src/function.rs +++ b/datafusion/expr/src/function.rs @@ -80,8 +80,7 @@ pub struct AccumulatorArgs<'a> { /// SELECT FIRST_VALUE(column1 ORDER BY column2) FROM t; /// ``` /// - /// If no `ORDER BY` is specified, `sort_exprs`` will be empty. - // pub sort_exprs: &'a [Expr], + /// If no `ORDER BY` is specified, `ordering_req` will be empty. pub ordering_req: &'a [PhysicalSortExpr], /// Whether the aggregation is running in reverse order @@ -100,8 +99,7 @@ pub struct AccumulatorArgs<'a> { /// The input types of the aggregate function. pub input_types: &'a [DataType], - /// The logical expression of arguments the aggregate function takes. - // pub input_exprs: &'a [Expr], + /// The physical expression of arguments the aggregate function takes. pub physical_exprs: &'a [Arc], } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 4044b0e8352e..c12edf994553 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -1219,61 +1219,6 @@ pub fn format_state_name(name: &str, state_name: &str) -> String { format!("{name}[{state_name}]") } -// /// Converts `datafusion_expr::Expr` into corresponding `Arc`. -// /// If conversion is not supported yet, returns Error. -// pub fn limited_convert_logical_expr_to_physical_expr_with_dfschema( -// expr: &Expr, -// dfschema: &DFSchema, -// ) -> Result> { -// match expr { -// Expr::Alias(Alias { expr, .. }) => Ok( -// limited_convert_logical_expr_to_physical_expr_with_dfschema(expr, dfschema)?, -// ), -// Expr::Column(col) => { -// let idx = dfschema.index_of_column(col)?; -// Ok(Arc::new(Column::new(&col.name, idx))) -// } -// Expr::Cast(cast_expr) => Ok(Arc::new(CastExpr::new( -// limited_convert_logical_expr_to_physical_expr_with_dfschema( -// cast_expr.expr.as_ref(), -// dfschema, -// )?, -// cast_expr.data_type.clone(), -// None, -// ))), -// Expr::Literal(value) => Ok(Arc::new(Literal::new(value.clone()))), -// _ => exec_err!( -// "Unsupported expression: {expr} for conversion to Arc" -// ), -// } -// } - -// /// Converts each [`Expr::Sort`] into a corresponding [`PhysicalSortExpr`]. -// /// Returns an error if the given logical expression is not a [`Expr::Sort`]. -// pub fn limited_convert_logical_sort_exprs_to_physical_with_dfschema( -// exprs: &[Expr], -// dfschema: &DFSchema, -// ) -> Result> { -// // Construct PhysicalSortExpr objects from Expr objects: -// let mut sort_exprs = vec![]; -// for expr in exprs { -// let Expr::Sort(sort) = expr else { -// return exec_err!("Expects to receive sort expression"); -// }; -// sort_exprs.push(PhysicalSortExpr::new( -// limited_convert_logical_expr_to_physical_expr_with_dfschema( -// sort.expr.as_ref(), -// dfschema, -// )?, -// SortOptions { -// descending: !sort.asc, -// nulls_first: sort.nulls_first, -// }, -// )) -// } -// Ok(sort_exprs) -// } - #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/functions-aggregate-common/Cargo.toml b/datafusion/functions-aggregate-common/Cargo.toml index 8473bc14c55c..1d0aaffa7974 100644 --- a/datafusion/functions-aggregate-common/Cargo.toml +++ b/datafusion/functions-aggregate-common/Cargo.toml @@ -42,8 +42,5 @@ ahash = { workspace = true } arrow = { workspace = true } datafusion-common = { workspace = true } datafusion-expr-common = { workspace = true } -# datafusion-expr = { workspace = true } datafusion-physical-expr-common = { workspace = true } rand = { workspace = true } -# strum = { version = "0.26.1", features = ["derive"] } -# strum_macros = "0.26.0" diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index 5c7d06d2bccd..db9ddaf7174f 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -19,7 +19,7 @@ use std::any::Any; use std::fmt::{Debug, Formatter}; use std::sync::Arc; -use arrow::array::{Array, RecordBatch}; +use arrow::array::Array; use arrow::compute::{filter, is_not_null}; use arrow::{ array::{ @@ -28,7 +28,7 @@ use arrow::{ }, datatypes::DataType, }; -use arrow_schema::{Field, Schema}; +use arrow_schema::Field; use datafusion_common::{ downcast_value, internal_err, not_impl_err, plan_err, DataFusionError, ScalarValue, @@ -37,11 +37,12 @@ use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::{INTEGERS, NUMERICS}; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ - Accumulator, AggregateUDFImpl, ColumnarValue, Signature, TypeSignature, Volatility, + Accumulator, AggregateUDFImpl, Signature, TypeSignature, Volatility, }; use datafusion_functions_aggregate_common::tdigest::{ TDigest, TryIntoF64, DEFAULT_MAX_SIZE, }; +use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; make_udaf_expr_and_func!( @@ -132,31 +133,27 @@ impl ApproxPercentileCont { } } -fn get_lit_value(expr: &Arc) -> datafusion_common::Result { - // TODO: use real schema - let empty_schema = Arc::new(Schema::empty()); - let empty_batch = RecordBatch::new_empty(Arc::clone(&empty_schema)); - let result = expr.evaluate(&empty_batch)?; - match result { - ColumnarValue::Array(_) => Err(DataFusionError::Internal(format!( - "The expr {:?} can't be evaluated to scalar value", - expr - ))), - ColumnarValue::Scalar(scalar_value) => Ok(scalar_value), - } -} - fn validate_input_percentile_expr( expr: &Arc, ) -> datafusion_common::Result { - let lit = get_lit_value(expr)?; - let percentile = match &lit { - ScalarValue::Float32(Some(q)) => *q as f64, - ScalarValue::Float64(Some(q)) => *q, - got => return not_impl_err!( - "Percentile value for 'APPROX_PERCENTILE_CONT' must be Float32 or Float64 literal (got data type {})", - got.data_type() - ) + let percentile = match expr.as_any().downcast_ref::() { + Some(lit) => match lit.value() { + ScalarValue::Float32(Some(value)) => { + *value as f64 + } + ScalarValue::Float64(Some(value)) => { + *value + } + sv => { + return not_impl_err!( + "Percentile value for 'APPROX_PERCENTILE_CONT' must be Float32 or Float64 literal (got data type {})", + sv.data_type() + ) + } + }, + None => { + return internal_err!("Expect to get a literal expr") + } }; // Ensure the percentile is between 0 and 1. @@ -171,21 +168,28 @@ fn validate_input_percentile_expr( fn validate_input_max_size_expr( expr: &Arc, ) -> datafusion_common::Result { - let lit = get_lit_value(expr)?; - let max_size = match &lit { - ScalarValue::UInt8(Some(q)) => *q as usize, - ScalarValue::UInt16(Some(q)) => *q as usize, - ScalarValue::UInt32(Some(q)) => *q as usize, - ScalarValue::UInt64(Some(q)) => *q as usize, - ScalarValue::Int32(Some(q)) if *q > 0 => *q as usize, - ScalarValue::Int64(Some(q)) if *q > 0 => *q as usize, - ScalarValue::Int16(Some(q)) if *q > 0 => *q as usize, - ScalarValue::Int8(Some(q)) if *q > 0 => *q as usize, - got => return not_impl_err!( - "Tdigest max_size value for 'APPROX_PERCENTILE_CONT' must be UInt > 0 literal (got data type {}).", - got.data_type() - ) + let max_size = match expr.as_any().downcast_ref::() { + Some(lit) => match lit.value() { + ScalarValue::UInt8(Some(q)) => *q as usize, + ScalarValue::UInt16(Some(q)) => *q as usize, + ScalarValue::UInt32(Some(q)) => *q as usize, + ScalarValue::UInt64(Some(q)) => *q as usize, + ScalarValue::Int32(Some(q)) if *q > 0 => *q as usize, + ScalarValue::Int64(Some(q)) if *q > 0 => *q as usize, + ScalarValue::Int16(Some(q)) if *q > 0 => *q as usize, + ScalarValue::Int8(Some(q)) if *q > 0 => *q as usize, + sv => { + return not_impl_err!( + "Tdigest max_size value for 'APPROX_PERCENTILE_CONT' must be UInt > 0 literal (got data type {}).", + sv.data_type() + ) + } + }, + None => { + return internal_err!("Expect to get a literal expr") + } }; + Ok(max_size) } diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index 60b8e818df15..a2ef3e678fe8 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -85,53 +85,35 @@ impl AggregateUDFImpl for NthValueAgg { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - // TODO: simplfiy this - let n = if let Some(lit) = acc_args.physical_exprs[1] + let n = match acc_args.physical_exprs[1] .as_any() .downcast_ref::() { - if let ScalarValue::Int64(Some(value)) = lit.value() { - if acc_args.is_reversed { - -*value - } else { - *value + Some(lit) => match lit.value() { + ScalarValue::Int64(Some(value)) => { + if acc_args.is_reversed { + -*value + } else { + *value + } } - } else { + _ => { + return not_impl_err!( + "{} not supported for n: {}", + self.name(), + &acc_args.physical_exprs[1] + ); + } + }, + None => { return not_impl_err!( "{} not supported for n: {}", self.name(), &acc_args.physical_exprs[1] ); } - } else { - return not_impl_err!( - "{} not supported for n: {}", - self.name(), - &acc_args.physical_exprs[1] - ); }; - // let n = match acc_args.physical_exprs[1] { - - // Expr::Literal(ScalarValue::Int64(Some(value))) => { - // if acc_args.is_reversed { - // Ok(-value) - // } else { - // Ok(value) - // } - // } - // _ => not_impl_err!( - // "{} not supported for n: {}", - // self.name(), - // &acc_args.physical_exprs[1] - // ), - // }?; - - // let ordering_req = limited_convert_logical_sort_exprs_to_physical_with_dfschema( - // acc_args.sort_exprs, - // acc_args.dfschema, - // )?; - let ordering_dtypes = acc_args .ordering_req .iter() diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index a04e191d3f0e..7f0fd47b7a98 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -337,7 +337,6 @@ mod tests { is_reversed: false, input_types: &[DataType::Float64], physical_exprs: &[col("a", schema)?], - // input_exprs: &[datafusion_expr::col("a")], }; let args2 = AccumulatorArgs { @@ -352,7 +351,6 @@ mod tests { is_reversed: false, input_types: &[DataType::Float64], physical_exprs: &[col("a", schema)?], - // input_exprs: &[datafusion_expr::col("a")], }; let mut accum1 = agg1.accumulator(args1)?; diff --git a/datafusion/functions-aggregate/src/string_agg.rs b/datafusion/functions-aggregate/src/string_agg.rs index fa83337f217b..9000f652c6b8 100644 --- a/datafusion/functions-aggregate/src/string_agg.rs +++ b/datafusion/functions-aggregate/src/string_agg.rs @@ -100,22 +100,6 @@ impl AggregateUDFImpl for StringAgg { } not_impl_err!("expect literal") - - // match &acc_args.physical_exprs[1] { - // Expr::Literal(ScalarValue::Utf8(Some(delimiter))) - // | Expr::Literal(ScalarValue::LargeUtf8(Some(delimiter))) => { - // Ok(Box::new(StringAggAccumulator::new(delimiter))) - // } - // Expr::Literal(ScalarValue::Utf8(None)) - // | Expr::Literal(ScalarValue::LargeUtf8(None)) - // | Expr::Literal(ScalarValue::Null) => { - // Ok(Box::new(StringAggAccumulator::new(""))) - // } - // _ => not_impl_err!( - // "StringAgg not supported for delimiter {}", - // &acc_args.input_exprs[1] - // ), - // } } } From bf394cc6b1c0ecdfba5f120fb8769527cdcc903e Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 6 Aug 2024 20:18:39 +0800 Subject: [PATCH 04/26] fix merge conflict Signed-off-by: jayzhan211 --- datafusion/core/src/physical_planner.rs | 15 ++++++++++++++- datafusion/core/src/test_util/mod.rs | 2 +- datafusion/expr-common/src/groups_accumulator.rs | 6 +++--- .../expr-functions-aggregate/src/aggregate.rs | 11 +++-------- .../functions-aggregate-common/src/aggregate.rs | 7 ++----- .../aggregate/groups_accumulator/accumulate.rs | 4 ++-- datafusion/functions-aggregate-common/src/lib.rs | 2 -- 7 files changed, 25 insertions(+), 22 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 3529e2b2f338..9e878c6597e2 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -73,7 +73,8 @@ use datafusion_common::{ }; use datafusion_expr::dml::CopyTo; use datafusion_expr::expr::{ - self, physical_name, AggregateFunction, Alias, GroupingSet, WindowFunction, + self, create_function_physical_name, physical_name, AggregateFunction, Alias, + GroupingSet, WindowFunction, }; use datafusion_expr::expr_rewriter::unnormalize_cols; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; @@ -1559,6 +1560,18 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( order_by, null_treatment, }) => { + let name = if name.is_none() { + let name = create_function_physical_name( + func.name(), + *distinct, + args, + order_by.as_ref(), + )?; + Some(name) + } else { + name + }; + let physical_args = create_physical_exprs(args, logical_input_schema, execution_props)?; let filter = match filter { diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 9610a7f20364..46c131a450c4 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -54,7 +54,7 @@ use datafusion_physical_expr::{ use async_trait::async_trait; use datafusion_catalog::Session; -use datafusion_physical_expr_common::aggregate::AggregateExprBuilder; +use datafusion_expr_functions_aggregate::aggregate::AggregateExprBuilder; use futures::Stream; use tempfile::TempDir; // backwards compatibility diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 055d731b114c..e66b27d073d1 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -75,7 +75,7 @@ impl EmitTo { /// expected that each `GroupAccumulator` will use something like `Vec<..>` /// to store the group states. /// -/// [`Accumulator`]: crate::Accumulator +/// [`Accumulator`]: crate::accumulator::Accumulator /// [Aggregating Millions of Groups Fast blog]: https://arrow.apache.org/blog/2023/08/05/datafusion_fast_grouping/ pub trait GroupsAccumulator: Send { /// Updates the accumulator's state from its arguments, encoded as @@ -140,7 +140,7 @@ pub trait GroupsAccumulator: Send { /// See [`Self::evaluate`] for details on the required output /// order and `emit_to`. /// - /// [`Accumulator::state`]: crate::Accumulator::state + /// [`Accumulator::state`]: crate::accumulator::Accumulator::state fn state(&mut self, emit_to: EmitTo) -> Result>; /// Merges intermediate state (the output from [`Self::state`]) @@ -197,7 +197,7 @@ pub trait GroupsAccumulator: Send { /// state directly to the next aggregation phase with minimal processing /// using this method. /// - /// [`Accumulator::state`]: crate::Accumulator::state + /// [`Accumulator::state`]: crate::accumulator::Accumulator::state fn convert_to_state( &self, _values: &[ArrayRef], diff --git a/datafusion/expr-functions-aggregate/src/aggregate.rs b/datafusion/expr-functions-aggregate/src/aggregate.rs index d63c40832b1e..aea301f2cd2e 100644 --- a/datafusion/expr-functions-aggregate/src/aggregate.rs +++ b/datafusion/expr-functions-aggregate/src/aggregate.rs @@ -17,7 +17,7 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::{internal_err, not_impl_err, DFSchema, Result}; -// use datafusion_expr::expr::create_function_physical_name; +use datafusion_expr::expr::create_function_physical_name; use datafusion_expr::function::StateFieldsArgs; use datafusion_expr::ReversedUDAF; use datafusion_expr::{function::AccumulatorArgs, AggregateUDF}; @@ -192,13 +192,8 @@ impl AggregateExprBuilder { let data_type = fun.return_type(&input_exprs_types)?; let name = match alias { - // TODO: fix this - None => create_function_physical_name( - fun.name(), - is_distinct, - &[], - args, - )?, + // TODO: Ideally, we should build the name from physical expressions + None => create_function_physical_name(fun.name(), is_distinct, &[], None)?, Some(alias) => alias, }; diff --git a/datafusion/functions-aggregate-common/src/aggregate.rs b/datafusion/functions-aggregate-common/src/aggregate.rs index 771e86969ce6..98eafd80e3bc 100644 --- a/datafusion/functions-aggregate-common/src/aggregate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate.rs @@ -20,14 +20,11 @@ use std::{any::Any, sync::Arc}; use arrow::datatypes::Field; +use crate::order::AggregateOrderSensitivity; use datafusion_common::exec_err; use datafusion_common::{not_impl_err, Result}; -// use datafusion_expr::utils::AggregateOrderSensitivity; -use crate::order::AggregateOrderSensitivity; use datafusion_expr_common::accumulator::Accumulator; - use datafusion_expr_common::groups_accumulator::GroupsAccumulator; - use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; @@ -155,7 +152,7 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { /// Rewrites [`AggregateExpr`], with new expressions given. The argument should be consistent /// with the return value of the [`AggregateExpr::all_expressions`] method. /// Returns `Some(Arc)` if re-write is supported, otherwise returns `None`. - /// TODO: This method only rewrites the [`PhysicalExpr`]s and does not handle [`Expr`]s. + /// TODO: This method only rewrites the [`PhysicalExpr`]s and does not handle `Expr`s. /// This can cause silent bugs and should be fixed in the future (possibly with physical-to-logical /// conversions). fn with_new_expressions( diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs index 18e497c63eb0..455fc5fec450 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -17,7 +17,7 @@ //! [`GroupsAccumulator`] helpers: [`NullState`] and [`accumulate_indices`] //! -//! [`GroupsAccumulator`]: datafusion_expr::GroupsAccumulator +//! [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; use arrow::buffer::{BooleanBuffer, NullBuffer}; @@ -48,7 +48,7 @@ use datafusion_expr_common::groups_accumulator::EmitTo; /// had at least one value to accumulate so they do not need to track /// if they have seen values for a particular group. /// -/// [`GroupsAccumulator`]: datafusion_expr::GroupsAccumulator +/// [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator #[derive(Debug)] pub struct NullState { /// Have we seen any non-filtered input values for `group_index`? diff --git a/datafusion/functions-aggregate-common/src/lib.rs b/datafusion/functions-aggregate-common/src/lib.rs index 43bc156f803e..92b143cff3e9 100644 --- a/datafusion/functions-aggregate-common/src/lib.rs +++ b/datafusion/functions-aggregate-common/src/lib.rs @@ -15,8 +15,6 @@ // specific language governing permissions and limitations // under the License. -//! [`AggregateUDF`]: User Defined Aggregate Functions - pub mod aggregate; pub mod merge_arrays; pub mod order; From a2e4cba3c3577cd3828fee7c91b5f74a4579d472 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 6 Aug 2024 20:42:51 +0800 Subject: [PATCH 05/26] mv accumulator out Signed-off-by: jayzhan211 --- datafusion/expr-common/src/signature.rs | 4 +- datafusion/expr/src/function.rs | 92 ++------------- datafusion/expr/src/udaf.rs | 2 +- .../src/accumulator.rs | 105 ++++++++++++++++++ .../functions-aggregate-common/src/lib.rs | 1 + 5 files changed, 115 insertions(+), 89 deletions(-) create mode 100644 datafusion/functions-aggregate-common/src/accumulator.rs diff --git a/datafusion/expr-common/src/signature.rs b/datafusion/expr-common/src/signature.rs index 32c828c992ed..8db4bc40d78c 100644 --- a/datafusion/expr-common/src/signature.rs +++ b/datafusion/expr-common/src/signature.rs @@ -93,9 +93,7 @@ pub enum TypeSignature { Variadic(Vec), /// The acceptable signature and coercions rules to coerce arguments to this /// signature are special for this function. If this signature is specified, - /// DataFusion will call [`ScalarUDFImpl::coerce_types`] to prepare argument types. - /// - /// [`ScalarUDFImpl::coerce_types`]: crate::udf::ScalarUDFImpl::coerce_types + /// DataFusion will call `ScalarUDFImpl::coerce_types` to prepare argument types. UserDefined, /// One or more arguments with arbitrary types VariadicAny, diff --git a/datafusion/expr/src/function.rs b/datafusion/expr/src/function.rs index 888916981321..cd7a0c8aa918 100644 --- a/datafusion/expr/src/function.rs +++ b/datafusion/expr/src/function.rs @@ -18,13 +18,15 @@ //! Function module contains typing and signature for built-in and user defined functions. use crate::ColumnarValue; -use crate::{Accumulator, Expr, PartitionEvaluator}; -use arrow::datatypes::{DataType, Field, Schema}; -use datafusion_common::{DFSchema, Result}; -use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use crate::{Expr, PartitionEvaluator}; +use arrow::datatypes::DataType; +use datafusion_common::Result; use std::sync::Arc; +pub use datafusion_functions_aggregate_common::accumulator::{ + AccumulatorArgs, AccumulatorFactoryFunction, StateFieldsArgs, +}; + #[derive(Debug, Clone, Copy)] pub enum Hint { /// Indicates the argument needs to be padded if it is scalar @@ -48,86 +50,6 @@ pub type ScalarFunctionImplementation = pub type ReturnTypeFunction = Arc Result> + Send + Sync>; -/// [`AccumulatorArgs`] contains information about how an aggregate -/// function was called, including the types of its arguments and any optional -/// ordering expressions. -#[derive(Debug)] -pub struct AccumulatorArgs<'a> { - /// The return type of the aggregate function. - pub data_type: &'a DataType, - - /// The schema of the input arguments - pub schema: &'a Schema, - - /// The schema of the input arguments - pub dfschema: &'a DFSchema, - - /// Whether to ignore nulls. - /// - /// SQL allows the user to specify `IGNORE NULLS`, for example: - /// - /// ```sql - /// SELECT FIRST_VALUE(column1) IGNORE NULLS FROM t; - /// ``` - pub ignore_nulls: bool, - - /// The expressions in the `ORDER BY` clause passed to this aggregator. - /// - /// SQL allows the user to specify the ordering of arguments to the - /// aggregate using an `ORDER BY`. For example: - /// - /// ```sql - /// SELECT FIRST_VALUE(column1 ORDER BY column2) FROM t; - /// ``` - /// - /// If no `ORDER BY` is specified, `ordering_req` will be empty. - pub ordering_req: &'a [PhysicalSortExpr], - - /// Whether the aggregation is running in reverse order - pub is_reversed: bool, - - /// The name of the aggregate expression - pub name: &'a str, - - /// Whether the aggregate function is distinct. - /// - /// ```sql - /// SELECT COUNT(DISTINCT column1) FROM t; - /// ``` - pub is_distinct: bool, - - /// The input types of the aggregate function. - pub input_types: &'a [DataType], - - /// The physical expression of arguments the aggregate function takes. - pub physical_exprs: &'a [Arc], -} - -/// [`StateFieldsArgs`] contains information about the fields that an -/// aggregate function's accumulator should have. Used for [`AggregateUDFImpl::state_fields`]. -/// -/// [`AggregateUDFImpl::state_fields`]: crate::udaf::AggregateUDFImpl::state_fields -pub struct StateFieldsArgs<'a> { - /// The name of the aggregate function. - pub name: &'a str, - - /// The input types of the aggregate function. - pub input_types: &'a [DataType], - - /// The return type of the aggregate function. - pub return_type: &'a DataType, - - /// The ordering fields of the aggregate function. - pub ordering_fields: &'a [Field], - - /// Whether the aggregate function is distinct. - pub is_distinct: bool, -} - -/// Factory that returns an accumulator for the given aggregate function. -pub type AccumulatorFactoryFunction = - Arc Result> + Send + Sync>; - /// Factory that creates a PartitionEvaluator for the given window /// function pub type PartitionEvaluatorFactory = diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 3a292b2b49bf..d136aeaf0908 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -422,7 +422,7 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// /// See [retract_batch] for more details. /// - /// [retract_batch]: crate::accumulator::Accumulator::retract_batch + /// [retract_batch]: datafusion_expr_common::accumulator::Accumulator::retract_batch fn create_sliding_accumulator( &self, args: AccumulatorArgs, diff --git a/datafusion/functions-aggregate-common/src/accumulator.rs b/datafusion/functions-aggregate-common/src/accumulator.rs new file mode 100644 index 000000000000..4e28da5f692f --- /dev/null +++ b/datafusion/functions-aggregate-common/src/accumulator.rs @@ -0,0 +1,105 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use arrow::datatypes::{DataType, Field, Schema}; +use datafusion_common::{DFSchema, Result}; +use datafusion_expr_common::accumulator::Accumulator; +use datafusion_physical_expr_common::{ + physical_expr::PhysicalExpr, sort_expr::PhysicalSortExpr, +}; + +/// [`AccumulatorArgs`] contains information about how an aggregate +/// function was called, including the types of its arguments and any optional +/// ordering expressions. +#[derive(Debug)] +pub struct AccumulatorArgs<'a> { + /// The return type of the aggregate function. + pub data_type: &'a DataType, + + /// The schema of the input arguments + pub schema: &'a Schema, + + /// The schema of the input arguments + pub dfschema: &'a DFSchema, + + /// Whether to ignore nulls. + /// + /// SQL allows the user to specify `IGNORE NULLS`, for example: + /// + /// ```sql + /// SELECT FIRST_VALUE(column1) IGNORE NULLS FROM t; + /// ``` + pub ignore_nulls: bool, + + /// The expressions in the `ORDER BY` clause passed to this aggregator. + /// + /// SQL allows the user to specify the ordering of arguments to the + /// aggregate using an `ORDER BY`. For example: + /// + /// ```sql + /// SELECT FIRST_VALUE(column1 ORDER BY column2) FROM t; + /// ``` + /// + /// If no `ORDER BY` is specified, `ordering_req` will be empty. + pub ordering_req: &'a [PhysicalSortExpr], + + /// Whether the aggregation is running in reverse order + pub is_reversed: bool, + + /// The name of the aggregate expression + pub name: &'a str, + + /// Whether the aggregate function is distinct. + /// + /// ```sql + /// SELECT COUNT(DISTINCT column1) FROM t; + /// ``` + pub is_distinct: bool, + + /// The input types of the aggregate function. + pub input_types: &'a [DataType], + + /// The physical expression of arguments the aggregate function takes. + pub physical_exprs: &'a [Arc], +} + +/// Factory that returns an accumulator for the given aggregate function. +pub type AccumulatorFactoryFunction = + Arc Result> + Send + Sync>; + +/// [`StateFieldsArgs`] contains information about the fields that an +/// aggregate function's accumulator should have. Used for [`AggregateUDFImpl::state_fields`]. +/// +/// [`AggregateUDFImpl::state_fields`]: crate::udaf::AggregateUDFImpl::state_fields +pub struct StateFieldsArgs<'a> { + /// The name of the aggregate function. + pub name: &'a str, + + /// The input types of the aggregate function. + pub input_types: &'a [DataType], + + /// The return type of the aggregate function. + pub return_type: &'a DataType, + + /// The ordering fields of the aggregate function. + pub ordering_fields: &'a [Field], + + /// Whether the aggregate function is distinct. + pub is_distinct: bool, +} diff --git a/datafusion/functions-aggregate-common/src/lib.rs b/datafusion/functions-aggregate-common/src/lib.rs index 92b143cff3e9..a4fcb4aa9de9 100644 --- a/datafusion/functions-aggregate-common/src/lib.rs +++ b/datafusion/functions-aggregate-common/src/lib.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +pub mod accumulator; pub mod aggregate; pub mod merge_arrays; pub mod order; From 4a634054e70de4973afa10d2c61c7f95c67bf958 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 6 Aug 2024 22:11:27 +0800 Subject: [PATCH 06/26] fix doc Signed-off-by: jayzhan211 --- datafusion/expr-functions-aggregate/src/aggregate.rs | 5 +++-- datafusion/functions-aggregate-common/src/accumulator.rs | 4 +--- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/datafusion/expr-functions-aggregate/src/aggregate.rs b/datafusion/expr-functions-aggregate/src/aggregate.rs index aea301f2cd2e..daabfcef3990 100644 --- a/datafusion/expr-functions-aggregate/src/aggregate.rs +++ b/datafusion/expr-functions-aggregate/src/aggregate.rs @@ -18,12 +18,13 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::{internal_err, not_impl_err, DFSchema, Result}; use datafusion_expr::expr::create_function_physical_name; -use datafusion_expr::function::StateFieldsArgs; +use datafusion_expr::AggregateUDF; use datafusion_expr::ReversedUDAF; -use datafusion_expr::{function::AccumulatorArgs, AggregateUDF}; use datafusion_expr_common::accumulator::Accumulator; use datafusion_expr_common::groups_accumulator::GroupsAccumulator; use datafusion_expr_common::type_coercion::aggregates::check_arg_count; +use datafusion_functions_aggregate_common::accumulator::AccumulatorArgs; +use datafusion_functions_aggregate_common::accumulator::StateFieldsArgs; use datafusion_functions_aggregate_common::aggregate::AggregateExpr; use datafusion_functions_aggregate_common::order::AggregateOrderSensitivity; use datafusion_functions_aggregate_common::utils::{self, down_cast_any_ref}; diff --git a/datafusion/functions-aggregate-common/src/accumulator.rs b/datafusion/functions-aggregate-common/src/accumulator.rs index 4e28da5f692f..3c31f4a5e9a6 100644 --- a/datafusion/functions-aggregate-common/src/accumulator.rs +++ b/datafusion/functions-aggregate-common/src/accumulator.rs @@ -84,9 +84,7 @@ pub type AccumulatorFactoryFunction = Arc Result> + Send + Sync>; /// [`StateFieldsArgs`] contains information about the fields that an -/// aggregate function's accumulator should have. Used for [`AggregateUDFImpl::state_fields`]. -/// -/// [`AggregateUDFImpl::state_fields`]: crate::udaf::AggregateUDFImpl::state_fields +/// aggregate function's accumulator should have. Used for `AggregateUDFImpl::state_fields`. pub struct StateFieldsArgs<'a> { /// The name of the aggregate function. pub name: &'a str, From f218184a990d1bda93df80b2514dc60f9ccba5c4 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 6 Aug 2024 22:37:57 +0800 Subject: [PATCH 07/26] rename Signed-off-by: jayzhan211 --- Cargo.toml | 8 ++--- datafusion-cli/Cargo.lock | 32 +++++++++---------- datafusion/core/Cargo.toml | 2 +- datafusion/core/src/lib.rs | 6 ++-- .../combine_partial_final_agg.rs | 2 +- datafusion/core/src/test_util/mod.rs | 2 +- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 2 +- .../Cargo.toml | 4 +-- .../src/aggregate.rs | 0 .../src/lib.rs | 0 datafusion/physical-plan/Cargo.toml | 2 +- .../physical-plan/src/aggregates/mod.rs | 6 ++-- datafusion/physical-plan/src/lib.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 2 +- datafusion/proto/src/physical_plan/mod.rs | 2 +- .../tests/cases/roundtrip_physical_plan.rs | 2 +- 16 files changed, 37 insertions(+), 37 deletions(-) rename datafusion/{expr-functions-aggregate => physical-expr-functions-aggregate}/Cargo.toml (93%) rename datafusion/{expr-functions-aggregate => physical-expr-functions-aggregate}/src/aggregate.rs (100%) rename datafusion/{expr-functions-aggregate => physical-expr-functions-aggregate}/src/lib.rs (100%) diff --git a/Cargo.toml b/Cargo.toml index 835fa1eabe02..3958c0264d11 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -24,15 +24,15 @@ members = [ "datafusion/core", "datafusion/expr", "datafusion/expr-common", - "datafusion/expr-functions-aggregate", "datafusion/execution", + "datafusion/functions", "datafusion/functions-aggregate", "datafusion/functions-aggregate-common", - "datafusion/functions", "datafusion/functions-nested", "datafusion/optimizer", - "datafusion/physical-expr-common", "datafusion/physical-expr", + "datafusion/physical-expr-common", + "datafusion/physical-expr-functions-aggregate", "datafusion/physical-optimizer", "datafusion/physical-plan", "datafusion/proto", @@ -98,7 +98,6 @@ datafusion-common-runtime = { path = "datafusion/common-runtime", version = "40. datafusion-execution = { path = "datafusion/execution", version = "40.0.0" } datafusion-expr = { path = "datafusion/expr", version = "40.0.0" } datafusion-expr-common = { path = "datafusion/expr-common", version = "40.0.0" } -datafusion-expr-functions-aggregate = { path = "datafusion/expr-functions-aggregate", version = "40.0.0" } datafusion-functions = { path = "datafusion/functions", version = "40.0.0" } datafusion-functions-aggregate = { path = "datafusion/functions-aggregate", version = "40.0.0" } datafusion-functions-aggregate-common = { path = "datafusion/functions-aggregate-common", version = "40.0.0" } @@ -106,6 +105,7 @@ datafusion-functions-nested = { path = "datafusion/functions-nested", version = datafusion-optimizer = { path = "datafusion/optimizer", version = "40.0.0", default-features = false } datafusion-physical-expr = { path = "datafusion/physical-expr", version = "40.0.0", default-features = false } datafusion-physical-expr-common = { path = "datafusion/physical-expr-common", version = "40.0.0", default-features = false } +datafusion-physical-expr-functions-aggregate = { path = "datafusion/physical-expr-functions-aggregate", version = "40.0.0" } datafusion-physical-optimizer = { path = "datafusion/physical-optimizer", version = "40.0.0" } datafusion-physical-plan = { path = "datafusion/physical-plan", version = "40.0.0" } datafusion-proto = { path = "datafusion/proto", version = "40.0.0" } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index c28644842bf3..ed554fddb772 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1147,13 +1147,13 @@ dependencies = [ "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", - "datafusion-expr-functions-aggregate", "datafusion-functions", "datafusion-functions-aggregate", "datafusion-functions-nested", "datafusion-optimizer", "datafusion-physical-expr", "datafusion-physical-expr-common", + "datafusion-physical-expr-functions-aggregate", "datafusion-physical-optimizer", "datafusion-physical-plan", "datafusion-sql", @@ -1298,20 +1298,6 @@ dependencies = [ "paste", ] -[[package]] -name = "datafusion-expr-functions-aggregate" -version = "40.0.0" -dependencies = [ - "ahash", - "arrow", - "datafusion-common", - "datafusion-expr", - "datafusion-expr-common", - "datafusion-functions-aggregate-common", - "datafusion-physical-expr-common", - "rand", -] - [[package]] name = "datafusion-functions" version = "40.0.0" @@ -1447,6 +1433,20 @@ dependencies = [ "rand", ] +[[package]] +name = "datafusion-physical-expr-functions-aggregate" +version = "40.0.0" +dependencies = [ + "ahash", + "arrow", + "datafusion-common", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions-aggregate-common", + "datafusion-physical-expr-common", + "rand", +] + [[package]] name = "datafusion-physical-optimizer" version = "40.0.0" @@ -1473,11 +1473,11 @@ dependencies = [ "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", - "datafusion-expr-functions-aggregate", "datafusion-functions-aggregate", "datafusion-functions-aggregate-common", "datafusion-physical-expr", "datafusion-physical-expr-common", + "datafusion-physical-expr-functions-aggregate", "futures", "half", "hashbrown 0.14.5", diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 3f18a8dfc04d..3911f882c7ff 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -103,7 +103,7 @@ datafusion-common = { workspace = true, features = ["object_store"] } datafusion-common-runtime = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } -datafusion-expr-functions-aggregate = { workspace = true } +datafusion-physical-expr-functions-aggregate = { workspace = true } datafusion-functions = { workspace = true } datafusion-functions-aggregate = { workspace = true } datafusion-functions-nested = { workspace = true, optional = true } diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index 6bee3dff9118..6b3773e4f6d5 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -556,9 +556,9 @@ pub mod physical_expr_common { pub use datafusion_physical_expr_common::*; } -/// re-export of [`datafusion_expr_functions_aggregate`] crate -pub mod expr_functions_aggregate { - pub use datafusion_expr_functions_aggregate::*; +/// re-export of [`datafusion_physical_expr_functions_aggregate`] crate +pub mod physical_expr_functions_aggregate { + pub use datafusion_physical_expr_functions_aggregate::*; } /// re-export of [`datafusion_physical_expr`] crate diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 6a958059d58b..f65a4c837a60 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -174,10 +174,10 @@ mod tests { use crate::physical_plan::{displayable, Partitioning}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - use datafusion_expr_functions_aggregate::aggregate::AggregateExprBuilder; use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::sum::sum_udaf; use datafusion_physical_expr::expressions::col; + use datafusion_physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; /// Runs the CombinePartialFinalAggregate optimizer and asserts the plan against the expected macro_rules! assert_optimized { diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 46c131a450c4..89206c6c3f5a 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -54,7 +54,7 @@ use datafusion_physical_expr::{ use async_trait::async_trait; use datafusion_catalog::Session; -use datafusion_expr_functions_aggregate::aggregate::AggregateExprBuilder; +use datafusion_physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; use futures::Stream; use tempfile::TempDir; // backwards compatibility diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 250abb74d65a..138e5bda7f39 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -25,7 +25,7 @@ use arrow::util::pretty::pretty_format_batches; use arrow_array::types::Int64Type; use datafusion::common::Result; use datafusion::datasource::MemTable; -use datafusion::expr_functions_aggregate::aggregate::AggregateExprBuilder; +use datafusion::physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; diff --git a/datafusion/expr-functions-aggregate/Cargo.toml b/datafusion/physical-expr-functions-aggregate/Cargo.toml similarity index 93% rename from datafusion/expr-functions-aggregate/Cargo.toml rename to datafusion/physical-expr-functions-aggregate/Cargo.toml index c6c78fc7c6ce..6eed89614c53 100644 --- a/datafusion/expr-functions-aggregate/Cargo.toml +++ b/datafusion/physical-expr-functions-aggregate/Cargo.toml @@ -16,7 +16,7 @@ # under the License. [package] -name = "datafusion-expr-functions-aggregate" +name = "datafusion-physical-expr-functions-aggregate" description = "Logical plan and expression representation for DataFusion query engine" keywords = ["datafusion", "logical", "plan", "expressions"] readme = "README.md" @@ -32,7 +32,7 @@ rust-version = { workspace = true } workspace = true [lib] -name = "datafusion_expr_functions_aggregate" +name = "datafusion_physical_expr_functions_aggregate" path = "src/lib.rs" [features] diff --git a/datafusion/expr-functions-aggregate/src/aggregate.rs b/datafusion/physical-expr-functions-aggregate/src/aggregate.rs similarity index 100% rename from datafusion/expr-functions-aggregate/src/aggregate.rs rename to datafusion/physical-expr-functions-aggregate/src/aggregate.rs diff --git a/datafusion/expr-functions-aggregate/src/lib.rs b/datafusion/physical-expr-functions-aggregate/src/lib.rs similarity index 100% rename from datafusion/expr-functions-aggregate/src/lib.rs rename to datafusion/physical-expr-functions-aggregate/src/lib.rs diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 102c66ab99bd..3a5ae6ec04ff 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -51,7 +51,7 @@ datafusion-common = { workspace = true, default-features = true } datafusion-common-runtime = { workspace = true, default-features = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } -datafusion-expr-functions-aggregate = { workspace = true } +datafusion-physical-expr-functions-aggregate = { workspace = true } datafusion-functions-aggregate = { workspace = true } datafusion-functions-aggregate-common = { workspace = true } datafusion-physical-expr = { workspace = true, default-features = true } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 31d598a89237..6f8aba197107 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1218,10 +1218,10 @@ mod tests { use datafusion_physical_expr::PhysicalSortExpr; use crate::common::collect; - use datafusion_expr_functions_aggregate::aggregate::{ + use datafusion_physical_expr::expressions::Literal; + use datafusion_physical_expr_functions_aggregate::aggregate::{ create_aggregate_expr_with_dfschema, AggregateExprBuilder, }; - use datafusion_physical_expr::expressions::Literal; use futures::{FutureExt, Stream}; // Generate a schema which consists of 5 columns (a, b, c, d, e) @@ -1987,7 +1987,7 @@ mod tests { }]; let args = vec![col("b", schema)?]; let func = datafusion_expr::AggregateUDF::new_from_impl(FirstValue::new()); - datafusion_expr_functions_aggregate::aggregate::create_aggregate_expr_with_dfschema( + datafusion_physical_expr_functions_aggregate::aggregate::create_aggregate_expr_with_dfschema( &func, &args, &ordering_req, diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 4c11b8d1a3dd..34dcc7d2858e 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -82,7 +82,7 @@ pub mod windows; pub mod work_table; pub mod udaf { - pub use datafusion_expr_functions_aggregate::aggregate::{ + pub use datafusion_physical_expr_functions_aggregate::aggregate::{ create_aggregate_expr, create_aggregate_expr_with_dfschema, AggregateFunctionExpr, }; } diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 0c8f87449f62..07ecd6d02527 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -35,7 +35,6 @@ use datafusion_expr::{ BuiltInWindowFunction, Expr, PartitionEvaluator, WindowFrame, WindowFunctionDefinition, WindowUDF, }; -use datafusion_expr_functions_aggregate::aggregate::AggregateExprBuilder; use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ reverse_order_bys, @@ -43,6 +42,7 @@ use datafusion_physical_expr::{ AggregateExpr, ConstExpr, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, }; +use datafusion_physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; use itertools::Itertools; mod bounded_window_agg_exec; diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index bf932e4a0f53..b5d28f40a68f 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -18,7 +18,7 @@ use std::fmt::Debug; use std::sync::Arc; -use datafusion::expr_functions_aggregate::aggregate::AggregateExprBuilder; +use datafusion::physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; use prost::bytes::BufMut; use prost::Message; diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 649e9854ae0c..1a9c6d40ebe6 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -24,7 +24,7 @@ use std::vec; use arrow::array::RecordBatch; use arrow::csv::WriterBuilder; -use datafusion::expr_functions_aggregate::aggregate::AggregateExprBuilder; +use datafusion::physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; use datafusion_functions_aggregate::min_max::max_udaf; use prost::Message; From c26e4d52f385546ee21aa7dfcb1b7852eb0ae147 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 6 Aug 2024 23:11:24 +0800 Subject: [PATCH 08/26] fix test Signed-off-by: jayzhan211 --- datafusion/core/Cargo.toml | 2 +- .../src/approx_percentile_cont.rs | 82 ++++++++++--------- datafusion/physical-plan/Cargo.toml | 2 +- 3 files changed, 44 insertions(+), 42 deletions(-) diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 3911f882c7ff..e678c93ede8b 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -103,13 +103,13 @@ datafusion-common = { workspace = true, features = ["object_store"] } datafusion-common-runtime = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } -datafusion-physical-expr-functions-aggregate = { workspace = true } datafusion-functions = { workspace = true } datafusion-functions-aggregate = { workspace = true } datafusion-functions-nested = { workspace = true, optional = true } datafusion-optimizer = { workspace = true } datafusion-physical-expr = { workspace = true } datafusion-physical-expr-common = { workspace = true } +datafusion-physical-expr-functions-aggregate = { workspace = true } datafusion-physical-optimizer = { workspace = true } datafusion-physical-plan = { workspace = true } datafusion-sql = { workspace = true } diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index db9ddaf7174f..9e108af293fd 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -19,7 +19,7 @@ use std::any::Any; use std::fmt::{Debug, Formatter}; use std::sync::Arc; -use arrow::array::Array; +use arrow::array::{Array, RecordBatch}; use arrow::compute::{filter, is_not_null}; use arrow::{ array::{ @@ -28,7 +28,7 @@ use arrow::{ }, datatypes::DataType, }; -use arrow_schema::Field; +use arrow_schema::{Field, Schema}; use datafusion_common::{ downcast_value, internal_err, not_impl_err, plan_err, DataFusionError, ScalarValue, @@ -37,12 +37,11 @@ use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::{INTEGERS, NUMERICS}; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ - Accumulator, AggregateUDFImpl, Signature, TypeSignature, Volatility, + Accumulator, AggregateUDFImpl, ColumnarValue, Signature, TypeSignature, Volatility, }; use datafusion_functions_aggregate_common::tdigest::{ TDigest, TryIntoF64, DEFAULT_MAX_SIZE, }; -use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; make_udaf_expr_and_func!( @@ -133,26 +132,34 @@ impl ApproxPercentileCont { } } +fn get_scalar_value( + expr: &Arc, +) -> datafusion_common::Result { + let empty_schema = Arc::new(Schema::empty()); + let batch = RecordBatch::new_empty(Arc::clone(&empty_schema)); + let val = expr.evaluate(&batch)?; + if let ColumnarValue::Scalar(s) = val { + Ok(s) + } else { + internal_err!("Didn't expect ColumnarValue::Array") + } +} + fn validate_input_percentile_expr( expr: &Arc, ) -> datafusion_common::Result { - let percentile = match expr.as_any().downcast_ref::() { - Some(lit) => match lit.value() { - ScalarValue::Float32(Some(value)) => { - *value as f64 - } - ScalarValue::Float64(Some(value)) => { - *value - } - sv => { - return not_impl_err!( - "Percentile value for 'APPROX_PERCENTILE_CONT' must be Float32 or Float64 literal (got data type {})", - sv.data_type() - ) - } - }, - None => { - return internal_err!("Expect to get a literal expr") + let percentile = match get_scalar_value(expr)? { + ScalarValue::Float32(Some(value)) => { + value as f64 + } + ScalarValue::Float64(Some(value)) => { + value + } + sv => { + return not_impl_err!( + "Percentile value for 'APPROX_PERCENTILE_CONT' must be Float32 or Float64 literal (got data type {})", + sv.data_type() + ) } }; @@ -168,25 +175,20 @@ fn validate_input_percentile_expr( fn validate_input_max_size_expr( expr: &Arc, ) -> datafusion_common::Result { - let max_size = match expr.as_any().downcast_ref::() { - Some(lit) => match lit.value() { - ScalarValue::UInt8(Some(q)) => *q as usize, - ScalarValue::UInt16(Some(q)) => *q as usize, - ScalarValue::UInt32(Some(q)) => *q as usize, - ScalarValue::UInt64(Some(q)) => *q as usize, - ScalarValue::Int32(Some(q)) if *q > 0 => *q as usize, - ScalarValue::Int64(Some(q)) if *q > 0 => *q as usize, - ScalarValue::Int16(Some(q)) if *q > 0 => *q as usize, - ScalarValue::Int8(Some(q)) if *q > 0 => *q as usize, - sv => { - return not_impl_err!( - "Tdigest max_size value for 'APPROX_PERCENTILE_CONT' must be UInt > 0 literal (got data type {}).", - sv.data_type() - ) - } - }, - None => { - return internal_err!("Expect to get a literal expr") + let max_size = match get_scalar_value(expr)? { + ScalarValue::UInt8(Some(q)) => q as usize, + ScalarValue::UInt16(Some(q)) => q as usize, + ScalarValue::UInt32(Some(q)) => q as usize, + ScalarValue::UInt64(Some(q)) => q as usize, + ScalarValue::Int32(Some(q)) if q > 0 => q as usize, + ScalarValue::Int64(Some(q)) if q > 0 => q as usize, + ScalarValue::Int16(Some(q)) if q > 0 => q as usize, + ScalarValue::Int8(Some(q)) if q > 0 => q as usize, + sv => { + return not_impl_err!( + "Tdigest max_size value for 'APPROX_PERCENTILE_CONT' must be UInt > 0 literal (got data type {}).", + sv.data_type() + ) } }; diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 3a5ae6ec04ff..78da4dc9c53f 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -51,11 +51,11 @@ datafusion-common = { workspace = true, default-features = true } datafusion-common-runtime = { workspace = true, default-features = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } -datafusion-physical-expr-functions-aggregate = { workspace = true } datafusion-functions-aggregate = { workspace = true } datafusion-functions-aggregate-common = { workspace = true } datafusion-physical-expr = { workspace = true, default-features = true } datafusion-physical-expr-common = { workspace = true } +datafusion-physical-expr-functions-aggregate = { workspace = true } futures = { workspace = true } half = { workspace = true } hashbrown = { workspace = true } From 366f2bf69a9cf0dfc085e7cc4ef3168ca292b7d3 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 6 Aug 2024 23:43:23 +0800 Subject: [PATCH 09/26] fix test Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/aggregates/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 6f8aba197107..76066350089d 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1992,7 +1992,7 @@ mod tests { &args, &ordering_req, dfschema, - None, + Some(String::from("first_value(b) ORDER BY [b ASC NULLS LAST]")), false, false, false, @@ -2016,7 +2016,7 @@ mod tests { &args, &ordering_req, dfschema, - None, + Some(String::from("last_value(b) ORDER BY [b ASC NULLS LAST]")), false, false, false, From a90b28effa04ae1885c6e3b6859bd165e21d373a Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 6 Aug 2024 23:47:50 +0800 Subject: [PATCH 10/26] doc Signed-off-by: jayzhan211 --- .../expr-common/src/interval_arithmetic.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/datafusion/expr-common/src/interval_arithmetic.rs b/datafusion/expr-common/src/interval_arithmetic.rs index d872b27d3c86..71f31d42d930 100644 --- a/datafusion/expr-common/src/interval_arithmetic.rs +++ b/datafusion/expr-common/src/interval_arithmetic.rs @@ -1177,7 +1177,7 @@ fn min_of_bounds(first: &ScalarValue, second: &ScalarValue) -> ScalarValue { /// Example usage: /// ``` /// use datafusion_common::DataFusionError; -/// use datafusion_expr::interval_arithmetic::{satisfy_greater, Interval}; +/// use crate::interval_arithmetic::{satisfy_greater, Interval}; /// /// let left = Interval::make(Some(-1000.0_f32), Some(1000.0_f32))?; /// let right = Interval::make(Some(500.0_f32), Some(2000.0_f32))?; @@ -1552,8 +1552,8 @@ fn cast_scalar_value( /// ``` /// use arrow::datatypes::DataType; /// use datafusion_common::ScalarValue; -/// use datafusion_expr::interval_arithmetic::Interval; -/// use datafusion_expr::interval_arithmetic::NullableInterval; +/// use crate::interval_arithmetic::Interval; +/// use crate::interval_arithmetic::NullableInterval; /// /// // [1, 2) U {NULL} /// let maybe_null = NullableInterval::MaybeNull { @@ -1674,9 +1674,9 @@ impl NullableInterval { /// /// ``` /// use datafusion_common::ScalarValue; - /// use datafusion_expr::Operator; - /// use datafusion_expr::interval_arithmetic::Interval; - /// use datafusion_expr::interval_arithmetic::NullableInterval; + /// use datafusion_expr_common::operator::Operator; + /// use datafusion_expr_common::interval_arithmetic::Interval; + /// use datafusion_expr_common::interval_arithmetic::NullableInterval; /// /// // 4 > 3 -> true /// let lhs = NullableInterval::from(ScalarValue::Int32(Some(4))); @@ -1798,8 +1798,8 @@ impl NullableInterval { /// /// ``` /// use datafusion_common::ScalarValue; - /// use datafusion_expr::interval_arithmetic::Interval; - /// use datafusion_expr::interval_arithmetic::NullableInterval; + /// use datafusion_expr_common::interval_arithmetic::Interval; + /// use datafusion_expr_common::interval_arithmetic::NullableInterval; /// /// let interval = NullableInterval::from(ScalarValue::Int32(Some(4))); /// assert_eq!(interval.single_value(), Some(ScalarValue::Int32(Some(4)))); From 93a514e58341aea165844ebae8b162712274cc41 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 7 Aug 2024 08:22:06 +0800 Subject: [PATCH 11/26] fix doc and cleanup Signed-off-by: jayzhan211 --- .../expr-common/src/interval_arithmetic.rs | 6 ++--- datafusion/expr-common/src/signature.rs | 2 +- .../functions-aggregate/src/nth_value.rs | 26 +++++++------------ 3 files changed, 13 insertions(+), 21 deletions(-) diff --git a/datafusion/expr-common/src/interval_arithmetic.rs b/datafusion/expr-common/src/interval_arithmetic.rs index 71f31d42d930..3cf9ff9d9284 100644 --- a/datafusion/expr-common/src/interval_arithmetic.rs +++ b/datafusion/expr-common/src/interval_arithmetic.rs @@ -1177,7 +1177,7 @@ fn min_of_bounds(first: &ScalarValue, second: &ScalarValue) -> ScalarValue { /// Example usage: /// ``` /// use datafusion_common::DataFusionError; -/// use crate::interval_arithmetic::{satisfy_greater, Interval}; +/// use datafusion_expr_common::interval_arithmetic::{satisfy_greater, Interval}; /// /// let left = Interval::make(Some(-1000.0_f32), Some(1000.0_f32))?; /// let right = Interval::make(Some(500.0_f32), Some(2000.0_f32))?; @@ -1552,8 +1552,8 @@ fn cast_scalar_value( /// ``` /// use arrow::datatypes::DataType; /// use datafusion_common::ScalarValue; -/// use crate::interval_arithmetic::Interval; -/// use crate::interval_arithmetic::NullableInterval; +/// use datafusion_expr_common::interval_arithmetic::Interval; +/// use datafusion_expr_common::interval_arithmetic::NullableInterval; /// /// // [1, 2) U {NULL} /// let maybe_null = NullableInterval::MaybeNull { diff --git a/datafusion/expr-common/src/signature.rs b/datafusion/expr-common/src/signature.rs index 8db4bc40d78c..33faf3e8a3b8 100644 --- a/datafusion/expr-common/src/signature.rs +++ b/datafusion/expr-common/src/signature.rs @@ -75,7 +75,7 @@ pub enum Volatility { /// /// ``` /// # use arrow::datatypes::{DataType, TimeUnit}; -/// # use datafusion_expr::{TIMEZONE_WILDCARD, TypeSignature}; +/// # use datafusion_expr_common::{TIMEZONE_WILDCARD, TypeSignature}; /// let type_signature = TypeSignature::Exact(vec![ /// // A nanosecond precision timestamp with ANY timezone /// // matches Timestamp(Nanosecond, Some("+0:00")) diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index a2ef3e678fe8..80f97cac5fe6 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -88,29 +88,21 @@ impl AggregateUDFImpl for NthValueAgg { let n = match acc_args.physical_exprs[1] .as_any() .downcast_ref::() + .and_then(|lit| Some(lit.value())) { - Some(lit) => match lit.value() { - ScalarValue::Int64(Some(value)) => { - if acc_args.is_reversed { - -*value - } else { - *value - } + Some(ScalarValue::Int64(Some(value))) => { + if acc_args.is_reversed { + -*value + } else { + *value } - _ => { - return not_impl_err!( - "{} not supported for n: {}", - self.name(), - &acc_args.physical_exprs[1] - ); - } - }, - None => { + } + _ => { return not_impl_err!( "{} not supported for n: {}", self.name(), &acc_args.physical_exprs[1] - ); + ) } }; From 38417364abb6985f66c29378d5709c41c786ff8f Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 7 Aug 2024 08:30:20 +0800 Subject: [PATCH 12/26] fix doc Signed-off-by: jayzhan211 --- datafusion/expr-common/src/signature.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/expr-common/src/signature.rs b/datafusion/expr-common/src/signature.rs index 33faf3e8a3b8..4dcfa423e371 100644 --- a/datafusion/expr-common/src/signature.rs +++ b/datafusion/expr-common/src/signature.rs @@ -75,7 +75,7 @@ pub enum Volatility { /// /// ``` /// # use arrow::datatypes::{DataType, TimeUnit}; -/// # use datafusion_expr_common::{TIMEZONE_WILDCARD, TypeSignature}; +/// # use datafusion_expr_common::signature::{TIMEZONE_WILDCARD, TypeSignature}; /// let type_signature = TypeSignature::Exact(vec![ /// // A nanosecond precision timestamp with ANY timezone /// // matches Timestamp(Nanosecond, Some("+0:00")) From 5ef93e117ba6ee09014b3fd70174cba31ff70098 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 7 Aug 2024 09:18:14 +0800 Subject: [PATCH 13/26] clippy + doc Signed-off-by: jayzhan211 --- datafusion/expr-common/src/lib.rs | 7 +++++++ datafusion/functions-aggregate-common/src/aggregate.rs | 2 +- datafusion/functions-aggregate-common/src/lib.rs | 8 ++++++++ datafusion/functions-aggregate/src/nth_value.rs | 2 +- datafusion/physical-expr-common/src/lib.rs | 5 +++++ datafusion/physical-expr/src/equivalence/class.rs | 2 +- datafusion/physical-expr/src/lib.rs | 1 + 7 files changed, 24 insertions(+), 3 deletions(-) diff --git a/datafusion/expr-common/src/lib.rs b/datafusion/expr-common/src/lib.rs index 2436b25271cf..4d5fdde38d32 100644 --- a/datafusion/expr-common/src/lib.rs +++ b/datafusion/expr-common/src/lib.rs @@ -15,6 +15,13 @@ // specific language governing permissions and limitations // under the License. +//! Logical Expr Common packages for [DataFusion] +//! +//! [DataFusion]: + +// Make cheap clones clear: https://github.com/apache/datafusion/issues/11143 +#![deny(clippy::clone_on_ref_ptr)] + pub mod accumulator; pub mod columnar_value; pub mod groups_accumulator; diff --git a/datafusion/functions-aggregate-common/src/aggregate.rs b/datafusion/functions-aggregate-common/src/aggregate.rs index 98eafd80e3bc..42c61817b806 100644 --- a/datafusion/functions-aggregate-common/src/aggregate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate.rs @@ -141,7 +141,7 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { let order_bys = self.order_bys().unwrap_or(&[]); let order_by_exprs = order_bys .iter() - .map(|sort_expr| sort_expr.expr.clone()) + .map(|sort_expr| Arc::clone(&sort_expr.expr)) .collect::>(); AggregatePhysicalExpressions { args, diff --git a/datafusion/functions-aggregate-common/src/lib.rs b/datafusion/functions-aggregate-common/src/lib.rs index a4fcb4aa9de9..eb86e05e2af1 100644 --- a/datafusion/functions-aggregate-common/src/lib.rs +++ b/datafusion/functions-aggregate-common/src/lib.rs @@ -15,6 +15,14 @@ // specific language governing permissions and limitations // under the License. +//! Aggregate Function Common packages for [DataFusion] +//! This package could be used to build for 3rd party aggregate function +//! +//! [DataFusion]: + +// Make cheap clones clear: https://github.com/apache/datafusion/issues/11143 +#![deny(clippy::clone_on_ref_ptr)] + pub mod accumulator; pub mod aggregate; pub mod merge_arrays; diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index 80f97cac5fe6..3073a84f74dd 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -88,7 +88,7 @@ impl AggregateUDFImpl for NthValueAgg { let n = match acc_args.physical_exprs[1] .as_any() .downcast_ref::() - .and_then(|lit| Some(lit.value())) + .map(|lit| lit.value()) { Some(ScalarValue::Int64(Some(value))) => { if acc_args.is_reversed { diff --git a/datafusion/physical-expr-common/src/lib.rs b/datafusion/physical-expr-common/src/lib.rs index c186d16aa1ea..7e2ea0c49397 100644 --- a/datafusion/physical-expr-common/src/lib.rs +++ b/datafusion/physical-expr-common/src/lib.rs @@ -15,6 +15,11 @@ // specific language governing permissions and limitations // under the License. +//! Physical Expr Common packages for [DataFusion] +//! This package contains high level PhysicalExpr trait +//! +//! [DataFusion]: + pub mod binary_map; pub mod binary_view_map; pub mod datum; diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index ffa58e385322..0296b7a247d6 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -47,7 +47,7 @@ use datafusion_common::JoinType; /// /// ```rust /// # use datafusion_physical_expr::ConstExpr; -/// # use datafusion_physical_expr_common::expressions::lit; +/// # use datafusion_physical_expr::expressions::lit; /// let col = lit(5); /// // Create a constant expression from a physical expression ref /// let const_expr = ConstExpr::from(&col); diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 3d2ae2651bfc..9a9120f1ddac 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -14,6 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. + // Make cheap clones clear: https://github.com/apache/datafusion/issues/11143 #![deny(clippy::clone_on_ref_ptr)] From 4b229959f1edbe1d90ac6ade56910bfa2fb4a722 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 7 Aug 2024 19:36:01 +0800 Subject: [PATCH 14/26] cleanup Signed-off-by: jayzhan211 --- .../src/physical_optimizer/limit_pushdown.rs | 3 +- .../expr-common/src/interval_arithmetic.rs | 4 +- datafusion/expr/src/lib.rs | 2 +- .../expr/src/{operator.rs => operation.rs} | 0 .../src/aggregate.rs | 3 -- .../src/approx_percentile_cont.rs | 53 +++++++++---------- 6 files changed, 29 insertions(+), 36 deletions(-) rename datafusion/expr/src/{operator.rs => operation.rs} (100%) diff --git a/datafusion/core/src/physical_optimizer/limit_pushdown.rs b/datafusion/core/src/physical_optimizer/limit_pushdown.rs index 732acc520498..6da227c73ecc 100644 --- a/datafusion/core/src/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/limit_pushdown.rs @@ -257,8 +257,7 @@ mod tests { use arrow_schema::{DataType, Field, Schema, SchemaRef}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr::expressions::lit; + use datafusion_physical_expr::expressions::{col, lit}; use datafusion_physical_expr::expressions::BinaryExpr; use datafusion_physical_expr::Partitioning; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; diff --git a/datafusion/expr-common/src/interval_arithmetic.rs b/datafusion/expr-common/src/interval_arithmetic.rs index 3cf9ff9d9284..e1b14abaa3e8 100644 --- a/datafusion/expr-common/src/interval_arithmetic.rs +++ b/datafusion/expr-common/src/interval_arithmetic.rs @@ -19,14 +19,12 @@ use crate::operator::Operator; use crate::type_coercion::binary::get_result_type; -use arrow::datatypes::{IntervalDayTime, IntervalMonthDayNano}; use std::borrow::Borrow; use std::fmt::{self, Display, Formatter}; use std::ops::{AddAssign, SubAssign}; use arrow::compute::{cast_with_options, CastOptions}; -use arrow::datatypes::DataType; -use arrow::datatypes::{IntervalUnit, TimeUnit}; +use arrow::datatypes::{DataType, IntervalDayTime, IntervalMonthDayNano, IntervalUnit, TimeUnit}; use datafusion_common::rounding::{alter_fp_rounding_mode, next_down, next_up}; use datafusion_common::{internal_err, Result, ScalarValue}; diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index 69fb9b61784b..260065f69af9 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -29,7 +29,7 @@ mod built_in_window_function; mod literal; -mod operator; +mod operation; mod partition_evaluator; mod table_source; mod udaf; diff --git a/datafusion/expr/src/operator.rs b/datafusion/expr/src/operation.rs similarity index 100% rename from datafusion/expr/src/operator.rs rename to datafusion/expr/src/operation.rs diff --git a/datafusion/functions-aggregate-common/src/aggregate.rs b/datafusion/functions-aggregate-common/src/aggregate.rs index 42c61817b806..d6d49a2e4bc6 100644 --- a/datafusion/functions-aggregate-common/src/aggregate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate.rs @@ -152,9 +152,6 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { /// Rewrites [`AggregateExpr`], with new expressions given. The argument should be consistent /// with the return value of the [`AggregateExpr::all_expressions`] method. /// Returns `Some(Arc)` if re-write is supported, otherwise returns `None`. - /// TODO: This method only rewrites the [`PhysicalExpr`]s and does not handle `Expr`s. - /// This can cause silent bugs and should be fixed in the future (possibly with physical-to-logical - /// conversions). fn with_new_expressions( &self, _args: Vec>, diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index 9e108af293fd..a1f7cb87accf 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -31,7 +31,7 @@ use arrow::{ use arrow_schema::{Field, Schema}; use datafusion_common::{ - downcast_value, internal_err, not_impl_err, plan_err, DataFusionError, ScalarValue, + downcast_value, internal_err, not_impl_err, plan_err, DataFusionError, ScalarValue, Result, }; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::{INTEGERS, NUMERICS}; @@ -95,7 +95,7 @@ impl ApproxPercentileCont { pub(crate) fn create_accumulator( &self, args: AccumulatorArgs, - ) -> datafusion_common::Result { + ) -> Result { let percentile = validate_input_percentile_expr(&args.physical_exprs[1])?; let tdigest_max_size = if args.physical_exprs.len() == 3 { Some(validate_input_max_size_expr(&args.physical_exprs[2])?) @@ -134,11 +134,10 @@ impl ApproxPercentileCont { fn get_scalar_value( expr: &Arc, -) -> datafusion_common::Result { +) -> Result { let empty_schema = Arc::new(Schema::empty()); let batch = RecordBatch::new_empty(Arc::clone(&empty_schema)); - let val = expr.evaluate(&batch)?; - if let ColumnarValue::Scalar(s) = val { + if let ColumnarValue::Scalar(s) = expr.evaluate(&batch)? { Ok(s) } else { internal_err!("Didn't expect ColumnarValue::Array") @@ -147,7 +146,7 @@ fn get_scalar_value( fn validate_input_percentile_expr( expr: &Arc, -) -> datafusion_common::Result { +) -> Result { let percentile = match get_scalar_value(expr)? { ScalarValue::Float32(Some(value)) => { value as f64 @@ -174,7 +173,7 @@ fn validate_input_percentile_expr( fn validate_input_max_size_expr( expr: &Arc, -) -> datafusion_common::Result { +) -> Result { let max_size = match get_scalar_value(expr)? { ScalarValue::UInt8(Some(q)) => q as usize, ScalarValue::UInt16(Some(q)) => q as usize, @@ -206,7 +205,7 @@ impl AggregateUDFImpl for ApproxPercentileCont { fn state_fields( &self, args: StateFieldsArgs, - ) -> datafusion_common::Result> { + ) -> Result> { Ok(vec![ Field::new( format_state_name(args.name, "max_size"), @@ -253,11 +252,11 @@ impl AggregateUDFImpl for ApproxPercentileCont { fn accumulator( &self, acc_args: AccumulatorArgs, - ) -> datafusion_common::Result> { + ) -> Result> { Ok(Box::new(self.create_accumulator(acc_args)?)) } - fn return_type(&self, arg_types: &[DataType]) -> datafusion_common::Result { + fn return_type(&self, arg_types: &[DataType]) -> Result { if !arg_types[0].is_numeric() { return plan_err!("approx_percentile_cont requires numeric input types"); } @@ -305,7 +304,7 @@ impl ApproxPercentileAccumulator { } // public for approx_percentile_cont_with_weight - pub fn convert_to_float(values: &ArrayRef) -> datafusion_common::Result> { + pub fn convert_to_float(values: &ArrayRef) -> Result> { match values.data_type() { DataType::Float64 => { let array = downcast_value!(values, Float64Array); @@ -313,7 +312,7 @@ impl ApproxPercentileAccumulator { .values() .iter() .filter_map(|v| v.try_as_f64().transpose()) - .collect::>>()?) + .collect::>>()?) } DataType::Float32 => { let array = downcast_value!(values, Float32Array); @@ -321,7 +320,7 @@ impl ApproxPercentileAccumulator { .values() .iter() .filter_map(|v| v.try_as_f64().transpose()) - .collect::>>()?) + .collect::>>()?) } DataType::Int64 => { let array = downcast_value!(values, Int64Array); @@ -329,7 +328,7 @@ impl ApproxPercentileAccumulator { .values() .iter() .filter_map(|v| v.try_as_f64().transpose()) - .collect::>>()?) + .collect::>>()?) } DataType::Int32 => { let array = downcast_value!(values, Int32Array); @@ -337,7 +336,7 @@ impl ApproxPercentileAccumulator { .values() .iter() .filter_map(|v| v.try_as_f64().transpose()) - .collect::>>()?) + .collect::>>()?) } DataType::Int16 => { let array = downcast_value!(values, Int16Array); @@ -345,7 +344,7 @@ impl ApproxPercentileAccumulator { .values() .iter() .filter_map(|v| v.try_as_f64().transpose()) - .collect::>>()?) + .collect::>>()?) } DataType::Int8 => { let array = downcast_value!(values, Int8Array); @@ -353,7 +352,7 @@ impl ApproxPercentileAccumulator { .values() .iter() .filter_map(|v| v.try_as_f64().transpose()) - .collect::>>()?) + .collect::>>()?) } DataType::UInt64 => { let array = downcast_value!(values, UInt64Array); @@ -361,7 +360,7 @@ impl ApproxPercentileAccumulator { .values() .iter() .filter_map(|v| v.try_as_f64().transpose()) - .collect::>>()?) + .collect::>>()?) } DataType::UInt32 => { let array = downcast_value!(values, UInt32Array); @@ -369,7 +368,7 @@ impl ApproxPercentileAccumulator { .values() .iter() .filter_map(|v| v.try_as_f64().transpose()) - .collect::>>()?) + .collect::>>()?) } DataType::UInt16 => { let array = downcast_value!(values, UInt16Array); @@ -377,7 +376,7 @@ impl ApproxPercentileAccumulator { .values() .iter() .filter_map(|v| v.try_as_f64().transpose()) - .collect::>>()?) + .collect::>>()?) } DataType::UInt8 => { let array = downcast_value!(values, UInt8Array); @@ -385,7 +384,7 @@ impl ApproxPercentileAccumulator { .values() .iter() .filter_map(|v| v.try_as_f64().transpose()) - .collect::>>()?) + .collect::>>()?) } e => internal_err!( "APPROX_PERCENTILE_CONT is not expected to receive the type {e:?}" @@ -395,11 +394,11 @@ impl ApproxPercentileAccumulator { } impl Accumulator for ApproxPercentileAccumulator { - fn state(&mut self) -> datafusion_common::Result> { + fn state(&mut self) -> Result> { Ok(self.digest.to_scalar_state().into_iter().collect()) } - fn update_batch(&mut self, values: &[ArrayRef]) -> datafusion_common::Result<()> { + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { // Remove any nulls before computing the percentile let mut values = Arc::clone(&values[0]); if values.nulls().is_some() { @@ -411,7 +410,7 @@ impl Accumulator for ApproxPercentileAccumulator { Ok(()) } - fn evaluate(&mut self) -> datafusion_common::Result { + fn evaluate(&mut self) -> Result { if self.digest.count() == 0 { return ScalarValue::try_from(self.return_type.clone()); } @@ -434,7 +433,7 @@ impl Accumulator for ApproxPercentileAccumulator { }) } - fn merge_batch(&mut self, states: &[ArrayRef]) -> datafusion_common::Result<()> { + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { if states.is_empty() { return Ok(()); } @@ -444,10 +443,10 @@ impl Accumulator for ApproxPercentileAccumulator { states .iter() .map(|array| ScalarValue::try_from_array(array, index)) - .collect::>>() + .collect::>>() .map(|state| TDigest::from_scalar_state(&state)) }) - .collect::>>()?; + .collect::>>()?; self.merge_digests(&states); From e7f0edc630cd4d764afd65562d055fad6b648627 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 7 Aug 2024 19:49:22 +0800 Subject: [PATCH 15/26] cleanup Signed-off-by: jayzhan211 --- .../src/physical_optimizer/limit_pushdown.rs | 2 +- .../expr-common/src/interval_arithmetic.rs | 4 ++- .../src/approx_percentile_cont.rs | 25 ++++++------------- datafusion/physical-expr-common/src/utils.rs | 3 --- datafusion/physical-expr/benches/case_when.rs | 3 +-- datafusion/physical-expr/benches/is_null.rs | 3 +-- .../physical-expr/src/expressions/binary.rs | 4 +-- .../physical-expr/src/expressions/case.rs | 6 ++--- 8 files changed, 16 insertions(+), 34 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/limit_pushdown.rs b/datafusion/core/src/physical_optimizer/limit_pushdown.rs index 6da227c73ecc..d02737ff0959 100644 --- a/datafusion/core/src/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/limit_pushdown.rs @@ -257,8 +257,8 @@ mod tests { use arrow_schema::{DataType, Field, Schema, SchemaRef}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::{col, lit}; use datafusion_physical_expr::expressions::BinaryExpr; + use datafusion_physical_expr::expressions::{col, lit}; use datafusion_physical_expr::Partitioning; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; diff --git a/datafusion/expr-common/src/interval_arithmetic.rs b/datafusion/expr-common/src/interval_arithmetic.rs index e1b14abaa3e8..e3ff412e785b 100644 --- a/datafusion/expr-common/src/interval_arithmetic.rs +++ b/datafusion/expr-common/src/interval_arithmetic.rs @@ -24,7 +24,9 @@ use std::fmt::{self, Display, Formatter}; use std::ops::{AddAssign, SubAssign}; use arrow::compute::{cast_with_options, CastOptions}; -use arrow::datatypes::{DataType, IntervalDayTime, IntervalMonthDayNano, IntervalUnit, TimeUnit}; +use arrow::datatypes::{ + DataType, IntervalDayTime, IntervalMonthDayNano, IntervalUnit, TimeUnit, +}; use datafusion_common::rounding::{alter_fp_rounding_mode, next_down, next_up}; use datafusion_common::{internal_err, Result, ScalarValue}; diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index a1f7cb87accf..77cc2798d93f 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -31,7 +31,8 @@ use arrow::{ use arrow_schema::{Field, Schema}; use datafusion_common::{ - downcast_value, internal_err, not_impl_err, plan_err, DataFusionError, ScalarValue, Result, + downcast_value, internal_err, not_impl_err, plan_err, DataFusionError, Result, + ScalarValue, }; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::{INTEGERS, NUMERICS}; @@ -132,9 +133,7 @@ impl ApproxPercentileCont { } } -fn get_scalar_value( - expr: &Arc, -) -> Result { +fn get_scalar_value(expr: &Arc) -> Result { let empty_schema = Arc::new(Schema::empty()); let batch = RecordBatch::new_empty(Arc::clone(&empty_schema)); if let ColumnarValue::Scalar(s) = expr.evaluate(&batch)? { @@ -144,9 +143,7 @@ fn get_scalar_value( } } -fn validate_input_percentile_expr( - expr: &Arc, -) -> Result { +fn validate_input_percentile_expr(expr: &Arc) -> Result { let percentile = match get_scalar_value(expr)? { ScalarValue::Float32(Some(value)) => { value as f64 @@ -171,9 +168,7 @@ fn validate_input_percentile_expr( Ok(percentile) } -fn validate_input_max_size_expr( - expr: &Arc, -) -> Result { +fn validate_input_max_size_expr(expr: &Arc) -> Result { let max_size = match get_scalar_value(expr)? { ScalarValue::UInt8(Some(q)) => q as usize, ScalarValue::UInt16(Some(q)) => q as usize, @@ -202,10 +197,7 @@ impl AggregateUDFImpl for ApproxPercentileCont { #[allow(rustdoc::private_intra_doc_links)] /// See [`TDigest::to_scalar_state()`] for a description of the serialised /// state. - fn state_fields( - &self, - args: StateFieldsArgs, - ) -> Result> { + fn state_fields(&self, args: StateFieldsArgs) -> Result> { Ok(vec![ Field::new( format_state_name(args.name, "max_size"), @@ -249,10 +241,7 @@ impl AggregateUDFImpl for ApproxPercentileCont { } #[inline] - fn accumulator( - &self, - acc_args: AccumulatorArgs, - ) -> Result> { + fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { Ok(Box::new(self.create_accumulator(acc_args)?)) } diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index 86cbb3ea8e05..d2c9bf1a2408 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -21,9 +21,6 @@ use arrow::array::{make_array, Array, ArrayRef, BooleanArray, MutableArrayData}; use arrow::compute::{and_kleene, is_not_null, SlicesIterator}; use datafusion_common::Result; -// use datafusion_expr::expr::Alias; -// use datafusion_expr::sort_properties::ExprProperties; -// use datafusion_expr::Expr; use datafusion_expr_common::sort_properties::ExprProperties; use crate::physical_expr::PhysicalExpr; diff --git a/datafusion/physical-expr/benches/case_when.rs b/datafusion/physical-expr/benches/case_when.rs index b863b79b6a5c..8a34f34a82db 100644 --- a/datafusion/physical-expr/benches/case_when.rs +++ b/datafusion/physical-expr/benches/case_when.rs @@ -22,8 +22,7 @@ use arrow_schema::DataType; use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_common::ScalarValue; use datafusion_expr::Operator; -use datafusion_physical_expr::expressions::{BinaryExpr, CaseExpr}; -use datafusion_physical_expr::expressions::{Column, Literal}; +use datafusion_physical_expr::expressions::{BinaryExpr, CaseExpr, Column, Literal}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use std::sync::Arc; diff --git a/datafusion/physical-expr/benches/is_null.rs b/datafusion/physical-expr/benches/is_null.rs index 2880a3a62fb9..7d26557afb1b 100644 --- a/datafusion/physical-expr/benches/is_null.rs +++ b/datafusion/physical-expr/benches/is_null.rs @@ -20,8 +20,7 @@ use arrow::record_batch::RecordBatch; use arrow_array::builder::Int32Builder; use arrow_schema::DataType; use criterion::{black_box, criterion_group, criterion_main, Criterion}; -use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::expressions::{IsNotNullExpr, IsNullExpr}; +use datafusion_physical_expr::expressions::{Column, IsNotNullExpr, IsNullExpr}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use std::sync::Arc; diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index 3896f89393ea..347a5d82dbec 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -681,9 +681,7 @@ pub fn binary( #[cfg(test)] mod tests { use super::*; - use crate::expressions::{col, lit, try_cast, Literal}; - - use crate::expressions::Column; + use crate::expressions::{col, lit, try_cast, Column, Literal}; use datafusion_common::plan_datafusion_err; use datafusion_expr::type_coercion::binary::get_input_types; diff --git a/datafusion/physical-expr/src/expressions/case.rs b/datafusion/physical-expr/src/expressions/case.rs index 4a3fe7178a77..bfab0dd59bd2 100644 --- a/datafusion/physical-expr/src/expressions/case.rs +++ b/datafusion/physical-expr/src/expressions/case.rs @@ -32,8 +32,7 @@ use datafusion_common::cast::as_boolean_array; use datafusion_common::{exec_err, internal_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::ColumnarValue; -use super::Column; -use super::Literal; +use super::{Column, Literal}; use itertools::Itertools; type WhenThen = (Arc, Arc); @@ -548,12 +547,11 @@ pub fn case( #[cfg(test)] mod tests { use super::*; - use crate::expressions::{binary, cast, col, lit, BinaryExpr}; - use super::Literal; use arrow::buffer::Buffer; use arrow::datatypes::DataType::Float64; use arrow::datatypes::*; + use crate::expressions::{binary, cast, col, lit, BinaryExpr}; use datafusion_common::cast::{as_float64_array, as_int32_array}; use datafusion_common::plan_err; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; From cfd173412b2fb1a1d507c4e799e8ef4dce7a4589 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 7 Aug 2024 20:01:04 +0800 Subject: [PATCH 16/26] rename exprs Signed-off-by: jayzhan211 --- datafusion/functions-aggregate-common/src/accumulator.rs | 2 +- datafusion/functions-aggregate/benches/count.rs | 2 +- datafusion/functions-aggregate/benches/sum.rs | 2 +- .../functions-aggregate/src/approx_percentile_cont.rs | 6 +++--- .../src/approx_percentile_cont_with_weight.rs | 8 ++++---- datafusion/functions-aggregate/src/count.rs | 4 ++-- datafusion/functions-aggregate/src/nth_value.rs | 4 ++-- datafusion/functions-aggregate/src/stddev.rs | 4 ++-- datafusion/functions-aggregate/src/string_agg.rs | 2 +- .../physical-expr-functions-aggregate/src/aggregate.rs | 8 ++++---- datafusion/physical-expr/src/expressions/case.rs | 2 +- 11 files changed, 22 insertions(+), 22 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/accumulator.rs b/datafusion/functions-aggregate-common/src/accumulator.rs index 3c31f4a5e9a6..b123184e35a8 100644 --- a/datafusion/functions-aggregate-common/src/accumulator.rs +++ b/datafusion/functions-aggregate-common/src/accumulator.rs @@ -76,7 +76,7 @@ pub struct AccumulatorArgs<'a> { pub input_types: &'a [DataType], /// The physical expression of arguments the aggregate function takes. - pub physical_exprs: &'a [Arc], + pub exprs: &'a [Arc], } /// Factory that returns an accumulator for the given aggregate function. diff --git a/datafusion/functions-aggregate/benches/count.rs b/datafusion/functions-aggregate/benches/count.rs index fa1051e945b1..28e4579f2a7e 100644 --- a/datafusion/functions-aggregate/benches/count.rs +++ b/datafusion/functions-aggregate/benches/count.rs @@ -39,7 +39,7 @@ fn prepare_accumulator() -> Box { name: "COUNT(f)", is_distinct: false, input_types: &[DataType::Int32], - physical_exprs: &[col("f", &schema).unwrap()], + exprs: &[col("f", &schema).unwrap()], }; let count_fn = Count::new(); diff --git a/datafusion/functions-aggregate/benches/sum.rs b/datafusion/functions-aggregate/benches/sum.rs index c800e09aa99b..a11b06cba54f 100644 --- a/datafusion/functions-aggregate/benches/sum.rs +++ b/datafusion/functions-aggregate/benches/sum.rs @@ -39,7 +39,7 @@ fn prepare_accumulator(data_type: &DataType) -> Box { name: "SUM(f)", is_distinct: false, input_types: &[data_type.clone()], - physical_exprs: &[col("f", &schema).unwrap()], + exprs: &[col("f", &schema).unwrap()], }; let sum_fn = Sum::new(); diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index 77cc2798d93f..f95b33bd0332 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -97,9 +97,9 @@ impl ApproxPercentileCont { &self, args: AccumulatorArgs, ) -> Result { - let percentile = validate_input_percentile_expr(&args.physical_exprs[1])?; - let tdigest_max_size = if args.physical_exprs.len() == 3 { - Some(validate_input_max_size_expr(&args.physical_exprs[2])?) + let percentile = validate_input_percentile_expr(&args.exprs[1])?; + let tdigest_max_size = if args.exprs.len() == 3 { + Some(validate_input_max_size_expr(&args.exprs[2])?) } else { None }; diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs b/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs index 917da47ed669..fee67ba1623d 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs @@ -124,16 +124,16 @@ impl AggregateUDFImpl for ApproxPercentileContWithWeight { ); } - if acc_args.physical_exprs.len() != 3 { + if acc_args.exprs.len() != 3 { return plan_err!( "approx_percentile_cont_with_weight requires three arguments: value, weight, percentile" ); } let sub_args = AccumulatorArgs { - physical_exprs: &[ - Arc::clone(&acc_args.physical_exprs[0]), - Arc::clone(&acc_args.physical_exprs[2]), + exprs: &[ + Arc::clone(&acc_args.exprs[0]), + Arc::clone(&acc_args.exprs[2]), ], ..acc_args }; diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index f41404eef880..b4c37fe09fe2 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -143,7 +143,7 @@ impl AggregateUDFImpl for Count { return Ok(Box::new(CountAccumulator::new())); } - if acc_args.physical_exprs.len() > 1 { + if acc_args.exprs.len() > 1 { return not_impl_err!("COUNT DISTINCT with multiple arguments"); } @@ -269,7 +269,7 @@ impl AggregateUDFImpl for Count { if args.is_distinct { return false; } - args.physical_exprs.len() == 1 + args.exprs.len() == 1 } fn create_groups_accumulator( diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index 3073a84f74dd..b19d32c35acd 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -85,7 +85,7 @@ impl AggregateUDFImpl for NthValueAgg { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - let n = match acc_args.physical_exprs[1] + let n = match acc_args.exprs[1] .as_any() .downcast_ref::() .map(|lit| lit.value()) @@ -101,7 +101,7 @@ impl AggregateUDFImpl for NthValueAgg { return not_impl_err!( "{} not supported for n: {}", self.name(), - &acc_args.physical_exprs[1] + &acc_args.exprs[1] ) } }; diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index 7f0fd47b7a98..7083ba1afbea 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -336,7 +336,7 @@ mod tests { is_distinct: false, is_reversed: false, input_types: &[DataType::Float64], - physical_exprs: &[col("a", schema)?], + exprs: &[col("a", schema)?], }; let args2 = AccumulatorArgs { @@ -350,7 +350,7 @@ mod tests { is_distinct: false, is_reversed: false, input_types: &[DataType::Float64], - physical_exprs: &[col("a", schema)?], + exprs: &[col("a", schema)?], }; let mut accum1 = agg1.accumulator(args1)?; diff --git a/datafusion/functions-aggregate/src/string_agg.rs b/datafusion/functions-aggregate/src/string_agg.rs index 9000f652c6b8..5b2548507c0d 100644 --- a/datafusion/functions-aggregate/src/string_agg.rs +++ b/datafusion/functions-aggregate/src/string_agg.rs @@ -83,7 +83,7 @@ impl AggregateUDFImpl for StringAgg { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - if let Some(lit) = acc_args.physical_exprs[1] + if let Some(lit) = acc_args.exprs[1] .as_any() .downcast_ref::() { diff --git a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs b/datafusion/physical-expr-functions-aggregate/src/aggregate.rs index daabfcef3990..cc19f16f8020 100644 --- a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs +++ b/datafusion/physical-expr-functions-aggregate/src/aggregate.rs @@ -345,7 +345,7 @@ impl AggregateExpr for AggregateFunctionExpr { input_types: &self.input_types, name: &self.name, is_reversed: self.is_reversed, - physical_exprs: &self.args, + exprs: &self.args, }; self.fun.accumulator(acc_args) @@ -362,7 +362,7 @@ impl AggregateExpr for AggregateFunctionExpr { input_types: &self.input_types, name: &self.name, is_reversed: self.is_reversed, - physical_exprs: &self.args, + exprs: &self.args, }; let accumulator = self.fun.create_sliding_accumulator(args)?; @@ -434,7 +434,7 @@ impl AggregateExpr for AggregateFunctionExpr { input_types: &self.input_types, name: &self.name, is_reversed: self.is_reversed, - physical_exprs: &self.args, + exprs: &self.args, }; self.fun.groups_accumulator_supported(args) } @@ -450,7 +450,7 @@ impl AggregateExpr for AggregateFunctionExpr { input_types: &self.input_types, name: &self.name, is_reversed: self.is_reversed, - physical_exprs: &self.args, + exprs: &self.args, }; self.fun.create_groups_accumulator(args) } diff --git a/datafusion/physical-expr/src/expressions/case.rs b/datafusion/physical-expr/src/expressions/case.rs index bfab0dd59bd2..583a4ef32542 100644 --- a/datafusion/physical-expr/src/expressions/case.rs +++ b/datafusion/physical-expr/src/expressions/case.rs @@ -548,10 +548,10 @@ pub fn case( mod tests { use super::*; + use crate::expressions::{binary, cast, col, lit, BinaryExpr}; use arrow::buffer::Buffer; use arrow::datatypes::DataType::Float64; use arrow::datatypes::*; - use crate::expressions::{binary, cast, col, lit, BinaryExpr}; use datafusion_common::cast::{as_float64_array, as_int32_array}; use datafusion_common::plan_err; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; From 061d4dd91951bed14f5748b0d141ed027769cb7f Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 7 Aug 2024 21:05:40 +0800 Subject: [PATCH 17/26] rm create_aggregate_expr_with_dfschema Signed-off-by: jayzhan211 --- datafusion/core/src/physical_planner.rs | 34 ++--- .../functions-aggregate/src/string_agg.rs | 5 +- .../src/aggregate.rs | 84 ++++-------- .../physical-plan/src/aggregates/mod.rs | 128 ++++++------------ datafusion/physical-plan/src/lib.rs | 4 +- 5 files changed, 88 insertions(+), 167 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 9e878c6597e2..5186ce3a96ea 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -58,7 +58,7 @@ use crate::physical_plan::unnest::UnnestExec; use crate::physical_plan::values::ValuesExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{ - displayable, udaf, windows, AggregateExpr, ExecutionPlan, ExecutionPlanProperties, + displayable, windows, AggregateExpr, ExecutionPlan, ExecutionPlanProperties, InputOrderMode, Partitioning, PhysicalExpr, WindowExpr, }; @@ -84,6 +84,7 @@ use datafusion_expr::{ }; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::LexOrdering; +use datafusion_physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_sql::utils::window_expr_common_partition_keys; @@ -1560,16 +1561,15 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( order_by, null_treatment, }) => { - let name = if name.is_none() { - let name = create_function_physical_name( + let name = if let Some(name) = name { + name + } else { + create_function_physical_name( func.name(), *distinct, args, order_by.as_ref(), - )?; - Some(name) - } else { - name + )? }; let physical_args = @@ -1600,16 +1600,16 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( let ordering_reqs: Vec = physical_sort_exprs.clone().unwrap_or(vec![]); - let agg_expr = udaf::create_aggregate_expr_with_dfschema( - func, - &physical_args, - &ordering_reqs, - logical_input_schema, - name, - ignore_nulls, - *distinct, - false, - )?; + let schema: Schema = logical_input_schema.clone().into(); + let agg_expr = + AggregateExprBuilder::new(func.to_owned(), physical_args.to_vec()) + .order_by(ordering_reqs.to_vec()) + .schema(Arc::new(schema)) + .dfschema(logical_input_schema.clone()) + .alias(name) + .with_ignore_nulls(ignore_nulls) + .with_distinct(*distinct) + .build()?; (agg_expr, filter, physical_sort_exprs) }; diff --git a/datafusion/functions-aggregate/src/string_agg.rs b/datafusion/functions-aggregate/src/string_agg.rs index 5b2548507c0d..a7e9a37e23ad 100644 --- a/datafusion/functions-aggregate/src/string_agg.rs +++ b/datafusion/functions-aggregate/src/string_agg.rs @@ -83,10 +83,7 @@ impl AggregateUDFImpl for StringAgg { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - if let Some(lit) = acc_args.exprs[1] - .as_any() - .downcast_ref::() - { + if let Some(lit) = acc_args.exprs[1].as_any().downcast_ref::() { return match lit.value() { ScalarValue::Utf8(Some(delimiter)) | ScalarValue::LargeUtf8(Some(delimiter)) => { diff --git a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs b/datafusion/physical-expr-functions-aggregate/src/aggregate.rs index cc19f16f8020..04a048a5f5d8 100644 --- a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs +++ b/datafusion/physical-expr-functions-aggregate/src/aggregate.rs @@ -52,6 +52,7 @@ use std::{any::Any, sync::Arc}; /// /// You can also create expression by [`AggregateExprBuilder`] #[allow(clippy::too_many_arguments)] +#[deprecated(since = "40.0.0", note = "Use `AggregateExprBuilder` instead")] pub fn create_aggregate_expr( fun: &AggregateUDF, input_phy_exprs: &[Arc], @@ -79,41 +80,6 @@ pub fn create_aggregate_expr( builder.build() } -#[allow(clippy::too_many_arguments)] -// This is not for external usage, consider creating with `create_aggregate_expr` instead. -pub fn create_aggregate_expr_with_dfschema( - fun: &AggregateUDF, - input_phy_exprs: &[Arc], - ordering_req: &[PhysicalSortExpr], - dfschema: &DFSchema, - alias: Option, - ignore_nulls: bool, - is_distinct: bool, - is_reversed: bool, -) -> Result> { - let mut builder = - AggregateExprBuilder::new(Arc::new(fun.clone()), input_phy_exprs.to_vec()); - builder = builder.order_by(ordering_req.to_vec()); - builder = builder.dfschema(dfschema.clone()); - let schema: Schema = dfschema.into(); - builder = builder.schema(Arc::new(schema)); - if let Some(alias) = alias { - builder = builder.alias(alias); - } - - if ignore_nulls { - builder = builder.ignore_nulls(); - } - if is_distinct { - builder = builder.distinct(); - } - if is_reversed { - builder = builder.reversed(); - } - - builder.build() -} - /// Builder for physical [`AggregateExpr`] /// /// `AggregateExpr` contains the information necessary to call @@ -488,17 +454,18 @@ impl AggregateExpr for AggregateFunctionExpr { else { return Ok(None); }; - create_aggregate_expr_with_dfschema( - &updated_fn, - &self.args, - &self.ordering_req, - &self.dfschema, - Some(self.name().to_string()), - self.ignore_nulls, - self.is_distinct, - self.is_reversed, - ) - .map(Some) + + let schema: Schema = self.dfschema.clone().into(); + AggregateExprBuilder::new(Arc::new(updated_fn), self.args.to_vec()) + .order_by(self.ordering_req.to_vec()) + .schema(Arc::new(schema)) + .dfschema(self.dfschema.clone()) + .alias(self.name().to_string()) + .with_ignore_nulls(self.ignore_nulls) + .with_distinct(self.is_distinct) + .with_reversed(self.is_reversed) + .build() + .map(Some) } fn reverse_expr(&self) -> Option> { @@ -515,19 +482,18 @@ impl AggregateExpr for AggregateFunctionExpr { replace_order_by_clause(&mut name); } replace_fn_name_clause(&mut name, self.fun.name(), reverse_udf.name()); - let reverse_aggr = create_aggregate_expr_with_dfschema( - &reverse_udf, - &self.args, - &reverse_ordering_req, - &self.dfschema, - Some(name), - self.ignore_nulls, - self.is_distinct, - !self.is_reversed, - ) - .unwrap(); - - Some(reverse_aggr) + + let schema: Schema = self.dfschema.clone().into(); + AggregateExprBuilder::new(reverse_udf, self.args.to_vec()) + .order_by(reverse_ordering_req.to_vec()) + .schema(Arc::new(schema)) + .dfschema(self.dfschema.clone()) + .alias(name) + .with_ignore_nulls(self.ignore_nulls) + .with_distinct(self.is_distinct) + .with_reversed(!self.is_reversed) + .build() + .ok() } } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 76066350089d..4d39eff42b5f 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1203,8 +1203,8 @@ mod tests { use arrow::datatypes::DataType; use arrow_array::{Float32Array, Int32Array}; use datafusion_common::{ - assert_batches_eq, assert_batches_sorted_eq, internal_err, DFSchema, DFSchemaRef, - DataFusionError, ScalarValue, + assert_batches_eq, assert_batches_sorted_eq, internal_err, DataFusionError, + ScalarValue, }; use datafusion_execution::config::SessionConfig; use datafusion_execution::memory_pool::FairSpillPool; @@ -1212,16 +1212,14 @@ mod tests { use datafusion_functions_aggregate::array_agg::array_agg_udaf; use datafusion_functions_aggregate::average::avg_udaf; use datafusion_functions_aggregate::count::count_udaf; - use datafusion_functions_aggregate::first_last::{FirstValue, LastValue}; + use datafusion_functions_aggregate::first_last::{first_value_udaf, last_value_udaf}; use datafusion_functions_aggregate::median::median_udaf; use datafusion_physical_expr::expressions::lit; use datafusion_physical_expr::PhysicalSortExpr; use crate::common::collect; use datafusion_physical_expr::expressions::Literal; - use datafusion_physical_expr_functions_aggregate::aggregate::{ - create_aggregate_expr_with_dfschema, AggregateExprBuilder, - }; + use datafusion_physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; use futures::{FutureExt, Stream}; // Generate a schema which consists of 5 columns (a, b, c, d, e) @@ -1269,22 +1267,19 @@ mod tests { } /// Generates some mock data for aggregate tests. - fn some_data_v2() -> (Arc, DFSchemaRef, Vec) { + fn some_data_v2() -> (Arc, Vec) { // Define a schema: let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::UInt32, false), Field::new("b", DataType::Float64, false), ])); - let df_schema = DFSchema::try_from(Arc::clone(&schema)).unwrap(); - // Generate data so that first and last value results are at 2nd and // 3rd partitions. With this construction, we guarantee we don't receive // the expected result by accident, but merging actually works properly; // i.e. it doesn't depend on the data insertion order. ( Arc::clone(&schema), - Arc::new(df_schema), vec![ RecordBatch::try_new( Arc::clone(&schema), @@ -1978,49 +1973,36 @@ mod tests { // FIRST_VALUE(b ORDER BY b ) fn test_first_value_agg_expr( schema: &Schema, - dfschema: &DFSchema, sort_options: SortOptions, ) -> Result> { - let ordering_req = vec![PhysicalSortExpr { + let ordering_req = [PhysicalSortExpr { expr: col("b", schema)?, options: sort_options, }]; - let args = vec![col("b", schema)?]; - let func = datafusion_expr::AggregateUDF::new_from_impl(FirstValue::new()); - datafusion_physical_expr_functions_aggregate::aggregate::create_aggregate_expr_with_dfschema( - &func, - &args, - &ordering_req, - dfschema, - Some(String::from("first_value(b) ORDER BY [b ASC NULLS LAST]")), - false, - false, - false, - ) + let args = [col("b", schema)?]; + + AggregateExprBuilder::new(first_value_udaf(), args.to_vec()) + .order_by(ordering_req.to_vec()) + .schema(Arc::new(schema.clone())) + .alias(String::from("first_value(b) ORDER BY [b ASC NULLS LAST]")) + .build() } // LAST_VALUE(b ORDER BY b ) fn test_last_value_agg_expr( schema: &Schema, - dfschema: &DFSchema, sort_options: SortOptions, ) -> Result> { - let ordering_req = vec![PhysicalSortExpr { + let ordering_req = [PhysicalSortExpr { expr: col("b", schema)?, options: sort_options, }]; - let args = vec![col("b", schema)?]; - let func = datafusion_expr::AggregateUDF::new_from_impl(LastValue::new()); - create_aggregate_expr_with_dfschema( - &func, - &args, - &ordering_req, - dfschema, - Some(String::from("last_value(b) ORDER BY [b ASC NULLS LAST]")), - false, - false, - false, - ) + let args = [col("b", schema)?]; + AggregateExprBuilder::new(last_value_udaf(), args.to_vec()) + .order_by(ordering_req.to_vec()) + .schema(Arc::new(schema.clone())) + .alias(String::from("last_value(b) ORDER BY [b ASC NULLS LAST]")) + .build() } // This function either constructs the physical plan below, @@ -2052,7 +2034,7 @@ mod tests { Arc::new(TaskContext::default()) }; - let (schema, df_schema, data) = some_data_v2(); + let (schema, data) = some_data_v2(); let partition1 = data[0].clone(); let partition2 = data[1].clone(); let partition3 = data[2].clone(); @@ -2066,13 +2048,9 @@ mod tests { nulls_first: false, }; let aggregates: Vec> = if is_first_acc { - vec![test_first_value_agg_expr( - &schema, - &df_schema, - sort_options, - )?] + vec![test_first_value_agg_expr(&schema, sort_options)?] } else { - vec![test_last_value_agg_expr(&schema, &df_schema, sort_options)?] + vec![test_last_value_agg_expr(&schema, sort_options)?] }; let memory_exec = Arc::new(MemoryExec::try_new( @@ -2139,7 +2117,6 @@ mod tests { #[tokio::test] async fn test_get_finest_requirements() -> Result<()> { let test_schema = create_test_schema()?; - let test_df_schema = DFSchema::try_from(Arc::clone(&test_schema)).unwrap(); // Assume column a and b are aliases // Assume also that a ASC and c DESC describe the same global ordering for the table. (Since they are ordering equivalent). @@ -2201,17 +2178,11 @@ mod tests { .into_iter() .map(|order_by_expr| { let ordering_req = order_by_expr.unwrap_or_default(); - create_aggregate_expr_with_dfschema( - &array_agg_udaf(), - &[Arc::clone(col_a)], - &ordering_req, - &test_df_schema, - None, - false, - false, - false, - ) - .unwrap() + AggregateExprBuilder::new(array_agg_udaf(), vec![Arc::clone(col_a)]) + .order_by(ordering_req.to_vec()) + .schema(Arc::clone(&test_schema)) + .build() + .unwrap() }) .collect::>(); let group_by = PhysicalGroupBy::new_single(vec![]); @@ -2232,7 +2203,6 @@ mod tests { Field::new("a", DataType::Float32, true), Field::new("b", DataType::Float32, true), ])); - let df_schema = DFSchema::try_from(Arc::clone(&schema)).unwrap(); let col_a = col("a", &schema)?; let option_desc = SortOptions { @@ -2242,8 +2212,8 @@ mod tests { let groups = PhysicalGroupBy::new_single(vec![(col_a, "a".to_string())]); let aggregates: Vec> = vec![ - test_first_value_agg_expr(&schema, &df_schema, option_desc)?, - test_last_value_agg_expr(&schema, &df_schema, option_desc)?, + test_first_value_agg_expr(&schema, option_desc)?, + test_last_value_agg_expr(&schema, option_desc)?, ]; let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); let aggregate_exec = Arc::new(AggregateExec::try_new( @@ -2353,22 +2323,17 @@ mod tests { Field::new("key", DataType::Int32, true), Field::new("val", DataType::Int32, true), ])); - let df_schema = DFSchema::try_from(Arc::clone(&schema))?; let group_by = PhysicalGroupBy::new_single(vec![(col("key", &schema)?, "key".to_string())]); - let aggr_expr: Vec> = - vec![create_aggregate_expr_with_dfschema( - &count_udaf(), - &[col("val", &schema)?], - &[], - &df_schema, - Some("COUNT(val)".to_string()), - false, - false, - false, - )?]; + let aggr_expr = + vec![ + AggregateExprBuilder::new(count_udaf(), vec![col("val", &schema)?]) + .schema(Arc::clone(&schema)) + .alias(String::from("COUNT(val)")) + .build()?, + ]; let input_data = vec![ RecordBatch::try_new( @@ -2439,22 +2404,17 @@ mod tests { Field::new("key", DataType::Int32, true), Field::new("val", DataType::Int32, true), ])); - let df_schema = DFSchema::try_from(Arc::clone(&schema))?; let group_by = PhysicalGroupBy::new_single(vec![(col("key", &schema)?, "key".to_string())]); - let aggr_expr: Vec> = - vec![create_aggregate_expr_with_dfschema( - &count_udaf(), - &[col("val", &schema)?], - &[], - &df_schema, - Some("COUNT(val)".to_string()), - false, - false, - false, - )?]; + let aggr_expr = + vec![ + AggregateExprBuilder::new(count_udaf(), vec![col("val", &schema)?]) + .schema(Arc::clone(&schema)) + .alias(String::from("COUNT(val)")) + .build()?, + ]; let input_data = vec![ RecordBatch::try_new( diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 34dcc7d2858e..59c5da6b6fb2 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -82,9 +82,7 @@ pub mod windows; pub mod work_table; pub mod udaf { - pub use datafusion_physical_expr_functions_aggregate::aggregate::{ - create_aggregate_expr, create_aggregate_expr_with_dfschema, AggregateFunctionExpr, - }; + pub use datafusion_physical_expr_functions_aggregate::aggregate::AggregateFunctionExpr; } #[cfg(test)] From c8846a6bee27e4b44e0e2c43450063247dfbc0b9 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 7 Aug 2024 21:13:47 +0800 Subject: [PATCH 18/26] revert change in calc_requirements Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/windows/mod.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 07ecd6d02527..2e6ad4e1a14f 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -379,8 +379,6 @@ impl BuiltInWindowFunctionExpr for WindowUDFExpr { } } -// TODO: Find a way to make clippy happy -#[allow(clippy::needless_borrow)] pub(crate) fn calc_requirements< T: Borrow>, S: Borrow, @@ -398,7 +396,7 @@ pub(crate) fn calc_requirements< let PhysicalSortExpr { expr, options } = element.borrow(); if !sort_reqs.iter().any(|e| e.expr.eq(expr)) { sort_reqs.push(PhysicalSortRequirement::new( - Arc::clone(&expr), + Arc::clone(expr), Some(*options), )); } From 87b5d428bca5081067999350b35df3326fdbb8cc Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 7 Aug 2024 21:15:19 +0800 Subject: [PATCH 19/26] fmt Signed-off-by: jayzhan211 --- datafusion/functions-aggregate-common/src/aggregate.rs | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate.rs b/datafusion/functions-aggregate-common/src/aggregate.rs index d6d49a2e4bc6..f0c98b4cc5cb 100644 --- a/datafusion/functions-aggregate-common/src/aggregate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate.rs @@ -15,18 +15,16 @@ // specific language governing permissions and limitations // under the License. -use std::fmt::Debug; -use std::{any::Any, sync::Arc}; - -use arrow::datatypes::Field; - use crate::order::AggregateOrderSensitivity; +use arrow::datatypes::Field; use datafusion_common::exec_err; use datafusion_common::{not_impl_err, Result}; use datafusion_expr_common::accumulator::Accumulator; use datafusion_expr_common::groups_accumulator::GroupsAccumulator; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use std::fmt::Debug; +use std::{any::Any, sync::Arc}; pub mod count_distinct; pub mod groups_accumulator; From b1a15db8c346035cc4ef00730e4431ed3234afb6 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 8 Aug 2024 08:22:53 +0800 Subject: [PATCH 20/26] doc and cleanup Signed-off-by: jayzhan211 --- datafusion/expr/src/operation.rs | 8 ++-- .../src/aggregate.rs | 3 ++ datafusion/functions-aggregate/src/stddev.rs | 1 - .../src/aggregate.rs | 45 ------------------- 4 files changed, 6 insertions(+), 51 deletions(-) diff --git a/datafusion/expr/src/operation.rs b/datafusion/expr/src/operation.rs index cdf550dae6c0..6b79a8248b29 100644 --- a/datafusion/expr/src/operation.rs +++ b/datafusion/expr/src/operation.rs @@ -15,14 +15,12 @@ // specific language governing permissions and limitations // under the License. -//! Operator module contains foundational types that are used to represent operators in DataFusion. +//! This module contains implementations of operations (unary, binary etc.) for DataFusion expressions. use crate::expr_fn::binary_expr; -use crate::Expr; -use crate::Like; +use crate::{Expr, Like}; use datafusion_expr_common::operator::Operator; -use std::ops; -use std::ops::Not; +use std::ops::{self, Not}; /// Support ` + ` fluent style impl ops::Add for Expr { diff --git a/datafusion/functions-aggregate-common/src/aggregate.rs b/datafusion/functions-aggregate-common/src/aggregate.rs index f0c98b4cc5cb..69a55b76317f 100644 --- a/datafusion/functions-aggregate-common/src/aggregate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +//! Contains the trait `AggregateExpr` which defines the interface all aggregate expressions +//! (built-in and custom) need to satisfy. + use crate::order::AggregateOrderSensitivity; use arrow::datatypes::Field; use datafusion_common::exec_err; diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index 7083ba1afbea..161a7a103bd2 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -344,7 +344,6 @@ mod tests { schema, dfschema: &dfschema, ignore_nulls: false, - // sort_exprs: &[], ordering_req: &[], name: "a", is_distinct: false, diff --git a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs b/datafusion/physical-expr-functions-aggregate/src/aggregate.rs index 04a048a5f5d8..cd1fb626088d 100644 --- a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs +++ b/datafusion/physical-expr-functions-aggregate/src/aggregate.rs @@ -35,51 +35,6 @@ use datafusion_physical_expr_common::utils::reverse_order_bys; use std::fmt::Debug; use std::{any::Any, sync::Arc}; -/// 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. -/// -/// `input_exprs` and `sort_exprs` are used for customizing Accumulator -/// whose behavior depends on arguments such as the `ORDER BY`. -/// -/// For example to call `ARRAY_AGG(x ORDER BY y)` would pass `y` to `sort_exprs`, `x` to `input_exprs` -/// -/// `input_exprs` and `sort_exprs` are used for customizing Accumulator as the arguments in `AccumulatorArgs`, -/// if you don't need them it is fine to pass empty slice `&[]`. -/// -/// `is_reversed` is used to indicate whether the aggregation is running in reverse order, -/// it could be used to hint Accumulator to accumulate in the reversed order, -/// you can just set to false if you are not reversing expression -/// -/// You can also create expression by [`AggregateExprBuilder`] -#[allow(clippy::too_many_arguments)] -#[deprecated(since = "40.0.0", note = "Use `AggregateExprBuilder` instead")] -pub fn create_aggregate_expr( - fun: &AggregateUDF, - input_phy_exprs: &[Arc], - ordering_req: &[PhysicalSortExpr], - schema: &Schema, - name: Option, - ignore_nulls: bool, - is_distinct: bool, -) -> Result> { - let mut builder = - AggregateExprBuilder::new(Arc::new(fun.clone()), input_phy_exprs.to_vec()); - builder = builder.order_by(ordering_req.to_vec()); - builder = builder.schema(Arc::new(schema.clone())); - if let Some(name) = name { - builder = builder.alias(name); - } - - if ignore_nulls { - builder = builder.ignore_nulls(); - } - if is_distinct { - builder = builder.distinct(); - } - - builder.build() -} - /// Builder for physical [`AggregateExpr`] /// /// `AggregateExpr` contains the information necessary to call From b76b8f03dfed8efa7880a174e8e79b73a641711e Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 8 Aug 2024 08:38:04 +0800 Subject: [PATCH 21/26] rm dfschema Signed-off-by: jayzhan211 --- datafusion/core/src/physical_planner.rs | 1 - .../src/accumulator.rs | 8 ++---- .../src/aggregate.rs | 25 +++---------------- 3 files changed, 5 insertions(+), 29 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 749057bc6196..321177f3b2da 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1605,7 +1605,6 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( AggregateExprBuilder::new(func.to_owned(), physical_args.to_vec()) .order_by(ordering_reqs.to_vec()) .schema(Arc::new(schema)) - .dfschema(logical_input_schema.clone()) .alias(name) .with_ignore_nulls(ignore_nulls) .with_distinct(*distinct) diff --git a/datafusion/functions-aggregate-common/src/accumulator.rs b/datafusion/functions-aggregate-common/src/accumulator.rs index b123184e35a8..12102cc98e7f 100644 --- a/datafusion/functions-aggregate-common/src/accumulator.rs +++ b/datafusion/functions-aggregate-common/src/accumulator.rs @@ -15,14 +15,13 @@ // specific language governing permissions and limitations // under the License. -use std::sync::Arc; - use arrow::datatypes::{DataType, Field, Schema}; -use datafusion_common::{DFSchema, Result}; +use datafusion_common::Result; use datafusion_expr_common::accumulator::Accumulator; use datafusion_physical_expr_common::{ physical_expr::PhysicalExpr, sort_expr::PhysicalSortExpr, }; +use std::sync::Arc; /// [`AccumulatorArgs`] contains information about how an aggregate /// function was called, including the types of its arguments and any optional @@ -35,9 +34,6 @@ pub struct AccumulatorArgs<'a> { /// The schema of the input arguments pub schema: &'a Schema, - /// The schema of the input arguments - pub dfschema: &'a DFSchema, - /// Whether to ignore nulls. /// /// SQL allows the user to specify `IGNORE NULLS`, for example: diff --git a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs b/datafusion/physical-expr-functions-aggregate/src/aggregate.rs index cd1fb626088d..302399a254b8 100644 --- a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs +++ b/datafusion/physical-expr-functions-aggregate/src/aggregate.rs @@ -16,7 +16,7 @@ // under the License. use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion_common::{internal_err, not_impl_err, DFSchema, Result}; +use datafusion_common::{internal_err, not_impl_err, Result}; use datafusion_expr::expr::create_function_physical_name; use datafusion_expr::AggregateUDF; use datafusion_expr::ReversedUDAF; @@ -47,8 +47,6 @@ pub struct AggregateExprBuilder { alias: Option, /// Arrow Schema for the aggregate function schema: SchemaRef, - /// Datafusion Schema for the aggregate function - dfschema: DFSchema, /// The physical order by expressions ordering_req: LexOrdering, /// Whether to ignore null values @@ -66,7 +64,6 @@ impl AggregateExprBuilder { args, alias: None, schema: Arc::new(Schema::empty()), - dfschema: DFSchema::empty(), ordering_req: vec![], ignore_nulls: false, is_distinct: false, @@ -80,7 +77,6 @@ impl AggregateExprBuilder { args, alias, schema, - dfschema, ordering_req, ignore_nulls, is_distinct, @@ -125,7 +121,6 @@ impl AggregateExprBuilder { data_type, name, schema: Arc::unwrap_or_clone(schema), - dfschema, ordering_req, ignore_nulls, ordering_fields, @@ -145,11 +140,6 @@ impl AggregateExprBuilder { self } - pub fn dfschema(mut self, dfschema: DFSchema) -> Self { - self.dfschema = dfschema; - self - } - pub fn order_by(mut self, order_by: LexOrdering) -> Self { self.ordering_req = order_by; self @@ -195,7 +185,6 @@ pub struct AggregateFunctionExpr { data_type: DataType, name: String, schema: Schema, - dfschema: DFSchema, // The physical order by expressions ordering_req: LexOrdering, // Whether to ignore null values @@ -259,7 +248,6 @@ impl AggregateExpr for AggregateFunctionExpr { let acc_args = AccumulatorArgs { data_type: &self.data_type, schema: &self.schema, - dfschema: &self.dfschema, ignore_nulls: self.ignore_nulls, ordering_req: &self.ordering_req, is_distinct: self.is_distinct, @@ -276,7 +264,6 @@ impl AggregateExpr for AggregateFunctionExpr { let args = AccumulatorArgs { data_type: &self.data_type, schema: &self.schema, - dfschema: &self.dfschema, ignore_nulls: self.ignore_nulls, ordering_req: &self.ordering_req, is_distinct: self.is_distinct, @@ -348,7 +335,6 @@ impl AggregateExpr for AggregateFunctionExpr { let args = AccumulatorArgs { data_type: &self.data_type, schema: &self.schema, - dfschema: &self.dfschema, ignore_nulls: self.ignore_nulls, ordering_req: &self.ordering_req, is_distinct: self.is_distinct, @@ -364,7 +350,6 @@ impl AggregateExpr for AggregateFunctionExpr { let args = AccumulatorArgs { data_type: &self.data_type, schema: &self.schema, - dfschema: &self.dfschema, ignore_nulls: self.ignore_nulls, ordering_req: &self.ordering_req, is_distinct: self.is_distinct, @@ -410,11 +395,9 @@ impl AggregateExpr for AggregateFunctionExpr { return Ok(None); }; - let schema: Schema = self.dfschema.clone().into(); AggregateExprBuilder::new(Arc::new(updated_fn), self.args.to_vec()) .order_by(self.ordering_req.to_vec()) - .schema(Arc::new(schema)) - .dfschema(self.dfschema.clone()) + .schema(Arc::new(self.schema.clone())) .alias(self.name().to_string()) .with_ignore_nulls(self.ignore_nulls) .with_distinct(self.is_distinct) @@ -438,11 +421,9 @@ impl AggregateExpr for AggregateFunctionExpr { } replace_fn_name_clause(&mut name, self.fun.name(), reverse_udf.name()); - let schema: Schema = self.dfschema.clone().into(); AggregateExprBuilder::new(reverse_udf, self.args.to_vec()) .order_by(reverse_ordering_req.to_vec()) - .schema(Arc::new(schema)) - .dfschema(self.dfschema.clone()) + .schema(Arc::new(self.schema.clone())) .alias(name) .with_ignore_nulls(self.ignore_nulls) .with_distinct(self.is_distinct) From 0c19a8db7b237c35fdca96d7e11fd862a54497f1 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 8 Aug 2024 09:03:09 +0800 Subject: [PATCH 22/26] rm input types Signed-off-by: jayzhan211 --- .../src/accumulator.rs | 3 --- .../functions-aggregate/benches/count.rs | 4 ---- datafusion/functions-aggregate/benches/sum.rs | 4 ---- .../functions-aggregate/src/approx_distinct.rs | 4 +++- .../functions-aggregate/src/approx_median.rs | 2 +- .../src/approx_percentile_cont.rs | 3 ++- .../functions-aggregate/src/array_agg.rs | 12 +++++------- datafusion/functions-aggregate/src/average.rs | 18 +++++++++++------- datafusion/functions-aggregate/src/count.rs | 2 +- datafusion/functions-aggregate/src/median.rs | 2 +- .../functions-aggregate/src/nth_value.rs | 3 ++- datafusion/functions-aggregate/src/stddev.rs | 13 ++----------- .../src/aggregate.rs | 4 ---- 13 files changed, 28 insertions(+), 46 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/accumulator.rs b/datafusion/functions-aggregate-common/src/accumulator.rs index 12102cc98e7f..33a52cae7fac 100644 --- a/datafusion/functions-aggregate-common/src/accumulator.rs +++ b/datafusion/functions-aggregate-common/src/accumulator.rs @@ -68,9 +68,6 @@ pub struct AccumulatorArgs<'a> { /// ``` pub is_distinct: bool, - /// The input types of the aggregate function. - pub input_types: &'a [DataType], - /// The physical expression of arguments the aggregate function takes. pub exprs: &'a [Arc], } diff --git a/datafusion/functions-aggregate/benches/count.rs b/datafusion/functions-aggregate/benches/count.rs index 28e4579f2a7e..767ae9f0a98e 100644 --- a/datafusion/functions-aggregate/benches/count.rs +++ b/datafusion/functions-aggregate/benches/count.rs @@ -20,7 +20,6 @@ use arrow::datatypes::Int32Type; use arrow::util::bench_util::{create_boolean_array, create_primitive_array}; use arrow_schema::{DataType, Field, Schema}; use criterion::{black_box, criterion_group, criterion_main, Criterion}; -use datafusion_common::DFSchema; use datafusion_expr::{function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator}; use datafusion_functions_aggregate::count::Count; use datafusion_physical_expr::expressions::col; @@ -28,17 +27,14 @@ use std::sync::Arc; fn prepare_accumulator() -> Box { let schema = Arc::new(Schema::new(vec![Field::new("f", DataType::Int32, true)])); - let df_schema = DFSchema::try_from(Arc::clone(&schema)).unwrap(); let accumulator_args = AccumulatorArgs { data_type: &DataType::Int64, schema: &schema, - dfschema: &df_schema, ignore_nulls: false, ordering_req: &[], is_reversed: false, name: "COUNT(f)", is_distinct: false, - input_types: &[DataType::Int32], exprs: &[col("f", &schema).unwrap()], }; let count_fn = Count::new(); diff --git a/datafusion/functions-aggregate/benches/sum.rs b/datafusion/functions-aggregate/benches/sum.rs index a11b06cba54f..944504f6647e 100644 --- a/datafusion/functions-aggregate/benches/sum.rs +++ b/datafusion/functions-aggregate/benches/sum.rs @@ -20,7 +20,6 @@ use arrow::datatypes::Int64Type; use arrow::util::bench_util::{create_boolean_array, create_primitive_array}; use arrow_schema::{DataType, Field, Schema}; use criterion::{black_box, criterion_group, criterion_main, Criterion}; -use datafusion_common::DFSchema; use datafusion_expr::{function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator}; use datafusion_functions_aggregate::sum::Sum; use datafusion_physical_expr::expressions::col; @@ -28,17 +27,14 @@ use std::sync::Arc; fn prepare_accumulator(data_type: &DataType) -> Box { let schema = Arc::new(Schema::new(vec![Field::new("f", data_type.clone(), true)])); - let df_schema = DFSchema::try_from(Arc::clone(&schema)).unwrap(); let accumulator_args = AccumulatorArgs { data_type, schema: &schema, - dfschema: &df_schema, ignore_nulls: false, ordering_req: &[], is_reversed: false, name: "SUM(f)", is_distinct: false, - input_types: &[data_type.clone()], exprs: &[col("f", &schema).unwrap()], }; let sum_fn = Sum::new(); diff --git a/datafusion/functions-aggregate/src/approx_distinct.rs b/datafusion/functions-aggregate/src/approx_distinct.rs index 56ef32e7ebe0..cf8217fe981d 100644 --- a/datafusion/functions-aggregate/src/approx_distinct.rs +++ b/datafusion/functions-aggregate/src/approx_distinct.rs @@ -277,7 +277,9 @@ impl AggregateUDFImpl for ApproxDistinct { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - let accumulator: Box = match &acc_args.input_types[0] { + let data_type = acc_args.exprs[0].data_type(acc_args.schema)?; + + let accumulator: Box = match data_type { // TODO u8, i8, u16, i16 shall really be done using bitmap, not HLL // TODO support for boolean (trivial case) // https://github.com/apache/datafusion/issues/1109 diff --git a/datafusion/functions-aggregate/src/approx_median.rs b/datafusion/functions-aggregate/src/approx_median.rs index c386ad89f0fb..7a7b12432544 100644 --- a/datafusion/functions-aggregate/src/approx_median.rs +++ b/datafusion/functions-aggregate/src/approx_median.rs @@ -113,7 +113,7 @@ impl AggregateUDFImpl for ApproxMedian { Ok(Box::new(ApproxPercentileAccumulator::new( 0.5_f64, - acc_args.input_types[0].clone(), + acc_args.exprs[0].data_type(acc_args.schema)?, ))) } } diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index f95b33bd0332..b292218beecd 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -104,7 +104,8 @@ impl ApproxPercentileCont { None }; - let accumulator: ApproxPercentileAccumulator = match &args.input_types[0] { + let data_type = args.exprs[0].data_type(args.schema)?; + let accumulator: ApproxPercentileAccumulator = match data_type { t @ (DataType::UInt8 | DataType::UInt16 | DataType::UInt32 diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index b451b87ab0ea..b641d388a7c5 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -114,16 +114,14 @@ impl AggregateUDFImpl for ArrayAgg { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { + let data_type = acc_args.exprs[0].data_type(acc_args.schema)?; + if acc_args.is_distinct { - return Ok(Box::new(DistinctArrayAggAccumulator::try_new( - &acc_args.input_types[0], - )?)); + return Ok(Box::new(DistinctArrayAggAccumulator::try_new(&data_type)?)); } if acc_args.ordering_req.is_empty() { - return Ok(Box::new(ArrayAggAccumulator::try_new( - &acc_args.input_types[0], - )?)); + return Ok(Box::new(ArrayAggAccumulator::try_new(&data_type)?)); } let ordering_dtypes = acc_args @@ -133,7 +131,7 @@ impl AggregateUDFImpl for ArrayAgg { .collect::>>()?; OrderSensitiveArrayAggAccumulator::try_new( - &acc_args.input_types[0], + &data_type, &ordering_dtypes, acc_args.ordering_req.to_vec(), acc_args.is_reversed, diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index c26a63340736..247ce3f31e99 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -92,8 +92,10 @@ impl AggregateUDFImpl for Avg { return exec_err!("avg(DISTINCT) aggregations are not available"); } use DataType::*; + + let data_type = acc_args.exprs[0].data_type(acc_args.schema)?; // instantiate specialized accumulator based for the type - match (&acc_args.input_types[0], acc_args.data_type) { + match (&data_type, acc_args.data_type) { (Float64, Float64) => Ok(Box::::default()), ( Decimal128(sum_precision, sum_scale), @@ -120,7 +122,7 @@ impl AggregateUDFImpl for Avg { })), _ => exec_err!( "AvgAccumulator for ({} --> {})", - &acc_args.input_types[0], + &data_type, acc_args.data_type ), } @@ -153,11 +155,13 @@ impl AggregateUDFImpl for Avg { args: AccumulatorArgs, ) -> Result> { use DataType::*; + + let data_type = args.exprs[0].data_type(args.schema)?; // instantiate specialized accumulator based for the type - match (&args.input_types[0], args.data_type) { + match (&data_type, args.data_type) { (Float64, Float64) => { Ok(Box::new(AvgGroupsAccumulator::::new( - &args.input_types[0], + &data_type, args.data_type, |sum: f64, count: u64| Ok(sum / count as f64), ))) @@ -176,7 +180,7 @@ impl AggregateUDFImpl for Avg { move |sum: i128, count: u64| decimal_averager.avg(sum, count as i128); Ok(Box::new(AvgGroupsAccumulator::::new( - &args.input_types[0], + &data_type, args.data_type, avg_fn, ))) @@ -197,7 +201,7 @@ impl AggregateUDFImpl for Avg { }; Ok(Box::new(AvgGroupsAccumulator::::new( - &args.input_types[0], + &data_type, args.data_type, avg_fn, ))) @@ -205,7 +209,7 @@ impl AggregateUDFImpl for Avg { _ => not_impl_err!( "AvgGroupsAccumulator for ({} --> {})", - &args.input_types[0], + &data_type, args.data_type ), } diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index b4c37fe09fe2..04b1921c7b9e 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -147,7 +147,7 @@ impl AggregateUDFImpl for Count { return not_impl_err!("COUNT DISTINCT with multiple arguments"); } - let data_type = &acc_args.input_types[0]; + let data_type = &acc_args.exprs[0].data_type(acc_args.schema)?; Ok(match data_type { // try and use a specialized accumulator if possible, otherwise fall back to generic accumulator DataType::Int8 => Box::new( diff --git a/datafusion/functions-aggregate/src/median.rs b/datafusion/functions-aggregate/src/median.rs index 9a70591e18cf..7dd0de14c3c0 100644 --- a/datafusion/functions-aggregate/src/median.rs +++ b/datafusion/functions-aggregate/src/median.rs @@ -133,7 +133,7 @@ impl AggregateUDFImpl for Median { }; } - let dt = &acc_args.input_types[0]; + let dt = acc_args.exprs[0].data_type(acc_args.schema)?; downcast_integer! { dt => (helper, dt), DataType::Float16 => helper!(Float16Type, dt), diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index b19d32c35acd..cb1ddd4738c4 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -112,9 +112,10 @@ impl AggregateUDFImpl for NthValueAgg { .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; + let data_type = acc_args.exprs[0].data_type(acc_args.schema)?; NthValueAccumulator::try_new( n, - &acc_args.input_types[0], + &data_type, &ordering_dtypes, acc_args.ordering_req.to_vec(), ) diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index 161a7a103bd2..43e3ff1e2ebc 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -269,16 +269,12 @@ impl Accumulator for StddevAccumulator { #[cfg(test)] mod tests { - use std::sync::Arc; - + use super::*; use arrow::{array::*, datatypes::*}; - - use datafusion_common::DFSchema; use datafusion_expr::AggregateUDF; use datafusion_functions_aggregate_common::utils::get_accum_scalar_values_as_arrays; use datafusion_physical_expr::expressions::col; - - use super::*; + use std::sync::Arc; #[test] fn stddev_f64_merge_1() -> Result<()> { @@ -325,30 +321,25 @@ mod tests { agg2: Arc, schema: &Schema, ) -> Result { - let dfschema = DFSchema::empty(); let args1 = AccumulatorArgs { data_type: &DataType::Float64, schema, - dfschema: &dfschema, ignore_nulls: false, ordering_req: &[], name: "a", is_distinct: false, is_reversed: false, - input_types: &[DataType::Float64], exprs: &[col("a", schema)?], }; let args2 = AccumulatorArgs { data_type: &DataType::Float64, schema, - dfschema: &dfschema, ignore_nulls: false, ordering_req: &[], name: "a", is_distinct: false, is_reversed: false, - input_types: &[DataType::Float64], exprs: &[col("a", schema)?], }; diff --git a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs b/datafusion/physical-expr-functions-aggregate/src/aggregate.rs index 302399a254b8..62c43c06559c 100644 --- a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs +++ b/datafusion/physical-expr-functions-aggregate/src/aggregate.rs @@ -251,7 +251,6 @@ impl AggregateExpr for AggregateFunctionExpr { ignore_nulls: self.ignore_nulls, ordering_req: &self.ordering_req, is_distinct: self.is_distinct, - input_types: &self.input_types, name: &self.name, is_reversed: self.is_reversed, exprs: &self.args, @@ -267,7 +266,6 @@ impl AggregateExpr for AggregateFunctionExpr { ignore_nulls: self.ignore_nulls, ordering_req: &self.ordering_req, is_distinct: self.is_distinct, - input_types: &self.input_types, name: &self.name, is_reversed: self.is_reversed, exprs: &self.args, @@ -338,7 +336,6 @@ impl AggregateExpr for AggregateFunctionExpr { ignore_nulls: self.ignore_nulls, ordering_req: &self.ordering_req, is_distinct: self.is_distinct, - input_types: &self.input_types, name: &self.name, is_reversed: self.is_reversed, exprs: &self.args, @@ -353,7 +350,6 @@ impl AggregateExpr for AggregateFunctionExpr { ignore_nulls: self.ignore_nulls, ordering_req: &self.ordering_req, is_distinct: self.is_distinct, - input_types: &self.input_types, name: &self.name, is_reversed: self.is_reversed, exprs: &self.args, From 722688915551a77f94a7ff719abc331aa1afe0c8 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 8 Aug 2024 09:12:29 +0800 Subject: [PATCH 23/26] rename return_type Signed-off-by: jayzhan211 --- .../src/accumulator.rs | 2 +- .../functions-aggregate/benches/count.rs | 2 +- datafusion/functions-aggregate/benches/sum.rs | 2 +- datafusion/functions-aggregate/src/average.rs | 16 ++++++++-------- .../functions-aggregate/src/bit_and_or_xor.rs | 6 +++--- .../functions-aggregate/src/bool_and_or.rs | 8 ++++---- .../functions-aggregate/src/first_last.rs | 4 ++-- datafusion/functions-aggregate/src/min_max.rs | 16 ++++++++-------- datafusion/functions-aggregate/src/stddev.rs | 4 ++-- datafusion/functions-aggregate/src/sum.rs | 18 +++++++++++------- .../src/aggregate.rs | 8 ++++---- 11 files changed, 45 insertions(+), 41 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/accumulator.rs b/datafusion/functions-aggregate-common/src/accumulator.rs index 33a52cae7fac..ddf0085b9de4 100644 --- a/datafusion/functions-aggregate-common/src/accumulator.rs +++ b/datafusion/functions-aggregate-common/src/accumulator.rs @@ -29,7 +29,7 @@ use std::sync::Arc; #[derive(Debug)] pub struct AccumulatorArgs<'a> { /// The return type of the aggregate function. - pub data_type: &'a DataType, + pub return_type: &'a DataType, /// The schema of the input arguments pub schema: &'a Schema, diff --git a/datafusion/functions-aggregate/benches/count.rs b/datafusion/functions-aggregate/benches/count.rs index 767ae9f0a98e..65956cb8a1de 100644 --- a/datafusion/functions-aggregate/benches/count.rs +++ b/datafusion/functions-aggregate/benches/count.rs @@ -28,7 +28,7 @@ use std::sync::Arc; fn prepare_accumulator() -> Box { let schema = Arc::new(Schema::new(vec![Field::new("f", DataType::Int32, true)])); let accumulator_args = AccumulatorArgs { - data_type: &DataType::Int64, + return_type: &DataType::Int64, schema: &schema, ignore_nulls: false, ordering_req: &[], diff --git a/datafusion/functions-aggregate/benches/sum.rs b/datafusion/functions-aggregate/benches/sum.rs index 944504f6647e..652d447129dc 100644 --- a/datafusion/functions-aggregate/benches/sum.rs +++ b/datafusion/functions-aggregate/benches/sum.rs @@ -28,7 +28,7 @@ use std::sync::Arc; fn prepare_accumulator(data_type: &DataType) -> Box { let schema = Arc::new(Schema::new(vec![Field::new("f", data_type.clone(), true)])); let accumulator_args = AccumulatorArgs { - data_type, + return_type: data_type, schema: &schema, ignore_nulls: false, ordering_req: &[], diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 247ce3f31e99..1be3cd6b0714 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -95,7 +95,7 @@ impl AggregateUDFImpl for Avg { let data_type = acc_args.exprs[0].data_type(acc_args.schema)?; // instantiate specialized accumulator based for the type - match (&data_type, acc_args.data_type) { + match (&data_type, acc_args.return_type) { (Float64, Float64) => Ok(Box::::default()), ( Decimal128(sum_precision, sum_scale), @@ -123,7 +123,7 @@ impl AggregateUDFImpl for Avg { _ => exec_err!( "AvgAccumulator for ({} --> {})", &data_type, - acc_args.data_type + acc_args.return_type ), } } @@ -145,7 +145,7 @@ impl AggregateUDFImpl for Avg { fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { matches!( - args.data_type, + args.return_type, DataType::Float64 | DataType::Decimal128(_, _) ) } @@ -158,11 +158,11 @@ impl AggregateUDFImpl for Avg { let data_type = args.exprs[0].data_type(args.schema)?; // instantiate specialized accumulator based for the type - match (&data_type, args.data_type) { + match (&data_type, args.return_type) { (Float64, Float64) => { Ok(Box::new(AvgGroupsAccumulator::::new( &data_type, - args.data_type, + args.return_type, |sum: f64, count: u64| Ok(sum / count as f64), ))) } @@ -181,7 +181,7 @@ impl AggregateUDFImpl for Avg { Ok(Box::new(AvgGroupsAccumulator::::new( &data_type, - args.data_type, + args.return_type, avg_fn, ))) } @@ -202,7 +202,7 @@ impl AggregateUDFImpl for Avg { Ok(Box::new(AvgGroupsAccumulator::::new( &data_type, - args.data_type, + args.return_type, avg_fn, ))) } @@ -210,7 +210,7 @@ impl AggregateUDFImpl for Avg { _ => not_impl_err!( "AvgGroupsAccumulator for ({} --> {})", &data_type, - args.data_type + args.return_type ), } } diff --git a/datafusion/functions-aggregate/src/bit_and_or_xor.rs b/datafusion/functions-aggregate/src/bit_and_or_xor.rs index b6b1862d281b..aa65062e3330 100644 --- a/datafusion/functions-aggregate/src/bit_and_or_xor.rs +++ b/datafusion/functions-aggregate/src/bit_and_or_xor.rs @@ -84,7 +84,7 @@ macro_rules! accumulator_helper { /// `is_distinct` is boolean value indicating whether the operation is distinct or not. macro_rules! downcast_bitwise_accumulator { ($args:ident, $opr:expr, $is_distinct: expr) => { - match $args.data_type { + match $args.return_type { DataType::Int8 => accumulator_helper!(Int8Type, $opr, $is_distinct), DataType::Int16 => accumulator_helper!(Int16Type, $opr, $is_distinct), DataType::Int32 => accumulator_helper!(Int32Type, $opr, $is_distinct), @@ -98,7 +98,7 @@ macro_rules! downcast_bitwise_accumulator { "{} not supported for {}: {}", stringify!($opr), $args.name, - $args.data_type + $args.return_type ) } } @@ -224,7 +224,7 @@ impl AggregateUDFImpl for BitwiseOperation { &self, args: AccumulatorArgs, ) -> Result> { - let data_type = args.data_type; + let data_type = args.return_type; let operation = &self.operation; downcast_integer! { data_type => (group_accumulator_helper, data_type, operation), diff --git a/datafusion/functions-aggregate/src/bool_and_or.rs b/datafusion/functions-aggregate/src/bool_and_or.rs index 3d5fedb5ca39..b993b2a4979c 100644 --- a/datafusion/functions-aggregate/src/bool_and_or.rs +++ b/datafusion/functions-aggregate/src/bool_and_or.rs @@ -149,14 +149,14 @@ impl AggregateUDFImpl for BoolAnd { &self, args: AccumulatorArgs, ) -> Result> { - match args.data_type { + match args.return_type { DataType::Boolean => { Ok(Box::new(BooleanGroupsAccumulator::new(|x, y| x && y))) } _ => not_impl_err!( "GroupsAccumulator not supported for {} with {}", args.name, - args.data_type + args.return_type ), } } @@ -269,14 +269,14 @@ impl AggregateUDFImpl for BoolOr { &self, args: AccumulatorArgs, ) -> Result> { - match args.data_type { + match args.return_type { DataType::Boolean => { Ok(Box::new(BooleanGroupsAccumulator::new(|x, y| x || y))) } _ => not_impl_err!( "GroupsAccumulator not supported for {} with {}", args.name, - args.data_type + args.return_type ), } } diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 28007503f754..2162442f054e 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -126,7 +126,7 @@ impl AggregateUDFImpl for FirstValue { acc_args.ordering_req.is_empty() || self.requirement_satisfied; FirstValueAccumulator::try_new( - acc_args.data_type, + acc_args.return_type, &ordering_dtypes, acc_args.ordering_req.to_vec(), acc_args.ignore_nulls, @@ -420,7 +420,7 @@ impl AggregateUDFImpl for LastValue { acc_args.ordering_req.is_empty() || self.requirement_satisfied; LastValueAccumulator::try_new( - acc_args.data_type, + acc_args.return_type, &ordering_dtypes, acc_args.ordering_req.to_vec(), acc_args.ignore_nulls, diff --git a/datafusion/functions-aggregate/src/min_max.rs b/datafusion/functions-aggregate/src/min_max.rs index 892b10b804c1..d790edd47a93 100644 --- a/datafusion/functions-aggregate/src/min_max.rs +++ b/datafusion/functions-aggregate/src/min_max.rs @@ -156,7 +156,7 @@ impl AggregateUDFImpl for Max { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - Ok(Box::new(MaxAccumulator::try_new(acc_args.data_type)?)) + Ok(Box::new(MaxAccumulator::try_new(acc_args.return_type)?)) } fn aliases(&self) -> &[String] { @@ -166,7 +166,7 @@ impl AggregateUDFImpl for Max { fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { use DataType::*; matches!( - args.data_type, + args.return_type, Int8 | Int16 | Int32 | Int64 @@ -192,7 +192,7 @@ impl AggregateUDFImpl for Max { ) -> Result> { use DataType::*; use TimeUnit::*; - let data_type = args.data_type; + let data_type = args.return_type; match data_type { Int8 => instantiate_max_accumulator!(data_type, i8, Int8Type), Int16 => instantiate_max_accumulator!(data_type, i16, Int16Type), @@ -253,7 +253,7 @@ impl AggregateUDFImpl for Max { &self, args: AccumulatorArgs, ) -> Result> { - Ok(Box::new(SlidingMaxAccumulator::try_new(args.data_type)?)) + Ok(Box::new(SlidingMaxAccumulator::try_new(args.return_type)?)) } fn is_descending(&self) -> Option { @@ -923,7 +923,7 @@ impl AggregateUDFImpl for Min { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - Ok(Box::new(MinAccumulator::try_new(acc_args.data_type)?)) + Ok(Box::new(MinAccumulator::try_new(acc_args.return_type)?)) } fn aliases(&self) -> &[String] { @@ -933,7 +933,7 @@ impl AggregateUDFImpl for Min { fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { use DataType::*; matches!( - args.data_type, + args.return_type, Int8 | Int16 | Int32 | Int64 @@ -959,7 +959,7 @@ impl AggregateUDFImpl for Min { ) -> Result> { use DataType::*; use TimeUnit::*; - let data_type = args.data_type; + let data_type = args.return_type; match data_type { Int8 => instantiate_min_accumulator!(data_type, i8, Int8Type), Int16 => instantiate_min_accumulator!(data_type, i16, Int16Type), @@ -1020,7 +1020,7 @@ impl AggregateUDFImpl for Min { &self, args: AccumulatorArgs, ) -> Result> { - Ok(Box::new(SlidingMinAccumulator::try_new(args.data_type)?)) + Ok(Box::new(SlidingMinAccumulator::try_new(args.return_type)?)) } fn is_descending(&self) -> Option { diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index 43e3ff1e2ebc..180f4ad3cf37 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -322,7 +322,7 @@ mod tests { schema: &Schema, ) -> Result { let args1 = AccumulatorArgs { - data_type: &DataType::Float64, + return_type: &DataType::Float64, schema, ignore_nulls: false, ordering_req: &[], @@ -333,7 +333,7 @@ mod tests { }; let args2 = AccumulatorArgs { - data_type: &DataType::Float64, + return_type: &DataType::Float64, schema, ignore_nulls: false, ordering_req: &[], diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index 40df5bf386ec..7e40c1bd17a8 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -58,14 +58,18 @@ make_udaf_expr_and_func!( /// `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), + match $args.return_type { + DataType::UInt64 => $helper!(UInt64Type, $args.return_type), + DataType::Int64 => $helper!(Int64Type, $args.return_type), + DataType::Float64 => $helper!(Float64Type, $args.return_type), + DataType::Decimal128(_, _) => $helper!(Decimal128Type, $args.return_type), + DataType::Decimal256(_, _) => $helper!(Decimal256Type, $args.return_type), _ => { - not_impl_err!("Sum not supported for {}: {}", $args.name, $args.data_type) + not_impl_err!( + "Sum not supported for {}: {}", + $args.name, + $args.return_type + ) } } }; diff --git a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs b/datafusion/physical-expr-functions-aggregate/src/aggregate.rs index 62c43c06559c..8185f0fdd51f 100644 --- a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs +++ b/datafusion/physical-expr-functions-aggregate/src/aggregate.rs @@ -246,7 +246,7 @@ impl AggregateExpr for AggregateFunctionExpr { fn create_accumulator(&self) -> Result> { let acc_args = AccumulatorArgs { - data_type: &self.data_type, + return_type: &self.data_type, schema: &self.schema, ignore_nulls: self.ignore_nulls, ordering_req: &self.ordering_req, @@ -261,7 +261,7 @@ impl AggregateExpr for AggregateFunctionExpr { fn create_sliding_accumulator(&self) -> Result> { let args = AccumulatorArgs { - data_type: &self.data_type, + return_type: &self.data_type, schema: &self.schema, ignore_nulls: self.ignore_nulls, ordering_req: &self.ordering_req, @@ -331,7 +331,7 @@ impl AggregateExpr for AggregateFunctionExpr { fn groups_accumulator_supported(&self) -> bool { let args = AccumulatorArgs { - data_type: &self.data_type, + return_type: &self.data_type, schema: &self.schema, ignore_nulls: self.ignore_nulls, ordering_req: &self.ordering_req, @@ -345,7 +345,7 @@ impl AggregateExpr for AggregateFunctionExpr { fn create_groups_accumulator(&self) -> Result> { let args = AccumulatorArgs { - data_type: &self.data_type, + return_type: &self.data_type, schema: &self.schema, ignore_nulls: self.ignore_nulls, ordering_req: &self.ordering_req, From 8d330fa78c714aa7e43d8fa793cee2a69eada7d6 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 8 Aug 2024 21:28:21 +0800 Subject: [PATCH 24/26] upd doc Signed-off-by: jayzhan211 --- datafusion/expr-common/src/lib.rs | 6 +++++- datafusion/functions-aggregate-common/Cargo.toml | 2 +- datafusion/functions-aggregate-common/src/aggregate.rs | 2 +- datafusion/functions-aggregate-common/src/lib.rs | 6 ++++-- datafusion/functions-aggregate/Cargo.toml | 2 +- datafusion/physical-expr/src/expressions/column.rs | 2 -- 6 files changed, 12 insertions(+), 8 deletions(-) diff --git a/datafusion/expr-common/src/lib.rs b/datafusion/expr-common/src/lib.rs index 4d5fdde38d32..179dd75ace85 100644 --- a/datafusion/expr-common/src/lib.rs +++ b/datafusion/expr-common/src/lib.rs @@ -15,7 +15,11 @@ // specific language governing permissions and limitations // under the License. -//! Logical Expr Common packages for [DataFusion] +//! Logical Expr types and traits for [DataFusion] +//! +//! This crate contains types and traits that are used by both Logical and Physical expressions. +//! They are kept in their own crate to avoid physical expressions depending on logical expressions. +//! //! //! [DataFusion]: diff --git a/datafusion/functions-aggregate-common/Cargo.toml b/datafusion/functions-aggregate-common/Cargo.toml index 1d0aaffa7974..a8296ce11f30 100644 --- a/datafusion/functions-aggregate-common/Cargo.toml +++ b/datafusion/functions-aggregate-common/Cargo.toml @@ -17,7 +17,7 @@ [package] name = "datafusion-functions-aggregate-common" -description = "Common aggregate function packages for the DataFusion query engine" +description = "Utility functions for implementing aggregate functions for the DataFusion query engine" keywords = ["datafusion", "logical", "plan", "expressions"] readme = "README.md" version = { workspace = true } diff --git a/datafusion/functions-aggregate-common/src/aggregate.rs b/datafusion/functions-aggregate-common/src/aggregate.rs index 69a55b76317f..016e54e68835 100644 --- a/datafusion/functions-aggregate-common/src/aggregate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Contains the trait `AggregateExpr` which defines the interface all aggregate expressions +//! [`AggregateExpr`] which defines the interface all aggregate expressions //! (built-in and custom) need to satisfy. use crate::order::AggregateOrderSensitivity; diff --git a/datafusion/functions-aggregate-common/src/lib.rs b/datafusion/functions-aggregate-common/src/lib.rs index eb86e05e2af1..cc50ff70913b 100644 --- a/datafusion/functions-aggregate-common/src/lib.rs +++ b/datafusion/functions-aggregate-common/src/lib.rs @@ -15,8 +15,10 @@ // specific language governing permissions and limitations // under the License. -//! Aggregate Function Common packages for [DataFusion] -//! This package could be used to build for 3rd party aggregate function +//! Common Aggregate functionality for [DataFusion] +//! +//! This crate contains traits and utilities commonly used to implement aggregate functions +//! They are kept in their own crate to avoid physical expressions depending on logical expressions. //! //! [DataFusion]: diff --git a/datafusion/functions-aggregate/Cargo.toml b/datafusion/functions-aggregate/Cargo.toml index 0006f2578145..636b2e42d236 100644 --- a/datafusion/functions-aggregate/Cargo.toml +++ b/datafusion/functions-aggregate/Cargo.toml @@ -17,7 +17,7 @@ [package] name = "datafusion-functions-aggregate" -description = "Aggregate function packages for the DataFusion query engine" +description = "Traits and types for logical plans and expressions for DataFusion query engine" keywords = ["datafusion", "logical", "plan", "expressions"] readme = "README.md" version = { workspace = true } diff --git a/datafusion/physical-expr/src/expressions/column.rs b/datafusion/physical-expr/src/expressions/column.rs index adf43ee545b4..79d15fdb02e8 100644 --- a/datafusion/physical-expr/src/expressions/column.rs +++ b/datafusion/physical-expr/src/expressions/column.rs @@ -138,8 +138,6 @@ pub fn col(name: &str, schema: &Schema) -> Result> { Ok(Arc::new(Column::new_with_schema(name, schema)?)) } -// TODO: Move expressions out of physical-expr? - /// Rewrites an expression according to new schema; i.e. changes the columns it /// refers to with the column at corresponding index in the new schema. Returns /// an error if the given schema has fewer columns than the original schema. From 96b72e883050b281d2a8018d7cc7c77479fc9a43 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 8 Aug 2024 21:38:46 +0800 Subject: [PATCH 25/26] move group accumulator adapter to functions-aggregate-common Signed-off-by: jayzhan211 --- .../src/aggregate/groups_accumulator.rs | 386 +++++++++++++++++ .../aggregate/groups_accumulator/adapter.rs | 403 ------------------ .../src/aggregate/groups_accumulator/mod.rs | 27 -- datafusion/physical-expr/src/aggregate/mod.rs | 28 -- .../physical-expr/src/aggregate/stats.rs | 18 - datafusion/physical-expr/src/lib.rs | 23 +- 6 files changed, 408 insertions(+), 477 deletions(-) delete mode 100644 datafusion/physical-expr/src/aggregate/groups_accumulator/adapter.rs delete mode 100644 datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs delete mode 100644 datafusion/physical-expr/src/aggregate/mod.rs delete mode 100644 datafusion/physical-expr/src/aggregate/stats.rs diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 5b0182c5db8a..644221edd04d 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -16,7 +16,393 @@ // under the License. //! Utilities for implementing GroupsAccumulator +//! Adapter that makes [`GroupsAccumulator`] out of [`Accumulator`] pub mod accumulate; pub mod bool_op; pub mod prim_op; + +use arrow::{ + array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray, UInt32Builder}, + compute, + datatypes::UInt32Type, +}; +use datafusion_common::{ + arrow_datafusion_err, utils::get_arrayref_at_indices, DataFusionError, Result, + ScalarValue, +}; +use datafusion_expr_common::accumulator::Accumulator; +use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; + +/// An adapter that implements [`GroupsAccumulator`] for any [`Accumulator`] +/// +/// While [`Accumulator`] are simpler to implement and can support +/// more general calculations (like retractable window functions), +/// they are not as fast as a specialized `GroupsAccumulator`. This +/// interface bridges the gap so the group by operator only operates +/// in terms of [`Accumulator`]. +pub struct GroupsAccumulatorAdapter { + factory: Box Result> + Send>, + + /// state for each group, stored in group_index order + states: Vec, + + /// Current memory usage, in bytes. + /// + /// Note this is incrementally updated with deltas to avoid the + /// call to size() being a bottleneck. We saw size() being a + /// bottleneck in earlier implementations when there were many + /// distinct groups. + allocation_bytes: usize, +} + +struct AccumulatorState { + /// [`Accumulator`] that stores the per-group state + accumulator: Box, + + // scratch space: indexes in the input array that will be fed to + // this accumulator. Stores indexes as `u32` to match the arrow + // `take` kernel input. + indices: Vec, +} + +impl AccumulatorState { + fn new(accumulator: Box) -> Self { + Self { + accumulator, + indices: vec![], + } + } + + /// Returns the amount of memory taken by this structure and its accumulator + fn size(&self) -> usize { + self.accumulator.size() + + std::mem::size_of_val(self) + + self.indices.allocated_size() + } +} + +impl GroupsAccumulatorAdapter { + /// Create a new adapter that will create a new [`Accumulator`] + /// for each group, using the specified factory function + pub fn new(factory: F) -> Self + where + F: Fn() -> Result> + Send + 'static, + { + Self { + factory: Box::new(factory), + states: vec![], + allocation_bytes: 0, + } + } + + /// Ensure that self.accumulators has total_num_groups + fn make_accumulators_if_needed(&mut self, total_num_groups: usize) -> Result<()> { + // can't shrink + assert!(total_num_groups >= self.states.len()); + let vec_size_pre = self.states.allocated_size(); + + // instantiate new accumulators + let new_accumulators = total_num_groups - self.states.len(); + for _ in 0..new_accumulators { + let accumulator = (self.factory)()?; + let state = AccumulatorState::new(accumulator); + self.add_allocation(state.size()); + self.states.push(state); + } + + self.adjust_allocation(vec_size_pre, self.states.allocated_size()); + Ok(()) + } + + /// invokes f(accumulator, values) for each group that has values + /// in group_indices. + /// + /// This function first reorders the input and filter so that + /// values for each group_index are contiguous and then invokes f + /// on the contiguous ranges, to minimize per-row overhead + /// + /// ```text + /// ┌─────────┐ ┌─────────┐ ┌ ─ ─ ─ ─ ┐ ┌─────────┐ ┌ ─ ─ ─ ─ ┐ + /// │ ┌─────┐ │ │ ┌─────┐ │ ┌─────┐ ┏━━━━━┓ │ ┌─────┐ │ ┌─────┐ + /// │ │ 2 │ │ │ │ 200 │ │ │ │ t │ │ ┃ 0 ┃ │ │ 200 │ │ │ │ t │ │ + /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ ┣━━━━━┫ │ ├─────┤ │ ├─────┤ + /// │ │ 2 │ │ │ │ 100 │ │ │ │ f │ │ ┃ 0 ┃ │ │ 300 │ │ │ │ t │ │ + /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ ┣━━━━━┫ │ ├─────┤ │ ├─────┤ + /// │ │ 0 │ │ │ │ 200 │ │ │ │ t │ │ ┃ 1 ┃ │ │ 200 │ │ │ │NULL │ │ + /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ ────────▶ ┣━━━━━┫ │ ├─────┤ │ ├─────┤ + /// │ │ 1 │ │ │ │ 200 │ │ │ │NULL │ │ ┃ 2 ┃ │ │ 200 │ │ │ │ t │ │ + /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ ┣━━━━━┫ │ ├─────┤ │ ├─────┤ + /// │ │ 0 │ │ │ │ 300 │ │ │ │ t │ │ ┃ 2 ┃ │ │ 100 │ │ │ │ f │ │ + /// │ └─────┘ │ │ └─────┘ │ └─────┘ ┗━━━━━┛ │ └─────┘ │ └─────┘ + /// └─────────┘ └─────────┘ └ ─ ─ ─ ─ ┘ └─────────┘ └ ─ ─ ─ ─ ┘ + /// + /// logical group values opt_filter logical group values opt_filter + /// + /// ``` + fn invoke_per_accumulator( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + f: F, + ) -> Result<()> + where + F: Fn(&mut dyn Accumulator, &[ArrayRef]) -> Result<()>, + { + self.make_accumulators_if_needed(total_num_groups)?; + + assert_eq!(values[0].len(), group_indices.len()); + + // figure out which input rows correspond to which groups. + // Note that self.state.indices starts empty for all groups + // (it is cleared out below) + for (idx, group_index) in group_indices.iter().enumerate() { + self.states[*group_index].indices.push(idx as u32); + } + + // groups_with_rows holds a list of group indexes that have + // any rows that need to be accumulated, stored in order of + // group_index + + let mut groups_with_rows = vec![]; + + // batch_indices holds indices into values, each group is contiguous + let mut batch_indices = UInt32Builder::with_capacity(0); + + // offsets[i] is index into batch_indices where the rows for + // group_index i starts + let mut offsets = vec![0]; + + let mut offset_so_far = 0; + for (group_index, state) in self.states.iter_mut().enumerate() { + let indices = &state.indices; + if indices.is_empty() { + continue; + } + + groups_with_rows.push(group_index); + batch_indices.append_slice(indices); + offset_so_far += indices.len(); + offsets.push(offset_so_far); + } + let batch_indices = batch_indices.finish(); + + // reorder the values and opt_filter by batch_indices so that + // all values for each group are contiguous, then invoke the + // accumulator once per group with values + let values = get_arrayref_at_indices(values, &batch_indices)?; + let opt_filter = get_filter_at_indices(opt_filter, &batch_indices)?; + + // invoke each accumulator with the appropriate rows, first + // pulling the input arguments for this group into their own + // RecordBatch(es) + let iter = groups_with_rows.iter().zip(offsets.windows(2)); + + let mut sizes_pre = 0; + let mut sizes_post = 0; + for (&group_idx, offsets) in iter { + let state = &mut self.states[group_idx]; + sizes_pre += state.size(); + + let values_to_accumulate = + slice_and_maybe_filter(&values, opt_filter.as_ref(), offsets)?; + (f)(state.accumulator.as_mut(), &values_to_accumulate)?; + + // clear out the state so they are empty for next + // iteration + state.indices.clear(); + sizes_post += state.size(); + } + + self.adjust_allocation(sizes_pre, sizes_post); + Ok(()) + } + + /// Increment the allocation by `n` + /// + /// See [`Self::allocation_bytes`] for rationale. + fn add_allocation(&mut self, size: usize) { + self.allocation_bytes += size; + } + + /// Decrease the allocation by `n` + /// + /// See [`Self::allocation_bytes`] for rationale. + fn free_allocation(&mut self, size: usize) { + // use saturating sub to avoid errors if the accumulators + // report erronious sizes + self.allocation_bytes = self.allocation_bytes.saturating_sub(size) + } + + /// Adjusts the allocation for something that started with + /// start_size and now has new_size avoiding overflow + /// + /// See [`Self::allocation_bytes`] for rationale. + fn adjust_allocation(&mut self, old_size: usize, new_size: usize) { + if new_size > old_size { + self.add_allocation(new_size - old_size) + } else { + self.free_allocation(old_size - new_size) + } + } +} + +impl GroupsAccumulator for GroupsAccumulatorAdapter { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + self.invoke_per_accumulator( + values, + group_indices, + opt_filter, + total_num_groups, + |accumulator, values_to_accumulate| { + accumulator.update_batch(values_to_accumulate) + }, + )?; + Ok(()) + } + + fn evaluate(&mut self, emit_to: EmitTo) -> Result { + let vec_size_pre = self.states.allocated_size(); + + let states = emit_to.take_needed(&mut self.states); + + let results: Vec = states + .into_iter() + .map(|mut state| { + self.free_allocation(state.size()); + state.accumulator.evaluate() + }) + .collect::>()?; + + let result = ScalarValue::iter_to_array(results); + + self.adjust_allocation(vec_size_pre, self.states.allocated_size()); + + result + } + + fn state(&mut self, emit_to: EmitTo) -> Result> { + let vec_size_pre = self.states.allocated_size(); + let states = emit_to.take_needed(&mut self.states); + + // each accumulator produces a potential vector of values + // which we need to form into columns + let mut results: Vec> = vec![]; + + for mut state in states { + self.free_allocation(state.size()); + let accumulator_state = state.accumulator.state()?; + results.resize_with(accumulator_state.len(), Vec::new); + for (idx, state_val) in accumulator_state.into_iter().enumerate() { + results[idx].push(state_val); + } + } + + // create an array for each intermediate column + let arrays = results + .into_iter() + .map(ScalarValue::iter_to_array) + .collect::>>()?; + + // double check each array has the same length (aka the + // accumulator was implemented correctly + if let Some(first_col) = arrays.first() { + for arr in &arrays { + assert_eq!(arr.len(), first_col.len()) + } + } + self.adjust_allocation(vec_size_pre, self.states.allocated_size()); + + Ok(arrays) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + self.invoke_per_accumulator( + values, + group_indices, + opt_filter, + total_num_groups, + |accumulator, values_to_accumulate| { + accumulator.merge_batch(values_to_accumulate)?; + Ok(()) + }, + )?; + Ok(()) + } + + fn size(&self) -> usize { + self.allocation_bytes + } +} + +/// Extension trait for [`Vec`] to account for allocations. +pub trait VecAllocExt { + /// Item type. + type T; + /// Return the amount of memory allocated by this Vec (not + /// recursively counting any heap allocations contained within the + /// structure). Does not include the size of `self` + fn allocated_size(&self) -> usize; +} + +impl VecAllocExt for Vec { + type T = T; + fn allocated_size(&self) -> usize { + std::mem::size_of::() * self.capacity() + } +} + +fn get_filter_at_indices( + opt_filter: Option<&BooleanArray>, + indices: &PrimitiveArray, +) -> Result> { + opt_filter + .map(|filter| { + compute::take( + &filter, indices, None, // None: no index check + ) + }) + .transpose() + .map_err(|e| arrow_datafusion_err!(e)) +} + +// Copied from physical-plan +pub(crate) fn slice_and_maybe_filter( + aggr_array: &[ArrayRef], + filter_opt: Option<&ArrayRef>, + offsets: &[usize], +) -> Result> { + let (offset, length) = (offsets[0], offsets[1] - offsets[0]); + let sliced_arrays: Vec = aggr_array + .iter() + .map(|array| array.slice(offset, length)) + .collect(); + + if let Some(f) = filter_opt { + let filter_array = f.slice(offset, length); + let filter_array = filter_array.as_boolean(); + + sliced_arrays + .iter() + .map(|array| { + compute::filter(array, filter_array).map_err(|e| arrow_datafusion_err!(e)) + }) + .collect() + } else { + Ok(sliced_arrays) + } +} diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/adapter.rs b/datafusion/physical-expr/src/aggregate/groups_accumulator/adapter.rs deleted file mode 100644 index 592c130b69d8..000000000000 --- a/datafusion/physical-expr/src/aggregate/groups_accumulator/adapter.rs +++ /dev/null @@ -1,403 +0,0 @@ -// 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. - -//! Adapter that makes [`GroupsAccumulator`] out of [`Accumulator`] - -use arrow::{ - array::{AsArray, UInt32Builder}, - compute, - datatypes::UInt32Type, -}; -use arrow_array::{ArrayRef, BooleanArray, PrimitiveArray}; -use datafusion_common::{ - arrow_datafusion_err, utils::get_arrayref_at_indices, DataFusionError, Result, - ScalarValue, -}; -use datafusion_expr::{Accumulator, EmitTo, GroupsAccumulator}; - -/// An adapter that implements [`GroupsAccumulator`] for any [`Accumulator`] -/// -/// While [`Accumulator`] are simpler to implement and can support -/// more general calculations (like retractable window functions), -/// they are not as fast as a specialized `GroupsAccumulator`. This -/// interface bridges the gap so the group by operator only operates -/// in terms of [`Accumulator`]. -pub struct GroupsAccumulatorAdapter { - factory: Box Result> + Send>, - - /// state for each group, stored in group_index order - states: Vec, - - /// Current memory usage, in bytes. - /// - /// Note this is incrementally updated with deltas to avoid the - /// call to size() being a bottleneck. We saw size() being a - /// bottleneck in earlier implementations when there were many - /// distinct groups. - allocation_bytes: usize, -} - -struct AccumulatorState { - /// [`Accumulator`] that stores the per-group state - accumulator: Box, - - // scratch space: indexes in the input array that will be fed to - // this accumulator. Stores indexes as `u32` to match the arrow - // `take` kernel input. - indices: Vec, -} - -impl AccumulatorState { - fn new(accumulator: Box) -> Self { - Self { - accumulator, - indices: vec![], - } - } - - /// Returns the amount of memory taken by this structure and its accumulator - fn size(&self) -> usize { - self.accumulator.size() - + std::mem::size_of_val(self) - + self.indices.allocated_size() - } -} - -impl GroupsAccumulatorAdapter { - /// Create a new adapter that will create a new [`Accumulator`] - /// for each group, using the specified factory function - pub fn new(factory: F) -> Self - where - F: Fn() -> Result> + Send + 'static, - { - Self { - factory: Box::new(factory), - states: vec![], - allocation_bytes: 0, - } - } - - /// Ensure that self.accumulators has total_num_groups - fn make_accumulators_if_needed(&mut self, total_num_groups: usize) -> Result<()> { - // can't shrink - assert!(total_num_groups >= self.states.len()); - let vec_size_pre = self.states.allocated_size(); - - // instantiate new accumulators - let new_accumulators = total_num_groups - self.states.len(); - for _ in 0..new_accumulators { - let accumulator = (self.factory)()?; - let state = AccumulatorState::new(accumulator); - self.add_allocation(state.size()); - self.states.push(state); - } - - self.adjust_allocation(vec_size_pre, self.states.allocated_size()); - Ok(()) - } - - /// invokes f(accumulator, values) for each group that has values - /// in group_indices. - /// - /// This function first reorders the input and filter so that - /// values for each group_index are contiguous and then invokes f - /// on the contiguous ranges, to minimize per-row overhead - /// - /// ```text - /// ┌─────────┐ ┌─────────┐ ┌ ─ ─ ─ ─ ┐ ┌─────────┐ ┌ ─ ─ ─ ─ ┐ - /// │ ┌─────┐ │ │ ┌─────┐ │ ┌─────┐ ┏━━━━━┓ │ ┌─────┐ │ ┌─────┐ - /// │ │ 2 │ │ │ │ 200 │ │ │ │ t │ │ ┃ 0 ┃ │ │ 200 │ │ │ │ t │ │ - /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ ┣━━━━━┫ │ ├─────┤ │ ├─────┤ - /// │ │ 2 │ │ │ │ 100 │ │ │ │ f │ │ ┃ 0 ┃ │ │ 300 │ │ │ │ t │ │ - /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ ┣━━━━━┫ │ ├─────┤ │ ├─────┤ - /// │ │ 0 │ │ │ │ 200 │ │ │ │ t │ │ ┃ 1 ┃ │ │ 200 │ │ │ │NULL │ │ - /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ ────────▶ ┣━━━━━┫ │ ├─────┤ │ ├─────┤ - /// │ │ 1 │ │ │ │ 200 │ │ │ │NULL │ │ ┃ 2 ┃ │ │ 200 │ │ │ │ t │ │ - /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ ┣━━━━━┫ │ ├─────┤ │ ├─────┤ - /// │ │ 0 │ │ │ │ 300 │ │ │ │ t │ │ ┃ 2 ┃ │ │ 100 │ │ │ │ f │ │ - /// │ └─────┘ │ │ └─────┘ │ └─────┘ ┗━━━━━┛ │ └─────┘ │ └─────┘ - /// └─────────┘ └─────────┘ └ ─ ─ ─ ─ ┘ └─────────┘ └ ─ ─ ─ ─ ┘ - /// - /// logical group values opt_filter logical group values opt_filter - /// - /// ``` - fn invoke_per_accumulator( - &mut self, - values: &[ArrayRef], - group_indices: &[usize], - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - f: F, - ) -> Result<()> - where - F: Fn(&mut dyn Accumulator, &[ArrayRef]) -> Result<()>, - { - self.make_accumulators_if_needed(total_num_groups)?; - - assert_eq!(values[0].len(), group_indices.len()); - - // figure out which input rows correspond to which groups. - // Note that self.state.indices starts empty for all groups - // (it is cleared out below) - for (idx, group_index) in group_indices.iter().enumerate() { - self.states[*group_index].indices.push(idx as u32); - } - - // groups_with_rows holds a list of group indexes that have - // any rows that need to be accumulated, stored in order of - // group_index - - let mut groups_with_rows = vec![]; - - // batch_indices holds indices into values, each group is contiguous - let mut batch_indices = UInt32Builder::with_capacity(0); - - // offsets[i] is index into batch_indices where the rows for - // group_index i starts - let mut offsets = vec![0]; - - let mut offset_so_far = 0; - for (group_index, state) in self.states.iter_mut().enumerate() { - let indices = &state.indices; - if indices.is_empty() { - continue; - } - - groups_with_rows.push(group_index); - batch_indices.append_slice(indices); - offset_so_far += indices.len(); - offsets.push(offset_so_far); - } - let batch_indices = batch_indices.finish(); - - // reorder the values and opt_filter by batch_indices so that - // all values for each group are contiguous, then invoke the - // accumulator once per group with values - let values = get_arrayref_at_indices(values, &batch_indices)?; - let opt_filter = get_filter_at_indices(opt_filter, &batch_indices)?; - - // invoke each accumulator with the appropriate rows, first - // pulling the input arguments for this group into their own - // RecordBatch(es) - let iter = groups_with_rows.iter().zip(offsets.windows(2)); - - let mut sizes_pre = 0; - let mut sizes_post = 0; - for (&group_idx, offsets) in iter { - let state = &mut self.states[group_idx]; - sizes_pre += state.size(); - - let values_to_accumulate = - slice_and_maybe_filter(&values, opt_filter.as_ref(), offsets)?; - (f)(state.accumulator.as_mut(), &values_to_accumulate)?; - - // clear out the state so they are empty for next - // iteration - state.indices.clear(); - sizes_post += state.size(); - } - - self.adjust_allocation(sizes_pre, sizes_post); - Ok(()) - } - - /// Increment the allocation by `n` - /// - /// See [`Self::allocation_bytes`] for rationale. - fn add_allocation(&mut self, size: usize) { - self.allocation_bytes += size; - } - - /// Decrease the allocation by `n` - /// - /// See [`Self::allocation_bytes`] for rationale. - fn free_allocation(&mut self, size: usize) { - // use saturating sub to avoid errors if the accumulators - // report erronious sizes - self.allocation_bytes = self.allocation_bytes.saturating_sub(size) - } - - /// Adjusts the allocation for something that started with - /// start_size and now has new_size avoiding overflow - /// - /// See [`Self::allocation_bytes`] for rationale. - fn adjust_allocation(&mut self, old_size: usize, new_size: usize) { - if new_size > old_size { - self.add_allocation(new_size - old_size) - } else { - self.free_allocation(old_size - new_size) - } - } -} - -impl GroupsAccumulator for GroupsAccumulatorAdapter { - fn update_batch( - &mut self, - values: &[ArrayRef], - group_indices: &[usize], - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - ) -> Result<()> { - self.invoke_per_accumulator( - values, - group_indices, - opt_filter, - total_num_groups, - |accumulator, values_to_accumulate| { - accumulator.update_batch(values_to_accumulate) - }, - )?; - Ok(()) - } - - fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let vec_size_pre = self.states.allocated_size(); - - let states = emit_to.take_needed(&mut self.states); - - let results: Vec = states - .into_iter() - .map(|mut state| { - self.free_allocation(state.size()); - state.accumulator.evaluate() - }) - .collect::>()?; - - let result = ScalarValue::iter_to_array(results); - - self.adjust_allocation(vec_size_pre, self.states.allocated_size()); - - result - } - - fn state(&mut self, emit_to: EmitTo) -> Result> { - let vec_size_pre = self.states.allocated_size(); - let states = emit_to.take_needed(&mut self.states); - - // each accumulator produces a potential vector of values - // which we need to form into columns - let mut results: Vec> = vec![]; - - for mut state in states { - self.free_allocation(state.size()); - let accumulator_state = state.accumulator.state()?; - results.resize_with(accumulator_state.len(), Vec::new); - for (idx, state_val) in accumulator_state.into_iter().enumerate() { - results[idx].push(state_val); - } - } - - // create an array for each intermediate column - let arrays = results - .into_iter() - .map(ScalarValue::iter_to_array) - .collect::>>()?; - - // double check each array has the same length (aka the - // accumulator was implemented correctly - if let Some(first_col) = arrays.first() { - for arr in &arrays { - assert_eq!(arr.len(), first_col.len()) - } - } - self.adjust_allocation(vec_size_pre, self.states.allocated_size()); - - Ok(arrays) - } - - fn merge_batch( - &mut self, - values: &[ArrayRef], - group_indices: &[usize], - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - ) -> Result<()> { - self.invoke_per_accumulator( - values, - group_indices, - opt_filter, - total_num_groups, - |accumulator, values_to_accumulate| { - accumulator.merge_batch(values_to_accumulate)?; - Ok(()) - }, - )?; - Ok(()) - } - - fn size(&self) -> usize { - self.allocation_bytes - } -} - -/// Extension trait for [`Vec`] to account for allocations. -pub trait VecAllocExt { - /// Item type. - type T; - /// Return the amount of memory allocated by this Vec (not - /// recursively counting any heap allocations contained within the - /// structure). Does not include the size of `self` - fn allocated_size(&self) -> usize; -} - -impl VecAllocExt for Vec { - type T = T; - fn allocated_size(&self) -> usize { - std::mem::size_of::() * self.capacity() - } -} - -fn get_filter_at_indices( - opt_filter: Option<&BooleanArray>, - indices: &PrimitiveArray, -) -> Result> { - opt_filter - .map(|filter| { - compute::take( - &filter, indices, None, // None: no index check - ) - }) - .transpose() - .map_err(|e| arrow_datafusion_err!(e)) -} - -// Copied from physical-plan -pub(crate) fn slice_and_maybe_filter( - aggr_array: &[ArrayRef], - filter_opt: Option<&ArrayRef>, - offsets: &[usize], -) -> Result> { - let (offset, length) = (offsets[0], offsets[1] - offsets[0]); - let sliced_arrays: Vec = aggr_array - .iter() - .map(|array| array.slice(offset, length)) - .collect(); - - if let Some(f) = filter_opt { - let filter_array = f.slice(offset, length); - let filter_array = filter_array.as_boolean(); - - sliced_arrays - .iter() - .map(|array| { - compute::filter(array, filter_array).map_err(|e| arrow_datafusion_err!(e)) - }) - .collect() - } else { - Ok(sliced_arrays) - } -} diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs b/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs deleted file mode 100644 index d2ea90ec3007..000000000000 --- a/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs +++ /dev/null @@ -1,27 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -mod adapter; -pub use adapter::GroupsAccumulatorAdapter; - -// Backward compatibility -#[allow(unused_imports)] -pub(crate) mod accumulate { - pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::NullState; -} - -pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::NullState; diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs deleted file mode 100644 index 673d5305f81d..000000000000 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ /dev/null @@ -1,28 +0,0 @@ -// 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(crate) mod groups_accumulator; -pub(crate) mod stats; - -pub mod utils { - pub use datafusion_functions_aggregate_common::utils::{ - adjust_output_array, down_cast_any_ref, get_accum_scalar_values_as_arrays, - get_sort_options, ordering_fields, DecimalAverager, Hashable, - }; -} - -pub use datafusion_functions_aggregate_common::aggregate::AggregateExpr; diff --git a/datafusion/physical-expr/src/aggregate/stats.rs b/datafusion/physical-expr/src/aggregate/stats.rs deleted file mode 100644 index db1934cd1ad9..000000000000 --- a/datafusion/physical-expr/src/aggregate/stats.rs +++ /dev/null @@ -1,18 +0,0 @@ -// 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 use datafusion_functions_aggregate_common::stats::StatsType; diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 9a9120f1ddac..c4255172d680 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -18,7 +18,28 @@ // Make cheap clones clear: https://github.com/apache/datafusion/issues/11143 #![deny(clippy::clone_on_ref_ptr)] -pub mod aggregate; +// Backward compatibility +pub mod aggregate { + pub(crate) mod groups_accumulator { + #[allow(unused_imports)] + pub(crate) mod accumulate { + pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::NullState; + } + pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ + accumulate::NullState, GroupsAccumulatorAdapter, + }; + } + pub(crate) mod stats { + pub use datafusion_functions_aggregate_common::stats::StatsType; + } + pub mod utils { + pub use datafusion_functions_aggregate_common::utils::{ + adjust_output_array, down_cast_any_ref, get_accum_scalar_values_as_arrays, + get_sort_options, ordering_fields, DecimalAverager, Hashable, + }; + } + pub use datafusion_functions_aggregate_common::aggregate::AggregateExpr; +} pub mod analysis; pub mod binary_map { pub use datafusion_physical_expr_common::binary_map::{ArrowBytesSet, OutputType}; From 4943b40e623b784924786a99637a6e8352febb6b Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 9 Aug 2024 17:49:45 +0800 Subject: [PATCH 26/26] fix Signed-off-by: jayzhan211 --- datafusion-cli/Cargo.lock | 6 +++--- datafusion/expr/src/udf.rs | 16 ++++++---------- .../src/approx_percentile_cont.rs | 3 ++- 3 files changed, 11 insertions(+), 14 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 5e1223e9f222..134cde8976d6 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1291,7 +1291,7 @@ dependencies = [ [[package]] name = "datafusion-expr-common" -version = "40.0.0" +version = "41.0.0" dependencies = [ "arrow", "datafusion-common", @@ -1343,7 +1343,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" -version = "40.0.0" +version = "41.0.0" dependencies = [ "ahash", "arrow", @@ -1435,7 +1435,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-functions-aggregate" -version = "40.0.0" +version = "41.0.0" dependencies = [ "ahash", "arrow", diff --git a/datafusion/expr/src/udf.rs b/datafusion/expr/src/udf.rs index 2ebf08fe68c0..f5434726e23d 100644 --- a/datafusion/expr/src/udf.rs +++ b/datafusion/expr/src/udf.rs @@ -17,23 +17,19 @@ //! [`ScalarUDF`]: Scalar User Defined Functions -use std::any::Any; -use std::fmt::{self, Debug, Formatter}; -use std::hash::{DefaultHasher, Hash, Hasher}; -use std::sync::Arc; - -use arrow::datatypes::DataType; - -use datafusion_common::{not_impl_err, ExprSchema, Result}; - use crate::expr::schema_name_from_exprs_comma_seperated_without_space; -use crate::interval_arithmetic::Interval; use crate::simplify::{ExprSimplifyResult, SimplifyInfo}; use crate::sort_properties::{ExprProperties, SortProperties}; use crate::{ ColumnarValue, Expr, ReturnTypeFunction, ScalarFunctionImplementation, Signature, }; +use arrow::datatypes::DataType; +use datafusion_common::{not_impl_err, ExprSchema, Result}; use datafusion_expr_common::interval_arithmetic::Interval; +use std::any::Any; +use std::fmt::{self, Debug, Formatter}; +use std::hash::{DefaultHasher, Hash, Hasher}; +use std::sync::Arc; /// Logical representation of a Scalar User Defined Function. /// diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index 79093b35b639..89d827e86859 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -38,7 +38,8 @@ use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::{INTEGERS, NUMERICS}; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ - Accumulator, AggregateUDFImpl, ColumnarValue, Signature, TypeSignature, Volatility, + Accumulator, AggregateUDFImpl, ColumnarValue, Expr, Signature, TypeSignature, + Volatility, }; use datafusion_functions_aggregate_common::tdigest::{ TDigest, TryIntoF64, DEFAULT_MAX_SIZE,