Skip to content

Commit

Permalink
Move array ArrayAgg to a UserDefinedAggregate (apache#11448)
Browse files Browse the repository at this point in the history
* Add input_nullable to UDAF args StateField/AccumulatorArgs

This follows how it done for input_type and only provide a single value.
But might need to be changed into a Vec in the future.

This is need when we are moving `arrag_agg` to udaf where one of the
states nullability will depend on the nullability of the input.

* Make ArragAgg (not ordered or distinct) into a UDAF

* Add roundtrip_expr_api test case

* Address PR comments

* Propegate input nullability for aggregates

* Remove from accumulator args

* first draft

Signed-off-by: jayzhan211 <[email protected]>

* cleanup

Signed-off-by: jayzhan211 <[email protected]>

* fix test

Signed-off-by: jayzhan211 <[email protected]>

* distinct

Signed-off-by: jayzhan211 <[email protected]>

* fix

Signed-off-by: jayzhan211 <[email protected]>

* address comment

Signed-off-by: jayzhan211 <[email protected]>

---------

Signed-off-by: jayzhan211 <[email protected]>
Co-authored-by: Emil Ejbyfeldt <[email protected]>
  • Loading branch information
2 people authored and Lordworms committed Jul 23, 2024
1 parent 8b9815a commit cf5f3be
Show file tree
Hide file tree
Showing 16 changed files with 328 additions and 892 deletions.
6 changes: 3 additions & 3 deletions datafusion/core/src/dataframe/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1696,10 +1696,10 @@ mod tests {
use datafusion_common::{Constraint, Constraints, ScalarValue};
use datafusion_common_runtime::SpawnedTask;
use datafusion_expr::{
array_agg, cast, create_udf, expr, lit, BuiltInWindowFunction,
ScalarFunctionImplementation, Volatility, WindowFrame, WindowFunctionDefinition,
cast, create_udf, expr, lit, BuiltInWindowFunction, ScalarFunctionImplementation,
Volatility, WindowFrame, WindowFunctionDefinition,
};
use datafusion_functions_aggregate::expr_fn::count_distinct;
use datafusion_functions_aggregate::expr_fn::{array_agg, count_distinct};
use datafusion_physical_expr::expressions::Column;
use datafusion_physical_plan::{get_plan_string, ExecutionPlanProperties};

Expand Down
28 changes: 28 additions & 0 deletions datafusion/core/src/physical_planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1839,7 +1839,34 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter(
.unwrap_or(sqlparser::ast::NullTreatment::RespectNulls)
== NullTreatment::IgnoreNulls;

// TODO: Remove this after array_agg are all udafs
let (agg_expr, filter, order_by) = match func_def {
AggregateFunctionDefinition::UDF(udf)
if udf.name() == "ARRAY_AGG" && order_by.is_some() =>
{
// not yet support UDAF, fallback to builtin
let physical_sort_exprs = match order_by {
Some(exprs) => Some(create_physical_sort_exprs(
exprs,
logical_input_schema,
execution_props,
)?),
None => None,
};
let ordering_reqs: Vec<PhysicalSortExpr> =
physical_sort_exprs.clone().unwrap_or(vec![]);
let fun = aggregates::AggregateFunction::ArrayAgg;
let agg_expr = aggregates::create_aggregate_expr(
&fun,
*distinct,
&physical_args,
&ordering_reqs,
physical_input_schema,
name,
ignore_nulls,
)?;
(agg_expr, filter, physical_sort_exprs)
}
AggregateFunctionDefinition::BuiltIn(fun) => {
let physical_sort_exprs = match order_by {
Some(exprs) => Some(create_physical_sort_exprs(
Expand Down Expand Up @@ -1888,6 +1915,7 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter(
(agg_expr, filter, physical_sort_exprs)
}
};

Ok((agg_expr, filter, order_by))
}
other => internal_err!("Invalid aggregate expression '{other:?}'"),
Expand Down
6 changes: 3 additions & 3 deletions datafusion/core/tests/dataframe/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -54,11 +54,11 @@ use datafusion_execution::runtime_env::RuntimeEnv;
use datafusion_expr::expr::{GroupingSet, Sort};
use datafusion_expr::var_provider::{VarProvider, VarType};
use datafusion_expr::{
array_agg, cast, col, exists, expr, in_subquery, lit, max, out_ref_col, placeholder,
cast, col, exists, expr, in_subquery, lit, max, out_ref_col, placeholder,
scalar_subquery, when, wildcard, Expr, ExprSchemable, WindowFrame, WindowFrameBound,
WindowFrameUnits, WindowFunctionDefinition,
};
use datafusion_functions_aggregate::expr_fn::{avg, count, sum};
use datafusion_functions_aggregate::expr_fn::{array_agg, avg, count, sum};

#[tokio::test]
async fn test_count_wildcard_on_sort() -> Result<()> {
Expand Down Expand Up @@ -1389,7 +1389,7 @@ async fn unnest_with_redundant_columns() -> Result<()> {
let expected = vec![
"Projection: shapes.shape_id [shape_id:UInt32]",
" Unnest: lists[shape_id2] structs[] [shape_id:UInt32, shape_id2:UInt32;N]",
" Aggregate: groupBy=[[shapes.shape_id]], aggr=[[ARRAY_AGG(shapes.shape_id) AS shape_id2]] [shape_id:UInt32, shape_id2:List(Field { name: \"item\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} });N]",
" Aggregate: groupBy=[[shapes.shape_id]], aggr=[[ARRAY_AGG(shapes.shape_id) AS shape_id2]] [shape_id:UInt32, shape_id2:List(Field { name: \"item\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} });N]",
" TableScan: shapes projection=[shape_id] [shape_id:UInt32]",
];

Expand Down
12 changes: 0 additions & 12 deletions datafusion/expr/src/expr_fn.rs
Original file line number Diff line number Diff line change
Expand Up @@ -171,18 +171,6 @@ pub fn max(expr: Expr) -> Expr {
))
}

/// Create an expression to represent the array_agg() aggregate function
pub fn array_agg(expr: Expr) -> Expr {
Expr::AggregateFunction(AggregateFunction::new(
aggregate_function::AggregateFunction::ArrayAgg,
vec![expr],
false,
None,
None,
None,
))
}

/// Return a new expression with bitwise AND
pub fn bitwise_and(left: Expr, right: Expr) -> Expr {
Expr::BinaryExpr(BinaryExpr::new(
Expand Down
261 changes: 261 additions & 0 deletions datafusion/functions-aggregate/src/array_agg.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,261 @@
// 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.

//! `ARRAY_AGG` aggregate implementation: [`ArrayAgg`]

use arrow::array::{Array, ArrayRef, AsArray};
use arrow::datatypes::DataType;
use arrow_schema::Field;

use datafusion_common::cast::as_list_array;
use datafusion_common::utils::array_into_list_array_nullable;
use datafusion_common::ScalarValue;
use datafusion_common::{internal_err, Result};
use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs};
use datafusion_expr::utils::format_state_name;
use datafusion_expr::AggregateUDFImpl;
use datafusion_expr::{Accumulator, Signature, Volatility};
use std::collections::HashSet;
use std::sync::Arc;

make_udaf_expr_and_func!(
ArrayAgg,
array_agg,
expression,
"input values, including nulls, concatenated into an array",
array_agg_udaf
);

#[derive(Debug)]
/// ARRAY_AGG aggregate expression
pub struct ArrayAgg {
signature: Signature,
alias: Vec<String>,
}

impl Default for ArrayAgg {
fn default() -> Self {
Self {
signature: Signature::any(1, Volatility::Immutable),
alias: vec!["array_agg".to_string()],
}
}
}

impl AggregateUDFImpl for ArrayAgg {
fn as_any(&self) -> &dyn std::any::Any {
self
}

// TODO: change name to lowercase
fn name(&self) -> &str {
"ARRAY_AGG"
}

fn aliases(&self) -> &[String] {
&self.alias
}

fn signature(&self) -> &Signature {
&self.signature
}

fn return_type(&self, arg_types: &[DataType]) -> Result<DataType> {
Ok(DataType::List(Arc::new(Field::new(
"item",
arg_types[0].clone(),
true,
))))
}

fn state_fields(&self, args: StateFieldsArgs) -> Result<Vec<Field>> {
if args.is_distinct {
return Ok(vec![Field::new_list(
format_state_name(args.name, "distinct_array_agg"),
Field::new("item", args.input_type.clone(), true),
true,
)]);
}

Ok(vec![Field::new_list(
format_state_name(args.name, "array_agg"),
Field::new("item", args.input_type.clone(), true),
true,
)])
}

fn accumulator(&self, acc_args: AccumulatorArgs) -> Result<Box<dyn Accumulator>> {
if acc_args.is_distinct {
return Ok(Box::new(DistinctArrayAggAccumulator::try_new(
acc_args.input_type,
)?));
}

Ok(Box::new(ArrayAggAccumulator::try_new(acc_args.input_type)?))
}
}

#[derive(Debug)]
pub struct ArrayAggAccumulator {
values: Vec<ArrayRef>,
datatype: DataType,
}

impl ArrayAggAccumulator {
/// new array_agg accumulator based on given item data type
pub fn try_new(datatype: &DataType) -> Result<Self> {
Ok(Self {
values: vec![],
datatype: datatype.clone(),
})
}
}

impl Accumulator for ArrayAggAccumulator {
fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> {
// Append value like Int64Array(1,2,3)
if values.is_empty() {
return Ok(());
}

if values.len() != 1 {
return internal_err!("expects single batch");
}

let val = Arc::clone(&values[0]);
if val.len() > 0 {
self.values.push(val);
}
Ok(())
}

fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> {
// Append value like ListArray(Int64Array(1,2,3), Int64Array(4,5,6))
if states.is_empty() {
return Ok(());
}

if states.len() != 1 {
return internal_err!("expects single state");
}

let list_arr = as_list_array(&states[0])?;
for arr in list_arr.iter().flatten() {
self.values.push(arr);
}
Ok(())
}

fn state(&mut self) -> Result<Vec<ScalarValue>> {
Ok(vec![self.evaluate()?])
}

fn evaluate(&mut self) -> Result<ScalarValue> {
// Transform Vec<ListArr> to ListArr
let element_arrays: Vec<&dyn Array> =
self.values.iter().map(|a| a.as_ref()).collect();

if element_arrays.is_empty() {
return Ok(ScalarValue::new_null_list(self.datatype.clone(), true, 1));
}

let concated_array = arrow::compute::concat(&element_arrays)?;
let list_array = array_into_list_array_nullable(concated_array);

Ok(ScalarValue::List(Arc::new(list_array)))
}

fn size(&self) -> usize {
std::mem::size_of_val(self)
+ (std::mem::size_of::<ArrayRef>() * self.values.capacity())
+ self
.values
.iter()
.map(|arr| arr.get_array_memory_size())
.sum::<usize>()
+ self.datatype.size()
- std::mem::size_of_val(&self.datatype)
}
}

#[derive(Debug)]
struct DistinctArrayAggAccumulator {
values: HashSet<ScalarValue>,
datatype: DataType,
}

impl DistinctArrayAggAccumulator {
pub fn try_new(datatype: &DataType) -> Result<Self> {
Ok(Self {
values: HashSet::new(),
datatype: datatype.clone(),
})
}
}

impl Accumulator for DistinctArrayAggAccumulator {
fn state(&mut self) -> Result<Vec<ScalarValue>> {
Ok(vec![self.evaluate()?])
}

fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> {
if values.len() != 1 {
return internal_err!("expects single batch");
}

let array = &values[0];

for i in 0..array.len() {
let scalar = ScalarValue::try_from_array(&array, i)?;
self.values.insert(scalar);
}

Ok(())
}

fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> {
if states.is_empty() {
return Ok(());
}

if states.len() != 1 {
return internal_err!("expects single state");
}

states[0]
.as_list::<i32>()
.iter()
.flatten()
.try_for_each(|val| self.update_batch(&[val]))
}

fn evaluate(&mut self) -> Result<ScalarValue> {
let values: Vec<ScalarValue> = self.values.iter().cloned().collect();
if values.is_empty() {
return Ok(ScalarValue::new_null_list(self.datatype.clone(), true, 1));
}
let arr = ScalarValue::new_list(&values, &self.datatype, true);
Ok(ScalarValue::List(arr))
}

fn size(&self) -> usize {
std::mem::size_of_val(self) + ScalarValue::size_of_hashset(&self.values)
- std::mem::size_of_val(&self.values)
+ self.datatype.size()
- std::mem::size_of_val(&self.datatype)
}
}
8 changes: 6 additions & 2 deletions datafusion/functions-aggregate/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@
pub mod macros;

pub mod approx_distinct;
pub mod array_agg;
pub mod correlation;
pub mod count;
pub mod covariance;
Expand Down Expand Up @@ -93,6 +94,7 @@ pub mod expr_fn {
pub use super::approx_median::approx_median;
pub use super::approx_percentile_cont::approx_percentile_cont;
pub use super::approx_percentile_cont_with_weight::approx_percentile_cont_with_weight;
pub use super::array_agg::array_agg;
pub use super::average::avg;
pub use super::bit_and_or_xor::bit_and;
pub use super::bit_and_or_xor::bit_or;
Expand Down Expand Up @@ -128,6 +130,7 @@ pub mod expr_fn {
/// Returns all default aggregate functions
pub fn all_default_aggregate_functions() -> Vec<Arc<AggregateUDF>> {
vec![
array_agg::array_agg_udaf(),
first_last::first_value_udaf(),
first_last::last_value_udaf(),
covariance::covar_samp_udaf(),
Expand Down Expand Up @@ -191,8 +194,9 @@ mod tests {
let mut names = HashSet::new();
for func in all_default_aggregate_functions() {
// TODO: remove this
// These functions are in intermediate migration state, skip them
if func.name().to_lowercase() == "count" {
// These functions are in intermidiate migration state, skip them
let name_lower_case = func.name().to_lowercase();
if name_lower_case == "count" || name_lower_case == "array_agg" {
continue;
}
assert!(
Expand Down
Loading

0 comments on commit cf5f3be

Please sign in to comment.