avantgardnerio commented on code in PR #7192:
URL: https://github.com/apache/arrow-datafusion/pull/7192#discussion_r1298348921


##########
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())

Review Comment:
   > Also if ID is not a string
   
   For this PR, it is specialized for only Strings. I would like to do a follow 
on PR that handles all combination of keys/vals.
   
   > moving it to insert
   
   The idea was that in future PRs, by keeping it here vs in `PriorityMap`, the 
code can be specialized. i.e. when we add `Row`/`OwnedRow` support, we can do 
the test here:
   
   1. we get `self.priority_map.worst_val()` which will be `OwnedRow`
   2. we call `worst.row()` for a cheap conversion to `Row`
   3. we compare `new_val: Row` with `worst: Row` and fail fast (the most 
important case)
   4. only then do we convert `new_val` to OwnedRow, and we do it here because 
each `impl` of `LimitedAggregator` will be able to handle its own type. The 
`Row` one will call `.to_owned()` the String one will call `.to_string()`, etc.
   
   I really liked your `Into<>` idea, but I'm not sure how we could apply it to 
each type?



-- 
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]

Reply via email to