forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Merge pull request apache#11290: [BEAM-9670] Fix nullability widening…
… in CoGroup key resolution
- Loading branch information
Showing
6 changed files
with
261 additions
and
15 deletions.
There are no files selected for viewing
101 changes: 101 additions & 0 deletions
101
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaUtils.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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:https://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()); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
107 changes: 107 additions & 0 deletions
107
sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaUtilsTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,107 @@ | ||
/* | ||
* 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:https://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 static org.junit.Assert.assertEquals; | ||
|
||
import org.apache.beam.sdk.schemas.Schema.FieldType; | ||
import org.junit.Test; | ||
|
||
/** Tests for {@link org.apache.beam.sdk.schemas.SchemaUtils}. */ | ||
public class SchemaUtilsTest { | ||
@Test | ||
public void testWidenPrimitives() { | ||
Schema schema1 = | ||
Schema.builder() | ||
.addField("field1", FieldType.INT32) | ||
.addNullableField("field2", FieldType.STRING) | ||
.build(); | ||
Schema schema2 = | ||
Schema.builder() | ||
.addNullableField("field3", FieldType.INT32) | ||
.addField("field4", FieldType.STRING) | ||
.build(); | ||
Schema expected = | ||
Schema.builder() | ||
.addNullableField("field1", FieldType.INT32) | ||
.addNullableField("field2", FieldType.STRING) | ||
.build(); | ||
assertEquals(expected, SchemaUtils.mergeWideningNullable(schema1, schema2)); | ||
} | ||
|
||
@Test | ||
public void testWidenNested() { | ||
Schema schema1 = | ||
Schema.builder() | ||
.addField("field1", FieldType.INT32) | ||
.addNullableField("field2", FieldType.STRING) | ||
.build(); | ||
Schema schema2 = | ||
Schema.builder() | ||
.addNullableField("field3", FieldType.INT32) | ||
.addField("field4", FieldType.STRING) | ||
.build(); | ||
Schema top1 = Schema.builder().addField("top1", FieldType.row(schema1)).build(); | ||
Schema top2 = Schema.builder().addField("top2", FieldType.row(schema2)).build(); | ||
Schema expected = | ||
Schema.builder() | ||
.addNullableField("field1", FieldType.INT32) | ||
.addNullableField("field2", FieldType.STRING) | ||
.build(); | ||
Schema expectedTop = Schema.builder().addField("top1", FieldType.row(expected)).build(); | ||
|
||
assertEquals(expectedTop, SchemaUtils.mergeWideningNullable(top1, top2)); | ||
} | ||
|
||
@Test | ||
public void testWidenArray() { | ||
Schema schema1 = Schema.builder().addArrayField("field1", FieldType.INT32).build(); | ||
Schema schema2 = | ||
Schema.builder().addArrayField("field1", FieldType.INT32.withNullable(true)).build(); | ||
Schema expected = | ||
Schema.builder().addArrayField("field1", FieldType.INT32.withNullable(true)).build(); | ||
assertEquals(expected, SchemaUtils.mergeWideningNullable(schema1, schema2)); | ||
} | ||
|
||
@Test | ||
public void testWidenIterable() { | ||
Schema schema1 = Schema.builder().addIterableField("field1", FieldType.INT32).build(); | ||
Schema schema2 = | ||
Schema.builder().addIterableField("field1", FieldType.INT32.withNullable(true)).build(); | ||
Schema expected = | ||
Schema.builder().addIterableField("field1", FieldType.INT32.withNullable(true)).build(); | ||
assertEquals(expected, SchemaUtils.mergeWideningNullable(schema1, schema2)); | ||
} | ||
|
||
@Test | ||
public void testWidenMap() { | ||
Schema schema1 = | ||
Schema.builder().addMapField("field1", FieldType.INT32, FieldType.INT32).build(); | ||
Schema schema2 = | ||
Schema.builder() | ||
.addMapField( | ||
"field1", FieldType.INT32.withNullable(true), FieldType.INT32.withNullable(true)) | ||
.build(); | ||
Schema expected = | ||
Schema.builder() | ||
.addMapField( | ||
"field1", FieldType.INT32.withNullable(true), FieldType.INT32.withNullable(true)) | ||
.build(); | ||
assertEquals(expected, SchemaUtils.mergeWideningNullable(schema1, schema2)); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters