diff --git a/api/src/main/java/org/apache/iceberg/types/Types.java b/api/src/main/java/org/apache/iceberg/types/Types.java index 82c1ce7e95..8218b83552 100644 --- a/api/src/main/java/org/apache/iceberg/types/Types.java +++ b/api/src/main/java/org/apache/iceberg/types/Types.java @@ -462,9 +462,8 @@ private static void validateDefaultValue(Object defaultValue, Type type) { if (defaultStruct.isEmpty()) { return; } - Preconditions.checkArgument(defaultStruct.size() == type.asStructType().fields().size()); - for (String fieldName : defaultStruct.keySet()) { - NestedField.validateDefaultValue(defaultStruct.get(fieldName), type.asStructType().field(fieldName).type); + for (NestedField field : type.asStructType().fields()) { + validateDefaultValue(defaultStruct.getOrDefault(field.name(), field.getDefaultValue()), field.type()); } break; diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java index e94d8c230d..0851ffa1c3 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java @@ -20,9 +20,11 @@ package org.apache.iceberg.avro; import java.util.ArrayList; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.avro.JsonProperties; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; @@ -466,4 +468,24 @@ static boolean hasNonNullDefaultValue(Schema.Field field) { return field.hasDefaultValue() && field.defaultVal() != JsonProperties.NULL_VALUE && !(field.defaultVal() instanceof String && ((String) field.defaultVal()).equalsIgnoreCase("null")); } + + static Object convertToJavaDefaultValue(Object defaultValue) { + if (defaultValue instanceof List) { + return ((List) defaultValue).stream() + .map(AvroSchemaUtil::convertToJavaDefaultValue) + .collect(Collectors.toList()); + } else if (defaultValue instanceof Map) { + // can't seem to use the java8 stream api on map correctly because of setting null value in map + Map retMap = new LinkedHashMap<>(); + for (Map.Entry entry : ((Map) defaultValue).entrySet()) { + retMap.put(entry.getKey(), convertToJavaDefaultValue(entry.getValue())); + } + return retMap; + } else if (defaultValue == JsonProperties.NULL_VALUE) { + // convert the JsonProperties.NULL_VALUE whenever we see it + return null; + } + // don't touch any other primitive values + return defaultValue; + } } diff --git a/core/src/main/java/org/apache/iceberg/avro/SchemaToType.java b/core/src/main/java/org/apache/iceberg/avro/SchemaToType.java index 285a520753..61e5cc9f92 100644 --- a/core/src/main/java/org/apache/iceberg/avro/SchemaToType.java +++ b/core/src/main/java/org/apache/iceberg/avro/SchemaToType.java @@ -93,7 +93,8 @@ public Type record(Schema record, List names, List fieldTypes) { Type fieldType = fieldTypes.get(i); int fieldId = getId(field); - Object defaultValue = AvroSchemaUtil.hasNonNullDefaultValue(field) ? field.defaultVal() : null; + Object defaultValue = AvroSchemaUtil.hasNonNullDefaultValue(field) ? + AvroSchemaUtil.convertToJavaDefaultValue(field.defaultVal()) : null; if (AvroSchemaUtil.isOptionSchema(field.schema()) || AvroSchemaUtil.isOptionalComplexUnion(field.schema())) { newFields.add(Types.NestedField.optional(fieldId, field.name(), fieldType, defaultValue, field.doc())); diff --git a/core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java b/core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java index 4d649dcde4..b3d1b769aa 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java @@ -325,4 +325,113 @@ public void testFieldDocsArePreserved() { List origFieldDocs = Lists.newArrayList(Iterables.transform(origSchema.columns(), Types.NestedField::doc)); Assert.assertEquals(origFieldDocs, fieldDocs); } + + @Test + public void testConversionOfRecordDefaultWithOptionalNestedField() { + String schemaString = "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"root\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"outer\",\n" + + " \"type\": {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"outerRecord\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"mapField\",\n" + + " \"type\": {\n" + + " \"type\": \"map\",\n" + + " \"values\": \"string\"\n" + + " }\n" + + " },\n" + + " {\n" + + " \"name\": \"recordField\",\n" + + " \"type\": [\n" + + " \"null\",\n" + + " {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"inner\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"innerString\",\n" + + " \"type\": \"string\"\n" + + " }\n" + + " ]\n" + + " }\n" + + " ],\n" + + " \"default\": null\n" + + " }\n" + + " ]\n" + + " },\n" + + " \"default\": {\n" + + " \"mapField\": {}\n" + + " }\n" + + " }\n" + + " ]\n" + + "}"; + Schema schema = new Schema.Parser().parse(schemaString); + org.apache.iceberg.Schema iSchema = AvroSchemaUtil.toIceberg(schema); + Assert.assertEquals("table {\n" + + " 0: outer: required struct<4: mapField: required map, " + + "5: recordField: optional struct<3: innerString: required string>>, default value: {mapField={}}, \n" + + "}", iSchema.toString()); + } + + @Test + public void testConversionOfRecordDefaultWithOptionalNestedField2() { + String schemaString = "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"root\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"outer\",\n" + + " \"type\": {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"outerRecord\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"mapField\",\n" + + " \"type\": {\n" + + " \"type\": \"map\",\n" + + " \"values\": \"string\"\n" + + " }\n" + + " },\n" + + " {\n" + + " \"name\": \"recordField\",\n" + + " \"type\": [\n" + + " \"null\",\n" + + " {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"inner\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"innerString\",\n" + + " \"type\": \"string\"\n" + + " }\n" + + " ]\n" + + " }\n" + + " ],\n" + + " \"default\": null\n" + + " }\n" + + " ]\n" + + " },\n" + + " \"default\": {\n" + + " \"mapField\": {\n" + + " \"foo\": \"bar\",\n" + + " \"x\": \"y\"\n" + + " },\n" + + " \"recordField\": null\n" + + " }\n" + + " }\n" + + " ]\n" + + "}"; + Schema schema = new Schema.Parser().parse(schemaString); + org.apache.iceberg.Schema iSchema = AvroSchemaUtil.toIceberg(schema); + Assert.assertEquals("table {\n" + + " 0: outer: required struct<4: mapField: required map, " + + "5: recordField: optional struct<3: innerString: required string>>, " + + "default value: {mapField={foo=bar, x=y}, recordField=null}, \n" + + "}", iSchema.toString()); + } }