ibzib commented on a change in pull request #15174:
URL: https://github.com/apache/beam/pull/15174#discussion_r699544775



##########
File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Group.java
##########
@@ -364,13 +369,29 @@
     }
   }
 
+  public interface AggregateCombinerInterface<InputT> {

Review comment:
       Nit: don't include `Interface` in the name of an interface.
   ```suggestion
     public interface AggregateCombiner<InputT> {
   ```

##########
File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Group.java
##########
@@ -364,13 +369,29 @@
     }
   }
 
+  public interface AggregateCombinerInterface<InputT> {
+    <CombineInputT, AccumT, CombineOutputT> AggregateCombinerInterface<InputT> 
aggregateField(
+        int inputFielId, CombineFn<CombineInputT, AccumT, CombineOutputT> fn, 
Field outputField);

Review comment:
       typo
   ```suggestion
           int inputFieldId, CombineFn<CombineInputT, AccumT, CombineOutputT> 
fn, Field outputField);
   ```

##########
File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java
##########
@@ -243,55 +247,72 @@ private Transform(
       if (windowFn != null) {
         windowedStream = assignTimestampsAndWindow(upstream);
       }
-
       validateWindowIsSupported(windowedStream);
+      // Check if have fields to be grouped
+      if (groupSetCount > 0) {
+        
org.apache.beam.sdk.schemas.transforms.Group.AggregateCombinerInterface<Row> 
byFields =
+            
org.apache.beam.sdk.schemas.transforms.Group.byFieldIds(keyFieldsIds);
+        PTransform<PCollection<Row>, PCollection<Row>> combiner = 
createCombiner(byFields);
+        boolean verifyRowValues =
+            
pinput.getPipeline().getOptions().as(BeamSqlPipelineOptions.class).getVerifyRowValues();
+        return windowedStream
+            .apply(combiner)
+            .apply(
+                "mergeRecord",
+                ParDo.of(
+                    mergeRecord(outputSchema, windowFieldIndex, ignoreValues, 
verifyRowValues)))
+            .setRowSchema(outputSchema);
+      }
+      
org.apache.beam.sdk.schemas.transforms.Group.AggregateCombinerInterface<Row> 
globally =
+          org.apache.beam.sdk.schemas.transforms.Group.globally();
+      PTransform<PCollection<Row>, PCollection<Row>> combiner = 
createCombiner(globally);
+      return windowedStream.apply(combiner).setRowSchema(outputSchema);
+    }
+
+    private PTransform<PCollection<Row>, PCollection<Row>> createCombiner(
+        
org.apache.beam.sdk.schemas.transforms.Group.AggregateCombinerInterface<Row>
+            initialCombiner) {
 
-      org.apache.beam.sdk.schemas.transforms.Group.ByFields<Row> byFields =
-          
org.apache.beam.sdk.schemas.transforms.Group.byFieldIds(keyFieldsIds);
-      org.apache.beam.sdk.schemas.transforms.Group.CombineFieldsByFields<Row> 
combined = null;
+      
org.apache.beam.sdk.schemas.transforms.Group.AggregateCombinerInterface<Row> 
combined = null;
       for (FieldAggregation fieldAggregation : fieldAggregations) {
         List<Integer> inputs = fieldAggregation.inputs;
         CombineFn combineFn = fieldAggregation.combineFn;
-        if (inputs.size() > 1 || inputs.isEmpty()) {
-          // In this path we extract a Row (an empty row if inputs.isEmpty).
+        if (inputs.size() == 1) {
+          // Combining over a single field, so extract just that field.
           combined =
               (combined == null)
-                  ? byFields.aggregateFieldsById(inputs, combineFn, 
fieldAggregation.outputField)
-                  : combined.aggregateFieldsById(inputs, combineFn, 
fieldAggregation.outputField);
+                  ? initialCombiner.aggregateField(
+                      inputs.get(0), combineFn, fieldAggregation.outputField)
+                  : combined.aggregateField(inputs.get(0), combineFn, 
fieldAggregation.outputField);
         } else {
-          // Combining over a single field, so extract just that field.
+          // In this path we extract a Row (an empty row if inputs.isEmpty).
           combined =
               (combined == null)
-                  ? byFields.aggregateField(inputs.get(0), combineFn, 
fieldAggregation.outputField)
-                  : combined.aggregateField(inputs.get(0), combineFn, 
fieldAggregation.outputField);
+                  ? initialCombiner.aggregateFieldsById(
+                      inputs, combineFn, fieldAggregation.outputField)
+                  : combined.aggregateFieldsById(inputs, combineFn, 
fieldAggregation.outputField);
         }
       }
 
-      PTransform<PCollection<Row>, PCollection<Row>> combiner = combined;
-      boolean ignoreValues = false;
+      PTransform<PCollection<Row>, PCollection<Row>> combiner =
+          (PTransform<PCollection<Row>, PCollection<Row>>) combined;

Review comment:
       These casts indicate that `AggregateCombinerInterface` should probably 
extend PTransform.




-- 
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-unsubscr...@beam.apache.org

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


Reply via email to