You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/05/25 17:13:56 UTC

[GitHub] [iceberg] rzhang10 opened a new pull request, #4871: Core: Implement default value parsing and unparsing

rzhang10 opened a new pull request, #4871:
URL: https://github.com/apache/iceberg/pull/4871

   This PR adds the default value parsing and un-parsing (serialization) from/to its JSON representation, as per spec #4301 .
   
   @rdblue Please review, cc @wmoustafa


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r908724681


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isFloatingPointNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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, 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);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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));
+        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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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));
+    }
+  }
+
+  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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")

Review Comment:
   Same here. For new code, we should not suppress the complexity limit.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#issuecomment-1241114855

   Hi @shiyancao, yes more PRs are under way for support reading default values in engines with different formats (Avro/ORC/Parquet), we will start with implementing support for Spark first.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892941755


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 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 BaseEncoding.base16().encode(((ByteBuffer) value).array());
+      case LIST:
+        List<Object> defaultList = (List<Object>) value;
+        List<Object> convertedList = Lists.newArrayListWithExpectedSize(defaultList.size());
+        for (Object element : defaultList) {

Review Comment:
   This should call generator methods instead of returning `List<Object>`. Same with map and struct.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893896929


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 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 BaseEncoding.base16().encode(((ByteBuffer) value).array());
+      case LIST:
+        List<Object> defaultList = (List<Object>) value;
+        List<Object> convertedList = Lists.newArrayListWithExpectedSize(defaultList.size());
+        for (Object element : defaultList) {

Review Comment:
   I refactored the code and created a new method :



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893902616


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 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 BaseEncoding.base16().encode(((ByteBuffer) value).array());

Review Comment:
   Sure, done. I think the `byte[]` will always come from the `fromJson` 's `wrap()` call.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r894948755


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,38 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String formatEpochDays(int days) {
+    return LocalDate.ofEpochDay(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String formatTimeOfDayMicros(long micros) {

Review Comment:
   Could you name these `microsToIsoTime` and `isoTimeToMicros`? I think the shorter names are more clear.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r899560963


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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.isoDateToDays(defaultValue.textValue());
+      case TIME:
+        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);
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          return DateTimeUtil.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static void toJson(Type type, Object defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Preconditions.checkArgument(
+            defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeBoolean((Boolean) defaultValue);
+        break;
+      case INTEGER:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Integer) defaultValue);
+        break;
+      case LONG:
+        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);
+        generator.writeNumber((Float) defaultValue);
+        break;
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Double) defaultValue);
+        break;
+      case DATE:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer && (Integer) defaultValue >= 0,
+            "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);
+        generator.writeString(DateTimeUtil.microsToIsoTime((Long) defaultValue));
+        break;
+      case TIMESTAMP:
+        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:
+        Preconditions.checkArgument(
+            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);
+        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);
+        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);
+        generator.writeNumber((BigDecimal) defaultValue);
+        break;
+      case LIST:
+        Preconditions.checkArgument(defaultValue instanceof List, "Invalid default %s value: %s", type, defaultValue);
+        List<Object> defaultList = (List<Object>) defaultValue;
+        Type elementType = type.asListType().elementType();
+        generator.writeStartArray();
+        for (Object element : defaultList) {
+          toJson(elementType, element, generator);
+        }
+        generator.writeEndArray();
+        break;
+      case MAP:
+        Preconditions.checkArgument(defaultValue instanceof Map, "Invalid default %s value: %s", type, defaultValue);
+        Map<Object, Object> defaultMap = (Map<Object, Object>) defaultValue;
+        Type keyType = type.asMapType().keyType();
+        Type valueType = type.asMapType().valueType();
+
+        List<Object> valueList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        generator.writeStartObject();
+        generator.writeArrayFieldStart(KEYS);
+        for (Map.Entry<Object, Object> entry : defaultMap.entrySet()) {
+          toJson(keyType, entry.getKey(), generator);
+          valueList.add(entry.getValue());
+        }
+        generator.writeEndArray();
+        generator.writeArrayFieldStart(VALUES);
+        for (Object value : valueList) {
+          toJson(valueType, value, generator);
+        }
+        generator.writeEndArray();
+        generator.writeEndObject();
+        break;
+      case STRUCT:
+        Preconditions.checkArgument(
+            defaultValue instanceof StructLike, "Invalid default %s value: %s", type, defaultValue);
+        Types.StructType structType = type.asStructType();
+        List<Types.NestedField> fields = structType.fields();
+        StructLike defaultStruct = (StructLike) defaultValue;
+
+        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);

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r899561487


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,40 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {

Review Comment:
   Discussed offline since Literals is in API module we are not able to do this refactor.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893863087


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,25 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  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)

Review Comment:
   Got it, refactored.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893899593


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 Object convertJavaDefaultForSerialization(Type type, Object value) {

Review Comment:
   I refactored it to be:



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 Object convertJavaDefaultForSerialization(Type type, Object value) {

Review Comment:
   I refactored it to be:
   ```
     public static String toJson(Type type, Object javaDefaultValue) throws IOException {
       return JsonUtil.mapper().writeValueAsString(DefaultValueParser.convertJavaDefaultForSerialization(
           type,
           javaDefaultValue));
     }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893896929


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 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 BaseEncoding.base16().encode(((ByteBuffer) value).array());
+      case LIST:
+        List<Object> defaultList = (List<Object>) value;
+        List<Object> convertedList = Lists.newArrayListWithExpectedSize(defaultList.size());
+        for (Object element : defaultList) {

Review Comment:
   I refactored the code and created a new method :
   
   ```
     public static String toJson(Type type, Object javaDefaultValue) throws IOException {
       return JsonUtil.mapper().writeValueAsString(DefaultValueParser.convertJavaDefaultForSerialization(
           type,
           javaDefaultValue));
     }
   ```
   
   that takes a java default and output a json string.
   
   I think the `SchemaParser` 's generator in `toJson` can then just call `generator.writeRawValue` to serialize the value.
   
   So I think the API `toJson` in this DefaultValueParser class itself doesn't need to take in a generator.
   
   Do you agree with my thoughts?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r894084179


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 Object convertJavaDefaultForSerialization(Type type, Object value) {

Review Comment:
   Sure, I refactored to use `JsonGenerator`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r894949156


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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:
+        generator.writeString(DateTimeUtil.formatEpochDays((int) javaDefaultValue));
+        break;
+      case TIME:
+        generator.writeString(DateTimeUtil.formatTimeOfDayMicros((long) javaDefaultValue));
+        break;
+      case TIMESTAMP:
+        generator.writeString(DateTimeUtil.formatEpochTimeMicros(
+            (long) javaDefaultValue,
+            ((Types.TimestampType) type).shouldAdjustToUTC()));
+        break;
+      case STRING:
+        generator.writeString((String) javaDefaultValue);

Review Comment:
   This is only required to be `CharSequence`, so you should cast to `CharSequence` and call `toString`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892943816


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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) {

Review Comment:
   Since the field can't actually carry a default value right now, I think we can put this off until the next PR.
   
   For the next step, I think this should add the API changes as package-private so we can add handling for child defaults in the same package. We can move the parser and make more things public as we make progress.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892933429


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> defaultStruct = Maps.newHashMap();

Review Comment:
   This should return a `StructLike`:
   
   ```java
     StructType struct = type.asStructType();
     StructLike defaultRecord = GenericRecord.create(struct);
   
     List<NestedField> fields = struct.fields();
     for (int pos = 0; pos < fields.size(); pos += 1) {
       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;
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892923512


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> defaultList = Lists.newArrayList();
+        for (JsonNode element : defaultValue) {
+          defaultList.add(parseDefaultFromJson(type.asListType().elementType(), element));

Review Comment:
   You can move `type.asListType().elementType()` out of the loop.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r899565283


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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.isoDateToDays(defaultValue.textValue());
+      case TIME:
+        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);
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          return DateTimeUtil.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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",

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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.isoDateToDays(defaultValue.textValue());
+      case TIME:
+        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);
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          return DateTimeUtil.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 IllegalArgumentException(String.format("Type: %s is not supported", type));

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r921640781


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+  private static final String VALUES = "values";
+
+  public static Object fromJson(Type type, JsonNode defaultValue) {
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        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);
+        BigDecimal retDecimal;
+        try {
+          retDecimal = new BigDecimal(defaultValue.textValue());
+        } catch (Exception e) {

Review Comment:
   This exception is too broad. Looks like `BigDecimal` will only throw `NumberFormatException` here, so you should catch that instead.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r921690656


##########
core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.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 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:30Z\""},

Review Comment:
   The format for this should be `+00:00` and not `Z`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r909053540


##########
core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.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 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+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];
+
+      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

Review Comment:
   I think this should probably reject values that have a non-zero offset, rather than allowing the serialized value to change.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r883927349


##########
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<Object> defaultList = Lists.newArrayList();
+        for (JsonNode element : defaultValue) {
+          defaultList.add(parseDefaultFromJson(type.asListType().elementType(), element));
+        }
+        return defaultList;
+      case MAP:
+        Map<Object, Object> defaultMap = Maps.newHashMap();
+        List<JsonNode> keysAndValues = StreamSupport
+            .stream(defaultValue.spliterator(), false)
+            .collect(Collectors.toList());
+        JsonNode keys = keysAndValues.get(0);
+        JsonNode values = keysAndValues.get(1);
+
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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<Integer, Object> 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)) {

Review Comment:
   I think this should match the style of the other JSON parsers, which don't do this work twice. Here, you're using a switch statement on the type to validate, and then using a switch statement on the type to extract the value. Instead, I think this should have one method that keeps the logic for each type in the same place.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r883941037


##########
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<Object> defaultList = Lists.newArrayList();
+        for (JsonNode element : defaultValue) {
+          defaultList.add(parseDefaultFromJson(type.asListType().elementType(), element));
+        }
+        return defaultList;
+      case MAP:
+        Map<Object, Object> defaultMap = Maps.newHashMap();
+        List<JsonNode> keysAndValues = StreamSupport
+            .stream(defaultValue.spliterator(), false)
+            .collect(Collectors.toList());
+        JsonNode keys = keysAndValues.get(0);
+        JsonNode values = keysAndValues.get(1);

Review Comment:
   Actually, I think we agreed in the spec that a map should be represented as a Json array of two underlying json arrays, the first array represents keys, and the second represents values, and their order aligns.
   eg.
   ```
           {Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()),
            stringToJsonNode("[[1,2], [\"foo\", \"bar\"]]")},
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r883932331


##########
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<Object> defaultList = Lists.newArrayList();
+        for (JsonNode element : defaultValue) {
+          defaultList.add(parseDefaultFromJson(type.asListType().elementType(), element));
+        }
+        return defaultList;
+      case MAP:
+        Map<Object, Object> defaultMap = Maps.newHashMap();
+        List<JsonNode> keysAndValues = StreamSupport
+            .stream(defaultValue.spliterator(), false)

Review Comment:
   Iceberg code should not use `spliterator`. Can you find another way?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r922453178


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+  private static final String VALUES = "values";
+
+  public static Object fromJson(Type type, JsonNode defaultValue) {
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        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);
+        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);
+        return retDecimal;
+      case STRING:
+        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);
+        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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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()) {
+          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.isoTimestampToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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));
+        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));
+        return ByteBuffer.wrap(binaryBytes);
+      case LIST:
+        return listFromJson(type, defaultValue);
+      case MAP:
+        return mapFromJson(type, defaultValue);
+      case STRUCT:
+        return structFromJson(type, defaultValue);
+      default:
+        throw new UnsupportedOperationException(String.format("Type: %s is not supported", type));
+    }
+  }
+
+  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);
+
+    List<Types.NestedField> 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;
+  }
+
+  private static Map<Object, Object> 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);
+
+    ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+    Iterator<JsonNode> keyIter = keys.iterator();
+    Type keyType = type.asMapType().keyType();
+    Iterator<JsonNode> 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<Object> 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) {
+    try {
+      JsonNode defaultValueJN = JsonUtil.mapper().readTree(defaultValue);
+      return fromJson(type, defaultValueJN);
+    } catch (IOException e) {
+      throw new IllegalArgumentException(
+          String.format("Failed to parse default as a %s value: %s", type, defaultValue), e);
+    }
+  }
+
+  public static String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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 defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Preconditions.checkArgument(
+            defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeBoolean((Boolean) defaultValue);
+        break;
+      case INTEGER:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Integer) defaultValue);
+        break;
+      case LONG:
+        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);
+        generator.writeNumber((Float) defaultValue);
+        break;
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Double) defaultValue);
+        break;
+      case DATE:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.daysToIsoDate((Integer) defaultValue));
+        break;
+      case TIME:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.microsToIsoTime((Long) defaultValue));
+        break;
+      case TIMESTAMP:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "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:
+        Preconditions.checkArgument(
+            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);
+        generator.writeString(defaultValue.toString());
+        break;
+      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);

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r922455421


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isFloatingPointNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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, 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);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "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);
+        }
+        return uuid;

Review Comment:
   @rdblue  I think we still need it, because the `UUID.fromString` method does more checking like:
   
   ```
   String[] components = name.split("-");
           if (components.length != 5)
               throw new IllegalArgumentException("Invalid UUID string: "+name);
   ```
   
   which testing the length itself doesn't validate.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r908731811


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isFloatingPointNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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, 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);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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));
+        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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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));
+    }
+  }
+
+  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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static void toJson(Type type, Object defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Preconditions.checkArgument(
+            defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeBoolean((Boolean) defaultValue);
+        break;
+      case INTEGER:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Integer) defaultValue);
+        break;
+      case LONG:
+        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);
+        generator.writeNumber((Float) defaultValue);
+        break;
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Double) defaultValue);
+        break;
+      case DATE:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.daysToIsoDate((Integer) defaultValue));
+        break;
+      case TIME:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.microsToIsoTime((Long) defaultValue));
+        break;
+      case TIMESTAMP:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "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:
+        Preconditions.checkArgument(
+            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);
+        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,
+            "Invalid default %s value: %s, incorrect length: %s",
+            type, defaultValue, valueLength);

Review Comment:
   Minor: this should check the `remaining` bytes in the incoming buffer rather than checking the final string value. That would avoid a conversion and get the check done as soon as possible.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r883934727


##########
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<Object> defaultList = Lists.newArrayList();
+        for (JsonNode element : defaultValue) {
+          defaultList.add(parseDefaultFromJson(type.asListType().elementType(), element));
+        }
+        return defaultList;
+      case MAP:
+        Map<Object, Object> defaultMap = Maps.newHashMap();
+        List<JsonNode> keysAndValues = StreamSupport
+            .stream(defaultValue.spliterator(), false)
+            .collect(Collectors.toList());
+        JsonNode keys = keysAndValues.get(0);
+        JsonNode values = keysAndValues.get(1);

Review Comment:
   According to the spec, the JSON node should be an object with two fields: `keys` and `values`. I think it would be much easier to validate that the node is an object and then read the fields, rather than trying to convert to a list. This needs to respect the names, not the order.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r885267388


##########
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<Object> defaultList = Lists.newArrayList();
+        for (JsonNode element : defaultValue) {
+          defaultList.add(parseDefaultFromJson(type.asListType().elementType(), element));
+        }
+        return defaultList;
+      case MAP:
+        Map<Object, Object> defaultMap = Maps.newHashMap();
+        List<JsonNode> keysAndValues = StreamSupport
+            .stream(defaultValue.spliterator(), false)
+            .collect(Collectors.toList());
+        JsonNode keys = keysAndValues.get(0);
+        JsonNode values = keysAndValues.get(1);
+
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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<Integer, Object> 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)) {

Review Comment:
   Sure, let me do this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r885272677


##########
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<Object> defaultList = Lists.newArrayList();
+        for (JsonNode element : defaultValue) {
+          defaultList.add(parseDefaultFromJson(type.asListType().elementType(), element));
+        }
+        return defaultList;
+      case MAP:
+        Map<Object, Object> defaultMap = Maps.newHashMap();
+        List<JsonNode> keysAndValues = StreamSupport
+            .stream(defaultValue.spliterator(), false)
+            .collect(Collectors.toList());
+        JsonNode keys = keysAndValues.get(0);
+        JsonNode values = keysAndValues.get(1);

Review Comment:
   Sure, fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892938544


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,25 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  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)

Review Comment:
   This should not produce a string and then parse it. Instead, it should update the conversion above to go directly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r934903711


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+  private static final String VALUES = "values";
+
+  public static Object fromJson(Type type, JsonNode defaultValue) {
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        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);
+        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);
+        }
+        Preconditions.checkArgument(
+            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(),
+            "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);
+        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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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()) {
+          String timestampTz = defaultValue.textValue();
+          Preconditions.checkArgument(DateTimeUtil.isUTCTimestamptz(timestampTz),
+              "Cannot parse default as a %s value: %s, timezone must be UTC",
+              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);
+        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));
+        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));
+        return ByteBuffer.wrap(binaryBytes);
+      case LIST:
+        return listFromJson(type, defaultValue);
+      case MAP:
+        return mapFromJson(type, defaultValue);
+      case STRUCT:
+        return structFromJson(type, defaultValue);
+      default:
+        throw new UnsupportedOperationException(String.format("Type: %s is not supported", type));
+    }
+  }
+
+  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);
+
+    List<Types.NestedField> 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;
+  }
+
+  private static Map<Object, Object> 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);
+
+    ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+    Iterator<JsonNode> keyIter = keys.iterator();
+    Type keyType = type.asMapType().keyType();
+    Iterator<JsonNode> 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<Object> 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) {
+    try {
+      JsonNode defaultValueJN = JsonUtil.mapper().readTree(defaultValue);
+      return fromJson(type, defaultValueJN);
+    } catch (IOException e) {
+      throw new IllegalArgumentException(
+          String.format("Failed to parse default as a %s value: %s", type, defaultValue), e);
+    }
+  }
+
+  public static String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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 defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Preconditions.checkArgument(
+            defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeBoolean((Boolean) defaultValue);
+        break;
+      case INTEGER:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Integer) defaultValue);
+        break;
+      case LONG:
+        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);
+        generator.writeNumber((Float) defaultValue);
+        break;
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Double) defaultValue);
+        break;
+      case DATE:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.daysToIsoDate((Integer) defaultValue));
+        break;
+      case TIME:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.microsToIsoTime((Long) defaultValue));
+        break;
+      case TIMESTAMP:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue);
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          generator.writeString(DateTimeUtil.microsToIsoTimestamptz((Long) defaultValue));
+        } else {
+          generator.writeString(DateTimeUtil.microsToIsoTimestamp((Long) defaultValue));
+        }
+        break;
+      case STRING:
+        Preconditions.checkArgument(
+            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);
+        generator.writeString(defaultValue.toString());
+        break;
+      case FIXED:
+        Preconditions.checkArgument(
+            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,
+            "Invalid default %s value: %s, incorrect length: %s",
+            type, defaultValue, byteBufferValue.remaining());

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r921689730


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,43 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {
+    return dateFromDays(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String microsToIsoTime(long micros) {
+    return timeFromMicros(micros).format(DateTimeFormatter.ISO_LOCAL_TIME);
+  }
+
+  public static String microsToIsoDateTimeTz(long micros) {

Review Comment:
   @rzhang10, here's a method name that wasn't fixed after my [earlier comment](https://github.com/apache/iceberg/pull/4871/files#discussion_r909039593).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892941902


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 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 BaseEncoding.base16().encode(((ByteBuffer) value).array());
+      case LIST:
+        List<Object> defaultList = (List<Object>) value;
+        List<Object> convertedList = Lists.newArrayListWithExpectedSize(defaultList.size());
+        for (Object element : defaultList) {
+          convertedList.add(convertJavaDefaultForSerialization(type.asListType().elementType(), element));
+        }
+        return convertedList;
+      case MAP:
+        Map<Object, Object> defaultMap = (Map<Object, Object>) value;
+        Map<String, List<Object>> convertedDefault = Maps.newHashMapWithExpectedSize(2);
+        List<Object> keyList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        List<Object> valueList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        for (Map.Entry<Object, Object> entry : defaultMap.entrySet()) {
+          keyList.add(convertJavaDefaultForSerialization(type.asMapType().keyType(), entry.getKey()));
+          valueList.add(convertJavaDefaultForSerialization(type.asMapType().valueType(), entry.getValue()));
+        }
+        convertedDefault.put("keys", keyList);
+        convertedDefault.put("values", valueList);
+        return convertedDefault;
+      case STRUCT:
+        Map<Integer, Object> defaultStruct = (Map<Integer, Object>) value;

Review Comment:
   This should deconstruct a `StructLike`, not a map.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r896341509


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,40 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {

Review Comment:
   @rzhang10, please refactor the literals to use these methods where possible. We don't want two separate implementations.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893901077


##########
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);

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r894084278


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 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 BaseEncoding.base16().encode(((ByteBuffer) value).array());
+      case LIST:
+        List<Object> defaultList = (List<Object>) value;
+        List<Object> convertedList = Lists.newArrayListWithExpectedSize(defaultList.size());
+        for (Object element : defaultList) {

Review Comment:
   I refactored to use `JsonGenerator`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r894950788


##########
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() {

Review Comment:
   I'd probably put these tests in the parser test.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r921640004


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isFloatingPointNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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, 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);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "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);
+        }
+        return uuid;

Review Comment:
   The try/catch is no longer needed. Can you remove it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893812828


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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());

Review Comment:
   `UUID.fromString` is already doing such validation, let me use a try-catch block to print out the error message with the same format?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892942577


##########
core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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<Object[]> 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.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()),
+         stringToJsonNode("{\"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\"}")}

Review Comment:
   Can you add test cases for nested types? One of each (list, map, struct) that contains a struct would be good.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r894029519


##########
core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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<Object[]> 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.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()),
+         stringToJsonNode("{\"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\"}")}

Review Comment:
   Added.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892922876


##########
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);

Review Comment:
   Why not use `ByteBuffer.wrap` here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893893017


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 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 BaseEncoding.base16().encode(((ByteBuffer) value).array());
+      case LIST:
+        List<Object> defaultList = (List<Object>) value;
+        List<Object> convertedList = Lists.newArrayListWithExpectedSize(defaultList.size());
+        for (Object element : defaultList) {
+          convertedList.add(convertJavaDefaultForSerialization(type.asListType().elementType(), element));
+        }
+        return convertedList;
+      case MAP:
+        Map<Object, Object> defaultMap = (Map<Object, Object>) value;
+        Map<String, List<Object>> convertedDefault = Maps.newHashMapWithExpectedSize(2);
+        List<Object> keyList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        List<Object> valueList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        for (Map.Entry<Object, Object> entry : defaultMap.entrySet()) {
+          keyList.add(convertJavaDefaultForSerialization(type.asMapType().keyType(), entry.getKey()));
+          valueList.add(convertJavaDefaultForSerialization(type.asMapType().valueType(), entry.getValue()));
+        }
+        convertedDefault.put("keys", keyList);
+        convertedDefault.put("values", valueList);
+        return convertedDefault;
+      case STRUCT:
+        Map<Integer, Object> defaultStruct = (Map<Integer, Object>) value;

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893908955


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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);

Review Comment:
   Done refactoring it to `Cannot parse default as a %s value: %s", type, defaultValue`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r909052221


##########
core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.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 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+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")),

Review Comment:
   Can you please remove all of the "doc" strings? Those are unnecessary and make it harder to read this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r908723007


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+
+  private static final String VALUES = "values";
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object fromJson(Type type, JsonNode defaultValue) {
+

Review Comment:
   Style: no need for an empty line starting a method.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r896342306


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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",

Review Comment:
   This error message can be more specific: the scale doesn't match.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r929270568


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+  private static final String VALUES = "values";
+
+  public static Object fromJson(Type type, JsonNode defaultValue) {
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        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);
+        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);
+        }
+        Preconditions.checkArgument(
+            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(),
+            "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);
+        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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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()) {
+          String timestampTz = defaultValue.textValue();
+          Preconditions.checkArgument(DateTimeUtil.isUTCTimestamptz(timestampTz),
+              "Cannot parse default as a %s value: %s, timezone must be UTC",
+              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);
+        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));
+        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));
+        return ByteBuffer.wrap(binaryBytes);
+      case LIST:
+        return listFromJson(type, defaultValue);
+      case MAP:
+        return mapFromJson(type, defaultValue);
+      case STRUCT:
+        return structFromJson(type, defaultValue);
+      default:
+        throw new UnsupportedOperationException(String.format("Type: %s is not supported", type));
+    }
+  }
+
+  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);
+
+    List<Types.NestedField> 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;
+  }
+
+  private static Map<Object, Object> 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);
+
+    ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+    Iterator<JsonNode> keyIter = keys.iterator();
+    Type keyType = type.asMapType().keyType();
+    Iterator<JsonNode> 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<Object> 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) {
+    try {
+      JsonNode defaultValueJN = JsonUtil.mapper().readTree(defaultValue);
+      return fromJson(type, defaultValueJN);
+    } catch (IOException e) {
+      throw new IllegalArgumentException(
+          String.format("Failed to parse default as a %s value: %s", type, defaultValue), e);
+    }
+  }
+
+  public static String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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 defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Preconditions.checkArgument(
+            defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeBoolean((Boolean) defaultValue);
+        break;
+      case INTEGER:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Integer) defaultValue);
+        break;
+      case LONG:
+        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);
+        generator.writeNumber((Float) defaultValue);
+        break;
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Double) defaultValue);
+        break;
+      case DATE:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.daysToIsoDate((Integer) defaultValue));
+        break;
+      case TIME:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.microsToIsoTime((Long) defaultValue));
+        break;
+      case TIMESTAMP:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue);
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          generator.writeString(DateTimeUtil.microsToIsoTimestamptz((Long) defaultValue));
+        } else {
+          generator.writeString(DateTimeUtil.microsToIsoTimestamp((Long) defaultValue));
+        }
+        break;
+      case STRING:
+        Preconditions.checkArgument(
+            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);
+        generator.writeString(defaultValue.toString());
+        break;
+      case FIXED:
+        Preconditions.checkArgument(
+            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,
+            "Invalid default %s value: %s, incorrect length: %s",
+            type, defaultValue, byteBufferValue.remaining());

Review Comment:
   `ByteBuffer#toString` isn't useful here. Just note that the size is wrong.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r899628752


##########
core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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<Object[]> 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 {

Review Comment:
   I made them into a single test class.



##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,40 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {
+    return LocalDate.ofEpochDay(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String microsToIsoTime(long micros) {
+    return LocalTime.ofNanoOfDay(micros * 1000).format(DateTimeFormatter.ISO_LOCAL_TIME);
+  }
+
+  public static String microsToIsoDateTimeTz(long micros) {
+    LocalDateTime localDateTime = LocalDateTime.ofEpochSecond(micros / 1000000,
+        (int) (micros % 1000000) * 1000, ZoneOffset.UTC);

Review Comment:
   Refactored



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r883941037


##########
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<Object> defaultList = Lists.newArrayList();
+        for (JsonNode element : defaultValue) {
+          defaultList.add(parseDefaultFromJson(type.asListType().elementType(), element));
+        }
+        return defaultList;
+      case MAP:
+        Map<Object, Object> defaultMap = Maps.newHashMap();
+        List<JsonNode> keysAndValues = StreamSupport
+            .stream(defaultValue.spliterator(), false)
+            .collect(Collectors.toList());
+        JsonNode keys = keysAndValues.get(0);
+        JsonNode values = keysAndValues.get(1);

Review Comment:
   Actually, I think we agreed in the spec that a map should be represented as a Json array of two underlying json arrays, the first array represents keys, and the second represents values, and their order aligns.
   eg.
   ```
   "[[1,2], [\"foo\", \"bar\"]]"
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r883927621


##########
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) {

Review Comment:
   This needs to check `isNull` as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r885267155


##########
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();

Review Comment:
   Fixed.



##########
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) {

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r883929081


##########
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();

Review Comment:
   This needs to validate the UUID and also return a `java.util.UUID` value.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892943265


##########
core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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 {

Review Comment:
   I think this should also have a few tests for cases that are caught above, like maps with different length key and value lists, binary and fixed values that are not the right length, UUID values that are not actually UUIDs, etc.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892895242


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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);

Review Comment:
   Actually, as long as this is in a type branch, you should just embed the type string: `"Cannot parse default as a boolean value: %s", defaultValue`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892938036


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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();

Review Comment:
   Rather than using `Literal`, could you just refactor to add the conversions to `DateTimeUtil` like the to string conversions? That way we have both in the util.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893826330


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893886632


##########
core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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<Object[]> 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.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()),
+         stringToJsonNode("{\"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\"}")}
+    });
+  }
+
+  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()

Review Comment:
   Got it, makes sense, refactored.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r895236051


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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:
+        generator.writeString(DateTimeUtil.formatEpochDays((int) javaDefaultValue));
+        break;
+      case TIME:
+        generator.writeString(DateTimeUtil.formatTimeOfDayMicros((long) javaDefaultValue));
+        break;
+      case TIMESTAMP:
+        generator.writeString(DateTimeUtil.formatEpochTimeMicros(
+            (long) javaDefaultValue,
+            ((Types.TimestampType) type).shouldAdjustToUTC()));
+        break;
+      case STRING:
+        generator.writeString((String) javaDefaultValue);
+        break;
+      case UUID:
+        generator.writeString(javaDefaultValue.toString());
+        break;
+      case FIXED:
+      case BINARY:
+        generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) javaDefaultValue))));
+        break;
+      case DECIMAL:
+        generator.writeNumber((BigDecimal) javaDefaultValue);
+        break;
+      case LIST:
+        List<Object> defaultList = (List<Object>) javaDefaultValue;
+        Type elementType = type.asListType().elementType();
+        generator.writeStartArray();
+        for (Object elementDefault : defaultList) {
+          toJson(elementType, elementDefault, generator);
+        }
+        generator.writeEndArray();
+        break;
+      case MAP:
+        Map<Object, Object> defaultMap = (Map<Object, Object>) javaDefaultValue;
+        List<Object> keyList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        List<Object> valueList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        Type keyType = type.asMapType().keyType();
+        Type valueType = type.asMapType().valueType();
+
+        for (Map.Entry<Object, Object> entry : defaultMap.entrySet()) {

Review Comment:
   fixed, nice suggestion/optimization.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#issuecomment-1201746636

   @rdblue I've addressed the comments and rebased on master and also did a `spotlessApply`, could you review again?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r929274993


##########
format/spec.md:
##########
@@ -1132,15 +1132,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 |

Review Comment:
   What does "as-is" mean? I think we need to be specific about the format here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#issuecomment-1158274877

   Hi Ryan, I've refactored the code according to the comments, can you please take a look again?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r921689880


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,43 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {
+    return dateFromDays(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String microsToIsoTime(long micros) {
+    return timeFromMicros(micros).format(DateTimeFormatter.ISO_LOCAL_TIME);
+  }
+
+  public static String microsToIsoDateTimeTz(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    return localDateTime.atOffset(ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+  }
+
+  public static String microsToIsoDateTime(long micros) {

Review Comment:
   @rzhang10, here's another method name that wasn't fixed after my [earlier comment](https://github.com/apache/iceberg/pull/4871/files#discussion_r909039593).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r922431584


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+  private static final String VALUES = "values";
+
+  public static Object fromJson(Type type, JsonNode defaultValue) {
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        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);
+        BigDecimal retDecimal;
+        try {
+          retDecimal = new BigDecimal(defaultValue.textValue());
+        } catch (Exception e) {

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r922427093


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,43 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {
+    return dateFromDays(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String microsToIsoTime(long micros) {
+    return timeFromMicros(micros).format(DateTimeFormatter.ISO_LOCAL_TIME);
+  }
+
+  public static String microsToIsoDateTimeTz(long micros) {

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,43 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {
+    return dateFromDays(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String microsToIsoTime(long micros) {
+    return timeFromMicros(micros).format(DateTimeFormatter.ISO_LOCAL_TIME);
+  }
+
+  public static String microsToIsoDateTimeTz(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    return localDateTime.atOffset(ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+  }
+
+  public static String microsToIsoDateTime(long micros) {

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r922296511


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,43 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {
+    return dateFromDays(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String microsToIsoTime(long micros) {
+    return timeFromMicros(micros).format(DateTimeFormatter.ISO_LOCAL_TIME);
+  }
+
+  public static String microsToIsoDateTimeTz(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    return localDateTime.atOffset(ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+  }
+
+  public static String microsToIsoDateTime(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    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 isoTimeToMicros(String timeString) {
+    return microsFromTime(LocalTime.parse(timeString, DateTimeFormatter.ISO_LOCAL_TIME));
+  }
+
+  public static long isoTimestamptzToMicros(String timestampString) {
+    return microsFromTimestamptz(OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME));
+  }
+
+  public static boolean timestamptzIsOfUTCZone(String timestampString) {
+    OffsetDateTime offsetDateTime = OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME);
+    return offsetDateTime.getOffset().equals(ZoneOffset.UTC) && timestampString.endsWith("Z");

Review Comment:
   As long as `ISO_DATE_TIME` can correctly parse, we shouldn't care about the underlying string. I think there isn't a need to check `endsWith("Z")`. This should just check that the `ZoneOffset` is UTC.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r908722392


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+

Review Comment:
   Nit: no need for an extra newline here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r908733853


##########
core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.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 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+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"},

Review Comment:
   Can you add a test that verifies trailing 0s are not suppressed by the JSON writer?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r909038178


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isFloatingPointNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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, the scale doesn't match", type, defaultValue);
+        return retDecimal;

Review Comment:
   I just tested this and Jackson produces incorrect results. When reading, the value gets parsed as a `DoubleNode` and that can be converted to a decimal with the wrong scale:
   
   ```java
         JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
         generator.writeNumber(new BigDecimal("12.4000"));
         generator.flush();
         String asJson = writer.toString();
         // asString = "12.4000"
   
         JsonNode node = JsonUtil.mapper().readValue(asJson, JsonNode.class);
         // node = DoubleNode(_value=12.4d)
         BigDecimal decimal = node.decimalValue();
         // decimal = BigDecimal("12.4")
   ```
   
   I also don't see a way to get the original text that was used to produce the node out of Jackson, so I think decimals will need to be stored as strings rather than as numbers. That makes sense anyway, since almost all languages have a way to initialize decimals from an exact string that can contain trailing 0s.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893824245


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> defaultList = Lists.newArrayList();
+        for (JsonNode element : defaultValue) {
+          defaultList.add(parseDefaultFromJson(type.asListType().elementType(), element));

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893856574


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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) {

Review Comment:
   I think handling child default will require a second pass to traverse the schema (with default) again. I plan to have another PR that implements a SchemaVisitor handle this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893950329


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 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 BaseEncoding.base16().encode(((ByteBuffer) value).array());
+      case LIST:
+        List<Object> defaultList = (List<Object>) value;
+        List<Object> convertedList = Lists.newArrayListWithExpectedSize(defaultList.size());
+        for (Object element : defaultList) {

Review Comment:
   No, as I said this should match the other parser classes and use a `JsonGenerator`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893805186


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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();

Review Comment:
   fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892940686


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 Object convertJavaDefaultForSerialization(Type type, Object value) {

Review Comment:
   Like the other parsers, this method should be passed a `JsonGenerator` that handles creating the JSON string.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892940320


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {

Review Comment:
   Can you rename this `fromJson`? And also add the variations of the method that accept String.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892893960


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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);

Review Comment:
   I think you're trying to copy the error messages from `JsonUtil`, but removed the wrong `%s`. The user value goes after the error message and should not be embedded in it. The `Cannot parse %s` in `JsonUtil` tells the reader which field was being parsed, like `Cannot parse snapshot-id to a long value: null`.
   
   This should be `"Cannot parse default as a %s value: %s", type, defaultValue`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892937113


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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) {

Review Comment:
   Here, I think we need to handle the child default values. If we make this independent of the child's default value, then there is no way to distinguish between an explicit null default and a missing default after this returns.
   
   When the default is missing and the child field has a default, this should fill in the child's default value.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r894947584


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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 {

Review Comment:
   What if the default value is null?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r894948601


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,38 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String formatEpochDays(int days) {

Review Comment:
   It doesn't look like the literals were updated to use these functions. Could you finish that refactor?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r895236051


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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:
+        generator.writeString(DateTimeUtil.formatEpochDays((int) javaDefaultValue));
+        break;
+      case TIME:
+        generator.writeString(DateTimeUtil.formatTimeOfDayMicros((long) javaDefaultValue));
+        break;
+      case TIMESTAMP:
+        generator.writeString(DateTimeUtil.formatEpochTimeMicros(
+            (long) javaDefaultValue,
+            ((Types.TimestampType) type).shouldAdjustToUTC()));
+        break;
+      case STRING:
+        generator.writeString((String) javaDefaultValue);
+        break;
+      case UUID:
+        generator.writeString(javaDefaultValue.toString());
+        break;
+      case FIXED:
+      case BINARY:
+        generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) javaDefaultValue))));
+        break;
+      case DECIMAL:
+        generator.writeNumber((BigDecimal) javaDefaultValue);
+        break;
+      case LIST:
+        List<Object> defaultList = (List<Object>) javaDefaultValue;
+        Type elementType = type.asListType().elementType();
+        generator.writeStartArray();
+        for (Object elementDefault : defaultList) {
+          toJson(elementType, elementDefault, generator);
+        }
+        generator.writeEndArray();
+        break;
+      case MAP:
+        Map<Object, Object> defaultMap = (Map<Object, Object>) javaDefaultValue;
+        List<Object> keyList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        List<Object> valueList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        Type keyType = type.asMapType().keyType();
+        Type valueType = type.asMapType().valueType();
+
+        for (Map.Entry<Object, Object> entry : defaultMap.entrySet()) {

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r896342524


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());

Review Comment:
   This should be returned rather than running the conversion twice.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r896343530


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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());

Review Comment:
   Actually, it isn't:
   
   ```java
       public static UUID fromString(String var0) {
           String[] var1 = var0.split("-");
           if (var1.length != 5) {
               throw new IllegalArgumentException("Invalid UUID string: " + var0);
           } else {
               for(int var2 = 0; var2 < 5; ++var2) {
                   var1[var2] = "0x" + var1[var2];
               }
   
               long var6 = Long.decode(var1[0]);
               var6 <<= 16;
               var6 |= Long.decode(var1[1]);
               var6 <<= 16;
               var6 |= Long.decode(var1[2]);
               long var4 = Long.decode(var1[3]);
               var4 <<= 48;
               var4 |= Long.decode(var1[4]);
               return new UUID(var6, var4);
           }
       }
   ```
   
   It's just validating that there are enough parts, and decoding those parts.
   
   Instead of doing the try/catch thing (which has its own problem) I think you should check that the string is the right length to be a UUID.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r918318560


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isFloatingPointNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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, 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);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "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);
+        }
+        return uuid;

Review Comment:
   Sorry that i forgot about this, done  this time.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r918273257


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,38 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {
+    return dateFromDays(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String microsToIsoTime(long micros) {
+    return timeFromMicros(micros).format(DateTimeFormatter.ISO_LOCAL_TIME);
+  }
+
+  public static String microsToIsoDateTimeTz(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    return OffsetDateTime.of(localDateTime, ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+  }
+
+  public static String microsToIsoDateTime(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    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 isoTimeToMicros(String timeString) {
+    return microsFromTime(LocalTime.parse(timeString, DateTimeFormatter.ISO_LOCAL_TIME));
+  }
+
+  public static long isoDateTimeTzToMicros(String timestampString) {

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r918273984


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,38 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {
+    return dateFromDays(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String microsToIsoTime(long micros) {
+    return timeFromMicros(micros).format(DateTimeFormatter.ISO_LOCAL_TIME);
+  }
+
+  public static String microsToIsoDateTimeTz(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    return OffsetDateTime.of(localDateTime, ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r921644740


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+  private static final String VALUES = "values";
+
+  public static Object fromJson(Type type, JsonNode defaultValue) {
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        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);
+        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);
+        return retDecimal;
+      case STRING:
+        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);
+        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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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()) {
+          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.isoTimestampToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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));
+        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));
+        return ByteBuffer.wrap(binaryBytes);
+      case LIST:
+        return listFromJson(type, defaultValue);
+      case MAP:
+        return mapFromJson(type, defaultValue);
+      case STRUCT:
+        return structFromJson(type, defaultValue);
+      default:
+        throw new UnsupportedOperationException(String.format("Type: %s is not supported", type));
+    }
+  }
+
+  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);
+
+    List<Types.NestedField> 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;
+  }
+
+  private static Map<Object, Object> 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);
+
+    ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+    Iterator<JsonNode> keyIter = keys.iterator();
+    Type keyType = type.asMapType().keyType();
+    Iterator<JsonNode> 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<Object> 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) {
+    try {
+      JsonNode defaultValueJN = JsonUtil.mapper().readTree(defaultValue);
+      return fromJson(type, defaultValueJN);
+    } catch (IOException e) {
+      throw new IllegalArgumentException(
+          String.format("Failed to parse default as a %s value: %s", type, defaultValue), e);
+    }
+  }
+
+  public static String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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 defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Preconditions.checkArgument(
+            defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeBoolean((Boolean) defaultValue);
+        break;
+      case INTEGER:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Integer) defaultValue);
+        break;
+      case LONG:
+        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);
+        generator.writeNumber((Float) defaultValue);
+        break;
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Double) defaultValue);
+        break;
+      case DATE:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.daysToIsoDate((Integer) defaultValue));
+        break;
+      case TIME:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.microsToIsoTime((Long) defaultValue));
+        break;
+      case TIMESTAMP:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "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:
+        Preconditions.checkArgument(
+            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);
+        generator.writeString(defaultValue.toString());
+        break;
+      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);

Review Comment:
   This is incorrect. Any `ByteBuffer` is a valid value that is supported by `ByteBuffers.toByteArray`.
   
   I think the confusion comes from [this comment](https://github.com/apache/iceberg/pull/4871/files#discussion_r908731811). The check I intended is `((ByteBuffer) defaultValue).remaining() == expectedLength`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r899572709


##########
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(

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r909045719


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isFloatingPointNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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, 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);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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));
+        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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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));
+    }
+  }
+
+  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);

Review Comment:
   We don't usually copy exception messages. As long as `e` is set as the cause, this should be printed elsewhere.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r883930229


##########
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);

Review Comment:
   This needs to validate the length of the byte array.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r885271858


##########
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);

Review Comment:
   Sure, fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r885288763


##########
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<Object> defaultList = Lists.newArrayList();
+        for (JsonNode element : defaultValue) {
+          defaultList.add(parseDefaultFromJson(type.asListType().elementType(), element));
+        }
+        return defaultList;
+      case MAP:
+        Map<Object, Object> defaultMap = Maps.newHashMap();
+        List<JsonNode> keysAndValues = StreamSupport
+            .stream(defaultValue.spliterator(), false)

Review Comment:
   No longer need this since we represent the Map using a JSON Object.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892898628


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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(),

Review Comment:
   I think this should check `isFloatingPointNumber`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892922600


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        Preconditions.checkArgument(
+            defaultValue.isTextual() && defaultValue.textValue().length() == ((Types.FixedType) type).length() * 2,
+            "Cannot parse %s to a %s value",

Review Comment:
   Can you produce a better error message for when the length is invalid?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892949166


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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);

Review Comment:
   Wouldn't reusing each type's `type.toString()` method better?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893946244


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 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 BaseEncoding.base16().encode(((ByteBuffer) value).array());

Review Comment:
   @rzhang10, yes, as I said it should be fine "most of the time".
   
   The problem is that you have no guarantee about the byte buffer that is passed in. Hence this is unsafe.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r918317143


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+
+  private static final String VALUES = "values";
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object fromJson(Type type, JsonNode defaultValue) {
+

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+
+  private static final String VALUES = "values";
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r918289836


##########
core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.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 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+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\": " +

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r918430699


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isFloatingPointNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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, the scale doesn't match", type, defaultValue);
+        return retDecimal;

Review Comment:
   Sure, I changed the the serializaiton of decimal to be a JSON string.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r922297083


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,43 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {
+    return dateFromDays(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String microsToIsoTime(long micros) {
+    return timeFromMicros(micros).format(DateTimeFormatter.ISO_LOCAL_TIME);
+  }
+
+  public static String microsToIsoDateTimeTz(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    return localDateTime.atOffset(ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+  }
+
+  public static String microsToIsoDateTime(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    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 isoTimeToMicros(String timeString) {
+    return microsFromTime(LocalTime.parse(timeString, DateTimeFormatter.ISO_LOCAL_TIME));
+  }
+
+  public static long isoTimestamptzToMicros(String timestampString) {
+    return microsFromTimestamptz(OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME));
+  }
+
+  public static boolean timestamptzIsOfUTCZone(String timestampString) {

Review Comment:
   Usually a boolean name would start with `is`, and "OfUTCZone" is a little confusing. How about naming this `isUTCTimestamptz` instead?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#issuecomment-1185900996

   Hi @rdblue , I addressed the comments again and updated the spec, could you please take another look?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r922444915


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+  private static final String VALUES = "values";
+
+  public static Object fromJson(Type type, JsonNode defaultValue) {
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        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);
+        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);
+        return retDecimal;
+      case STRING:
+        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);
+        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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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()) {
+          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.isoTimestampToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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));
+        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));
+        return ByteBuffer.wrap(binaryBytes);
+      case LIST:
+        return listFromJson(type, defaultValue);
+      case MAP:
+        return mapFromJson(type, defaultValue);
+      case STRUCT:
+        return structFromJson(type, defaultValue);
+      default:
+        throw new UnsupportedOperationException(String.format("Type: %s is not supported", type));
+    }
+  }
+
+  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);
+
+    List<Types.NestedField> 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;
+  }
+
+  private static Map<Object, Object> 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);
+
+    ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+    Iterator<JsonNode> keyIter = keys.iterator();
+    Type keyType = type.asMapType().keyType();
+    Iterator<JsonNode> 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<Object> 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) {
+    try {
+      JsonNode defaultValueJN = JsonUtil.mapper().readTree(defaultValue);
+      return fromJson(type, defaultValueJN);
+    } catch (IOException e) {
+      throw new IllegalArgumentException(
+          String.format("Failed to parse default as a %s value: %s", type, defaultValue), e);
+    }
+  }
+
+  public static String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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 defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Preconditions.checkArgument(
+            defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeBoolean((Boolean) defaultValue);
+        break;
+      case INTEGER:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Integer) defaultValue);
+        break;
+      case LONG:
+        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);
+        generator.writeNumber((Float) defaultValue);
+        break;
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Double) defaultValue);
+        break;
+      case DATE:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.daysToIsoDate((Integer) defaultValue));
+        break;
+      case TIME:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.microsToIsoTime((Long) defaultValue));
+        break;
+      case TIMESTAMP:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "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:
+        Preconditions.checkArgument(
+            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);
+        generator.writeString(defaultValue.toString());
+        break;
+      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();
+        int expectedLength = ((Types.FixedType) type).length();
+        Preconditions.checkArgument(actualLength == expectedLength,
+            "Invalid default %s value: %s, incorrect length: %s",
+            type, defaultValue, actualLength);
+        String fixedString = BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) defaultValue)));
+        generator.writeString(fixedString);
+        break;
+      case BINARY:
+        Preconditions.checkArgument(
+            defaultValue instanceof ByteBuffer, "Invalid default %s value: %s", type, 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);
+        generator.writeString(((BigDecimal) defaultValue).toPlainString());

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r909052858


##########
core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.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 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+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\": " +

Review Comment:
   This should not reuse field IDs. That's not allowed and could cause these tests to fail later.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r896341253


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,40 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {
+    return LocalDate.ofEpochDay(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String microsToIsoTime(long micros) {
+    return LocalTime.ofNanoOfDay(micros * 1000).format(DateTimeFormatter.ISO_LOCAL_TIME);
+  }
+
+  public static String microsToIsoDateTimeTz(long micros) {
+    LocalDateTime localDateTime = LocalDateTime.ofEpochSecond(micros / 1000000,
+        (int) (micros % 1000000) * 1000, ZoneOffset.UTC);

Review Comment:
   This conversion isn't safe. Can you update this to use `timestampFromMicros` and `timestamptzFromMicros` instead? The problem here is that this doesn't use `floorMod` and `floorDiv` so it handles negative values incorrectly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r934917396


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+  private static final String VALUES = "values";
+
+  public static Object fromJson(Type type, JsonNode defaultValue) {
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        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);
+        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);
+        }
+        Preconditions.checkArgument(
+            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(),
+            "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);
+        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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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()) {
+          String timestampTz = defaultValue.textValue();
+          Preconditions.checkArgument(DateTimeUtil.isUTCTimestamptz(timestampTz),
+              "Cannot parse default as a %s value: %s, timezone must be UTC",

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#issuecomment-1203290076

   Thanks, @rzhang10! The latest changes look good. I merged this.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue merged PR #4871:
URL: https://github.com/apache/iceberg/pull/4871


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] shiyancao commented on pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
shiyancao commented on PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#issuecomment-1239809915

   hi @rdblue / @rzhang10, are there more PRs to be developed before we can support default value in Iceberg? 
   
   I read in [PR 4732](https://github.com/apache/iceberg/pull/4732) and it seems that these are still pending to create but just want to double check and confirm.
   
   ```
   Add the JSON value parser
   Add as much as possible to Parquet, Avro, and ORC readers, like being able to read with a fake map of default values.
   ```
   
   Also, is there a place I can have a holistic view of the full issue? It seems that [issue 2039](https://github.com/apache/iceberg/issues/2039) was the one but it was not updated.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#issuecomment-1142680198

   Hi @rdblue I've updated the PR addressing your comments, could you please take a look again? Thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r883929911


##########
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",

Review Comment:
   What is the value of `0x`? I think I'd rather just remove it than have all this extra handling for it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r885268381


##########
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",

Review Comment:
   Sure, we can remove this, should we also update the spec according to this?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r908726977


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isFloatingPointNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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, 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);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "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);
+        }
+        return uuid;

Review Comment:
   @rzhang10, please fix this: https://github.com/apache/iceberg/pull/4871#discussion_r892899915



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r921688169


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+  private static final String VALUES = "values";
+
+  public static Object fromJson(Type type, JsonNode defaultValue) {
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        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);
+        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);
+        return retDecimal;
+      case STRING:
+        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);
+        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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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()) {
+          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.isoTimestampToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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));
+        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));
+        return ByteBuffer.wrap(binaryBytes);
+      case LIST:
+        return listFromJson(type, defaultValue);
+      case MAP:
+        return mapFromJson(type, defaultValue);
+      case STRUCT:
+        return structFromJson(type, defaultValue);
+      default:
+        throw new UnsupportedOperationException(String.format("Type: %s is not supported", type));
+    }
+  }
+
+  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);
+
+    List<Types.NestedField> 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;
+  }
+
+  private static Map<Object, Object> 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);
+
+    ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+    Iterator<JsonNode> keyIter = keys.iterator();
+    Type keyType = type.asMapType().keyType();
+    Iterator<JsonNode> 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<Object> 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) {
+    try {
+      JsonNode defaultValueJN = JsonUtil.mapper().readTree(defaultValue);
+      return fromJson(type, defaultValueJN);
+    } catch (IOException e) {
+      throw new IllegalArgumentException(
+          String.format("Failed to parse default as a %s value: %s", type, defaultValue), e);
+    }
+  }
+
+  public static String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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 defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Preconditions.checkArgument(
+            defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeBoolean((Boolean) defaultValue);
+        break;
+      case INTEGER:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Integer) defaultValue);
+        break;
+      case LONG:
+        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);
+        generator.writeNumber((Float) defaultValue);
+        break;
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Double) defaultValue);
+        break;
+      case DATE:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.daysToIsoDate((Integer) defaultValue));
+        break;
+      case TIME:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.microsToIsoTime((Long) defaultValue));
+        break;
+      case TIMESTAMP:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "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:
+        Preconditions.checkArgument(
+            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);
+        generator.writeString(defaultValue.toString());
+        break;
+      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();
+        int expectedLength = ((Types.FixedType) type).length();
+        Preconditions.checkArgument(actualLength == expectedLength,
+            "Invalid default %s value: %s, incorrect length: %s",
+            type, defaultValue, actualLength);
+        String fixedString = BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) defaultValue)));
+        generator.writeString(fixedString);
+        break;
+      case BINARY:
+        Preconditions.checkArgument(
+            defaultValue instanceof ByteBuffer, "Invalid default %s value: %s", type, 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);
+        generator.writeString(((BigDecimal) defaultValue).toPlainString());

Review Comment:
   From the documentation for `toPlainString`, I think this is a problem:
   
   > if this BigDecimal has a negative scale, the string resulting from this method will have a scale of zero when processed by the string constructor.
   
   That means there's still a case where round-trip conversion will fail. We have two choices:
   * Document this in the spec and use the scale of the type (don't fail if the type scale is negative and the number's scale is 0)
   * Use a different representation that encodes the scale, like `toString` that is guaranteed to round-trip correctly
   
   I would probably opt for the result of `toPlainString` when the scale is positive and `toString` when the scale is negative.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r918253590


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isFloatingPointNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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, 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);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "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);
+        }
+        return uuid;

Review Comment:
   Sorry, I forget this, I just adds testing the length to be 36.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#issuecomment-1194581269

   @rzhang10, looks like the PR needs to be rebased for conflicts.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893852759


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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;

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892899915


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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());

Review Comment:
   I think this should validate that the string's length is the length of a UUID string.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892927493


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");

Review Comment:
   I think you should check that the size of these array nodes matches.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892926551


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> defaultList = Lists.newArrayList();
+        for (JsonNode element : defaultValue) {
+          defaultList.add(parseDefaultFromJson(type.asListType().elementType(), element));

Review Comment:
   It may also be shorter to do it this way:
   
   ```java
   Type elementType = type.asListType().elementType();
   return Lists.newArrayList(Iterables.transform(arrayNode, e -> DefaultValueParser.fromJson(elementType, e)));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893847440


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> defaultStruct = Maps.newHashMap();

Review Comment:
   Makes sense, updated.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893948807


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 Object convertJavaDefaultForSerialization(Type type, Object value) {

Review Comment:
   @rzhang10, please look at the other parsers and match what they do. You should be using a `JsonGenerator`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r894949470


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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:
+        generator.writeString(DateTimeUtil.formatEpochDays((int) javaDefaultValue));
+        break;
+      case TIME:
+        generator.writeString(DateTimeUtil.formatTimeOfDayMicros((long) javaDefaultValue));
+        break;
+      case TIMESTAMP:
+        generator.writeString(DateTimeUtil.formatEpochTimeMicros(
+            (long) javaDefaultValue,
+            ((Types.TimestampType) type).shouldAdjustToUTC()));
+        break;
+      case STRING:
+        generator.writeString((String) javaDefaultValue);
+        break;
+      case UUID:
+        generator.writeString(javaDefaultValue.toString());
+        break;
+      case FIXED:
+      case BINARY:
+        generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) javaDefaultValue))));
+        break;
+      case DECIMAL:
+        generator.writeNumber((BigDecimal) javaDefaultValue);
+        break;
+      case LIST:
+        List<Object> defaultList = (List<Object>) javaDefaultValue;
+        Type elementType = type.asListType().elementType();
+        generator.writeStartArray();
+        for (Object elementDefault : defaultList) {

Review Comment:
   Minor: this isn't the element default, it is one element in the default list. I think this should just be `element`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r896344794


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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.isoDateToDays(defaultValue.textValue());
+      case TIME:
+        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);
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          return DateTimeUtil.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 IllegalArgumentException(String.format("Type: %s is not supported", type));

Review Comment:
   This should also be `UnsupportedOperationException`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r896342854


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("Cannot parse default as a %s value: %s", type,
+              defaultValue));

Review Comment:
   In Iceberg, you should never suppress an exception that was caught. You can add context, but always wrap the exception.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r896342157


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);

Review Comment:
   Sorry if it wasn't clear, but double needs to check `isFloatingPointNumber` as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r899561162


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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.isoDateToDays(defaultValue.textValue());
+      case TIME:
+        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);
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          return DateTimeUtil.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static void toJson(Type type, Object defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Preconditions.checkArgument(
+            defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeBoolean((Boolean) defaultValue);
+        break;
+      case INTEGER:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Integer) defaultValue);
+        break;
+      case LONG:
+        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);
+        generator.writeNumber((Float) defaultValue);
+        break;
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Double) defaultValue);
+        break;
+      case DATE:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer && (Integer) defaultValue >= 0,
+            "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);
+        generator.writeString(DateTimeUtil.microsToIsoTime((Long) defaultValue));
+        break;
+      case TIMESTAMP:
+        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:
+        Preconditions.checkArgument(
+            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);
+        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);
+        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);
+        generator.writeNumber((BigDecimal) defaultValue);
+        break;
+      case LIST:
+        Preconditions.checkArgument(defaultValue instanceof List, "Invalid default %s value: %s", type, defaultValue);
+        List<Object> defaultList = (List<Object>) defaultValue;
+        Type elementType = type.asListType().elementType();
+        generator.writeStartArray();
+        for (Object element : defaultList) {
+          toJson(elementType, element, generator);
+        }
+        generator.writeEndArray();
+        break;
+      case MAP:
+        Preconditions.checkArgument(defaultValue instanceof Map, "Invalid default %s value: %s", type, defaultValue);
+        Map<Object, Object> defaultMap = (Map<Object, Object>) defaultValue;
+        Type keyType = type.asMapType().keyType();
+        Type valueType = type.asMapType().valueType();
+
+        List<Object> valueList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        generator.writeStartObject();
+        generator.writeArrayFieldStart(KEYS);
+        for (Map.Entry<Object, Object> entry : defaultMap.entrySet()) {
+          toJson(keyType, entry.getKey(), generator);
+          valueList.add(entry.getValue());
+        }
+        generator.writeEndArray();
+        generator.writeArrayFieldStart(VALUES);
+        for (Object value : valueList) {
+          toJson(valueType, value, generator);
+        }
+        generator.writeEndArray();
+        generator.writeEndObject();
+        break;
+      case STRUCT:
+        Preconditions.checkArgument(
+            defaultValue instanceof StructLike, "Invalid default %s value: %s", type, defaultValue);
+        Types.StructType structType = type.asStructType();
+        List<Types.NestedField> fields = structType.fields();
+        StructLike defaultStruct = (StructLike) defaultValue;
+
+        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) {
+            generator.writeFieldName(String.valueOf(fieldId));
+            toJson(field.type(), fieldJavaDefaultValue, generator);
+          }
+        }
+        generator.writeEndObject();
+        break;
+      default:
+        throw new IllegalArgumentException(String.format("Type: %s is not supported", type));

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r899559980


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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.isoDateToDays(defaultValue.textValue());
+      case TIME:
+        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);
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          return DateTimeUtil.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static void toJson(Type type, Object defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Preconditions.checkArgument(
+            defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeBoolean((Boolean) defaultValue);
+        break;
+      case INTEGER:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Integer) defaultValue);
+        break;
+      case LONG:
+        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);
+        generator.writeNumber((Float) defaultValue);
+        break;
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Double) defaultValue);
+        break;
+      case DATE:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer && (Integer) defaultValue >= 0,
+            "Invalid default %s value: %s",
+            type,
+            defaultValue);

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r934915885


##########
format/spec.md:
##########
@@ -1132,15 +1132,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 |

Review Comment:
   @rdblue I added a detailed explanation of the string format, please take a look and let me know if there are any issues.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r909039593


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,38 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {
+    return dateFromDays(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String microsToIsoTime(long micros) {
+    return timeFromMicros(micros).format(DateTimeFormatter.ISO_LOCAL_TIME);
+  }
+
+  public static String microsToIsoDateTimeTz(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    return OffsetDateTime.of(localDateTime, ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+  }
+
+  public static String microsToIsoDateTime(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    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 isoTimeToMicros(String timeString) {
+    return microsFromTime(LocalTime.parse(timeString, DateTimeFormatter.ISO_LOCAL_TIME));
+  }
+
+  public static long isoDateTimeTzToMicros(String timestampString) {

Review Comment:
   These methods should not be named "DateTime". The conversion is between `Timestamp` or `Timestamptz` and `Micros`. This should be `isoTimestamptzToMicros`, for example.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r929269435


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isFloatingPointNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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, 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);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "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);
+        }
+        return uuid;

Review Comment:
   @rzhang10, please remove this.
   
   There is no need to wrap an `IllegalArgumentException` in a different `IllegalArgumentException`. You're correct that the checking done by `UUID.fromString` is insufficient, but that's why I requested that you add the precondition above that checks for the correct length.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r921688631


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+  private static final String VALUES = "values";
+
+  public static Object fromJson(Type type, JsonNode defaultValue) {
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        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);
+        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);
+        return retDecimal;
+      case STRING:
+        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);
+        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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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()) {
+          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.isoTimestampToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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));
+        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));
+        return ByteBuffer.wrap(binaryBytes);
+      case LIST:
+        return listFromJson(type, defaultValue);
+      case MAP:
+        return mapFromJson(type, defaultValue);
+      case STRUCT:
+        return structFromJson(type, defaultValue);
+      default:
+        throw new UnsupportedOperationException(String.format("Type: %s is not supported", type));
+    }
+  }
+
+  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);
+
+    List<Types.NestedField> 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;
+  }
+
+  private static Map<Object, Object> 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);
+
+    ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+    Iterator<JsonNode> keyIter = keys.iterator();
+    Type keyType = type.asMapType().keyType();
+    Iterator<JsonNode> 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<Object> 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) {
+    try {
+      JsonNode defaultValueJN = JsonUtil.mapper().readTree(defaultValue);
+      return fromJson(type, defaultValueJN);
+    } catch (IOException e) {
+      throw new IllegalArgumentException(
+          String.format("Failed to parse default as a %s value: %s", type, defaultValue), e);
+    }
+  }
+
+  public static String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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 defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Preconditions.checkArgument(
+            defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeBoolean((Boolean) defaultValue);
+        break;
+      case INTEGER:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Integer) defaultValue);
+        break;
+      case LONG:
+        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);
+        generator.writeNumber((Float) defaultValue);
+        break;
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Double) defaultValue);
+        break;
+      case DATE:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.daysToIsoDate((Integer) defaultValue));
+        break;
+      case TIME:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.microsToIsoTime((Long) defaultValue));
+        break;
+      case TIMESTAMP:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "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:
+        Preconditions.checkArgument(
+            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);
+        generator.writeString(defaultValue.toString());
+        break;
+      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();
+        int expectedLength = ((Types.FixedType) type).length();
+        Preconditions.checkArgument(actualLength == expectedLength,
+            "Invalid default %s value: %s, incorrect length: %s",
+            type, defaultValue, actualLength);
+        String fixedString = BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) defaultValue)));
+        generator.writeString(fixedString);
+        break;
+      case BINARY:
+        Preconditions.checkArgument(
+            defaultValue instanceof ByteBuffer, "Invalid default %s value: %s", type, 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);
+        generator.writeString(((BigDecimal) defaultValue).toPlainString());

Review Comment:
   We're also going to need to add more test cases for this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r921691614


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+  private static final String VALUES = "values";
+
+  public static Object fromJson(Type type, JsonNode defaultValue) {
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        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);
+        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);
+        return retDecimal;
+      case STRING:
+        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);
+        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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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()) {
+          String timestampTz = defaultValue.textValue();
+          Preconditions.checkArgument(DateTimeUtil.timestamptzIsOfUTCZone(timestampTz),
+              "Cannot parse default as a %s value: %s, timezone must be UTC",
+              type, defaultValue);

Review Comment:
   Non-UTC offsets are allowed by the spec. Can you remove this and the helper function, `timestamptzIsOfUTCZone`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r921690656


##########
core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.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 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:30Z\""},

Review Comment:
   The format for this should be `+00:00` and not `Z`. The spec says "must include a zone offset" and that means "+00:00". It also specifically allows non-UTC offsets, so we should probably update it to use +00:00.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r921690656


##########
core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.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 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:30Z\""},

Review Comment:
   The format for this should be `+00:00` and not `Z`. The spec says "must include a zone offset" and that means "+00:00". It also specifically allows non-UTC offsets.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r918268230


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isFloatingPointNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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, 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);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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));
+        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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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));
+    }
+  }
+
+  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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static void toJson(Type type, Object defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Preconditions.checkArgument(
+            defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeBoolean((Boolean) defaultValue);
+        break;
+      case INTEGER:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Integer) defaultValue);
+        break;
+      case LONG:
+        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);
+        generator.writeNumber((Float) defaultValue);
+        break;
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Double) defaultValue);
+        break;
+      case DATE:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.daysToIsoDate((Integer) defaultValue));
+        break;
+      case TIME:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeString(DateTimeUtil.microsToIsoTime((Long) defaultValue));
+        break;
+      case TIMESTAMP:
+        Preconditions.checkArgument(
+            defaultValue instanceof Long, "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:
+        Preconditions.checkArgument(
+            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);
+        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,
+            "Invalid default %s value: %s, incorrect length: %s",
+            type, defaultValue, valueLength);

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r918309519


##########
core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.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 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+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];
+
+      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

Review Comment:
   Sure, I adapted this change.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r922476411


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,43 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {
+    return dateFromDays(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String microsToIsoTime(long micros) {
+    return timeFromMicros(micros).format(DateTimeFormatter.ISO_LOCAL_TIME);
+  }
+
+  public static String microsToIsoDateTimeTz(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    return localDateTime.atOffset(ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+  }
+
+  public static String microsToIsoDateTime(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    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 isoTimeToMicros(String timeString) {
+    return microsFromTime(LocalTime.parse(timeString, DateTimeFormatter.ISO_LOCAL_TIME));
+  }
+
+  public static long isoTimestamptzToMicros(String timestampString) {
+    return microsFromTimestamptz(OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME));
+  }
+
+  public static boolean timestamptzIsOfUTCZone(String timestampString) {

Review Comment:
   done



##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,43 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {
+    return dateFromDays(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String microsToIsoTime(long micros) {
+    return timeFromMicros(micros).format(DateTimeFormatter.ISO_LOCAL_TIME);
+  }
+
+  public static String microsToIsoDateTimeTz(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    return localDateTime.atOffset(ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+  }
+
+  public static String microsToIsoDateTime(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    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 isoTimeToMicros(String timeString) {
+    return microsFromTime(LocalTime.parse(timeString, DateTimeFormatter.ISO_LOCAL_TIME));
+  }
+
+  public static long isoTimestamptzToMicros(String timestampString) {
+    return microsFromTimestamptz(OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME));
+  }
+
+  public static boolean timestamptzIsOfUTCZone(String timestampString) {
+    OffsetDateTime offsetDateTime = OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME);
+    return offsetDateTime.getOffset().equals(ZoneOffset.UTC) && timestampString.endsWith("Z");

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892929140


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> valueList = Lists.newArrayList(values.iterator());

Review Comment:
   It shouldn't be necessary to copy these into lists. Instead, you can iterate over them simultaneously after checking that the size is the same:
   
   ```java
     ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
   
     Iterator<JsonNode> keyIter = keys.iterator();
     Type keyType = type.asMapType().keyType();
     Iterator<JsonNode> valueIter = values.iterator();
     Type valueType = type.asMapType().valueType();
   
     while (keyIter.hasNext()) {
       mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
     }
   
     return mapBuilder.build();
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892934921


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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;

Review Comment:
   Shouldn't this throw an exception if the type is not supported?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892942142


##########
core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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<Object[]> 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.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()),
+         stringToJsonNode("{\"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\"}")}
+    });
+  }
+
+  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()

Review Comment:
   The parser should produce and accept strings, rather than doing it here in tests.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892963953


##########
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",

Review Comment:
   Yes, please update the spec as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892964286


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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);

Review Comment:
   I don't feel strongly about this. Either way is fine, but the first comment about the error message should be fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r894948344


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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)

Review Comment:
   Minor: Can you rename `javaDefaultValue` to `defaultValue`? Adding `java` is not very helpful because all of these are Java values. And the choice of which classes are used is actually the set of Iceberg internal representations.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r894950440


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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:
+        generator.writeString(DateTimeUtil.formatEpochDays((int) javaDefaultValue));
+        break;
+      case TIME:
+        generator.writeString(DateTimeUtil.formatTimeOfDayMicros((long) javaDefaultValue));
+        break;
+      case TIMESTAMP:
+        generator.writeString(DateTimeUtil.formatEpochTimeMicros(
+            (long) javaDefaultValue,
+            ((Types.TimestampType) type).shouldAdjustToUTC()));
+        break;
+      case STRING:
+        generator.writeString((String) javaDefaultValue);
+        break;
+      case UUID:
+        generator.writeString(javaDefaultValue.toString());
+        break;
+      case FIXED:
+      case BINARY:
+        generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) javaDefaultValue))));
+        break;
+      case DECIMAL:
+        generator.writeNumber((BigDecimal) javaDefaultValue);
+        break;
+      case LIST:
+        List<Object> defaultList = (List<Object>) javaDefaultValue;
+        Type elementType = type.asListType().elementType();
+        generator.writeStartArray();
+        for (Object elementDefault : defaultList) {
+          toJson(elementType, elementDefault, generator);
+        }
+        generator.writeEndArray();
+        break;
+      case MAP:
+        Map<Object, Object> defaultMap = (Map<Object, Object>) javaDefaultValue;
+        List<Object> keyList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        List<Object> valueList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        Type keyType = type.asMapType().keyType();
+        Type valueType = type.asMapType().valueType();
+
+        for (Map.Entry<Object, Object> entry : defaultMap.entrySet()) {
+          keyList.add(entry.getKey());
+          valueList.add(entry.getValue());
+        }
+        generator.writeStartObject();
+        generator.writeFieldName("keys");

Review Comment:
   Strings that are part of the serialization format should be constants at the top, like the other parsers use.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r894950365


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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:
+        generator.writeString(DateTimeUtil.formatEpochDays((int) javaDefaultValue));
+        break;
+      case TIME:
+        generator.writeString(DateTimeUtil.formatTimeOfDayMicros((long) javaDefaultValue));
+        break;
+      case TIMESTAMP:
+        generator.writeString(DateTimeUtil.formatEpochTimeMicros(
+            (long) javaDefaultValue,
+            ((Types.TimestampType) type).shouldAdjustToUTC()));
+        break;
+      case STRING:
+        generator.writeString((String) javaDefaultValue);
+        break;
+      case UUID:
+        generator.writeString(javaDefaultValue.toString());
+        break;
+      case FIXED:
+      case BINARY:
+        generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) javaDefaultValue))));
+        break;
+      case DECIMAL:
+        generator.writeNumber((BigDecimal) javaDefaultValue);
+        break;
+      case LIST:
+        List<Object> defaultList = (List<Object>) javaDefaultValue;
+        Type elementType = type.asListType().elementType();
+        generator.writeStartArray();
+        for (Object elementDefault : defaultList) {
+          toJson(elementType, elementDefault, generator);
+        }
+        generator.writeEndArray();
+        break;
+      case MAP:
+        Map<Object, Object> defaultMap = (Map<Object, Object>) javaDefaultValue;
+        List<Object> keyList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        List<Object> valueList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        Type keyType = type.asMapType().keyType();
+        Type valueType = type.asMapType().valueType();
+
+        for (Map.Entry<Object, Object> entry : defaultMap.entrySet()) {

Review Comment:
   You can get rid of an extra list allocation by writing keys as you iterate:
   
   ```java
     Map<Object, Object> defaultMap = (Map<Object, Object>) javaDefaultValue;
     Type keyType = type.asMapType().keyType();
     Type valueType = type.asMapType().valueType();
   
     generator.writeStartObject();
     generator.writeArrayFieldStart("keys");
     List<Object> values = Lists.newArrayListWithExpectedSize(defaultMap.size());
     for (Map.Entry<Object, Object> entry : defaultMap.entrySet()) {
       toJson(keyType, key, generator);
     }
     generator.writeEndArray();
     generator.writeArrayFieldStart("values");
     for (Object value : values) {
       toJson(valueType, value, generator);
     }
     generator.writeEndArray();
     generator.writeEndObject();
     break;
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893792033


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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(),

Review Comment:
   That means when the user can't specify the float value as `1`, but instead need to specify `1.0` for the parser to parse. Do you think this restriction is preferred?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893821728


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        Preconditions.checkArgument(
+            defaultValue.isTextual() && defaultValue.textValue().length() == ((Types.FixedType) type).length() * 2,
+            "Cannot parse %s to a %s value",

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893873313


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {

Review Comment:
   sure



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r894067363


##########
core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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 {

Review Comment:
   added



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r895235389


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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 {

Review Comment:
   add the logic to handle it



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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 {

Review Comment:
   added the logic to handle it



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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)

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r896340312


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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.isoDateToDays(defaultValue.textValue());
+      case TIME:
+        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);
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          return DateTimeUtil.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static void toJson(Type type, Object defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Preconditions.checkArgument(
+            defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeBoolean((Boolean) defaultValue);
+        break;
+      case INTEGER:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Integer) defaultValue);
+        break;
+      case LONG:
+        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);
+        generator.writeNumber((Float) defaultValue);
+        break;
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Double) defaultValue);
+        break;
+      case DATE:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer && (Integer) defaultValue >= 0,
+            "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);
+        generator.writeString(DateTimeUtil.microsToIsoTime((Long) defaultValue));
+        break;
+      case TIMESTAMP:
+        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:
+        Preconditions.checkArgument(
+            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);
+        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);
+        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);
+        generator.writeNumber((BigDecimal) defaultValue);
+        break;
+      case LIST:
+        Preconditions.checkArgument(defaultValue instanceof List, "Invalid default %s value: %s", type, defaultValue);
+        List<Object> defaultList = (List<Object>) defaultValue;
+        Type elementType = type.asListType().elementType();
+        generator.writeStartArray();
+        for (Object element : defaultList) {
+          toJson(elementType, element, generator);
+        }
+        generator.writeEndArray();
+        break;
+      case MAP:
+        Preconditions.checkArgument(defaultValue instanceof Map, "Invalid default %s value: %s", type, defaultValue);
+        Map<Object, Object> defaultMap = (Map<Object, Object>) defaultValue;
+        Type keyType = type.asMapType().keyType();
+        Type valueType = type.asMapType().valueType();
+
+        List<Object> valueList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        generator.writeStartObject();
+        generator.writeArrayFieldStart(KEYS);
+        for (Map.Entry<Object, Object> entry : defaultMap.entrySet()) {
+          toJson(keyType, entry.getKey(), generator);
+          valueList.add(entry.getValue());
+        }
+        generator.writeEndArray();
+        generator.writeArrayFieldStart(VALUES);
+        for (Object value : valueList) {
+          toJson(valueType, value, generator);
+        }
+        generator.writeEndArray();
+        generator.writeEndObject();
+        break;
+      case STRUCT:
+        Preconditions.checkArgument(
+            defaultValue instanceof StructLike, "Invalid default %s value: %s", type, defaultValue);
+        Types.StructType structType = type.asStructType();
+        List<Types.NestedField> fields = structType.fields();
+        StructLike defaultStruct = (StructLike) defaultValue;
+
+        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) {
+            generator.writeFieldName(String.valueOf(fieldId));
+            toJson(field.type(), fieldJavaDefaultValue, generator);
+          }
+        }
+        generator.writeEndObject();
+        break;
+      default:
+        throw new IllegalArgumentException(String.format("Type: %s is not supported", type));

Review Comment:
   This should be `UnsupportedOperationException`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r896340024


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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.isoDateToDays(defaultValue.textValue());
+      case TIME:
+        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);
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          return DateTimeUtil.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static void toJson(Type type, Object defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Preconditions.checkArgument(
+            defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeBoolean((Boolean) defaultValue);
+        break;
+      case INTEGER:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Integer) defaultValue);
+        break;
+      case LONG:
+        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);
+        generator.writeNumber((Float) defaultValue);
+        break;
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Double) defaultValue);
+        break;
+      case DATE:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer && (Integer) defaultValue >= 0,
+            "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);
+        generator.writeString(DateTimeUtil.microsToIsoTime((Long) defaultValue));
+        break;
+      case TIMESTAMP:
+        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:
+        Preconditions.checkArgument(
+            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);
+        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);
+        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);
+        generator.writeNumber((BigDecimal) defaultValue);
+        break;
+      case LIST:
+        Preconditions.checkArgument(defaultValue instanceof List, "Invalid default %s value: %s", type, defaultValue);
+        List<Object> defaultList = (List<Object>) defaultValue;
+        Type elementType = type.asListType().elementType();
+        generator.writeStartArray();
+        for (Object element : defaultList) {
+          toJson(elementType, element, generator);
+        }
+        generator.writeEndArray();
+        break;
+      case MAP:
+        Preconditions.checkArgument(defaultValue instanceof Map, "Invalid default %s value: %s", type, defaultValue);
+        Map<Object, Object> defaultMap = (Map<Object, Object>) defaultValue;
+        Type keyType = type.asMapType().keyType();
+        Type valueType = type.asMapType().valueType();
+
+        List<Object> valueList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        generator.writeStartObject();
+        generator.writeArrayFieldStart(KEYS);
+        for (Map.Entry<Object, Object> entry : defaultMap.entrySet()) {
+          toJson(keyType, entry.getKey(), generator);
+          valueList.add(entry.getValue());
+        }
+        generator.writeEndArray();
+        generator.writeArrayFieldStart(VALUES);
+        for (Object value : valueList) {
+          toJson(valueType, value, generator);
+        }
+        generator.writeEndArray();
+        generator.writeEndObject();
+        break;
+      case STRUCT:
+        Preconditions.checkArgument(
+            defaultValue instanceof StructLike, "Invalid default %s value: %s", type, defaultValue);
+        Types.StructType structType = type.asStructType();
+        List<Types.NestedField> fields = structType.fields();
+        StructLike defaultStruct = (StructLike) defaultValue;
+
+        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);

Review Comment:
   Nit: this still has "Java" in the variable name.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r896338553


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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.isoDateToDays(defaultValue.textValue());
+      case TIME:
+        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);
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          return DateTimeUtil.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static void toJson(Type type, Object defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+    switch (type.typeId()) {

Review Comment:
   Style: In Iceberg, there should be an empty newline between a control flow block and the following statement.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r896341844


##########
core/src/test/java/org/apache/iceberg/TestDefaultValuesParsingAndUnParsing.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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<Object[]> 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 {

Review Comment:
   Instead of having a parameterized test suite with just one test, can you refactor this and embed the list of types you want to test?
   
   Then you can combine all of the default values tests into a single suite.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r918287219


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isFloatingPointNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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, 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);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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));
+        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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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));
+    }
+  }
+
+  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);

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r918289008


##########
core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.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 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+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")),

Review Comment:
   done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r918315660


##########
core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.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 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+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];
+
+      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

Review Comment:
   @rdblue  There is one trivia worth mentioning, even the zero offset itself has 2 representations: `Z` or `+00:00` , and the java `DateTimeFormatter.ISO_OFFSET_DATE_TIME` will always output the `Z` representation. Do you mean to require users always set the timestamptz string to end with `Z` ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r921692739


##########
core/src/test/java/org/apache/iceberg/TestDefaultValueParser.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.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 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+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];
+
+      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

Review Comment:
   Thanks for asking! Sorry that I didn't see this, but I think we should use `+00:00` because that's the form in the spec.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r918318206


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isFloatingPointNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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, 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);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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));
+        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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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));
+    }
+  }
+
+  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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")

Review Comment:
   seems I can remove the suppressing without causing any stylecheck issues.



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isFloatingPointNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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, 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);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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));
+        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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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));
+    }
+  }
+
+  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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")

Review Comment:
   seems I can remove this suppressing without causing any stylecheck issues.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892899008


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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();

Review Comment:
   I think this also needs to validate that the decimal's scale matches the expected scale. That must always match or else it should throw an exception.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892940101


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 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 BaseEncoding.base16().encode(((ByteBuffer) value).array());

Review Comment:
   This is an incorrect and unsafe use of `array()`:
   * The buffer may not be backed by `byte[]`
   * The buffer may use an `arrayOffset()`
   * The buffer may have a non-zero position
   * The buffer may have a limit before the end of the array
   
   You can use `ByteBuffers.toByteArray(value)` if you need `byte[]`, although that may copy in some cases. I think it should be fine here since most of the time the backing array can be used directly because it will be created by `fromJson`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r892949166


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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);

Review Comment:
   Wouldn't reusing each type's `type.toString()` method better? As I see that is defined for each type.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893829101


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> valueList = Lists.newArrayList(values.iterator());

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r893906920


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class DefaultValueParser {
+  private DefaultValueParser() {
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static Object parseDefaultFromJson(Type type, JsonNode defaultValue) {
+
+    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:
+        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:
+        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:
+        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<Object> 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<Object, Object> defaultMap = Maps.newHashMap();
+        JsonNode keys = defaultValue.get("keys");
+        JsonNode values = defaultValue.get("values");
+        List<JsonNode> keyList = Lists.newArrayList(keys.iterator());
+        List<JsonNode> 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:
+        Preconditions.checkArgument(defaultValue.isObject(),
+            "Cannot parse %s to a %s value", defaultValue, type);
+        Map<Integer, Object> 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 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 BaseEncoding.base16().encode(((ByteBuffer) value).array());

Review Comment:
   Actually, just to argue about my original code, I think it's safe, upon reading the java API in 
   
   - https://docs.oracle.com/javase/7/docs/api/java/nio/ByteBuffer.html#wrap(byte[])
   - https://docs.oracle.com/javase/7/docs/api/java/nio/ByteBuffer.html#array()
   I think if we are sure that the ByteBuffer is always backed by an undrelying byte array (and this is indeed the case in `fromJson`), then calling `array()` should be safe.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r929278293


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+  private static final String VALUES = "values";
+
+  public static Object fromJson(Type type, JsonNode defaultValue) {
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        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);
+        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);
+        }
+        Preconditions.checkArgument(
+            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(),
+            "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);
+        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);
+        }
+        return uuid;
+      case DATE:
+        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);
+        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()) {
+          String timestampTz = defaultValue.textValue();
+          Preconditions.checkArgument(DateTimeUtil.isUTCTimestamptz(timestampTz),
+              "Cannot parse default as a %s value: %s, timezone must be UTC",

Review Comment:
   This isn't quite correct. The supported timestamp strings don't use zones because they can change. Instead, Iceberg uses offsets. This should be `offset must be +00:00`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r908723845


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  private static final String KEYS = "keys";
+
+  private static final String VALUES = "values";
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")

Review Comment:
   There's no need to suppress this. It is probably a good idea to refactor the complex cases into private methods.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r896339072


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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.isoDateToDays(defaultValue.textValue());
+      case TIME:
+        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);
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          return DateTimeUtil.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static void toJson(Type type, Object defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Preconditions.checkArgument(
+            defaultValue instanceof Boolean, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeBoolean((Boolean) defaultValue);
+        break;
+      case INTEGER:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Integer) defaultValue);
+        break;
+      case LONG:
+        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);
+        generator.writeNumber((Float) defaultValue);
+        break;
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue instanceof Double, "Invalid default %s value: %s", type, defaultValue);
+        generator.writeNumber((Double) defaultValue);
+        break;
+      case DATE:
+        Preconditions.checkArgument(
+            defaultValue instanceof Integer && (Integer) defaultValue >= 0,
+            "Invalid default %s value: %s",
+            type,
+            defaultValue);

Review Comment:
   Style (minor): the format arguments should be on the same line as the format string unless they need to be wrapped, not one per line.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r899557481


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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",

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("Cannot parse default as a %s value: %s", type,
+              defaultValue));

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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.isoDateToDays(defaultValue.textValue());
+      case TIME:
+        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);
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          return DateTimeUtil.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  public static void toJson(Type type, Object defaultValue, JsonGenerator generator)
+      throws IOException {
+    if (defaultValue == null) {
+      generator.writeNull();
+      return;
+    }
+    switch (type.typeId()) {

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r899559121


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("Cannot parse default as a %s value: %s", type,
+              defaultValue));

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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",

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r899561757


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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",

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("Cannot parse default as a %s value: %s", type,
+              defaultValue));

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r896345110


##########
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(

Review Comment:
   When using `assertThrows`, you should always test the exception message.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r894948094


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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)

Review Comment:
   Looks like this uses unchecked casts. Can you also add validation that the incoming values are the expected type?
   
   ```java
     case BOOLEAN:
       Preconditions.checkArgument(javaDefaultValue instanceof Boolean, "Invalid default %s value: %s", type, javaDefaultValue);
       generator.writeBoolean((Boolean) javaDefaultValue);
       break;
     ...
   ```
   
   Note that this also casts the the class that was checked, so it uses the object type, `Boolean` rather than primitive `boolean`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r894948446


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,38 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  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) {

Review Comment:
   Since there are two separate functions for the "from string" functions, I think this should match.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rzhang10 commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rzhang10 commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r895231088


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,38 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String formatEpochDays(int days) {

Review Comment:
   @rdblue Yes I tried to do that, but the `Literals` class is in API module, which can't call the util methods in `DateTImeUtil` since it's in the Core module. Do you mean to move the `DateTImeUtil` to API?



##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,38 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  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) {

Review Comment:
   done



##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,38 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String formatEpochDays(int days) {
+    return LocalDate.ofEpochDay(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String formatTimeOfDayMicros(long micros) {

Review Comment:
   done



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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:
+        generator.writeString(DateTimeUtil.formatEpochDays((int) javaDefaultValue));
+        break;
+      case TIME:
+        generator.writeString(DateTimeUtil.formatTimeOfDayMicros((long) javaDefaultValue));
+        break;
+      case TIMESTAMP:
+        generator.writeString(DateTimeUtil.formatEpochTimeMicros(
+            (long) javaDefaultValue,
+            ((Types.TimestampType) type).shouldAdjustToUTC()));
+        break;
+      case STRING:
+        generator.writeString((String) javaDefaultValue);
+        break;
+      case UUID:
+        generator.writeString(javaDefaultValue.toString());
+        break;
+      case FIXED:
+      case BINARY:
+        generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) javaDefaultValue))));
+        break;
+      case DECIMAL:
+        generator.writeNumber((BigDecimal) javaDefaultValue);
+        break;
+      case LIST:
+        List<Object> defaultList = (List<Object>) javaDefaultValue;
+        Type elementType = type.asListType().elementType();
+        generator.writeStartArray();
+        for (Object elementDefault : defaultList) {

Review Comment:
   fixed



##########
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() {

Review Comment:
   The `TestDefaultValuesParsingAndUnParsing` test is a Parameterized test class, I think leaving them separate might achieve better clarity?



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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)

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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:
+        generator.writeString(DateTimeUtil.formatEpochDays((int) javaDefaultValue));
+        break;
+      case TIME:
+        generator.writeString(DateTimeUtil.formatTimeOfDayMicros((long) javaDefaultValue));
+        break;
+      case TIMESTAMP:
+        generator.writeString(DateTimeUtil.formatEpochTimeMicros(
+            (long) javaDefaultValue,
+            ((Types.TimestampType) type).shouldAdjustToUTC()));
+        break;
+      case STRING:
+        generator.writeString((String) javaDefaultValue);
+        break;
+      case UUID:
+        generator.writeString(javaDefaultValue.toString());
+        break;
+      case FIXED:
+      case BINARY:
+        generator.writeString(BaseEncoding.base16().encode(ByteBuffers.toByteArray(((ByteBuffer) javaDefaultValue))));
+        break;
+      case DECIMAL:
+        generator.writeNumber((BigDecimal) javaDefaultValue);
+        break;
+      case LIST:
+        List<Object> defaultList = (List<Object>) javaDefaultValue;
+        Type elementType = type.asListType().elementType();
+        generator.writeStartArray();
+        for (Object elementDefault : defaultList) {
+          toJson(elementType, elementDefault, generator);
+        }
+        generator.writeEndArray();
+        break;
+      case MAP:
+        Map<Object, Object> defaultMap = (Map<Object, Object>) javaDefaultValue;
+        List<Object> keyList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        List<Object> valueList = Lists.newArrayListWithExpectedSize(defaultMap.size());
+        Type keyType = type.asMapType().keyType();
+        Type valueType = type.asMapType().valueType();
+
+        for (Map.Entry<Object, Object> entry : defaultMap.entrySet()) {
+          keyList.add(entry.getKey());
+          valueList.add(entry.getValue());
+        }
+        generator.writeStartObject();
+        generator.writeFieldName("keys");

Review Comment:
   fixed



##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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 fromJson(Type type, JsonNode defaultValue) {
+
+    if (defaultValue == null || defaultValue.isNull()) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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 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(),
+            "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.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));
+        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)));
+      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);
+
+        ImmutableMap.Builder<Object, Object> mapBuilder = ImmutableMap.builder();
+
+        Iterator<JsonNode> keyIter = keys.iterator();
+        Type keyType = type.asMapType().keyType();
+        Iterator<JsonNode> valueIter = values.iterator();
+        Type valueType = type.asMapType().valueType();
+
+        while (keyIter.hasNext()) {
+          mapBuilder.put(fromJson(keyType, keyIter.next()), fromJson(valueType, valueIter.next()));
+        }
+
+        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);
+
+        List<Types.NestedField> 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 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 String toJson(Type type, Object defaultValue) {
+    return toJson(type, defaultValue, false);
+  }
+
+  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:
+        generator.writeString(DateTimeUtil.formatEpochDays((int) javaDefaultValue));
+        break;
+      case TIME:
+        generator.writeString(DateTimeUtil.formatTimeOfDayMicros((long) javaDefaultValue));
+        break;
+      case TIMESTAMP:
+        generator.writeString(DateTimeUtil.formatEpochTimeMicros(
+            (long) javaDefaultValue,
+            ((Types.TimestampType) type).shouldAdjustToUTC()));
+        break;
+      case STRING:
+        generator.writeString((String) javaDefaultValue);

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r896344347


##########
core/src/main/java/org/apache/iceberg/DefaultValueParser.java:
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.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;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+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.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() {
+  }
+
+  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;
+    }
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        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);
+        return defaultValue.intValue();
+      case LONG:
+        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);
+        return defaultValue.floatValue();
+      case DOUBLE:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        return defaultValue.doubleValue();
+      case DECIMAL:
+        Preconditions.checkArgument(defaultValue.isNumber(),
+            "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 default as a %s value: %s", type, defaultValue);
+        return defaultValue.textValue();
+      case UUID:
+        Preconditions.checkArgument(defaultValue.isTextual(),
+            "Cannot parse default as a %s value: %s", type, defaultValue);
+        try {
+          UUID.fromString(defaultValue.textValue());
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format("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.isoDateToDays(defaultValue.textValue());
+      case TIME:
+        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);
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          return DateTimeUtil.isoDateTimeTzToMicros(defaultValue.textValue());
+        } else {
+          return DateTimeUtil.isoDateTimeToMicros(defaultValue.textValue());
+        }
+      case FIXED:
+        Preconditions.checkArgument(
+            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",

Review Comment:
   Can you rephrase this error message? We try to make messages helpful, but short. Instead, this should be "Cannot parse default %s value, incorrect length: %s"



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4871: Core: Implement default value parsing and unparsing

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4871:
URL: https://github.com/apache/iceberg/pull/4871#discussion_r909042607


##########
core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java:
##########
@@ -88,4 +88,38 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) {
   public static String formatTimestampMillis(long millis) {
     return DATE_FORMAT.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(millis), ZoneOffset.UTC));
   }
+
+  public static String daysToIsoDate(int days) {
+    return dateFromDays(days).format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  public static String microsToIsoTime(long micros) {
+    return timeFromMicros(micros).format(DateTimeFormatter.ISO_LOCAL_TIME);
+  }
+
+  public static String microsToIsoDateTimeTz(long micros) {
+    LocalDateTime localDateTime = timestampFromMicros(micros);
+    return OffsetDateTime.of(localDateTime, ZoneOffset.UTC).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);

Review Comment:
   Minor: other places use `localDateTime.atOffset(ZoneOffset.UTC)` instead. Can you use that for consistency?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org