avantgardnerio commented on code in PR #7192: URL: https://github.com/apache/arrow-datafusion/pull/7192#discussion_r1298415543
########## datafusion/core/src/physical_plan/aggregates/priority_map.rs: ########## @@ -0,0 +1,820 @@ +// 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. + +//! A memory-conscious aggregation implementation that limits group buckets to a fixed number + +use crate::physical_plan::aggregates::{ + aggregate_expressions, evaluate_group_by, evaluate_many, AggregateExec, + PhysicalGroupBy, +}; +use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; +use ahash::RandomState; +use arrow::util::pretty::print_batches; +use arrow_array::cast::AsArray; +use arrow_array::downcast_primitive; +use arrow_array::{ + Array, ArrayRef, ArrowNativeTypeOp, ArrowPrimitiveType, PrimitiveArray, RecordBatch, + StringArray, +}; +use arrow_schema::{DataType, SchemaRef, SortOptions}; +use datafusion_common::DataFusionError; +use datafusion_common::Result; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::PhysicalExpr; +use futures::stream::{Stream, StreamExt}; +use hashbrown::raw::RawTable; +use log::{trace, Level}; +use std::cmp::Ordering; +use std::collections::BTreeSet; +use std::hash::{BuildHasher, Hash, Hasher}; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + +pub struct GroupedTopKAggregateStream { + partition: usize, + row_count: usize, + started: bool, + schema: SchemaRef, + input: SendableRecordBatchStream, + aggregate_arguments: Vec<Vec<Arc<dyn PhysicalExpr>>>, + group_by: PhysicalGroupBy, + aggregator: Box<dyn LimitedAggregator>, +} + +impl GroupedTopKAggregateStream { + pub fn new( + agg: &AggregateExec, + context: Arc<TaskContext>, + partition: usize, + limit: usize, + ) -> Result<Self> { + let agg_schema = Arc::clone(&agg.schema); + let group_by = agg.group_by.clone(); + + let input = agg.input.execute(partition, Arc::clone(&context))?; + + let aggregate_arguments = + aggregate_expressions(&agg.aggr_expr, &agg.mode, group_by.expr.len())?; + + let (val_field, _) = agg + .get_minmax_desc() + .ok_or_else(|| DataFusionError::Execution("Min/max required".to_string()))?; + + let vt = val_field.data_type().clone(); + let ag = new_group_values(limit, agg.sort, vt)?; + + Ok(GroupedTopKAggregateStream { + partition, + started: false, + row_count: 0, + schema: agg_schema, + input, + aggregate_arguments, + group_by, + aggregator: ag, + }) + } +} + +pub fn new_group_values( + limit: usize, + sort: SortOptions, + vt: DataType, +) -> Result<Box<dyn LimitedAggregator>> { + macro_rules! downcast_helper { + ($vt:ty, $d:ident) => { + return Ok(Box::new(PrimitiveAggregator::<$vt>::new( + limit, + limit * 10, + sort, + ))) + }; + } + + downcast_primitive! { + vt => (downcast_helper, vt), + _ => {} + } + + Err(DataFusionError::Execution(format!( + "Can't group type: {vt:?}" + ))) +} + +impl RecordBatchStream for GroupedTopKAggregateStream { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } +} + +pub trait LimitedAggregator: Send { + fn intern(&mut self, ids: ArrayRef, vals: ArrayRef) -> Result<()>; + fn emit(&mut self) -> Result<Vec<ArrayRef>>; + fn is_empty(&self) -> bool; +} + +pub trait ValueType: ArrowNativeTypeOp + Clone {} + +impl<T> ValueType for T where T: ArrowNativeTypeOp + Clone {} + +pub trait KeyType: Clone + Eq + Hash {} + +impl<T> KeyType for T where T: Clone + Eq + Hash {} + +struct PrimitiveAggregator<VAL: ArrowPrimitiveType> +where + <VAL as ArrowPrimitiveType>::Native: Clone, +{ + priority_map: PriorityMap<Option<String>, VAL::Native>, +} + +impl<VAL: ArrowPrimitiveType> PrimitiveAggregator<VAL> +where + <VAL as ArrowPrimitiveType>::Native: Clone, +{ + pub fn new(limit: usize, capacity: usize, sort: SortOptions) -> Self { + Self { + priority_map: PriorityMap::new(limit, capacity, sort), + } + } +} + +unsafe impl<VAL: ArrowPrimitiveType> Send for PrimitiveAggregator<VAL> where + <VAL as ArrowPrimitiveType>::Native: Clone +{ +} + +impl<VAL: ArrowPrimitiveType> LimitedAggregator for PrimitiveAggregator<VAL> +where + <VAL as ArrowPrimitiveType>::Native: Clone, +{ + fn intern(&mut self, ids: ArrayRef, vals: ArrayRef) -> Result<()> { + let ids = ids.as_any().downcast_ref::<StringArray>().ok_or_else(|| { + DataFusionError::Execution("Expected StringArray".to_string()) + })?; + let sort = self.priority_map.sort; + let vals = vals.as_primitive::<VAL>(); + for row_idx in 0..ids.len() { + let val = if vals.is_null(row_idx) { + None + } else { + Some(vals.value(row_idx)) + }; + let id = if ids.is_null(row_idx) { + None + } else { + // Check goes here, because it is generalizable between str/String and Row/OwnedRow + let id = ids.value(row_idx); + if let Some(worst) = self.priority_map.worst_val() { + if cmp(&sort, &val, worst) == Ordering::Greater { + continue; + } + } + Some(id.to_string()) + }; + + self.priority_map.insert(id, val)?; + } + Ok(()) + } + + fn emit(&mut self) -> Result<Vec<ArrayRef>> { + let (keys, vals): (Vec<_>, Vec<_>) = + self.priority_map.drain().into_iter().unzip(); + let keys = Arc::new(StringArray::from(keys)); + let vals = Arc::new(PrimitiveArray::<VAL>::from_iter(vals)); + Ok(vec![keys, vals]) + } + + fn is_empty(&self) -> bool { + self.priority_map.is_empty() + } +} + +impl Stream for GroupedTopKAggregateStream { + type Item = Result<RecordBatch>; + + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll<Option<Self::Item>> { + while let Poll::Ready(res) = self.input.poll_next_unpin(cx) { + match res { + // got a batch, convert to rows and append to our TreeMap + Some(Ok(batch)) => { + self.started = true; + trace!( + "partition {} has {} rows and got batch with {} rows", + self.partition, + self.row_count, + batch.num_rows() + ); + if log::log_enabled!(Level::Trace) && batch.num_rows() < 20 { + print_batches(&[batch.clone()])?; + } + self.row_count += batch.num_rows(); + let batches = &[batch]; + let group_by_values = + evaluate_group_by(&self.group_by, batches.first().unwrap())?; + let group_by_values = + group_by_values.into_iter().last().expect("values"); + let group_by_values = + group_by_values.into_iter().last().expect("values"); + let input_values = evaluate_many( + &self.aggregate_arguments, + batches.first().unwrap(), + )?; + let input_values = match input_values.as_slice() { + [] => { + Err(DataFusionError::Execution("vals required".to_string()))? + } + [vals] => vals, + _ => { + Err(DataFusionError::Execution("1 val required".to_string()))? + } + }; + let input_values = match input_values.as_slice() { + [] => { + Err(DataFusionError::Execution("vals required".to_string()))? + } + [vals] => vals, + _ => { + Err(DataFusionError::Execution("1 val required".to_string()))? + } + } + .clone(); + + // iterate over each column of group_by values + (*self.aggregator).intern(group_by_values, input_values)?; + } + // inner is done, emit all rows and switch to producing output + None => { + if self.aggregator.is_empty() { + trace!("partition {} emit None", self.partition); + return Poll::Ready(None); + } + let cols = self.aggregator.emit()?; + let batch = RecordBatch::try_new(self.schema.clone(), cols)?; + trace!( + "partition {} emit batch with {} rows", + self.partition, + batch.num_rows() + ); + if log::log_enabled!(Level::Trace) { + print_batches(&[batch.clone()])?; + } + return Poll::Ready(Some(Ok(batch))); + } + // inner had error, return to caller + Some(Err(e)) => { + return Poll::Ready(Some(Err(e))); + } + } + } + Poll::Pending + } +} + +pub struct PriorityMap<ID: KeyType, VAL: ValueType> { + limit: usize, + sort: SortOptions, + rnd: RandomState, + id_to_val: RawTable<MapItem<ID, VAL>>, + val_to_idx: BTreeSet<HeapItem<VAL>>, +} + +pub struct MapItem<ID: KeyType, VAL: ValueType> { + hash: u64, + pub id: ID, + pub val: Option<VAL>, +} + +impl<ID: KeyType, VAL: ValueType> MapItem<ID, VAL> { + pub fn new(hash: u64, id: ID, val: Option<VAL>) -> Self { + Self { hash, id, val } + } +} + +struct HeapItem<VAL: ValueType> { + val: Option<VAL>, + buk_idx: usize, + sort: SortOptions, +} + +impl<VAL: ValueType> HeapItem<VAL> { + pub fn new(val: Option<VAL>, buk_idx: usize, sort: SortOptions) -> Self { + Self { val, buk_idx, sort } + } +} + +impl<VAL: ValueType> Eq for HeapItem<VAL> {} + +impl<VAL: ValueType> PartialEq<Self> for HeapItem<VAL> { + fn eq(&self, other: &Self) -> bool { + self.cmp(other) == Ordering::Equal + } +} + +impl<VAL: ValueType> PartialOrd<Self> for HeapItem<VAL> { + fn partial_cmp(&self, other: &Self) -> Option<Ordering> { + Some(self.cmp(other)) + } +} + +impl<VAL: ValueType> Ord for HeapItem<VAL> { + fn cmp(&self, other: &Self) -> Ordering { + let res = cmp(&self.sort, &self.val, &other.val); + if res != Ordering::Equal { + return res; + } + self.buk_idx.cmp(&other.buk_idx) + } +} + +impl<ID: KeyType, VAL: ValueType> PriorityMap<ID, VAL> +where + VAL: PartialEq<VAL>, +{ + pub fn new(limit: usize, capacity: usize, sort: SortOptions) -> Self { + Self { + limit, + sort, + rnd: Default::default(), + id_to_val: RawTable::with_capacity(capacity), + val_to_idx: Default::default(), + } + } + + pub fn insert(&mut self, new_id: ID, new_val: Option<VAL>) -> Result<()> { + let is_full = self.is_full(); + assert!(self.len() <= self.limit, "Overflow"); + let (id_to_val, val_to_idx) = (&mut self.id_to_val, &mut self.val_to_idx); + + // if we're full, and the new val is worse than all our values, just bail + if is_full { + let worst_entry = val_to_idx.last().expect("Missing value!"); + if cmp(&self.sort, &new_val, &worst_entry.val) == Ordering::Greater { + return Ok(()); + } + } + + // handle new groups we haven't seen yet + let mut hasher = self.rnd.build_hasher(); + new_id.hash(&mut hasher); + let new_hash = hasher.finish(); + let old_bucket = match id_to_val.find(new_hash, |mi| new_id == mi.id) { + None => { + // we're full and this is a better value, so remove the worst + if is_full { + let worst_hi = val_to_idx.pop_last().expect("Missing value!"); + unsafe { id_to_val.erase(id_to_val.bucket(worst_hi.buk_idx)) }; Review Comment: Ah, I think [this documentation](https://github.com/apache/arrow-rs/blob/master/arrow/CONTRIBUTING.md#considerations-when-introducing-unsafe) is what I was looking for. I'll try to follow that template. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
