datafusion_functions_aggregate/
median.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18use std::cmp::Ordering;
19use std::fmt::{Debug, Formatter};
20use std::mem::{size_of, size_of_val};
21use std::sync::Arc;
22
23use arrow::array::{
24    downcast_integer, ArrowNumericType, BooleanArray, ListArray, PrimitiveArray,
25    PrimitiveBuilder,
26};
27use arrow::buffer::{OffsetBuffer, ScalarBuffer};
28use arrow::{
29    array::{ArrayRef, AsArray},
30    datatypes::{
31        DataType, Decimal128Type, Decimal256Type, Field, Float16Type, Float32Type,
32        Float64Type,
33    },
34};
35
36use arrow::array::Array;
37use arrow::array::ArrowNativeTypeOp;
38use arrow::datatypes::{ArrowNativeType, ArrowPrimitiveType, FieldRef};
39
40use datafusion_common::{
41    internal_datafusion_err, internal_err, DataFusionError, HashSet, Result, ScalarValue,
42};
43use datafusion_expr::function::StateFieldsArgs;
44use datafusion_expr::{
45    function::AccumulatorArgs, utils::format_state_name, Accumulator, AggregateUDFImpl,
46    Documentation, Signature, Volatility,
47};
48use datafusion_expr::{EmitTo, GroupsAccumulator};
49use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::accumulate;
50use datafusion_functions_aggregate_common::aggregate::groups_accumulator::nulls::filtered_null_mask;
51use datafusion_functions_aggregate_common::utils::Hashable;
52use datafusion_macros::user_doc;
53
54make_udaf_expr_and_func!(
55    Median,
56    median,
57    expression,
58    "Computes the median of a set of numbers",
59    median_udaf
60);
61
62#[user_doc(
63    doc_section(label = "General Functions"),
64    description = "Returns the median value in the specified column.",
65    syntax_example = "median(expression)",
66    sql_example = r#"```sql
67> SELECT median(column_name) FROM table_name;
68+----------------------+
69| median(column_name)   |
70+----------------------+
71| 45.5                 |
72+----------------------+
73```"#,
74    standard_argument(name = "expression", prefix = "The")
75)]
76/// MEDIAN aggregate expression. If using the non-distinct variation, then this uses a
77/// lot of memory because all values need to be stored in memory before a result can be
78/// computed. If an approximation is sufficient then APPROX_MEDIAN provides a much more
79/// efficient solution.
80///
81/// If using the distinct variation, the memory usage will be similarly high if the
82/// cardinality is high as it stores all distinct values in memory before computing the
83/// result, but if cardinality is low then memory usage will also be lower.
84pub struct Median {
85    signature: Signature,
86}
87
88impl Debug for Median {
89    fn fmt(&self, f: &mut Formatter) -> std::fmt::Result {
90        f.debug_struct("Median")
91            .field("name", &self.name())
92            .field("signature", &self.signature)
93            .finish()
94    }
95}
96
97impl Default for Median {
98    fn default() -> Self {
99        Self::new()
100    }
101}
102
103impl Median {
104    pub fn new() -> Self {
105        Self {
106            signature: Signature::numeric(1, Volatility::Immutable),
107        }
108    }
109}
110
111impl AggregateUDFImpl for Median {
112    fn as_any(&self) -> &dyn std::any::Any {
113        self
114    }
115
116    fn name(&self) -> &str {
117        "median"
118    }
119
120    fn signature(&self) -> &Signature {
121        &self.signature
122    }
123
124    fn return_type(&self, arg_types: &[DataType]) -> Result<DataType> {
125        Ok(arg_types[0].clone())
126    }
127
128    fn state_fields(&self, args: StateFieldsArgs) -> Result<Vec<FieldRef>> {
129        //Intermediate state is a list of the elements we have collected so far
130        let field = Field::new_list_field(args.input_fields[0].data_type().clone(), true);
131        let state_name = if args.is_distinct {
132            "distinct_median"
133        } else {
134            "median"
135        };
136
137        Ok(vec![Field::new(
138            format_state_name(args.name, state_name),
139            DataType::List(Arc::new(field)),
140            true,
141        )
142        .into()])
143    }
144
145    fn accumulator(&self, acc_args: AccumulatorArgs) -> Result<Box<dyn Accumulator>> {
146        macro_rules! helper {
147            ($t:ty, $dt:expr) => {
148                if acc_args.is_distinct {
149                    Ok(Box::new(DistinctMedianAccumulator::<$t> {
150                        data_type: $dt.clone(),
151                        distinct_values: HashSet::new(),
152                    }))
153                } else {
154                    Ok(Box::new(MedianAccumulator::<$t> {
155                        data_type: $dt.clone(),
156                        all_values: vec![],
157                    }))
158                }
159            };
160        }
161
162        let dt = acc_args.exprs[0].data_type(acc_args.schema)?;
163        downcast_integer! {
164            dt => (helper, dt),
165            DataType::Float16 => helper!(Float16Type, dt),
166            DataType::Float32 => helper!(Float32Type, dt),
167            DataType::Float64 => helper!(Float64Type, dt),
168            DataType::Decimal128(_, _) => helper!(Decimal128Type, dt),
169            DataType::Decimal256(_, _) => helper!(Decimal256Type, dt),
170            _ => Err(DataFusionError::NotImplemented(format!(
171                "MedianAccumulator not supported for {} with {}",
172                acc_args.name,
173                dt,
174            ))),
175        }
176    }
177
178    fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool {
179        !args.is_distinct
180    }
181
182    fn create_groups_accumulator(
183        &self,
184        args: AccumulatorArgs,
185    ) -> Result<Box<dyn GroupsAccumulator>> {
186        let num_args = args.exprs.len();
187        if num_args != 1 {
188            return internal_err!(
189                "median should only have 1 arg, but found num args:{}",
190                args.exprs.len()
191            );
192        }
193
194        let dt = args.exprs[0].data_type(args.schema)?;
195
196        macro_rules! helper {
197            ($t:ty, $dt:expr) => {
198                Ok(Box::new(MedianGroupsAccumulator::<$t>::new($dt)))
199            };
200        }
201
202        downcast_integer! {
203            dt => (helper, dt),
204            DataType::Float16 => helper!(Float16Type, dt),
205            DataType::Float32 => helper!(Float32Type, dt),
206            DataType::Float64 => helper!(Float64Type, dt),
207            DataType::Decimal128(_, _) => helper!(Decimal128Type, dt),
208            DataType::Decimal256(_, _) => helper!(Decimal256Type, dt),
209            _ => Err(DataFusionError::NotImplemented(format!(
210                "MedianGroupsAccumulator not supported for {} with {}",
211                args.name,
212                dt,
213            ))),
214        }
215    }
216
217    fn documentation(&self) -> Option<&Documentation> {
218        self.doc()
219    }
220}
221
222/// The median accumulator accumulates the raw input values
223/// as `ScalarValue`s
224///
225/// The intermediate state is represented as a List of scalar values updated by
226/// `merge_batch` and a `Vec` of `ArrayRef` that are converted to scalar values
227/// in the final evaluation step so that we avoid expensive conversions and
228/// allocations during `update_batch`.
229struct MedianAccumulator<T: ArrowNumericType> {
230    data_type: DataType,
231    all_values: Vec<T::Native>,
232}
233
234impl<T: ArrowNumericType> Debug for MedianAccumulator<T> {
235    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
236        write!(f, "MedianAccumulator({})", self.data_type)
237    }
238}
239
240impl<T: ArrowNumericType> Accumulator for MedianAccumulator<T> {
241    fn state(&mut self) -> Result<Vec<ScalarValue>> {
242        // Convert `all_values` to `ListArray` and return a single List ScalarValue
243
244        // Build offsets
245        let offsets =
246            OffsetBuffer::new(ScalarBuffer::from(vec![0, self.all_values.len() as i32]));
247
248        // Build inner array
249        let values_array = PrimitiveArray::<T>::new(
250            ScalarBuffer::from(std::mem::take(&mut self.all_values)),
251            None,
252        )
253        .with_data_type(self.data_type.clone());
254
255        // Build the result list array
256        let list_array = ListArray::new(
257            Arc::new(Field::new_list_field(self.data_type.clone(), true)),
258            offsets,
259            Arc::new(values_array),
260            None,
261        );
262
263        Ok(vec![ScalarValue::List(Arc::new(list_array))])
264    }
265
266    fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> {
267        let values = values[0].as_primitive::<T>();
268        self.all_values.reserve(values.len() - values.null_count());
269        self.all_values.extend(values.iter().flatten());
270        Ok(())
271    }
272
273    fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> {
274        let array = states[0].as_list::<i32>();
275        for v in array.iter().flatten() {
276            self.update_batch(&[v])?
277        }
278        Ok(())
279    }
280
281    fn evaluate(&mut self) -> Result<ScalarValue> {
282        let d = std::mem::take(&mut self.all_values);
283        let median = calculate_median::<T>(d);
284        ScalarValue::new_primitive::<T>(median, &self.data_type)
285    }
286
287    fn size(&self) -> usize {
288        size_of_val(self) + self.all_values.capacity() * size_of::<T::Native>()
289    }
290}
291
292/// The median groups accumulator accumulates the raw input values
293///
294/// For calculating the accurate medians of groups, we need to store all values
295/// of groups before final evaluation.
296/// So values in each group will be stored in a `Vec<T>`, and the total group values
297/// will be actually organized as a `Vec<Vec<T>>`.
298///
299#[derive(Debug)]
300struct MedianGroupsAccumulator<T: ArrowNumericType + Send> {
301    data_type: DataType,
302    group_values: Vec<Vec<T::Native>>,
303}
304
305impl<T: ArrowNumericType + Send> MedianGroupsAccumulator<T> {
306    pub fn new(data_type: DataType) -> Self {
307        Self {
308            data_type,
309            group_values: Vec::new(),
310        }
311    }
312}
313
314impl<T: ArrowNumericType + Send> GroupsAccumulator for MedianGroupsAccumulator<T> {
315    fn update_batch(
316        &mut self,
317        values: &[ArrayRef],
318        group_indices: &[usize],
319        opt_filter: Option<&BooleanArray>,
320        total_num_groups: usize,
321    ) -> Result<()> {
322        assert_eq!(values.len(), 1, "single argument to update_batch");
323        let values = values[0].as_primitive::<T>();
324
325        // Push the `not nulls + not filtered` row into its group
326        self.group_values.resize(total_num_groups, Vec::new());
327        accumulate(
328            group_indices,
329            values,
330            opt_filter,
331            |group_index, new_value| {
332                self.group_values[group_index].push(new_value);
333            },
334        );
335
336        Ok(())
337    }
338
339    fn merge_batch(
340        &mut self,
341        values: &[ArrayRef],
342        group_indices: &[usize],
343        // Since aggregate filter should be applied in partial stage, in final stage there should be no filter
344        _opt_filter: Option<&BooleanArray>,
345        total_num_groups: usize,
346    ) -> Result<()> {
347        assert_eq!(values.len(), 1, "one argument to merge_batch");
348
349        // The merged values should be organized like as a `ListArray` which is nullable
350        // (input with nulls usually generated from `convert_to_state`), but `inner array` of
351        // `ListArray`  is `non-nullable`.
352        //
353        // Following is the possible and impossible input `values`:
354        //
355        // # Possible values
356        // ```text
357        //   group 0: [1, 2, 3]
358        //   group 1: null (list array is nullable)
359        //   group 2: [6, 7, 8]
360        //   ...
361        //   group n: [...]
362        // ```
363        //
364        // # Impossible values
365        // ```text
366        //   group x: [1, 2, null] (values in list array is non-nullable)
367        // ```
368        //
369        let input_group_values = values[0].as_list::<i32>();
370
371        // Ensure group values big enough
372        self.group_values.resize(total_num_groups, Vec::new());
373
374        // Extend values to related groups
375        // TODO: avoid using iterator of the `ListArray`, this will lead to
376        // many calls of `slice` of its ``inner array`, and `slice` is not
377        // so efficient(due to the calculation of `null_count` for each `slice`).
378        group_indices
379            .iter()
380            .zip(input_group_values.iter())
381            .for_each(|(&group_index, values_opt)| {
382                if let Some(values) = values_opt {
383                    let values = values.as_primitive::<T>();
384                    self.group_values[group_index].extend(values.values().iter());
385                }
386            });
387
388        Ok(())
389    }
390
391    fn state(&mut self, emit_to: EmitTo) -> Result<Vec<ArrayRef>> {
392        // Emit values
393        let emit_group_values = emit_to.take_needed(&mut self.group_values);
394
395        // Build offsets
396        let mut offsets = Vec::with_capacity(self.group_values.len() + 1);
397        offsets.push(0);
398        let mut cur_len = 0_i32;
399        for group_value in &emit_group_values {
400            cur_len += group_value.len() as i32;
401            offsets.push(cur_len);
402        }
403        // TODO: maybe we can use `OffsetBuffer::new_unchecked` like what in `convert_to_state`,
404        // but safety should be considered more carefully here(and I am not sure if it can get
405        // performance improvement when we introduce checks to keep the safety...).
406        //
407        // Can see more details in:
408        // https://github.com/apache/datafusion/pull/13681#discussion_r1931209791
409        //
410        let offsets = OffsetBuffer::new(ScalarBuffer::from(offsets));
411
412        // Build inner array
413        let flatten_group_values =
414            emit_group_values.into_iter().flatten().collect::<Vec<_>>();
415        let group_values_array =
416            PrimitiveArray::<T>::new(ScalarBuffer::from(flatten_group_values), None)
417                .with_data_type(self.data_type.clone());
418
419        // Build the result list array
420        let result_list_array = ListArray::new(
421            Arc::new(Field::new_list_field(self.data_type.clone(), true)),
422            offsets,
423            Arc::new(group_values_array),
424            None,
425        );
426
427        Ok(vec![Arc::new(result_list_array)])
428    }
429
430    fn evaluate(&mut self, emit_to: EmitTo) -> Result<ArrayRef> {
431        // Emit values
432        let emit_group_values = emit_to.take_needed(&mut self.group_values);
433
434        // Calculate median for each group
435        let mut evaluate_result_builder =
436            PrimitiveBuilder::<T>::new().with_data_type(self.data_type.clone());
437        for values in emit_group_values {
438            let median = calculate_median::<T>(values);
439            evaluate_result_builder.append_option(median);
440        }
441
442        Ok(Arc::new(evaluate_result_builder.finish()))
443    }
444
445    fn convert_to_state(
446        &self,
447        values: &[ArrayRef],
448        opt_filter: Option<&BooleanArray>,
449    ) -> Result<Vec<ArrayRef>> {
450        assert_eq!(values.len(), 1, "one argument to merge_batch");
451
452        let input_array = values[0].as_primitive::<T>();
453
454        // Directly convert the input array to states, each row will be
455        // seen as a respective group.
456        // For detail, the `input_array` will be converted to a `ListArray`.
457        // And if row is `not null + not filtered`, it will be converted to a list
458        // with only one element; otherwise, this row in `ListArray` will be set
459        // to null.
460
461        // Reuse values buffer in `input_array` to build `values` in `ListArray`
462        let values = PrimitiveArray::<T>::new(input_array.values().clone(), None)
463            .with_data_type(self.data_type.clone());
464
465        // `offsets` in `ListArray`, each row as a list element
466        let offset_end = i32::try_from(input_array.len()).map_err(|e| {
467            internal_datafusion_err!(
468                "cast array_len to i32 failed in convert_to_state of group median, err:{e:?}"
469            )
470        })?;
471        let offsets = (0..=offset_end).collect::<Vec<_>>();
472        // Safety: all checks in `OffsetBuffer::new` are ensured to pass
473        let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(offsets)) };
474
475        // `nulls` for converted `ListArray`
476        let nulls = filtered_null_mask(opt_filter, input_array);
477
478        let converted_list_array = ListArray::new(
479            Arc::new(Field::new_list_field(self.data_type.clone(), true)),
480            offsets,
481            Arc::new(values),
482            nulls,
483        );
484
485        Ok(vec![Arc::new(converted_list_array)])
486    }
487
488    fn supports_convert_to_state(&self) -> bool {
489        true
490    }
491
492    fn size(&self) -> usize {
493        self.group_values
494            .iter()
495            .map(|values| values.capacity() * size_of::<T>())
496            .sum::<usize>()
497            // account for size of self.grou_values too
498            + self.group_values.capacity() * size_of::<Vec<T>>()
499    }
500}
501
502/// The distinct median accumulator accumulates the raw input values
503/// as `ScalarValue`s
504///
505/// The intermediate state is represented as a List of scalar values updated by
506/// `merge_batch` and a `Vec` of `ArrayRef` that are converted to scalar values
507/// in the final evaluation step so that we avoid expensive conversions and
508/// allocations during `update_batch`.
509struct DistinctMedianAccumulator<T: ArrowNumericType> {
510    data_type: DataType,
511    distinct_values: HashSet<Hashable<T::Native>>,
512}
513
514impl<T: ArrowNumericType> Debug for DistinctMedianAccumulator<T> {
515    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
516        write!(f, "DistinctMedianAccumulator({})", self.data_type)
517    }
518}
519
520impl<T: ArrowNumericType> Accumulator for DistinctMedianAccumulator<T> {
521    fn state(&mut self) -> Result<Vec<ScalarValue>> {
522        let all_values = self
523            .distinct_values
524            .iter()
525            .map(|x| ScalarValue::new_primitive::<T>(Some(x.0), &self.data_type))
526            .collect::<Result<Vec<_>>>()?;
527
528        let arr = ScalarValue::new_list_nullable(&all_values, &self.data_type);
529        Ok(vec![ScalarValue::List(arr)])
530    }
531
532    fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> {
533        if values.is_empty() {
534            return Ok(());
535        }
536
537        let array = values[0].as_primitive::<T>();
538        match array.nulls().filter(|x| x.null_count() > 0) {
539            Some(n) => {
540                for idx in n.valid_indices() {
541                    self.distinct_values.insert(Hashable(array.value(idx)));
542                }
543            }
544            None => array.values().iter().for_each(|x| {
545                self.distinct_values.insert(Hashable(*x));
546            }),
547        }
548        Ok(())
549    }
550
551    fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> {
552        let array = states[0].as_list::<i32>();
553        for v in array.iter().flatten() {
554            self.update_batch(&[v])?
555        }
556        Ok(())
557    }
558
559    fn evaluate(&mut self) -> Result<ScalarValue> {
560        let d = std::mem::take(&mut self.distinct_values)
561            .into_iter()
562            .map(|v| v.0)
563            .collect::<Vec<_>>();
564        let median = calculate_median::<T>(d);
565        ScalarValue::new_primitive::<T>(median, &self.data_type)
566    }
567
568    fn size(&self) -> usize {
569        size_of_val(self) + self.distinct_values.capacity() * size_of::<T::Native>()
570    }
571}
572
573/// Get maximum entry in the slice,
574fn slice_max<T>(array: &[T::Native]) -> T::Native
575where
576    T: ArrowPrimitiveType,
577    T::Native: PartialOrd, // Ensure the type supports PartialOrd for comparison
578{
579    // Make sure that, array is not empty.
580    debug_assert!(!array.is_empty());
581    // `.unwrap()` is safe here as the array is supposed to be non-empty
582    *array
583        .iter()
584        .max_by(|x, y| x.partial_cmp(y).unwrap_or(Ordering::Less))
585        .unwrap()
586}
587
588fn calculate_median<T: ArrowNumericType>(
589    mut values: Vec<T::Native>,
590) -> Option<T::Native> {
591    let cmp = |x: &T::Native, y: &T::Native| x.compare(*y);
592
593    let len = values.len();
594    if len == 0 {
595        None
596    } else if len % 2 == 0 {
597        let (low, high, _) = values.select_nth_unstable_by(len / 2, cmp);
598        // Get the maximum of the low (left side after bi-partitioning)
599        let left_max = slice_max::<T>(low);
600        let median = left_max
601            .add_wrapping(*high)
602            .div_wrapping(T::Native::usize_as(2));
603        Some(median)
604    } else {
605        let (_, median, _) = values.select_nth_unstable_by(len / 2, cmp);
606        Some(*median)
607    }
608}