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 2020/04/02 07:09:54 UTC

[GitHub] [beam] reuvenlax opened a new pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution

reuvenlax opened a new pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution
URL: https://github.com/apache/beam/pull/11290
 
 
   This PR also adds some zetasql unit tests that expose the previous bug.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] reuvenlax commented on issue #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution

Posted by GitBox <gi...@apache.org>.
reuvenlax commented on issue #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution
URL: https://github.com/apache/beam/pull/11290#issuecomment-608034108
 
 
   Run Java PreCommit

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] TheNeuralBit commented on a change in pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution
URL: https://github.com/apache/beam/pull/11290#discussion_r402450787
 
 

 ##########
 File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
 ##########
 @@ -63,16 +62,6 @@ public BeamSetOperatorRelBase(BeamRelNode beamRelNode, OpType opType, boolean al
         inputs);
     PCollection<Row> leftRows = inputs.get(0);
     PCollection<Row> rightRows = inputs.get(1);
-    Schema leftSchema = leftRows.getSchema();
-    Schema rightSchema = rightRows.getSchema();
-    if (!leftSchema.typesEqual(rightSchema)) {
-      throw new IllegalArgumentException(
-          "Can't intersect two tables with different schemas."
-              + "lhsSchema: "
-              + leftSchema
-              + "  rhsSchema: "
-              + rightSchema);
-    }
 
 Review comment:
   Shouldn't we still verify that the schemas have equivalent types without nullability?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] TheNeuralBit commented on a change in pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution
URL: https://github.com/apache/beam/pull/11290#discussion_r402477845
 
 

 ##########
 File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
 ##########
 @@ -63,16 +62,6 @@ public BeamSetOperatorRelBase(BeamRelNode beamRelNode, OpType opType, boolean al
         inputs);
     PCollection<Row> leftRows = inputs.get(0);
     PCollection<Row> rightRows = inputs.get(1);
-    Schema leftSchema = leftRows.getSchema();
-    Schema rightSchema = rightRows.getSchema();
-    if (!leftSchema.typesEqual(rightSchema)) {
-      throw new IllegalArgumentException(
-          "Can't intersect two tables with different schemas."
-              + "lhsSchema: "
-              + leftSchema
-              + "  rhsSchema: "
-              + rightSchema);
-    }
 
 Review comment:
   It could be nice to have something in ZetaSQLDialectSpecTest to verify the set operation just in case we break the logic there and not in CoGroup, but I guess it's fine either way

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] reuvenlax commented on issue #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution

Posted by GitBox <gi...@apache.org>.
reuvenlax commented on issue #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution
URL: https://github.com/apache/beam/pull/11290#issuecomment-608075431
 
 
   Run Java PreCommit

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] reuvenlax commented on issue #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution

Posted by GitBox <gi...@apache.org>.
reuvenlax commented on issue #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution
URL: https://github.com/apache/beam/pull/11290#issuecomment-607980938
 
 
   @TheNeuralBit thanks. I'll merge once tests complete

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] reuvenlax merged pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution

Posted by GitBox <gi...@apache.org>.
reuvenlax merged pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution
URL: https://github.com/apache/beam/pull/11290
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] TheNeuralBit commented on a change in pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution
URL: https://github.com/apache/beam/pull/11290#discussion_r402472001
 
 

 ##########
 File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
 ##########
 @@ -63,16 +62,6 @@ public BeamSetOperatorRelBase(BeamRelNode beamRelNode, OpType opType, boolean al
         inputs);
     PCollection<Row> leftRows = inputs.get(0);
     PCollection<Row> rightRows = inputs.get(1);
-    Schema leftSchema = leftRows.getSchema();
-    Schema rightSchema = rightRows.getSchema();
-    if (!leftSchema.typesEqual(rightSchema)) {
-      throw new IllegalArgumentException(
-          "Can't intersect two tables with different schemas."
-              + "lhsSchema: "
-              + leftSchema
-              + "  rhsSchema: "
-              + rightSchema);
-    }
 
 Review comment:
   Ah I see. If there aren't any tests verifying this already could you add one?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] reuvenlax commented on issue #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution

Posted by GitBox <gi...@apache.org>.
reuvenlax commented on issue #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution
URL: https://github.com/apache/beam/pull/11290#issuecomment-607943772
 
 
   Run Java PreCommit

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] reuvenlax commented on issue #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution

Posted by GitBox <gi...@apache.org>.
reuvenlax commented on issue #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution
URL: https://github.com/apache/beam/pull/11290#issuecomment-607676240
 
 
   run sql postcommit

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] reuvenlax commented on a change in pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution

Posted by GitBox <gi...@apache.org>.
reuvenlax commented on a change in pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution
URL: https://github.com/apache/beam/pull/11290#discussion_r402457753
 
 

 ##########
 File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaUtils.java
 ##########
 @@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.schemas;
+
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+
+/** A set of utility functions for schemas. */
+public class SchemaUtils {
+  /**
+   * Given two schema that have matching types, return a nullable-widened schema.
+   *
+   * <p>The schemas must have matching types, except for field names which can differ. The returned
+   * schema will contain the field names in the first schema. All field types will be nullable if
+   * the corresponding field type is nullable in either of the input schemas.
+   */
+  public static Schema mergeWideningNullable(Schema schema1, Schema schema2) {
+    if (schema1.getFieldCount() != schema2.getFieldCount()) {
+      throw new IllegalArgumentException(
+          "Cannot merge schemas with different numbers of fields. "
+              + "schema1: "
+              + schema1
+              + " schema2: "
+              + schema2);
+    }
+    Schema.Builder builder = Schema.builder();
+    for (int i = 0; i < schema1.getFieldCount(); ++i) {
+      String name = schema1.getField(i).getName();
+      builder.addField(
+          name, widenNullableTypes(schema1.getField(i).getType(), schema2.getField(i).getType()));
+    }
+    return builder.build();
+  }
+
+  static FieldType widenNullableTypes(FieldType fieldType1, FieldType fieldType2) {
+    if (fieldType1.getTypeName() != fieldType2.getTypeName()) {
+      throw new IllegalArgumentException(
+          "Cannot merge two types: "
+              + fieldType1.getTypeName()
+              + " and "
+              + fieldType2.getTypeName());
+    }
+
+    FieldType result;
+    switch (fieldType1.getTypeName()) {
+      case ROW:
+        result =
+            FieldType.row(
+                mergeWideningNullable(fieldType1.getRowSchema(), fieldType2.getRowSchema()));
+        break;
+      case ARRAY:
+        FieldType arrayElementType =
+            widenNullableTypes(
+                fieldType1.getCollectionElementType(), fieldType2.getCollectionElementType());
+        result = FieldType.array(arrayElementType);
+        break;
+      case ITERABLE:
+        FieldType iterableElementType =
+            widenNullableTypes(
+                fieldType1.getCollectionElementType(), fieldType2.getCollectionElementType());
+        result = FieldType.iterable(iterableElementType);
+        break;
+      case MAP:
+        FieldType keyType =
+            widenNullableTypes(fieldType1.getMapKeyType(), fieldType2.getMapKeyType());
+        FieldType valueType =
+            widenNullableTypes(fieldType1.getMapValueType(), fieldType2.getMapValueType());
+        result = FieldType.map(keyType, valueType);
+        break;
+      case LOGICAL_TYPE:
+        if (!fieldType1
+            .getLogicalType()
+            .getIdentifier()
+            .equals(fieldType2.getLogicalType().getIdentifier())) {
+          throw new IllegalArgumentException(
+              "Logical types don't match and cannot be merged: "
+                  + fieldType1.getLogicalType().getIdentifier()
+                  + ".v.s"
+                  + fieldType2.getLogicalType().getIdentifier());
+        }
+        // fall through
+      default:
+        result = fieldType1;
+    }
+    return result.withNullable(fieldType1.getNullable() || fieldType2.getNullable());
 
 Review comment:
   Maybe? I'm not sure visitor per se, as we don't require double dispatch. However we might consider generalizing the recursive tree walk.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] reuvenlax commented on a change in pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution

Posted by GitBox <gi...@apache.org>.
reuvenlax commented on a change in pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution
URL: https://github.com/apache/beam/pull/11290#discussion_r402457726
 
 

 ##########
 File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
 ##########
 @@ -63,16 +62,6 @@ public BeamSetOperatorRelBase(BeamRelNode beamRelNode, OpType opType, boolean al
         inputs);
     PCollection<Row> leftRows = inputs.get(0);
     PCollection<Row> rightRows = inputs.get(1);
-    Schema leftSchema = leftRows.getSchema();
-    Schema rightSchema = rightRows.getSchema();
-    if (!leftSchema.typesEqual(rightSchema)) {
-      throw new IllegalArgumentException(
-          "Can't intersect two tables with different schemas."
-              + "lhsSchema: "
-              + leftSchema
-              + "  rhsSchema: "
-              + rightSchema);
-    }
 
 Review comment:
   The merge function itself now verifies this.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] TheNeuralBit commented on a change in pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution
URL: https://github.com/apache/beam/pull/11290#discussion_r402454719
 
 

 ##########
 File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaUtils.java
 ##########
 @@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.schemas;
+
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+
+/** A set of utility functions for schemas. */
+public class SchemaUtils {
+  /**
+   * Given two schema that have matching types, return a nullable-widened schema.
+   *
+   * <p>The schemas must have matching types, except for field names which can differ. The returned
+   * schema will contain the field names in the first schema. All field types will be nullable if
+   * the corresponding field type is nullable in either of the input schemas.
+   */
+  public static Schema mergeWideningNullable(Schema schema1, Schema schema2) {
+    if (schema1.getFieldCount() != schema2.getFieldCount()) {
+      throw new IllegalArgumentException(
+          "Cannot merge schemas with different numbers of fields. "
+              + "schema1: "
+              + schema1
+              + " schema2: "
+              + schema2);
+    }
+    Schema.Builder builder = Schema.builder();
+    for (int i = 0; i < schema1.getFieldCount(); ++i) {
+      String name = schema1.getField(i).getName();
+      builder.addField(
+          name, widenNullableTypes(schema1.getField(i).getType(), schema2.getField(i).getType()));
+    }
+    return builder.build();
+  }
+
+  static FieldType widenNullableTypes(FieldType fieldType1, FieldType fieldType2) {
+    if (fieldType1.getTypeName() != fieldType2.getTypeName()) {
+      throw new IllegalArgumentException(
+          "Cannot merge two types: "
+              + fieldType1.getTypeName()
+              + " and "
+              + fieldType2.getTypeName());
+    }
+
+    FieldType result;
+    switch (fieldType1.getTypeName()) {
+      case ROW:
+        result =
+            FieldType.row(
+                mergeWideningNullable(fieldType1.getRowSchema(), fieldType2.getRowSchema()));
+        break;
+      case ARRAY:
+        FieldType arrayElementType =
+            widenNullableTypes(
+                fieldType1.getCollectionElementType(), fieldType2.getCollectionElementType());
+        result = FieldType.array(arrayElementType);
+        break;
+      case ITERABLE:
+        FieldType iterableElementType =
+            widenNullableTypes(
+                fieldType1.getCollectionElementType(), fieldType2.getCollectionElementType());
+        result = FieldType.iterable(iterableElementType);
+        break;
+      case MAP:
+        FieldType keyType =
+            widenNullableTypes(fieldType1.getMapKeyType(), fieldType2.getMapKeyType());
+        FieldType valueType =
+            widenNullableTypes(fieldType1.getMapValueType(), fieldType2.getMapValueType());
+        result = FieldType.map(keyType, valueType);
+        break;
+      case LOGICAL_TYPE:
+        if (!fieldType1
+            .getLogicalType()
+            .getIdentifier()
+            .equals(fieldType2.getLogicalType().getIdentifier())) {
+          throw new IllegalArgumentException(
+              "Logical types don't match and cannot be merged: "
+                  + fieldType1.getLogicalType().getIdentifier()
+                  + ".v.s"
+                  + fieldType2.getLogicalType().getIdentifier());
+        }
+        // fall through
+      default:
+        result = fieldType1;
+    }
+    return result.withNullable(fieldType1.getNullable() || fieldType2.getNullable());
 
 Review comment:
   Just a general comment: do you think we should add a visitor pattern for this sort of thing?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] reuvenlax commented on a change in pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution

Posted by GitBox <gi...@apache.org>.
reuvenlax commented on a change in pull request #11290: [BEAM-9670] Fix nullability widening in CoGroup key resolution
URL: https://github.com/apache/beam/pull/11290#discussion_r402473331
 
 

 ##########
 File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
 ##########
 @@ -63,16 +62,6 @@ public BeamSetOperatorRelBase(BeamRelNode beamRelNode, OpType opType, boolean al
         inputs);
     PCollection<Row> leftRows = inputs.get(0);
     PCollection<Row> rightRows = inputs.get(1);
-    Schema leftSchema = leftRows.getSchema();
-    Schema rightSchema = rightRows.getSchema();
-    if (!leftSchema.typesEqual(rightSchema)) {
-      throw new IllegalArgumentException(
-          "Can't intersect two tables with different schemas."
-              + "lhsSchema: "
-              + leftSchema
-              + "  rhsSchema: "
-              + rightSchema);
-    }
 
 Review comment:
   There is a test in CoGroupTest.java that verifies this. Should I add an additional one 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services