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 tests 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 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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]