@@ -241,84 +241,106 @@ public void shouldHandleUnions() {
241241 }
242242
243243 @ Test
244- public void shouldHandleSingleElementUnionsInArraysAndMaps () {
245- // This test verifies that single-element unions in array items and map values are properly unwrapped
244+ public void shouldHandleSingleElementUnionsInAllTypes () {
245+ // This test verifies that single-element unions in primitives, array items, and map values are properly unwrapped
246246 // and the nested field nullability is preserved during schema merging.
247247 // This reproduces the fix for handling avro.schema.literal with single-element unions like:
248+ // - Primitives: "type": ["string"]
248249 // - Array items: "items": [{"type":"record",...}]
249250 // - Map values: "values": [{"type":"record",...}]
250251 // These single-element unions appear in real-world Avro schemas stored as avro.schema.literal
251252
252- String hive = "struct<id:bigint,items:array<struct<fooconfiguration:struct<name:string,urlvalue:string,source:string>,"
253+ // seu = single-element union
254+ String hive = "struct<id:bigint,name:string,active:boolean,"
255+ + "items:array<struct<fooconfiguration:struct<name:string,urlvalue:string,source:string>,"
253256 + "barconfiguration:struct<name:string,domain:string>>>,"
254- + "metadata:map<string,struct<category:string,priority:int>>>" ;
255-
256- // Define an Avro schema literal similar to what would be stored in avro.schema.literal table property
257- // Note the single-element unions in array items and map values: [{"type":"record",...}]
258- String avroSchemaLiteral =
259- "{\" type\" :\" record\" ,\" name\" :\" test_complex_array_table\" ,\" namespace\" :\" com.example.test\" ,\" fields\" :["
260- + "{\" name\" :\" id\" ,\" type\" :[\" null\" ,\" long\" ],\" default\" :null},"
261- + "{\" name\" :\" items\" ,\" type\" :[\" null\" ,{\" type\" :\" array\" ,\" items\" :[{\" type\" :\" record\" ,\" name\" :\" ItemConfig\" ,\" namespace\" :\" com.example.data\" ,\" fields\" :["
262- + "{\" name\" :\" fooConfiguration\" ,\" type\" :[\" null\" ,{\" type\" :\" record\" ,\" name\" :\" FooConfiguration\" ,\" fields\" :["
263- + "{\" name\" :\" name\" ,\" type\" :\" string\" },"
264- + "{\" name\" :\" urlValue\" ,\" type\" :\" string\" },"
265- + "{\" name\" :\" source\" ,\" type\" :\" string\" }"
266- + "]}],\" default\" :null},"
267- + "{\" name\" :\" barConfiguration\" ,\" type\" :[\" null\" ,{\" type\" :\" record\" ,\" name\" :\" BarConfiguration\" ,\" fields\" :["
268- + "{\" name\" :\" name\" ,\" type\" :\" string\" },"
269- + "{\" name\" :\" domain\" ,\" type\" :\" string\" }"
270- + "]}],\" default\" :null}"
271- + "]}]}],\" default\" :null},"
272- + "{\" name\" :\" metadata\" ,\" type\" :[\" null\" ,{\" type\" :\" map\" ,\" values\" :[{\" type\" :\" record\" ,\" name\" :\" MetadataValue\" ,\" namespace\" :\" com.example.data\" ,\" fields\" :["
273- + "{\" name\" :\" category\" ,\" type\" :\" string\" },"
274- + "{\" name\" :\" priority\" ,\" type\" :\" int\" }"
275- + "]}]}],\" default\" :null}"
276- + "]}" ;
277-
278- Schema avro = new Schema .Parser ().parse (avroSchemaLiteral );
279- Schema merged = merge (hive , avro );
257+ + "metadata:map<string,struct<category:string,priority:int>>,"
258+ + "tags:array<string>>" ;
259+
260+ // Nested record schemas
261+ Schema fooConfigSchema = struct ("FooConfiguration" , "doc-foo" , "com.example.data" ,
262+ required ("name" , Schema .Type .STRING ), required ("urlValue" , Schema .Type .STRING ),
263+ required ("source" , Schema .Type .STRING ));
264+
265+ Schema barConfigSchema = struct ("BarConfiguration" , "doc-bar" , "com.example.data" ,
266+ required ("name" , Schema .Type .STRING ), required ("domain" , Schema .Type .STRING ));
267+
268+ Schema seu_arrayItemConfigSchema = struct ("ItemConfig" , "doc-item" , "com.example.data" ,
269+ optional ("fooConfiguration" , fooConfigSchema ), optional ("barConfiguration" , barConfigSchema ));
270+
271+ Schema seu_mapValueMetadataSchema = struct ("MetadataValue" , "doc-metadata" , "com.example.data" ,
272+ required ("category" , Schema .Type .STRING ), required ("priority" , Schema .Type .INT ));
273+
274+ // Construct Avro schema with single-element unions for primitives, array items, and map values
275+ Schema avro = struct ("test_complex_array_table" , "doc-test" , "com.example.test" , optional ("id" , Schema .Type .LONG ),
276+ field ("name" , singleElementUnion (Schema .create (Schema .Type .STRING )), null , "unknown" , null ),
277+ field ("active" , singleElementUnion (Schema .create (Schema .Type .BOOLEAN )), null , false , null ),
278+ optional ("items" , array (singleElementUnion (seu_arrayItemConfigSchema ))),
279+ optional ("metadata" , map (singleElementUnion (seu_mapValueMetadataSchema ))),
280+ optional ("tags" , array (singleElementUnion (Schema .create (Schema .Type .STRING )))));
281+
282+ // Expected schema after merge: single-element unions should be preserved
283+ // The structure of the Avro schema is maintained, including single-element unions
284+ // Expected output should match the input avro schema
285+ Schema expected = avro ;
280286
281- // Verify that single-element unions were properly handled
282- // Extract items array
283- Schema mergedItemsArray = SchemaUtilities .extractIfOption (merged .getField ("items" ).schema ());
284- Schema mergedItemConfig = mergedItemsArray .getElementType ();
285-
286- // The fix ensures that single-element union [ItemConfig] is unwrapped to ItemConfig
287- // Without the fix, this would fail because the union wouldn't be unwrapped
288- assertEquals (mergedItemConfig .getType (), Schema .Type .RECORD , "Array element should be a record, not a union" );
289-
290- // Extract fooConfiguration and verify nested field nullability is preserved
291- Schema mergedFooConfig =
292- SchemaUtilities .extractIfOption (mergedItemConfig .getField ("fooConfiguration" ).schema ());
293-
294- // Nested fields should be non-nullable (required) as defined in the avro.schema.literal
295- assertEquals (mergedFooConfig .getField ("name" ).schema ().getType (), Schema .Type .STRING ,
296- "name field should be non-nullable string" );
297- assertEquals (mergedFooConfig .getField ("urlValue" ).schema ().getType (), Schema .Type .STRING ,
298- "urlValue field should be non-nullable string" );
299- assertEquals (mergedFooConfig .getField ("source" ).schema ().getType (), Schema .Type .STRING ,
300- "source field should be non-nullable string" );
301-
302- // Verify barConfiguration nested fields
303- Schema mergedBarConfig =
304- SchemaUtilities .extractIfOption (mergedItemConfig .getField ("barConfiguration" ).schema ());
305- assertEquals (mergedBarConfig .getField ("name" ).schema ().getType (), Schema .Type .STRING ,
306- "bar name field should be non-nullable string" );
307- assertEquals (mergedBarConfig .getField ("domain" ).schema ().getType (), Schema .Type .STRING ,
308- "domain field should be non-nullable string" );
309-
310- // Extract metadata map value and verify
311- // Ensures that single-element union [MetadataValue] is unwrapped to MetadataValue
312- Schema mergedMetadataMap = SchemaUtilities .extractIfOption (merged .getField ("metadata" ).schema ());
313- Schema mergedMetadataValue = mergedMetadataMap .getValueType ();
314-
315- assertEquals (mergedMetadataValue .getType (), Schema .Type .RECORD , "Map value should be a record, not a union" );
316-
317- // Fields in MetadataValue should be non-nullable as defined in avro.schema.literal
318- assertEquals (mergedMetadataValue .getField ("category" ).schema ().getType (), Schema .Type .STRING ,
319- "category field should be non-nullable string" );
320- assertEquals (mergedMetadataValue .getField ("priority" ).schema ().getType (), Schema .Type .INT ,
321- "priority field should be non-nullable int" );
287+ Schema actual = merge (hive , avro );
288+
289+ System .out .println ("\n === INPUT AVRO SCHEMA ===" );
290+ System .out .println (avro .toString (true ));
291+ System .out .println ("\n === EXPECTED OUTPUT SCHEMA ===" );
292+ System .out .println (expected .toString (true ));
293+ System .out .println ("\n === ACTUAL OUTPUT SCHEMA ===" );
294+ System .out .println (actual .toString (true ));
295+ System .out .println ("\n === END ===\n " );
296+
297+ assertSchema (expected , actual );
298+ }
299+
300+ @ Test
301+ public void shouldHandleSingleElementUnionsWithHiveUnionType () {
302+ // This test ensures backward compatibility with Hive union-encoded-as-struct format
303+ // when single-element unions are present in array items and map values.
304+ // In Hive, unions are represented as `uniontype<type1,type2>` and encoded as
305+ // struct<tag:int,field0:type1,field1:type2> in the schema.
306+ // This test verifies that extractIfOption correctly handles single-element unions
307+ // in nested structures even when the Hive schema uses uniontype format.
308+
309+ // seu = single-element union
310+ String hive = "struct<id:bigint,"
311+ + "status:uniontype<string,int>,"
312+ + "items:array<uniontype<struct<value:string>>>,"
313+ + "metadata:map<string,uniontype<struct<priority:int>>>>" ;
314+
315+ Schema seu_arrayItemSchema =
316+ struct ("Item" , "doc-item" , "com.example.data" , required ("value" , Schema .Type .STRING ));
317+
318+ Schema seu_mapValueMetadataSchema =
319+ struct ("MetadataInfo" , "doc-metadata" , "com.example.data" , required ("priority" , Schema .Type .INT ));
320+
321+ // Construct Avro schema with single-element unions in array items and map values
322+ // Also includes a regular union for the status field (to test Hive uniontype compatibility)
323+ Schema avro = struct ("test_union_compat" , "doc-test" , "com.example.test" , optional ("id" , Schema .Type .LONG ),
324+ required ("status" , union (Schema .Type .NULL , Schema .Type .STRING , Schema .Type .INT )),
325+ optional ("items" , array (singleElementUnion (seu_arrayItemSchema ))),
326+ optional ("metadata" , map (singleElementUnion (seu_mapValueMetadataSchema ))));
327+
328+ // Expected schema: single-element unions should be preserved, regular union preserved
329+ // The Avro schema structure is maintained
330+ // Expected output should match the input avro schema
331+ Schema expected = avro ;
332+
333+ Schema actual = merge (hive , avro );
334+
335+ System .out .println ("\n === INPUT AVRO SCHEMA (HiveUnionType test) ===" );
336+ System .out .println (avro .toString (true ));
337+ System .out .println ("\n === EXPECTED OUTPUT SCHEMA (HiveUnionType test) ===" );
338+ System .out .println (expected .toString (true ));
339+ System .out .println ("\n === ACTUAL OUTPUT SCHEMA (HiveUnionType test) ===" );
340+ System .out .println (actual .toString (true ));
341+ System .out .println ("\n === END ===\n " );
342+
343+ assertSchema (expected , actual );
322344 }
323345
324346 // TODO: tests to retain schema props
@@ -331,6 +353,10 @@ private Schema union(Schema.Type... types) {
331353 return Schema .createUnion (Arrays .stream (types ).map (Schema ::create ).collect (Collectors .toList ()));
332354 }
333355
356+ private Schema singleElementUnion (Schema schema ) {
357+ return Schema .createUnion (Arrays .asList (schema ));
358+ }
359+
334360 private void assertSchema (Schema expected , Schema actual ) {
335361 assertEquals (actual .toString (true ), expected .toString (true ));
336362 }
0 commit comments