Skip to content

Commit a8bf3fb

Browse files
committed
[LI] Align default value validation align with avro semantics in terms of nullable (nested) fields (linkedin#92)
* Align default value validation align with avro semantics in terms of nullable (nested) fields * Allow setting null as default value for nested fields in record default
1 parent 1891a1a commit a8bf3fb

File tree

4 files changed

+135
-4
lines changed

4 files changed

+135
-4
lines changed

api/src/main/java/org/apache/iceberg/types/Types.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -462,9 +462,8 @@ private static void validateDefaultValue(Object defaultValue, Type type) {
462462
if (defaultStruct.isEmpty()) {
463463
return;
464464
}
465-
Preconditions.checkArgument(defaultStruct.size() == type.asStructType().fields().size());
466-
for (String fieldName : defaultStruct.keySet()) {
467-
NestedField.validateDefaultValue(defaultStruct.get(fieldName), type.asStructType().field(fieldName).type);
465+
for (NestedField field : type.asStructType().fields()) {
466+
validateDefaultValue(defaultStruct.getOrDefault(field.name(), field.getDefaultValue()), field.type());
468467
}
469468
break;
470469

core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java

Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,9 +20,11 @@
2020
package org.apache.iceberg.avro;
2121

2222
import java.util.ArrayList;
23+
import java.util.LinkedHashMap;
2324
import java.util.List;
2425
import java.util.Map;
2526
import java.util.Set;
27+
import java.util.stream.Collectors;
2628
import org.apache.avro.JsonProperties;
2729
import org.apache.avro.LogicalType;
2830
import org.apache.avro.LogicalTypes;
@@ -505,4 +507,24 @@ static boolean hasNonNullDefaultValue(Schema.Field field) {
505507
return field.hasDefaultValue() && field.defaultVal() != JsonProperties.NULL_VALUE &&
506508
!(field.defaultVal() instanceof String && ((String) field.defaultVal()).equalsIgnoreCase("null"));
507509
}
510+
511+
static Object convertToJavaDefaultValue(Object defaultValue) {
512+
if (defaultValue instanceof List) {
513+
return ((List<?>) defaultValue).stream()
514+
.map(AvroSchemaUtil::convertToJavaDefaultValue)
515+
.collect(Collectors.toList());
516+
} else if (defaultValue instanceof Map) {
517+
// can't seem to use the java8 stream api on map correctly because of setting null value in map
518+
Map<Object, Object> retMap = new LinkedHashMap<>();
519+
for (Map.Entry<Object, Object> entry : ((Map<Object, Object>) defaultValue).entrySet()) {
520+
retMap.put(entry.getKey(), convertToJavaDefaultValue(entry.getValue()));
521+
}
522+
return retMap;
523+
} else if (defaultValue == JsonProperties.NULL_VALUE) {
524+
// convert the JsonProperties.NULL_VALUE whenever we see it
525+
return null;
526+
}
527+
// don't touch any other primitive values
528+
return defaultValue;
529+
}
508530
}

core/src/main/java/org/apache/iceberg/avro/SchemaToType.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -94,7 +94,8 @@ public Type record(Schema record, List<String> names, List<Type> fieldTypes) {
9494
Type fieldType = fieldTypes.get(i);
9595
int fieldId = getId(field);
9696

97-
Object defaultValue = AvroSchemaUtil.hasNonNullDefaultValue(field) ? field.defaultVal() : null;
97+
Object defaultValue = AvroSchemaUtil.hasNonNullDefaultValue(field) ?
98+
AvroSchemaUtil.convertToJavaDefaultValue(field.defaultVal()) : null;
9899

99100
if (AvroSchemaUtil.isOptionSchema(field.schema()) || AvroSchemaUtil.isOptionalComplexUnion(field.schema())) {
100101
newFields.add(Types.NestedField.optional(fieldId, field.name(), fieldType, defaultValue, field.doc()));

core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java

Lines changed: 109 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -325,4 +325,113 @@ public void testFieldDocsArePreserved() {
325325
List<String> origFieldDocs = Lists.newArrayList(Iterables.transform(origSchema.columns(), Types.NestedField::doc));
326326
Assert.assertEquals(origFieldDocs, fieldDocs);
327327
}
328+
329+
@Test
330+
public void testConversionOfRecordDefaultWithOptionalNestedField() {
331+
String schemaString = "{\n" +
332+
" \"type\": \"record\",\n" +
333+
" \"name\": \"root\",\n" +
334+
" \"fields\": [\n" +
335+
" {\n" +
336+
" \"name\": \"outer\",\n" +
337+
" \"type\": {\n" +
338+
" \"type\": \"record\",\n" +
339+
" \"name\": \"outerRecord\",\n" +
340+
" \"fields\": [\n" +
341+
" {\n" +
342+
" \"name\": \"mapField\",\n" +
343+
" \"type\": {\n" +
344+
" \"type\": \"map\",\n" +
345+
" \"values\": \"string\"\n" +
346+
" }\n" +
347+
" },\n" +
348+
" {\n" +
349+
" \"name\": \"recordField\",\n" +
350+
" \"type\": [\n" +
351+
" \"null\",\n" +
352+
" {\n" +
353+
" \"type\": \"record\",\n" +
354+
" \"name\": \"inner\",\n" +
355+
" \"fields\": [\n" +
356+
" {\n" +
357+
" \"name\": \"innerString\",\n" +
358+
" \"type\": \"string\"\n" +
359+
" }\n" +
360+
" ]\n" +
361+
" }\n" +
362+
" ],\n" +
363+
" \"default\": null\n" +
364+
" }\n" +
365+
" ]\n" +
366+
" },\n" +
367+
" \"default\": {\n" +
368+
" \"mapField\": {}\n" +
369+
" }\n" +
370+
" }\n" +
371+
" ]\n" +
372+
"}";
373+
Schema schema = new Schema.Parser().parse(schemaString);
374+
org.apache.iceberg.Schema iSchema = AvroSchemaUtil.toIceberg(schema);
375+
Assert.assertEquals("table {\n" +
376+
" 0: outer: required struct<4: mapField: required map<string, string>, " +
377+
"5: recordField: optional struct<3: innerString: required string>>, default value: {mapField={}}, \n" +
378+
"}", iSchema.toString());
379+
}
380+
381+
@Test
382+
public void testConversionOfRecordDefaultWithOptionalNestedField2() {
383+
String schemaString = "{\n" +
384+
" \"type\": \"record\",\n" +
385+
" \"name\": \"root\",\n" +
386+
" \"fields\": [\n" +
387+
" {\n" +
388+
" \"name\": \"outer\",\n" +
389+
" \"type\": {\n" +
390+
" \"type\": \"record\",\n" +
391+
" \"name\": \"outerRecord\",\n" +
392+
" \"fields\": [\n" +
393+
" {\n" +
394+
" \"name\": \"mapField\",\n" +
395+
" \"type\": {\n" +
396+
" \"type\": \"map\",\n" +
397+
" \"values\": \"string\"\n" +
398+
" }\n" +
399+
" },\n" +
400+
" {\n" +
401+
" \"name\": \"recordField\",\n" +
402+
" \"type\": [\n" +
403+
" \"null\",\n" +
404+
" {\n" +
405+
" \"type\": \"record\",\n" +
406+
" \"name\": \"inner\",\n" +
407+
" \"fields\": [\n" +
408+
" {\n" +
409+
" \"name\": \"innerString\",\n" +
410+
" \"type\": \"string\"\n" +
411+
" }\n" +
412+
" ]\n" +
413+
" }\n" +
414+
" ],\n" +
415+
" \"default\": null\n" +
416+
" }\n" +
417+
" ]\n" +
418+
" },\n" +
419+
" \"default\": {\n" +
420+
" \"mapField\": {\n" +
421+
" \"foo\": \"bar\",\n" +
422+
" \"x\": \"y\"\n" +
423+
" },\n" +
424+
" \"recordField\": null\n" +
425+
" }\n" +
426+
" }\n" +
427+
" ]\n" +
428+
"}";
429+
Schema schema = new Schema.Parser().parse(schemaString);
430+
org.apache.iceberg.Schema iSchema = AvroSchemaUtil.toIceberg(schema);
431+
Assert.assertEquals("table {\n" +
432+
" 0: outer: required struct<4: mapField: required map<string, string>, " +
433+
"5: recordField: optional struct<3: innerString: required string>>, " +
434+
"default value: {mapField={foo=bar, x=y}, recordField=null}, \n" +
435+
"}", iSchema.toString());
436+
}
328437
}

0 commit comments

Comments
 (0)