Skip to content
Merged
Show file tree
Hide file tree
Changes from all 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
5 changes: 2 additions & 3 deletions api/src/main/java/org/apache/iceberg/types/Types.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down
22 changes: 22 additions & 0 deletions core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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<Object, Object> retMap = new LinkedHashMap<>();
for (Map.Entry<Object, Object> entry : ((Map<Object, Object>) 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;
}
}
3 changes: 2 additions & 1 deletion core/src/main/java/org/apache/iceberg/avro/SchemaToType.java
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,8 @@ public Type record(Schema record, List<String> names, List<Type> 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()));
Expand Down
109 changes: 109 additions & 0 deletions core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java
Original file line number Diff line number Diff line change
Expand Up @@ -325,4 +325,113 @@ public void testFieldDocsArePreserved() {
List<String> 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<string, string>, " +
"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<string, string>, " +
"5: recordField: optional struct<3: innerString: required string>>, " +
"default value: {mapField={foo=bar, x=y}, recordField=null}, \n" +
"}", iSchema.toString());
}
}