use std::any::Any;
use std::fmt::Debug;
use std::ops::BitAnd;
use std::sync::Arc;
use crate::aggregate::utils::down_cast_any_ref;
use crate::{AggregateExpr, GroupsAccumulator, PhysicalExpr};
use arrow::array::{Array, Int64Array};
use arrow::compute;
use arrow::datatypes::DataType;
use arrow::{array::ArrayRef, datatypes::Field};
use arrow_array::cast::AsArray;
use arrow_array::types::Int64Type;
use arrow_array::PrimitiveArray;
use arrow_buffer::BooleanBuffer;
use datafusion_common::{downcast_value, ScalarValue};
use datafusion_common::{DataFusionError, Result};
use datafusion_expr::Accumulator;
use crate::expressions::format_state_name;
use super::groups_accumulator::accumulate::accumulate_indices;
use super::groups_accumulator::EmitTo;
#[derive(Debug, Clone)]
pub struct Count {
name: String,
data_type: DataType,
nullable: bool,
exprs: Vec<Arc<dyn PhysicalExpr>>,
}
impl Count {
pub fn new(
expr: Arc<dyn PhysicalExpr>,
name: impl Into<String>,
data_type: DataType,
) -> Self {
Self {
name: name.into(),
exprs: vec![expr],
data_type,
nullable: true,
}
}
pub fn new_with_multiple_exprs(
exprs: Vec<Arc<dyn PhysicalExpr>>,
name: impl Into<String>,
data_type: DataType,
) -> Self {
Self {
name: name.into(),
exprs,
data_type,
nullable: true,
}
}
}
#[derive(Debug)]
struct CountGroupsAccumulator {
counts: Vec<i64>,
}
impl CountGroupsAccumulator {
pub fn new() -> Self {
Self { counts: vec![] }
}
}
impl GroupsAccumulator for CountGroupsAccumulator {
fn update_batch(
&mut self,
values: &[ArrayRef],
group_indices: &[usize],
opt_filter: Option<&arrow_array::BooleanArray>,
total_num_groups: usize,
) -> Result<()> {
assert_eq!(values.len(), 1, "single argument to update_batch");
let values = &values[0];
self.counts.resize(total_num_groups, 0);
accumulate_indices(
group_indices,
values.nulls(), opt_filter,
|group_index| {
self.counts[group_index] += 1;
},
);
Ok(())
}
fn merge_batch(
&mut self,
values: &[ArrayRef],
group_indices: &[usize],
opt_filter: Option<&arrow_array::BooleanArray>,
total_num_groups: usize,
) -> Result<()> {
assert_eq!(values.len(), 1, "one argument to merge_batch");
let partial_counts = values[0].as_primitive::<Int64Type>();
assert_eq!(partial_counts.null_count(), 0);
let partial_counts = partial_counts.values();
self.counts.resize(total_num_groups, 0);
match opt_filter {
Some(filter) => filter
.iter()
.zip(group_indices.iter())
.zip(partial_counts.iter())
.for_each(|((filter_value, &group_index), partial_count)| {
if let Some(true) = filter_value {
self.counts[group_index] += partial_count;
}
}),
None => group_indices.iter().zip(partial_counts.iter()).for_each(
|(&group_index, partial_count)| {
self.counts[group_index] += partial_count;
},
),
}
Ok(())
}
fn evaluate(&mut self, emit_to: EmitTo) -> Result<ArrayRef> {
let counts = emit_to.take_needed(&mut self.counts);
let nulls = None;
let array = PrimitiveArray::<Int64Type>::new(counts.into(), nulls);
Ok(Arc::new(array))
}
fn state(&mut self, emit_to: EmitTo) -> Result<Vec<ArrayRef>> {
let counts = emit_to.take_needed(&mut self.counts);
let counts: PrimitiveArray<Int64Type> = Int64Array::from(counts); Ok(vec![Arc::new(counts) as ArrayRef])
}
fn size(&self) -> usize {
self.counts.capacity() * std::mem::size_of::<usize>()
}
}
fn null_count_for_multiple_cols(values: &[ArrayRef]) -> usize {
if values.len() > 1 {
let result_bool_buf: Option<BooleanBuffer> = values
.iter()
.map(|a| a.nulls())
.fold(None, |acc, b| match (acc, b) {
(Some(acc), Some(b)) => Some(acc.bitand(b.inner())),
(Some(acc), None) => Some(acc),
(None, Some(b)) => Some(b.inner().clone()),
_ => None,
});
result_bool_buf.map_or(0, |b| values[0].len() - b.count_set_bits())
} else {
values[0].null_count()
}
}
impl AggregateExpr for Count {
fn as_any(&self) -> &dyn Any {
self
}
fn field(&self) -> Result<Field> {
Ok(Field::new(&self.name, DataType::Int64, self.nullable))
}
fn state_fields(&self) -> Result<Vec<Field>> {
Ok(vec![Field::new(
format_state_name(&self.name, "count"),
DataType::Int64,
true,
)])
}
fn expressions(&self) -> Vec<Arc<dyn PhysicalExpr>> {
self.exprs.clone()
}
fn create_accumulator(&self) -> Result<Box<dyn Accumulator>> {
Ok(Box::new(CountAccumulator::new()))
}
fn name(&self) -> &str {
&self.name
}
fn groups_accumulator_supported(&self) -> bool {
self.exprs.len() == 1
}
fn reverse_expr(&self) -> Option<Arc<dyn AggregateExpr>> {
Some(Arc::new(self.clone()))
}
fn create_sliding_accumulator(&self) -> Result<Box<dyn Accumulator>> {
Ok(Box::new(CountAccumulator::new()))
}
fn create_groups_accumulator(&self) -> Result<Box<dyn GroupsAccumulator>> {
Ok(Box::new(CountGroupsAccumulator::new()))
}
}
impl PartialEq<dyn Any> for Count {
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.nullable == x.nullable
&& self.exprs.len() == x.exprs.len()
&& self
.exprs
.iter()
.zip(x.exprs.iter())
.all(|(expr1, expr2)| expr1.eq(expr2))
})
.unwrap_or(false)
}
}
#[derive(Debug)]
struct CountAccumulator {
count: i64,
}
impl CountAccumulator {
pub fn new() -> Self {
Self { count: 0 }
}
}
impl Accumulator for CountAccumulator {
fn state(&self) -> Result<Vec<ScalarValue>> {
Ok(vec![ScalarValue::Int64(Some(self.count))])
}
fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> {
let array = &values[0];
self.count += (array.len() - null_count_for_multiple_cols(values)) as i64;
Ok(())
}
fn retract_batch(&mut self, values: &[ArrayRef]) -> Result<()> {
let array = &values[0];
self.count -= (array.len() - null_count_for_multiple_cols(values)) as i64;
Ok(())
}
fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> {
let counts = downcast_value!(states[0], Int64Array);
let delta = &compute::sum(counts);
if let Some(d) = delta {
self.count += *d;
}
Ok(())
}
fn evaluate(&self) -> Result<ScalarValue> {
Ok(ScalarValue::Int64(Some(self.count)))
}
fn supports_retract_batch(&self) -> bool {
true
}
fn size(&self) -> usize {
std::mem::size_of_val(self)
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::expressions::tests::aggregate;
use crate::expressions::{col, lit};
use crate::generic_test_op;
use arrow::record_batch::RecordBatch;
use arrow::{array::*, datatypes::*};
use datafusion_common::Result;
#[test]
fn count_elements() -> Result<()> {
let a: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5]));
generic_test_op!(a, DataType::Int32, Count, ScalarValue::from(5i64))
}
#[test]
fn count_with_nulls() -> Result<()> {
let a: ArrayRef = Arc::new(Int32Array::from(vec![
Some(1),
Some(2),
None,
None,
Some(3),
None,
]));
generic_test_op!(a, DataType::Int32, Count, ScalarValue::from(3i64))
}
#[test]
fn count_all_nulls() -> Result<()> {
let a: ArrayRef = Arc::new(BooleanArray::from(vec![
None, None, None, None, None, None, None, None,
]));
generic_test_op!(a, DataType::Boolean, Count, ScalarValue::from(0i64))
}
#[test]
fn count_empty() -> Result<()> {
let a: Vec<bool> = vec![];
let a: ArrayRef = Arc::new(BooleanArray::from(a));
generic_test_op!(a, DataType::Boolean, Count, ScalarValue::from(0i64))
}
#[test]
fn count_utf8() -> Result<()> {
let a: ArrayRef =
Arc::new(StringArray::from(vec!["a", "bb", "ccc", "dddd", "ad"]));
generic_test_op!(a, DataType::Utf8, Count, ScalarValue::from(5i64))
}
#[test]
fn count_large_utf8() -> Result<()> {
let a: ArrayRef =
Arc::new(LargeStringArray::from(vec!["a", "bb", "ccc", "dddd", "ad"]));
generic_test_op!(a, DataType::LargeUtf8, Count, ScalarValue::from(5i64))
}
#[test]
fn count_multi_cols() -> Result<()> {
let a: ArrayRef = Arc::new(Int32Array::from(vec![
Some(1),
Some(2),
None,
None,
Some(3),
None,
]));
let b: ArrayRef = Arc::new(Int32Array::from(vec![
Some(1),
None,
Some(2),
None,
Some(3),
Some(4),
]));
let schema = Schema::new(vec![
Field::new("a", DataType::Int32, true),
Field::new("b", DataType::Int32, true),
]);
let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![a, b])?;
let agg = Arc::new(Count::new_with_multiple_exprs(
vec![col("a", &schema)?, col("b", &schema)?],
"bla".to_string(),
DataType::Int64,
));
let actual = aggregate(&batch, agg)?;
let expected = ScalarValue::from(2i64);
assert_eq!(expected, actual);
Ok(())
}
#[test]
fn count_eq() -> Result<()> {
let count = Count::new(lit(1i8), "COUNT(1)".to_string(), DataType::Int64);
let arc_count: Arc<dyn AggregateExpr> = Arc::new(Count::new(
lit(1i8),
"COUNT(1)".to_string(),
DataType::Int64,
));
let box_count: Box<dyn AggregateExpr> = Box::new(Count::new(
lit(1i8),
"COUNT(1)".to_string(),
DataType::Int64,
));
let count2 = Count::new(lit(1i8), "COUNT(2)".to_string(), DataType::Int64);
assert!(arc_count.eq(&box_count));
assert!(box_count.eq(&arc_count));
assert!(arc_count.eq(&count));
assert!(count.eq(&box_count));
assert!(count.eq(&arc_count));
assert!(count2.ne(&arc_count));
Ok(())
}
}