You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2021/08/04 23:13:47 UTC

[GitHub] [beam] ibzib commented on a change in pull request #15174: [WIP][BEAM-12100][BEAM-10379][BEAM-9514] AssertionError type mismatch from AggregateScanConverter

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



##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java
##########
@@ -246,6 +250,67 @@ private Transform(
 
       validateWindowIsSupported(windowedStream);
 
+      PTransform<PCollection<Row>, PCollection<Row>> combiner = createCombiner();
+
+      boolean verifyRowValues =
+          pinput.getPipeline().getOptions().as(BeamSqlPipelineOptions.class).getVerifyRowValues();
+      return windowedStream
+          .apply(combiner)
+          .apply(
+              "mergeRecord",
+              ParDo.of(mergeRecord(outputSchema, windowFieldIndex, ignoreValues, verifyRowValues)))
+          .setRowSchema(outputSchema);
+    }
+
+    private PTransform<PCollection<Row>, PCollection<Row>> createCombiner() {
+      // Check if have fields to be grouped
+      if (groupSetCount > 0) {
+        return createGroupCombiner();
+      }
+      return createGlobalCombiner();
+    }
+
+    private PTransform<PCollection<Row>, PCollection<Row>> createGlobalCombiner() {
+      org.apache.beam.sdk.schemas.transforms.Group.Global<Row> globally =
+          org.apache.beam.sdk.schemas.transforms.Group.globally();
+      org.apache.beam.sdk.schemas.transforms.Group.CombineFieldsGlobally<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).
+          combined =
+              (combined == null)
+                  ? globally.aggregateFieldsById(inputs, combineFn, fieldAggregation.outputField)
+                  : combined.aggregateFieldsById(inputs, combineFn, fieldAggregation.outputField);
+        } else {
+          // Combining over a single field, so extract just that field.
+          combined =
+              (combined == null)
+                  ? globally.aggregateField(inputs.get(0), combineFn, fieldAggregation.outputField)
+                  : combined.aggregateField(inputs.get(0), combineFn, fieldAggregation.outputField);
+        }
+      }
+
+      PTransform<PCollection<Row>, PCollection<Row>> combiner = combined;
+      if (combiner == null) {
+        // If no field aggregations were specified, we run a constant combiner that always returns
+        // a single empty row for each key. This is used by the SELECT DISTINCT query plan - in this
+        // case a group by is generated to determine unique keys, and a constant null combiner is
+        // used.
+        combiner =
+            globally.aggregateField(
+                "*",
+                AggregationCombineFnAdapter.createConstantCombineFn(),
+                Field.of(
+                    "e",
+                    FieldType.row(AggregationCombineFnAdapter.EMPTY_SCHEMA).withNullable(true)));
+        ignoreValues = true;
+      }
+      return combiner;
+    }

Review comment:
       It looks like the incorrect null value is coming from here: https://github.com/apache/beam/blob/39cf3fccbb3950b9fd0c8938301d192c3ca85cce/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java#L515
   
   You can fix it by overriding the `identity()` method in `LongSum`. Or maybe you could change it back to `Sum.ofLongs`, I'm not sure why it needed a separate definition here?




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

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