Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[BEAM-12100][BEAM-10379][BEAM-9514][BEAM-12647][BEAM-12099] AssertionError type mismatch from AggregateScanConverter #15174

Merged
merged 26 commits into from Sep 15, 2021

Conversation

benWize
Copy link
Contributor

@benWize benWize commented Jul 15, 2021

Working on related issues

[BEAM-9514] For aggregate return type, use Calcite type inference instead of ZetaSQL. Reference #14392
[BEAM-12100] LongSum uses Math.addExact that handles overflow and underflow sum.
[BEAM-10379] Remove default filter of nulls in AggregationCombineFn and create a DropNull wrapper for the aggregations that rely on dropping nulls.
[BEAM-12647] Branch aggregation functions to use Combine.GroupedValues or Combine.Globally when using or not GROUP BY.
[BEAM-12099] Fix return value of BitOr aggregation when input is an empty array.


Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

  • Choose reviewer(s) and mention them in a comment (R: @username).
  • Format the pull request title like [BEAM-XXX] Fixes bug in ApproximateQuantiles, where you replace BEAM-XXX with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
  • Update CHANGES.md with noteworthy changes.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

See the Contributor Guide for more tips on how to make review process smoother.

ValidatesRunner compliance status (on master branch)

Lang ULR Dataflow Flink Samza Spark Twister2
Go --- Build Status Build Status Build Status Build Status ---
Java Build Status Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Python --- Build Status
Build Status
Build Status
Build Status
Build Status
--- Build Status ---
XLang Build Status Build Status Build Status --- Build Status ---

Examples testing status on various runners

Lang ULR Dataflow Flink Samza Spark Twister2
Go --- --- --- --- --- --- ---
Java --- Build Status
Build Status
Build Status
--- --- --- --- ---
Python --- --- --- --- --- --- ---
XLang --- --- --- --- --- --- ---

Post-Commit SDK/Transform Integration Tests Status (on master branch)

Go Java Python
Build Status Build Status Build Status
Build Status
Build Status

Pre-Commit Tests Status (on master branch)

--- Java Python Go Website Whitespace Typescript
Non-portable Build Status
Build Status
Build Status
Build Status
Build Status
Build Status Build Status Build Status Build Status
Portable --- Build Status Build Status --- --- ---

See .test-infra/jenkins/README for trigger phrase, status and link of all Jenkins jobs.

GitHub Actions Tests Status (on master branch)

Build python source distribution and wheels
Python tests
Java tests

See CI.md for more information about GitHub Actions CI.

@benWize
Copy link
Contributor Author

benWize commented Jul 15, 2021

Hi @ibzib, I am working on a group of related issues using some of your work in #14392. Could you help me to take a look at the progress so far and provide some guidance on the next steps?

I am planning to include BEAM-12098 and BEAM-12099 in this PR, but for those, I have a problem, in the transformation related to UNNEST, when it receives an empty array, does not return anything so it fails before aggregation transforms (COUNT, BIT_OR).

I think the problem could be in this part https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRel.java#L115 because when the array is empty, this transform does not output anything.

Any guidance on this would be appreciated.
Thanks!

@ibzib
Copy link
Contributor

ibzib commented Jul 22, 2021

Hi @ibzib, I am working on a group of related issues using some of your work in #14392. Could you help me to take a look at the progress so far and provide some guidance on the next steps?

I am planning to include BEAM-12098 and BEAM-12099 in this PR, but for those, I have a problem, in the transformation related to UNNEST, when it receives an empty array, does not return anything so it fails before aggregation transforms (COUNT, BIT_OR).

I think the problem could be in this part https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRel.java#L115 because when the array is empty, this transform does not output anything.

Any guidance on this would be appreciated.
Thanks!

Hi Benjamin, thanks for working on this.

If I understand correctly, UNNEST([]) returning no values (an empty pcollection) is intended behavior.

I experimented with other queries and found that SELECT COUNT(*) FROM table_empty also returned no values. So I think the problem is Beam SQL's aggregation implementation. Aggregations on empty pcollections are supposed to return some default value, for example Count on an empty pcollection is expected to return 0. I filed a bug for this: BEAM-12647

Comment on lines 273 to 311
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;
}
Copy link
Contributor Author

@benWize benWize Aug 3, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

are failing with this change, I am not sure why, but when this transform is applied, the data is dropped and does not execute the combineFn aggregation methods..

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It looks like the incorrect null value is coming from here:

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?

Copy link
Contributor Author

@benWize benWize Aug 5, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I created LongSum to handle ArithmeticExceptions like Sumoverflow and Sumunderflow, without modifying the original Sum.ofLongs.

@codecov
Copy link

codecov bot commented Aug 5, 2021

Codecov Report

Merging #15174 (2a9182b) into master (823ab5a) will increase coverage by 0.00%.
The diff coverage is n/a.

❗ Current head 2a9182b differs from pull request most recent head 384750a. Consider uploading reports for the commit 384750a to get more accurate results
Impacted file tree graph

@@           Coverage Diff           @@
##           master   #15174   +/-   ##
=======================================
  Coverage   83.74%   83.74%           
=======================================
  Files         443      443           
  Lines       60077    60077           
=======================================
+ Hits        50312    50313    +1     
+ Misses       9765     9764    -1     
Impacted Files Coverage Δ
...hon/apache_beam/runners/worker/bundle_processor.py 93.39% <0.00%> (+0.12%) ⬆️

Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update 762a619...384750a. Read the comment docs.

@benWize
Copy link
Contributor Author

benWize commented Aug 9, 2021

Hi @ibzib, I overrode identity() in LongSum as you suggested, it was necessary to override identity() for the other sum types as well.
I found out that VarianceFn and CovarianceFn were throwing an ArithmeticException like in https://ci-beam.apache.org/job/beam_PreCommit_SQL_Commit/4013/testReport/junit/org.apache.beam.sdk.extensions.sql/BeamSqlDslSqlStdOperatorsTest/testVARPOP/ so I added a check when count and variance are zero to avoid the division undefined exception.
I am stuck now with a couple of errors related to the incompatibility of Coders like in https://ci-beam.apache.org/job/beam_PreCommit_SQL_Java11_Commit/2966/testReport/junit/org.apache.beam.sdk.extensions.sql.zetasql/ZetaSqlDialectSpecTest/testArrayAggZetasql/ and https://ci-beam.apache.org/job/beam_PreCommit_SQL_Java11_Commit/2966/testReport/junit/org.apache.beam.sdk.extensions.sql.zetasql/ZetaSqlDialectSpecTest/testCountIfZetaSQLDialect/.
Do you have a suggestion of how should I proceed with these?
Thanks.

@ibzib
Copy link
Contributor

ibzib commented Aug 11, 2021

I am stuck now with a couple of errors related to the incompatibility of Coders like in https://ci-beam.apache.org/job/beam_PreCommit_SQL_Java11_Commit/2966/testReport/junit/org.apache.beam.sdk.extensions.sql.zetasql/ZetaSqlDialectSpecTest/testArrayAggZetasql/ and https://ci-beam.apache.org/job/beam_PreCommit_SQL_Java11_Commit/2966/testReport/junit/org.apache.beam.sdk.extensions.sql.zetasql/ZetaSqlDialectSpecTest/testCountIfZetaSQLDialect/.
Do you have a suggestion of how should I proceed with these?

I'm guessing the reason the error message mentions "elements of type class java.lang.Object" is because of the type signature of DropNullFn. DropNullFn<T> extends CombineFn<T, Object, Object> DropNullFn should use generics for all three types, not just the input type. Try changing it to DropNullFn<InputT, AccumT, OutputT> extends CombineFn<InputT, AccumT, OutputT>

@benWize
Copy link
Contributor Author

benWize commented Aug 19, 2021

I am stuck now with a couple of errors related to the incompatibility of Coders like in https://ci-beam.apache.org/job/beam_PreCommit_SQL_Java11_Commit/2966/testReport/junit/org.apache.beam.sdk.extensions.sql.zetasql/ZetaSqlDialectSpecTest/testArrayAggZetasql/ and https://ci-beam.apache.org/job/beam_PreCommit_SQL_Java11_Commit/2966/testReport/junit/org.apache.beam.sdk.extensions.sql.zetasql/ZetaSqlDialectSpecTest/testCountIfZetaSQLDialect/.
Do you have a suggestion of how should I proceed with these?

I'm guessing the reason the error message mentions "elements of type class java.lang.Object" is because of the type signature of DropNullFn. DropNullFn<T> extends CombineFn<T, Object, Object> DropNullFn should use generics for all three types, not just the input type. Try changing it to DropNullFn<InputT, AccumT, OutputT> extends CombineFn<InputT, AccumT, OutputT>

I used generics in DropNullFn, I did the same in ArrayAgg and that fixed testArrayAggZetasql.
For testCountIfZetaSQLDialect the problem was with the Coder for the accumulator class, implementing Accum as Serializable fixed the problem. Now the only failing test is this https://ci-beam.apache.org/job/beam_PreCommit_SQL_Java11_Commit/2992/testReport/junit/org.apache.beam.sdk.extensions.sql/BeamSqlDslAggregationTest/testAggregationNonGroupedFunctionsWithUnbounded/.
I've tried adding .withoutDefaults() in https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Group.java#L556 but that causes a failure in

because of the default value.

@ibzib
Copy link
Contributor

ibzib commented Aug 19, 2021

@benWize the other option it suggested in the error message was to use Combine.globally().asSingletonView(). Did you try that?

@benWize
Copy link
Contributor Author

benWize commented Aug 19, 2021

@benWize the other option it suggested in the error message was to use Combine.globally().asSingletonView(). Did you try that?

Yes, I tried but it causes a compilation error because of the return type of .asSingletonView(). I tried setting .withoutDefaults() just for Unbounded PCollections, and it is working now. 7473306

@benWize benWize marked this pull request as ready for review August 19, 2021 22:51
@benWize benWize changed the title [WIP][BEAM-12100][BEAM-10379][BEAM-9514] AssertionError type mismatch from AggregateScanConverter [BEAM-12100][BEAM-10379][BEAM-9514][BEAM-12647][BEAM-12099] AssertionError type mismatch from AggregateScanConverter Aug 19, 2021
@ibzib
Copy link
Contributor

ibzib commented Aug 19, 2021

Yes, I tried but it causes a compilation error because of the return type of .asSingletonView(). I tried setting .withoutDefaults() just for Unbounded PCollections, and it is working now. 7473306

I think what we really want is to check whether or not the PCollection is in the global window, which is orthogonal to whether or not it's bounded. (In the test data, the bounded input is globally windowed while the unbounded input is not, but that's not always true generally.)

I'll try to do a full review of this PR tomorrow, but I might not get to it till next week.

@benWize
Copy link
Contributor Author

benWize commented Aug 20, 2021

I think what we really want is to check whether or not the PCollection is in the global window, which is orthogonal to whether or not it's bounded. (In the test data, the bounded input is globally windowed while the unbounded input is not, but that's not always true generally.)

I'll try to do a full review of this PR tomorrow, but I might not get to it till next week.

I changed the condition to validate if the PCollection is in the global window.
Thanks!

@ibzib
Copy link
Contributor

ibzib commented Aug 27, 2021

I changed the condition to validate if the PCollection is in the global window.

@benWize Sorry I haven't been able to review this promptly. I'm planning on reviewing and merging #14729 first, since it's critical we upgrade Calcite. Unfortunately there's a chance that could break this PR. Just a heads up. cc @kileys

@benWize
Copy link
Contributor Author

benWize commented Aug 27, 2021

I changed the condition to validate if the PCollection is in the global window.

@benWize Sorry I haven't been able to review this promptly. I'm planning on reviewing and merging #14729 first, since it's critical we upgrade Calcite. Unfortunately there's a chance that could break this PR. Just a heads up. cc @kileys

Ok Kyle, thanks for the heads up!

Copy link
Contributor

@ibzib ibzib left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you @benWize ! Comments inline.

@@ -216,6 +216,8 @@ public RelWriter explainTerms(RelWriter pw) {
private WindowFn<Row, IntervalWindow> windowFn;
private int windowFieldIndex;
private List<FieldAggregation> fieldAggregations;
private int groupSetCount;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should these be final?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, fixed!

for (FieldAggregation fieldAggregation : fieldAggregations) {
List<Integer> inputs = fieldAggregation.inputs;
CombineFn combineFn = fieldAggregation.combineFn;
if (inputs.size() > 1 || inputs.isEmpty()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: this logic is simpler if we put the single field case first.

if (inputs.size() == 1) {

} else {

}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Changed


private PTransform<PCollection<Row>, PCollection<Row>> createGlobalCombiner() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

createGroupCombiner and createGlobalCombiner are mostly the same. Can we find a way to reduce code duplication?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I created a new interface in order to make the code reusable for both global and byFields combiners.

.put("SUM", BeamBuiltinAggregations::createSum)
.put("$SUM0", BeamBuiltinAggregations::createSum)
.put("AVG", BeamBuiltinAggregations::createAvg)
// Drop null elements for these aggregations BEAM-10379
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It might be confusing to link to that jira here, since it's referring to functions that don't want to drop null elements.

Suggested change
// Drop null elements for these aggregations BEAM-10379
// Drop null elements for these aggregations.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Comment changed

}

static class DropNullFn<InputT, AccumT, OutputT> extends CombineFn<InputT, AccumT, OutputT> {
CombineFn<InputT, AccumT, OutputT> combineFn;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can this be private final?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, fixed

}
}

static class DropNullFn<InputT, AccumT, OutputT> extends CombineFn<InputT, AccumT, OutputT> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this be private?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, fixed

public Long addInput(Long accum, T input) {
return accum | input.longValue();
public Accum addInput(Accum accum, T input) {
accum.bitOr |= input.longValue();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What if input is null?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I added a validation similar to BitAnd

@@ -124,6 +124,11 @@ private BigDecimal getCovariance(CovarianceAccumulator covariance) {
BigDecimal adjustedCount =
this.isSample ? covariance.count().subtract(BigDecimal.ONE) : covariance.count();

// Avoid ArithmeticException: Division is undefined when adjustedCount and covariance are 0
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Isn't division undefined whenever the denominator is 0, regardless of the numerator?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I changed the validation and comment

Copy link
Contributor

@ibzib ibzib left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is one ZetaSQL compliance test failure. (ZetaSQL is mostly open source but its compliance tests are not yet, so we usually end up duplicating the tests in Beam where necessary.)

It seems we are not handling this case properly: Returns NULL if the input contains only NULLs. (from https://github.com/google/zetasql/blob/master/docs/aggregate_functions.md#sum)

Can you write a test for this case and fix it?

@@ -364,13 +369,29 @@
}
}

public interface AggregateCombinerInterface<InputT> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: don't include Interface in the name of an interface.

Suggested change
public interface AggregateCombinerInterface<InputT> {
public interface AggregateCombiner<InputT> {

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Changed

@@ -364,13 +369,29 @@
}
}

public interface AggregateCombinerInterface<InputT> {
<CombineInputT, AccumT, CombineOutputT> AggregateCombinerInterface<InputT> aggregateField(
int inputFielId, CombineFn<CombineInputT, AccumT, CombineOutputT> fn, Field outputField);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

typo

Suggested change
int inputFielId, CombineFn<CombineInputT, AccumT, CombineOutputT> fn, Field outputField);
int inputFieldId, CombineFn<CombineInputT, AccumT, CombineOutputT> fn, Field outputField);

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed

PTransform<PCollection<Row>, PCollection<Row>> combiner = combined;
boolean ignoreValues = false;
PTransform<PCollection<Row>, PCollection<Row>> combiner =
(PTransform<PCollection<Row>, PCollection<Row>>) combined;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These casts indicate that AggregateCombinerInterface should probably extend PTransform.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I tried to extend from PTransform, but I got several conflicts because of the different output types for Global, CombineFieldsGlobally and CombineFieldsByFields classes.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It looks like the problem is that CombineFieldsGlobally and CombineFieldsByFields extend PTransform<PCollection<InputT>, PCollection<Row>>, but Global extends PTransform<PCollection<InputT>, PCollection<Iterable<InputT>>>. However, it looks like you only use Global.globally().aggregateField(...), which returns a CombineFieldsGlobally transform anyway.

So can you try using CombineFieldsGlobally as the initial combiner instead of Global? Then Global doesn't have to extend AggregateCombiner at all.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I did this, but I had to create a new method to make an instance of CombineFieldsGlobally with null schemaAggregateFn because that param is initialized here

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we always turn the null schemaAggregateFn into SchemaAggregateFn.create(), why not initialize with SchemaAggregateFn.create() instead of null?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I made the change and it worked, thanks!

@benWize
Copy link
Contributor Author

benWize commented Sep 1, 2021

There is one ZetaSQL compliance test failure. (ZetaSQL is mostly open source but its compliance tests are not yet, so we usually end up duplicating the tests in Beam where necessary.)

It seems we are not handling this case properly: Returns NULL if the input contains only NULLs. (from https://github.com/google/zetasql/blob/master/docs/aggregate_functions.md#sum)

Can you write a test for this case and fix it?

I added the test, and fix the null handling case, but I had to branch SUM and SUM0 logic.

return new BigDecimalSum0();
default:
throw new UnsupportedOperationException(
String.format("[%s] is not supported in SUM", fieldType));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
String.format("[%s] is not supported in SUM", fieldType));
String.format("[%s] is not supported in SUM0", fieldType));

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed

PTransform<PCollection<Row>, PCollection<Row>> combiner = combined;
boolean ignoreValues = false;
PTransform<PCollection<Row>, PCollection<Row>> combiner =
(PTransform<PCollection<Row>, PCollection<Row>>) combined;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It looks like the problem is that CombineFieldsGlobally and CombineFieldsByFields extend PTransform<PCollection<InputT>, PCollection<Row>>, but Global extends PTransform<PCollection<InputT>, PCollection<Iterable<InputT>>>. However, it looks like you only use Global.globally().aggregateField(...), which returns a CombineFieldsGlobally transform anyway.

So can you try using CombineFieldsGlobally as the initial combiner instead of Global? Then Global doesn't have to extend AggregateCombiner at all.

@ibzib
Copy link
Contributor

ibzib commented Sep 2, 2021

We just merged #14729 so you will need to rebase this PR.

Copy link
Contributor

@ibzib ibzib left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please rebase this PR onto the master branch (instead of merging it).

private final SchemaAggregateFn.Inner schemaAggregateFn;

CombineFieldsGlobally(SchemaAggregateFn.Inner schemaAggregateFn) {
this.schemaAggregateFn = schemaAggregateFn;
}

public static CombineFieldsGlobally initialCombiner() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a javadoc comment for this method.

Also consider renaming it to something more standard. Factory methods like this are usually just called create (like SchemaAggregateFn.create()) so we can follow that pattern here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

@ibzib
Copy link
Contributor

ibzib commented Sep 10, 2021

Run Java PreCommit

@ibzib
Copy link
Contributor

ibzib commented Sep 10, 2021

run sql postcommit

@ibzib
Copy link
Contributor

ibzib commented Sep 10, 2021

@benWize I tried running Google compliance tests on this PR again. Most tests pass, but this test crashes:

https://github.com/google/zetasql/blob/dd883180de6387ad80bcf7534b5aae8191e66621/zetasql/compliance/testdata/aggregation_queries.test#L1591

java.lang.RuntimeException: Could not construct an accumulator Coder for Inner{inputSchema=Fields:
Field{name=$col1, description=, type=INT64 NOT NULL, options={{}}}
Encoding positions:
{$col1=0}
Options:{{}}UUID: 0e4f58b4-f7ac-4531-a10d-df770ec2f974, outputSchema=Fields:
Field{name=$col1, description=, type=INT64, options={{}}}
Encoding positions:
{$col1=0}
Options:{{}}UUID: null, composedCombineFn=org.apache.beam.sdk.transforms.CombineFns$ComposedCombineFn@46d36c10, fieldAggregations=[org.apache.beam.sdk.schemas.transforms.SchemaAggregateFn$Inner$FieldAggregation@2e410f14]}. Accumulator Coder for a Combine may be null, but may not throw an exception
	at org.apache.beam.runners.direct.MultiStepCombine$1.isApplicable(MultiStepCombine.java:114)
	at org.apache.beam.runners.direct.MultiStepCombine$1.matches(MultiStepCombine.java:83)
	at org.apache.beam.sdk.Pipeline$2.enterCompositeTransform(Pipeline.java:278)
	at org.apache.beam.sdk.runners.TransformHierarchy$Node.visit(TransformHierarchy.java:581)
	at org.apache.beam.sdk.runners.TransformHierarchy$Node.visit(TransformHierarchy.java:585)
	at org.apache.beam.sdk.runners.TransformHierarchy$Node.visit(TransformHierarchy.java:585)
	at org.apache.beam.sdk.runners.TransformHierarchy$Node.visit(TransformHierarchy.java:585)
	at org.apache.beam.sdk.runners.TransformHierarchy$Node.visit(TransformHierarchy.java:585)
	at org.apache.beam.sdk.runners.TransformHierarchy$Node.access$500(TransformHierarchy.java:240)
	at org.apache.beam.sdk.runners.TransformHierarchy.visit(TransformHierarchy.java:214)
	at org.apache.beam.sdk.Pipeline.traverseTopologically(Pipeline.java:469)
	at org.apache.beam.sdk.Pipeline.replace(Pipeline.java:268)
	at org.apache.beam.sdk.Pipeline.replaceAll(Pipeline.java:218)
	at org.apache.beam.runners.direct.DirectRunner.performRewrites(DirectRunner.java:246)
	at org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:175)
	at org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:67)
	at org.apache.beam.sdk.Pipeline.run(Pipeline.java:323)
	at org.apache.beam.sdk.Pipeline.run(Pipeline.java:309)
	at org.apache.beam.sdk.extensions.sql.impl.rel.BeamEnumerableConverter.runCollector(BeamEnumerableConverter.java:217)
	at org.apache.beam.sdk.extensions.sql.impl.rel.BeamEnumerableConverter.collectRows(BeamEnumerableConverter.java:236)
	at org.apache.beam.sdk.extensions.sql.impl.rel.BeamEnumerableConverter.toRowList(BeamEnumerableConverter.java:163)
	at org.apache.beam.sdk.extensions.sql.impl.rel.BeamEnumerableConverter.toRowList(BeamEnumerableConverter.java:139)
	at cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl.executeQuery(ExecuteQueryServiceServer.java:309)
	at com.google.zetasql.testing.SqlComplianceServiceGrpc$MethodHandlers.invoke(SqlComplianceServiceGrpc.java:423)
	at com.google.zetasql.io.grpc.stub.ServerCalls$UnaryServerCallHandler$UnaryServerCallListener.onHalfClose(ServerCalls.java:171)
	at com.google.zetasql.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.halfClosed(ServerCallImpl.java:283)
	at com.google.zetasql.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1HalfClosed.runInContext(ServerImpl.java:711)
	at com.google.zetasql.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
	at com.google.zetasql.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:123)
	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
	at java.base/java.lang.Thread.run(Thread.java:830)
Caused by: org.apache.beam.sdk.coders.CannotProvideCoderException: Cannot infer coder for type parameter AccumT
	at org.apache.beam.sdk.coders.CoderRegistry.getCoder(CoderRegistry.java:328)
	at org.apache.beam.sdk.transforms.CombineFnBase$AbstractGlobalCombineFn.getAccumulatorCoder(CombineFnBase.java:119)
	at org.apache.beam.sdk.transforms.Combine$CombineFn.getAccumulatorCoder(Combine.java:389)
	at org.apache.beam.sdk.extensions.sql.impl.transform.agg.AggregationCombineFnAdapter$WrappedCombinerBase.getAccumulatorCoder(AggregationCombineFnAdapter.java:72)
	at org.apache.beam.sdk.transforms.CombineFns$ComposedCombineFn.getAccumulatorCoder(CombineFns.java:430)
	at org.apache.beam.sdk.schemas.transforms.SchemaAggregateFn$Inner.getAccumulatorCoder(SchemaAggregateFn.java:335)
	at org.apache.beam.runners.direct.MultiStepCombine$1.isApplicable(MultiStepCombine.java:106)
	... 31 more

@benWize
Copy link
Contributor Author

benWize commented Sep 10, 2021

I copied the test and fixed the error in bitAnd

@ibzib
Copy link
Contributor

ibzib commented Sep 10, 2021

Run SQL PreCommit

@ibzib
Copy link
Contributor

ibzib commented Sep 10, 2021

Run Java_Examples_Dataflow PreCommit

@ibzib
Copy link
Contributor

ibzib commented Sep 10, 2021

@benWize Thanks, that fixed the error. The tests are no longer crashing, but 6 failures remain:

  • aggregation_queries_test:aggregation_array_agg_1
  • aggregation_queries_test:array_agg_empty_input
  • aggregation_queries_test:aggregation_36
  • aggregation_queries_test:aggregation_53
  • aggregation_queries_test:aggregation_58
  • aggregation_queries_test:aggregation_62

https://github.com/google/zetasql/blob/dd883180de6387ad80bcf7534b5aae8191e66621/zetasql/compliance/testdata/aggregation_queries.test

I took a quick look at the errors, and it seems they are all related to null handling. (I'm guessing these are all test cases where the SQL query failed to compile before, but now they're compiling and giving wrong answers.)

  • Some functions are expected to return null when the input is empty.
  • BIT_AND, BIT_OR, etc. are expected to ignore null input mixed with non-null inputs. I think the current implementations of these functions in Beam are incorrect. The compliance tests are the source of truth.

@benWize
Copy link
Contributor Author

benWize commented Sep 13, 2021

@ibzib I copied most of the tests to check the fixes, I was able to fix null handling for BitOr and BitAnd, but in the case of BitXOr and Array_Agg it shows an error because the inferedType by Calcite is BIGINT NOT NULL, so when trying to return null (Like in an empty input) fails by type mismatch (ARRAY<INT64 NOT NULL> NOT NULL is not nullable).

@ibzib
Copy link
Contributor

ibzib commented Sep 14, 2021

Hi @benWize the operators for BIT_OR etc. are taken from Calcite's SqlStdOperatorTable.

To override their return type inference, we will need replace the operators from SqlStdOperatorTable with custom operators defined in SqlOperators, and make sure nullability is set correctly. For example, BIT_XOR already has a custom operator, but it looks like the return type is non-null.

@benWize
Copy link
Contributor Author

benWize commented Sep 14, 2021

Thanks, @ibzib I fixed null handling for BitXor with your suggestion, and for the case of array_agg I tried overriding type to
x -> createSqlType(SqlTypeName.ARRAY, true)


but an assert fails with java.lang.AssertionError: use createArrayType() instead
I looked into createArrayType() implementation, and it creates an ArraySqlType with nullable = false.

@ibzib
Copy link
Contributor

ibzib commented Sep 14, 2021

Thanks, @ibzib I fixed null handling for BitXor with your suggestion, and for the case of array_agg I tried overriding type to
x -> createSqlType(SqlTypeName.ARRAY, true)

but an assert fails with java.lang.AssertionError: use createArrayType() instead
I looked into createArrayType() implementation, and it creates an ArraySqlType with nullable = false.

ArraySqlType has a public constructor, so can't we just call it directly with nullable = true? https://calcite.apache.org/javadocAggregate/org/apache/calcite/sql/type/ArraySqlType.html#%3Cinit%3E(org.apache.calcite.rel.type.RelDataType,boolean)

@benWize
Copy link
Contributor Author

benWize commented Sep 14, 2021

ArraySqlType has a public constructor, so can't we just call it directly with nullable = true? https://calcite.apache.org/javadocAggregate/org/apache/calcite/sql/type/ArraySqlType.html#%3Cinit%3E(org.apache.calcite.rel.type.RelDataType,boolean)

Yes, it is fixed now.

Copy link
Contributor

@ibzib ibzib left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

All compliance tests are passing now. Thanks for all your hard work!

LGTM

@ibzib ibzib merged commit f9dea10 into apache:master Sep 15, 2021
@benWize
Copy link
Contributor Author

benWize commented Sep 15, 2021

All compliance tests are passing now. Thanks for all your hard work!

LGTM

Thank you for all your guidance!

ibzib added a commit to ibzib/beam that referenced this pull request Sep 15, 2021
ibzib added a commit to ibzib/beam that referenced this pull request Sep 15, 2021
calvinleungyk pushed a commit to calvinleungyk/beam that referenced this pull request Sep 22, 2021
…Error type mismatch from AggregateScanConverter (apache#15174)

* [BEAM-12100] SUM throws error when overflow/underflow occurs

* [BEAM-10379] Remove filter of nulls in AggregateCombineFnAdapter and create a DropNull wrapper for the aggregations that rely on dropping nulls

* Fix checkstyle issues

* [BEAM-12647] Branch combiner in Aggregations

* [BEAM-12647] Create different paths for aggregations with and without GroupBy

* [BEAM-12647] Override identity in LongSums

* [BEAM-12647] Override identity in CombineFns, add condition to avoid undefined division

* [BEAM-12647] Fix Coder Cast Exceptions

* [BEAM-12647] Fix CountIf coder and result bug

* Fix spotless

* [BEAM-12099] Fix return value in Bit_OR for empty arrays

* [BEAM-12647] Add .withoutDefaults() to GloballyCombineFn for unbounded pcollections

* [BEAM-12647] Change condition in Group GloballyCombine to check if the input has a GlobalWindow

* Fix accesors and comments

* [BEAM-12647] Refactor aggregation combineFn to avoid code duplication

* Change interface name and fix typo

* Branch SUM and SUM0 for null management

* Change AggregateCombiner from interface to abstract class to extend PTransform

* Fix SUM0 message

* Change schemaFn initialize from null to SchemaAggregateFn.create()

* Add javadocs to AggregationCombiner

* Fix Coder error in bitAnd operator

* Fix null handling for bitOr and bitAnd ZetaSQL

* Fix null handling BitXor

* Fix null handling array_agg

* Fix @nullable checker error
calvinleungyk pushed a commit to calvinleungyk/beam that referenced this pull request Sep 22, 2021
dmitriikuzinepam pushed a commit to dmitriikuzinepam/beam that referenced this pull request Nov 2, 2021
…Error type mismatch from AggregateScanConverter (apache#15174)

* [BEAM-12100] SUM throws error when overflow/underflow occurs

* [BEAM-10379] Remove filter of nulls in AggregateCombineFnAdapter and create a DropNull wrapper for the aggregations that rely on dropping nulls

* Fix checkstyle issues

* [BEAM-12647] Branch combiner in Aggregations

* [BEAM-12647] Create different paths for aggregations with and without GroupBy

* [BEAM-12647] Override identity in LongSums

* [BEAM-12647] Override identity in CombineFns, add condition to avoid undefined division

* [BEAM-12647] Fix Coder Cast Exceptions

* [BEAM-12647] Fix CountIf coder and result bug

* Fix spotless

* [BEAM-12099] Fix return value in Bit_OR for empty arrays

* [BEAM-12647] Add .withoutDefaults() to GloballyCombineFn for unbounded pcollections

* [BEAM-12647] Change condition in Group GloballyCombine to check if the input has a GlobalWindow

* Fix accesors and comments

* [BEAM-12647] Refactor aggregation combineFn to avoid code duplication

* Change interface name and fix typo

* Branch SUM and SUM0 for null management

* Change AggregateCombiner from interface to abstract class to extend PTransform

* Fix SUM0 message

* Change schemaFn initialize from null to SchemaAggregateFn.create()

* Add javadocs to AggregationCombiner

* Fix Coder error in bitAnd operator

* Fix null handling for bitOr and bitAnd ZetaSQL

* Fix null handling BitXor

* Fix null handling array_agg

* Fix @nullable checker error
dmitriikuzinepam pushed a commit to dmitriikuzinepam/beam that referenced this pull request Nov 2, 2021
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

2 participants