Skip to content

Commit

Permalink
feat: impl the basic string_agg function (#8148)
Browse files Browse the repository at this point in the history
* init impl

* add support for larget utf8

* add some test

* support null

* remove redundance code

* remove redundance code

* add more test

* Update datafusion/physical-expr/src/aggregate/string_agg.rs

Co-authored-by: universalmind303 <[email protected]>

* Update datafusion/physical-expr/src/aggregate/string_agg.rs

Co-authored-by: universalmind303 <[email protected]>

* add suggest

* Update datafusion/physical-expr/src/aggregate/string_agg.rs

Co-authored-by: Andrew Lamb <[email protected]>

* Update datafusion/sqllogictest/test_files/aggregate.slt

Co-authored-by: Andrew Lamb <[email protected]>

* Update datafusion/sqllogictest/test_files/aggregate.slt

Co-authored-by: Andrew Lamb <[email protected]>

* fix ci

---------

Co-authored-by: universalmind303 <[email protected]>
Co-authored-by: Andrew Lamb <[email protected]>
  • Loading branch information
3 people authored Nov 18, 2023
1 parent a984f08 commit 76ced31
Show file tree
Hide file tree
Showing 12 changed files with 386 additions and 0 deletions.
8 changes: 8 additions & 0 deletions datafusion/expr/src/aggregate_function.rs
Original file line number Diff line number Diff line change
Expand Up @@ -100,6 +100,8 @@ pub enum AggregateFunction {
BoolAnd,
/// Bool Or
BoolOr,
/// string_agg
StringAgg,
}

impl AggregateFunction {
Expand Down Expand Up @@ -141,6 +143,7 @@ impl AggregateFunction {
BitXor => "BIT_XOR",
BoolAnd => "BOOL_AND",
BoolOr => "BOOL_OR",
StringAgg => "STRING_AGG",
}
}
}
Expand Down Expand Up @@ -171,6 +174,7 @@ impl FromStr for AggregateFunction {
"array_agg" => AggregateFunction::ArrayAgg,
"first_value" => AggregateFunction::FirstValue,
"last_value" => AggregateFunction::LastValue,
"string_agg" => AggregateFunction::StringAgg,
// statistical
"corr" => AggregateFunction::Correlation,
"covar" => AggregateFunction::Covariance,
Expand Down Expand Up @@ -299,6 +303,7 @@ impl AggregateFunction {
AggregateFunction::FirstValue | AggregateFunction::LastValue => {
Ok(coerced_data_types[0].clone())
}
AggregateFunction::StringAgg => Ok(DataType::LargeUtf8),
}
}
}
Expand Down Expand Up @@ -408,6 +413,9 @@ impl AggregateFunction {
.collect(),
Volatility::Immutable,
),
AggregateFunction::StringAgg => {
Signature::uniform(2, STRINGS.to_vec(), Volatility::Immutable)
}
}
}
}
Expand Down
26 changes: 26 additions & 0 deletions datafusion/expr/src/type_coercion/aggregates.rs
Original file line number Diff line number Diff line change
Expand Up @@ -298,6 +298,23 @@ pub fn coerce_types(
| AggregateFunction::FirstValue
| AggregateFunction::LastValue => Ok(input_types.to_vec()),
AggregateFunction::Grouping => Ok(vec![input_types[0].clone()]),
AggregateFunction::StringAgg => {
if !is_string_agg_supported_arg_type(&input_types[0]) {
return plan_err!(
"The function {:?} does not support inputs of type {:?}",
agg_fun,
input_types[0]
);
}
if !is_string_agg_supported_arg_type(&input_types[1]) {
return plan_err!(
"The function {:?} does not support inputs of type {:?}",
agg_fun,
input_types[1]
);
}
Ok(vec![LargeUtf8, input_types[1].clone()])
}
}
}

Expand Down Expand Up @@ -565,6 +582,15 @@ pub fn is_approx_percentile_cont_supported_arg_type(arg_type: &DataType) -> bool
)
}

/// Return `true` if `arg_type` is of a [`DataType`] that the
/// [`AggregateFunction::StringAgg`] aggregation can operate on.
pub fn is_string_agg_supported_arg_type(arg_type: &DataType) -> bool {
matches!(
arg_type,
DataType::Utf8 | DataType::LargeUtf8 | DataType::Null
)
}

#[cfg(test)]
mod tests {
use super::*;
Expand Down
16 changes: 16 additions & 0 deletions datafusion/physical-expr/src/aggregate/build_in.rs
Original file line number Diff line number Diff line change
Expand Up @@ -369,6 +369,22 @@ pub fn create_aggregate_expr(
ordering_req.to_vec(),
ordering_types,
)),
(AggregateFunction::StringAgg, false) => {
if !ordering_req.is_empty() {
return not_impl_err!(
"STRING_AGG(ORDER BY a ASC) order-sensitive aggregations are not available"
);
}
Arc::new(expressions::StringAgg::new(
input_phy_exprs[0].clone(),
input_phy_exprs[1].clone(),
name,
data_type,
))
}
(AggregateFunction::StringAgg, true) => {
return not_impl_err!("STRING_AGG(DISTINCT) aggregations are not available");
}
})
}

Expand Down
1 change: 1 addition & 0 deletions datafusion/physical-expr/src/aggregate/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ pub(crate) mod covariance;
pub(crate) mod first_last;
pub(crate) mod grouping;
pub(crate) mod median;
pub(crate) mod string_agg;
#[macro_use]
pub(crate) mod min_max;
pub mod build_in;
Expand Down
246 changes: 246 additions & 0 deletions datafusion/physical-expr/src/aggregate/string_agg.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,246 @@
// 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.

//! [`StringAgg`] and [`StringAggAccumulator`] accumulator for the `string_agg` function
use crate::aggregate::utils::down_cast_any_ref;
use crate::expressions::{format_state_name, Literal};
use crate::{AggregateExpr, PhysicalExpr};
use arrow::array::ArrayRef;
use arrow::datatypes::{DataType, Field};
use datafusion_common::cast::as_generic_string_array;
use datafusion_common::{not_impl_err, DataFusionError, Result, ScalarValue};
use datafusion_expr::Accumulator;
use std::any::Any;
use std::sync::Arc;

/// STRING_AGG aggregate expression
#[derive(Debug)]
pub struct StringAgg {
name: String,
data_type: DataType,
expr: Arc<dyn PhysicalExpr>,
delimiter: Arc<dyn PhysicalExpr>,
nullable: bool,
}

impl StringAgg {
/// Create a new StringAgg aggregate function
pub fn new(
expr: Arc<dyn PhysicalExpr>,
delimiter: Arc<dyn PhysicalExpr>,
name: impl Into<String>,
data_type: DataType,
) -> Self {
Self {
name: name.into(),
data_type,
delimiter,
expr,
nullable: true,
}
}
}

impl AggregateExpr for StringAgg {
fn as_any(&self) -> &dyn Any {
self
}

fn field(&self) -> Result<Field> {
Ok(Field::new(
&self.name,
self.data_type.clone(),
self.nullable,
))
}

fn create_accumulator(&self) -> Result<Box<dyn Accumulator>> {
if let Some(delimiter) = self.delimiter.as_any().downcast_ref::<Literal>() {
match delimiter.value() {
ScalarValue::Utf8(Some(delimiter))
| ScalarValue::LargeUtf8(Some(delimiter)) => {
return Ok(Box::new(StringAggAccumulator::new(delimiter)));
}
ScalarValue::Null => {
return Ok(Box::new(StringAggAccumulator::new("")));
}
_ => return not_impl_err!("StringAgg not supported for {}", self.name),
}
}
not_impl_err!("StringAgg not supported for {}", self.name)
}

fn state_fields(&self) -> Result<Vec<Field>> {
Ok(vec![Field::new(
format_state_name(&self.name, "string_agg"),
self.data_type.clone(),
self.nullable,
)])
}

fn expressions(&self) -> Vec<Arc<dyn PhysicalExpr>> {
vec![self.expr.clone(), self.delimiter.clone()]
}

fn name(&self) -> &str {
&self.name
}
}

impl PartialEq<dyn Any> for StringAgg {
fn eq(&self, other: &dyn Any) -> bool {
down_cast_any_ref(other)
.downcast_ref::<Self>()
.map(|x| {
self.name == x.name
&& self.data_type == x.data_type
&& self.expr.eq(&x.expr)
&& self.delimiter.eq(&x.delimiter)
})
.unwrap_or(false)
}
}

#[derive(Debug)]
pub(crate) struct StringAggAccumulator {
values: Option<String>,
delimiter: String,
}

impl StringAggAccumulator {
pub fn new(delimiter: &str) -> Self {
Self {
values: None,
delimiter: delimiter.to_string(),
}
}
}

impl Accumulator for StringAggAccumulator {
fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> {
let string_array: Vec<_> = as_generic_string_array::<i64>(&values[0])?
.iter()
.filter_map(|v| v.as_ref().map(ToString::to_string))
.collect();
if !string_array.is_empty() {
let s = string_array.join(self.delimiter.as_str());
let v = self.values.get_or_insert("".to_string());
if !v.is_empty() {
v.push_str(self.delimiter.as_str());
}
v.push_str(s.as_str());
}
Ok(())
}

fn merge_batch(&mut self, values: &[ArrayRef]) -> Result<()> {
self.update_batch(values)?;
Ok(())
}

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

fn evaluate(&self) -> Result<ScalarValue> {
Ok(ScalarValue::LargeUtf8(self.values.clone()))
}

fn size(&self) -> usize {
std::mem::size_of_val(self)
+ self.values.as_ref().map(|v| v.capacity()).unwrap_or(0)
+ self.delimiter.capacity()
}
}

#[cfg(test)]
mod tests {
use super::*;
use crate::expressions::tests::aggregate;
use crate::expressions::{col, create_aggregate_expr, try_cast};
use arrow::array::ArrayRef;
use arrow::datatypes::*;
use arrow::record_batch::RecordBatch;
use arrow_array::LargeStringArray;
use arrow_array::StringArray;
use datafusion_expr::type_coercion::aggregates::coerce_types;
use datafusion_expr::AggregateFunction;

fn assert_string_aggregate(
array: ArrayRef,
function: AggregateFunction,
distinct: bool,
expected: ScalarValue,
delimiter: String,
) {
let data_type = array.data_type();
let sig = function.signature();
let coerced =
coerce_types(&function, &[data_type.clone(), DataType::Utf8], &sig).unwrap();

let input_schema = Schema::new(vec![Field::new("a", data_type.clone(), true)]);
let batch =
RecordBatch::try_new(Arc::new(input_schema.clone()), vec![array]).unwrap();

let input = try_cast(
col("a", &input_schema).unwrap(),
&input_schema,
coerced[0].clone(),
)
.unwrap();

let delimiter = Arc::new(Literal::new(ScalarValue::Utf8(Some(delimiter))));
let schema = Schema::new(vec![Field::new("a", coerced[0].clone(), true)]);
let agg = create_aggregate_expr(
&function,
distinct,
&[input, delimiter],
&[],
&schema,
"agg",
)
.unwrap();

let result = aggregate(&batch, agg).unwrap();
assert_eq!(expected, result);
}

#[test]
fn string_agg_utf8() {
let a: ArrayRef = Arc::new(StringArray::from(vec!["h", "e", "l", "l", "o"]));
assert_string_aggregate(
a,
AggregateFunction::StringAgg,
false,
ScalarValue::LargeUtf8(Some("h,e,l,l,o".to_owned())),
",".to_owned(),
);
}

#[test]
fn string_agg_largeutf8() {
let a: ArrayRef = Arc::new(LargeStringArray::from(vec!["h", "e", "l", "l", "o"]));
assert_string_aggregate(
a,
AggregateFunction::StringAgg,
false,
ScalarValue::LargeUtf8(Some("h|e|l|l|o".to_owned())),
"|".to_owned(),
);
}
}
1 change: 1 addition & 0 deletions datafusion/physical-expr/src/expressions/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@ pub use crate::aggregate::min_max::{MaxAccumulator, MinAccumulator};
pub use crate::aggregate::regr::{Regr, RegrType};
pub use crate::aggregate::stats::StatsType;
pub use crate::aggregate::stddev::{Stddev, StddevPop};
pub use crate::aggregate::string_agg::StringAgg;
pub use crate::aggregate::sum::Sum;
pub use crate::aggregate::sum_distinct::DistinctSum;
pub use crate::aggregate::variance::{Variance, VariancePop};
Expand Down
1 change: 1 addition & 0 deletions datafusion/proto/proto/datafusion.proto
Original file line number Diff line number Diff line change
Expand Up @@ -686,6 +686,7 @@ enum AggregateFunction {
REGR_SXX = 32;
REGR_SYY = 33;
REGR_SXY = 34;
STRING_AGG = 35;
}

message AggregateExprNode {
Expand Down
Loading

0 comments on commit 76ced31

Please sign in to comment.