You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "Dandandan (via GitHub)" <gi...@apache.org> on 2023/06/29 18:20:53 UTC

[GitHub] [arrow-datafusion] Dandandan commented on a diff in pull request #6800: (NOT READY FOR REVIEW YET) POC: Demonstrate new `GroupHashAggregate` stream approach

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


##########
datafusion/core/src/physical_plan/aggregates/row_hash2.rs:
##########
@@ -0,0 +1,511 @@
+// 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.
+
+//! Hash aggregation through row format
+//!
+//! POC demonstration of GroupByHashApproach
+
+use log::info;
+use std::sync::Arc;
+use std::task::{Context, Poll};
+use std::vec;
+
+use ahash::RandomState;
+use arrow::row::{OwnedRow, RowConverter, SortField};
+use datafusion_physical_expr::hash_utils::create_hashes;
+use futures::ready;
+use futures::stream::{Stream, StreamExt};
+
+use crate::physical_plan::aggregates::{
+    evaluate_group_by, evaluate_many, evaluate_optional, group_schema, AggregateMode,
+    PhysicalGroupBy,
+};
+use crate::physical_plan::metrics::{BaselineMetrics, RecordOutput};
+use crate::physical_plan::{aggregates, AggregateExpr, PhysicalExpr};
+use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream};
+use arrow::array::*;
+use arrow::{datatypes::SchemaRef, record_batch::RecordBatch};
+use datafusion_common::Result;
+use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt};
+use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation};
+use datafusion_execution::TaskContext;
+use hashbrown::raw::RawTable;
+
+#[derive(Debug, Clone)]
+/// This object tracks the aggregation phase (input/output)
+pub(crate) enum ExecutionState {
+    ReadingInput,
+    /// When producing output, the remaining rows to output are stored
+    /// here and are sliced off as needed in batch_size chunks
+    ProducingOutput(RecordBatch),
+    Done,
+}
+
+use super::AggregateExec;
+
+/// Grouping aggregate
+///
+/// For each aggregation entry, we use:
+/// - [Arrow-row] represents grouping keys for fast hash computation and comparison directly on raw bytes.
+/// - [GroupsAccumulator] to store per group aggregates
+///
+/// The architecture is the following:
+///
+/// TODO
+///
+/// [WordAligned]: datafusion_row::layout
+pub(crate) struct GroupedHashAggregateStream2 {
+    schema: SchemaRef,
+    input: SendableRecordBatchStream,
+    mode: AggregateMode,
+
+    /// Accumulators, one for each `AggregateExpr` in the query
+    accumulators: Vec<Box<dyn GroupsAccumulator>>,
+    /// Arguments expressionf or each accumulator
+    aggregate_arguments: Vec<Vec<Arc<dyn PhysicalExpr>>>,
+    /// Filter expression to evaluate for each aggregate
+    filter_expressions: Vec<Option<Arc<dyn PhysicalExpr>>>,
+
+    /// Converter for each row
+    row_converter: RowConverter,
+    group_by: PhysicalGroupBy,
+
+    /// The memory reservation for this grouping
+    reservation: MemoryReservation,
+
+    /// Logically maps group values to a group_index `group_states`
+    ///
+    /// Uses the raw API of hashbrown to avoid actually storing the
+    /// keys in the table
+    ///
+    /// keys: u64 hashes of the GroupValue
+    /// values: (hash, index into `group_states`)
+    map: RawTable<(u64, usize)>,
+
+    /// The actual group by values, stored in arrow Row format
+    /// the index of group_by_values is the index
+    /// https://github.com/apache/arrow-rs/issues/4466
+    group_by_values: Vec<OwnedRow>,

Review Comment:
   This should probably be a buffer of some sort? `OwnedRow` has a copy of the `RowConfig` per value. If we want to keep using rows(?), something like the following would do:
   
   ```
   pub struct AppendableRows {
       /// Underlying row bytes
       buffer: Vec<[u8]>,
       /// Row `i` has data `&buffer[offsets[i]..offsets[i+1]]`
       offsets: Vec<usize>,
       /// The config for these rows
       config: RowConfig,
   }
   ```
   



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org