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


##########
datafusion/core/src/physical_plan/aggregates/priority_queue.rs:
##########
@@ -0,0 +1,415 @@
+// 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, group_schema, 
AggregateExec,
+    PhysicalGroupBy,
+};
+use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream};
+use arrow::row::{OwnedRow, Row, RowConverter, SortField};
+use arrow_array::RecordBatch;
+use arrow_schema::{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 std::collections::btree_map::Entry;
+use std::collections::hash_map::{Keys, Values};
+use std::collections::{hash_map, BTreeMap, HashMap, HashSet};
+use std::pin::Pin;
+use std::sync::Arc;
+use std::task::{Context, Poll};
+
+pub struct GroupedPriorityQueueAggregateStream {
+    schema: SchemaRef,
+    input: SendableRecordBatchStream,
+    aggregate_arguments: Vec<Vec<Arc<dyn PhysicalExpr>>>,
+    group_by: PhysicalGroupBy,
+    group_converter: RowConverter,
+    value_converter: RowConverter,
+    priority_map: PriorityQueueBiMap,
+}
+
+impl GroupedPriorityQueueAggregateStream {
+    pub fn new(
+        agg: &AggregateExec,
+        context: Arc<TaskContext>,
+        partition: usize,
+        limit: usize,
+    ) -> Result<Self> {
+        let agg_schema = Arc::clone(&agg.schema);
+        let agg_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, 
agg_group_by.expr.len())?;
+
+        let (field, descending) = agg
+            .get_minmax_desc()
+            .ok_or_else(|| DataFusionError::Execution("Min/max 
required".to_string()))?;
+        let sort = SortOptions {
+            descending,
+            nulls_first: false,
+        };
+        let value_fields =
+            vec![SortField::new_with_options(field.data_type().clone(), sort)];
+        let value_converter = RowConverter::new(value_fields.clone())?;
+
+        // TODO: filters
+
+        let group_schema = group_schema(&agg_schema, agg_group_by.expr.len());
+        let group_fields = group_schema
+            .fields()
+            .iter()
+            .map(|f| SortField::new(f.data_type().clone()))
+            .collect::<Vec<_>>();
+        let group_converter = RowConverter::new(group_fields.clone())?;
+
+        Ok(GroupedPriorityQueueAggregateStream {
+            schema: agg_schema,
+            input,
+            aggregate_arguments,
+            group_by: agg_group_by,
+            priority_map: PriorityQueueBiMap::new(limit),
+            group_converter,
+            value_converter,
+        })
+    }
+}
+
+impl RecordBatchStream for GroupedPriorityQueueAggregateStream {
+    fn schema(&self) -> SchemaRef {
+        self.schema.clone()
+    }
+}
+
+impl Stream for GroupedPriorityQueueAggregateStream {
+    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)) => {
+                    let group_by_values = evaluate_group_by(&self.group_by, 
&batch)?;
+                    let group_by_values = match group_by_values.as_slice() {
+                        [] => {
+                            Err(DataFusionError::Execution("vals 
required".to_string()))?
+                        }
+                        [vals] => vals,
+                        _ => {
+                            Err(DataFusionError::Execution("1 val 
required".to_string()))?
+                        }
+                    };
+                    let input_values = 
evaluate_many(&self.aggregate_arguments, &batch)?;

Review Comment:
   I guess it would be most optimal to:
   1. execute the min/max aggrgate first
   2. create a mask based on resp. the max/min value (i.e. do they make it to 
the top `n` values for each group)
   3. only do row conversion and rest of the logic on filtered values



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