You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2021/08/03 23:34:00 UTC

[jira] [Work logged] (BEAM-12100) SUM should error when overflow/underflow occurs.

     [ https://issues.apache.org/jira/browse/BEAM-12100?focusedWorklogId=633225&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-633225 ]

ASF GitHub Bot logged work on BEAM-12100:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 03/Aug/21 23:33
            Start Date: 03/Aug/21 23:33
    Worklog Time Spent: 10m 
      Work Description: benWize commented on a change in pull request #15174:
URL: https://github.com/apache/beam/pull/15174#discussion_r682168175



##########
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:
       I created this method to process aggregations without "GROUP BY".
   Some test related to aggregations on empty tables are passing now, but different tests like this https://github.com/apache/beam/blob/bdee3bd963d3213c207ea124a6bfd8256deb220b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java#L2216 are failing with this change, I am not sure why, but I when this [transform](https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Group.java#L555 ) is applied, the data is dropped and does not execute the combineFn aggregation methods..




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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 633225)
    Time Spent: 40m  (was: 0.5h)

> SUM should error when overflow/underflow occurs.
> ------------------------------------------------
>
>                 Key: BEAM-12100
>                 URL: https://issues.apache.org/jira/browse/BEAM-12100
>             Project: Beam
>          Issue Type: Bug
>          Components: dsl-sql-zetasql
>            Reporter: Kyle Weaver
>            Assignee: Benjamin Gonzalez
>            Priority: P3
>          Time Spent: 40m
>  Remaining Estimate: 0h
>
> SELECT SUM(col1) FROM (SELECT CAST(9223372036854775807 as int64) as col1 UNION ALL SELECT CAST(1 as int64))
> should return an error.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)