From 4869453f033344eafd27aead1dd70abe0558a9ad Mon Sep 17 00:00:00 2001 From: Limian Zhang <75445426+rzhang10@users.noreply.github.com> Date: Wed, 25 May 2022 01:21:25 -0700 Subject: [PATCH 01/23] Core: Implement default value parsing and unparsing --- .../apache/iceberg/DefaultValueParser.java | 249 ++++++++++++++++++ .../org/apache/iceberg/util/DateTimeUtil.java | 21 ++ .../TestDefaultValuesParsingAndUnParsing.java | 112 ++++++++ 3 files changed, 382 insertions(+) create mode 100644 core/src/main/java/org/apache/iceberg/DefaultValueParser.java create mode 100644 core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java new file mode 100644 index 000000000000..9119f18f499f --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -0,0 +1,249 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +import com.fasterxml.jackson.databind.JsonNode; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; + +public class DefaultValueParser { + private DefaultValueParser() { + } + + public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) { + validateDefault(type, defaultValue); + + if (defaultValue == null) { + return null; + } + + switch (type.typeId()) { + case BOOLEAN: + return defaultValue.booleanValue(); + case INTEGER: + return defaultValue.intValue(); + case LONG: + return defaultValue.longValue(); + case FLOAT: + return defaultValue.floatValue(); + case DOUBLE: + return defaultValue.doubleValue(); + case DECIMAL: + return defaultValue.decimalValue(); + case STRING: + case UUID: + return defaultValue.textValue(); + case DATE: + case TIME: + case TIMESTAMP: + return Literal.of(defaultValue.textValue()).to(type).value(); + case FIXED: + byte[] fixedBytes = BaseEncoding.base16().decode(defaultValue.textValue().toUpperCase(Locale.ROOT).replaceFirst( + "^0X", + "")); + return ByteBuffer.allocate(((Types.FixedType) type).length()).put(fixedBytes); + case BINARY: + byte[] binaryBytes = + BaseEncoding.base16().decode(defaultValue.textValue().toUpperCase(Locale.ROOT).replaceFirst( + "^0X", "")); + return ByteBuffer.wrap(binaryBytes); + case LIST: + List defaultList = Lists.newArrayList(); + for (JsonNode element : defaultValue) { + defaultList.add(parseDefaultFromJson(type.asListType().elementType(), element)); + } + return defaultList; + case MAP: + Map defaultMap = Maps.newHashMap(); + List keysAndValues = StreamSupport + .stream(defaultValue.spliterator(), false) + .collect(Collectors.toList()); + JsonNode keys = keysAndValues.get(0); + JsonNode values = keysAndValues.get(1); + + List keyList = Lists.newArrayList(keys.iterator()); + List valueList = Lists.newArrayList(values.iterator()); + + for (int i = 0; i < keyList.size(); i++) { + defaultMap.put( + parseDefaultFromJson(type.asMapType().keyType(), keyList.get(i)), + parseDefaultFromJson(type.asMapType().valueType(), valueList.get(i))); + } + return defaultMap; + case STRUCT: + Map defaultStruct = Maps.newHashMap(); + for (Types.NestedField subField : type.asStructType().fields()) { + String fieldIdAsString = String.valueOf(subField.fieldId()); + Object value = defaultValue.has(fieldIdAsString) ? parseDefaultFromJson( + subField.type(), + defaultValue.get(fieldIdAsString)) : null; + if (value != null) { + defaultStruct.put(subField.fieldId(), value); + } + } + return defaultStruct; + default: + return null; + } + } + + public static JsonNode validateDefault(Type type, JsonNode defaultValue) { + if (defaultValue != null && !isValidDefault(type, defaultValue)) { + throw new ValidationException("Invalid default value for type %s: %s not a %s", type, defaultValue, type); + } + return defaultValue; + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + public static boolean isValidDefault(Type type, JsonNode defaultValue) { + if (defaultValue == null) { + return false; + } + switch (type.typeId()) { + case BOOLEAN: + return defaultValue.isBoolean(); + case INTEGER: + return defaultValue.isIntegralNumber() && defaultValue.canConvertToInt(); + case LONG: + return defaultValue.isIntegralNumber() && defaultValue.canConvertToLong(); + case FLOAT: + case DOUBLE: + case DECIMAL: + return defaultValue.isNumber(); + case STRING: + case UUID: + case DATE: + case TIME: + case TIMESTAMP: + return defaultValue.isTextual(); + case FIXED: + case BINARY: + return defaultValue.isTextual() && + (defaultValue.textValue().startsWith("0x") || defaultValue.textValue().startsWith("0X")); + case LIST: + if (!defaultValue.isArray()) { + return false; + } + for (JsonNode element : defaultValue) { + if (!isValidDefault(type.asListType().elementType(), element)) { + return false; + } + } + return true; + case MAP: + if (!defaultValue.isArray()) { + return false; + } + List keysAndValues = StreamSupport + .stream(defaultValue.spliterator(), false) + .collect(Collectors.toList()); + if (keysAndValues.size() != 2) { + return false; + } + JsonNode keys = keysAndValues.get(0); + JsonNode values = keysAndValues.get(1); + if (!keys.isArray() || !values.isArray()) { + return false; + } + List keyList = Lists.newArrayList(keys.iterator()); + List valueList = Lists.newArrayList(values.iterator()); + if (keyList.size() != valueList.size()) { + return false; + } + for (int i = 0; i < keyList.size(); i++) { + if (!isValidDefault(type.asMapType().keyType(), keyList.get(i)) || + !isValidDefault(type.asMapType().valueType(), valueList.get(i))) { + return false; + } + } + return true; + case STRUCT: + if (!defaultValue.isObject()) { + return false; + } + for (Types.NestedField subType : type.asStructType().fields()) { + String fieldId = String.valueOf(subType.fieldId()); + if (!isValidDefault(subType.type(), defaultValue.has(fieldId) ? defaultValue.get(fieldId) : null)) { + return false; + } + } + return true; + default: + return false; + } + } + + public static Object convertJavaDefaultForSerialization(Type type, Object value) { + switch (type.typeId()) { + case DATE: + return DateTimeUtil.formatEpochDays((int) value); + case TIME: + return DateTimeUtil.formatTimeOfDayMicros((long) value); + case TIMESTAMP: + return DateTimeUtil.formatEpochTimeMicros((long) value, ((Types.TimestampType) type).shouldAdjustToUTC()); + case FIXED: + case BINARY: + return "0X" + BaseEncoding.base16().encode(((ByteBuffer) value).array()); + case LIST: + List defaultList = (List) value; + List convertedList = Lists.newArrayListWithExpectedSize(defaultList.size()); + for (Object element : defaultList) { + convertedList.add(convertJavaDefaultForSerialization(type.asListType().elementType(), element)); + } + return convertedList; + case MAP: + Map defaultMap = (Map) value; + List> convertedDefault = Lists.newArrayListWithExpectedSize(2); + List keyList = Lists.newArrayListWithExpectedSize(defaultMap.size()); + List valueList = Lists.newArrayListWithExpectedSize(defaultMap.size()); + for (Map.Entry entry : defaultMap.entrySet()) { + keyList.add(convertJavaDefaultForSerialization(type.asMapType().keyType(), entry.getKey())); + valueList.add(convertJavaDefaultForSerialization(type.asMapType().valueType(), entry.getValue())); + } + convertedDefault.add(keyList); + convertedDefault.add(valueList); + return convertedDefault; + case STRUCT: + Map defaultStruct = (Map) value; + Map convertedStruct = Maps.newHashMap(); + for (Map.Entry entry : defaultStruct.entrySet()) { + int fieldId = entry.getKey(); + convertedStruct.put(fieldId, convertJavaDefaultForSerialization( + type.asStructType().field(fieldId).type(), + entry.getValue())); + } + return convertedStruct; + default: + return value; + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java index a3d5b219de92..33b7b05eb394 100644 --- a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java @@ -83,4 +83,25 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) { public static String formatTimestampMillis(long millis) { return Instant.ofEpochMilli(millis).toString().replace("Z", "+00:00"); } + + public static String formatEpochDays(int days) { + return LocalDate.ofEpochDay(days).format(DateTimeFormatter.ISO_LOCAL_DATE); + } + + public static String formatTimeOfDayMicros(long micros) { + return LocalTime.ofNanoOfDay(micros * 1000).format(DateTimeFormatter.ISO_LOCAL_TIME); + } + + public static String formatEpochTimeMicros(long micros, boolean withUTCZone) { + String localDateTime = LocalDateTime.ofEpochSecond(micros / 1000000, + (int) (micros % 1000000) * 1000, ZoneOffset.UTC).format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); + if (withUTCZone) { + // We standardize the format by always using the UTC zone + return LocalDateTime.parse(localDateTime, DateTimeFormatter.ISO_LOCAL_DATE_TIME) + .atOffset(ZoneOffset.UTC) + .format(DateTimeFormatter.ISO_OFFSET_DATE_TIME); + } else { + return localDateTime; + } + } } diff --git a/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java b/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java new file mode 100644 index 000000000000..082c9b8ad629 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.time.OffsetDateTime; +import java.util.Arrays; +import java.util.Collection; +import java.util.Locale; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.JsonUtil; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +@RunWith(Parameterized.class) +public class TestDefaultValuesParsingAndUnParsing { + + private final Type type; + private final JsonNode defaultValue; + + public TestDefaultValuesParsingAndUnParsing(Type type, JsonNode defaultValue) { + this.type = type; + this.defaultValue = defaultValue; + } + + @Parameterized.Parameters + public static Collection data() { + return Arrays.asList(new Object[][] { + {Types.BooleanType.get(), stringToJsonNode("true")}, + {Types.IntegerType.get(), stringToJsonNode("1")}, + {Types.LongType.get(), stringToJsonNode("9999999")}, + {Types.FloatType.get(), stringToJsonNode("1.23")}, + {Types.DoubleType.get(), stringToJsonNode("123.456")}, + {Types.DateType.get(), stringToJsonNode("\"2007-12-03\"")}, + {Types.TimeType.get(), stringToJsonNode("\"10:15:30\"")}, + {Types.TimestampType.withoutZone(), stringToJsonNode("\"2007-12-03T10:15:30\"")}, + {Types.TimestampType.withZone(), stringToJsonNode("\"2007-12-03T10:15:30+01:00\"")}, + {Types.StringType.get(), stringToJsonNode("\"foo\"")}, + {Types.UUIDType.get(), stringToJsonNode("\"eb26bdb1-a1d8-4aa6-990e-da940875492c\"")}, + {Types.FixedType.ofLength(2), stringToJsonNode("\"0x111f\"")}, + {Types.BinaryType.get(), stringToJsonNode("\"0x0000ff\"")}, + {Types.DecimalType.of(9, 2), stringToJsonNode("123.45")}, + {Types.ListType.ofOptional(1, Types.IntegerType.get()), stringToJsonNode("[1, 2, 3]")}, + {Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()), + stringToJsonNode("[[1,2], [\"foo\", \"bar\"]]")}, + {Types.StructType.of( + required(1, "f1", Types.IntegerType.get(), "doc"), + optional(2, "f2", Types.StringType.get(), "doc")), + stringToJsonNode("{\"1\": 1, \"2\": \"bar\"}")} + }); + } + + private static JsonNode stringToJsonNode(String json) { + try { + ObjectMapper mapper = new ObjectMapper(); + return mapper.readTree(json); + } catch (JsonProcessingException e) { + throw new RuntimeException("Failed to parse: " + json + "; reason: " + e.getMessage(), e); + } + } + + // serialize to json and deserialize back should return the same result + private static String defaultValueParseAndUnParseRoundTrip(Type type, JsonNode defaultValue) + throws JsonProcessingException { + Object javaDefaultValue = DefaultValueParser.parseDefaultFromJson(type, defaultValue); + String jsonDefaultValue = JsonUtil.mapper() + .writeValueAsString(DefaultValueParser.convertJavaDefaultForSerialization(type, javaDefaultValue)); + return jsonDefaultValue; + } + + @Test + public void testTypeWithDefaultValue() throws JsonProcessingException { + Assert.assertTrue(DefaultValueParser.isValidDefault(type, defaultValue)); + String parseThenUnParseDefaultValue = defaultValueParseAndUnParseRoundTrip(type, defaultValue); + // Only if the type is a timestampWithZone type, the round-trip default value will always be standardized to the + // UTC time zone, which might be different in the original value, but they should represent the same instant + if (type.typeId() == Type.TypeID.TIMESTAMP && ((Types.TimestampType) type).shouldAdjustToUTC()) { + Assert.assertTrue( + OffsetDateTime.parse(defaultValue.textValue()).isEqual( + OffsetDateTime.parse(stringToJsonNode(parseThenUnParseDefaultValue).textValue()))); + } else { + Assert.assertEquals( + defaultValue.toString().toLowerCase(Locale.ROOT), + parseThenUnParseDefaultValue.toLowerCase(Locale.ROOT)); + } + } +} From f5e55f1629079713bd9d4d3a6a80d5586247747a Mon Sep 17 00:00:00 2001 From: Limian Zhang <75445426+rzhang10@users.noreply.github.com> Date: Tue, 31 May 2022 00:36:32 -0700 Subject: [PATCH 02/23] Refactor default value parser to combine validation and parsing --- .../apache/iceberg/DefaultValueParser.java | 157 ++++++------------ .../TestDefaultValuesParsingAndUnParsing.java | 7 +- 2 files changed, 52 insertions(+), 112 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index 9119f18f499f..31d47c3eb84e 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -24,10 +24,9 @@ import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; -import org.apache.iceberg.exceptions.ValidationException; +import java.util.UUID; import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; @@ -39,57 +38,83 @@ public class DefaultValueParser { private DefaultValueParser() { } + @SuppressWarnings("checkstyle:CyclomaticComplexity") public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) { - validateDefault(type, defaultValue); - if (defaultValue == null) { + if (defaultValue == null || defaultValue.isNull()) { return null; } switch (type.typeId()) { case BOOLEAN: + Preconditions.checkArgument(defaultValue.isBoolean(), + "Cannot parse %s to a %s value", defaultValue, type); return defaultValue.booleanValue(); case INTEGER: + Preconditions.checkArgument(defaultValue.isIntegralNumber() && defaultValue.canConvertToInt(), + "Cannot parse %s to a %s value", defaultValue, type); return defaultValue.intValue(); case LONG: + Preconditions.checkArgument(defaultValue.isIntegralNumber() && defaultValue.canConvertToLong(), + "Cannot parse %s to a %s value", defaultValue, type); return defaultValue.longValue(); case FLOAT: + Preconditions.checkArgument(defaultValue.isNumber(), + "Cannot parse %s to a %s value", defaultValue, type); return defaultValue.floatValue(); case DOUBLE: + Preconditions.checkArgument(defaultValue.isNumber(), + "Cannot parse %s to a %s value", defaultValue, type); return defaultValue.doubleValue(); case DECIMAL: + Preconditions.checkArgument(defaultValue.isNumber(), + "Cannot parse %s to a %s value", defaultValue, type); return defaultValue.decimalValue(); case STRING: - case UUID: + Preconditions.checkArgument(defaultValue.isTextual(), + "Cannot parse %s to a %s value", defaultValue, type); return defaultValue.textValue(); + case UUID: + Preconditions.checkArgument(defaultValue.isTextual(), + "Cannot parse %s to a %s value", defaultValue, type); + return UUID.fromString(defaultValue.textValue()); case DATE: case TIME: case TIMESTAMP: + Preconditions.checkArgument(defaultValue.isTextual(), + "Cannot parse %s to a %s value", defaultValue, type); return Literal.of(defaultValue.textValue()).to(type).value(); case FIXED: - byte[] fixedBytes = BaseEncoding.base16().decode(defaultValue.textValue().toUpperCase(Locale.ROOT).replaceFirst( - "^0X", - "")); + Preconditions.checkArgument( + defaultValue.isTextual() && defaultValue.textValue().length() == ((Types.FixedType) type).length() * 2, + "Cannot parse %s to a %s value", + defaultValue, + type); + byte[] fixedBytes = BaseEncoding.base16().decode(defaultValue.textValue().toUpperCase(Locale.ROOT)); return ByteBuffer.allocate(((Types.FixedType) type).length()).put(fixedBytes); case BINARY: - byte[] binaryBytes = - BaseEncoding.base16().decode(defaultValue.textValue().toUpperCase(Locale.ROOT).replaceFirst( - "^0X", "")); + Preconditions.checkArgument(defaultValue.isTextual(), + "Cannot parse %s to a %s value", defaultValue, type); + byte[] binaryBytes = BaseEncoding.base16().decode(defaultValue.textValue().toUpperCase(Locale.ROOT)); return ByteBuffer.wrap(binaryBytes); case LIST: + Preconditions.checkArgument(defaultValue.isArray(), + "Cannot parse %s to a %s value", defaultValue, type); List defaultList = Lists.newArrayList(); for (JsonNode element : defaultValue) { defaultList.add(parseDefaultFromJson(type.asListType().elementType(), element)); } return defaultList; case MAP: + Preconditions.checkArgument( + defaultValue.isObject() && defaultValue.has("keys") && defaultValue.has("values") && + defaultValue.get("keys").isArray() && defaultValue.get("values").isArray(), + "Cannot parse %s to a %s value", + defaultValue, + type); Map defaultMap = Maps.newHashMap(); - List keysAndValues = StreamSupport - .stream(defaultValue.spliterator(), false) - .collect(Collectors.toList()); - JsonNode keys = keysAndValues.get(0); - JsonNode values = keysAndValues.get(1); - + JsonNode keys = defaultValue.get("keys"); + JsonNode values = defaultValue.get("values"); List keyList = Lists.newArrayList(keys.iterator()); List valueList = Lists.newArrayList(values.iterator()); @@ -100,6 +125,8 @@ public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) { } return defaultMap; case STRUCT: + Preconditions.checkArgument(defaultValue.isObject(), + "Cannot parse %s to a %s value", defaultValue, type); Map defaultStruct = Maps.newHashMap(); for (Types.NestedField subField : type.asStructType().fields()) { String fieldIdAsString = String.valueOf(subField.fieldId()); @@ -116,92 +143,6 @@ public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) { } } - public static JsonNode validateDefault(Type type, JsonNode defaultValue) { - if (defaultValue != null && !isValidDefault(type, defaultValue)) { - throw new ValidationException("Invalid default value for type %s: %s not a %s", type, defaultValue, type); - } - return defaultValue; - } - - @SuppressWarnings("checkstyle:CyclomaticComplexity") - public static boolean isValidDefault(Type type, JsonNode defaultValue) { - if (defaultValue == null) { - return false; - } - switch (type.typeId()) { - case BOOLEAN: - return defaultValue.isBoolean(); - case INTEGER: - return defaultValue.isIntegralNumber() && defaultValue.canConvertToInt(); - case LONG: - return defaultValue.isIntegralNumber() && defaultValue.canConvertToLong(); - case FLOAT: - case DOUBLE: - case DECIMAL: - return defaultValue.isNumber(); - case STRING: - case UUID: - case DATE: - case TIME: - case TIMESTAMP: - return defaultValue.isTextual(); - case FIXED: - case BINARY: - return defaultValue.isTextual() && - (defaultValue.textValue().startsWith("0x") || defaultValue.textValue().startsWith("0X")); - case LIST: - if (!defaultValue.isArray()) { - return false; - } - for (JsonNode element : defaultValue) { - if (!isValidDefault(type.asListType().elementType(), element)) { - return false; - } - } - return true; - case MAP: - if (!defaultValue.isArray()) { - return false; - } - List keysAndValues = StreamSupport - .stream(defaultValue.spliterator(), false) - .collect(Collectors.toList()); - if (keysAndValues.size() != 2) { - return false; - } - JsonNode keys = keysAndValues.get(0); - JsonNode values = keysAndValues.get(1); - if (!keys.isArray() || !values.isArray()) { - return false; - } - List keyList = Lists.newArrayList(keys.iterator()); - List valueList = Lists.newArrayList(values.iterator()); - if (keyList.size() != valueList.size()) { - return false; - } - for (int i = 0; i < keyList.size(); i++) { - if (!isValidDefault(type.asMapType().keyType(), keyList.get(i)) || - !isValidDefault(type.asMapType().valueType(), valueList.get(i))) { - return false; - } - } - return true; - case STRUCT: - if (!defaultValue.isObject()) { - return false; - } - for (Types.NestedField subType : type.asStructType().fields()) { - String fieldId = String.valueOf(subType.fieldId()); - if (!isValidDefault(subType.type(), defaultValue.has(fieldId) ? defaultValue.get(fieldId) : null)) { - return false; - } - } - return true; - default: - return false; - } - } - public static Object convertJavaDefaultForSerialization(Type type, Object value) { switch (type.typeId()) { case DATE: @@ -212,7 +153,7 @@ public static Object convertJavaDefaultForSerialization(Type type, Object value) return DateTimeUtil.formatEpochTimeMicros((long) value, ((Types.TimestampType) type).shouldAdjustToUTC()); case FIXED: case BINARY: - return "0X" + BaseEncoding.base16().encode(((ByteBuffer) value).array()); + return BaseEncoding.base16().encode(((ByteBuffer) value).array()); case LIST: List defaultList = (List) value; List convertedList = Lists.newArrayListWithExpectedSize(defaultList.size()); @@ -222,15 +163,15 @@ public static Object convertJavaDefaultForSerialization(Type type, Object value) return convertedList; case MAP: Map defaultMap = (Map) value; - List> convertedDefault = Lists.newArrayListWithExpectedSize(2); + Map> convertedDefault = Maps.newHashMapWithExpectedSize(2); List keyList = Lists.newArrayListWithExpectedSize(defaultMap.size()); List valueList = Lists.newArrayListWithExpectedSize(defaultMap.size()); for (Map.Entry entry : defaultMap.entrySet()) { keyList.add(convertJavaDefaultForSerialization(type.asMapType().keyType(), entry.getKey())); valueList.add(convertJavaDefaultForSerialization(type.asMapType().valueType(), entry.getValue())); } - convertedDefault.add(keyList); - convertedDefault.add(valueList); + convertedDefault.put("keys", keyList); + convertedDefault.put("values", valueList); return convertedDefault; case STRUCT: Map defaultStruct = (Map) value; diff --git a/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java b/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java index 082c9b8ad629..bf069800fd2c 100644 --- a/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java +++ b/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java @@ -62,12 +62,12 @@ public static Collection data() { {Types.TimestampType.withZone(), stringToJsonNode("\"2007-12-03T10:15:30+01:00\"")}, {Types.StringType.get(), stringToJsonNode("\"foo\"")}, {Types.UUIDType.get(), stringToJsonNode("\"eb26bdb1-a1d8-4aa6-990e-da940875492c\"")}, - {Types.FixedType.ofLength(2), stringToJsonNode("\"0x111f\"")}, - {Types.BinaryType.get(), stringToJsonNode("\"0x0000ff\"")}, + {Types.FixedType.ofLength(2), stringToJsonNode("\"111f\"")}, + {Types.BinaryType.get(), stringToJsonNode("\"0000ff\"")}, {Types.DecimalType.of(9, 2), stringToJsonNode("123.45")}, {Types.ListType.ofOptional(1, Types.IntegerType.get()), stringToJsonNode("[1, 2, 3]")}, {Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()), - stringToJsonNode("[[1,2], [\"foo\", \"bar\"]]")}, + stringToJsonNode("{\"keys\": [1, 2], \"values\": [\"foo\", \"bar\"]}")}, {Types.StructType.of( required(1, "f1", Types.IntegerType.get(), "doc"), optional(2, "f2", Types.StringType.get(), "doc")), @@ -95,7 +95,6 @@ private static String defaultValueParseAndUnParseRoundTrip(Type type, JsonNode d @Test public void testTypeWithDefaultValue() throws JsonProcessingException { - Assert.assertTrue(DefaultValueParser.isValidDefault(type, defaultValue)); String parseThenUnParseDefaultValue = defaultValueParseAndUnParseRoundTrip(type, defaultValue); // Only if the type is a timestampWithZone type, the round-trip default value will always be standardized to the // UTC time zone, which might be different in the original value, but they should represent the same instant From c5970b5bdda7d4551f1ad6c25e6a30748f9dabd8 Mon Sep 17 00:00:00 2001 From: Limian Zhang <75445426+rzhang10@users.noreply.github.com> Date: Thu, 9 Jun 2022 16:12:11 -0700 Subject: [PATCH 03/23] Refactor code against comments --- .../apache/iceberg/DefaultValueParser.java | 177 ++++++++++++------ .../org/apache/iceberg/util/DateTimeUtil.java | 27 ++- .../TestDefaultValuesParsingAndUnParsing.java | 95 +++++----- 3 files changed, 186 insertions(+), 113 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index 31d47c3eb84e..f658d47974b8 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -20,26 +20,33 @@ package org.apache.iceberg; import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.math.BigDecimal; import java.nio.ByteBuffer; +import java.util.Iterator; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.UUID; -import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.JsonUtil; public class DefaultValueParser { private DefaultValueParser() { } @SuppressWarnings("checkstyle:CyclomaticComplexity") - public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) { + public static Object fromJson(Type type, JsonNode defaultValue) { if (defaultValue == null || defaultValue.isNull()) { return null; @@ -48,121 +55,164 @@ public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) { switch (type.typeId()) { case BOOLEAN: Preconditions.checkArgument(defaultValue.isBoolean(), - "Cannot parse %s to a %s value", defaultValue, type); + "Cannot parse default as a %s value: %s", type, defaultValue); return defaultValue.booleanValue(); case INTEGER: Preconditions.checkArgument(defaultValue.isIntegralNumber() && defaultValue.canConvertToInt(), - "Cannot parse %s to a %s value", defaultValue, type); + "Cannot parse default as a %s value: %s", type, defaultValue); return defaultValue.intValue(); case LONG: Preconditions.checkArgument(defaultValue.isIntegralNumber() && defaultValue.canConvertToLong(), - "Cannot parse %s to a %s value", defaultValue, type); + "Cannot parse default as a %s value: %s", type, defaultValue); return defaultValue.longValue(); case FLOAT: - Preconditions.checkArgument(defaultValue.isNumber(), - "Cannot parse %s to a %s value", defaultValue, type); + Preconditions.checkArgument(defaultValue.isFloatingPointNumber(), + "Cannot parse default as a %s value: %s", type, defaultValue); return defaultValue.floatValue(); case DOUBLE: Preconditions.checkArgument(defaultValue.isNumber(), - "Cannot parse %s to a %s value", defaultValue, type); + "Cannot parse default as a %s value: %s", type, defaultValue); return defaultValue.doubleValue(); case DECIMAL: Preconditions.checkArgument(defaultValue.isNumber(), - "Cannot parse %s to a %s value", defaultValue, type); - return defaultValue.decimalValue(); + "Cannot parse default as a %s value: %s", type, defaultValue); + BigDecimal retDecimal = defaultValue.decimalValue(); + Preconditions.checkArgument( + retDecimal.scale() == ((Types.DecimalType) type).scale(), "Cannot parse default as a %s value: %s", + type, + defaultValue); + return retDecimal; case STRING: Preconditions.checkArgument(defaultValue.isTextual(), - "Cannot parse %s to a %s value", defaultValue, type); + "Cannot parse default as a %s value: %s", type, defaultValue); return defaultValue.textValue(); case UUID: Preconditions.checkArgument(defaultValue.isTextual(), - "Cannot parse %s to a %s value", defaultValue, type); + "Cannot parse default as a %s value: %s", type, defaultValue); + try { + UUID.fromString(defaultValue.textValue()); + } catch (IllegalArgumentException e) { + Preconditions.checkArgument(false, "Cannot parse default as a %s value: %s", type, defaultValue); + } return UUID.fromString(defaultValue.textValue()); case DATE: + Preconditions.checkArgument(defaultValue.isTextual(), + "Cannot parse default as a %s value: %s", type, defaultValue); + return DateTimeUtil.daysFromISODateString(defaultValue.textValue()); case TIME: + Preconditions.checkArgument(defaultValue.isTextual(), + "Cannot parse default as a %s value: %s", type, defaultValue); + return DateTimeUtil.microsFromISOTimeString(defaultValue.textValue()); case TIMESTAMP: Preconditions.checkArgument(defaultValue.isTextual(), - "Cannot parse %s to a %s value", defaultValue, type); - return Literal.of(defaultValue.textValue()).to(type).value(); + "Cannot parse default as a %s value: %s", type, defaultValue); + if (((Types.TimestampType) type).shouldAdjustToUTC()) { + return DateTimeUtil.microsFromISOOffsetTsString(defaultValue.textValue()); + } else { + return DateTimeUtil.microsFromISOTsString(defaultValue.textValue()); + } case FIXED: Preconditions.checkArgument( - defaultValue.isTextual() && defaultValue.textValue().length() == ((Types.FixedType) type).length() * 2, - "Cannot parse %s to a %s value", - defaultValue, - type); + defaultValue.isTextual(), + "Cannot parse default as a %s value: %s", type, defaultValue); + int defaultLength = defaultValue.textValue().length(); + int fixedLength = ((Types.FixedType) type).length(); + Preconditions.checkArgument(defaultLength == fixedLength * 2, + "Default value %s is not compatible with the expected fixed type, the fixed type is expected to store " + + "exactly %s bytes, which means the default value should be of exactly 2 * %s length hex string", + defaultValue, fixedLength, fixedLength); byte[] fixedBytes = BaseEncoding.base16().decode(defaultValue.textValue().toUpperCase(Locale.ROOT)); - return ByteBuffer.allocate(((Types.FixedType) type).length()).put(fixedBytes); + return ByteBuffer.wrap(fixedBytes); case BINARY: Preconditions.checkArgument(defaultValue.isTextual(), - "Cannot parse %s to a %s value", defaultValue, type); + "Cannot parse default as a %s value: %s", type, defaultValue); byte[] binaryBytes = BaseEncoding.base16().decode(defaultValue.textValue().toUpperCase(Locale.ROOT)); return ByteBuffer.wrap(binaryBytes); case LIST: Preconditions.checkArgument(defaultValue.isArray(), - "Cannot parse %s to a %s value", defaultValue, type); - List defaultList = Lists.newArrayList(); - for (JsonNode element : defaultValue) { - defaultList.add(parseDefaultFromJson(type.asListType().elementType(), element)); - } - return defaultList; + "Cannot parse default as a %s value: %s", type, defaultValue); + Type elementType = type.asListType().elementType(); + return Lists.newArrayList(Iterables.transform(defaultValue, e -> fromJson(elementType, e))); case MAP: Preconditions.checkArgument( defaultValue.isObject() && defaultValue.has("keys") && defaultValue.has("values") && defaultValue.get("keys").isArray() && defaultValue.get("values").isArray(), "Cannot parse %s to a %s value", - defaultValue, - type); - Map defaultMap = Maps.newHashMap(); + defaultValue, type); JsonNode keys = defaultValue.get("keys"); JsonNode values = defaultValue.get("values"); - List keyList = Lists.newArrayList(keys.iterator()); - List valueList = Lists.newArrayList(values.iterator()); + Preconditions.checkArgument(keys.size() == values.size(), "Cannot parse default as a %s value: %s", type, + defaultValue); + + ImmutableMap.Builder mapBuilder = ImmutableMap.builder(); + + Iterator keyIter = keys.iterator(); + Type keyType = type.asMapType().keyType(); + Iterator valueIter = values.iterator(); + Type valueType = type.asMapType().valueType(); - for (int i = 0; i < keyList.size(); i++) { - defaultMap.put( - parseDefaultFromJson(type.asMapType().keyType(), keyList.get(i)), - parseDefaultFromJson(type.asMapType().valueType(), valueList.get(i))); + while (keyIter.hasNext()) { + mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next())); } - return defaultMap; + + return mapBuilder.build(); case STRUCT: Preconditions.checkArgument(defaultValue.isObject(), - "Cannot parse %s to a %s value", defaultValue, type); - Map defaultStruct = Maps.newHashMap(); - for (Types.NestedField subField : type.asStructType().fields()) { - String fieldIdAsString = String.valueOf(subField.fieldId()); - Object value = defaultValue.has(fieldIdAsString) ? parseDefaultFromJson( - subField.type(), - defaultValue.get(fieldIdAsString)) : null; - if (value != null) { - defaultStruct.put(subField.fieldId(), value); + "Cannot parse default as a %s value: %s", type, defaultValue); + Types.StructType struct = type.asStructType(); + StructLike defaultRecord = GenericRecord.create(struct); + + List fields = struct.fields(); + for (int pos = 0; pos < fields.size(); pos += 1) { + Types.NestedField field = fields.get(pos); + String idString = String.valueOf(field.fieldId()); + if (defaultValue.has(idString)) { + defaultRecord.set(pos, fromJson(field.type(), defaultValue.get(idString))); } } - return defaultStruct; + return defaultRecord; default: - return null; + throw new IllegalArgumentException(String.format("Type: %s is not supported", type)); + } + } + + public static Object fromJson(Type type, String defaultValue) { + try { + JsonNode defaultValueJN = JsonUtil.mapper().readTree(defaultValue); + return fromJson(type, defaultValueJN); + } catch (IOException e) { + throw new IllegalArgumentException("Failed to parse: " + defaultValue + "; reason: " + e.getMessage(), e); } } - public static Object convertJavaDefaultForSerialization(Type type, Object value) { + public static String toJson(Type type, Object javaDefaultValue) throws IOException { + return JsonUtil.mapper().writeValueAsString(DefaultValueParser.convertJavaDefaultForSerialization( + type, + javaDefaultValue)); + } + + private static Object convertJavaDefaultForSerialization(Type type, Object javaDefaultValue) { switch (type.typeId()) { case DATE: - return DateTimeUtil.formatEpochDays((int) value); + return DateTimeUtil.formatEpochDays((int) javaDefaultValue); case TIME: - return DateTimeUtil.formatTimeOfDayMicros((long) value); + return DateTimeUtil.formatTimeOfDayMicros((long) javaDefaultValue); case TIMESTAMP: - return DateTimeUtil.formatEpochTimeMicros((long) value, ((Types.TimestampType) type).shouldAdjustToUTC()); + return DateTimeUtil.formatEpochTimeMicros( + (long) javaDefaultValue, + ((Types.TimestampType) type).shouldAdjustToUTC()); case FIXED: case BINARY: - return BaseEncoding.base16().encode(((ByteBuffer) value).array()); + return BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) javaDefaultValue))); case LIST: - List defaultList = (List) value; + List defaultList = (List) javaDefaultValue; List convertedList = Lists.newArrayListWithExpectedSize(defaultList.size()); for (Object element : defaultList) { convertedList.add(convertJavaDefaultForSerialization(type.asListType().elementType(), element)); } return convertedList; case MAP: - Map defaultMap = (Map) value; + Map defaultMap = (Map) javaDefaultValue; Map> convertedDefault = Maps.newHashMapWithExpectedSize(2); List keyList = Lists.newArrayListWithExpectedSize(defaultMap.size()); List valueList = Lists.newArrayListWithExpectedSize(defaultMap.size()); @@ -174,17 +224,22 @@ public static Object convertJavaDefaultForSerialization(Type type, Object value) convertedDefault.put("values", valueList); return convertedDefault; case STRUCT: - Map defaultStruct = (Map) value; + Types.StructType structType = type.asStructType(); + List fields = structType.fields(); + StructLike defaultStruct = (StructLike) javaDefaultValue; Map convertedStruct = Maps.newHashMap(); - for (Map.Entry entry : defaultStruct.entrySet()) { - int fieldId = entry.getKey(); - convertedStruct.put(fieldId, convertJavaDefaultForSerialization( - type.asStructType().field(fieldId).type(), - entry.getValue())); + + for (int i = 0; i < defaultStruct.size(); i++) { + Types.NestedField field = fields.get(i); + int fieldId = field.fieldId(); + Object fieldJavaDefaultValue = defaultStruct.get(i, Object.class); + if (fieldJavaDefaultValue != null) { + convertedStruct.put(fieldId, convertJavaDefaultForSerialization(field.type(), fieldJavaDefaultValue)); + } } return convertedStruct; default: - return value; + return javaDefaultValue; } } } diff --git a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java index 33b7b05eb394..404a5fd5582f 100644 --- a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java @@ -93,15 +93,28 @@ public static String formatTimeOfDayMicros(long micros) { } public static String formatEpochTimeMicros(long micros, boolean withUTCZone) { - String localDateTime = LocalDateTime.ofEpochSecond(micros / 1000000, - (int) (micros % 1000000) * 1000, ZoneOffset.UTC).format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); + LocalDateTime localDateTime = LocalDateTime.ofEpochSecond(micros / 1000000, + (int) (micros % 1000000) * 1000, ZoneOffset.UTC); if (withUTCZone) { - // We standardize the format by always using the UTC zone - return LocalDateTime.parse(localDateTime, DateTimeFormatter.ISO_LOCAL_DATE_TIME) - .atOffset(ZoneOffset.UTC) - .format(DateTimeFormatter.ISO_OFFSET_DATE_TIME); + return OffsetDateTime.of(localDateTime, ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME); } else { - return localDateTime; + return localDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); } } + + public static int daysFromISODateString(String dateString) { + return daysFromDate(LocalDate.parse(dateString, DateTimeFormatter.ISO_LOCAL_DATE)); + } + + public static long microsFromISOTimeString(String timeString) { + return microsFromTime(LocalTime.parse(timeString, DateTimeFormatter.ISO_LOCAL_TIME)); + } + + public static long microsFromISOOffsetTsString(String timestampString) { + return microsFromTimestamptz(OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME)); + } + + public static long microsFromISOTsString(String timestampString) { + return microsFromTimestamp(LocalDateTime.parse(timestampString, DateTimeFormatter.ISO_LOCAL_DATE_TIME)); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java b/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java index bf069800fd2c..03181fc69ec0 100644 --- a/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java +++ b/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java @@ -19,9 +19,7 @@ package org.apache.iceberg; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; import java.time.OffsetDateTime; import java.util.Arrays; import java.util.Collection; @@ -41,9 +39,9 @@ public class TestDefaultValuesParsingAndUnParsing { private final Type type; - private final JsonNode defaultValue; + private final String defaultValue; - public TestDefaultValuesParsingAndUnParsing(Type type, JsonNode defaultValue) { + public TestDefaultValuesParsingAndUnParsing(Type type, String defaultValue) { this.type = type; this.defaultValue = defaultValue; } @@ -51,61 +49,68 @@ public TestDefaultValuesParsingAndUnParsing(Type type, JsonNode defaultValue) { @Parameterized.Parameters public static Collection data() { return Arrays.asList(new Object[][] { - {Types.BooleanType.get(), stringToJsonNode("true")}, - {Types.IntegerType.get(), stringToJsonNode("1")}, - {Types.LongType.get(), stringToJsonNode("9999999")}, - {Types.FloatType.get(), stringToJsonNode("1.23")}, - {Types.DoubleType.get(), stringToJsonNode("123.456")}, - {Types.DateType.get(), stringToJsonNode("\"2007-12-03\"")}, - {Types.TimeType.get(), stringToJsonNode("\"10:15:30\"")}, - {Types.TimestampType.withoutZone(), stringToJsonNode("\"2007-12-03T10:15:30\"")}, - {Types.TimestampType.withZone(), stringToJsonNode("\"2007-12-03T10:15:30+01:00\"")}, - {Types.StringType.get(), stringToJsonNode("\"foo\"")}, - {Types.UUIDType.get(), stringToJsonNode("\"eb26bdb1-a1d8-4aa6-990e-da940875492c\"")}, - {Types.FixedType.ofLength(2), stringToJsonNode("\"111f\"")}, - {Types.BinaryType.get(), stringToJsonNode("\"0000ff\"")}, - {Types.DecimalType.of(9, 2), stringToJsonNode("123.45")}, - {Types.ListType.ofOptional(1, Types.IntegerType.get()), stringToJsonNode("[1, 2, 3]")}, + {Types.BooleanType.get(), "true"}, + {Types.IntegerType.get(), "1"}, + {Types.LongType.get(), "9999999"}, + {Types.FloatType.get(), "1.23"}, + {Types.DoubleType.get(), "123.456"}, + {Types.DateType.get(), "\"2007-12-03\""}, + {Types.TimeType.get(), "\"10:15:30\""}, + {Types.TimestampType.withoutZone(), "\"2007-12-03T10:15:30\""}, + {Types.TimestampType.withZone(), "\"2007-12-03T10:15:30+01:00\""}, + {Types.StringType.get(), "\"foo\""}, + {Types.UUIDType.get(), "\"eb26bdb1-a1d8-4aa6-990e-da940875492c\""}, + {Types.FixedType.ofLength(2), "\"111f\""}, + {Types.BinaryType.get(), "\"0000ff\""}, + {Types.DecimalType.of(9, 2), "123.45"}, + {Types.ListType.ofOptional(1, Types.IntegerType.get()), "[1, 2, 3]"}, {Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()), - stringToJsonNode("{\"keys\": [1, 2], \"values\": [\"foo\", \"bar\"]}")}, + "{\"keys\": [1, 2], \"values\": [\"foo\", \"bar\"]}"}, {Types.StructType.of( required(1, "f1", Types.IntegerType.get(), "doc"), optional(2, "f2", Types.StringType.get(), "doc")), - stringToJsonNode("{\"1\": 1, \"2\": \"bar\"}")} + "{\"1\": 1, \"2\": \"bar\"}"}, + // deeply nested complex types + {Types.ListType.ofOptional(1, Types.StructType.of( + required(1, "f1", Types.IntegerType.get(), "doc"), + optional(2, "f2", Types.StringType.get(), "doc"))), "[{\"1\": 1, \"2\": \"bar\"}, {\"1\": 2, \"2\": " + + "\"foo\"}]"}, + {Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StructType.of( + required(1, "f1", Types.IntegerType.get(), "doc"), + optional(2, "f2", Types.StringType.get(), "doc"))), + "{\"keys\": [1, 2], \"values\": [{\"1\": 1, \"2\": \"bar\"}, {\"1\": 2, \"2\": \"foo\"}]}"}, + {Types.StructType.of( + required(1, "f1", Types.StructType.of( + optional(2, "ff1", Types.IntegerType.get(), "doc"), + optional(3, "ff2", Types.StringType.get(), "doc")), "doc"), + optional(4, "f2", Types.StructType.of( + optional(5, "ff1", Types.StringType.get(), "doc"), + optional(6, "ff2", Types.IntegerType.get(), "doc")), "doc")), + "{\"1\": {\"2\": 1, \"3\": \"bar\"}, \"4\": {\"5\": \"bar\", \"6\": 1}}"}, }); } - private static JsonNode stringToJsonNode(String json) { - try { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readTree(json); - } catch (JsonProcessingException e) { - throw new RuntimeException("Failed to parse: " + json + "; reason: " + e.getMessage(), e); - } - } - // serialize to json and deserialize back should return the same result - private static String defaultValueParseAndUnParseRoundTrip(Type type, JsonNode defaultValue) - throws JsonProcessingException { - Object javaDefaultValue = DefaultValueParser.parseDefaultFromJson(type, defaultValue); - String jsonDefaultValue = JsonUtil.mapper() - .writeValueAsString(DefaultValueParser.convertJavaDefaultForSerialization(type, javaDefaultValue)); - return jsonDefaultValue; + private static String defaultValueParseAndUnParseRoundTrip(Type type, String defaultValue) throws IOException { + Object javaDefaultValue = DefaultValueParser.fromJson(type, defaultValue); + return DefaultValueParser.toJson(type, javaDefaultValue); } @Test - public void testTypeWithDefaultValue() throws JsonProcessingException { - String parseThenUnParseDefaultValue = defaultValueParseAndUnParseRoundTrip(type, defaultValue); + public void testTypeWithDefaultValue() throws IOException { + String roundTripDefaultValue = defaultValueParseAndUnParseRoundTrip(type, defaultValue); // Only if the type is a timestampWithZone type, the round-trip default value will always be standardized to the // UTC time zone, which might be different in the original value, but they should represent the same instant if (type.typeId() == Type.TypeID.TIMESTAMP && ((Types.TimestampType) type).shouldAdjustToUTC()) { - Assert.assertTrue( - OffsetDateTime.parse(defaultValue.textValue()).isEqual( - OffsetDateTime.parse(stringToJsonNode(parseThenUnParseDefaultValue).textValue()))); + Assert.assertTrue(OffsetDateTime.parse(JsonUtil.mapper().readTree(defaultValue).textValue()) + .isEqual(OffsetDateTime.parse(JsonUtil.mapper().readTree(roundTripDefaultValue).textValue()))); } else { - Assert.assertEquals( - defaultValue.toString().toLowerCase(Locale.ROOT), - parseThenUnParseDefaultValue.toLowerCase(Locale.ROOT)); + jsonStringEquals(defaultValue.toLowerCase(Locale.ROOT), roundTripDefaultValue.toLowerCase(Locale.ROOT)); + System.out.println(roundTripDefaultValue); } } + + private static void jsonStringEquals(String s1, String s2) throws IOException { + Assert.assertEquals(JsonUtil.mapper().readTree(s1), JsonUtil.mapper().readTree(s2)); + } } From bcc83509d622e699f99a7c578cf7f99e4910404d Mon Sep 17 00:00:00 2001 From: Limian Zhang <75445426+rzhang10@users.noreply.github.com> Date: Thu, 9 Jun 2022 17:22:07 -0700 Subject: [PATCH 04/23] Refactor toJson to use JsonGenerator --- .../apache/iceberg/DefaultValueParser.java | 111 ++++++++++++++---- .../TestDefaultValuesParsingAndUnParsing.java | 5 +- 2 files changed, 89 insertions(+), 27 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index f658d47974b8..9df45aca781d 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -19,8 +19,10 @@ package org.apache.iceberg; +import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.JsonNode; import java.io.IOException; +import java.io.StringWriter; import java.math.BigDecimal; import java.nio.ByteBuffer; import java.util.Iterator; @@ -29,6 +31,7 @@ import java.util.Map; import java.util.UUID; import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -185,61 +188,121 @@ public static Object fromJson(Type type, String defaultValue) { } } - public static String toJson(Type type, Object javaDefaultValue) throws IOException { - return JsonUtil.mapper().writeValueAsString(DefaultValueParser.convertJavaDefaultForSerialization( - type, - javaDefaultValue)); + public static String toJson(Type type, Object defaultValue) { + return toJson(type, defaultValue, false); } - private static Object convertJavaDefaultForSerialization(Type type, Object javaDefaultValue) { + public static String toJson(Type type, Object defaultValue, boolean pretty) { + try { + StringWriter writer = new StringWriter(); + JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + if (pretty) { + generator.useDefaultPrettyPrinter(); + } + toJson(type, defaultValue, generator); + generator.flush(); + return writer.toString(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public static void toJson(Type type, Object javaDefaultValue, JsonGenerator generator) + throws IOException { switch (type.typeId()) { + case BOOLEAN: + generator.writeBoolean((boolean) javaDefaultValue); + break; + case INTEGER: + generator.writeNumber((int) javaDefaultValue); + break; + case LONG: + generator.writeNumber((long) javaDefaultValue); + break; + case FLOAT: + generator.writeNumber((float) javaDefaultValue); + break; + case DOUBLE: + generator.writeNumber((double) javaDefaultValue); + break; case DATE: - return DateTimeUtil.formatEpochDays((int) javaDefaultValue); + generator.writeString(DateTimeUtil.formatEpochDays((int) javaDefaultValue)); + break; case TIME: - return DateTimeUtil.formatTimeOfDayMicros((long) javaDefaultValue); + generator.writeString(DateTimeUtil.formatTimeOfDayMicros((long) javaDefaultValue)); + break; case TIMESTAMP: - return DateTimeUtil.formatEpochTimeMicros( + generator.writeString(DateTimeUtil.formatEpochTimeMicros( (long) javaDefaultValue, - ((Types.TimestampType) type).shouldAdjustToUTC()); + ((Types.TimestampType) type).shouldAdjustToUTC())); + break; + case STRING: + generator.writeString((String) javaDefaultValue); + break; + case UUID: + generator.writeString(javaDefaultValue.toString()); + break; case FIXED: case BINARY: - return BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) javaDefaultValue))); + generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) javaDefaultValue)))); + break; + case DECIMAL: + generator.writeNumber((BigDecimal) javaDefaultValue); + break; case LIST: List defaultList = (List) javaDefaultValue; - List convertedList = Lists.newArrayListWithExpectedSize(defaultList.size()); - for (Object element : defaultList) { - convertedList.add(convertJavaDefaultForSerialization(type.asListType().elementType(), element)); + Type elementType = type.asListType().elementType(); + generator.writeStartArray(); + for (Object elementDefault : defaultList) { + toJson(elementType, elementDefault, generator); } - return convertedList; + generator.writeEndArray(); + break; case MAP: Map defaultMap = (Map) javaDefaultValue; - Map> convertedDefault = Maps.newHashMapWithExpectedSize(2); List keyList = Lists.newArrayListWithExpectedSize(defaultMap.size()); List valueList = Lists.newArrayListWithExpectedSize(defaultMap.size()); + Type keyType = type.asMapType().keyType(); + Type valueType = type.asMapType().valueType(); + for (Map.Entry entry : defaultMap.entrySet()) { - keyList.add(convertJavaDefaultForSerialization(type.asMapType().keyType(), entry.getKey())); - valueList.add(convertJavaDefaultForSerialization(type.asMapType().valueType(), entry.getValue())); + keyList.add(entry.getKey()); + valueList.add(entry.getValue()); + } + generator.writeStartObject(); + generator.writeFieldName("keys"); + generator.writeStartArray(); + for (Object key : keyList) { + toJson(keyType, key, generator); } - convertedDefault.put("keys", keyList); - convertedDefault.put("values", valueList); - return convertedDefault; + generator.writeEndArray(); + generator.writeFieldName("values"); + generator.writeStartArray(); + for (Object value : valueList) { + toJson(valueType, value, generator); + } + generator.writeEndArray(); + generator.writeEndObject(); + break; case STRUCT: Types.StructType structType = type.asStructType(); List fields = structType.fields(); StructLike defaultStruct = (StructLike) javaDefaultValue; - Map convertedStruct = Maps.newHashMap(); + generator.writeStartObject(); for (int i = 0; i < defaultStruct.size(); i++) { Types.NestedField field = fields.get(i); int fieldId = field.fieldId(); Object fieldJavaDefaultValue = defaultStruct.get(i, Object.class); if (fieldJavaDefaultValue != null) { - convertedStruct.put(fieldId, convertJavaDefaultForSerialization(field.type(), fieldJavaDefaultValue)); + generator.writeFieldName(String.valueOf(fieldId)); + toJson(field.type(), fieldJavaDefaultValue, generator); } } - return convertedStruct; + generator.writeEndObject(); + break; default: - return javaDefaultValue; + throw new IllegalArgumentException(String.format("Type: %s is not supported", type)); } } } diff --git a/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java b/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java index 03181fc69ec0..7342d0f40600 100644 --- a/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java +++ b/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java @@ -87,7 +87,7 @@ public static Collection data() { optional(5, "ff1", Types.StringType.get(), "doc"), optional(6, "ff2", Types.IntegerType.get(), "doc")), "doc")), "{\"1\": {\"2\": 1, \"3\": \"bar\"}, \"4\": {\"5\": \"bar\", \"6\": 1}}"}, - }); + }); } // serialize to json and deserialize back should return the same result @@ -103,10 +103,9 @@ public void testTypeWithDefaultValue() throws IOException { // UTC time zone, which might be different in the original value, but they should represent the same instant if (type.typeId() == Type.TypeID.TIMESTAMP && ((Types.TimestampType) type).shouldAdjustToUTC()) { Assert.assertTrue(OffsetDateTime.parse(JsonUtil.mapper().readTree(defaultValue).textValue()) - .isEqual(OffsetDateTime.parse(JsonUtil.mapper().readTree(roundTripDefaultValue).textValue()))); + .isEqual(OffsetDateTime.parse(JsonUtil.mapper().readTree(roundTripDefaultValue).textValue()))); } else { jsonStringEquals(defaultValue.toLowerCase(Locale.ROOT), roundTripDefaultValue.toLowerCase(Locale.ROOT)); - System.out.println(roundTripDefaultValue); } } From e3c8f469942ca3842b4001815c8cbb689e522a62 Mon Sep 17 00:00:00 2001 From: Limian Zhang <75445426+rzhang10@users.noreply.github.com> Date: Thu, 9 Jun 2022 17:40:11 -0700 Subject: [PATCH 05/23] Add TestInvalidDefaultValues --- .../apache/iceberg/DefaultValueParser.java | 2 - .../TestDefaultValuesParsingAndUnParsing.java | 2 +- .../iceberg/TestInvalidDefaultValues.java | 66 +++++++++++++++++++ 3 files changed, 67 insertions(+), 3 deletions(-) create mode 100644 core/src/test/java/org/apache/iceberg/TestInvalidDefaultValues.java diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index 9df45aca781d..5d9d7d88295b 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -31,12 +31,10 @@ import java.util.Map; import java.util.UUID; import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; diff --git a/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java b/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java index 7342d0f40600..8b9fd8b2da1e 100644 --- a/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java +++ b/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java @@ -91,7 +91,7 @@ public static Collection data() { } // serialize to json and deserialize back should return the same result - private static String defaultValueParseAndUnParseRoundTrip(Type type, String defaultValue) throws IOException { + static String defaultValueParseAndUnParseRoundTrip(Type type, String defaultValue) throws IOException { Object javaDefaultValue = DefaultValueParser.fromJson(type, defaultValue); return DefaultValueParser.toJson(type, javaDefaultValue); } diff --git a/core/src/test/java/org/apache/iceberg/TestInvalidDefaultValues.java b/core/src/test/java/org/apache/iceberg/TestInvalidDefaultValues.java new file mode 100644 index 000000000000..1ea5eac9dbc2 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestInvalidDefaultValues.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.iceberg.TestDefaultValuesParsingAndUnParsing.defaultValueParseAndUnParseRoundTrip; + +public class TestInvalidDefaultValues { + + @Test + public void testInvalidFixed() { + Type expectedType = Types.FixedType.ofLength(2); + String defaultJson = "\"111ff\""; + Assert.assertThrows(IllegalArgumentException.class, () -> defaultValueParseAndUnParseRoundTrip( + expectedType, + defaultJson)); + } + + @Test + public void testInvalidUUID() { + Type expectedType = Types.FixedType.ofLength(2); + String defaultJson = "\"eb26bdb1-a1d8-4aa6-990e-da940875492c-abcde\""; + Assert.assertThrows(IllegalArgumentException.class, () -> defaultValueParseAndUnParseRoundTrip( + expectedType, + defaultJson)); + } + + @Test + public void testInvalidMap() { + Type expectedType = Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()); + String defaultJson = "{\"keys\": [1, 2, 3], \"values\": [\"foo\", \"bar\"]}"; + Assert.assertThrows(IllegalArgumentException.class, () -> defaultValueParseAndUnParseRoundTrip( + expectedType, + defaultJson)); + } + + @Test + public void testInvalidDecimal() { + Type expectedType = Types.DecimalType.of(5, 2); + String defaultJson = "123.456"; + Assert.assertThrows(IllegalArgumentException.class, () -> defaultValueParseAndUnParseRoundTrip( + expectedType, + defaultJson)); + } +} From ac34d57af58a3a0e1374e361601cd5d0355ebda8 Mon Sep 17 00:00:00 2001 From: Limian Zhang <75445426+rzhang10@users.noreply.github.com> Date: Thu, 9 Jun 2022 18:42:26 -0700 Subject: [PATCH 06/23] Minor change: UUID error message --- core/src/main/java/org/apache/iceberg/DefaultValueParser.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index 5d9d7d88295b..7c8b9bf8296e 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -93,7 +93,8 @@ public static Object fromJson(Type type, JsonNode defaultValue) { try { UUID.fromString(defaultValue.textValue()); } catch (IllegalArgumentException e) { - Preconditions.checkArgument(false, "Cannot parse default as a %s value: %s", type, defaultValue); + throw new IllegalArgumentException(String.format("Cannot parse default as a %s value: %s", type, + defaultValue)); } return UUID.fromString(defaultValue.textValue()); case DATE: From a64937b08dad841c7517f1fd5f3206c0696aa67c Mon Sep 17 00:00:00 2001 From: Limian Zhang <75445426+rzhang10@users.noreply.github.com> Date: Sun, 12 Jun 2022 14:23:24 -0700 Subject: [PATCH 07/23] Add validation in toJson --- .../apache/iceberg/DefaultValueParser.java | 109 +++++++++++++----- .../org/apache/iceberg/util/DateTimeUtil.java | 26 +++-- 2 files changed, 92 insertions(+), 43 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index 7c8b9bf8296e..480d07cd998a 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -46,6 +46,10 @@ public class DefaultValueParser { private DefaultValueParser() { } + private static final String KEYS = "keys"; + + private static final String VALUES = "values"; + @SuppressWarnings("checkstyle:CyclomaticComplexity") public static Object fromJson(Type type, JsonNode defaultValue) { @@ -100,18 +104,18 @@ public static Object fromJson(Type type, JsonNode defaultValue) { case DATE: Preconditions.checkArgument(defaultValue.isTextual(), "Cannot parse default as a %s value: %s", type, defaultValue); - return DateTimeUtil.daysFromISODateString(defaultValue.textValue()); + return DateTimeUtil.isoDateToDays(defaultValue.textValue()); case TIME: Preconditions.checkArgument(defaultValue.isTextual(), "Cannot parse default as a %s value: %s", type, defaultValue); - return DateTimeUtil.microsFromISOTimeString(defaultValue.textValue()); + return DateTimeUtil.isoTimeToMicros(defaultValue.textValue()); case TIMESTAMP: Preconditions.checkArgument(defaultValue.isTextual(), "Cannot parse default as a %s value: %s", type, defaultValue); if (((Types.TimestampType) type).shouldAdjustToUTC()) { - return DateTimeUtil.microsFromISOOffsetTsString(defaultValue.textValue()); + return DateTimeUtil.isoDateTimeTzToMicros(defaultValue.textValue()); } else { - return DateTimeUtil.microsFromISOTsString(defaultValue.textValue()); + return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue()); } case FIXED: Preconditions.checkArgument( @@ -137,13 +141,16 @@ public static Object fromJson(Type type, JsonNode defaultValue) { return Lists.newArrayList(Iterables.transform(defaultValue, e -> fromJson(elementType, e))); case MAP: Preconditions.checkArgument( - defaultValue.isObject() && defaultValue.has("keys") && defaultValue.has("values") && - defaultValue.get("keys").isArray() && defaultValue.get("values").isArray(), + defaultValue.isObject() && defaultValue.has(KEYS) && defaultValue.has(VALUES) && + defaultValue.get(KEYS).isArray() && defaultValue.get(VALUES).isArray(), "Cannot parse %s to a %s value", defaultValue, type); - JsonNode keys = defaultValue.get("keys"); - JsonNode values = defaultValue.get("values"); - Preconditions.checkArgument(keys.size() == values.size(), "Cannot parse default as a %s value: %s", type, + JsonNode keys = defaultValue.get(KEYS); + JsonNode values = defaultValue.get(VALUES); + Preconditions.checkArgument( + keys.size() == values.size(), + "Cannot parse default as a %s value: %s", + type, defaultValue); ImmutableMap.Builder mapBuilder = ImmutableMap.builder(); @@ -206,59 +213,97 @@ public static String toJson(Type type, Object defaultValue, boolean pretty) { } } - public static void toJson(Type type, Object javaDefaultValue, JsonGenerator generator) + @SuppressWarnings("checkstyle:CyclomaticComplexity") + public static void toJson(Type type, Object defaultValue, JsonGenerator generator) throws IOException { switch (type.typeId()) { case BOOLEAN: - generator.writeBoolean((boolean) javaDefaultValue); + Preconditions.checkArgument( + defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue); + generator.writeBoolean((Boolean) defaultValue); break; case INTEGER: - generator.writeNumber((int) javaDefaultValue); + Preconditions.checkArgument( + defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue); + generator.writeNumber((Integer) defaultValue); break; case LONG: - generator.writeNumber((long) javaDefaultValue); + Preconditions.checkArgument(defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue); + generator.writeNumber((Long) defaultValue); break; case FLOAT: - generator.writeNumber((float) javaDefaultValue); + Preconditions.checkArgument(defaultValue instanceof Float, "Invalid default %s value: %s", type, defaultValue); + generator.writeNumber((Float) defaultValue); break; case DOUBLE: - generator.writeNumber((double) javaDefaultValue); + Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue); + generator.writeNumber((Double) defaultValue); break; case DATE: - generator.writeString(DateTimeUtil.formatEpochDays((int) javaDefaultValue)); + Preconditions.checkArgument( + defaultValue instanceof Integer && (Integer) defaultValue >= 0, + "Invalid default %s value: %s", + type, + defaultValue); + generator.writeString(DateTimeUtil.daysToIsoDate((Integer) defaultValue)); break; case TIME: - generator.writeString(DateTimeUtil.formatTimeOfDayMicros((long) javaDefaultValue)); + Preconditions.checkArgument( + defaultValue instanceof Long && (Long) defaultValue >= 0, + "Invalid default %s value: %s", + type, + defaultValue); + generator.writeString(DateTimeUtil.microsToIsoTime((Long) defaultValue)); break; case TIMESTAMP: - generator.writeString(DateTimeUtil.formatEpochTimeMicros( - (long) javaDefaultValue, - ((Types.TimestampType) type).shouldAdjustToUTC())); + Preconditions.checkArgument( + defaultValue instanceof Long && (Long) defaultValue >= 0, + "Invalid default %s value: %s", + type, + defaultValue); + if (((Types.TimestampType) type).shouldAdjustToUTC()) { + generator.writeString(DateTimeUtil.microsToIsoDateTimeTz((Long) defaultValue)); + } else { + generator.writeString(DateTimeUtil.microsToIsoDateTime((Long) defaultValue)); + } break; case STRING: - generator.writeString((String) javaDefaultValue); + Preconditions.checkArgument( + defaultValue instanceof CharSequence, "Invalid default %s value: %s", type, defaultValue); + generator.writeString(((CharSequence) defaultValue).toString()); break; case UUID: - generator.writeString(javaDefaultValue.toString()); + Preconditions.checkArgument(defaultValue instanceof UUID, "Invalid default %s value: %s", type, defaultValue); + generator.writeString(defaultValue.toString()); break; case FIXED: case BINARY: - generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) javaDefaultValue)))); + Preconditions.checkArgument( + defaultValue instanceof ByteBuffer, "Invalid default %s value: %s", type, defaultValue); + generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) defaultValue)))); break; case DECIMAL: - generator.writeNumber((BigDecimal) javaDefaultValue); + Preconditions.checkArgument( + defaultValue instanceof BigDecimal && + ((BigDecimal) defaultValue).scale() == ((Types.DecimalType) type).scale(), + "Invalid default %s value: %s", + type, + defaultValue); + generator.writeNumber((BigDecimal) defaultValue); break; case LIST: - List defaultList = (List) javaDefaultValue; + Preconditions.checkArgument(defaultValue instanceof List, "Invalid default %s value: %s", type, defaultValue); + List defaultList = (List) defaultValue; Type elementType = type.asListType().elementType(); generator.writeStartArray(); - for (Object elementDefault : defaultList) { - toJson(elementType, elementDefault, generator); + for (Object element : defaultList) { + toJson(elementType, element, generator); } generator.writeEndArray(); break; case MAP: - Map defaultMap = (Map) javaDefaultValue; + Preconditions.checkArgument(defaultValue instanceof Map, "Invalid default %s value: %s", type, defaultValue); + Map defaultMap = (Map) defaultValue; List keyList = Lists.newArrayListWithExpectedSize(defaultMap.size()); List valueList = Lists.newArrayListWithExpectedSize(defaultMap.size()); Type keyType = type.asMapType().keyType(); @@ -269,13 +314,13 @@ public static void toJson(Type type, Object javaDefaultValue, JsonGenerator gene valueList.add(entry.getValue()); } generator.writeStartObject(); - generator.writeFieldName("keys"); + generator.writeFieldName(KEYS); generator.writeStartArray(); for (Object key : keyList) { toJson(keyType, key, generator); } generator.writeEndArray(); - generator.writeFieldName("values"); + generator.writeFieldName(VALUES); generator.writeStartArray(); for (Object value : valueList) { toJson(valueType, value, generator); @@ -284,9 +329,11 @@ public static void toJson(Type type, Object javaDefaultValue, JsonGenerator gene generator.writeEndObject(); break; case STRUCT: + Preconditions.checkArgument( + defaultValue instanceof StructLike, "Invalid default %s value: %s", type, defaultValue); Types.StructType structType = type.asStructType(); List fields = structType.fields(); - StructLike defaultStruct = (StructLike) javaDefaultValue; + StructLike defaultStruct = (StructLike) defaultValue; generator.writeStartObject(); for (int i = 0; i < defaultStruct.size(); i++) { diff --git a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java index 404a5fd5582f..6961c736e120 100644 --- a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java @@ -84,37 +84,39 @@ public static String formatTimestampMillis(long millis) { return Instant.ofEpochMilli(millis).toString().replace("Z", "+00:00"); } - public static String formatEpochDays(int days) { + public static String daysToIsoDate(int days) { return LocalDate.ofEpochDay(days).format(DateTimeFormatter.ISO_LOCAL_DATE); } - public static String formatTimeOfDayMicros(long micros) { + public static String microsToIsoTime(long micros) { return LocalTime.ofNanoOfDay(micros * 1000).format(DateTimeFormatter.ISO_LOCAL_TIME); } - public static String formatEpochTimeMicros(long micros, boolean withUTCZone) { + public static String microsToIsoDateTimeTz(long micros) { LocalDateTime localDateTime = LocalDateTime.ofEpochSecond(micros / 1000000, (int) (micros % 1000000) * 1000, ZoneOffset.UTC); - if (withUTCZone) { - return OffsetDateTime.of(localDateTime, ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME); - } else { - return localDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); - } + return OffsetDateTime.of(localDateTime, ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME); } - public static int daysFromISODateString(String dateString) { + public static String microsToIsoDateTime(long micros) { + LocalDateTime localDateTime = LocalDateTime.ofEpochSecond(micros / 1000000, + (int) (micros % 1000000) * 1000, ZoneOffset.UTC); + return localDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); + } + + public static int isoDateToDays(String dateString) { return daysFromDate(LocalDate.parse(dateString, DateTimeFormatter.ISO_LOCAL_DATE)); } - public static long microsFromISOTimeString(String timeString) { + public static long isoTimeToMicros(String timeString) { return microsFromTime(LocalTime.parse(timeString, DateTimeFormatter.ISO_LOCAL_TIME)); } - public static long microsFromISOOffsetTsString(String timestampString) { + public static long isoDateTimeTzToMicros(String timestampString) { return microsFromTimestamptz(OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME)); } - public static long microsFromISOTsString(String timestampString) { + public static long isoDateTimeToMicros(String timestampString) { return microsFromTimestamp(LocalDateTime.parse(timestampString, DateTimeFormatter.ISO_LOCAL_DATE_TIME)); } } From af1a8d93287d8072077567b20bb8b36677d63ded Mon Sep 17 00:00:00 2001 From: Limian Zhang <75445426+rzhang10@users.noreply.github.com> Date: Sun, 12 Jun 2022 14:34:28 -0700 Subject: [PATCH 08/23] Add handle null and map refactor in toJson --- .../apache/iceberg/DefaultValueParser.java | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index 480d07cd998a..7f11c3ee1f0d 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -216,6 +216,9 @@ public static String toJson(Type type, Object defaultValue, boolean pretty) { @SuppressWarnings("checkstyle:CyclomaticComplexity") public static void toJson(Type type, Object defaultValue, JsonGenerator generator) throws IOException { + if (defaultValue == null) { + generator.writeNull(); + } switch (type.typeId()) { case BOOLEAN: Preconditions.checkArgument( @@ -304,24 +307,18 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato case MAP: Preconditions.checkArgument(defaultValue instanceof Map, "Invalid default %s value: %s", type, defaultValue); Map defaultMap = (Map) defaultValue; - List keyList = Lists.newArrayListWithExpectedSize(defaultMap.size()); - List valueList = Lists.newArrayListWithExpectedSize(defaultMap.size()); Type keyType = type.asMapType().keyType(); Type valueType = type.asMapType().valueType(); + List valueList = Lists.newArrayListWithExpectedSize(defaultMap.size()); + generator.writeStartObject(); + generator.writeArrayFieldStart(KEYS); for (Map.Entry entry : defaultMap.entrySet()) { - keyList.add(entry.getKey()); + toJson(keyType, entry.getKey(), generator); valueList.add(entry.getValue()); } - generator.writeStartObject(); - generator.writeFieldName(KEYS); - generator.writeStartArray(); - for (Object key : keyList) { - toJson(keyType, key, generator); - } generator.writeEndArray(); - generator.writeFieldName(VALUES); - generator.writeStartArray(); + generator.writeArrayFieldStart(VALUES); for (Object value : valueList) { toJson(valueType, value, generator); } From 84c699498c73eebeb019216e13e3b0927f04ea4f Mon Sep 17 00:00:00 2001 From: Limian Zhang <75445426+rzhang10@users.noreply.github.com> Date: Sun, 12 Jun 2022 14:47:13 -0700 Subject: [PATCH 09/23] Minor fix --- core/src/main/java/org/apache/iceberg/DefaultValueParser.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index 7f11c3ee1f0d..e48708c8a302 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -218,6 +218,7 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato throws IOException { if (defaultValue == null) { generator.writeNull(); + return; } switch (type.typeId()) { case BOOLEAN: From 3ef04c82328e70da97a8b7bb840c3a2b42def581 Mon Sep 17 00:00:00 2001 From: Raymond Zhang Date: Mon, 13 Jun 2022 11:39:09 -0700 Subject: [PATCH 10/23] Add null unit test and add additional validation for FIXED in toJson --- .../java/org/apache/iceberg/DefaultValueParser.java | 11 +++++++++++ .../iceberg/TestDefaultValuesParsingAndUnParsing.java | 1 + 2 files changed, 12 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index e48708c8a302..161e46c5d400 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -281,6 +281,17 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato generator.writeString(defaultValue.toString()); break; case FIXED: + Preconditions.checkArgument( + defaultValue instanceof ByteBuffer, "Invalid default %s value: %s", type, defaultValue); + String fixedString = BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) defaultValue))); + int valueLength = fixedString.length(); + int expectedLength = ((Types.FixedType) type).length(); + Preconditions.checkArgument(valueLength == expectedLength * 2, + "Default value %s is not compatible with the expected fixed type, the fixed type is expected to store " + + "exactly %s bytes, which means the default value should be of exactly 2 * %s length hex string", + fixedString, expectedLength, expectedLength); + generator.writeString(fixedString); + break; case BINARY: Preconditions.checkArgument( defaultValue instanceof ByteBuffer, "Invalid default %s value: %s", type, defaultValue); diff --git a/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java b/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java index 8b9fd8b2da1e..5d7874ea2c8b 100644 --- a/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java +++ b/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java @@ -49,6 +49,7 @@ public TestDefaultValuesParsingAndUnParsing(Type type, String defaultValue) { @Parameterized.Parameters public static Collection data() { return Arrays.asList(new Object[][] { + {Types.BooleanType.get(), "null"}, {Types.BooleanType.get(), "true"}, {Types.IntegerType.get(), "1"}, {Types.LongType.get(), "9999999"}, From 1e8f98ef85c6d1f60df6b36282dd64a240108d8b Mon Sep 17 00:00:00 2001 From: Raymond Zhang Date: Thu, 16 Jun 2022 15:48:46 -0700 Subject: [PATCH 11/23] Address comments --- .../apache/iceberg/DefaultValueParser.java | 57 +++++++------------ .../org/apache/iceberg/util/DateTimeUtil.java | 10 ++-- .../iceberg/TestInvalidDefaultValues.java | 26 ++++----- .../spark/source/TestRuntimeFiltering.java | 2 +- 4 files changed, 39 insertions(+), 56 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index 161e46c5d400..aa0ba478377b 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -75,7 +75,7 @@ public static Object fromJson(Type type, JsonNode defaultValue) { "Cannot parse default as a %s value: %s", type, defaultValue); return defaultValue.floatValue(); case DOUBLE: - Preconditions.checkArgument(defaultValue.isNumber(), + Preconditions.checkArgument(defaultValue.isFloatingPointNumber(), "Cannot parse default as a %s value: %s", type, defaultValue); return defaultValue.doubleValue(); case DECIMAL: @@ -83,9 +83,8 @@ public static Object fromJson(Type type, JsonNode defaultValue) { "Cannot parse default as a %s value: %s", type, defaultValue); BigDecimal retDecimal = defaultValue.decimalValue(); Preconditions.checkArgument( - retDecimal.scale() == ((Types.DecimalType) type).scale(), "Cannot parse default as a %s value: %s", - type, - defaultValue); + retDecimal.scale() == ((Types.DecimalType) type).scale(), + "Cannot parse default as a %s value: %s, the scale doesn't match", type, defaultValue); return retDecimal; case STRING: Preconditions.checkArgument(defaultValue.isTextual(), @@ -94,13 +93,14 @@ public static Object fromJson(Type type, JsonNode defaultValue) { case UUID: Preconditions.checkArgument(defaultValue.isTextual(), "Cannot parse default as a %s value: %s", type, defaultValue); + UUID uuid; try { - UUID.fromString(defaultValue.textValue()); + uuid = UUID.fromString(defaultValue.textValue()); } catch (IllegalArgumentException e) { throw new IllegalArgumentException(String.format("Cannot parse default as a %s value: %s", type, - defaultValue)); + defaultValue), e); } - return UUID.fromString(defaultValue.textValue()); + return uuid; case DATE: Preconditions.checkArgument(defaultValue.isTextual(), "Cannot parse default as a %s value: %s", type, defaultValue); @@ -124,9 +124,7 @@ public static Object fromJson(Type type, JsonNode defaultValue) { int defaultLength = defaultValue.textValue().length(); int fixedLength = ((Types.FixedType) type).length(); Preconditions.checkArgument(defaultLength == fixedLength * 2, - "Default value %s is not compatible with the expected fixed type, the fixed type is expected to store " + - "exactly %s bytes, which means the default value should be of exactly 2 * %s length hex string", - defaultValue, fixedLength, fixedLength); + "Cannot parse default %s value: %s, incorrect length: %s", type, defaultValue, defaultLength); byte[] fixedBytes = BaseEncoding.base16().decode(defaultValue.textValue().toUpperCase(Locale.ROOT)); return ByteBuffer.wrap(fixedBytes); case BINARY: @@ -149,9 +147,7 @@ public static Object fromJson(Type type, JsonNode defaultValue) { JsonNode values = defaultValue.get(VALUES); Preconditions.checkArgument( keys.size() == values.size(), - "Cannot parse default as a %s value: %s", - type, - defaultValue); + "Cannot parse default as a %s value: %s", type, defaultValue); ImmutableMap.Builder mapBuilder = ImmutableMap.builder(); @@ -181,7 +177,7 @@ public static Object fromJson(Type type, JsonNode defaultValue) { } return defaultRecord; default: - throw new IllegalArgumentException(String.format("Type: %s is not supported", type)); + throw new UnsupportedOperationException(String.format("Type: %s is not supported", type)); } } @@ -220,6 +216,7 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato generator.writeNull(); return; } + switch (type.typeId()) { case BOOLEAN: Preconditions.checkArgument( @@ -245,26 +242,17 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato break; case DATE: Preconditions.checkArgument( - defaultValue instanceof Integer && (Integer) defaultValue >= 0, - "Invalid default %s value: %s", - type, - defaultValue); + defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue); generator.writeString(DateTimeUtil.daysToIsoDate((Integer) defaultValue)); break; case TIME: Preconditions.checkArgument( - defaultValue instanceof Long && (Long) defaultValue >= 0, - "Invalid default %s value: %s", - type, - defaultValue); + defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue); generator.writeString(DateTimeUtil.microsToIsoTime((Long) defaultValue)); break; case TIMESTAMP: Preconditions.checkArgument( - defaultValue instanceof Long && (Long) defaultValue >= 0, - "Invalid default %s value: %s", - type, - defaultValue); + defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue); if (((Types.TimestampType) type).shouldAdjustToUTC()) { generator.writeString(DateTimeUtil.microsToIsoDateTimeTz((Long) defaultValue)); } else { @@ -287,9 +275,8 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato int valueLength = fixedString.length(); int expectedLength = ((Types.FixedType) type).length(); Preconditions.checkArgument(valueLength == expectedLength * 2, - "Default value %s is not compatible with the expected fixed type, the fixed type is expected to store " + - "exactly %s bytes, which means the default value should be of exactly 2 * %s length hex string", - fixedString, expectedLength, expectedLength); + "Invalid default %s value: %s, incorrect length: %s", + type, defaultValue, valueLength); generator.writeString(fixedString); break; case BINARY: @@ -301,9 +288,7 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato Preconditions.checkArgument( defaultValue instanceof BigDecimal && ((BigDecimal) defaultValue).scale() == ((Types.DecimalType) type).scale(), - "Invalid default %s value: %s", - type, - defaultValue); + "Invalid default %s value: %s", type, defaultValue); generator.writeNumber((BigDecimal) defaultValue); break; case LIST: @@ -348,16 +333,16 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato for (int i = 0; i < defaultStruct.size(); i++) { Types.NestedField field = fields.get(i); int fieldId = field.fieldId(); - Object fieldJavaDefaultValue = defaultStruct.get(i, Object.class); - if (fieldJavaDefaultValue != null) { + Object fieldDefaultValue = defaultStruct.get(i, Object.class); + if (fieldDefaultValue != null) { generator.writeFieldName(String.valueOf(fieldId)); - toJson(field.type(), fieldJavaDefaultValue, generator); + toJson(field.type(), fieldDefaultValue, generator); } } generator.writeEndObject(); break; default: - throw new IllegalArgumentException(String.format("Type: %s is not supported", type)); + throw new UnsupportedOperationException(String.format("Type: %s is not supported", type)); } } } diff --git a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java index 6961c736e120..489ab4955080 100644 --- a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java @@ -85,22 +85,20 @@ public static String formatTimestampMillis(long millis) { } public static String daysToIsoDate(int days) { - return LocalDate.ofEpochDay(days).format(DateTimeFormatter.ISO_LOCAL_DATE); + return dateFromDays(days).format(DateTimeFormatter.ISO_LOCAL_DATE); } public static String microsToIsoTime(long micros) { - return LocalTime.ofNanoOfDay(micros * 1000).format(DateTimeFormatter.ISO_LOCAL_TIME); + return timeFromMicros(micros).format(DateTimeFormatter.ISO_LOCAL_TIME); } public static String microsToIsoDateTimeTz(long micros) { - LocalDateTime localDateTime = LocalDateTime.ofEpochSecond(micros / 1000000, - (int) (micros % 1000000) * 1000, ZoneOffset.UTC); + LocalDateTime localDateTime = timestampFromMicros(micros); return OffsetDateTime.of(localDateTime, ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME); } public static String microsToIsoDateTime(long micros) { - LocalDateTime localDateTime = LocalDateTime.ofEpochSecond(micros / 1000000, - (int) (micros % 1000000) * 1000, ZoneOffset.UTC); + LocalDateTime localDateTime = timestampFromMicros(micros); return localDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); } diff --git a/core/src/test/java/org/apache/iceberg/TestInvalidDefaultValues.java b/core/src/test/java/org/apache/iceberg/TestInvalidDefaultValues.java index 1ea5eac9dbc2..02288baae9e4 100644 --- a/core/src/test/java/org/apache/iceberg/TestInvalidDefaultValues.java +++ b/core/src/test/java/org/apache/iceberg/TestInvalidDefaultValues.java @@ -32,35 +32,35 @@ public class TestInvalidDefaultValues { public void testInvalidFixed() { Type expectedType = Types.FixedType.ofLength(2); String defaultJson = "\"111ff\""; - Assert.assertThrows(IllegalArgumentException.class, () -> defaultValueParseAndUnParseRoundTrip( - expectedType, - defaultJson)); + Exception exception = Assert.assertThrows(IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default fixed[2] value")); } @Test public void testInvalidUUID() { - Type expectedType = Types.FixedType.ofLength(2); + Type expectedType = Types.UUIDType.get(); String defaultJson = "\"eb26bdb1-a1d8-4aa6-990e-da940875492c-abcde\""; - Assert.assertThrows(IllegalArgumentException.class, () -> defaultValueParseAndUnParseRoundTrip( - expectedType, - defaultJson)); + Exception exception = Assert.assertThrows(IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a uuid value")); } @Test public void testInvalidMap() { Type expectedType = Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()); String defaultJson = "{\"keys\": [1, 2, 3], \"values\": [\"foo\", \"bar\"]}"; - Assert.assertThrows(IllegalArgumentException.class, () -> defaultValueParseAndUnParseRoundTrip( - expectedType, - defaultJson)); + Exception exception = Assert.assertThrows(IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a map value")); } @Test public void testInvalidDecimal() { Type expectedType = Types.DecimalType.of(5, 2); String defaultJson = "123.456"; - Assert.assertThrows(IllegalArgumentException.class, () -> defaultValueParseAndUnParseRoundTrip( - expectedType, - defaultJson)); + Exception exception = Assert.assertThrows(IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a decimal(5, 2) value")); } } diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java index beaf7b75c6c0..642b72db62ad 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java @@ -66,7 +66,7 @@ public void testIdentityPartitionedTable() throws NoSuchTableException { .withColumn("data", expr("CAST(date AS STRING)")) .select("id", "data", "date", "ts"); - df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").option(SparkWriteOptions.WRITE_FORMAT, "avro").append(); sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); Dataset dimDF = From dad18117ad7697fe496be5e85e6487d4867f1c3a Mon Sep 17 00:00:00 2001 From: Raymond Zhang Date: Thu, 16 Jun 2022 15:58:22 -0700 Subject: [PATCH 12/23] Refactor unit tests --- .../iceberg/TestDefaultValueParser.java | 160 ++++++++++++++++++ .../TestDefaultValuesParsingAndUnParsing.java | 116 ------------- .../iceberg/TestInvalidDefaultValues.java | 66 -------- 3 files changed, 160 insertions(+), 182 deletions(-) create mode 100644 core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java delete mode 100644 core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java delete mode 100644 core/src/test/java/org/apache/iceberg/TestInvalidDefaultValues.java diff --git a/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java b/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java new file mode 100644 index 000000000000..811240e48bb0 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +import java.io.IOException; +import java.time.OffsetDateTime; +import java.util.Arrays; +import java.util.Collection; +import java.util.Locale; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.JsonUtil; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.runners.Enclosed; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +@RunWith(Enclosed.class) +public class TestDefaultValueParser { + + @RunWith(Parameterized.class) + public static class TestValidDefaultValues { + + private final Type type; + private final String defaultValue; + + public TestValidDefaultValues(Type type, String defaultValue) { + this.type = type; + this.defaultValue = defaultValue; + } + + @Parameterized.Parameters + public static Collection data() { + return Arrays.asList(new Object[][] { + {Types.BooleanType.get(), "null"}, + {Types.BooleanType.get(), "true"}, + {Types.IntegerType.get(), "1"}, + {Types.LongType.get(), "9999999"}, + {Types.FloatType.get(), "1.23"}, + {Types.DoubleType.get(), "123.456"}, + {Types.DateType.get(), "\"2007-12-03\""}, + {Types.TimeType.get(), "\"10:15:30\""}, + {Types.TimestampType.withoutZone(), "\"2007-12-03T10:15:30\""}, + {Types.TimestampType.withZone(), "\"2007-12-03T10:15:30+01:00\""}, + {Types.StringType.get(), "\"foo\""}, + {Types.UUIDType.get(), "\"eb26bdb1-a1d8-4aa6-990e-da940875492c\""}, + {Types.FixedType.ofLength(2), "\"111f\""}, + {Types.BinaryType.get(), "\"0000ff\""}, + {Types.DecimalType.of(9, 2), "123.45"}, + {Types.ListType.ofOptional(1, Types.IntegerType.get()), "[1, 2, 3]"}, + {Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()), + "{\"keys\": [1, 2], \"values\": [\"foo\", \"bar\"]}"}, + {Types.StructType.of( + required(1, "f1", Types.IntegerType.get(), "doc"), + optional(2, "f2", Types.StringType.get(), "doc")), + "{\"1\": 1, \"2\": \"bar\"}"}, + // deeply nested complex types + {Types.ListType.ofOptional(1, Types.StructType.of( + required(1, "f1", Types.IntegerType.get(), "doc"), + optional(2, "f2", Types.StringType.get(), "doc"))), "[{\"1\": 1, \"2\": \"bar\"}, {\"1\": 2, \"2\": " + + "\"foo\"}]"}, + {Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StructType.of( + required(1, "f1", Types.IntegerType.get(), "doc"), + optional(2, "f2", Types.StringType.get(), "doc"))), + "{\"keys\": [1, 2], \"values\": [{\"1\": 1, \"2\": \"bar\"}, {\"1\": 2, \"2\": \"foo\"}]}"}, + {Types.StructType.of( + required(1, "f1", Types.StructType.of( + optional(2, "ff1", Types.IntegerType.get(), "doc"), + optional(3, "ff2", Types.StringType.get(), "doc")), "doc"), + optional(4, "f2", Types.StructType.of( + optional(5, "ff1", Types.StringType.get(), "doc"), + optional(6, "ff2", Types.IntegerType.get(), "doc")), "doc")), + "{\"1\": {\"2\": 1, \"3\": \"bar\"}, \"4\": {\"5\": \"bar\", \"6\": 1}}"}, + }); + } + + @Test + public void testTypeWithDefaultValue() throws IOException { + String roundTripDefaultValue = defaultValueParseAndUnParseRoundTrip(type, defaultValue); + // Only if the type is a timestampWithZone type, the round-trip default value will always be standardized to the + // UTC time zone, which might be different in the original value, but they should represent the same instant + if (type.typeId() == Type.TypeID.TIMESTAMP && ((Types.TimestampType) type).shouldAdjustToUTC()) { + Assert.assertTrue(OffsetDateTime.parse(JsonUtil.mapper().readTree(defaultValue).textValue()) + .isEqual(OffsetDateTime.parse(JsonUtil.mapper().readTree(roundTripDefaultValue).textValue()))); + } else { + jsonStringEquals(defaultValue.toLowerCase(Locale.ROOT), roundTripDefaultValue.toLowerCase(Locale.ROOT)); + } + } + } + + public static class TestInvalidDefaultValues { + + @Test + public void testInvalidFixed() { + Type expectedType = Types.FixedType.ofLength(2); + String defaultJson = "\"111ff\""; + Exception exception = Assert.assertThrows(IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default fixed[2] value")); + } + + @Test + public void testInvalidUUID() { + Type expectedType = Types.UUIDType.get(); + String defaultJson = "\"eb26bdb1-a1d8-4aa6-990e-da940875492c-abcde\""; + Exception exception = Assert.assertThrows(IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a uuid value")); + } + + @Test + public void testInvalidMap() { + Type expectedType = Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()); + String defaultJson = "{\"keys\": [1, 2, 3], \"values\": [\"foo\", \"bar\"]}"; + Exception exception = Assert.assertThrows(IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a map value")); + } + + @Test + public void testInvalidDecimal() { + Type expectedType = Types.DecimalType.of(5, 2); + String defaultJson = "123.456"; + Exception exception = Assert.assertThrows(IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a decimal(5, 2) value")); + } + } + + // serialize to json and deserialize back should return the same result + private static String defaultValueParseAndUnParseRoundTrip(Type type, String defaultValue) throws IOException { + Object javaDefaultValue = DefaultValueParser.fromJson(type, defaultValue); + return DefaultValueParser.toJson(type, javaDefaultValue); + } + + private static void jsonStringEquals(String s1, String s2) throws IOException { + Assert.assertEquals(JsonUtil.mapper().readTree(s1), JsonUtil.mapper().readTree(s2)); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java b/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java deleted file mode 100644 index 5d7874ea2c8b..000000000000 --- a/core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iceberg; - -import java.io.IOException; -import java.time.OffsetDateTime; -import java.util.Arrays; -import java.util.Collection; -import java.util.Locale; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.JsonUtil; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; - -@RunWith(Parameterized.class) -public class TestDefaultValuesParsingAndUnParsing { - - private final Type type; - private final String defaultValue; - - public TestDefaultValuesParsingAndUnParsing(Type type, String defaultValue) { - this.type = type; - this.defaultValue = defaultValue; - } - - @Parameterized.Parameters - public static Collection data() { - return Arrays.asList(new Object[][] { - {Types.BooleanType.get(), "null"}, - {Types.BooleanType.get(), "true"}, - {Types.IntegerType.get(), "1"}, - {Types.LongType.get(), "9999999"}, - {Types.FloatType.get(), "1.23"}, - {Types.DoubleType.get(), "123.456"}, - {Types.DateType.get(), "\"2007-12-03\""}, - {Types.TimeType.get(), "\"10:15:30\""}, - {Types.TimestampType.withoutZone(), "\"2007-12-03T10:15:30\""}, - {Types.TimestampType.withZone(), "\"2007-12-03T10:15:30+01:00\""}, - {Types.StringType.get(), "\"foo\""}, - {Types.UUIDType.get(), "\"eb26bdb1-a1d8-4aa6-990e-da940875492c\""}, - {Types.FixedType.ofLength(2), "\"111f\""}, - {Types.BinaryType.get(), "\"0000ff\""}, - {Types.DecimalType.of(9, 2), "123.45"}, - {Types.ListType.ofOptional(1, Types.IntegerType.get()), "[1, 2, 3]"}, - {Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()), - "{\"keys\": [1, 2], \"values\": [\"foo\", \"bar\"]}"}, - {Types.StructType.of( - required(1, "f1", Types.IntegerType.get(), "doc"), - optional(2, "f2", Types.StringType.get(), "doc")), - "{\"1\": 1, \"2\": \"bar\"}"}, - // deeply nested complex types - {Types.ListType.ofOptional(1, Types.StructType.of( - required(1, "f1", Types.IntegerType.get(), "doc"), - optional(2, "f2", Types.StringType.get(), "doc"))), "[{\"1\": 1, \"2\": \"bar\"}, {\"1\": 2, \"2\": " + - "\"foo\"}]"}, - {Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StructType.of( - required(1, "f1", Types.IntegerType.get(), "doc"), - optional(2, "f2", Types.StringType.get(), "doc"))), - "{\"keys\": [1, 2], \"values\": [{\"1\": 1, \"2\": \"bar\"}, {\"1\": 2, \"2\": \"foo\"}]}"}, - {Types.StructType.of( - required(1, "f1", Types.StructType.of( - optional(2, "ff1", Types.IntegerType.get(), "doc"), - optional(3, "ff2", Types.StringType.get(), "doc")), "doc"), - optional(4, "f2", Types.StructType.of( - optional(5, "ff1", Types.StringType.get(), "doc"), - optional(6, "ff2", Types.IntegerType.get(), "doc")), "doc")), - "{\"1\": {\"2\": 1, \"3\": \"bar\"}, \"4\": {\"5\": \"bar\", \"6\": 1}}"}, - }); - } - - // serialize to json and deserialize back should return the same result - static String defaultValueParseAndUnParseRoundTrip(Type type, String defaultValue) throws IOException { - Object javaDefaultValue = DefaultValueParser.fromJson(type, defaultValue); - return DefaultValueParser.toJson(type, javaDefaultValue); - } - - @Test - public void testTypeWithDefaultValue() throws IOException { - String roundTripDefaultValue = defaultValueParseAndUnParseRoundTrip(type, defaultValue); - // Only if the type is a timestampWithZone type, the round-trip default value will always be standardized to the - // UTC time zone, which might be different in the original value, but they should represent the same instant - if (type.typeId() == Type.TypeID.TIMESTAMP && ((Types.TimestampType) type).shouldAdjustToUTC()) { - Assert.assertTrue(OffsetDateTime.parse(JsonUtil.mapper().readTree(defaultValue).textValue()) - .isEqual(OffsetDateTime.parse(JsonUtil.mapper().readTree(roundTripDefaultValue).textValue()))); - } else { - jsonStringEquals(defaultValue.toLowerCase(Locale.ROOT), roundTripDefaultValue.toLowerCase(Locale.ROOT)); - } - } - - private static void jsonStringEquals(String s1, String s2) throws IOException { - Assert.assertEquals(JsonUtil.mapper().readTree(s1), JsonUtil.mapper().readTree(s2)); - } -} diff --git a/core/src/test/java/org/apache/iceberg/TestInvalidDefaultValues.java b/core/src/test/java/org/apache/iceberg/TestInvalidDefaultValues.java deleted file mode 100644 index 02288baae9e4..000000000000 --- a/core/src/test/java/org/apache/iceberg/TestInvalidDefaultValues.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iceberg; - -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; - -import static org.apache.iceberg.TestDefaultValuesParsingAndUnParsing.defaultValueParseAndUnParseRoundTrip; - -public class TestInvalidDefaultValues { - - @Test - public void testInvalidFixed() { - Type expectedType = Types.FixedType.ofLength(2); - String defaultJson = "\"111ff\""; - Exception exception = Assert.assertThrows(IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default fixed[2] value")); - } - - @Test - public void testInvalidUUID() { - Type expectedType = Types.UUIDType.get(); - String defaultJson = "\"eb26bdb1-a1d8-4aa6-990e-da940875492c-abcde\""; - Exception exception = Assert.assertThrows(IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a uuid value")); - } - - @Test - public void testInvalidMap() { - Type expectedType = Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()); - String defaultJson = "{\"keys\": [1, 2, 3], \"values\": [\"foo\", \"bar\"]}"; - Exception exception = Assert.assertThrows(IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a map value")); - } - - @Test - public void testInvalidDecimal() { - Type expectedType = Types.DecimalType.of(5, 2); - String defaultJson = "123.456"; - Exception exception = Assert.assertThrows(IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a decimal(5, 2) value")); - } -} From 2042fb624953a10ea2c060672a9beba753fe05ec Mon Sep 17 00:00:00 2001 From: Raymond Zhang Date: Thu, 16 Jun 2022 16:36:21 -0700 Subject: [PATCH 13/23] refactor unit tests again --- .../iceberg/TestDefaultValueParser.java | 179 ++++++++---------- 1 file changed, 82 insertions(+), 97 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java b/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java index 811240e48bb0..68a2d36b2ce2 100644 --- a/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java +++ b/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java @@ -21,82 +21,66 @@ import java.io.IOException; import java.time.OffsetDateTime; -import java.util.Arrays; -import java.util.Collection; import java.util.Locale; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.JsonUtil; import org.junit.Assert; import org.junit.Test; -import org.junit.experimental.runners.Enclosed; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; -@RunWith(Enclosed.class) public class TestDefaultValueParser { - @RunWith(Parameterized.class) - public static class TestValidDefaultValues { + @Test + public void testValidDefaults() throws IOException { + Object[][] typesWithDefaults = new Object[][] { + {Types.BooleanType.get(), "null"}, + {Types.BooleanType.get(), "true"}, + {Types.IntegerType.get(), "1"}, + {Types.LongType.get(), "9999999"}, + {Types.FloatType.get(), "1.23"}, + {Types.DoubleType.get(), "123.456"}, + {Types.DateType.get(), "\"2007-12-03\""}, + {Types.TimeType.get(), "\"10:15:30\""}, + {Types.TimestampType.withoutZone(), "\"2007-12-03T10:15:30\""}, + {Types.TimestampType.withZone(), "\"2007-12-03T10:15:30+01:00\""}, + {Types.StringType.get(), "\"foo\""}, + {Types.UUIDType.get(), "\"eb26bdb1-a1d8-4aa6-990e-da940875492c\""}, + {Types.FixedType.ofLength(2), "\"111f\""}, + {Types.BinaryType.get(), "\"0000ff\""}, + {Types.DecimalType.of(9, 2), "123.45"}, + {Types.ListType.ofOptional(1, Types.IntegerType.get()), "[1, 2, 3]"}, + {Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()), + "{\"keys\": [1, 2], \"values\": [\"foo\", \"bar\"]}"}, + {Types.StructType.of( + required(1, "f1", Types.IntegerType.get(), "doc"), + optional(2, "f2", Types.StringType.get(), "doc")), + "{\"1\": 1, \"2\": \"bar\"}"}, + // deeply nested complex types + {Types.ListType.ofOptional(1, Types.StructType.of( + required(1, "f1", Types.IntegerType.get(), "doc"), + optional(2, "f2", Types.StringType.get(), "doc"))), "[{\"1\": 1, \"2\": \"bar\"}, {\"1\": 2, \"2\": " + + "\"foo\"}]"}, + {Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StructType.of( + required(1, "f1", Types.IntegerType.get(), "doc"), + optional(2, "f2", Types.StringType.get(), "doc"))), + "{\"keys\": [1, 2], \"values\": [{\"1\": 1, \"2\": \"bar\"}, {\"1\": 2, \"2\": \"foo\"}]}"}, + {Types.StructType.of( + required(1, "f1", Types.StructType.of( + optional(2, "ff1", Types.IntegerType.get(), "doc"), + optional(3, "ff2", Types.StringType.get(), "doc")), "doc"), + optional(4, "f2", Types.StructType.of( + optional(5, "ff1", Types.StringType.get(), "doc"), + optional(6, "ff2", Types.IntegerType.get(), "doc")), "doc")), + "{\"1\": {\"2\": 1, \"3\": \"bar\"}, \"4\": {\"5\": \"bar\", \"6\": 1}}"}, + }; + + for (Object[] typeWithDefault : typesWithDefaults) { + Type type = (Type) typeWithDefault[0]; + String defaultValue = (String) typeWithDefault[1]; - private final Type type; - private final String defaultValue; - - public TestValidDefaultValues(Type type, String defaultValue) { - this.type = type; - this.defaultValue = defaultValue; - } - - @Parameterized.Parameters - public static Collection data() { - return Arrays.asList(new Object[][] { - {Types.BooleanType.get(), "null"}, - {Types.BooleanType.get(), "true"}, - {Types.IntegerType.get(), "1"}, - {Types.LongType.get(), "9999999"}, - {Types.FloatType.get(), "1.23"}, - {Types.DoubleType.get(), "123.456"}, - {Types.DateType.get(), "\"2007-12-03\""}, - {Types.TimeType.get(), "\"10:15:30\""}, - {Types.TimestampType.withoutZone(), "\"2007-12-03T10:15:30\""}, - {Types.TimestampType.withZone(), "\"2007-12-03T10:15:30+01:00\""}, - {Types.StringType.get(), "\"foo\""}, - {Types.UUIDType.get(), "\"eb26bdb1-a1d8-4aa6-990e-da940875492c\""}, - {Types.FixedType.ofLength(2), "\"111f\""}, - {Types.BinaryType.get(), "\"0000ff\""}, - {Types.DecimalType.of(9, 2), "123.45"}, - {Types.ListType.ofOptional(1, Types.IntegerType.get()), "[1, 2, 3]"}, - {Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()), - "{\"keys\": [1, 2], \"values\": [\"foo\", \"bar\"]}"}, - {Types.StructType.of( - required(1, "f1", Types.IntegerType.get(), "doc"), - optional(2, "f2", Types.StringType.get(), "doc")), - "{\"1\": 1, \"2\": \"bar\"}"}, - // deeply nested complex types - {Types.ListType.ofOptional(1, Types.StructType.of( - required(1, "f1", Types.IntegerType.get(), "doc"), - optional(2, "f2", Types.StringType.get(), "doc"))), "[{\"1\": 1, \"2\": \"bar\"}, {\"1\": 2, \"2\": " + - "\"foo\"}]"}, - {Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StructType.of( - required(1, "f1", Types.IntegerType.get(), "doc"), - optional(2, "f2", Types.StringType.get(), "doc"))), - "{\"keys\": [1, 2], \"values\": [{\"1\": 1, \"2\": \"bar\"}, {\"1\": 2, \"2\": \"foo\"}]}"}, - {Types.StructType.of( - required(1, "f1", Types.StructType.of( - optional(2, "ff1", Types.IntegerType.get(), "doc"), - optional(3, "ff2", Types.StringType.get(), "doc")), "doc"), - optional(4, "f2", Types.StructType.of( - optional(5, "ff1", Types.StringType.get(), "doc"), - optional(6, "ff2", Types.IntegerType.get(), "doc")), "doc")), - "{\"1\": {\"2\": 1, \"3\": \"bar\"}, \"4\": {\"5\": \"bar\", \"6\": 1}}"}, - }); - } - - @Test - public void testTypeWithDefaultValue() throws IOException { String roundTripDefaultValue = defaultValueParseAndUnParseRoundTrip(type, defaultValue); // Only if the type is a timestampWithZone type, the round-trip default value will always be standardized to the // UTC time zone, which might be different in the original value, but they should represent the same instant @@ -109,43 +93,44 @@ public void testTypeWithDefaultValue() throws IOException { } } - public static class TestInvalidDefaultValues { - - @Test - public void testInvalidFixed() { - Type expectedType = Types.FixedType.ofLength(2); - String defaultJson = "\"111ff\""; - Exception exception = Assert.assertThrows(IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default fixed[2] value")); - } + @Test + public void testInvalidFixed() { + Type expectedType = Types.FixedType.ofLength(2); + String defaultJson = "\"111ff\""; + Exception exception = Assert.assertThrows( + IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default fixed[2] value")); + } - @Test - public void testInvalidUUID() { - Type expectedType = Types.UUIDType.get(); - String defaultJson = "\"eb26bdb1-a1d8-4aa6-990e-da940875492c-abcde\""; - Exception exception = Assert.assertThrows(IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a uuid value")); - } + @Test + public void testInvalidUUID() { + Type expectedType = Types.UUIDType.get(); + String defaultJson = "\"eb26bdb1-a1d8-4aa6-990e-da940875492c-abcde\""; + Exception exception = Assert.assertThrows( + IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a uuid value")); + } - @Test - public void testInvalidMap() { - Type expectedType = Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()); - String defaultJson = "{\"keys\": [1, 2, 3], \"values\": [\"foo\", \"bar\"]}"; - Exception exception = Assert.assertThrows(IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a map value")); - } + @Test + public void testInvalidMap() { + Type expectedType = Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()); + String defaultJson = "{\"keys\": [1, 2, 3], \"values\": [\"foo\", \"bar\"]}"; + Exception exception = Assert.assertThrows( + IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a map value")); + } - @Test - public void testInvalidDecimal() { - Type expectedType = Types.DecimalType.of(5, 2); - String defaultJson = "123.456"; - Exception exception = Assert.assertThrows(IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a decimal(5, 2) value")); - } + @Test + public void testInvalidDecimal() { + Type expectedType = Types.DecimalType.of(5, 2); + String defaultJson = "123.456"; + Exception exception = Assert.assertThrows( + IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a decimal(5, 2) value")); } // serialize to json and deserialize back should return the same result From 9cca9c606495a82809bee75476749d27939e92d6 Mon Sep 17 00:00:00 2001 From: Raymond Zhang Date: Thu, 16 Jun 2022 16:39:11 -0700 Subject: [PATCH 14/23] revert mistaken changes --- .../org/apache/iceberg/spark/source/TestRuntimeFiltering.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java index 642b72db62ad..beaf7b75c6c0 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java @@ -66,7 +66,7 @@ public void testIdentityPartitionedTable() throws NoSuchTableException { .withColumn("data", expr("CAST(date AS STRING)")) .select("id", "data", "date", "ts"); - df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").option(SparkWriteOptions.WRITE_FORMAT, "avro").append(); + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); Dataset dimDF = From 976e936da76ef3683612b8271b139964e96726ce Mon Sep 17 00:00:00 2001 From: Raymond Zhang Date: Mon, 11 Jul 2022 13:20:04 -0700 Subject: [PATCH 15/23] Address comments and refactor --- .../apache/iceberg/DefaultValueParser.java | 121 ++++++++++-------- .../org/apache/iceberg/util/DateTimeUtil.java | 11 +- .../iceberg/TestDefaultValueParser.java | 58 +++++---- 3 files changed, 108 insertions(+), 82 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index aa0ba478377b..ff0f37a3af55 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -47,12 +47,9 @@ private DefaultValueParser() { } private static final String KEYS = "keys"; - private static final String VALUES = "values"; - @SuppressWarnings("checkstyle:CyclomaticComplexity") public static Object fromJson(Type type, JsonNode defaultValue) { - if (defaultValue == null || defaultValue.isNull()) { return null; } @@ -91,7 +88,7 @@ public static Object fromJson(Type type, JsonNode defaultValue) { "Cannot parse default as a %s value: %s", type, defaultValue); return defaultValue.textValue(); case UUID: - Preconditions.checkArgument(defaultValue.isTextual(), + Preconditions.checkArgument(defaultValue.isTextual() && defaultValue.textValue().length() == 36, "Cannot parse default as a %s value: %s", type, defaultValue); UUID uuid; try { @@ -113,9 +110,13 @@ public static Object fromJson(Type type, JsonNode defaultValue) { Preconditions.checkArgument(defaultValue.isTextual(), "Cannot parse default as a %s value: %s", type, defaultValue); if (((Types.TimestampType) type).shouldAdjustToUTC()) { - return DateTimeUtil.isoDateTimeTzToMicros(defaultValue.textValue()); + String timestampTz = defaultValue.textValue(); + Preconditions.checkArgument(DateTimeUtil.timestamptzIsOfUTCZone(timestampTz), + "Cannot parse default as a %s value: %s, timezone must be UTC", + type, defaultValue); + return DateTimeUtil.isoTimestamptzToMicros(timestampTz); } else { - return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue()); + return DateTimeUtil.isoTimestampToMicros(defaultValue.textValue()); } case FIXED: Preconditions.checkArgument( @@ -133,52 +134,64 @@ public static Object fromJson(Type type, JsonNode defaultValue) { byte[] binaryBytes = BaseEncoding.base16().decode(defaultValue.textValue().toUpperCase(Locale.ROOT)); return ByteBuffer.wrap(binaryBytes); case LIST: - Preconditions.checkArgument(defaultValue.isArray(), - "Cannot parse default as a %s value: %s", type, defaultValue); - Type elementType = type.asListType().elementType(); - return Lists.newArrayList(Iterables.transform(defaultValue, e -> fromJson(elementType, e))); + return listFromJson(type, defaultValue); case MAP: - Preconditions.checkArgument( - defaultValue.isObject() && defaultValue.has(KEYS) && defaultValue.has(VALUES) && - defaultValue.get(KEYS).isArray() && defaultValue.get(VALUES).isArray(), - "Cannot parse %s to a %s value", - defaultValue, type); - JsonNode keys = defaultValue.get(KEYS); - JsonNode values = defaultValue.get(VALUES); - Preconditions.checkArgument( - keys.size() == values.size(), - "Cannot parse default as a %s value: %s", type, defaultValue); + return mapFromJson(type, defaultValue); + case STRUCT: + return structFromJson(type, defaultValue); + default: + throw new UnsupportedOperationException(String.format("Type: %s is not supported", type)); + } + } - ImmutableMap.Builder mapBuilder = ImmutableMap.builder(); + private static StructLike structFromJson(Type type, JsonNode defaultValue) { + Preconditions.checkArgument(defaultValue.isObject(), + "Cannot parse default as a %s value: %s", type, defaultValue); + Types.StructType struct = type.asStructType(); + StructLike defaultRecord = GenericRecord.create(struct); - Iterator keyIter = keys.iterator(); - Type keyType = type.asMapType().keyType(); - Iterator valueIter = values.iterator(); - Type valueType = type.asMapType().valueType(); + List fields = struct.fields(); + for (int pos = 0; pos < fields.size(); pos += 1) { + Types.NestedField field = fields.get(pos); + String idString = String.valueOf(field.fieldId()); + if (defaultValue.has(idString)) { + defaultRecord.set(pos, fromJson(field.type(), defaultValue.get(idString))); + } + } + return defaultRecord; + } - while (keyIter.hasNext()) { - mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next())); - } + private static Map mapFromJson(Type type, JsonNode defaultValue) { + Preconditions.checkArgument( + defaultValue.isObject() && defaultValue.has(KEYS) && defaultValue.has(VALUES) && + defaultValue.get(KEYS).isArray() && defaultValue.get(VALUES).isArray(), + "Cannot parse %s to a %s value", + defaultValue, type); + JsonNode keys = defaultValue.get(KEYS); + JsonNode values = defaultValue.get(VALUES); + Preconditions.checkArgument( + keys.size() == values.size(), + "Cannot parse default as a %s value: %s", type, defaultValue); - return mapBuilder.build(); - case STRUCT: - Preconditions.checkArgument(defaultValue.isObject(), - "Cannot parse default as a %s value: %s", type, defaultValue); - Types.StructType struct = type.asStructType(); - StructLike defaultRecord = GenericRecord.create(struct); + ImmutableMap.Builder mapBuilder = ImmutableMap.builder(); - List fields = struct.fields(); - for (int pos = 0; pos < fields.size(); pos += 1) { - Types.NestedField field = fields.get(pos); - String idString = String.valueOf(field.fieldId()); - if (defaultValue.has(idString)) { - defaultRecord.set(pos, fromJson(field.type(), defaultValue.get(idString))); - } - } - return defaultRecord; - default: - throw new UnsupportedOperationException(String.format("Type: %s is not supported", type)); + Iterator keyIter = keys.iterator(); + Type keyType = type.asMapType().keyType(); + Iterator valueIter = values.iterator(); + Type valueType = type.asMapType().valueType(); + + while (keyIter.hasNext()) { + mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next())); } + + return mapBuilder.build(); + } + + private static List listFromJson(Type type, JsonNode defaultValue) { + Preconditions.checkArgument(defaultValue.isArray(), + "Cannot parse default as a %s value: %s", type, defaultValue); + Type elementType = type.asListType().elementType(); + return Lists.newArrayList(Iterables.transform(defaultValue, e -> fromJson(elementType, e))); } public static Object fromJson(Type type, String defaultValue) { @@ -186,7 +199,8 @@ public static Object fromJson(Type type, String defaultValue) { JsonNode defaultValueJN = JsonUtil.mapper().readTree(defaultValue); return fromJson(type, defaultValueJN); } catch (IOException e) { - throw new IllegalArgumentException("Failed to parse: " + defaultValue + "; reason: " + e.getMessage(), e); + throw new IllegalArgumentException( + String.format("Failed to parse default as a %s value: %s", type, defaultValue), e); } } @@ -209,7 +223,6 @@ public static String toJson(Type type, Object defaultValue, boolean pretty) { } } - @SuppressWarnings("checkstyle:CyclomaticComplexity") public static void toJson(Type type, Object defaultValue, JsonGenerator generator) throws IOException { if (defaultValue == null) { @@ -271,12 +284,18 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato case FIXED: Preconditions.checkArgument( defaultValue instanceof ByteBuffer, "Invalid default %s value: %s", type, defaultValue); - String fixedString = BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) defaultValue))); - int valueLength = fixedString.length(); + ByteBuffer byteBufferDefaultValue = (ByteBuffer) defaultValue; + Preconditions.checkArgument( + byteBufferDefaultValue.hasArray() && byteBufferDefaultValue.arrayOffset() == 0 && + byteBufferDefaultValue.position() == 0, + "Invalid default %s value: %s, not a valid bytebuffer representation", + type, defaultValue); + int actualLength = byteBufferDefaultValue.remaining(); int expectedLength = ((Types.FixedType) type).length(); - Preconditions.checkArgument(valueLength == expectedLength * 2, + Preconditions.checkArgument(actualLength == expectedLength, "Invalid default %s value: %s, incorrect length: %s", - type, defaultValue, valueLength); + type, defaultValue, actualLength); + String fixedString = BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) defaultValue))); generator.writeString(fixedString); break; case BINARY: diff --git a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java index 489ab4955080..d0e9242bf2b4 100644 --- a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java @@ -94,7 +94,7 @@ public static String microsToIsoTime(long micros) { public static String microsToIsoDateTimeTz(long micros) { LocalDateTime localDateTime = timestampFromMicros(micros); - return OffsetDateTime.of(localDateTime, ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME); + return localDateTime.atOffset(ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME); } public static String microsToIsoDateTime(long micros) { @@ -110,11 +110,16 @@ public static long isoTimeToMicros(String timeString) { return microsFromTime(LocalTime.parse(timeString, DateTimeFormatter.ISO_LOCAL_TIME)); } - public static long isoDateTimeTzToMicros(String timestampString) { + public static long isoTimestamptzToMicros(String timestampString) { return microsFromTimestamptz(OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME)); } - public static long isoDateTimeToMicros(String timestampString) { + public static boolean timestamptzIsOfUTCZone(String timestampString) { + OffsetDateTime offsetDateTime = OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME); + return offsetDateTime.getOffset().equals(ZoneOffset.UTC) && timestampString.endsWith("Z"); + } + + public static long isoTimestampToMicros(String timestampString) { return microsFromTimestamp(LocalDateTime.parse(timestampString, DateTimeFormatter.ISO_LOCAL_DATE_TIME)); } } diff --git a/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java b/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java index 68a2d36b2ce2..55be90aab914 100644 --- a/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java +++ b/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java @@ -20,7 +20,6 @@ package org.apache.iceberg; import java.io.IOException; -import java.time.OffsetDateTime; import java.util.Locale; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -45,36 +44,36 @@ public void testValidDefaults() throws IOException { {Types.DateType.get(), "\"2007-12-03\""}, {Types.TimeType.get(), "\"10:15:30\""}, {Types.TimestampType.withoutZone(), "\"2007-12-03T10:15:30\""}, - {Types.TimestampType.withZone(), "\"2007-12-03T10:15:30+01:00\""}, + {Types.TimestampType.withZone(), "\"2007-12-03T10:15:30Z\""}, {Types.StringType.get(), "\"foo\""}, {Types.UUIDType.get(), "\"eb26bdb1-a1d8-4aa6-990e-da940875492c\""}, {Types.FixedType.ofLength(2), "\"111f\""}, {Types.BinaryType.get(), "\"0000ff\""}, {Types.DecimalType.of(9, 2), "123.45"}, {Types.ListType.ofOptional(1, Types.IntegerType.get()), "[1, 2, 3]"}, - {Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()), + {Types.MapType.ofOptional(2, 3, Types.IntegerType.get(), Types.StringType.get()), "{\"keys\": [1, 2], \"values\": [\"foo\", \"bar\"]}"}, {Types.StructType.of( - required(1, "f1", Types.IntegerType.get(), "doc"), - optional(2, "f2", Types.StringType.get(), "doc")), - "{\"1\": 1, \"2\": \"bar\"}"}, + required(4, "f1", Types.IntegerType.get()), + optional(5, "f2", Types.StringType.get())), + "{\"4\": 1, \"5\": \"bar\"}"}, // deeply nested complex types - {Types.ListType.ofOptional(1, Types.StructType.of( - required(1, "f1", Types.IntegerType.get(), "doc"), - optional(2, "f2", Types.StringType.get(), "doc"))), "[{\"1\": 1, \"2\": \"bar\"}, {\"1\": 2, \"2\": " + + {Types.ListType.ofOptional(6, Types.StructType.of( + required(7, "f1", Types.IntegerType.get()), + optional(8, "f2", Types.StringType.get()))), "[{\"7\": 1, \"8\": \"bar\"}, {\"7\": 2, \"8\": " + "\"foo\"}]"}, - {Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StructType.of( - required(1, "f1", Types.IntegerType.get(), "doc"), - optional(2, "f2", Types.StringType.get(), "doc"))), - "{\"keys\": [1, 2], \"values\": [{\"1\": 1, \"2\": \"bar\"}, {\"1\": 2, \"2\": \"foo\"}]}"}, + {Types.MapType.ofOptional(9, 10, Types.IntegerType.get(), Types.StructType.of( + required(11, "f1", Types.IntegerType.get()), + optional(12, "f2", Types.StringType.get()))), + "{\"keys\": [1, 2], \"values\": [{\"11\": 1, \"12\": \"bar\"}, {\"11\": 2, \"12\": \"foo\"}]}"}, {Types.StructType.of( - required(1, "f1", Types.StructType.of( - optional(2, "ff1", Types.IntegerType.get(), "doc"), - optional(3, "ff2", Types.StringType.get(), "doc")), "doc"), - optional(4, "f2", Types.StructType.of( - optional(5, "ff1", Types.StringType.get(), "doc"), - optional(6, "ff2", Types.IntegerType.get(), "doc")), "doc")), - "{\"1\": {\"2\": 1, \"3\": \"bar\"}, \"4\": {\"5\": \"bar\", \"6\": 1}}"}, + required(13, "f1", Types.StructType.of( + optional(14, "ff1", Types.IntegerType.get()), + optional(15, "ff2", Types.StringType.get()))), + optional(16, "f2", Types.StructType.of( + optional(17, "ff1", Types.StringType.get()), + optional(18, "ff2", Types.IntegerType.get())))), + "{\"13\": {\"14\": 1, \"15\": \"bar\"}, \"16\": {\"17\": \"bar\", \"18\": 1}}"}, }; for (Object[] typeWithDefault : typesWithDefaults) { @@ -82,14 +81,7 @@ public void testValidDefaults() throws IOException { String defaultValue = (String) typeWithDefault[1]; String roundTripDefaultValue = defaultValueParseAndUnParseRoundTrip(type, defaultValue); - // Only if the type is a timestampWithZone type, the round-trip default value will always be standardized to the - // UTC time zone, which might be different in the original value, but they should represent the same instant - if (type.typeId() == Type.TypeID.TIMESTAMP && ((Types.TimestampType) type).shouldAdjustToUTC()) { - Assert.assertTrue(OffsetDateTime.parse(JsonUtil.mapper().readTree(defaultValue).textValue()) - .isEqual(OffsetDateTime.parse(JsonUtil.mapper().readTree(roundTripDefaultValue).textValue()))); - } else { - jsonStringEquals(defaultValue.toLowerCase(Locale.ROOT), roundTripDefaultValue.toLowerCase(Locale.ROOT)); - } + jsonStringEquals(defaultValue.toLowerCase(Locale.ROOT), roundTripDefaultValue.toLowerCase(Locale.ROOT)); } } @@ -133,6 +125,16 @@ public void testInvalidDecimal() { Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a decimal(5, 2) value")); } + @Test + public void testInvalidTimestamptz() { + Type expectedType = Types.TimestampType.withZone(); + String defaultJson = "\"2007-12-03T10:15:30+01:00\""; + Exception exception = Assert.assertThrows( + IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a timestamptz value")); + } + // serialize to json and deserialize back should return the same result private static String defaultValueParseAndUnParseRoundTrip(Type type, String defaultValue) throws IOException { Object javaDefaultValue = DefaultValueParser.fromJson(type, defaultValue); From a908d53b5931e4b54b3ad4bf7bb6d84a81ef85f4 Mon Sep 17 00:00:00 2001 From: Raymond Zhang Date: Mon, 11 Jul 2022 16:45:55 -0700 Subject: [PATCH 16/23] Change decimal serialization to string --- .../java/org/apache/iceberg/DefaultValueParser.java | 13 ++++++++++--- .../org/apache/iceberg/TestDefaultValueParser.java | 4 ++-- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index ff0f37a3af55..dff46ff92d77 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -76,9 +76,16 @@ public static Object fromJson(Type type, JsonNode defaultValue) { "Cannot parse default as a %s value: %s", type, defaultValue); return defaultValue.doubleValue(); case DECIMAL: - Preconditions.checkArgument(defaultValue.isNumber(), + Preconditions.checkArgument(defaultValue.isTextual(), "Cannot parse default as a %s value: %s", type, defaultValue); - BigDecimal retDecimal = defaultValue.decimalValue(); + BigDecimal retDecimal; + try { + retDecimal = new BigDecimal(defaultValue.textValue()); + } catch (Exception e) { + throw new IllegalArgumentException(String.format( + "Cannot parse default as a %s value: %s", + type, defaultValue), e); + } Preconditions.checkArgument( retDecimal.scale() == ((Types.DecimalType) type).scale(), "Cannot parse default as a %s value: %s, the scale doesn't match", type, defaultValue); @@ -308,7 +315,7 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato defaultValue instanceof BigDecimal && ((BigDecimal) defaultValue).scale() == ((Types.DecimalType) type).scale(), "Invalid default %s value: %s", type, defaultValue); - generator.writeNumber((BigDecimal) defaultValue); + generator.writeString(((BigDecimal) defaultValue).toPlainString()); break; case LIST: Preconditions.checkArgument(defaultValue instanceof List, "Invalid default %s value: %s", type, defaultValue); diff --git a/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java b/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java index 55be90aab914..c9378cda6f69 100644 --- a/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java +++ b/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java @@ -49,7 +49,7 @@ public void testValidDefaults() throws IOException { {Types.UUIDType.get(), "\"eb26bdb1-a1d8-4aa6-990e-da940875492c\""}, {Types.FixedType.ofLength(2), "\"111f\""}, {Types.BinaryType.get(), "\"0000ff\""}, - {Types.DecimalType.of(9, 2), "123.45"}, + {Types.DecimalType.of(9, 4), "\"123.4500\""}, {Types.ListType.ofOptional(1, Types.IntegerType.get()), "[1, 2, 3]"}, {Types.MapType.ofOptional(2, 3, Types.IntegerType.get(), Types.StringType.get()), "{\"keys\": [1, 2], \"values\": [\"foo\", \"bar\"]}"}, @@ -136,7 +136,7 @@ public void testInvalidTimestamptz() { } // serialize to json and deserialize back should return the same result - private static String defaultValueParseAndUnParseRoundTrip(Type type, String defaultValue) throws IOException { + private static String defaultValueParseAndUnParseRoundTrip(Type type, String defaultValue) { Object javaDefaultValue = DefaultValueParser.fromJson(type, defaultValue); return DefaultValueParser.toJson(type, javaDefaultValue); } From 0e063f37a5786475b1c7c323fe39c6e3073d907a Mon Sep 17 00:00:00 2001 From: Raymond Zhang Date: Fri, 15 Jul 2022 13:00:58 -0700 Subject: [PATCH 17/23] Address comments --- .../apache/iceberg/DefaultValueParser.java | 30 +++++++++---------- .../org/apache/iceberg/util/DateTimeUtil.java | 8 ++--- .../iceberg/TestDefaultValueParser.java | 22 +++++++------- 3 files changed, 30 insertions(+), 30 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index dff46ff92d77..004d50d34bd3 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -81,7 +81,7 @@ public static Object fromJson(Type type, JsonNode defaultValue) { BigDecimal retDecimal; try { retDecimal = new BigDecimal(defaultValue.textValue()); - } catch (Exception e) { + } catch (NumberFormatException e) { throw new IllegalArgumentException(String.format( "Cannot parse default as a %s value: %s", type, defaultValue), e); @@ -118,7 +118,7 @@ public static Object fromJson(Type type, JsonNode defaultValue) { "Cannot parse default as a %s value: %s", type, defaultValue); if (((Types.TimestampType) type).shouldAdjustToUTC()) { String timestampTz = defaultValue.textValue(); - Preconditions.checkArgument(DateTimeUtil.timestamptzIsOfUTCZone(timestampTz), + Preconditions.checkArgument(DateTimeUtil.isUTCTimestamptz(timestampTz), "Cannot parse default as a %s value: %s, timezone must be UTC", type, defaultValue); return DateTimeUtil.isoTimestamptzToMicros(timestampTz); @@ -274,9 +274,9 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato Preconditions.checkArgument( defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue); if (((Types.TimestampType) type).shouldAdjustToUTC()) { - generator.writeString(DateTimeUtil.microsToIsoDateTimeTz((Long) defaultValue)); + generator.writeString(DateTimeUtil.microsToIsoTimestamptz((Long) defaultValue)); } else { - generator.writeString(DateTimeUtil.microsToIsoDateTime((Long) defaultValue)); + generator.writeString(DateTimeUtil.microsToIsoTimestamp((Long) defaultValue)); } break; case STRING: @@ -291,19 +291,12 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato case FIXED: Preconditions.checkArgument( defaultValue instanceof ByteBuffer, "Invalid default %s value: %s", type, defaultValue); - ByteBuffer byteBufferDefaultValue = (ByteBuffer) defaultValue; - Preconditions.checkArgument( - byteBufferDefaultValue.hasArray() && byteBufferDefaultValue.arrayOffset() == 0 && - byteBufferDefaultValue.position() == 0, - "Invalid default %s value: %s, not a valid bytebuffer representation", - type, defaultValue); - int actualLength = byteBufferDefaultValue.remaining(); + ByteBuffer byteBufferValue = (ByteBuffer) defaultValue; int expectedLength = ((Types.FixedType) type).length(); - Preconditions.checkArgument(actualLength == expectedLength, + Preconditions.checkArgument(byteBufferValue.remaining() == expectedLength, "Invalid default %s value: %s, incorrect length: %s", - type, defaultValue, actualLength); - String fixedString = BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) defaultValue))); - generator.writeString(fixedString); + type, defaultValue, byteBufferValue.remaining()); + generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) defaultValue)))); break; case BINARY: Preconditions.checkArgument( @@ -315,7 +308,12 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato defaultValue instanceof BigDecimal && ((BigDecimal) defaultValue).scale() == ((Types.DecimalType) type).scale(), "Invalid default %s value: %s", type, defaultValue); - generator.writeString(((BigDecimal) defaultValue).toPlainString()); + BigDecimal decimalValue = (BigDecimal) defaultValue; + if (decimalValue.scale() >= 0) { + generator.writeString(decimalValue.toPlainString()); + } else { + generator.writeString(decimalValue.toString()); + } break; case LIST: Preconditions.checkArgument(defaultValue instanceof List, "Invalid default %s value: %s", type, defaultValue); diff --git a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java index d0e9242bf2b4..89628555dae7 100644 --- a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java @@ -92,12 +92,12 @@ public static String microsToIsoTime(long micros) { return timeFromMicros(micros).format(DateTimeFormatter.ISO_LOCAL_TIME); } - public static String microsToIsoDateTimeTz(long micros) { + public static String microsToIsoTimestamptz(long micros) { LocalDateTime localDateTime = timestampFromMicros(micros); return localDateTime.atOffset(ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME); } - public static String microsToIsoDateTime(long micros) { + public static String microsToIsoTimestamp(long micros) { LocalDateTime localDateTime = timestampFromMicros(micros); return localDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); } @@ -114,9 +114,9 @@ public static long isoTimestamptzToMicros(String timestampString) { return microsFromTimestamptz(OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME)); } - public static boolean timestamptzIsOfUTCZone(String timestampString) { + public static boolean isUTCTimestamptz(String timestampString) { OffsetDateTime offsetDateTime = OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME); - return offsetDateTime.getOffset().equals(ZoneOffset.UTC) && timestampString.endsWith("Z"); + return offsetDateTime.getOffset().equals(ZoneOffset.UTC); } public static long isoTimestampToMicros(String timestampString) { diff --git a/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java b/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java index c9378cda6f69..1290219882b6 100644 --- a/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java +++ b/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java @@ -35,21 +35,23 @@ public class TestDefaultValueParser { @Test public void testValidDefaults() throws IOException { Object[][] typesWithDefaults = new Object[][] { - {Types.BooleanType.get(), "null"}, - {Types.BooleanType.get(), "true"}, - {Types.IntegerType.get(), "1"}, - {Types.LongType.get(), "9999999"}, - {Types.FloatType.get(), "1.23"}, - {Types.DoubleType.get(), "123.456"}, - {Types.DateType.get(), "\"2007-12-03\""}, - {Types.TimeType.get(), "\"10:15:30\""}, - {Types.TimestampType.withoutZone(), "\"2007-12-03T10:15:30\""}, - {Types.TimestampType.withZone(), "\"2007-12-03T10:15:30Z\""}, + // {Types.BooleanType.get(), "null"}, + // {Types.BooleanType.get(), "true"}, + // {Types.IntegerType.get(), "1"}, + // {Types.LongType.get(), "9999999"}, + // {Types.FloatType.get(), "1.23"}, + // {Types.DoubleType.get(), "123.456"}, + // {Types.DateType.get(), "\"2007-12-03\""}, + // {Types.TimeType.get(), "\"10:15:30\""}, + // {Types.TimestampType.withoutZone(), "\"2007-12-03T10:15:30\""}, + {Types.TimestampType.withZone(), "\"2007-12-03T10:15:30+00:00\""}, {Types.StringType.get(), "\"foo\""}, {Types.UUIDType.get(), "\"eb26bdb1-a1d8-4aa6-990e-da940875492c\""}, {Types.FixedType.ofLength(2), "\"111f\""}, {Types.BinaryType.get(), "\"0000ff\""}, {Types.DecimalType.of(9, 4), "\"123.4500\""}, + {Types.DecimalType.of(9, 0), "\"2\""}, + {Types.DecimalType.of(9, -20), "\"2E+20\""}, {Types.ListType.ofOptional(1, Types.IntegerType.get()), "[1, 2, 3]"}, {Types.MapType.ofOptional(2, 3, Types.IntegerType.get(), Types.StringType.get()), "{\"keys\": [1, 2], \"values\": [\"foo\", \"bar\"]}"}, From 94421c3a28b6f377e743fdf6ba8c175369cda5c5 Mon Sep 17 00:00:00 2001 From: Raymond Zhang Date: Fri, 15 Jul 2022 13:03:18 -0700 Subject: [PATCH 18/23] Use custom formatter for serializing timestamptz value --- .../org/apache/iceberg/util/DateTimeUtil.java | 9 ++++++++- .../apache/iceberg/TestDefaultValueParser.java | 18 +++++++++--------- 2 files changed, 17 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java index 89628555dae7..554b9ed027fd 100644 --- a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java @@ -24,6 +24,8 @@ import java.time.LocalTime; import java.time.OffsetDateTime; import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; import java.time.temporal.ChronoUnit; public class DateTimeUtil { @@ -94,7 +96,12 @@ public static String microsToIsoTime(long micros) { public static String microsToIsoTimestamptz(long micros) { LocalDateTime localDateTime = timestampFromMicros(micros); - return localDateTime.atOffset(ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME); + DateTimeFormatter customNoOffsetTextISOFormatter = new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) + .appendOffset("+HH:MM:ss", "+00:00") + .toFormatter(); + return localDateTime.atOffset(ZoneOffset.UTC).format(customNoOffsetTextISOFormatter); } public static String microsToIsoTimestamp(long micros) { diff --git a/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java b/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java index 1290219882b6..d1e70593f398 100644 --- a/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java +++ b/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java @@ -35,15 +35,15 @@ public class TestDefaultValueParser { @Test public void testValidDefaults() throws IOException { Object[][] typesWithDefaults = new Object[][] { - // {Types.BooleanType.get(), "null"}, - // {Types.BooleanType.get(), "true"}, - // {Types.IntegerType.get(), "1"}, - // {Types.LongType.get(), "9999999"}, - // {Types.FloatType.get(), "1.23"}, - // {Types.DoubleType.get(), "123.456"}, - // {Types.DateType.get(), "\"2007-12-03\""}, - // {Types.TimeType.get(), "\"10:15:30\""}, - // {Types.TimestampType.withoutZone(), "\"2007-12-03T10:15:30\""}, + {Types.BooleanType.get(), "null"}, + {Types.BooleanType.get(), "true"}, + {Types.IntegerType.get(), "1"}, + {Types.LongType.get(), "9999999"}, + {Types.FloatType.get(), "1.23"}, + {Types.DoubleType.get(), "123.456"}, + {Types.DateType.get(), "\"2007-12-03\""}, + {Types.TimeType.get(), "\"10:15:30\""}, + {Types.TimestampType.withoutZone(), "\"2007-12-03T10:15:30\""}, {Types.TimestampType.withZone(), "\"2007-12-03T10:15:30+00:00\""}, {Types.StringType.get(), "\"foo\""}, {Types.UUIDType.get(), "\"eb26bdb1-a1d8-4aa6-990e-da940875492c\""}, From 08e740ef0f57b4866a2b967a1058377583f83f58 Mon Sep 17 00:00:00 2001 From: Raymond Zhang Date: Fri, 15 Jul 2022 13:31:55 -0700 Subject: [PATCH 19/23] Change default spec according to the changes in this PR --- format/spec.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/format/spec.md b/format/spec.md index c5d312865ea7..b243d2e4f22b 100644 --- a/format/spec.md +++ b/format/spec.md @@ -1162,15 +1162,15 @@ This serialization scheme is for storing single values as individual binary valu | **`long`** | **`JSON long`** | `34` | | | **`float`** | **`JSON number`** | `1.0` | | | **`double`** | **`JSON number`** | `1.0` | | -| **`decimal(P,S)`** | **`JSON number`** | `14.20` | Stores the decimal as a number with S places after the decimal | +| **`decimal(P,S)`** | **`JSON string`** | `"14.20"`, `"2E+20"` | Stores the decimal string as-is | | **`date`** | **`JSON string`** | `"2017-11-16"` | Stores ISO-8601 standard date | | **`time`** | **`JSON string`** | `"22:31:08.123456"` | Stores ISO-8601 standard time with microsecond precision | | **`timestamp`** | **`JSON string`** | `"2017-11-16T22:31:08.123456"` | Stores ISO-8601 standard timestamp with microsecond precision; must not include a zone offset | -| **`timestamptz`** | **`JSON string`** | `"2017-11-16T22:31:08.123456-07:00"` | Stores ISO-8601 standard timestamp with microsecond precision; must include a zone offset | +| **`timestamptz`** | **`JSON string`** | `"2017-11-16T22:31:08.123456+00:00"` | Stores ISO-8601 standard timestamp with microsecond precision; must include a zone offset and it must be '+00:00' | | **`string`** | **`JSON string`** | `"iceberg"` | | | **`uuid`** | **`JSON string`** | `"f79c3e09-677c-4bbd-a479-3f349cb785e7"` | Stores the lowercase uuid string | -| **`fixed(L)`** | **`JSON string`** | `"0x00010203"` | Stored as a hexadecimal string, prefixed by `0x` | -| **`binary`** | **`JSON string`** | `"0x00010203"` | Stored as a hexadecimal string, prefixed by `0x` | +| **`fixed(L)`** | **`JSON string`** | `"000102ff"` | Stored as a hexadecimal string | +| **`binary`** | **`JSON string`** | `"000102ff"` | Stored as a hexadecimal string | | **`struct`** | **`JSON object by field ID`** | `{"1": 1, "2": "bar"}` | Stores struct fields using the field ID as the JSON field name; field values are stored using this JSON single-value format | | **`list`** | **`JSON array of values`** | `[1, 2, 3]` | Stores a JSON array of values that are serialized using this JSON single-value format | | **`map`** | **`JSON object of key and value arrays`** | `{ "keys": ["a", "b"], "values": [1, 2] }` | Stores arrays of keys and values; individual keys and values are serialized using this JSON single-value format | From e85de0917920f387ad90b37ab73851958f393cb2 Mon Sep 17 00:00:00 2001 From: Raymond Zhang Date: Fri, 15 Jul 2022 13:36:58 -0700 Subject: [PATCH 20/23] Minor fix --- core/src/main/java/org/apache/iceberg/DefaultValueParser.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index 004d50d34bd3..b3f55c235af9 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -296,12 +296,12 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato Preconditions.checkArgument(byteBufferValue.remaining() == expectedLength, "Invalid default %s value: %s, incorrect length: %s", type, defaultValue, byteBufferValue.remaining()); - generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) defaultValue)))); + generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(byteBufferValue))); break; case BINARY: Preconditions.checkArgument( defaultValue instanceof ByteBuffer, "Invalid default %s value: %s", type, defaultValue); - generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) defaultValue)))); + generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray((ByteBuffer) defaultValue))); break; case DECIMAL: Preconditions.checkArgument( From 16e95598c832da9e64934e40a0ace9430498f037 Mon Sep 17 00:00:00 2001 From: Raymond Zhang Date: Mon, 1 Aug 2022 14:02:11 -0700 Subject: [PATCH 21/23] Fix all comments --- .../main/java/org/apache/iceberg/DefaultValueParser.java | 6 +++--- format/spec.md | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index b3f55c235af9..20d00aa8e4d1 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -119,7 +119,7 @@ public static Object fromJson(Type type, JsonNode defaultValue) { if (((Types.TimestampType) type).shouldAdjustToUTC()) { String timestampTz = defaultValue.textValue(); Preconditions.checkArgument(DateTimeUtil.isUTCTimestamptz(timestampTz), - "Cannot parse default as a %s value: %s, timezone must be UTC", + "Cannot parse default as a %s value: %s, offset must be +00:00", type, defaultValue); return DateTimeUtil.isoTimestamptzToMicros(timestampTz); } else { @@ -294,8 +294,8 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato ByteBuffer byteBufferValue = (ByteBuffer) defaultValue; int expectedLength = ((Types.FixedType) type).length(); Preconditions.checkArgument(byteBufferValue.remaining() == expectedLength, - "Invalid default %s value: %s, incorrect length: %s", - type, defaultValue, byteBufferValue.remaining()); + "Invalid default %s value, incorrect length: %s", + type, byteBufferValue.remaining()); generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(byteBufferValue))); break; case BINARY: diff --git a/format/spec.md b/format/spec.md index b243d2e4f22b..2ce1e6c9b513 100644 --- a/format/spec.md +++ b/format/spec.md @@ -1162,7 +1162,7 @@ This serialization scheme is for storing single values as individual binary valu | **`long`** | **`JSON long`** | `34` | | | **`float`** | **`JSON number`** | `1.0` | | | **`double`** | **`JSON number`** | `1.0` | | -| **`decimal(P,S)`** | **`JSON string`** | `"14.20"`, `"2E+20"` | Stores the decimal string as-is | +| **`decimal(P,S)`** | **`JSON string`** | `"14.20"`, `"2E+20"` | Stores the string representation of the decimal value, specifically, for values with a positive scale, the number of digits to the right of the decimal point is used to indicate scale, for values with a negative scale, the scientific notation is used and the exponent must equal the negated scale | | **`date`** | **`JSON string`** | `"2017-11-16"` | Stores ISO-8601 standard date | | **`time`** | **`JSON string`** | `"22:31:08.123456"` | Stores ISO-8601 standard time with microsecond precision | | **`timestamp`** | **`JSON string`** | `"2017-11-16T22:31:08.123456"` | Stores ISO-8601 standard timestamp with microsecond precision; must not include a zone offset | From 3b855ad0fd2fb191fa7ca8da166be2013cfbf350 Mon Sep 17 00:00:00 2001 From: Raymond Zhang Date: Mon, 1 Aug 2022 14:17:37 -0700 Subject: [PATCH 22/23] Refactor variable name --- core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java index 554b9ed027fd..8914e3b9dbd5 100644 --- a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java @@ -96,12 +96,12 @@ public static String microsToIsoTime(long micros) { public static String microsToIsoTimestamptz(long micros) { LocalDateTime localDateTime = timestampFromMicros(micros); - DateTimeFormatter customNoOffsetTextISOFormatter = new DateTimeFormatterBuilder() + DateTimeFormatter zeroOffsetFormatter = new DateTimeFormatterBuilder() .parseCaseInsensitive() .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) .appendOffset("+HH:MM:ss", "+00:00") .toFormatter(); - return localDateTime.atOffset(ZoneOffset.UTC).format(customNoOffsetTextISOFormatter); + return localDateTime.atOffset(ZoneOffset.UTC).format(zeroOffsetFormatter); } public static String microsToIsoTimestamp(long micros) { From 64d949a19d70d5471273ca6ad7b3554b05dc447e Mon Sep 17 00:00:00 2001 From: Raymond Zhang Date: Mon, 1 Aug 2022 14:32:47 -0700 Subject: [PATCH 23/23] SpotlessApply --- .../apache/iceberg/DefaultValueParser.java | 168 +++++++++++------- .../org/apache/iceberg/util/DateTimeUtil.java | 20 ++- .../iceberg/TestDefaultValueParser.java | 164 ++++++++++------- 3 files changed, 214 insertions(+), 138 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java index 20d00aa8e4d1..1f693303c965 100644 --- a/core/src/main/java/org/apache/iceberg/DefaultValueParser.java +++ b/core/src/main/java/org/apache/iceberg/DefaultValueParser.java @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.iceberg; import com.fasterxml.jackson.core.JsonGenerator; @@ -43,8 +42,7 @@ import org.apache.iceberg.util.JsonUtil; public class DefaultValueParser { - private DefaultValueParser() { - } + private DefaultValueParser() {} private static final String KEYS = "keys"; private static final String VALUES = "values"; @@ -56,89 +54,112 @@ public static Object fromJson(Type type, JsonNode defaultValue) { switch (type.typeId()) { case BOOLEAN: - Preconditions.checkArgument(defaultValue.isBoolean(), - "Cannot parse default as a %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue.isBoolean(), "Cannot parse default as a %s value: %s", type, defaultValue); return defaultValue.booleanValue(); case INTEGER: - Preconditions.checkArgument(defaultValue.isIntegralNumber() && defaultValue.canConvertToInt(), - "Cannot parse default as a %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue.isIntegralNumber() && defaultValue.canConvertToInt(), + "Cannot parse default as a %s value: %s", + type, + defaultValue); return defaultValue.intValue(); case LONG: - Preconditions.checkArgument(defaultValue.isIntegralNumber() && defaultValue.canConvertToLong(), - "Cannot parse default as a %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue.isIntegralNumber() && defaultValue.canConvertToLong(), + "Cannot parse default as a %s value: %s", + type, + defaultValue); return defaultValue.longValue(); case FLOAT: - Preconditions.checkArgument(defaultValue.isFloatingPointNumber(), - "Cannot parse default as a %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue.isFloatingPointNumber(), + "Cannot parse default as a %s value: %s", + type, + defaultValue); return defaultValue.floatValue(); case DOUBLE: - Preconditions.checkArgument(defaultValue.isFloatingPointNumber(), - "Cannot parse default as a %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue.isFloatingPointNumber(), + "Cannot parse default as a %s value: %s", + type, + defaultValue); return defaultValue.doubleValue(); case DECIMAL: - Preconditions.checkArgument(defaultValue.isTextual(), - "Cannot parse default as a %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue.isTextual(), "Cannot parse default as a %s value: %s", type, defaultValue); BigDecimal retDecimal; try { retDecimal = new BigDecimal(defaultValue.textValue()); } catch (NumberFormatException e) { - throw new IllegalArgumentException(String.format( - "Cannot parse default as a %s value: %s", - type, defaultValue), e); + throw new IllegalArgumentException( + String.format("Cannot parse default as a %s value: %s", type, defaultValue), e); } Preconditions.checkArgument( retDecimal.scale() == ((Types.DecimalType) type).scale(), - "Cannot parse default as a %s value: %s, the scale doesn't match", type, defaultValue); + "Cannot parse default as a %s value: %s, the scale doesn't match", + type, + defaultValue); return retDecimal; case STRING: - Preconditions.checkArgument(defaultValue.isTextual(), - "Cannot parse default as a %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue.isTextual(), "Cannot parse default as a %s value: %s", type, defaultValue); return defaultValue.textValue(); case UUID: - Preconditions.checkArgument(defaultValue.isTextual() && defaultValue.textValue().length() == 36, - "Cannot parse default as a %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue.isTextual() && defaultValue.textValue().length() == 36, + "Cannot parse default as a %s value: %s", + type, + defaultValue); UUID uuid; try { uuid = UUID.fromString(defaultValue.textValue()); } catch (IllegalArgumentException e) { - throw new IllegalArgumentException(String.format("Cannot parse default as a %s value: %s", type, - defaultValue), e); + throw new IllegalArgumentException( + String.format("Cannot parse default as a %s value: %s", type, defaultValue), e); } return uuid; case DATE: - Preconditions.checkArgument(defaultValue.isTextual(), - "Cannot parse default as a %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue.isTextual(), "Cannot parse default as a %s value: %s", type, defaultValue); return DateTimeUtil.isoDateToDays(defaultValue.textValue()); case TIME: - Preconditions.checkArgument(defaultValue.isTextual(), - "Cannot parse default as a %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue.isTextual(), "Cannot parse default as a %s value: %s", type, defaultValue); return DateTimeUtil.isoTimeToMicros(defaultValue.textValue()); case TIMESTAMP: - Preconditions.checkArgument(defaultValue.isTextual(), - "Cannot parse default as a %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue.isTextual(), "Cannot parse default as a %s value: %s", type, defaultValue); if (((Types.TimestampType) type).shouldAdjustToUTC()) { String timestampTz = defaultValue.textValue(); - Preconditions.checkArgument(DateTimeUtil.isUTCTimestamptz(timestampTz), + Preconditions.checkArgument( + DateTimeUtil.isUTCTimestamptz(timestampTz), "Cannot parse default as a %s value: %s, offset must be +00:00", - type, defaultValue); + type, + defaultValue); return DateTimeUtil.isoTimestamptzToMicros(timestampTz); } else { return DateTimeUtil.isoTimestampToMicros(defaultValue.textValue()); } case FIXED: Preconditions.checkArgument( - defaultValue.isTextual(), - "Cannot parse default as a %s value: %s", type, defaultValue); + defaultValue.isTextual(), "Cannot parse default as a %s value: %s", type, defaultValue); int defaultLength = defaultValue.textValue().length(); int fixedLength = ((Types.FixedType) type).length(); - Preconditions.checkArgument(defaultLength == fixedLength * 2, - "Cannot parse default %s value: %s, incorrect length: %s", type, defaultValue, defaultLength); - byte[] fixedBytes = BaseEncoding.base16().decode(defaultValue.textValue().toUpperCase(Locale.ROOT)); + Preconditions.checkArgument( + defaultLength == fixedLength * 2, + "Cannot parse default %s value: %s, incorrect length: %s", + type, + defaultValue, + defaultLength); + byte[] fixedBytes = + BaseEncoding.base16().decode(defaultValue.textValue().toUpperCase(Locale.ROOT)); return ByteBuffer.wrap(fixedBytes); case BINARY: - Preconditions.checkArgument(defaultValue.isTextual(), - "Cannot parse default as a %s value: %s", type, defaultValue); - byte[] binaryBytes = BaseEncoding.base16().decode(defaultValue.textValue().toUpperCase(Locale.ROOT)); + Preconditions.checkArgument( + defaultValue.isTextual(), "Cannot parse default as a %s value: %s", type, defaultValue); + byte[] binaryBytes = + BaseEncoding.base16().decode(defaultValue.textValue().toUpperCase(Locale.ROOT)); return ByteBuffer.wrap(binaryBytes); case LIST: return listFromJson(type, defaultValue); @@ -152,8 +173,8 @@ public static Object fromJson(Type type, JsonNode defaultValue) { } private static StructLike structFromJson(Type type, JsonNode defaultValue) { - Preconditions.checkArgument(defaultValue.isObject(), - "Cannot parse default as a %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue.isObject(), "Cannot parse default as a %s value: %s", type, defaultValue); Types.StructType struct = type.asStructType(); StructLike defaultRecord = GenericRecord.create(struct); @@ -170,15 +191,18 @@ private static StructLike structFromJson(Type type, JsonNode defaultValue) { private static Map mapFromJson(Type type, JsonNode defaultValue) { Preconditions.checkArgument( - defaultValue.isObject() && defaultValue.has(KEYS) && defaultValue.has(VALUES) && - defaultValue.get(KEYS).isArray() && defaultValue.get(VALUES).isArray(), + defaultValue.isObject() + && defaultValue.has(KEYS) + && defaultValue.has(VALUES) + && defaultValue.get(KEYS).isArray() + && defaultValue.get(VALUES).isArray(), "Cannot parse %s to a %s value", - defaultValue, type); + defaultValue, + type); JsonNode keys = defaultValue.get(KEYS); JsonNode values = defaultValue.get(VALUES); Preconditions.checkArgument( - keys.size() == values.size(), - "Cannot parse default as a %s value: %s", type, defaultValue); + keys.size() == values.size(), "Cannot parse default as a %s value: %s", type, defaultValue); ImmutableMap.Builder mapBuilder = ImmutableMap.builder(); @@ -195,8 +219,8 @@ private static Map mapFromJson(Type type, JsonNode defaultValue) } private static List listFromJson(Type type, JsonNode defaultValue) { - Preconditions.checkArgument(defaultValue.isArray(), - "Cannot parse default as a %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue.isArray(), "Cannot parse default as a %s value: %s", type, defaultValue); Type elementType = type.asListType().elementType(); return Lists.newArrayList(Iterables.transform(defaultValue, e -> fromJson(elementType, e))); } @@ -230,6 +254,7 @@ public static String toJson(Type type, Object defaultValue, boolean pretty) { } } + @SuppressWarnings("checkstyle:MethodLength") public static void toJson(Type type, Object defaultValue, JsonGenerator generator) throws IOException { if (defaultValue == null) { @@ -249,15 +274,18 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato generator.writeNumber((Integer) defaultValue); break; case LONG: - Preconditions.checkArgument(defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue); generator.writeNumber((Long) defaultValue); break; case FLOAT: - Preconditions.checkArgument(defaultValue instanceof Float, "Invalid default %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue instanceof Float, "Invalid default %s value: %s", type, defaultValue); generator.writeNumber((Float) defaultValue); break; case DOUBLE: - Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue); generator.writeNumber((Double) defaultValue); break; case DATE: @@ -281,11 +309,15 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato break; case STRING: Preconditions.checkArgument( - defaultValue instanceof CharSequence, "Invalid default %s value: %s", type, defaultValue); + defaultValue instanceof CharSequence, + "Invalid default %s value: %s", + type, + defaultValue); generator.writeString(((CharSequence) defaultValue).toString()); break; case UUID: - Preconditions.checkArgument(defaultValue instanceof UUID, "Invalid default %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue instanceof UUID, "Invalid default %s value: %s", type, defaultValue); generator.writeString(defaultValue.toString()); break; case FIXED: @@ -293,21 +325,27 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato defaultValue instanceof ByteBuffer, "Invalid default %s value: %s", type, defaultValue); ByteBuffer byteBufferValue = (ByteBuffer) defaultValue; int expectedLength = ((Types.FixedType) type).length(); - Preconditions.checkArgument(byteBufferValue.remaining() == expectedLength, + Preconditions.checkArgument( + byteBufferValue.remaining() == expectedLength, "Invalid default %s value, incorrect length: %s", - type, byteBufferValue.remaining()); - generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(byteBufferValue))); + type, + byteBufferValue.remaining()); + generator.writeString( + BaseEncoding.base16().encode(ByteBuffers.toByteArray(byteBufferValue))); break; case BINARY: Preconditions.checkArgument( defaultValue instanceof ByteBuffer, "Invalid default %s value: %s", type, defaultValue); - generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray((ByteBuffer) defaultValue))); + generator.writeString( + BaseEncoding.base16().encode(ByteBuffers.toByteArray((ByteBuffer) defaultValue))); break; case DECIMAL: Preconditions.checkArgument( - defaultValue instanceof BigDecimal && - ((BigDecimal) defaultValue).scale() == ((Types.DecimalType) type).scale(), - "Invalid default %s value: %s", type, defaultValue); + defaultValue instanceof BigDecimal + && ((BigDecimal) defaultValue).scale() == ((Types.DecimalType) type).scale(), + "Invalid default %s value: %s", + type, + defaultValue); BigDecimal decimalValue = (BigDecimal) defaultValue; if (decimalValue.scale() >= 0) { generator.writeString(decimalValue.toPlainString()); @@ -316,7 +354,8 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato } break; case LIST: - Preconditions.checkArgument(defaultValue instanceof List, "Invalid default %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue instanceof List, "Invalid default %s value: %s", type, defaultValue); List defaultList = (List) defaultValue; Type elementType = type.asListType().elementType(); generator.writeStartArray(); @@ -326,7 +365,8 @@ public static void toJson(Type type, Object defaultValue, JsonGenerator generato generator.writeEndArray(); break; case MAP: - Preconditions.checkArgument(defaultValue instanceof Map, "Invalid default %s value: %s", type, defaultValue); + Preconditions.checkArgument( + defaultValue instanceof Map, "Invalid default %s value: %s", type, defaultValue); Map defaultMap = (Map) defaultValue; Type keyType = type.asMapType().keyType(); Type valueType = type.asMapType().valueType(); diff --git a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java index 8914e3b9dbd5..29214ff8831d 100644 --- a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java @@ -96,11 +96,12 @@ public static String microsToIsoTime(long micros) { public static String microsToIsoTimestamptz(long micros) { LocalDateTime localDateTime = timestampFromMicros(micros); - DateTimeFormatter zeroOffsetFormatter = new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) - .appendOffset("+HH:MM:ss", "+00:00") - .toFormatter(); + DateTimeFormatter zeroOffsetFormatter = + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) + .appendOffset("+HH:MM:ss", "+00:00") + .toFormatter(); return localDateTime.atOffset(ZoneOffset.UTC).format(zeroOffsetFormatter); } @@ -118,15 +119,18 @@ public static long isoTimeToMicros(String timeString) { } public static long isoTimestamptzToMicros(String timestampString) { - return microsFromTimestamptz(OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME)); + return microsFromTimestamptz( + OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME)); } public static boolean isUTCTimestamptz(String timestampString) { - OffsetDateTime offsetDateTime = OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME); + OffsetDateTime offsetDateTime = + OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME); return offsetDateTime.getOffset().equals(ZoneOffset.UTC); } public static long isoTimestampToMicros(String timestampString) { - return microsFromTimestamp(LocalDateTime.parse(timestampString, DateTimeFormatter.ISO_LOCAL_DATE_TIME)); + return microsFromTimestamp( + LocalDateTime.parse(timestampString, DateTimeFormatter.ISO_LOCAL_DATE_TIME)); } } diff --git a/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java b/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java index d1e70593f398..eacc8c2fee56 100644 --- a/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java +++ b/core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java @@ -16,9 +16,11 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.iceberg; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + import java.io.IOException; import java.util.Locale; import org.apache.iceberg.types.Type; @@ -27,55 +29,75 @@ import org.junit.Assert; import org.junit.Test; -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; - public class TestDefaultValueParser { @Test public void testValidDefaults() throws IOException { - Object[][] typesWithDefaults = new Object[][] { - {Types.BooleanType.get(), "null"}, - {Types.BooleanType.get(), "true"}, - {Types.IntegerType.get(), "1"}, - {Types.LongType.get(), "9999999"}, - {Types.FloatType.get(), "1.23"}, - {Types.DoubleType.get(), "123.456"}, - {Types.DateType.get(), "\"2007-12-03\""}, - {Types.TimeType.get(), "\"10:15:30\""}, - {Types.TimestampType.withoutZone(), "\"2007-12-03T10:15:30\""}, - {Types.TimestampType.withZone(), "\"2007-12-03T10:15:30+00:00\""}, - {Types.StringType.get(), "\"foo\""}, - {Types.UUIDType.get(), "\"eb26bdb1-a1d8-4aa6-990e-da940875492c\""}, - {Types.FixedType.ofLength(2), "\"111f\""}, - {Types.BinaryType.get(), "\"0000ff\""}, - {Types.DecimalType.of(9, 4), "\"123.4500\""}, - {Types.DecimalType.of(9, 0), "\"2\""}, - {Types.DecimalType.of(9, -20), "\"2E+20\""}, - {Types.ListType.ofOptional(1, Types.IntegerType.get()), "[1, 2, 3]"}, - {Types.MapType.ofOptional(2, 3, Types.IntegerType.get(), Types.StringType.get()), - "{\"keys\": [1, 2], \"values\": [\"foo\", \"bar\"]}"}, - {Types.StructType.of( - required(4, "f1", Types.IntegerType.get()), - optional(5, "f2", Types.StringType.get())), - "{\"4\": 1, \"5\": \"bar\"}"}, - // deeply nested complex types - {Types.ListType.ofOptional(6, Types.StructType.of( - required(7, "f1", Types.IntegerType.get()), - optional(8, "f2", Types.StringType.get()))), "[{\"7\": 1, \"8\": \"bar\"}, {\"7\": 2, \"8\": " + - "\"foo\"}]"}, - {Types.MapType.ofOptional(9, 10, Types.IntegerType.get(), Types.StructType.of( - required(11, "f1", Types.IntegerType.get()), - optional(12, "f2", Types.StringType.get()))), - "{\"keys\": [1, 2], \"values\": [{\"11\": 1, \"12\": \"bar\"}, {\"11\": 2, \"12\": \"foo\"}]}"}, - {Types.StructType.of( - required(13, "f1", Types.StructType.of( - optional(14, "ff1", Types.IntegerType.get()), - optional(15, "ff2", Types.StringType.get()))), - optional(16, "f2", Types.StructType.of( - optional(17, "ff1", Types.StringType.get()), - optional(18, "ff2", Types.IntegerType.get())))), - "{\"13\": {\"14\": 1, \"15\": \"bar\"}, \"16\": {\"17\": \"bar\", \"18\": 1}}"}, + Object[][] typesWithDefaults = + new Object[][] { + {Types.BooleanType.get(), "null"}, + {Types.BooleanType.get(), "true"}, + {Types.IntegerType.get(), "1"}, + {Types.LongType.get(), "9999999"}, + {Types.FloatType.get(), "1.23"}, + {Types.DoubleType.get(), "123.456"}, + {Types.DateType.get(), "\"2007-12-03\""}, + {Types.TimeType.get(), "\"10:15:30\""}, + {Types.TimestampType.withoutZone(), "\"2007-12-03T10:15:30\""}, + {Types.TimestampType.withZone(), "\"2007-12-03T10:15:30+00:00\""}, + {Types.StringType.get(), "\"foo\""}, + {Types.UUIDType.get(), "\"eb26bdb1-a1d8-4aa6-990e-da940875492c\""}, + {Types.FixedType.ofLength(2), "\"111f\""}, + {Types.BinaryType.get(), "\"0000ff\""}, + {Types.DecimalType.of(9, 4), "\"123.4500\""}, + {Types.DecimalType.of(9, 0), "\"2\""}, + {Types.DecimalType.of(9, -20), "\"2E+20\""}, + {Types.ListType.ofOptional(1, Types.IntegerType.get()), "[1, 2, 3]"}, + { + Types.MapType.ofOptional(2, 3, Types.IntegerType.get(), Types.StringType.get()), + "{\"keys\": [1, 2], \"values\": [\"foo\", \"bar\"]}" + }, + { + Types.StructType.of( + required(4, "f1", Types.IntegerType.get()), + optional(5, "f2", Types.StringType.get())), + "{\"4\": 1, \"5\": \"bar\"}" + }, + // deeply nested complex types + { + Types.ListType.ofOptional( + 6, + Types.StructType.of( + required(7, "f1", Types.IntegerType.get()), + optional(8, "f2", Types.StringType.get()))), + "[{\"7\": 1, \"8\": \"bar\"}, {\"7\": 2, \"8\": " + "\"foo\"}]" + }, + { + Types.MapType.ofOptional( + 9, + 10, + Types.IntegerType.get(), + Types.StructType.of( + required(11, "f1", Types.IntegerType.get()), + optional(12, "f2", Types.StringType.get()))), + "{\"keys\": [1, 2], \"values\": [{\"11\": 1, \"12\": \"bar\"}, {\"11\": 2, \"12\": \"foo\"}]}" + }, + { + Types.StructType.of( + required( + 13, + "f1", + Types.StructType.of( + optional(14, "ff1", Types.IntegerType.get()), + optional(15, "ff2", Types.StringType.get()))), + optional( + 16, + "f2", + Types.StructType.of( + optional(17, "ff1", Types.StringType.get()), + optional(18, "ff2", Types.IntegerType.get())))), + "{\"13\": {\"14\": 1, \"15\": \"bar\"}, \"16\": {\"17\": \"bar\", \"18\": 1}}" + }, }; for (Object[] typeWithDefault : typesWithDefaults) { @@ -83,7 +105,8 @@ public void testValidDefaults() throws IOException { String defaultValue = (String) typeWithDefault[1]; String roundTripDefaultValue = defaultValueParseAndUnParseRoundTrip(type, defaultValue); - jsonStringEquals(defaultValue.toLowerCase(Locale.ROOT), roundTripDefaultValue.toLowerCase(Locale.ROOT)); + jsonStringEquals( + defaultValue.toLowerCase(Locale.ROOT), roundTripDefaultValue.toLowerCase(Locale.ROOT)); } } @@ -91,9 +114,10 @@ public void testValidDefaults() throws IOException { public void testInvalidFixed() { Type expectedType = Types.FixedType.ofLength(2); String defaultJson = "\"111ff\""; - Exception exception = Assert.assertThrows( - IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Exception exception = + Assert.assertThrows( + IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default fixed[2] value")); } @@ -101,40 +125,48 @@ public void testInvalidFixed() { public void testInvalidUUID() { Type expectedType = Types.UUIDType.get(); String defaultJson = "\"eb26bdb1-a1d8-4aa6-990e-da940875492c-abcde\""; - Exception exception = Assert.assertThrows( - IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Exception exception = + Assert.assertThrows( + IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a uuid value")); } @Test public void testInvalidMap() { - Type expectedType = Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()); + Type expectedType = + Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()); String defaultJson = "{\"keys\": [1, 2, 3], \"values\": [\"foo\", \"bar\"]}"; - Exception exception = Assert.assertThrows( - IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a map value")); + Exception exception = + Assert.assertThrows( + IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue( + exception.getMessage().startsWith("Cannot parse default as a map value")); } @Test public void testInvalidDecimal() { Type expectedType = Types.DecimalType.of(5, 2); String defaultJson = "123.456"; - Exception exception = Assert.assertThrows( - IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a decimal(5, 2) value")); + Exception exception = + Assert.assertThrows( + IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue( + exception.getMessage().startsWith("Cannot parse default as a decimal(5, 2) value")); } @Test public void testInvalidTimestamptz() { Type expectedType = Types.TimestampType.withZone(); String defaultJson = "\"2007-12-03T10:15:30+01:00\""; - Exception exception = Assert.assertThrows( - IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a timestamptz value")); + Exception exception = + Assert.assertThrows( + IllegalArgumentException.class, + () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); + Assert.assertTrue( + exception.getMessage().startsWith("Cannot parse default as a timestamptz value")); } // serialize to json and deserialize back should return the same result