You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "yjshen (via GitHub)" <gi...@apache.org> on 2023/04/04 17:40:19 UTC

[GitHub] [arrow-datafusion] yjshen commented on a diff in pull request #5868: feat: Support SQL filter clause for aggregate expressions

yjshen commented on code in PR #5868:
URL: https://github.com/apache/arrow-datafusion/pull/5868#discussion_r1157564945


##########
datafusion/core/src/physical_plan/aggregates/row_hash.rs:
##########
@@ -463,27 +482,20 @@ impl GroupedHashAggregateStream {
                                     accumulator.merge_batch(&values, &mut state_accessor)
                                 }
                             }
-                        })
-                        // 2.5
-                        .and(Ok(()))?;
+                        })?;
                     // normal accumulators
                     group_state
                         .accumulator_set
                         .iter_mut()
                         .zip(normal_values.iter())
-                        .map(|(accumulator, aggr_array)| {
-                            (
-                                accumulator,
-                                aggr_array
-                                    .iter()
-                                    .map(|array| {
-                                        // 2.3
-                                        array.slice(offsets[0], offsets[1] - offsets[0])
-                                    })
-                                    .collect::<Vec<ArrayRef>>(),
-                            )
-                        })
-                        .try_for_each(|(accumulator, values)| {
+                        .zip(normal_filter_values.iter())

Review Comment:
   ditto



##########
datafusion/core/src/physical_plan/aggregates/no_grouping.rs:
##########
@@ -172,26 +177,34 @@ fn aggregate_batch(
     batch: &RecordBatch,
     accumulators: &mut [AccumulatorItem],
     expressions: &[Vec<Arc<dyn PhysicalExpr>>],
+    filters: &[Option<Arc<dyn PhysicalExpr>>],
 ) -> Result<usize> {
     let mut allocated = 0usize;
 
     // 1.1 iterate accumulators and respective expressions together
-    // 1.2 evaluate expressions
-    // 1.3 update / merge accumulators with the expressions' values
+    // 1.2 filter the batch if necessary

Review Comment:
   Here is another main change.



##########
datafusion/core/src/physical_plan/aggregates/row_hash.rs:
##########
@@ -532,7 +544,7 @@ pub struct RowGroupState {
 }
 
 /// The state of all the groups
-pub struct RowAggregationState {
+pub struct AggregationState {

Review Comment:
   ditto



##########
datafusion/core/src/physical_plan/aggregates/row_hash.rs:
##########
@@ -430,22 +453,18 @@ impl GroupedHashAggregateStream {
                 .try_for_each(|(group_idx, offsets)| {
                     let group_state = &mut row_group_states[*group_idx];
                     // 2.2
+                    // Process row accumulators
                     self.row_accumulators
                         .iter_mut()
                         .zip(row_values.iter())
-                        .map(|(accumulator, aggr_array)| {
-                            (
-                                accumulator,
-                                aggr_array
-                                    .iter()
-                                    .map(|array| {
-                                        // 2.3
-                                        array.slice(offsets[0], offsets[1] - offsets[0])
-                                    })
-                                    .collect::<Vec<ArrayRef>>(),
-                            )
-                        })
-                        .try_for_each(|(accumulator, values)| {
+                        .zip(row_filter_values.iter())

Review Comment:
   This is one of the main changes.



##########
datafusion/sql/src/parser.rs:
##########
@@ -134,7 +134,7 @@ impl<'a> DFParser<'a> {
     /// Parse a sql string into one or [`Statement`]s using the
     /// [`GenericDialect`].
     pub fn parse_sql(sql: &str) -> Result<VecDeque<Statement>, ParserError> {
-        let dialect = &GenericDialect {};
+        let dialect = &PostgreSqlDialect {};

Review Comment:
   I am unsure about the best approach to tackle this. One potential solution is to provide a configuration option that users can set at the beginning.



##########
datafusion/core/src/physical_plan/aggregates/row_hash.rs:
##########
@@ -516,7 +528,7 @@ impl GroupedHashAggregateStream {
 
 /// The state that is built for each output group.
 #[derive(Debug)]
-pub struct RowGroupState {
+pub struct GroupState {

Review Comment:
   The State actually contains both row-wise state and non-row state.



##########
datafusion/core/src/physical_plan/aggregates/row_hash.rs:
##########
@@ -200,19 +214,21 @@ impl GroupedHashAggregateStream {
 
         Ok(GroupedHashAggregateStream {
             schema: Arc::clone(&schema),
-            mode,
-            exec_state,
             input,
-            group_by,
+            mode,
             normal_aggr_expr,
+            normal_aggregate_expressions,
+            normal_filter_expressions,
+            row_aggregate_expressions,
+            row_filter_expressions,
             row_accumulators,
             row_converter,
             row_aggr_schema,
             row_aggr_layout,
+            group_by,
+            aggr_state,
+            exec_state,

Review Comment:
   Group the members based on their usage and arrange them accordingly.



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