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/04 20:38:25 UTC

[GitHub] [iceberg] kbendick opened a new pull request, #4693: Core - Add Parser for UpdateRequirement

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

   We need a parser for `UpdateTableRequest.UpdateRequirement` class.
   
   - Adds `UpdateRequirementParser`
   - Adds json serializer / deserializer to the RESTSerializers class
   - Adds tests for all of the individual implementations of `UpdateRequirement`, similar to `MetadataUpdate` tests.
   
   Open questions:
   Some of the fields don't necessarily line up with the spec.
   
   I've noted them on the PR. I was initially going to change them, but they are intertwined with some other classes (namely `MetadataUpdate` and would also possibly cause shadowing issues in some cases.
   
   Seeking feedback on naming for these.


-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  private static final String NAME = "name";
+  private static final String SNAPSHOT_ID = "snapshot-id";
+
+  // AssertLastAssignedFieldId
+  private static final String LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id";
+
+  // AssertCurrentSchemaID
+  private static final String SCHEMA_ID = "current-schema-id";
+
+  // AssertLastAssignedPartitionId
+  private static final String LAST_ASSIGNED_PARTITION_ID = "last-assigned-partition-id";
+
+  // AssertDefaultSpecID
+  private static final String SPEC_ID = "default-spec-id";
+
+  // AssertDefaultSortOrderID
+  private static final String SORT_ORDER_ID = "default-write-order-id";
+
+  private static final Map<Class<? extends UpdateTableRequest.UpdateRequirement>, String> TYPES = ImmutableMap
+      .<Class<? extends UpdateTableRequest.UpdateRequirement>, String>builder()
+      .put(UpdateRequirement.AssertTableUUID.class, ASSERT_TABLE_UUID)
+      .put(UpdateRequirement.AssertTableDoesNotExist.class, ASSERT_TABLE_DOES_NOT_EXIST)
+      .put(UpdateRequirement.AssertRefSnapshotID.class, ASSERT_REF_SNAPSHOT_ID)
+      .put(UpdateRequirement.AssertLastAssignedFieldId.class, ASSERT_LAST_ASSIGNED_FIELD_ID)
+      .put(UpdateRequirement.AssertCurrentSchemaID.class, ASSERT_CURRENT_SCHEMA_ID)
+      .put(UpdateRequirement.AssertLastAssignedPartitionId.class, ASSERT_LAST_ASSIGNED_PARTITION_ID)
+      .put(UpdateRequirement.AssertDefaultSpecID.class, ASSERT_DEFAULT_SPEC_ID)
+      .put(UpdateRequirement.AssertDefaultSortOrderID.class, ASSERT_DEFAULT_SORT_ORDER_ID)
+      .build();
+
+  public static String toJson(UpdateRequirement updateRequirement) {
+    return toJson(updateRequirement, false);
+  }
+
+  public static String toJson(UpdateRequirement updateRequirement, boolean pretty) {
+    try {
+      StringWriter writer = new StringWriter();
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      if (pretty) {
+        generator.useDefaultPrettyPrinter();
+      }
+      toJson(updateRequirement, generator);
+      generator.flush();
+      return writer.toString();
+    } catch (IOException e) {
+      throw new UncheckedIOException(
+          String.format("Failed to write update requirement json for: %s", updateRequirement), e);
+    }
+  }
+
+  public static void toJson(UpdateRequirement updateRequirement, JsonGenerator generator) throws IOException {
+    String requirementType = TYPES.get(updateRequirement.getClass());
+
+    generator.writeStartObject();
+    generator.writeStringField(TYPE, requirementType);
+
+    switch (requirementType) {
+      case ASSERT_TABLE_DOES_NOT_EXIST:
+        // No fields beyond the requirement itself
+        break;
+      case ASSERT_TABLE_UUID:
+        writeAssertTableUUID((UpdateRequirement.AssertTableUUID) updateRequirement, generator);
+        break;
+      case ASSERT_REF_SNAPSHOT_ID:
+        writeAssertRefSnapshotId((UpdateRequirement.AssertRefSnapshotID) updateRequirement, generator);
+        break;
+      case ASSERT_LAST_ASSIGNED_FIELD_ID:
+        writeAssertLastAssignedFieldId((UpdateRequirement.AssertLastAssignedFieldId) updateRequirement, generator);
+        break;
+      case ASSERT_LAST_ASSIGNED_PARTITION_ID:
+        writeAssertLastAssignedPartitionId(
+            (UpdateRequirement.AssertLastAssignedPartitionId) updateRequirement, generator);
+        break;
+      case ASSERT_CURRENT_SCHEMA_ID:
+        writeAssertCurrentSchemaId((UpdateRequirement.AssertCurrentSchemaID) updateRequirement, generator);
+        break;
+      case ASSERT_DEFAULT_SPEC_ID:
+        writeAssertDefaultSpecId((UpdateRequirement.AssertDefaultSpecID) updateRequirement, generator);
+        break;
+      case ASSERT_DEFAULT_SORT_ORDER_ID:
+        writeAssertDefaultSortOrderId((UpdateRequirement.AssertDefaultSortOrderID) updateRequirement, generator);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot convert update requirement to json. Unrecognized type: %s", requirementType));
+    }
+
+    generator.writeEndObject();
+  }
+
+  /**
+   * Read MetadataUpdate from a JSON string.
+   *
+   * @param json a JSON string of a MetadataUpdate
+   * @return a MetadataUpdate object
+   */
+  public static UpdateRequirement fromJson(String json) {
+    try {
+      return fromJson(JsonUtil.mapper().readValue(json, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to read JSON string: " + json, e);
+    }
+  }
+
+  public static UpdateRequirement fromJson(JsonNode jsonNode) {
+    Preconditions.checkArgument(jsonNode != null && jsonNode.isObject(),
+        "Cannot parse update requirement from non-object value: %s", jsonNode);
+    Preconditions.checkArgument(jsonNode.hasNonNull(TYPE), "Cannot parse update requirement. Missing field: type");
+    String type = JsonUtil.getString(TYPE, jsonNode).toLowerCase(Locale.ROOT);
+
+    switch (type) {
+      case ASSERT_TABLE_DOES_NOT_EXIST:
+        return readAssertTableDoesNotExist(jsonNode);
+      case ASSERT_TABLE_UUID:
+        return readAssertTableUUID(jsonNode);
+      case ASSERT_REF_SNAPSHOT_ID:
+        return readAssertRefSnapshotId(jsonNode);
+      case ASSERT_LAST_ASSIGNED_FIELD_ID:
+        return readAssertLastAssignedFieldId(jsonNode);
+      case ASSERT_LAST_ASSIGNED_PARTITION_ID:
+        return readAssertLastAssignedPartitionId(jsonNode);
+      case ASSERT_CURRENT_SCHEMA_ID:
+        return readAssertCurrentSchemaId(jsonNode);
+      case ASSERT_DEFAULT_SPEC_ID:
+        return readAssertDefaultSpecId(jsonNode);
+      case ASSERT_DEFAULT_SORT_ORDER_ID:
+        return readAssertDefaultSortOrderId(jsonNode);
+      default:
+        throw new UnsupportedOperationException(
+            String.format("Unrecognized update requirement. Cannot convert to json: %s", type));
+    }
+  }
+
+  private static void writeAssertTableUUID(UpdateRequirement.AssertTableUUID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeStringField(UUID, requirement.uuid());
+  }
+
+  private static void writeAssertRefSnapshotId(UpdateRequirement.AssertRefSnapshotID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeStringField(NAME, requirement.refName());
+    gen.writeNumberField(SNAPSHOT_ID, requirement.snapshotId());
+  }
+
+  private static void writeAssertLastAssignedFieldId(UpdateRequirement.AssertLastAssignedFieldId requirement,
+      JsonGenerator gen) throws IOException {
+    gen.writeNumberField(LAST_ASSIGNED_FIELD_ID, requirement.lastAssignedFieldId());
+  }
+
+  private static void writeAssertLastAssignedPartitionId(UpdateRequirement.AssertLastAssignedPartitionId requirement,
+      JsonGenerator gen) throws IOException {
+    gen.writeNumberField(LAST_ASSIGNED_PARTITION_ID, requirement.lastAssignedPartitionId());
+  }
+
+  private static void writeAssertCurrentSchemaId(UpdateRequirement.AssertCurrentSchemaID requirement,
+      JsonGenerator gen) throws IOException {
+    gen.writeNumberField(SCHEMA_ID, requirement.schemaId());
+  }
+
+  private static void writeAssertDefaultSpecId(UpdateRequirement.AssertDefaultSpecID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeNumberField(SPEC_ID, requirement.specId());
+  }
+
+  private static void writeAssertDefaultSortOrderId(UpdateRequirement.AssertDefaultSortOrderID requirement,
+      JsonGenerator gen) throws IOException {
+    gen.writeNumberField(SORT_ORDER_ID, requirement.sortOrderId());
+  }
+
+  @SuppressWarnings("unused")  // Keep same signature in case this requirement class evolves and gets fields
+  private static UpdateRequirement readAssertTableDoesNotExist(JsonNode node) {
+    return new UpdateRequirement.AssertTableDoesNotExist();
+  }
+
+  private static UpdateRequirement readAssertTableUUID(JsonNode node) {
+    String uuid = JsonUtil.getString(UUID, node);
+    return new UpdateRequirement.AssertTableUUID(uuid);
+  }
+
+  private static UpdateRequirement readAssertRefSnapshotId(JsonNode node) {
+    String name = JsonUtil.getStringOrNull(NAME, node);

Review Comment:
   Looking at the class definition, there are `null` checks with an else statement. That's what I based it on:
   
   https://github.com/apache/iceberg/blob/a19ac7835c8e251fd6a042e056f29ee63262adec/core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java#L279-L297
   



-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  // TODO - This is called `ref` in the spec.
+  //   https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1359-L1360
+  private static final String NAME = "name";
+  private static final String SNAPSHOT_ID = "snapshot-id";
+
+  // AssertLastAssignedFieldId
+  private static final String LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id";
+
+  // AssertCurrentSchemaID
+  private static final String SCHEMA_ID = "current-schema-id";
+
+  // AssertLastAssignedPartitionId
+  private static final String LAST_ASSIGNED_PARTITION_ID = "last-assigned-partition-id";
+
+  // AssertDefaultSpecID
+  private static final String SPEC_ID = "default-spec-id";
+
+  // AssertDefaultSortOrderID
+  // TODO - Is currently referred to as default-write-order-id in spec but class and comments use sort-order. Need to
+  //  update it in class or in spec
+  private static final String SORT_ORDER_ID = "default-write-order-id";

Review Comment:
   This doesn't necessarily need to be updated in the class or the spec, as we do use `sort-order-id` and `write-order-id` somewhat interchangeably.
   
   I hesitated to update the class, because the associated `MetadataUpdate` class uses `order-id` in the spec, but the classes are all named `SortOrder`. https://github.com/apache/iceberg/blob/a19ac7835c8e251fd6a042e056f29ee63262adec/open-api/rest-catalog-open-api.yaml#L1231-L1240



-- 
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] kbendick commented on pull request #4693: Core - Add Parser for UpdateRequirement

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

   These tests failed because of a flakey Flink 1.15 test. Several tests in that file needed to be updated for Flink 1.15 as they were order dependent, and Flink no longer is returning the records in a defined ordering - and we have no ordering guarantees per the spec unless we have an `ORDER BY` clause _in the select statement_.
   
   This PR should fix the issue: https://github.com/apache/iceberg/pull/4697


-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  // TODO - This is called `ref` in the spec.
+  //   https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1359-L1360
+  private static final String NAME = "name";

Review Comment:
   For reference, the spec field is listed here: https://github.com/apache/iceberg/blob/a19ac7835c8e251fd6a042e056f29ee63262adec/open-api/rest-catalog-open-api.yaml#L1359-L1360



-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  // TODO - This is called `ref` in the spec.
+  //   https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1359-L1360
+  private static final String NAME = "name";

Review Comment:
   For reference, the spec field is listed here: https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1359-L1360



-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  private static final String NAME = "name";
+  private static final String SNAPSHOT_ID = "snapshot-id";
+
+  // AssertLastAssignedFieldId
+  private static final String LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id";
+
+  // AssertCurrentSchemaID
+  private static final String SCHEMA_ID = "current-schema-id";
+
+  // AssertLastAssignedPartitionId
+  private static final String LAST_ASSIGNED_PARTITION_ID = "last-assigned-partition-id";
+
+  // AssertDefaultSpecID
+  private static final String SPEC_ID = "default-spec-id";
+
+  // AssertDefaultSortOrderID
+  private static final String SORT_ORDER_ID = "default-write-order-id";
+
+  private static final Map<Class<? extends UpdateTableRequest.UpdateRequirement>, String> TYPES = ImmutableMap
+      .<Class<? extends UpdateTableRequest.UpdateRequirement>, String>builder()
+      .put(UpdateRequirement.AssertTableUUID.class, ASSERT_TABLE_UUID)
+      .put(UpdateRequirement.AssertTableDoesNotExist.class, ASSERT_TABLE_DOES_NOT_EXIST)
+      .put(UpdateRequirement.AssertRefSnapshotID.class, ASSERT_REF_SNAPSHOT_ID)
+      .put(UpdateRequirement.AssertLastAssignedFieldId.class, ASSERT_LAST_ASSIGNED_FIELD_ID)
+      .put(UpdateRequirement.AssertCurrentSchemaID.class, ASSERT_CURRENT_SCHEMA_ID)
+      .put(UpdateRequirement.AssertLastAssignedPartitionId.class, ASSERT_LAST_ASSIGNED_PARTITION_ID)
+      .put(UpdateRequirement.AssertDefaultSpecID.class, ASSERT_DEFAULT_SPEC_ID)
+      .put(UpdateRequirement.AssertDefaultSortOrderID.class, ASSERT_DEFAULT_SORT_ORDER_ID)
+      .build();
+
+  public static String toJson(UpdateRequirement updateRequirement) {
+    return toJson(updateRequirement, false);
+  }
+
+  public static String toJson(UpdateRequirement updateRequirement, boolean pretty) {
+    try {
+      StringWriter writer = new StringWriter();
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      if (pretty) {
+        generator.useDefaultPrettyPrinter();
+      }
+      toJson(updateRequirement, generator);
+      generator.flush();
+      return writer.toString();
+    } catch (IOException e) {
+      throw new UncheckedIOException(
+          String.format("Failed to write update requirement json for: %s", updateRequirement), e);
+    }
+  }
+
+  public static void toJson(UpdateRequirement updateRequirement, JsonGenerator generator) throws IOException {
+    String requirementType = TYPES.get(updateRequirement.getClass());
+
+    generator.writeStartObject();
+    generator.writeStringField(TYPE, requirementType);
+
+    switch (requirementType) {
+      case ASSERT_TABLE_DOES_NOT_EXIST:
+        // No fields beyond the requirement itself
+        break;
+      case ASSERT_TABLE_UUID:
+        writeAssertTableUUID((UpdateRequirement.AssertTableUUID) updateRequirement, generator);
+        break;
+      case ASSERT_REF_SNAPSHOT_ID:
+        writeAssertRefSnapshotId((UpdateRequirement.AssertRefSnapshotID) updateRequirement, generator);
+        break;
+      case ASSERT_LAST_ASSIGNED_FIELD_ID:
+        writeAssertLastAssignedFieldId((UpdateRequirement.AssertLastAssignedFieldId) updateRequirement, generator);
+        break;
+      case ASSERT_LAST_ASSIGNED_PARTITION_ID:
+        writeAssertLastAssignedPartitionId(
+            (UpdateRequirement.AssertLastAssignedPartitionId) updateRequirement, generator);
+        break;
+      case ASSERT_CURRENT_SCHEMA_ID:
+        writeAssertCurrentSchemaId((UpdateRequirement.AssertCurrentSchemaID) updateRequirement, generator);
+        break;
+      case ASSERT_DEFAULT_SPEC_ID:
+        writeAssertDefaultSpecId((UpdateRequirement.AssertDefaultSpecID) updateRequirement, generator);
+        break;
+      case ASSERT_DEFAULT_SORT_ORDER_ID:
+        writeAssertDefaultSortOrderId((UpdateRequirement.AssertDefaultSortOrderID) updateRequirement, generator);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot convert update requirement to json. Unrecognized type: %s", requirementType));
+    }
+
+    generator.writeEndObject();
+  }
+
+  /**
+   * Read MetadataUpdate from a JSON string.
+   *
+   * @param json a JSON string of a MetadataUpdate
+   * @return a MetadataUpdate object
+   */
+  public static UpdateRequirement fromJson(String json) {
+    try {
+      return fromJson(JsonUtil.mapper().readValue(json, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to read JSON string: " + json, e);
+    }
+  }
+
+  public static UpdateRequirement fromJson(JsonNode jsonNode) {
+    Preconditions.checkArgument(jsonNode != null && jsonNode.isObject(),
+        "Cannot parse update requirement from non-object value: %s", jsonNode);
+    Preconditions.checkArgument(jsonNode.hasNonNull(TYPE), "Cannot parse update requirement. Missing field: type");
+    String type = JsonUtil.getString(TYPE, jsonNode).toLowerCase(Locale.ROOT);
+
+    switch (type) {
+      case ASSERT_TABLE_DOES_NOT_EXIST:
+        return readAssertTableDoesNotExist(jsonNode);
+      case ASSERT_TABLE_UUID:
+        return readAssertTableUUID(jsonNode);
+      case ASSERT_REF_SNAPSHOT_ID:
+        return readAssertRefSnapshotId(jsonNode);
+      case ASSERT_LAST_ASSIGNED_FIELD_ID:
+        return readAssertLastAssignedFieldId(jsonNode);
+      case ASSERT_LAST_ASSIGNED_PARTITION_ID:
+        return readAssertLastAssignedPartitionId(jsonNode);
+      case ASSERT_CURRENT_SCHEMA_ID:
+        return readAssertCurrentSchemaId(jsonNode);
+      case ASSERT_DEFAULT_SPEC_ID:
+        return readAssertDefaultSpecId(jsonNode);
+      case ASSERT_DEFAULT_SORT_ORDER_ID:
+        return readAssertDefaultSortOrderId(jsonNode);
+      default:
+        throw new UnsupportedOperationException(
+            String.format("Unrecognized update requirement. Cannot convert to json: %s", type));
+    }
+  }
+
+  private static void writeAssertTableUUID(UpdateRequirement.AssertTableUUID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeStringField(UUID, requirement.uuid());
+  }
+
+  private static void writeAssertRefSnapshotId(UpdateRequirement.AssertRefSnapshotID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeStringField(NAME, requirement.refName());
+    gen.writeNumberField(SNAPSHOT_ID, requirement.snapshotId());

Review Comment:
   I though this would write `null` if the `Long` value were null. I'll write a test to ensure it though.



-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  // TODO - This is called `ref` in the spec.
+  //   https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1359-L1360
+  private static final String NAME = "name";
+  private static final String SNAPSHOT_ID = "snapshot-id";
+
+  // AssertLastAssignedFieldId
+  private static final String LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id";
+
+  // AssertCurrentSchemaID
+  private static final String SCHEMA_ID = "current-schema-id";
+
+  // AssertLastAssignedPartitionId
+  private static final String LAST_ASSIGNED_PARTITION_ID = "last-assigned-partition-id";
+
+  // AssertDefaultSpecID
+  private static final String SPEC_ID = "default-spec-id";
+
+  // AssertDefaultSortOrderID
+  // TODO - Is currently referred to as default-write-order-id in spec but class and comments use sort-order. Need to
+  //  update it in class or in spec
+  private static final String SORT_ORDER_ID = "default-write-order-id";

Review Comment:
   Ok. I'm somewhat partial to `write-order-id` as the condition is about how the data is written (and not how the data is necessarily read), but `sort-order-id` does match much more of the existing code.



-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  // TODO - This is called `ref` in the spec.
+  //   https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1359-L1360
+  private static final String NAME = "name";
+  private static final String SNAPSHOT_ID = "snapshot-id";
+
+  // AssertLastAssignedFieldId
+  private static final String LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id";
+
+  // AssertCurrentSchemaID
+  private static final String SCHEMA_ID = "current-schema-id";
+
+  // AssertLastAssignedPartitionId
+  private static final String LAST_ASSIGNED_PARTITION_ID = "last-assigned-partition-id";
+
+  // AssertDefaultSpecID
+  private static final String SPEC_ID = "default-spec-id";
+
+  // AssertDefaultSortOrderID
+  // TODO - Is currently referred to as default-write-order-id in spec but class and comments use sort-order. Need to
+  //  update it in class or in spec
+  private static final String SORT_ORDER_ID = "default-write-order-id";

Review Comment:
   I updated the spec for this class, as well as for the `SetDefaultSortOrder` metadata update.
   
   I'll be opening a PR for that serializer shortly where I'll use `sort-order-id` instead of `order-id`. If you'd like me to update the spec in another PR or hold off on updating the spec for `MetadataUpdate.SetDefaultSortOrder` to be `sort-order-id`, let me know. I did it all in this PR.



-- 
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 #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  private static final String NAME = "name";
+  private static final String SNAPSHOT_ID = "snapshot-id";
+
+  // AssertLastAssignedFieldId
+  private static final String LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id";
+
+  // AssertCurrentSchemaID
+  private static final String SCHEMA_ID = "current-schema-id";
+
+  // AssertLastAssignedPartitionId
+  private static final String LAST_ASSIGNED_PARTITION_ID = "last-assigned-partition-id";
+
+  // AssertDefaultSpecID
+  private static final String SPEC_ID = "default-spec-id";
+
+  // AssertDefaultSortOrderID
+  private static final String SORT_ORDER_ID = "default-write-order-id";
+
+  private static final Map<Class<? extends UpdateTableRequest.UpdateRequirement>, String> TYPES = ImmutableMap
+      .<Class<? extends UpdateTableRequest.UpdateRequirement>, String>builder()
+      .put(UpdateRequirement.AssertTableUUID.class, ASSERT_TABLE_UUID)
+      .put(UpdateRequirement.AssertTableDoesNotExist.class, ASSERT_TABLE_DOES_NOT_EXIST)
+      .put(UpdateRequirement.AssertRefSnapshotID.class, ASSERT_REF_SNAPSHOT_ID)
+      .put(UpdateRequirement.AssertLastAssignedFieldId.class, ASSERT_LAST_ASSIGNED_FIELD_ID)
+      .put(UpdateRequirement.AssertCurrentSchemaID.class, ASSERT_CURRENT_SCHEMA_ID)
+      .put(UpdateRequirement.AssertLastAssignedPartitionId.class, ASSERT_LAST_ASSIGNED_PARTITION_ID)
+      .put(UpdateRequirement.AssertDefaultSpecID.class, ASSERT_DEFAULT_SPEC_ID)
+      .put(UpdateRequirement.AssertDefaultSortOrderID.class, ASSERT_DEFAULT_SORT_ORDER_ID)
+      .build();
+
+  public static String toJson(UpdateRequirement updateRequirement) {
+    return toJson(updateRequirement, false);
+  }
+
+  public static String toJson(UpdateRequirement updateRequirement, boolean pretty) {
+    try {
+      StringWriter writer = new StringWriter();
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      if (pretty) {
+        generator.useDefaultPrettyPrinter();
+      }
+      toJson(updateRequirement, generator);
+      generator.flush();
+      return writer.toString();
+    } catch (IOException e) {
+      throw new UncheckedIOException(
+          String.format("Failed to write update requirement json for: %s", updateRequirement), e);
+    }
+  }
+
+  public static void toJson(UpdateRequirement updateRequirement, JsonGenerator generator) throws IOException {
+    String requirementType = TYPES.get(updateRequirement.getClass());
+
+    generator.writeStartObject();
+    generator.writeStringField(TYPE, requirementType);
+
+    switch (requirementType) {
+      case ASSERT_TABLE_DOES_NOT_EXIST:
+        // No fields beyond the requirement itself
+        break;
+      case ASSERT_TABLE_UUID:
+        writeAssertTableUUID((UpdateRequirement.AssertTableUUID) updateRequirement, generator);
+        break;
+      case ASSERT_REF_SNAPSHOT_ID:
+        writeAssertRefSnapshotId((UpdateRequirement.AssertRefSnapshotID) updateRequirement, generator);
+        break;
+      case ASSERT_LAST_ASSIGNED_FIELD_ID:
+        writeAssertLastAssignedFieldId((UpdateRequirement.AssertLastAssignedFieldId) updateRequirement, generator);
+        break;
+      case ASSERT_LAST_ASSIGNED_PARTITION_ID:
+        writeAssertLastAssignedPartitionId(
+            (UpdateRequirement.AssertLastAssignedPartitionId) updateRequirement, generator);
+        break;
+      case ASSERT_CURRENT_SCHEMA_ID:
+        writeAssertCurrentSchemaId((UpdateRequirement.AssertCurrentSchemaID) updateRequirement, generator);
+        break;
+      case ASSERT_DEFAULT_SPEC_ID:
+        writeAssertDefaultSpecId((UpdateRequirement.AssertDefaultSpecID) updateRequirement, generator);
+        break;
+      case ASSERT_DEFAULT_SORT_ORDER_ID:
+        writeAssertDefaultSortOrderId((UpdateRequirement.AssertDefaultSortOrderID) updateRequirement, generator);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot convert update requirement to json. Unrecognized type: %s", requirementType));
+    }
+
+    generator.writeEndObject();
+  }
+
+  /**
+   * Read MetadataUpdate from a JSON string.
+   *
+   * @param json a JSON string of a MetadataUpdate
+   * @return a MetadataUpdate object
+   */
+  public static UpdateRequirement fromJson(String json) {
+    try {
+      return fromJson(JsonUtil.mapper().readValue(json, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to read JSON string: " + json, e);
+    }
+  }
+
+  public static UpdateRequirement fromJson(JsonNode jsonNode) {
+    Preconditions.checkArgument(jsonNode != null && jsonNode.isObject(),
+        "Cannot parse update requirement from non-object value: %s", jsonNode);
+    Preconditions.checkArgument(jsonNode.hasNonNull(TYPE), "Cannot parse update requirement. Missing field: type");
+    String type = JsonUtil.getString(TYPE, jsonNode).toLowerCase(Locale.ROOT);
+
+    switch (type) {
+      case ASSERT_TABLE_DOES_NOT_EXIST:
+        return readAssertTableDoesNotExist(jsonNode);
+      case ASSERT_TABLE_UUID:
+        return readAssertTableUUID(jsonNode);
+      case ASSERT_REF_SNAPSHOT_ID:
+        return readAssertRefSnapshotId(jsonNode);
+      case ASSERT_LAST_ASSIGNED_FIELD_ID:
+        return readAssertLastAssignedFieldId(jsonNode);
+      case ASSERT_LAST_ASSIGNED_PARTITION_ID:
+        return readAssertLastAssignedPartitionId(jsonNode);
+      case ASSERT_CURRENT_SCHEMA_ID:
+        return readAssertCurrentSchemaId(jsonNode);
+      case ASSERT_DEFAULT_SPEC_ID:
+        return readAssertDefaultSpecId(jsonNode);
+      case ASSERT_DEFAULT_SORT_ORDER_ID:
+        return readAssertDefaultSortOrderId(jsonNode);
+      default:
+        throw new UnsupportedOperationException(
+            String.format("Unrecognized update requirement. Cannot convert to json: %s", type));
+    }
+  }
+
+  private static void writeAssertTableUUID(UpdateRequirement.AssertTableUUID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeStringField(UUID, requirement.uuid());
+  }
+
+  private static void writeAssertRefSnapshotId(UpdateRequirement.AssertRefSnapshotID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeStringField(NAME, requirement.refName());
+    gen.writeNumberField(SNAPSHOT_ID, requirement.snapshotId());
+  }
+
+  private static void writeAssertLastAssignedFieldId(UpdateRequirement.AssertLastAssignedFieldId requirement,
+      JsonGenerator gen) throws IOException {
+    gen.writeNumberField(LAST_ASSIGNED_FIELD_ID, requirement.lastAssignedFieldId());
+  }
+
+  private static void writeAssertLastAssignedPartitionId(UpdateRequirement.AssertLastAssignedPartitionId requirement,
+      JsonGenerator gen) throws IOException {
+    gen.writeNumberField(LAST_ASSIGNED_PARTITION_ID, requirement.lastAssignedPartitionId());
+  }
+
+  private static void writeAssertCurrentSchemaId(UpdateRequirement.AssertCurrentSchemaID requirement,
+      JsonGenerator gen) throws IOException {
+    gen.writeNumberField(SCHEMA_ID, requirement.schemaId());
+  }
+
+  private static void writeAssertDefaultSpecId(UpdateRequirement.AssertDefaultSpecID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeNumberField(SPEC_ID, requirement.specId());
+  }
+
+  private static void writeAssertDefaultSortOrderId(UpdateRequirement.AssertDefaultSortOrderID requirement,
+      JsonGenerator gen) throws IOException {
+    gen.writeNumberField(SORT_ORDER_ID, requirement.sortOrderId());
+  }
+
+  @SuppressWarnings("unused")  // Keep same signature in case this requirement class evolves and gets fields
+  private static UpdateRequirement readAssertTableDoesNotExist(JsonNode node) {
+    return new UpdateRequirement.AssertTableDoesNotExist();
+  }
+
+  private static UpdateRequirement readAssertTableUUID(JsonNode node) {
+    String uuid = JsonUtil.getString(UUID, node);
+    return new UpdateRequirement.AssertTableUUID(uuid);
+  }
+
+  private static UpdateRequirement readAssertRefSnapshotId(JsonNode node) {
+    String name = JsonUtil.getStringOrNull(NAME, node);

Review Comment:
   Isn't `ref` (or `name`) required? I would expect this to not allow `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] kbendick commented on pull request #4693: Core - Add Parser for UpdateRequirement

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

   I've rebased this now that https://github.com/apache/iceberg/pull/4697 has been merged.


-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  // TODO - This is called `ref` in the spec.
+  //   https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1359-L1360
+  private static final String NAME = "name";
+  private static final String SNAPSHOT_ID = "snapshot-id";
+
+  // AssertLastAssignedFieldId
+  private static final String LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id";
+
+  // AssertCurrentSchemaID
+  private static final String SCHEMA_ID = "current-schema-id";
+
+  // AssertLastAssignedPartitionId
+  private static final String LAST_ASSIGNED_PARTITION_ID = "last-assigned-partition-id";
+
+  // AssertDefaultSpecID
+  private static final String SPEC_ID = "default-spec-id";
+
+  // AssertDefaultSortOrderID
+  // TODO - Is currently referred to as default-write-order-id in spec but class and comments use sort-order. Need to
+  //  update it in class or in spec
+  private static final String SORT_ORDER_ID = "default-write-order-id";

Review Comment:
   Here's the associated class for the equivalent `MetadataUpdate`: https://github.com/apache/iceberg/blob/a19ac7835c8e251fd6a042e056f29ee63262adec/core/src/main/java/org/apache/iceberg/MetadataUpdate.java#L167-L182



-- 
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 #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  // TODO - This is called `ref` in the spec.
+  //   https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1359-L1360
+  private static final String NAME = "name";

Review Comment:
   How about `ref-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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateRequirementParser.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.rest.requests;
+
+import java.util.List;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestUpdateRequirementParser {
+
+  @Test
+  public void testUpdateRequirementWithoutRequirementTypeCannotParse() {
+    List<String> invalidJson = ImmutableList.of(
+        "{\"type\":null,\"uuid\":\"2cc52516-5e73-41f2-b139-545d41a4e151\"}",
+        "{\"uuid\":\"2cc52516-5e73-41f2-b139-545d41a4e151\"}"
+    );
+
+    for (String json : invalidJson) {
+      AssertHelpers.assertThrows(
+          "UpdateRequirement without a recognized requirement type should fail to deserialize",
+          IllegalArgumentException.class,
+          "Cannot parse update requirement. Missing field: type",
+          () -> UpdateRequirementParser.fromJson(json));
+    }
+  }
+
+  @Test
+  public void testAssertUUIDFromJson() {
+    String requirementType = UpdateRequirementParser.ASSERT_TABLE_UUID;
+    String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151";
+    String json = String.format("{\"type\":\"assert-table-uuid\",\"uuid\":\"%s\"}", uuid);
+    UpdateRequirement expected = new UpdateRequirement.AssertTableUUID(uuid);
+    assertEquals(requirementType, expected, UpdateRequirementParser.fromJson(json));
+  }
+
+  @Test
+  public void testAssertUUIDToJson() {
+    String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151";
+    String expected = String.format("{\"type\":\"assert-table-uuid\",\"uuid\":\"%s\"}", uuid);
+    UpdateRequirement actual = new UpdateRequirement.AssertTableUUID(uuid);
+    Assert.assertEquals("AssertTableUUID should convert to the correct JSON value",
+        expected, UpdateRequirementParser.toJson(actual));
+  }
+
+  @Test
+  public void testAssertTableDoesNotExistFromJson() {
+    String requirementType = UpdateRequirementParser.ASSERT_TABLE_DOES_NOT_EXIST;
+    String json = "{\"type\":\"assert-create\"}";
+    UpdateRequirement expected = new UpdateRequirement.AssertTableDoesNotExist();
+    assertEquals(requirementType, expected, UpdateRequirementParser.fromJson(json));
+  }
+
+  @Test
+  public void testAssertTableDoesNotExistToJson() {
+    String expected  = "{\"type\":\"assert-create\"}";
+    UpdateRequirement actual = new UpdateRequirement.AssertTableDoesNotExist();
+    Assert.assertEquals("AssertTableDoesNotExist should convert to the correct JSON value",
+        expected, UpdateRequirementParser.toJson(actual));
+  }
+
+  @Test
+  public void testAssertRefSnapshotIdToJson() {
+    String requirementType = UpdateRequirementParser.ASSERT_REF_SNAPSHOT_ID;
+    String name = "snapshot-name";
+    Long snapshotId = 1L;
+    String json = String.format("{\"type\":\"%s\",\"name\":\"%s\",\"snapshot-id\":%d}",
+        requirementType, name, snapshotId);
+    UpdateRequirement expected = new UpdateRequirement.AssertRefSnapshotID(name, snapshotId);
+    assertEquals(requirementType, expected, UpdateRequirementParser.fromJson(json));
+  }
+
+  @Test
+  public void testAssertRefSnapshotIdFromJson() {

Review Comment:
   Opened a PR to update this, as the error also exists in `getIntOrNull` as well.
   
   https://github.com/apache/iceberg/pull/4696



-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  // TODO - This is called `ref` in the spec.
+  //   https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1359-L1360
+  private static final String NAME = "name";
+  private static final String SNAPSHOT_ID = "snapshot-id";
+
+  // AssertLastAssignedFieldId
+  private static final String LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id";
+
+  // AssertCurrentSchemaID
+  private static final String SCHEMA_ID = "current-schema-id";
+
+  // AssertLastAssignedPartitionId
+  private static final String LAST_ASSIGNED_PARTITION_ID = "last-assigned-partition-id";
+
+  // AssertDefaultSpecID
+  private static final String SPEC_ID = "default-spec-id";
+
+  // AssertDefaultSortOrderID
+  // TODO - Is currently referred to as default-write-order-id in spec but class and comments use sort-order. Need to
+  //  update it in class or in spec
+  private static final String SORT_ORDER_ID = "default-write-order-id";

Review Comment:
   This doesn't necessarily need to be updated in the class or the spec, as we do use `sort-order-id` and `write-order-id` somewhat interchangeably.
   
   I hesitated to update the class, because the associated `MetadataUpdate` class uses `order-id` in the spec, but the classes are all named `SortOrder`. https://github.com/apache/iceberg/blob/a19ac7835c8e251fd6a042e056f29ee63262adec/open-api/rest-catalog-open-api.yaml#L1231-L1240
   
   Happy to update either one. We could also leave these two as not implemented, review the easier parts of this PR, and then add a second PR to handle it in isolation.



-- 
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 #4693: Core - Add Parser for UpdateRequirement

Posted by GitBox <gi...@apache.org>.
rdblue merged PR #4693:
URL: https://github.com/apache/iceberg/pull/4693


-- 
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 #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  // TODO - This is called `ref` in the spec.
+  //   https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1359-L1360
+  private static final String NAME = "name";

Review Comment:
   I'm not sure that this actually needs to change. This parser is the only place where this name is used, so it doesn't really conflict or cause confusion if this is "ref". The `name` field of the assertion could still be set as "ref" in the serialization.



-- 
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] kbendick commented on pull request #4693: Core - Add Parser for UpdateRequirement

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

   It looks like `null` handling is incorrect in `JsonUtil::getLongOrNull`. I have a PR to update it that should likely go in first (though the same changes for `getLongOrNull` are reflected in this PR,  but not the changes for `getIntOrNull`): https://github.com/apache/iceberg/pull/4696


-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  // TODO - This is called `ref` in the spec.
+  //   https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1359-L1360
+  private static final String NAME = "name";

Review Comment:
   Yeah. I'll just make the `NAME` field as `ref` in the parser.



-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1233,9 +1233,9 @@ components:
         - $ref: '#/components/schemas/BaseUpdate'
         - type: object
           required:
-            - order-id
+            - sort-order-id
           properties:
-            order-id:
+            sort-order-id:

Review Comment:
   I am moving this change to https://github.com/apache/iceberg/pull/4716, as that's where the associated parser is being written.



-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateRequirementParser.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.rest.requests;
+
+import java.util.List;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestUpdateRequirementParser {
+
+  @Test
+  public void testUpdateRequirementWithoutRequirementTypeCannotParse() {
+    List<String> invalidJson = ImmutableList.of(
+        "{\"type\":null,\"uuid\":\"2cc52516-5e73-41f2-b139-545d41a4e151\"}",
+        "{\"uuid\":\"2cc52516-5e73-41f2-b139-545d41a4e151\"}"
+    );
+
+    for (String json : invalidJson) {
+      AssertHelpers.assertThrows(
+          "UpdateRequirement without a recognized requirement type should fail to deserialize",
+          IllegalArgumentException.class,
+          "Cannot parse update requirement. Missing field: type",
+          () -> UpdateRequirementParser.fromJson(json));
+    }
+  }
+
+  @Test
+  public void testAssertUUIDFromJson() {
+    String requirementType = UpdateRequirementParser.ASSERT_TABLE_UUID;
+    String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151";
+    String json = String.format("{\"type\":\"assert-table-uuid\",\"uuid\":\"%s\"}", uuid);
+    UpdateRequirement expected = new UpdateRequirement.AssertTableUUID(uuid);
+    assertEquals(requirementType, expected, UpdateRequirementParser.fromJson(json));
+  }
+
+  @Test
+  public void testAssertUUIDToJson() {
+    String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151";
+    String expected = String.format("{\"type\":\"assert-table-uuid\",\"uuid\":\"%s\"}", uuid);
+    UpdateRequirement actual = new UpdateRequirement.AssertTableUUID(uuid);
+    Assert.assertEquals("AssertTableUUID should convert to the correct JSON value",
+        expected, UpdateRequirementParser.toJson(actual));
+  }
+
+  @Test
+  public void testAssertTableDoesNotExistFromJson() {
+    String requirementType = UpdateRequirementParser.ASSERT_TABLE_DOES_NOT_EXIST;
+    String json = "{\"type\":\"assert-create\"}";
+    UpdateRequirement expected = new UpdateRequirement.AssertTableDoesNotExist();
+    assertEquals(requirementType, expected, UpdateRequirementParser.fromJson(json));
+  }
+
+  @Test
+  public void testAssertTableDoesNotExistToJson() {
+    String expected  = "{\"type\":\"assert-create\"}";
+    UpdateRequirement actual = new UpdateRequirement.AssertTableDoesNotExist();
+    Assert.assertEquals("AssertTableDoesNotExist should convert to the correct JSON value",
+        expected, UpdateRequirementParser.toJson(actual));
+  }
+
+  @Test
+  public void testAssertRefSnapshotIdToJson() {
+    String requirementType = UpdateRequirementParser.ASSERT_REF_SNAPSHOT_ID;
+    String name = "snapshot-name";
+    Long snapshotId = 1L;
+    String json = String.format("{\"type\":\"%s\",\"name\":\"%s\",\"snapshot-id\":%d}",
+        requirementType, name, snapshotId);
+    UpdateRequirement expected = new UpdateRequirement.AssertRefSnapshotID(name, snapshotId);
+    assertEquals(requirementType, expected, UpdateRequirementParser.fromJson(json));
+  }
+
+  @Test
+  public void testAssertRefSnapshotIdFromJson() {

Review Comment:
   Added the tests. There seems to be a bug in `JsonUtils.getLongOrNull`. Should I open a separate PR for that for the benefit of people who cherry-pick?
   
   The check should be `node.hasNonNull`. I'm going to open a separate PR because there's also issues with the int one I think. 
   
   https://github.com/apache/iceberg/blob/4ae2002bd46bf8e1c20db03cffc6319237e4d74a/core/src/main/java/org/apache/iceberg/util/JsonUtil.java#L71-L79_



-- 
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 #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  // TODO - This is called `ref` in the spec.
+  //   https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1359-L1360
+  private static final String NAME = "name";
+  private static final String SNAPSHOT_ID = "snapshot-id";
+
+  // AssertLastAssignedFieldId
+  private static final String LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id";
+
+  // AssertCurrentSchemaID
+  private static final String SCHEMA_ID = "current-schema-id";
+
+  // AssertLastAssignedPartitionId
+  private static final String LAST_ASSIGNED_PARTITION_ID = "last-assigned-partition-id";
+
+  // AssertDefaultSpecID
+  private static final String SPEC_ID = "default-spec-id";
+
+  // AssertDefaultSortOrderID
+  // TODO - Is currently referred to as default-write-order-id in spec but class and comments use sort-order. Need to
+  //  update it in class or in spec
+  private static final String SORT_ORDER_ID = "default-write-order-id";

Review Comment:
   I'd be fine renaming this to `assert-default-sort-order-id` and `default-sort-order-id` so that it matches `SetDefaultSortOrder` with `sort-order-id`. We'd need to update 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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  private static final String NAME = "name";
+  private static final String SNAPSHOT_ID = "snapshot-id";
+
+  // AssertLastAssignedFieldId
+  private static final String LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id";
+
+  // AssertCurrentSchemaID
+  private static final String SCHEMA_ID = "current-schema-id";
+
+  // AssertLastAssignedPartitionId
+  private static final String LAST_ASSIGNED_PARTITION_ID = "last-assigned-partition-id";
+
+  // AssertDefaultSpecID
+  private static final String SPEC_ID = "default-spec-id";
+
+  // AssertDefaultSortOrderID
+  private static final String SORT_ORDER_ID = "default-write-order-id";
+
+  private static final Map<Class<? extends UpdateTableRequest.UpdateRequirement>, String> TYPES = ImmutableMap
+      .<Class<? extends UpdateTableRequest.UpdateRequirement>, String>builder()
+      .put(UpdateRequirement.AssertTableUUID.class, ASSERT_TABLE_UUID)
+      .put(UpdateRequirement.AssertTableDoesNotExist.class, ASSERT_TABLE_DOES_NOT_EXIST)
+      .put(UpdateRequirement.AssertRefSnapshotID.class, ASSERT_REF_SNAPSHOT_ID)
+      .put(UpdateRequirement.AssertLastAssignedFieldId.class, ASSERT_LAST_ASSIGNED_FIELD_ID)
+      .put(UpdateRequirement.AssertCurrentSchemaID.class, ASSERT_CURRENT_SCHEMA_ID)
+      .put(UpdateRequirement.AssertLastAssignedPartitionId.class, ASSERT_LAST_ASSIGNED_PARTITION_ID)
+      .put(UpdateRequirement.AssertDefaultSpecID.class, ASSERT_DEFAULT_SPEC_ID)
+      .put(UpdateRequirement.AssertDefaultSortOrderID.class, ASSERT_DEFAULT_SORT_ORDER_ID)
+      .build();
+
+  public static String toJson(UpdateRequirement updateRequirement) {
+    return toJson(updateRequirement, false);
+  }
+
+  public static String toJson(UpdateRequirement updateRequirement, boolean pretty) {
+    try {
+      StringWriter writer = new StringWriter();
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      if (pretty) {
+        generator.useDefaultPrettyPrinter();
+      }
+      toJson(updateRequirement, generator);
+      generator.flush();
+      return writer.toString();
+    } catch (IOException e) {
+      throw new UncheckedIOException(
+          String.format("Failed to write update requirement json for: %s", updateRequirement), e);
+    }
+  }
+
+  public static void toJson(UpdateRequirement updateRequirement, JsonGenerator generator) throws IOException {
+    String requirementType = TYPES.get(updateRequirement.getClass());
+
+    generator.writeStartObject();
+    generator.writeStringField(TYPE, requirementType);
+
+    switch (requirementType) {
+      case ASSERT_TABLE_DOES_NOT_EXIST:
+        // No fields beyond the requirement itself
+        break;
+      case ASSERT_TABLE_UUID:
+        writeAssertTableUUID((UpdateRequirement.AssertTableUUID) updateRequirement, generator);
+        break;
+      case ASSERT_REF_SNAPSHOT_ID:
+        writeAssertRefSnapshotId((UpdateRequirement.AssertRefSnapshotID) updateRequirement, generator);
+        break;
+      case ASSERT_LAST_ASSIGNED_FIELD_ID:
+        writeAssertLastAssignedFieldId((UpdateRequirement.AssertLastAssignedFieldId) updateRequirement, generator);
+        break;
+      case ASSERT_LAST_ASSIGNED_PARTITION_ID:
+        writeAssertLastAssignedPartitionId(
+            (UpdateRequirement.AssertLastAssignedPartitionId) updateRequirement, generator);
+        break;
+      case ASSERT_CURRENT_SCHEMA_ID:
+        writeAssertCurrentSchemaId((UpdateRequirement.AssertCurrentSchemaID) updateRequirement, generator);
+        break;
+      case ASSERT_DEFAULT_SPEC_ID:
+        writeAssertDefaultSpecId((UpdateRequirement.AssertDefaultSpecID) updateRequirement, generator);
+        break;
+      case ASSERT_DEFAULT_SORT_ORDER_ID:
+        writeAssertDefaultSortOrderId((UpdateRequirement.AssertDefaultSortOrderID) updateRequirement, generator);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot convert update requirement to json. Unrecognized type: %s", requirementType));
+    }
+
+    generator.writeEndObject();
+  }
+
+  /**
+   * Read MetadataUpdate from a JSON string.
+   *
+   * @param json a JSON string of a MetadataUpdate
+   * @return a MetadataUpdate object
+   */
+  public static UpdateRequirement fromJson(String json) {
+    try {
+      return fromJson(JsonUtil.mapper().readValue(json, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to read JSON string: " + json, e);
+    }
+  }
+
+  public static UpdateRequirement fromJson(JsonNode jsonNode) {
+    Preconditions.checkArgument(jsonNode != null && jsonNode.isObject(),
+        "Cannot parse update requirement from non-object value: %s", jsonNode);
+    Preconditions.checkArgument(jsonNode.hasNonNull(TYPE), "Cannot parse update requirement. Missing field: type");
+    String type = JsonUtil.getString(TYPE, jsonNode).toLowerCase(Locale.ROOT);
+
+    switch (type) {
+      case ASSERT_TABLE_DOES_NOT_EXIST:
+        return readAssertTableDoesNotExist(jsonNode);
+      case ASSERT_TABLE_UUID:
+        return readAssertTableUUID(jsonNode);
+      case ASSERT_REF_SNAPSHOT_ID:
+        return readAssertRefSnapshotId(jsonNode);
+      case ASSERT_LAST_ASSIGNED_FIELD_ID:
+        return readAssertLastAssignedFieldId(jsonNode);
+      case ASSERT_LAST_ASSIGNED_PARTITION_ID:
+        return readAssertLastAssignedPartitionId(jsonNode);
+      case ASSERT_CURRENT_SCHEMA_ID:
+        return readAssertCurrentSchemaId(jsonNode);
+      case ASSERT_DEFAULT_SPEC_ID:
+        return readAssertDefaultSpecId(jsonNode);
+      case ASSERT_DEFAULT_SORT_ORDER_ID:
+        return readAssertDefaultSortOrderId(jsonNode);
+      default:
+        throw new UnsupportedOperationException(
+            String.format("Unrecognized update requirement. Cannot convert to json: %s", type));
+    }
+  }
+
+  private static void writeAssertTableUUID(UpdateRequirement.AssertTableUUID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeStringField(UUID, requirement.uuid());
+  }
+
+  private static void writeAssertRefSnapshotId(UpdateRequirement.AssertRefSnapshotID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeStringField(NAME, requirement.refName());
+    gen.writeNumberField(SNAPSHOT_ID, requirement.snapshotId());
+  }
+
+  private static void writeAssertLastAssignedFieldId(UpdateRequirement.AssertLastAssignedFieldId requirement,
+      JsonGenerator gen) throws IOException {
+    gen.writeNumberField(LAST_ASSIGNED_FIELD_ID, requirement.lastAssignedFieldId());
+  }
+
+  private static void writeAssertLastAssignedPartitionId(UpdateRequirement.AssertLastAssignedPartitionId requirement,
+      JsonGenerator gen) throws IOException {
+    gen.writeNumberField(LAST_ASSIGNED_PARTITION_ID, requirement.lastAssignedPartitionId());
+  }
+
+  private static void writeAssertCurrentSchemaId(UpdateRequirement.AssertCurrentSchemaID requirement,
+      JsonGenerator gen) throws IOException {
+    gen.writeNumberField(SCHEMA_ID, requirement.schemaId());
+  }
+
+  private static void writeAssertDefaultSpecId(UpdateRequirement.AssertDefaultSpecID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeNumberField(SPEC_ID, requirement.specId());
+  }
+
+  private static void writeAssertDefaultSortOrderId(UpdateRequirement.AssertDefaultSortOrderID requirement,
+      JsonGenerator gen) throws IOException {
+    gen.writeNumberField(SORT_ORDER_ID, requirement.sortOrderId());
+  }
+
+  @SuppressWarnings("unused")  // Keep same signature in case this requirement class evolves and gets fields
+  private static UpdateRequirement readAssertTableDoesNotExist(JsonNode node) {
+    return new UpdateRequirement.AssertTableDoesNotExist();
+  }
+
+  private static UpdateRequirement readAssertTableUUID(JsonNode node) {
+    String uuid = JsonUtil.getString(UUID, node);
+    return new UpdateRequirement.AssertTableUUID(uuid);
+  }
+
+  private static UpdateRequirement readAssertRefSnapshotId(JsonNode node) {
+    String name = JsonUtil.getStringOrNull(NAME, node);

Review Comment:
   Oh wait I read that wrong from when I renamed it to `ref` temporarily. You're right.



-- 
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 #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  private static final String NAME = "name";
+  private static final String SNAPSHOT_ID = "snapshot-id";
+
+  // AssertLastAssignedFieldId
+  private static final String LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id";
+
+  // AssertCurrentSchemaID
+  private static final String SCHEMA_ID = "current-schema-id";
+
+  // AssertLastAssignedPartitionId
+  private static final String LAST_ASSIGNED_PARTITION_ID = "last-assigned-partition-id";
+
+  // AssertDefaultSpecID
+  private static final String SPEC_ID = "default-spec-id";
+
+  // AssertDefaultSortOrderID
+  private static final String SORT_ORDER_ID = "default-write-order-id";
+
+  private static final Map<Class<? extends UpdateTableRequest.UpdateRequirement>, String> TYPES = ImmutableMap
+      .<Class<? extends UpdateTableRequest.UpdateRequirement>, String>builder()
+      .put(UpdateRequirement.AssertTableUUID.class, ASSERT_TABLE_UUID)
+      .put(UpdateRequirement.AssertTableDoesNotExist.class, ASSERT_TABLE_DOES_NOT_EXIST)
+      .put(UpdateRequirement.AssertRefSnapshotID.class, ASSERT_REF_SNAPSHOT_ID)
+      .put(UpdateRequirement.AssertLastAssignedFieldId.class, ASSERT_LAST_ASSIGNED_FIELD_ID)
+      .put(UpdateRequirement.AssertCurrentSchemaID.class, ASSERT_CURRENT_SCHEMA_ID)
+      .put(UpdateRequirement.AssertLastAssignedPartitionId.class, ASSERT_LAST_ASSIGNED_PARTITION_ID)
+      .put(UpdateRequirement.AssertDefaultSpecID.class, ASSERT_DEFAULT_SPEC_ID)
+      .put(UpdateRequirement.AssertDefaultSortOrderID.class, ASSERT_DEFAULT_SORT_ORDER_ID)
+      .build();
+
+  public static String toJson(UpdateRequirement updateRequirement) {
+    return toJson(updateRequirement, false);
+  }
+
+  public static String toJson(UpdateRequirement updateRequirement, boolean pretty) {
+    try {
+      StringWriter writer = new StringWriter();
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      if (pretty) {
+        generator.useDefaultPrettyPrinter();
+      }
+      toJson(updateRequirement, generator);
+      generator.flush();
+      return writer.toString();
+    } catch (IOException e) {
+      throw new UncheckedIOException(
+          String.format("Failed to write update requirement json for: %s", updateRequirement), e);
+    }
+  }
+
+  public static void toJson(UpdateRequirement updateRequirement, JsonGenerator generator) throws IOException {
+    String requirementType = TYPES.get(updateRequirement.getClass());
+
+    generator.writeStartObject();
+    generator.writeStringField(TYPE, requirementType);
+
+    switch (requirementType) {
+      case ASSERT_TABLE_DOES_NOT_EXIST:
+        // No fields beyond the requirement itself
+        break;
+      case ASSERT_TABLE_UUID:
+        writeAssertTableUUID((UpdateRequirement.AssertTableUUID) updateRequirement, generator);
+        break;
+      case ASSERT_REF_SNAPSHOT_ID:
+        writeAssertRefSnapshotId((UpdateRequirement.AssertRefSnapshotID) updateRequirement, generator);
+        break;
+      case ASSERT_LAST_ASSIGNED_FIELD_ID:
+        writeAssertLastAssignedFieldId((UpdateRequirement.AssertLastAssignedFieldId) updateRequirement, generator);
+        break;
+      case ASSERT_LAST_ASSIGNED_PARTITION_ID:
+        writeAssertLastAssignedPartitionId(
+            (UpdateRequirement.AssertLastAssignedPartitionId) updateRequirement, generator);
+        break;
+      case ASSERT_CURRENT_SCHEMA_ID:
+        writeAssertCurrentSchemaId((UpdateRequirement.AssertCurrentSchemaID) updateRequirement, generator);
+        break;
+      case ASSERT_DEFAULT_SPEC_ID:
+        writeAssertDefaultSpecId((UpdateRequirement.AssertDefaultSpecID) updateRequirement, generator);
+        break;
+      case ASSERT_DEFAULT_SORT_ORDER_ID:
+        writeAssertDefaultSortOrderId((UpdateRequirement.AssertDefaultSortOrderID) updateRequirement, generator);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot convert update requirement to json. Unrecognized type: %s", requirementType));
+    }
+
+    generator.writeEndObject();
+  }
+
+  /**
+   * Read MetadataUpdate from a JSON string.
+   *
+   * @param json a JSON string of a MetadataUpdate
+   * @return a MetadataUpdate object
+   */
+  public static UpdateRequirement fromJson(String json) {
+    try {
+      return fromJson(JsonUtil.mapper().readValue(json, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to read JSON string: " + json, e);
+    }
+  }
+
+  public static UpdateRequirement fromJson(JsonNode jsonNode) {
+    Preconditions.checkArgument(jsonNode != null && jsonNode.isObject(),
+        "Cannot parse update requirement from non-object value: %s", jsonNode);
+    Preconditions.checkArgument(jsonNode.hasNonNull(TYPE), "Cannot parse update requirement. Missing field: type");
+    String type = JsonUtil.getString(TYPE, jsonNode).toLowerCase(Locale.ROOT);
+
+    switch (type) {
+      case ASSERT_TABLE_DOES_NOT_EXIST:
+        return readAssertTableDoesNotExist(jsonNode);
+      case ASSERT_TABLE_UUID:
+        return readAssertTableUUID(jsonNode);
+      case ASSERT_REF_SNAPSHOT_ID:
+        return readAssertRefSnapshotId(jsonNode);
+      case ASSERT_LAST_ASSIGNED_FIELD_ID:
+        return readAssertLastAssignedFieldId(jsonNode);
+      case ASSERT_LAST_ASSIGNED_PARTITION_ID:
+        return readAssertLastAssignedPartitionId(jsonNode);
+      case ASSERT_CURRENT_SCHEMA_ID:
+        return readAssertCurrentSchemaId(jsonNode);
+      case ASSERT_DEFAULT_SPEC_ID:
+        return readAssertDefaultSpecId(jsonNode);
+      case ASSERT_DEFAULT_SORT_ORDER_ID:
+        return readAssertDefaultSortOrderId(jsonNode);
+      default:
+        throw new UnsupportedOperationException(
+            String.format("Unrecognized update requirement. Cannot convert to json: %s", type));
+    }
+  }
+
+  private static void writeAssertTableUUID(UpdateRequirement.AssertTableUUID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeStringField(UUID, requirement.uuid());
+  }
+
+  private static void writeAssertRefSnapshotId(UpdateRequirement.AssertRefSnapshotID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeStringField(NAME, requirement.refName());
+    gen.writeNumberField(SNAPSHOT_ID, requirement.snapshotId());

Review Comment:
   (This is the only one that can be 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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  // TODO - This is called `ref` in the spec.
+  //   https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1359-L1360
+  private static final String NAME = "name";

Review Comment:
   This field, `name`, is the name of the branch or tag. In the spec it's referred to as `ref`, but using `ref` could potentially cause confusion with `TableMetadata.ref` in this code here: https://github.com/apache/iceberg/blob/a19ac7835c8e251fd6a042e056f29ee63262adec/core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java#L261-L297



-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1361,7 +1361,8 @@ components:
         uuid:
           type: string
         snapshot-id:
-          type: integer
+          type: interger
+          format: int64

Review Comment:
   This is a long, so updated the doc to reflect that.



-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  private static final String NAME = "name";
+  private static final String SNAPSHOT_ID = "snapshot-id";
+
+  // AssertLastAssignedFieldId
+  private static final String LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id";
+
+  // AssertCurrentSchemaID
+  private static final String SCHEMA_ID = "current-schema-id";
+
+  // AssertLastAssignedPartitionId
+  private static final String LAST_ASSIGNED_PARTITION_ID = "last-assigned-partition-id";
+
+  // AssertDefaultSpecID
+  private static final String SPEC_ID = "default-spec-id";
+
+  // AssertDefaultSortOrderID
+  private static final String SORT_ORDER_ID = "default-write-order-id";
+
+  private static final Map<Class<? extends UpdateTableRequest.UpdateRequirement>, String> TYPES = ImmutableMap
+      .<Class<? extends UpdateTableRequest.UpdateRequirement>, String>builder()
+      .put(UpdateRequirement.AssertTableUUID.class, ASSERT_TABLE_UUID)
+      .put(UpdateRequirement.AssertTableDoesNotExist.class, ASSERT_TABLE_DOES_NOT_EXIST)
+      .put(UpdateRequirement.AssertRefSnapshotID.class, ASSERT_REF_SNAPSHOT_ID)
+      .put(UpdateRequirement.AssertLastAssignedFieldId.class, ASSERT_LAST_ASSIGNED_FIELD_ID)
+      .put(UpdateRequirement.AssertCurrentSchemaID.class, ASSERT_CURRENT_SCHEMA_ID)
+      .put(UpdateRequirement.AssertLastAssignedPartitionId.class, ASSERT_LAST_ASSIGNED_PARTITION_ID)
+      .put(UpdateRequirement.AssertDefaultSpecID.class, ASSERT_DEFAULT_SPEC_ID)
+      .put(UpdateRequirement.AssertDefaultSortOrderID.class, ASSERT_DEFAULT_SORT_ORDER_ID)
+      .build();
+
+  public static String toJson(UpdateRequirement updateRequirement) {
+    return toJson(updateRequirement, false);
+  }
+
+  public static String toJson(UpdateRequirement updateRequirement, boolean pretty) {
+    try {
+      StringWriter writer = new StringWriter();
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      if (pretty) {
+        generator.useDefaultPrettyPrinter();
+      }
+      toJson(updateRequirement, generator);
+      generator.flush();
+      return writer.toString();
+    } catch (IOException e) {
+      throw new UncheckedIOException(
+          String.format("Failed to write update requirement json for: %s", updateRequirement), e);
+    }
+  }
+
+  public static void toJson(UpdateRequirement updateRequirement, JsonGenerator generator) throws IOException {
+    String requirementType = TYPES.get(updateRequirement.getClass());
+
+    generator.writeStartObject();
+    generator.writeStringField(TYPE, requirementType);
+
+    switch (requirementType) {
+      case ASSERT_TABLE_DOES_NOT_EXIST:
+        // No fields beyond the requirement itself
+        break;
+      case ASSERT_TABLE_UUID:
+        writeAssertTableUUID((UpdateRequirement.AssertTableUUID) updateRequirement, generator);
+        break;
+      case ASSERT_REF_SNAPSHOT_ID:
+        writeAssertRefSnapshotId((UpdateRequirement.AssertRefSnapshotID) updateRequirement, generator);
+        break;
+      case ASSERT_LAST_ASSIGNED_FIELD_ID:
+        writeAssertLastAssignedFieldId((UpdateRequirement.AssertLastAssignedFieldId) updateRequirement, generator);
+        break;
+      case ASSERT_LAST_ASSIGNED_PARTITION_ID:
+        writeAssertLastAssignedPartitionId(
+            (UpdateRequirement.AssertLastAssignedPartitionId) updateRequirement, generator);
+        break;
+      case ASSERT_CURRENT_SCHEMA_ID:
+        writeAssertCurrentSchemaId((UpdateRequirement.AssertCurrentSchemaID) updateRequirement, generator);
+        break;
+      case ASSERT_DEFAULT_SPEC_ID:
+        writeAssertDefaultSpecId((UpdateRequirement.AssertDefaultSpecID) updateRequirement, generator);
+        break;
+      case ASSERT_DEFAULT_SORT_ORDER_ID:
+        writeAssertDefaultSortOrderId((UpdateRequirement.AssertDefaultSortOrderID) updateRequirement, generator);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot convert update requirement to json. Unrecognized type: %s", requirementType));
+    }
+
+    generator.writeEndObject();
+  }
+
+  /**
+   * Read MetadataUpdate from a JSON string.
+   *
+   * @param json a JSON string of a MetadataUpdate
+   * @return a MetadataUpdate object
+   */
+  public static UpdateRequirement fromJson(String json) {
+    try {
+      return fromJson(JsonUtil.mapper().readValue(json, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to read JSON string: " + json, e);
+    }
+  }
+
+  public static UpdateRequirement fromJson(JsonNode jsonNode) {
+    Preconditions.checkArgument(jsonNode != null && jsonNode.isObject(),
+        "Cannot parse update requirement from non-object value: %s", jsonNode);
+    Preconditions.checkArgument(jsonNode.hasNonNull(TYPE), "Cannot parse update requirement. Missing field: type");
+    String type = JsonUtil.getString(TYPE, jsonNode).toLowerCase(Locale.ROOT);
+
+    switch (type) {
+      case ASSERT_TABLE_DOES_NOT_EXIST:
+        return readAssertTableDoesNotExist(jsonNode);
+      case ASSERT_TABLE_UUID:
+        return readAssertTableUUID(jsonNode);
+      case ASSERT_REF_SNAPSHOT_ID:
+        return readAssertRefSnapshotId(jsonNode);
+      case ASSERT_LAST_ASSIGNED_FIELD_ID:
+        return readAssertLastAssignedFieldId(jsonNode);
+      case ASSERT_LAST_ASSIGNED_PARTITION_ID:
+        return readAssertLastAssignedPartitionId(jsonNode);
+      case ASSERT_CURRENT_SCHEMA_ID:
+        return readAssertCurrentSchemaId(jsonNode);
+      case ASSERT_DEFAULT_SPEC_ID:
+        return readAssertDefaultSpecId(jsonNode);
+      case ASSERT_DEFAULT_SORT_ORDER_ID:
+        return readAssertDefaultSortOrderId(jsonNode);
+      default:
+        throw new UnsupportedOperationException(
+            String.format("Unrecognized update requirement. Cannot convert to json: %s", type));
+    }
+  }
+
+  private static void writeAssertTableUUID(UpdateRequirement.AssertTableUUID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeStringField(UUID, requirement.uuid());
+  }
+
+  private static void writeAssertRefSnapshotId(UpdateRequirement.AssertRefSnapshotID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeStringField(NAME, requirement.refName());
+    gen.writeNumberField(SNAPSHOT_ID, requirement.snapshotId());

Review Comment:
   You're right. `writeNumberField` doesn't handle null properly.
   
   Updated and added a 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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  // TODO - This is called `ref` in the spec.
+  //   https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1359-L1360
+  private static final String NAME = "name";

Review Comment:
   For reference, the spec fields are listed here. Notice that `ref` is one of the fields: https://github.com/apache/iceberg/blob/a19ac7835c8e251fd6a042e056f29ee63262adec/open-api/rest-catalog-open-api.yaml#L1325-L1374



-- 
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 #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  private static final String NAME = "name";
+  private static final String SNAPSHOT_ID = "snapshot-id";
+
+  // AssertLastAssignedFieldId
+  private static final String LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id";
+
+  // AssertCurrentSchemaID
+  private static final String SCHEMA_ID = "current-schema-id";
+
+  // AssertLastAssignedPartitionId
+  private static final String LAST_ASSIGNED_PARTITION_ID = "last-assigned-partition-id";
+
+  // AssertDefaultSpecID
+  private static final String SPEC_ID = "default-spec-id";
+
+  // AssertDefaultSortOrderID
+  private static final String SORT_ORDER_ID = "default-write-order-id";
+
+  private static final Map<Class<? extends UpdateTableRequest.UpdateRequirement>, String> TYPES = ImmutableMap
+      .<Class<? extends UpdateTableRequest.UpdateRequirement>, String>builder()
+      .put(UpdateRequirement.AssertTableUUID.class, ASSERT_TABLE_UUID)
+      .put(UpdateRequirement.AssertTableDoesNotExist.class, ASSERT_TABLE_DOES_NOT_EXIST)
+      .put(UpdateRequirement.AssertRefSnapshotID.class, ASSERT_REF_SNAPSHOT_ID)
+      .put(UpdateRequirement.AssertLastAssignedFieldId.class, ASSERT_LAST_ASSIGNED_FIELD_ID)
+      .put(UpdateRequirement.AssertCurrentSchemaID.class, ASSERT_CURRENT_SCHEMA_ID)
+      .put(UpdateRequirement.AssertLastAssignedPartitionId.class, ASSERT_LAST_ASSIGNED_PARTITION_ID)
+      .put(UpdateRequirement.AssertDefaultSpecID.class, ASSERT_DEFAULT_SPEC_ID)
+      .put(UpdateRequirement.AssertDefaultSortOrderID.class, ASSERT_DEFAULT_SORT_ORDER_ID)
+      .build();
+
+  public static String toJson(UpdateRequirement updateRequirement) {
+    return toJson(updateRequirement, false);
+  }
+
+  public static String toJson(UpdateRequirement updateRequirement, boolean pretty) {
+    try {
+      StringWriter writer = new StringWriter();
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      if (pretty) {
+        generator.useDefaultPrettyPrinter();
+      }
+      toJson(updateRequirement, generator);
+      generator.flush();
+      return writer.toString();
+    } catch (IOException e) {
+      throw new UncheckedIOException(
+          String.format("Failed to write update requirement json for: %s", updateRequirement), e);
+    }
+  }
+
+  public static void toJson(UpdateRequirement updateRequirement, JsonGenerator generator) throws IOException {
+    String requirementType = TYPES.get(updateRequirement.getClass());
+
+    generator.writeStartObject();
+    generator.writeStringField(TYPE, requirementType);
+
+    switch (requirementType) {
+      case ASSERT_TABLE_DOES_NOT_EXIST:
+        // No fields beyond the requirement itself
+        break;
+      case ASSERT_TABLE_UUID:
+        writeAssertTableUUID((UpdateRequirement.AssertTableUUID) updateRequirement, generator);
+        break;
+      case ASSERT_REF_SNAPSHOT_ID:
+        writeAssertRefSnapshotId((UpdateRequirement.AssertRefSnapshotID) updateRequirement, generator);
+        break;
+      case ASSERT_LAST_ASSIGNED_FIELD_ID:
+        writeAssertLastAssignedFieldId((UpdateRequirement.AssertLastAssignedFieldId) updateRequirement, generator);
+        break;
+      case ASSERT_LAST_ASSIGNED_PARTITION_ID:
+        writeAssertLastAssignedPartitionId(
+            (UpdateRequirement.AssertLastAssignedPartitionId) updateRequirement, generator);
+        break;
+      case ASSERT_CURRENT_SCHEMA_ID:
+        writeAssertCurrentSchemaId((UpdateRequirement.AssertCurrentSchemaID) updateRequirement, generator);
+        break;
+      case ASSERT_DEFAULT_SPEC_ID:
+        writeAssertDefaultSpecId((UpdateRequirement.AssertDefaultSpecID) updateRequirement, generator);
+        break;
+      case ASSERT_DEFAULT_SORT_ORDER_ID:
+        writeAssertDefaultSortOrderId((UpdateRequirement.AssertDefaultSortOrderID) updateRequirement, generator);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot convert update requirement to json. Unrecognized type: %s", requirementType));
+    }
+
+    generator.writeEndObject();
+  }
+
+  /**
+   * Read MetadataUpdate from a JSON string.
+   *
+   * @param json a JSON string of a MetadataUpdate
+   * @return a MetadataUpdate object
+   */
+  public static UpdateRequirement fromJson(String json) {
+    try {
+      return fromJson(JsonUtil.mapper().readValue(json, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to read JSON string: " + json, e);
+    }
+  }
+
+  public static UpdateRequirement fromJson(JsonNode jsonNode) {
+    Preconditions.checkArgument(jsonNode != null && jsonNode.isObject(),
+        "Cannot parse update requirement from non-object value: %s", jsonNode);
+    Preconditions.checkArgument(jsonNode.hasNonNull(TYPE), "Cannot parse update requirement. Missing field: type");
+    String type = JsonUtil.getString(TYPE, jsonNode).toLowerCase(Locale.ROOT);
+
+    switch (type) {
+      case ASSERT_TABLE_DOES_NOT_EXIST:
+        return readAssertTableDoesNotExist(jsonNode);
+      case ASSERT_TABLE_UUID:
+        return readAssertTableUUID(jsonNode);
+      case ASSERT_REF_SNAPSHOT_ID:
+        return readAssertRefSnapshotId(jsonNode);
+      case ASSERT_LAST_ASSIGNED_FIELD_ID:
+        return readAssertLastAssignedFieldId(jsonNode);
+      case ASSERT_LAST_ASSIGNED_PARTITION_ID:
+        return readAssertLastAssignedPartitionId(jsonNode);
+      case ASSERT_CURRENT_SCHEMA_ID:
+        return readAssertCurrentSchemaId(jsonNode);
+      case ASSERT_DEFAULT_SPEC_ID:
+        return readAssertDefaultSpecId(jsonNode);
+      case ASSERT_DEFAULT_SORT_ORDER_ID:
+        return readAssertDefaultSortOrderId(jsonNode);
+      default:
+        throw new UnsupportedOperationException(
+            String.format("Unrecognized update requirement. Cannot convert to json: %s", type));
+    }
+  }
+
+  private static void writeAssertTableUUID(UpdateRequirement.AssertTableUUID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeStringField(UUID, requirement.uuid());
+  }
+
+  private static void writeAssertRefSnapshotId(UpdateRequirement.AssertRefSnapshotID requirement, JsonGenerator gen)
+      throws IOException {
+    gen.writeStringField(NAME, requirement.refName());
+    gen.writeNumberField(SNAPSHOT_ID, requirement.snapshotId());

Review Comment:
   I think this should test whether the required snapshot ID is null and explicitly write `null` if it is.



-- 
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 #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateRequirementParser.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.rest.requests;
+
+import java.util.List;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestUpdateRequirementParser {
+
+  @Test
+  public void testUpdateRequirementWithoutRequirementTypeCannotParse() {
+    List<String> invalidJson = ImmutableList.of(
+        "{\"type\":null,\"uuid\":\"2cc52516-5e73-41f2-b139-545d41a4e151\"}",
+        "{\"uuid\":\"2cc52516-5e73-41f2-b139-545d41a4e151\"}"
+    );
+
+    for (String json : invalidJson) {
+      AssertHelpers.assertThrows(
+          "UpdateRequirement without a recognized requirement type should fail to deserialize",
+          IllegalArgumentException.class,
+          "Cannot parse update requirement. Missing field: type",
+          () -> UpdateRequirementParser.fromJson(json));
+    }
+  }
+
+  @Test
+  public void testAssertUUIDFromJson() {
+    String requirementType = UpdateRequirementParser.ASSERT_TABLE_UUID;
+    String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151";
+    String json = String.format("{\"type\":\"assert-table-uuid\",\"uuid\":\"%s\"}", uuid);
+    UpdateRequirement expected = new UpdateRequirement.AssertTableUUID(uuid);
+    assertEquals(requirementType, expected, UpdateRequirementParser.fromJson(json));
+  }
+
+  @Test
+  public void testAssertUUIDToJson() {
+    String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151";
+    String expected = String.format("{\"type\":\"assert-table-uuid\",\"uuid\":\"%s\"}", uuid);
+    UpdateRequirement actual = new UpdateRequirement.AssertTableUUID(uuid);
+    Assert.assertEquals("AssertTableUUID should convert to the correct JSON value",
+        expected, UpdateRequirementParser.toJson(actual));
+  }
+
+  @Test
+  public void testAssertTableDoesNotExistFromJson() {
+    String requirementType = UpdateRequirementParser.ASSERT_TABLE_DOES_NOT_EXIST;
+    String json = "{\"type\":\"assert-create\"}";
+    UpdateRequirement expected = new UpdateRequirement.AssertTableDoesNotExist();
+    assertEquals(requirementType, expected, UpdateRequirementParser.fromJson(json));
+  }
+
+  @Test
+  public void testAssertTableDoesNotExistToJson() {
+    String expected  = "{\"type\":\"assert-create\"}";
+    UpdateRequirement actual = new UpdateRequirement.AssertTableDoesNotExist();
+    Assert.assertEquals("AssertTableDoesNotExist should convert to the correct JSON value",
+        expected, UpdateRequirementParser.toJson(actual));
+  }
+
+  @Test
+  public void testAssertRefSnapshotIdToJson() {
+    String requirementType = UpdateRequirementParser.ASSERT_REF_SNAPSHOT_ID;
+    String name = "snapshot-name";
+    Long snapshotId = 1L;
+    String json = String.format("{\"type\":\"%s\",\"name\":\"%s\",\"snapshot-id\":%d}",
+        requirementType, name, snapshotId);
+    UpdateRequirement expected = new UpdateRequirement.AssertRefSnapshotID(name, snapshotId);
+    assertEquals(requirementType, expected, UpdateRequirementParser.fromJson(json));
+  }
+
+  @Test
+  public void testAssertRefSnapshotIdFromJson() {

Review Comment:
   This needs to validate that `snapshot-id` serializes correctly when it 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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateRequirementParser.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.rest.requests;
+
+import java.util.List;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestUpdateRequirementParser {
+
+  @Test
+  public void testUpdateRequirementWithoutRequirementTypeCannotParse() {
+    List<String> invalidJson = ImmutableList.of(
+        "{\"type\":null,\"uuid\":\"2cc52516-5e73-41f2-b139-545d41a4e151\"}",
+        "{\"uuid\":\"2cc52516-5e73-41f2-b139-545d41a4e151\"}"
+    );
+
+    for (String json : invalidJson) {
+      AssertHelpers.assertThrows(
+          "UpdateRequirement without a recognized requirement type should fail to deserialize",
+          IllegalArgumentException.class,
+          "Cannot parse update requirement. Missing field: type",
+          () -> UpdateRequirementParser.fromJson(json));
+    }
+  }
+
+  @Test
+  public void testAssertUUIDFromJson() {
+    String requirementType = UpdateRequirementParser.ASSERT_TABLE_UUID;
+    String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151";
+    String json = String.format("{\"type\":\"assert-table-uuid\",\"uuid\":\"%s\"}", uuid);
+    UpdateRequirement expected = new UpdateRequirement.AssertTableUUID(uuid);
+    assertEquals(requirementType, expected, UpdateRequirementParser.fromJson(json));
+  }
+
+  @Test
+  public void testAssertUUIDToJson() {
+    String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151";
+    String expected = String.format("{\"type\":\"assert-table-uuid\",\"uuid\":\"%s\"}", uuid);
+    UpdateRequirement actual = new UpdateRequirement.AssertTableUUID(uuid);
+    Assert.assertEquals("AssertTableUUID should convert to the correct JSON value",
+        expected, UpdateRequirementParser.toJson(actual));
+  }
+
+  @Test
+  public void testAssertTableDoesNotExistFromJson() {
+    String requirementType = UpdateRequirementParser.ASSERT_TABLE_DOES_NOT_EXIST;
+    String json = "{\"type\":\"assert-create\"}";
+    UpdateRequirement expected = new UpdateRequirement.AssertTableDoesNotExist();
+    assertEquals(requirementType, expected, UpdateRequirementParser.fromJson(json));
+  }
+
+  @Test
+  public void testAssertTableDoesNotExistToJson() {
+    String expected  = "{\"type\":\"assert-create\"}";
+    UpdateRequirement actual = new UpdateRequirement.AssertTableDoesNotExist();
+    Assert.assertEquals("AssertTableDoesNotExist should convert to the correct JSON value",
+        expected, UpdateRequirementParser.toJson(actual));
+  }
+
+  @Test
+  public void testAssertRefSnapshotIdToJson() {
+    String requirementType = UpdateRequirementParser.ASSERT_REF_SNAPSHOT_ID;
+    String name = "snapshot-name";
+    Long snapshotId = 1L;
+    String json = String.format("{\"type\":\"%s\",\"name\":\"%s\",\"snapshot-id\":%d}",
+        requirementType, name, snapshotId);
+    UpdateRequirement expected = new UpdateRequirement.AssertRefSnapshotID(name, snapshotId);
+    assertEquals(requirementType, expected, UpdateRequirementParser.fromJson(json));
+  }
+
+  @Test
+  public void testAssertRefSnapshotIdFromJson() {

Review Comment:
   Oh and added a test for explicit 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 pull request #4693: Core - Add Parser for UpdateRequirement

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

   Thanks, @kbendick!


-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1233,9 +1233,9 @@ components:
         - $ref: '#/components/schemas/BaseUpdate'
         - type: object
           required:
-            - order-id
+            - sort-order-id
           properties:
-            order-id:
+            sort-order-id:

Review Comment:
   I can update the corresponding fields for `MetadataUpdate` in another PR.
   
   I'm opening a PR for the last `MetadataUpdate` parsers today.



-- 
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] kbendick commented on a diff in pull request #4693: Core - Add Parser for UpdateRequirement

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
+import org.apache.iceberg.util.JsonUtil;
+
+public class UpdateRequirementParser {
+
+  private UpdateRequirementParser() {
+  }
+
+  private static final String TYPE = "type";
+
+  // assertion types
+  static final String ASSERT_TABLE_UUID = "assert-table-uuid";
+  static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create";
+  static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id";
+  static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id";
+  static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id";
+  static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id";
+  static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id";
+  static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-write-order-id";
+
+  // AssertTableUUID
+  private static final String UUID = "uuid";
+
+  // AssertRefSnapshotID
+  // TODO - This is called `ref` in the spec.
+  //   https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1359-L1360
+  private static final String NAME = "name";

Review Comment:
   If we made this `ref`, we'd just need to rename the current variable `ref` in the class's validate method to `baseRefType` or something: https://github.com/apache/iceberg/blob/a19ac7835c8e251fd6a042e056f29ee63262adec/core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java#L279-L281



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