Skip to content
Open
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -157,6 +157,27 @@ private static class AvroPartnerAccessor implements PartnerAccessor<Schema, Sche

private static final Schema MAP_KEY = Schema.create(Schema.Type.STRING);

/**
* Extracts the actual type from a schema that may be a union.
* Handles two cases:
* 1. Single-element unions like [{"type":"record",...}] - extracts the single type
* 2. Nullable unions like ["null", "string"] - extracts the non-null type
*
* @param schema The schema to extract from
* @return The extracted schema
*/
private static Schema extractFromUnion(Schema schema) {
if (schema.getType() == Schema.Type.UNION) {
List<Schema> types = schema.getTypes();
if (types.size() == 1) {
// Single-element union: just extract that single type
return types.get(0);
}
}
// Extract the non-null type from nullable unions like ["null", "string"]
return SchemaUtilities.extractIfOption(schema);
}

@Override
public Schema.Field fieldPartner(Schema partner, String fieldName) {
Schema schema = SchemaUtilities.extractIfOption(partner);
Expand All @@ -177,13 +198,19 @@ public Schema mapKeyPartner(Schema partner) {
@Override
public Schema mapValuePartner(Schema partner) {
Schema schema = SchemaUtilities.extractIfOption(partner);
return (schema.getType() == Schema.Type.MAP) ? schema.getValueType() : null;
if (schema.getType() != Schema.Type.MAP) {
return null;
}
return extractFromUnion(schema.getValueType());
}

@Override
public Schema listElementPartner(Schema partner) {
Schema schema = SchemaUtilities.extractIfOption(partner);
return (schema.getType() == Schema.Type.ARRAY) ? schema.getElementType() : null;
if (schema.getType() != Schema.Type.ARRAY) {
return null;
}
return extractFromUnion(schema.getElementType());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -240,6 +240,87 @@ public void shouldHandleUnions() {
assertSchema(expected, merge(hive, avro));
}

@Test
public void shouldHandleSingleElementUnionsInArraysAndMaps() {
// This test verifies that single-element unions in array items and map values are properly unwrapped
// and the nested field nullability is preserved during schema merging.
// This reproduces the fix for handling avro.schema.literal with single-element unions like:
// - Array items: "items": [{"type":"record",...}]
// - Map values: "values": [{"type":"record",...}]
// These single-element unions appear in real-world Avro schemas stored as avro.schema.literal

String hive = "struct<id:bigint,items:array<struct<fooconfiguration:struct<name:string,urlvalue:string,source:string>,"
+ "barconfiguration:struct<name:string,domain:string>>>,"
+ "metadata:map<string,struct<category:string,priority:int>>>";

// Define an Avro schema literal similar to what would be stored in avro.schema.literal table property
// Note the single-element unions in array items and map values: [{"type":"record",...}]
String avroSchemaLiteral =
"{\"type\":\"record\",\"name\":\"test_complex_array_table\",\"namespace\":\"com.example.test\",\"fields\":["
+ "{\"name\":\"id\",\"type\":[\"null\",\"long\"],\"default\":null},"
+ "{\"name\":\"items\",\"type\":[\"null\",{\"type\":\"array\",\"items\":[{\"type\":\"record\",\"name\":\"ItemConfig\",\"namespace\":\"com.example.data\",\"fields\":["
+ "{\"name\":\"fooConfiguration\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"FooConfiguration\",\"fields\":["
+ "{\"name\":\"name\",\"type\":\"string\"},"
+ "{\"name\":\"urlValue\",\"type\":\"string\"},"
+ "{\"name\":\"source\",\"type\":\"string\"}"
+ "]}],\"default\":null},"
+ "{\"name\":\"barConfiguration\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"BarConfiguration\",\"fields\":["
+ "{\"name\":\"name\",\"type\":\"string\"},"
+ "{\"name\":\"domain\",\"type\":\"string\"}"
+ "]}],\"default\":null}"
+ "]}]}],\"default\":null},"
+ "{\"name\":\"metadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":[{\"type\":\"record\",\"name\":\"MetadataValue\",\"namespace\":\"com.example.data\",\"fields\":["
+ "{\"name\":\"category\",\"type\":\"string\"},"
+ "{\"name\":\"priority\",\"type\":\"int\"}"
+ "]}]}],\"default\":null}"
+ "]}";

Schema avro = new Schema.Parser().parse(avroSchemaLiteral);
Schema merged = merge(hive, avro);

// Verify that single-element unions were properly handled
// Extract items array
Schema mergedItemsArray = SchemaUtilities.extractIfOption(merged.getField("items").schema());
Schema mergedItemConfig = mergedItemsArray.getElementType();

// The fix ensures that single-element union [ItemConfig] is unwrapped to ItemConfig
// Without the fix, this would fail because the union wouldn't be unwrapped
assertEquals(mergedItemConfig.getType(), Schema.Type.RECORD, "Array element should be a record, not a union");

// Extract fooConfiguration and verify nested field nullability is preserved
Schema mergedFooConfig =
SchemaUtilities.extractIfOption(mergedItemConfig.getField("fooConfiguration").schema());

// Nested fields should be non-nullable (required) as defined in the avro.schema.literal
assertEquals(mergedFooConfig.getField("name").schema().getType(), Schema.Type.STRING,
"name field should be non-nullable string");
assertEquals(mergedFooConfig.getField("urlValue").schema().getType(), Schema.Type.STRING,
"urlValue field should be non-nullable string");
assertEquals(mergedFooConfig.getField("source").schema().getType(), Schema.Type.STRING,
"source field should be non-nullable string");

// Verify barConfiguration nested fields
Schema mergedBarConfig =
SchemaUtilities.extractIfOption(mergedItemConfig.getField("barConfiguration").schema());
assertEquals(mergedBarConfig.getField("name").schema().getType(), Schema.Type.STRING,
"bar name field should be non-nullable string");
assertEquals(mergedBarConfig.getField("domain").schema().getType(), Schema.Type.STRING,
"domain field should be non-nullable string");

// Extract metadata map value and verify
// Ensures that single-element union [MetadataValue] is unwrapped to MetadataValue
Schema mergedMetadataMap = SchemaUtilities.extractIfOption(merged.getField("metadata").schema());
Schema mergedMetadataValue = mergedMetadataMap.getValueType();

assertEquals(mergedMetadataValue.getType(), Schema.Type.RECORD, "Map value should be a record, not a union");

// Fields in MetadataValue should be non-nullable as defined in avro.schema.literal
assertEquals(mergedMetadataValue.getField("category").schema().getType(), Schema.Type.STRING,
"category field should be non-nullable string");
assertEquals(mergedMetadataValue.getField("priority").schema().getType(), Schema.Type.INT,
"priority field should be non-nullable int");
}

// TODO: tests to retain schema props
// TODO: tests for explicit type compatibility check between hive and avro primitives, once we implement it
// TODO: tests for error case => default value in Avro does not match with type from hive
Expand Down