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

[GOBBLIN-496] Support nullable union when getting field schema #2367

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils;
import org.codehaus.jackson.JsonNode;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -126,6 +127,9 @@ private static Optional<Schema> getFieldSchemaHelper(Schema schema, List<String>
}
switch (schema.getType()) {
case UNION:
if (AvroSerdeUtils.isNullableType(schema)) {
return AvroUtils.getFieldSchemaHelper(AvroSerdeUtils.getOtherTypeFromNullableType(schema), pathList, field);
}
throw new AvroRuntimeException("Union of complex types cannot be handled : " + schema);
case MAP:
if ((field + 1) == pathList.size()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -279,4 +279,24 @@ public void testComplexTypesInUnionNotSupported()

AvroUtils.getFieldSchema(record.getSchema(), TEST_LOCATION);
}

@Test
public void testUnionWithNull() {
Schema nestedRecord = SchemaBuilder.record("nested").fields().requiredDouble("double")
.requiredString("string").endRecord();
Schema union = SchemaBuilder.unionOf().nullType().and().type(nestedRecord).endUnion();
Schema schema = SchemaBuilder.record("record").fields().name("union").type(union).noDefault().endRecord();

Schema doubleSchema = AvroUtils.getFieldSchema(schema, "union.double").get();
Assert.assertEquals(doubleSchema.getType(), Schema.Type.DOUBLE);

GenericRecord nested = new GenericData.Record(nestedRecord);
nested.put("double", 10);
nested.put("string", "testString");
GenericRecord record = new GenericData.Record(schema);
record.put("union", nested);

String stringValue = AvroUtils.getFieldValue(record, "union.string").get().toString();
Assert.assertEquals(stringValue, "testString");
}
}