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


##########
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>,

Review Comment:
   Correct, I think the options are:
   1. continue to use generics, create a macro that generates a concrete `impl` 
for all `26^2` primitive types
   2. switch to runtime polymorphism, create a `Wrapper` struct on the heap for 
each row, compare via `.as_any()`
   3. limit this PR to a fixed number of specialized cases, i.e. `<String, 
ArrowPrimitiveType>`
   4. break `Map<ID>` and `Heap<VAL>` into separate structs that communicate 
via passing messages so we are `26*2` concrete `impl`s, instead of `26^2`
   
   This PR takes approach 3. In a subsequent PR, I would like to switch to 4 
(and I think it will give us a very significant performance boost as well).



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