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/06/23 21:34:58 UTC

[GitHub] [iceberg] kbendick opened a new pull request, #5118: CORE - Load Table Response Test

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

   This PR adds serde tests for `LoadTableResponse` for both V1 and V2.
   
   Presently, this PR makes a number of things `public` which were previously package-private, and that I would like to keep package-private.
   
   So I'm opening this as draft to get feedback on ways to possibly do that and which ones do we care very much about opening up further in terms of public vs package-private (e.g. I think the BaseSnapshot constructors should remain package-private, but some of the named `Snapshot` constants are probably ok to make public).
   
   We can possibly access the protected constructors via test utilities that expose the necessary functions in code that lives in `test` only (like we did for testing some parts of `CachingCatalog`). Or possibly some reflection based accession would be sufficient.


-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    // Add location
+    TableMetadata metadataWithLocation = TableMetadata
+        .buildFrom(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadataWithLocation)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerDe() throws JsonProcessingException {
+
+    // Build table metadata so that its uuid and last updated millis can be reused.
+    TableMetadata metaV1 = TableMetadata.buildFrom(
+        TableMetadata.newTableMetadata(
+            TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+
+    LoadTableResponse respV1 = LoadTableResponse.builder()
+        .withTableMetadata(metaV1)
+        .addAllConfig(CONFIG)
+        .build();
+
+    String fullJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":1," +
+        "    \"table-uuid\":\"" + metaV1.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-updated-ms\":" + metaV1.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "       ]" +
+        "    }," +
+        "  \"current-schema-id\":0," +
+        "  \"schemas\":[" +
+        "    {\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "      {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "      {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "      {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"partition-spec\":[]," +
+        "  \"default-spec-id\":0," +
+        "  \"partition-specs\":[{\"spec-id\":0,\"fields\":[]}]," +
+        "  \"last-partition-id\":999,\"default-sort-order-id\":1," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":1,\"fields\":[" +
+        "        {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "        {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"properties\":{\"owner\":\"hank\"}," +
+        "  \"current-snapshot-id\":-1," +
+        "  \"refs\":{},\"snapshots\":[],\"snapshot-log\":[],\"metadata-log\":[]},\"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullJson.replaceAll("\\s", ""), respV1);
+
+    TableMetadata metaV2 = buildV2TableMetadataWithTagsAndSnapshotHistory();
+    LoadTableResponse respV2 = LoadTableResponse.builder()
+        .withTableMetadata(metaV2)
+        .addAllConfig(CONFIG)
+        .build();
+    Snapshot currentSnapshot = metaV2.snapshots().get(1);
+    Snapshot previousSnapshot = metaV2.snapshots().get(0);
+    String fullExampleInstanceJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":2," +
+        "    \"table-uuid\":\"" + metaV2.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-sequence-number\":34," +
+        "    \"last-updated-ms\":" + metaV2.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"current-schema-id\":7," +
+        "    \"schemas\":[{" +
+        "      \"type\":\"struct\",\"schema-id\":7,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]},{" +
+        "      \"type\":\"struct\",\"schema-id\":6,\"fields\":[" +
+        "        {\"id\":10,\"name\":\"x\",\"required\":true,\"type\":\"string\"}]" +
+        "    }]," +
+        "  \"default-spec-id\":5," +
+        "  \"partition-specs\":[{\"spec-id\":5,\"fields\":[]}]," +
+        "  \"last-partition-id\":999," +
+        "  \"default-sort-order-id\":3," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":3,\"fields\":[" +
+        "      {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "      {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}]}]," +
+        "  \"properties\":{\"property\":\"value\"}," +
+        "  \"current-snapshot-id\":" + metaV2.currentSnapshot().snapshotId() + "," +
+        "  \"refs\":{" +
+        "    \"main\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("main").snapshotId() + "," +
+        "      \"type\":\"branch\"}," +
+        "    \"previous\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("previous").snapshotId() + "," +
+        "      \"type\":\"tag\"}," +
+        "    \"test\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("test").snapshotId() + "," +
+        "     \"type\":\"branch\"}" +
+        "  }," +
+        "  \"snapshots\":[{" +
+        "    \"snapshot-id\":" + metaV2.snapshots().get(0).snapshotId() + "," +
+        "    \"timestamp-ms\":" + metaV2.snapshots().get(0).timestampMillis() + "," +
+        "    \"manifests\":[" +
+        "      \"/tmp/manifest.1.avro\"" +
+        "    ]},{" +
+        "    \"snapshot-id\":" + metaV2.snapshots().get(1).snapshotId() + "," +
+        "    \"parent-snapshot-id\":" + metaV2.snapshots().get(1).parentId() + "," +
+        "    \"timestamp-ms\":" + metaV2.snapshots().get(1).timestampMillis() + "," +
+        "    \"manifests\":[" +
+        "      \"/tmp/manifest.2.avro\"" +
+        "    ],\"schema-id\":7" +
+        "  }]," +
+        "  \"snapshot-log\":[{" +
+        "      \"timestamp-ms\":" + previousSnapshot.timestampMillis() + "," +
+        "      \"snapshot-id\":" + previousSnapshot.snapshotId() +
+        "    },{" +
+        "      \"timestamp-ms\":" + currentSnapshot.timestampMillis() + "," +
+        "      \"snapshot-id\":" + currentSnapshot.snapshotId() +
+        "  }]," +
+        "  \"metadata-log\":[]}," +
+        "  \"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullExampleInstanceJson.replaceAll("\\s", ""), respV2);
+
+  }
+
+
+  private TableMetadata buildV2TableMetadataWithTagsAndSnapshotHistory() {
+    Snapshot previousSnapshot = new BaseSnapshot(
+        ops.io(), PREVIOUS_SNAPSHOT_ID, null, PREVIOUS_SNAPSHOT_ID, null, null, null, ImmutableList.of(
+            new GenericManifestFile(localInput("file:/tmp/manifest.1.avro"), SPEC_5.specId())));
+    long currentSnapshotId = System.currentTimeMillis();
+    Snapshot currentSnapshot = new BaseSnapshot(
+        ops.io(), currentSnapshotId, PREVIOUS_SNAPSHOT_ID, currentSnapshotId, null, null, 7, ImmutableList.of(
+            new GenericManifestFile(localInput("file:/tmp/manifest.2.avro"), SPEC_5.specId())));
+
+    List<HistoryEntry> snapshotLog = ImmutableList.<HistoryEntry>builder()
+        .add(new TableMetadata.SnapshotLogEntry(previousSnapshot.timestampMillis(), previousSnapshot.snapshotId()))
+        .add(new TableMetadata.SnapshotLogEntry(currentSnapshot.timestampMillis(), currentSnapshot.snapshotId()))
+        .build();
+
+    Schema schema = new Schema(6,
+        Types.NestedField.required(10, "x", Types.StringType.get()));
+
+    Map<String, SnapshotRef> refs = ImmutableMap.of(
+        "main", SnapshotRef.branchBuilder(currentSnapshotId).build(),
+        "previous", SnapshotRef.tagBuilder(PREVIOUS_SNAPSHOT_ID).build(),
+        "test", SnapshotRef.branchBuilder(PREVIOUS_SNAPSHOT_ID).build()
+    );
+
+    // TODO - Use builder to get here instead - possibly even just writing out to a .json file and reading in from
+    //  there.

Review Comment:
   I agree with this. You can use `TableMetadata.buildFrom` and `TableMetadata.newTableMetadata` instead of making the constructor public.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    // Add location
+    TableMetadata metadataWithLocation = TableMetadata
+        .buildFrom(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadataWithLocation)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerDe() throws JsonProcessingException {
+
+    // Build table metadata so that its uuid and last updated millis can be reused.
+    TableMetadata metaV1 = TableMetadata.buildFrom(
+        TableMetadata.newTableMetadata(
+            TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();

Review Comment:
   I don't think that we need to check v1 or v2. Just one round-trip test should be fine.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTObjectMapper.java:
##########
@@ -39,6 +40,7 @@ static ObjectMapper mapper() {
         if (!isInitialized) {
           MAPPER.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY);
           MAPPER.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+          MAPPER.setPropertyNamingStrategy(new PropertyNamingStrategy.KebabCaseStrategy());

Review Comment:
   This will come from https://github.com/apache/iceberg/pull/5135



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    // Add location
+    TableMetadata metadataWithLocation = TableMetadata
+        .buildFrom(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadataWithLocation)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {

Review Comment:
   It's not quite required by the builder though. I'll update the builder as well 👍 



##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";

Review Comment:
   Removed



##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    // Add location
+    TableMetadata metadataWithLocation = TableMetadata
+        .buildFrom(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadataWithLocation)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerDe() throws JsonProcessingException {
+
+    // Build table metadata so that its uuid and last updated millis can be reused.
+    TableMetadata metaV1 = TableMetadata.buildFrom(
+        TableMetadata.newTableMetadata(
+            TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+
+    LoadTableResponse respV1 = LoadTableResponse.builder()
+        .withTableMetadata(metaV1)
+        .addAllConfig(CONFIG)
+        .build();
+
+    String fullJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":1," +
+        "    \"table-uuid\":\"" + metaV1.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-updated-ms\":" + metaV1.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "       ]" +
+        "    }," +
+        "  \"current-schema-id\":0," +
+        "  \"schemas\":[" +
+        "    {\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "      {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "      {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "      {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"partition-spec\":[]," +
+        "  \"default-spec-id\":0," +
+        "  \"partition-specs\":[{\"spec-id\":0,\"fields\":[]}]," +
+        "  \"last-partition-id\":999,\"default-sort-order-id\":1," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":1,\"fields\":[" +
+        "        {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "        {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"properties\":{\"owner\":\"hank\"}," +
+        "  \"current-snapshot-id\":-1," +
+        "  \"refs\":{},\"snapshots\":[],\"snapshot-log\":[],\"metadata-log\":[]},\"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullJson.replaceAll("\\s", ""), respV1);
+
+    TableMetadata metaV2 = buildV2TableMetadataWithTagsAndSnapshotHistory();
+    LoadTableResponse respV2 = LoadTableResponse.builder()
+        .withTableMetadata(metaV2)
+        .addAllConfig(CONFIG)
+        .build();
+    Snapshot currentSnapshot = metaV2.snapshots().get(1);
+    Snapshot previousSnapshot = metaV2.snapshots().get(0);
+    String fullExampleInstanceJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":2," +
+        "    \"table-uuid\":\"" + metaV2.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-sequence-number\":34," +
+        "    \"last-updated-ms\":" + metaV2.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"current-schema-id\":7," +
+        "    \"schemas\":[{" +
+        "      \"type\":\"struct\",\"schema-id\":7,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]},{" +
+        "      \"type\":\"struct\",\"schema-id\":6,\"fields\":[" +
+        "        {\"id\":10,\"name\":\"x\",\"required\":true,\"type\":\"string\"}]" +
+        "    }]," +
+        "  \"default-spec-id\":5," +
+        "  \"partition-specs\":[{\"spec-id\":5,\"fields\":[]}]," +
+        "  \"last-partition-id\":999," +
+        "  \"default-sort-order-id\":3," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":3,\"fields\":[" +
+        "      {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "      {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}]}]," +
+        "  \"properties\":{\"property\":\"value\"}," +
+        "  \"current-snapshot-id\":" + metaV2.currentSnapshot().snapshotId() + "," +
+        "  \"refs\":{" +
+        "    \"main\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("main").snapshotId() + "," +
+        "      \"type\":\"branch\"}," +
+        "    \"previous\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("previous").snapshotId() + "," +
+        "      \"type\":\"tag\"}," +
+        "    \"test\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("test").snapshotId() + "," +
+        "     \"type\":\"branch\"}" +
+        "  }," +
+        "  \"snapshots\":[{" +
+        "    \"snapshot-id\":" + metaV2.snapshots().get(0).snapshotId() + "," +
+        "    \"timestamp-ms\":" + metaV2.snapshots().get(0).timestampMillis() + "," +
+        "    \"manifests\":[" +
+        "      \"/tmp/manifest.1.avro\"" +
+        "    ]},{" +
+        "    \"snapshot-id\":" + metaV2.snapshots().get(1).snapshotId() + "," +
+        "    \"parent-snapshot-id\":" + metaV2.snapshots().get(1).parentId() + "," +
+        "    \"timestamp-ms\":" + metaV2.snapshots().get(1).timestampMillis() + "," +
+        "    \"manifests\":[" +
+        "      \"/tmp/manifest.2.avro\"" +
+        "    ],\"schema-id\":7" +
+        "  }]," +
+        "  \"snapshot-log\":[{" +
+        "      \"timestamp-ms\":" + previousSnapshot.timestampMillis() + "," +
+        "      \"snapshot-id\":" + previousSnapshot.snapshotId() +
+        "    },{" +
+        "      \"timestamp-ms\":" + currentSnapshot.timestampMillis() + "," +
+        "      \"snapshot-id\":" + currentSnapshot.snapshotId() +
+        "  }]," +
+        "  \"metadata-log\":[]}," +
+        "  \"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullExampleInstanceJson.replaceAll("\\s", ""), respV2);
+
+  }
+
+
+  private TableMetadata buildV2TableMetadataWithTagsAndSnapshotHistory() {
+    Snapshot previousSnapshot = new BaseSnapshot(
+        ops.io(), PREVIOUS_SNAPSHOT_ID, null, PREVIOUS_SNAPSHOT_ID, null, null, null, ImmutableList.of(

Review Comment:
   They aren't. My bad. I'll remove them.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema SCHEMA_7 = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(SCHEMA_7).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(SCHEMA_7)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadata-location", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata =
+        TableMetadata
+            .buildFrom(
+                TableMetadata.newTableMetadata(SCHEMA_7, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+            .withMetadataLocation(TEST_METADATA_LOCATION)
+            .build();
+
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadata)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerde() throws Exception {
+    // Default fields are missing in this JSON
+    String tableMetadataV1Json = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(null, TEST_METADATA_LOCATION, tableMetadataV1Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV1));
+    LoadTableResponse resp1 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV1)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json1, resp1);
+
+    String tableMetadataV2Json = readTableMetadataInputFile("TableMetadataV2Valid.json");
+    TableMetadata metadataV2 = TableMetadataParser.fromJson(null, TEST_METADATA_LOCATION, tableMetadataV2Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json2 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV2));
+    LoadTableResponse resp2 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV2)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json2, resp2);
+  }
+
+  @Test
+  public void testCanDeserializeWithoutDefaultValues() throws Exception {
+    String metadataJson = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, metadataJson);
+
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(null, TEST_METADATA_LOCATION, metadataJson);
+    LoadTableResponse resp1 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV1)
+        .addAllConfig(CONFIG)
+        .build();
+    assertEquals(deserialize(json1), resp1);

Review Comment:
   > From the test name, I think what you intended to test is that all values that can be omitted will be deserialized to defaults. Should this omit "config" from the JSON string and validate that the response is equal to the one created without calling addAllConfig?
   
   Yeah that's what this is testing. It's a little unclear from the way it's written admittedly.
   
   I'll try removing `config` and then ensuring we get the default back.
   
   And update the names.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -53,6 +53,8 @@ private LoadTableResponse(String metadataLocation, TableMetadata metadata, Map<S
 
   @Override
   public void validate() {
+    Preconditions.checkNotNull(metadata, "Invalid metadata: null");
+    Preconditions.checkNotNull(metadataLocation, "Invalid metadata location: null");

Review Comment:
   Since `validate` is for ensuring that all requirements are met from results received externally, it makes sense to have the `metadata-location` check here.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/LocalTableOperations.java:
##########
@@ -27,13 +27,13 @@
 import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.junit.rules.TemporaryFolder;
 
-class LocalTableOperations implements TableOperations {
+public class LocalTableOperations implements TableOperations {
   private final TemporaryFolder temp;
   private final FileIO io;
 
   private final Map<String, String> createdMetadataFilePaths = Maps.newHashMap();
 
-  LocalTableOperations(TemporaryFolder temp) {
+  public LocalTableOperations(TemporaryFolder temp) {

Review Comment:
   Is this still needed?



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -28,6 +28,7 @@
 import org.apache.iceberg.BaseMetadataTable;
 import org.apache.iceberg.BaseTable;
 import org.apache.iceberg.BaseTransaction;
+import org.apache.iceberg.HasTableOperations;

Review Comment:
   Why are those changes more correct?



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/RequestResponseTestBase.java:
##########
@@ -104,7 +103,7 @@ protected void assertRoundTripSerializesEquallyFrom(String json, T expected) thr
     T actual = deserialize(json);
     assertEquals(actual, expected);
 
-    // Check that the deserialized value serializes back into the original JSON
-    Assertions.assertThat(serialize(expected)).isEqualTo(json);
+    Assert.assertEquals("The serialized version of the Java record should match the original JSON",
+        serialize(expected), json);

Review Comment:
   I updated this as the comparison of two JSON strings can be quite cumbersome to diff using `Assertions` (it just shows the whole string as they're both one single line).
   
   Using `Assert.assertEquals`, the error message is much more clear about exactly what part of the string is different as it truncates the rest of the string and makes it clear what characters are different.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/RequestResponseTestBase.java:
##########
@@ -104,7 +103,7 @@ protected void assertRoundTripSerializesEquallyFrom(String json, T expected) thr
     T actual = deserialize(json);
     assertEquals(actual, expected);
 
-    // Check that the deserialized value serializes back into the original JSON
-    Assertions.assertThat(serialize(expected)).isEqualTo(json);
+    Assert.assertEquals("The serialized version of the Java record should match the original JSON",
+        serialize(expected), json);

Review Comment:
   Nope. This is just a beneficial upgrade. I'll revert it now.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema SCHEMA_7 = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(SCHEMA_7).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(SCHEMA_7)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadata-location", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata =
+        TableMetadata
+            .buildFrom(
+                TableMetadata.newTableMetadata(SCHEMA_7, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+            .withMetadataLocation(TEST_METADATA_LOCATION)
+            .build();
+
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadata)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerde() throws Exception {
+    // Default fields are missing in this JSON
+    String tableMetadataV1Json = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(null, TEST_METADATA_LOCATION, tableMetadataV1Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV1));
+    LoadTableResponse resp1 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV1)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json1, resp1);
+
+    String tableMetadataV2Json = readTableMetadataInputFile("TableMetadataV2Valid.json");
+    TableMetadata metadataV2 = TableMetadataParser.fromJson(null, TEST_METADATA_LOCATION, tableMetadataV2Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json2 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV2));
+    LoadTableResponse resp2 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV2)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json2, resp2);
+  }
+
+  @Test
+  public void testCanDeserializeWithoutDefaultValues() throws Exception {
+    String metadataJson = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, metadataJson);
+
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(null, TEST_METADATA_LOCATION, metadataJson);
+    LoadTableResponse resp1 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV1)
+        .addAllConfig(CONFIG)
+        .build();
+    assertEquals(deserialize(json1), resp1);

Review Comment:
   In the version you were looking at, this was testing that the JSON parses into the class as expected, even if default values are not present.
   
   Because default values are not present in the external JSON, I used those to represent missing values (which was the source of confusion).
   
   For the `round trip` tests, it tests that the JSON deserializes into the object _as well as that the JSON matches exactly as expected the serialiazed output of the object_
   
   That's the `round trip` in the `assertRoundTripSerialzesEquallyFrom`: https://github.com/apache/iceberg/blob/22298818cb7eba13294eda3b2dd696b773e0f170/core/src/test/java/org/apache/iceberg/rest/RequestResponseTestBase.java#L102-L109



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -89,8 +90,16 @@ private Builder() {
     }
 
     public Builder withTableMetadata(TableMetadata tableMetadata) {
-      this.metadataLocation = tableMetadata.metadataFileLocation();
       this.metadata = tableMetadata;
+      if (tableMetadata.metadataFileLocation() != null) {
+        this.metadataLocation = tableMetadata.metadataFileLocation();
+      }
+
+      return this;
+    }
+
+    public Builder withMetadataLocation(String metaLocation) {

Review Comment:
   Why is changing this builder necessary?



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema SCHEMA_7 = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(SCHEMA_7).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(SCHEMA_7)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadata-location", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata =
+        TableMetadata
+            .buildFrom(
+                TableMetadata.newTableMetadata(SCHEMA_7, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+            .withMetadataLocation(TEST_METADATA_LOCATION)
+            .build();
+
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadata)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerde() throws Exception {

Review Comment:
   There's no need for this to put 2 different test cases in the same method. Can you test v1 and v2 separately?



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -28,6 +28,7 @@
 import org.apache.iceberg.BaseMetadataTable;
 import org.apache.iceberg.BaseTable;
 import org.apache.iceberg.BaseTransaction;
+import org.apache.iceberg.HasTableOperations;

Review Comment:
   They've been reverted so once the tests pass, this is the minimum change set for these tests.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -225,6 +225,16 @@ public static LoadTableResponse loadTable(Catalog catalog, TableIdentifier ident
     Table table = catalog.loadTable(ident);
 
     if (table instanceof BaseTable) {
+      // This gives
+      // TestRESTCatalog > testCompleteReplaceTransaction() FAILED
+      // java.lang.AssertionError: Table should have correct number of previous metadata locations expected:<1> but was
+      //   :<2>
+      // TableMetadata metadataWithoutLocation = ((BaseTable) table).operations().current();
+      // String metadataLocation = ((BaseTable) table).operations().metadataFileLocation("metadataV1.json");
+      // TableMetadata metadata = TableMetadata.buildFrom(metadataWithoutLocation)
+      //     .withMetadataLocation(metadataLocation)
+      //     .build();
+      // String metadataLocation = ((BaseTable) table).operations().current().previousFiles();
       return LoadTableResponse.builder()
           .withTableMetadata(((BaseTable) table).operations().current())

Review Comment:
   It actually does. There's some internal usage of `LoadTableResponse` where the `metadataLocation` isn't known (e.g. during `stageCreate`).
   
   Removing the requirement that the result of the builder have `metadataLocation` at all times fixes this.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -106,6 +113,8 @@ public Builder addAllConfig(Map<String, String> properties) {
 
     public LoadTableResponse build() {
       Preconditions.checkNotNull(metadata, "Invalid metadata: null");
+      // This is causing errors in TestRESTCatalog via CatalogHandlers
+      // Preconditions.checkNotNull(metadataLocation, "Invalid metadata location: null");

Review Comment:
   Here's an example call that fails, which _shouldnt_ have `metadataLocation` yet because the table hasn't been created (and the metadata location hasn't been determined yet): https://github.com/apache/iceberg/blob/e0ff3c6e9876ce04c890908e375c876d14240c3e/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java#L1450-L1458



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -106,6 +113,8 @@ public Builder addAllConfig(Map<String, String> properties) {
 
     public LoadTableResponse build() {
       Preconditions.checkNotNull(metadata, "Invalid metadata: null");
+      // This is causing errors in TestRESTCatalog via CatalogHandlers
+      Preconditions.checkNotNull(metadataLocation, "Invalid metadata location: null");

Review Comment:
   This is the check that is causing 16 test failures in `TestRESTCatalog` - which has been reduced to 13 by the changes in `CatalogHandlers`.
   
   Without this check, all tests in `TestRESTCatalog` pass. See notes here too 



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    // Add location
+    TableMetadata metadataWithLocation = TableMetadata
+        .buildFrom(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadataWithLocation)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerDe() throws JsonProcessingException {
+
+    // Build table metadata so that its uuid and last updated millis can be reused.
+    TableMetadata metaV1 = TableMetadata.buildFrom(
+        TableMetadata.newTableMetadata(
+            TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+
+    LoadTableResponse respV1 = LoadTableResponse.builder()
+        .withTableMetadata(metaV1)
+        .addAllConfig(CONFIG)
+        .build();
+
+    String fullJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":1," +
+        "    \"table-uuid\":\"" + metaV1.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-updated-ms\":" + metaV1.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "       ]" +
+        "    }," +
+        "  \"current-schema-id\":0," +
+        "  \"schemas\":[" +
+        "    {\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "      {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "      {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "      {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"partition-spec\":[]," +
+        "  \"default-spec-id\":0," +
+        "  \"partition-specs\":[{\"spec-id\":0,\"fields\":[]}]," +
+        "  \"last-partition-id\":999,\"default-sort-order-id\":1," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":1,\"fields\":[" +
+        "        {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "        {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"properties\":{\"owner\":\"hank\"}," +
+        "  \"current-snapshot-id\":-1," +
+        "  \"refs\":{},\"snapshots\":[],\"snapshot-log\":[],\"metadata-log\":[]},\"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullJson.replaceAll("\\s", ""), respV1);
+
+    TableMetadata metaV2 = buildV2TableMetadataWithTagsAndSnapshotHistory();
+    LoadTableResponse respV2 = LoadTableResponse.builder()
+        .withTableMetadata(metaV2)
+        .addAllConfig(CONFIG)
+        .build();
+    Snapshot currentSnapshot = metaV2.snapshots().get(1);
+    Snapshot previousSnapshot = metaV2.snapshots().get(0);
+    String fullExampleInstanceJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":2," +
+        "    \"table-uuid\":\"" + metaV2.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-sequence-number\":34," +
+        "    \"last-updated-ms\":" + metaV2.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"current-schema-id\":7," +
+        "    \"schemas\":[{" +
+        "      \"type\":\"struct\",\"schema-id\":7,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]},{" +
+        "      \"type\":\"struct\",\"schema-id\":6,\"fields\":[" +
+        "        {\"id\":10,\"name\":\"x\",\"required\":true,\"type\":\"string\"}]" +
+        "    }]," +
+        "  \"default-spec-id\":5," +
+        "  \"partition-specs\":[{\"spec-id\":5,\"fields\":[]}]," +
+        "  \"last-partition-id\":999," +
+        "  \"default-sort-order-id\":3," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":3,\"fields\":[" +
+        "      {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "      {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}]}]," +
+        "  \"properties\":{\"property\":\"value\"}," +
+        "  \"current-snapshot-id\":" + metaV2.currentSnapshot().snapshotId() + "," +
+        "  \"refs\":{" +
+        "    \"main\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("main").snapshotId() + "," +
+        "      \"type\":\"branch\"}," +
+        "    \"previous\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("previous").snapshotId() + "," +
+        "      \"type\":\"tag\"}," +
+        "    \"test\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("test").snapshotId() + "," +
+        "     \"type\":\"branch\"}" +
+        "  }," +
+        "  \"snapshots\":[{" +
+        "    \"snapshot-id\":" + metaV2.snapshots().get(0).snapshotId() + "," +
+        "    \"timestamp-ms\":" + metaV2.snapshots().get(0).timestampMillis() + "," +
+        "    \"manifests\":[" +
+        "      \"/tmp/manifest.1.avro\"" +
+        "    ]},{" +
+        "    \"snapshot-id\":" + metaV2.snapshots().get(1).snapshotId() + "," +
+        "    \"parent-snapshot-id\":" + metaV2.snapshots().get(1).parentId() + "," +
+        "    \"timestamp-ms\":" + metaV2.snapshots().get(1).timestampMillis() + "," +
+        "    \"manifests\":[" +
+        "      \"/tmp/manifest.2.avro\"" +
+        "    ],\"schema-id\":7" +
+        "  }]," +
+        "  \"snapshot-log\":[{" +
+        "      \"timestamp-ms\":" + previousSnapshot.timestampMillis() + "," +
+        "      \"snapshot-id\":" + previousSnapshot.snapshotId() +
+        "    },{" +
+        "      \"timestamp-ms\":" + currentSnapshot.timestampMillis() + "," +
+        "      \"snapshot-id\":" + currentSnapshot.snapshotId() +
+        "  }]," +
+        "  \"metadata-log\":[]}," +
+        "  \"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullExampleInstanceJson.replaceAll("\\s", ""), respV2);
+
+  }
+
+
+  private TableMetadata buildV2TableMetadataWithTagsAndSnapshotHistory() {
+    Snapshot previousSnapshot = new BaseSnapshot(
+        ops.io(), PREVIOUS_SNAPSHOT_ID, null, PREVIOUS_SNAPSHOT_ID, null, null, null, ImmutableList.of(
+            new GenericManifestFile(localInput("file:/tmp/manifest.1.avro"), SPEC_5.specId())));
+    long currentSnapshotId = System.currentTimeMillis();
+    Snapshot currentSnapshot = new BaseSnapshot(
+        ops.io(), currentSnapshotId, PREVIOUS_SNAPSHOT_ID, currentSnapshotId, null, null, 7, ImmutableList.of(
+            new GenericManifestFile(localInput("file:/tmp/manifest.2.avro"), SPEC_5.specId())));
+
+    List<HistoryEntry> snapshotLog = ImmutableList.<HistoryEntry>builder()
+        .add(new TableMetadata.SnapshotLogEntry(previousSnapshot.timestampMillis(), previousSnapshot.snapshotId()))
+        .add(new TableMetadata.SnapshotLogEntry(currentSnapshot.timestampMillis(), currentSnapshot.snapshotId()))
+        .build();
+
+    Schema schema = new Schema(6,
+        Types.NestedField.required(10, "x", Types.StringType.get()));
+
+    Map<String, SnapshotRef> refs = ImmutableMap.of(
+        "main", SnapshotRef.branchBuilder(currentSnapshotId).build(),
+        "previous", SnapshotRef.tagBuilder(PREVIOUS_SNAPSHOT_ID).build(),
+        "test", SnapshotRef.branchBuilder(PREVIOUS_SNAPSHOT_ID).build()
+    );
+
+    // TODO - Use builder to get here instead - possibly even just writing out to a .json file and reading in from
+    //  there.
+    TableMetadata expected = new TableMetadata(TEST_METADATA_LOCATION, 2, UUID.randomUUID().toString(),
+        TEST_TABLE_LOCATION,
+        SEQ_NO, System.currentTimeMillis(), 3,
+        7, ImmutableList.of(TEST_SCHEMA, schema),
+        5, ImmutableList.of(SPEC_5), SPEC_5.lastAssignedFieldId(),
+        3, ImmutableList.of(SORT_ORDER_3), ImmutableMap.of("property", "value"), currentSnapshotId,
+        Arrays.asList(previousSnapshot, currentSnapshot), snapshotLog, ImmutableList.of(), refs,
+        ImmutableList.of());
+
+    return expected;
+  }
+
+  @Override
+  public void assertEquals(LoadTableResponse actual, LoadTableResponse expected) {
+    // For printing JSON
+    // try {
+    //   Assert.assertEquals("Should have the same string representation",
+    //       mapper().writeValueAsString(expected), mapper().writeValueAsString(actual));
+    // } catch (JsonProcessingException e) {
+    //   LOG.error("\tRAUL Received exceptiom when comparing actual = {}, expected = {}", actual, expected, e);
+    //   LOG.error("\tRAUL exception is: ", e);
+    // }

Review Comment:
   Can you remove print statements or comments used for debugging?



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };

Review Comment:
   This doesn't match the spec. The spec uses `metadata-location`, not `metadataLocation`. It looks like this needs to be fixed 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 #5118: CORE - Load Table Response Test

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

   Since we’re only testing `LoadTableResponse` here and not `TableMetadata` specifically, it might be better to write the JSON out as external files and use `TableMetadataParser` after reading them in.
   
   We’d still be building the `LoadTableResponse` via the builder and testing that, but we wouldn’t need to create the `TableMetadata` instances programmaticallly.


-- 
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 #5118: CORE - Load Table Response Test

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

   There's currently one failure `TestCreateTableRequest#testReoundTripSerDe`, that will be fixed once either https://github.com/apache/iceberg/pull/5135 or https://github.com/apache/iceberg/pull/5133 is merged.
   
   ```
   TestCreateTableRequest > testRoundTripSerDe FAILED
       org.junit.ComparisonFailure: The serialized version of the Java record should match the original JSON expected:<...wner":"Hank"},"stage[-c]reate":false}> but was:<...wner":"Hank"},"stage[C]reate":false}>
           at org.junit.Assert.assertEquals(Assert.java:117)
           at org.apache.iceberg.rest.RequestResponseTestBase.assertRoundTripSerializesEquallyFrom(RequestResponseTestBase.java:90)
           at org.apache.iceberg.rest.requests.TestCreateTableRequest.testRoundTripSerDe(TestCreateTableRequest.java:83)
   ```


-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -94,6 +96,11 @@ public Builder withTableMetadata(TableMetadata tableMetadata) {
       return this;
     }
 
+    public Builder withMetadataLocation(String metaLocation) {
+      this.metadataLocation = metaLocation;
+      return this;
+    }

Review Comment:
   This isn't used presently, but I've found myself having to rebuild the `TableMetadata` in a few places to get `metadataLocation` to be non-null. 
   
   Given that the accessor in `LoadTableResponse` _also_ calls the builder and adds `metadataLocation` to it, this could be useful.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -89,8 +90,16 @@ private Builder() {
     }
 
     public Builder withTableMetadata(TableMetadata tableMetadata) {
-      this.metadataLocation = tableMetadata.metadataFileLocation();
       this.metadata = tableMetadata;
+      if (tableMetadata.metadataFileLocation() != null) {
+        this.metadataLocation = tableMetadata.metadataFileLocation();
+      }
+
+      return this;
+    }
+
+    public Builder withMetadataLocation(String metaLocation) {

Review Comment:
   I found myself having to create a new `TableMetadata` instance via the builder to pass in that had metadata location on it.
   
   Given that same builder pattern is used to add the metadata location to the `metadata` accessor in `LoadTableResult`, so I figured it would be convenient.
   
   Can remove 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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };

Review Comment:
   Updated.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -89,8 +90,11 @@ private Builder() {
     }
 
     public Builder withTableMetadata(TableMetadata tableMetadata) {
-      this.metadataLocation = tableMetadata.metadataFileLocation();
       this.metadata = tableMetadata;
+      if (tableMetadata.metadataFileLocation() != null) {
+        this.metadataLocation = tableMetadata.metadataFileLocation();
+      }

Review Comment:
   As we've removed the `withMetadataLocation` method from the builder, should I revert this change?
   
   I do like that it shows the nullability a bit further to the reader, but it is not a necessary change.



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

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

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


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


[GitHub] [iceberg] rdblue merged pull request #5118: CORE - Load Table Response Test

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


-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    // Add location
+    TableMetadata metadataWithLocation = TableMetadata
+        .buildFrom(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadataWithLocation)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerDe() throws JsonProcessingException {
+
+    // Build table metadata so that its uuid and last updated millis can be reused.
+    TableMetadata metaV1 = TableMetadata.buildFrom(
+        TableMetadata.newTableMetadata(
+            TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+
+    LoadTableResponse respV1 = LoadTableResponse.builder()
+        .withTableMetadata(metaV1)
+        .addAllConfig(CONFIG)
+        .build();
+
+    String fullJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":1," +
+        "    \"table-uuid\":\"" + metaV1.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-updated-ms\":" + metaV1.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "       ]" +
+        "    }," +
+        "  \"current-schema-id\":0," +
+        "  \"schemas\":[" +
+        "    {\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "      {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "      {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "      {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"partition-spec\":[]," +
+        "  \"default-spec-id\":0," +
+        "  \"partition-specs\":[{\"spec-id\":0,\"fields\":[]}]," +
+        "  \"last-partition-id\":999,\"default-sort-order-id\":1," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":1,\"fields\":[" +
+        "        {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "        {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"properties\":{\"owner\":\"hank\"}," +
+        "  \"current-snapshot-id\":-1," +
+        "  \"refs\":{},\"snapshots\":[],\"snapshot-log\":[],\"metadata-log\":[]},\"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullJson.replaceAll("\\s", ""), respV1);
+
+    TableMetadata metaV2 = buildV2TableMetadataWithTagsAndSnapshotHistory();
+    LoadTableResponse respV2 = LoadTableResponse.builder()
+        .withTableMetadata(metaV2)
+        .addAllConfig(CONFIG)
+        .build();
+    Snapshot currentSnapshot = metaV2.snapshots().get(1);
+    Snapshot previousSnapshot = metaV2.snapshots().get(0);
+    String fullExampleInstanceJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":2," +
+        "    \"table-uuid\":\"" + metaV2.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-sequence-number\":34," +
+        "    \"last-updated-ms\":" + metaV2.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"current-schema-id\":7," +
+        "    \"schemas\":[{" +
+        "      \"type\":\"struct\",\"schema-id\":7,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]},{" +
+        "      \"type\":\"struct\",\"schema-id\":6,\"fields\":[" +
+        "        {\"id\":10,\"name\":\"x\",\"required\":true,\"type\":\"string\"}]" +
+        "    }]," +
+        "  \"default-spec-id\":5," +
+        "  \"partition-specs\":[{\"spec-id\":5,\"fields\":[]}]," +
+        "  \"last-partition-id\":999," +
+        "  \"default-sort-order-id\":3," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":3,\"fields\":[" +
+        "      {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "      {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}]}]," +
+        "  \"properties\":{\"property\":\"value\"}," +
+        "  \"current-snapshot-id\":" + metaV2.currentSnapshot().snapshotId() + "," +
+        "  \"refs\":{" +
+        "    \"main\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("main").snapshotId() + "," +
+        "      \"type\":\"branch\"}," +
+        "    \"previous\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("previous").snapshotId() + "," +
+        "      \"type\":\"tag\"}," +
+        "    \"test\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("test").snapshotId() + "," +
+        "     \"type\":\"branch\"}" +
+        "  }," +
+        "  \"snapshots\":[{" +
+        "    \"snapshot-id\":" + metaV2.snapshots().get(0).snapshotId() + "," +
+        "    \"timestamp-ms\":" + metaV2.snapshots().get(0).timestampMillis() + "," +
+        "    \"manifests\":[" +
+        "      \"/tmp/manifest.1.avro\"" +
+        "    ]},{" +
+        "    \"snapshot-id\":" + metaV2.snapshots().get(1).snapshotId() + "," +
+        "    \"parent-snapshot-id\":" + metaV2.snapshots().get(1).parentId() + "," +
+        "    \"timestamp-ms\":" + metaV2.snapshots().get(1).timestampMillis() + "," +
+        "    \"manifests\":[" +
+        "      \"/tmp/manifest.2.avro\"" +
+        "    ],\"schema-id\":7" +
+        "  }]," +
+        "  \"snapshot-log\":[{" +
+        "      \"timestamp-ms\":" + previousSnapshot.timestampMillis() + "," +
+        "      \"snapshot-id\":" + previousSnapshot.snapshotId() +
+        "    },{" +
+        "      \"timestamp-ms\":" + currentSnapshot.timestampMillis() + "," +
+        "      \"snapshot-id\":" + currentSnapshot.snapshotId() +
+        "  }]," +
+        "  \"metadata-log\":[]}," +
+        "  \"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullExampleInstanceJson.replaceAll("\\s", ""), respV2);
+
+  }
+
+
+  private TableMetadata buildV2TableMetadataWithTagsAndSnapshotHistory() {
+    Snapshot previousSnapshot = new BaseSnapshot(
+        ops.io(), PREVIOUS_SNAPSHOT_ID, null, PREVIOUS_SNAPSHOT_ID, null, null, null, ImmutableList.of(

Review Comment:
   I thought Snapshots no longer needed to have IO passed in?



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    // Add location
+    TableMetadata metadataWithLocation = TableMetadata
+        .buildFrom(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadataWithLocation)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerDe() throws JsonProcessingException {
+
+    // Build table metadata so that its uuid and last updated millis can be reused.
+    TableMetadata metaV1 = TableMetadata.buildFrom(
+        TableMetadata.newTableMetadata(
+            TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+
+    LoadTableResponse respV1 = LoadTableResponse.builder()
+        .withTableMetadata(metaV1)
+        .addAllConfig(CONFIG)
+        .build();
+
+    String fullJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":1," +
+        "    \"table-uuid\":\"" + metaV1.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-updated-ms\":" + metaV1.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "       ]" +
+        "    }," +
+        "  \"current-schema-id\":0," +
+        "  \"schemas\":[" +
+        "    {\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "      {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "      {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "      {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"partition-spec\":[]," +
+        "  \"default-spec-id\":0," +
+        "  \"partition-specs\":[{\"spec-id\":0,\"fields\":[]}]," +
+        "  \"last-partition-id\":999,\"default-sort-order-id\":1," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":1,\"fields\":[" +
+        "        {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "        {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"properties\":{\"owner\":\"hank\"}," +
+        "  \"current-snapshot-id\":-1," +
+        "  \"refs\":{},\"snapshots\":[],\"snapshot-log\":[],\"metadata-log\":[]},\"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullJson.replaceAll("\\s", ""), respV1);
+
+    TableMetadata metaV2 = buildV2TableMetadataWithTagsAndSnapshotHistory();
+    LoadTableResponse respV2 = LoadTableResponse.builder()
+        .withTableMetadata(metaV2)
+        .addAllConfig(CONFIG)
+        .build();
+    Snapshot currentSnapshot = metaV2.snapshots().get(1);
+    Snapshot previousSnapshot = metaV2.snapshots().get(0);
+    String fullExampleInstanceJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":2," +
+        "    \"table-uuid\":\"" + metaV2.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-sequence-number\":34," +
+        "    \"last-updated-ms\":" + metaV2.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"current-schema-id\":7," +
+        "    \"schemas\":[{" +
+        "      \"type\":\"struct\",\"schema-id\":7,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]},{" +
+        "      \"type\":\"struct\",\"schema-id\":6,\"fields\":[" +
+        "        {\"id\":10,\"name\":\"x\",\"required\":true,\"type\":\"string\"}]" +
+        "    }]," +
+        "  \"default-spec-id\":5," +
+        "  \"partition-specs\":[{\"spec-id\":5,\"fields\":[]}]," +
+        "  \"last-partition-id\":999," +
+        "  \"default-sort-order-id\":3," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":3,\"fields\":[" +
+        "      {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "      {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}]}]," +
+        "  \"properties\":{\"property\":\"value\"}," +
+        "  \"current-snapshot-id\":" + metaV2.currentSnapshot().snapshotId() + "," +
+        "  \"refs\":{" +
+        "    \"main\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("main").snapshotId() + "," +
+        "      \"type\":\"branch\"}," +
+        "    \"previous\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("previous").snapshotId() + "," +
+        "      \"type\":\"tag\"}," +
+        "    \"test\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("test").snapshotId() + "," +
+        "     \"type\":\"branch\"}" +
+        "  }," +
+        "  \"snapshots\":[{" +
+        "    \"snapshot-id\":" + metaV2.snapshots().get(0).snapshotId() + "," +
+        "    \"timestamp-ms\":" + metaV2.snapshots().get(0).timestampMillis() + "," +
+        "    \"manifests\":[" +
+        "      \"/tmp/manifest.1.avro\"" +
+        "    ]},{" +
+        "    \"snapshot-id\":" + metaV2.snapshots().get(1).snapshotId() + "," +
+        "    \"parent-snapshot-id\":" + metaV2.snapshots().get(1).parentId() + "," +
+        "    \"timestamp-ms\":" + metaV2.snapshots().get(1).timestampMillis() + "," +
+        "    \"manifests\":[" +
+        "      \"/tmp/manifest.2.avro\"" +
+        "    ],\"schema-id\":7" +
+        "  }]," +
+        "  \"snapshot-log\":[{" +
+        "      \"timestamp-ms\":" + previousSnapshot.timestampMillis() + "," +
+        "      \"snapshot-id\":" + previousSnapshot.snapshotId() +
+        "    },{" +
+        "      \"timestamp-ms\":" + currentSnapshot.timestampMillis() + "," +
+        "      \"snapshot-id\":" + currentSnapshot.snapshotId() +
+        "  }]," +
+        "  \"metadata-log\":[]}," +
+        "  \"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullExampleInstanceJson.replaceAll("\\s", ""), respV2);
+
+  }
+
+
+  private TableMetadata buildV2TableMetadataWithTagsAndSnapshotHistory() {
+    Snapshot previousSnapshot = new BaseSnapshot(
+        ops.io(), PREVIOUS_SNAPSHOT_ID, null, PREVIOUS_SNAPSHOT_ID, null, null, null, ImmutableList.of(

Review Comment:
   These are removed entirely.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/BaseSnapshot.java:
##########
@@ -32,7 +32,8 @@
 import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 
-class BaseSnapshot implements Snapshot {
+// visible for testing
+public class BaseSnapshot implements Snapshot {

Review Comment:
   Reverted



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -106,6 +113,8 @@ public Builder addAllConfig(Map<String, String> properties) {
 
     public LoadTableResponse build() {
       Preconditions.checkNotNull(metadata, "Invalid metadata: null");
+      // This is causing errors in TestRESTCatalog via CatalogHandlers
+      Preconditions.checkNotNull(metadataLocation, "Invalid metadata location: null");

Review Comment:
   This is the check that is causing 16 test failures in `TestRESTCatalog` - which has been reduced to 13 by the changes in `CatalogHandlers`.
   
   Without this check, all tests in `TestRESTCatalog` pass. See notes here in this comment thread as well: https://github.com/apache/iceberg/pull/5118#discussion_r907805539



-- 
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 #5118: CORE - Load Table Response Test

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1464,19 +1464,23 @@ components:
     LoadTableResult:
       description:
         Result used when a table is successfully loaded.
-        
-        
-        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata must be returned in the `metadata-location` field. Clients can check whether metadata has changed by comparing metadata locations.
-        
-        
+
+
+        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata should be returned in the `metadata-location` field, unless it hasn't been determined yet - such as when a transaction begins to stage a table for creation but has not commit.
+
+
+        Clients can check whether metadata has changed by comparing metadata locations after the table has been created.

Review Comment:
   Let's leave this as part of the previous paragraph.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema SCHEMA_7 = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(SCHEMA_7).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(SCHEMA_7)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadata-location", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        SCHEMA_7, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Override
+  public void testHasOnlyKnownFields() {
+    Set<String> fieldsFromSpec = Sets.newHashSet();
+    Collections.addAll(fieldsFromSpec, allFieldsFromSpec());
+    try {
+      JsonNode node = mapper().readValue(serialize(createExampleInstance()), JsonNode.class);
+      for (String field : fieldsFromSpec) {
+        Assert.assertTrue("Should have field: " + field, node.has(field));
+      }
+
+      for (String field : ((Iterable<? extends String>) node::fieldNames)) {
+        Assert.assertTrue("Should not have field: " + field, fieldsFromSpec.contains(field));
+      }
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerde() throws Exception {
+    // Default fields are missing in this JSON
+    String tableMetadataV1Json = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(ops.io(), TEST_METADATA_LOCATION, tableMetadataV1Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV1));
+    LoadTableResponse resp1 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV1)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json1, resp1);
+
+    String tableMetadataV2Json = readTableMetadataInputFile("TableMetadataV2Valid.json");
+    TableMetadata metadataV2 = TableMetadataParser.fromJson(ops.io(), TEST_METADATA_LOCATION, tableMetadataV2Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json2 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV2));
+    LoadTableResponse resp2 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV2)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json2, resp2);
+  }
+
+  @Test
+  public void testCanDeserializeWithoutDefaultValues() throws Exception {
+    String metadataJson = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, metadataJson);
+
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(ops.io(), metadataJson);

Review Comment:
   Why is this passing `ops.io()`? I thought that we already determined that we could leave `io` null.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema SCHEMA_7 = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(SCHEMA_7).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(SCHEMA_7)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadata-location", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        SCHEMA_7, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Override
+  public void testHasOnlyKnownFields() {
+    Set<String> fieldsFromSpec = Sets.newHashSet();
+    Collections.addAll(fieldsFromSpec, allFieldsFromSpec());
+    try {
+      JsonNode node = mapper().readValue(serialize(createExampleInstance()), JsonNode.class);
+      for (String field : fieldsFromSpec) {
+        Assert.assertTrue("Should have field: " + field, node.has(field));
+      }
+
+      for (String field : ((Iterable<? extends String>) node::fieldNames)) {
+        Assert.assertTrue("Should not have field: " + field, fieldsFromSpec.contains(field));
+      }
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerde() throws Exception {
+    // Default fields are missing in this JSON
+    String tableMetadataV1Json = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(ops.io(), TEST_METADATA_LOCATION, tableMetadataV1Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV1));
+    LoadTableResponse resp1 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV1)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json1, resp1);
+
+    String tableMetadataV2Json = readTableMetadataInputFile("TableMetadataV2Valid.json");
+    TableMetadata metadataV2 = TableMetadataParser.fromJson(ops.io(), TEST_METADATA_LOCATION, tableMetadataV2Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json2 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV2));
+    LoadTableResponse resp2 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV2)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json2, resp2);
+  }
+
+  @Test
+  public void testCanDeserializeWithoutDefaultValues() throws Exception {
+    String metadataJson = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, metadataJson);
+
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(ops.io(), metadataJson);

Review Comment:
   Why is this passing `ops`?



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -89,8 +90,16 @@ private Builder() {
     }
 
     public Builder withTableMetadata(TableMetadata tableMetadata) {
-      this.metadataLocation = tableMetadata.metadataFileLocation();
       this.metadata = tableMetadata;
+      if (tableMetadata.metadataFileLocation() != null) {
+        this.metadataLocation = tableMetadata.metadataFileLocation();
+      }
+
+      return this;
+    }
+
+    public Builder withMetadataLocation(String metaLocation) {

Review Comment:
   Yeah, please remove any unnecessary changes.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1464,19 +1464,23 @@ components:
     LoadTableResult:
       description:
         Result used when a table is successfully loaded.
-        
-        
-        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata must be returned in the `metadata-location` field. Clients can check whether metadata has changed by comparing metadata locations.
-        
-        
+
+
+        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata should be returned in the `metadata-location` field, unless it hasn't been determined yet - such as when a transaction begins to stage a table for creation but has not commit.

Review Comment:
   Updated to use this language.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/RequestResponseTestBase.java:
##########
@@ -104,7 +103,7 @@ protected void assertRoundTripSerializesEquallyFrom(String json, T expected) thr
     T actual = deserialize(json);
     assertEquals(actual, expected);
 
-    // Check that the deserialized value serializes back into the original JSON
-    Assertions.assertThat(serialize(expected)).isEqualTo(json);
+    Assert.assertEquals("The serialized version of the Java record should match the original JSON",
+        serialize(expected), json);

Review Comment:
   I updated this as the comparison of two JSON strings can be quite cumbersome to diff using `Assertions`.
   
   This way, the error message is much more clear about exactly what part of the string is different.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -89,8 +90,11 @@ private Builder() {
     }
 
     public Builder withTableMetadata(TableMetadata tableMetadata) {
-      this.metadataLocation = tableMetadata.metadataFileLocation();
       this.metadata = tableMetadata;
+      if (tableMetadata.metadataFileLocation() != null) {
+        this.metadataLocation = tableMetadata.metadataFileLocation();
+      }

Review Comment:
   As we've removed the `withMetadataLocation` method from the builder, should I revert this change?
   
   I do like that it shows the nullability a bit further to the reader, but it is arguably an unnecessary change.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema SCHEMA_7 = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(SCHEMA_7).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(SCHEMA_7)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadata-location", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata =
+        TableMetadata
+            .buildFrom(
+                TableMetadata.newTableMetadata(SCHEMA_7, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+            .withMetadataLocation(TEST_METADATA_LOCATION)
+            .build();
+
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadata)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerde() throws Exception {

Review Comment:
   Yeah. Split it into two tests 👍 



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    // Add location
+    TableMetadata metadataWithLocation = TableMetadata
+        .buildFrom(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadataWithLocation)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {

Review Comment:
   Updating the builder to require it is causing a number of failures in `TestRESTCatalog`.
   
   I'll uncomment it and push and collect the stacktrace.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -106,6 +113,8 @@ public Builder addAllConfig(Map<String, String> properties) {
 
     public LoadTableResponse build() {
       Preconditions.checkNotNull(metadata, "Invalid metadata: null");
+      // This is causing errors in TestRESTCatalog via CatalogHandlers
+      // Preconditions.checkNotNull(metadataLocation, "Invalid metadata location: null");

Review Comment:
   It's this `LoadTableResponse` https://github.com/apache/iceberg/blob/e0ff3c6e9876ce04c890908e375c876d14240c3e/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java#L483-L484, which is called in the failing tests via `catalog.buildTable(TABLE, OTHER_SCHEMA).createTransaction()` which then fails the builder because there's no metadata location yet.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -170,16 +171,20 @@ public static LoadTableResponse stageTableCreate(Catalog catalog, Namespace name
     properties.putAll(request.properties());
 
     String location;
+    String metadataLocation;
     if (request.location() != null) {
       location = request.location();
+      metadataLocation = location + "/metadata";
     } else {
-      location = catalog.buildTable(ident, request.schema())
+      Table tbl = catalog.buildTable(ident, request.schema())

Review Comment:
   In Iceberg, we don't avoid abbreviations like `tbl`. It's no easier to type and is harder to read. Use `table` instead.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -106,6 +113,8 @@ public Builder addAllConfig(Map<String, String> properties) {
 
     public LoadTableResponse build() {
       Preconditions.checkNotNull(metadata, "Invalid metadata: null");
+      // This is causing errors in TestRESTCatalog via CatalogHandlers
+      // Preconditions.checkNotNull(metadataLocation, "Invalid metadata location: null");

Review Comment:
   We've determined that `metadata-location` is not required (can be null), due to this very case.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/BaseSnapshot.java:
##########
@@ -32,7 +32,8 @@
 import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 
-class BaseSnapshot implements Snapshot {
+// visible for testing
+public class BaseSnapshot implements Snapshot {

Review Comment:
   I did mean that they are all in `core`. As opposed to having this be in `api` where we have tests and where stability is guaranteed.
   
   I agree with your statement though, and I will update the tests to read in a JSON file with a `TableMetadata` instance and then rely on `TableMetadataParser`, as we're testing `LoadTableResponse`.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/TableMetadataParser.java:
##########
@@ -83,30 +83,30 @@ private TableMetadataParser() {
   }
 
   // visible for testing
-  static final String FORMAT_VERSION = "format-version";
-  static final String TABLE_UUID = "table-uuid";
-  static final String LOCATION = "location";
-  static final String LAST_SEQUENCE_NUMBER = "last-sequence-number";
-  static final String LAST_UPDATED_MILLIS = "last-updated-ms";
-  static final String LAST_COLUMN_ID = "last-column-id";
-  static final String SCHEMA = "schema";
-  static final String SCHEMAS = "schemas";
-  static final String CURRENT_SCHEMA_ID = "current-schema-id";
-  static final String PARTITION_SPEC = "partition-spec";
-  static final String PARTITION_SPECS = "partition-specs";
-  static final String DEFAULT_SPEC_ID = "default-spec-id";
-  static final String LAST_PARTITION_ID = "last-partition-id";
-  static final String DEFAULT_SORT_ORDER_ID = "default-sort-order-id";
-  static final String SORT_ORDERS = "sort-orders";
-  static final String PROPERTIES = "properties";
-  static final String CURRENT_SNAPSHOT_ID = "current-snapshot-id";
-  static final String REFS = "refs";
-  static final String SNAPSHOTS = "snapshots";
-  static final String SNAPSHOT_ID = "snapshot-id";
-  static final String TIMESTAMP_MS = "timestamp-ms";
-  static final String SNAPSHOT_LOG = "snapshot-log";
-  static final String METADATA_FILE = "metadata-file";
-  static final String METADATA_LOG = "metadata-log";
+  public static final String FORMAT_VERSION = "format-version";
+  public static final String TABLE_UUID = "table-uuid";
+  public static final String LOCATION = "location";
+  public static final String LAST_SEQUENCE_NUMBER = "last-sequence-number";
+  public static final String LAST_UPDATED_MILLIS = "last-updated-ms";
+  public static final String LAST_COLUMN_ID = "last-column-id";
+  public static final String SCHEMA = "schema";
+  public static final String SCHEMAS = "schemas";
+  public static final String CURRENT_SCHEMA_ID = "current-schema-id";
+  public static final String PARTITION_SPEC = "partition-spec";
+  public static final String PARTITION_SPECS = "partition-specs";
+  public static final String DEFAULT_SPEC_ID = "default-spec-id";
+  public static final String LAST_PARTITION_ID = "last-partition-id";
+  public static final String DEFAULT_SORT_ORDER_ID = "default-sort-order-id";
+  public static final String SORT_ORDERS = "sort-orders";
+  public static final String PROPERTIES = "properties";
+  public static final String CURRENT_SNAPSHOT_ID = "current-snapshot-id";
+  public static final String REFS = "refs";
+  public static final String SNAPSHOTS = "snapshots";
+  public static final String SNAPSHOT_ID = "snapshot-id";
+  public static final String TIMESTAMP_MS = "timestamp-ms";
+  public static final String SNAPSHOT_LOG = "snapshot-log";
+  public static final String METADATA_FILE = "metadata-file";
+  public static final String METADATA_LOG = "metadata-log";

Review Comment:
   Also, I just checked and it doesn't look like `SNAPSHOT_ID` is used in the tests. The string is embedded instead, so it should be even easier to make this change.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -225,6 +225,16 @@ public static LoadTableResponse loadTable(Catalog catalog, TableIdentifier ident
     Table table = catalog.loadTable(ident);
 
     if (table instanceof BaseTable) {
+      // This gives
+      // TestRESTCatalog > testCompleteReplaceTransaction() FAILED
+      // java.lang.AssertionError: Table should have correct number of previous metadata locations expected:<1> but was
+      //   :<2>
+      // TableMetadata metadataWithoutLocation = ((BaseTable) table).operations().current();
+      // String metadataLocation = ((BaseTable) table).operations().metadataFileLocation("metadataV1.json");
+      // TableMetadata metadata = TableMetadata.buildFrom(metadataWithoutLocation)
+      //     .withMetadataLocation(metadataLocation)
+      //     .build();
+      // String metadataLocation = ((BaseTable) table).operations().current().previousFiles();
       return LoadTableResponse.builder()
           .withTableMetadata(((BaseTable) table).operations().current())

Review Comment:
   This `TableMetadata` has no metadataLocation value on it. Tracking down where this is built and will update it.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -336,4 +348,8 @@ private static TableMetadata commit(TableOperations ops, UpdateTableRequest requ
 
     return ops.current();
   }
+
+  private static String metadataLocation(Table tbl) {
+    return ((HasTableOperations) tbl).operations().current().metadataFileLocation();
+  }

Review Comment:
   Currently unused - was using to check the metadata file location in some of the failing tests. It's only intermediate results that have a `LoadTableResponse` without a metadataLocation (e.g. if `stageCreate` is being called).
   
   So maybe `metadata-location` should be nullable as `LoadTableResponse` can be returned without it? Or we need to be adding it somewhere in the responses that we're not inside of `RESTSessionCatalog`.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1464,19 +1464,23 @@ components:
     LoadTableResult:
       description:
         Result used when a table is successfully loaded.
-        
-        
-        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata must be returned in the `metadata-location` field. Clients can check whether metadata has changed by comparing metadata locations.
-        
-        
+
+
+        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata should be returned in the `metadata-location` field, unless it hasn't been determined yet - such as when a transaction begins to stage a table for creation but has not commit.

Review Comment:
   Instead of adding ` - ` and extending the sentence, start a new one with "For example".
   
   It should be something like "unless the metadata is not yet committed. For example, a create transaction may return metadata that is staged but not committed."



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -170,16 +171,20 @@ public static LoadTableResponse stageTableCreate(Catalog catalog, Namespace name
     properties.putAll(request.properties());
 
     String location;
+    String metadataLocation;
     if (request.location() != null) {
       location = request.location();
+      metadataLocation = location + "/metadata";

Review Comment:
   Why does this hard-code `/metadata`?



-- 
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 #5118: CORE - Load Table Response Test

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

   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] rdblue commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    // Add location
+    TableMetadata metadataWithLocation = TableMetadata
+        .buildFrom(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadataWithLocation)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerDe() throws JsonProcessingException {
+
+    // Build table metadata so that its uuid and last updated millis can be reused.
+    TableMetadata metaV1 = TableMetadata.buildFrom(
+        TableMetadata.newTableMetadata(
+            TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+
+    LoadTableResponse respV1 = LoadTableResponse.builder()
+        .withTableMetadata(metaV1)
+        .addAllConfig(CONFIG)
+        .build();
+
+    String fullJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":1," +
+        "    \"table-uuid\":\"" + metaV1.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-updated-ms\":" + metaV1.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "       ]" +
+        "    }," +
+        "  \"current-schema-id\":0," +
+        "  \"schemas\":[" +
+        "    {\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "      {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "      {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "      {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"partition-spec\":[]," +
+        "  \"default-spec-id\":0," +
+        "  \"partition-specs\":[{\"spec-id\":0,\"fields\":[]}]," +
+        "  \"last-partition-id\":999,\"default-sort-order-id\":1," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":1,\"fields\":[" +
+        "        {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "        {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"properties\":{\"owner\":\"hank\"}," +
+        "  \"current-snapshot-id\":-1," +
+        "  \"refs\":{},\"snapshots\":[],\"snapshot-log\":[],\"metadata-log\":[]},\"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullJson.replaceAll("\\s", ""), respV1);
+
+    TableMetadata metaV2 = buildV2TableMetadataWithTagsAndSnapshotHistory();
+    LoadTableResponse respV2 = LoadTableResponse.builder()
+        .withTableMetadata(metaV2)
+        .addAllConfig(CONFIG)
+        .build();
+    Snapshot currentSnapshot = metaV2.snapshots().get(1);
+    Snapshot previousSnapshot = metaV2.snapshots().get(0);
+    String fullExampleInstanceJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":2," +
+        "    \"table-uuid\":\"" + metaV2.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-sequence-number\":34," +
+        "    \"last-updated-ms\":" + metaV2.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"current-schema-id\":7," +
+        "    \"schemas\":[{" +
+        "      \"type\":\"struct\",\"schema-id\":7,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]},{" +
+        "      \"type\":\"struct\",\"schema-id\":6,\"fields\":[" +
+        "        {\"id\":10,\"name\":\"x\",\"required\":true,\"type\":\"string\"}]" +
+        "    }]," +
+        "  \"default-spec-id\":5," +
+        "  \"partition-specs\":[{\"spec-id\":5,\"fields\":[]}]," +
+        "  \"last-partition-id\":999," +
+        "  \"default-sort-order-id\":3," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":3,\"fields\":[" +
+        "      {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "      {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}]}]," +
+        "  \"properties\":{\"property\":\"value\"}," +
+        "  \"current-snapshot-id\":" + metaV2.currentSnapshot().snapshotId() + "," +
+        "  \"refs\":{" +
+        "    \"main\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("main").snapshotId() + "," +
+        "      \"type\":\"branch\"}," +
+        "    \"previous\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("previous").snapshotId() + "," +
+        "      \"type\":\"tag\"}," +
+        "    \"test\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("test").snapshotId() + "," +
+        "     \"type\":\"branch\"}" +
+        "  }," +
+        "  \"snapshots\":[{" +
+        "    \"snapshot-id\":" + metaV2.snapshots().get(0).snapshotId() + "," +
+        "    \"timestamp-ms\":" + metaV2.snapshots().get(0).timestampMillis() + "," +
+        "    \"manifests\":[" +
+        "      \"/tmp/manifest.1.avro\"" +
+        "    ]},{" +
+        "    \"snapshot-id\":" + metaV2.snapshots().get(1).snapshotId() + "," +
+        "    \"parent-snapshot-id\":" + metaV2.snapshots().get(1).parentId() + "," +
+        "    \"timestamp-ms\":" + metaV2.snapshots().get(1).timestampMillis() + "," +
+        "    \"manifests\":[" +
+        "      \"/tmp/manifest.2.avro\"" +
+        "    ],\"schema-id\":7" +
+        "  }]," +
+        "  \"snapshot-log\":[{" +
+        "      \"timestamp-ms\":" + previousSnapshot.timestampMillis() + "," +
+        "      \"snapshot-id\":" + previousSnapshot.snapshotId() +
+        "    },{" +
+        "      \"timestamp-ms\":" + currentSnapshot.timestampMillis() + "," +
+        "      \"snapshot-id\":" + currentSnapshot.snapshotId() +
+        "  }]," +
+        "  \"metadata-log\":[]}," +
+        "  \"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullExampleInstanceJson.replaceAll("\\s", ""), respV2);
+
+  }
+
+

Review Comment:
   Style: only one empty line is needed between methods.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    // Add location
+    TableMetadata metadataWithLocation = TableMetadata
+        .buildFrom(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadataWithLocation)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {

Review Comment:
   Metadata location is also required according to 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] rdblue commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema SCHEMA_7 = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(SCHEMA_7).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(SCHEMA_7)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadata-location", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        SCHEMA_7, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Override
+  public void testHasOnlyKnownFields() {

Review Comment:
   Now that #5135 has been merged, can you update 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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/RequestResponseTestBase.java:
##########
@@ -104,7 +103,7 @@ protected void assertRoundTripSerializesEquallyFrom(String json, T expected) thr
     T actual = deserialize(json);
     assertEquals(actual, expected);
 
-    // Check that the deserialized value serializes back into the original JSON
-    Assertions.assertThat(serialize(expected)).isEqualTo(json);
+    Assert.assertEquals("The serialized version of the Java record should match the original JSON",
+        serialize(expected), json);

Review Comment:
   Let's do this separately. I don't think this is needed to finish the tests, 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] kbendick commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -89,8 +90,11 @@ private Builder() {
     }
 
     public Builder withTableMetadata(TableMetadata tableMetadata) {
-      this.metadataLocation = tableMetadata.metadataFileLocation();
       this.metadata = tableMetadata;
+      if (tableMetadata.metadataFileLocation() != null) {
+        this.metadataLocation = tableMetadata.metadataFileLocation();
+      }

Review Comment:
   This has been reverted.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -225,6 +225,16 @@ public static LoadTableResponse loadTable(Catalog catalog, TableIdentifier ident
     Table table = catalog.loadTable(ident);
 
     if (table instanceof BaseTable) {
+      // This gives
+      // TestRESTCatalog > testCompleteReplaceTransaction() FAILED
+      // java.lang.AssertionError: Table should have correct number of previous metadata locations expected:<1> but was
+      //   :<2>
+      // TableMetadata metadataWithoutLocation = ((BaseTable) table).operations().current();
+      // String metadataLocation = ((BaseTable) table).operations().metadataFileLocation("metadataV1.json");
+      // TableMetadata metadata = TableMetadata.buildFrom(metadataWithoutLocation)
+      //     .withMetadataLocation(metadataLocation)
+      //     .build();
+      // String metadataLocation = ((BaseTable) table).operations().current().previousFiles();
       return LoadTableResponse.builder()
           .withTableMetadata(((BaseTable) table).operations().current())

Review Comment:
   It actually does have metadata location by this point in time. There's some internal usage of `LoadTableResponse` where the `metadataLocation` isn't known (e.g. during `stageCreate` inside of `RESTSessionCatalog`).
   
   Removing the requirement that the `LoadTableResponse` builder have `metadataLocation` at all times fixes this.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1464,19 +1464,23 @@ components:
     LoadTableResult:
       description:
         Result used when a table is successfully loaded.
-        
-        
-        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata must be returned in the `metadata-location` field. Clients can check whether metadata has changed by comparing metadata locations.
-        
-        
+
+
+        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata should be returned in the `metadata-location` field, unless it hasn't been determined yet - such as when a transaction begins to stage a table for creation but has not commit.
+
+
+        Clients can check whether metadata has changed by comparing metadata locations after the table has been created.
+
+
         The `config` map returns table-specific configuration for the table's resources, including its HTTP client and FileIO. For example, config may contain a specific FileIO implementation class for the table depending on its underlying storage.
       type: object
       required:
-        - metadata-location
         - metadata
       properties:
         metadata-location:
           type: string
+          nullable: true
+          description: Current location of the table's metadata file. Should only be null if the table has been staged for creation as part of a transaction but does not yet exist.

Review Comment:
   I think this should be shorter. "May be null if the table is staged as part of a transaction".



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema SCHEMA_7 = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(SCHEMA_7).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(SCHEMA_7)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadata-location", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata =
+        TableMetadata
+            .buildFrom(
+                TableMetadata.newTableMetadata(SCHEMA_7, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+            .withMetadataLocation(TEST_METADATA_LOCATION)
+            .build();
+
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadata)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerde() throws Exception {
+    // Default fields are missing in this JSON
+    String tableMetadataV1Json = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(null, TEST_METADATA_LOCATION, tableMetadataV1Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV1));
+    LoadTableResponse resp1 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV1)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json1, resp1);
+
+    String tableMetadataV2Json = readTableMetadataInputFile("TableMetadataV2Valid.json");
+    TableMetadata metadataV2 = TableMetadataParser.fromJson(null, TEST_METADATA_LOCATION, tableMetadataV2Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json2 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV2));
+    LoadTableResponse resp2 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV2)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json2, resp2);
+  }
+
+  @Test
+  public void testCanDeserializeWithoutDefaultValues() throws Exception {
+    String metadataJson = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, metadataJson);
+
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(null, TEST_METADATA_LOCATION, metadataJson);
+    LoadTableResponse resp1 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV1)
+        .addAllConfig(CONFIG)
+        .build();
+    assertEquals(deserialize(json1), resp1);

Review Comment:
   > Yeah that's what this is testing. It's a little unclear from the way it's written admittedly.
   
   I'm not sure I understand. What was this testing in the version I was looking at?
   
   Initially, I was going to comment that it duplicates the first part of the round-trip test for v1. Then I realized that it was intended to test something else.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -89,8 +90,11 @@ private Builder() {
     }
 
     public Builder withTableMetadata(TableMetadata tableMetadata) {
-      this.metadataLocation = tableMetadata.metadataFileLocation();
       this.metadata = tableMetadata;
+      if (tableMetadata.metadataFileLocation() != null) {
+        this.metadataLocation = tableMetadata.metadataFileLocation();
+      }

Review Comment:
   Yes, please remove unnecessary changes.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -189,8 +194,12 @@ public static LoadTableResponse stageTableCreate(Catalog catalog, Namespace name
         location,
         properties);
 
+    TableMetadata metadataWithLocation = TableMetadata.buildFrom(metadata)
+        .withMetadataLocation(metadataLocation)
+        .build();
+
     return LoadTableResponse.builder()
-        .withTableMetadata(metadata)
+        .withTableMetadata(metadataWithLocation)

Review Comment:
   We've determined that `metadata-location` should not be required, specifically as a `LoadTableResponse` might be returned in response to a request to stage the creation of a table (i.e. during a transaction).
   
   But given that the table _does_ have `metadata-location` by this point in time, we should add it to the response.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1464,19 +1464,23 @@ components:
     LoadTableResult:
       description:
         Result used when a table is successfully loaded.
-        
-        
-        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata must be returned in the `metadata-location` field. Clients can check whether metadata has changed by comparing metadata locations.
-        
-        
+
+
+        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata should be returned in the `metadata-location` field, unless it hasn't been determined yet - such as when a transaction begins to stage a table for creation but has not commit.
+
+
+        Clients can check whether metadata has changed by comparing metadata locations after the table has been created.
+
+
         The `config` map returns table-specific configuration for the table's resources, including its HTTP client and FileIO. For example, config may contain a specific FileIO implementation class for the table depending on its underlying storage.
       type: object
       required:
-        - metadata-location
         - metadata
       properties:
         metadata-location:
           type: string
+          nullable: true

Review Comment:
   Yeah we don't tend to make usage of `nullable`. Removing.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema SCHEMA_7 = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(SCHEMA_7).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(SCHEMA_7)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadata-location", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        SCHEMA_7, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Override
+  public void testHasOnlyKnownFields() {
+    Set<String> fieldsFromSpec = Sets.newHashSet();
+    Collections.addAll(fieldsFromSpec, allFieldsFromSpec());
+    try {
+      JsonNode node = mapper().readValue(serialize(createExampleInstance()), JsonNode.class);
+      for (String field : fieldsFromSpec) {
+        Assert.assertTrue("Should have field: " + field, node.has(field));
+      }
+
+      for (String field : ((Iterable<? extends String>) node::fieldNames)) {
+        Assert.assertTrue("Should not have field: " + field, fieldsFromSpec.contains(field));
+      }
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerde() throws Exception {
+    // Default fields are missing in this JSON
+    String tableMetadataV1Json = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(ops.io(), TEST_METADATA_LOCATION, tableMetadataV1Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV1));
+    LoadTableResponse resp1 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV1)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json1, resp1);
+
+    String tableMetadataV2Json = readTableMetadataInputFile("TableMetadataV2Valid.json");
+    TableMetadata metadataV2 = TableMetadataParser.fromJson(ops.io(), TEST_METADATA_LOCATION, tableMetadataV2Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json2 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV2));
+    LoadTableResponse resp2 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV2)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json2, resp2);
+  }
+
+  @Test
+  public void testCanDeserializeWithoutDefaultValues() throws Exception {
+    String metadataJson = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, metadataJson);
+
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(ops.io(), metadataJson);

Review Comment:
   Changed to null and removed `ops` from the class entirely.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema SCHEMA_7 = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(SCHEMA_7).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(SCHEMA_7)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadata-location", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        SCHEMA_7, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Override
+  public void testHasOnlyKnownFields() {

Review Comment:
   Rebased and updated.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";

Review Comment:
   Typo: `UUIID` should be `UUID`.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/TableMetadataParser.java:
##########
@@ -83,30 +83,30 @@ private TableMetadataParser() {
   }
 
   // visible for testing
-  static final String FORMAT_VERSION = "format-version";
-  static final String TABLE_UUID = "table-uuid";
-  static final String LOCATION = "location";
-  static final String LAST_SEQUENCE_NUMBER = "last-sequence-number";
-  static final String LAST_UPDATED_MILLIS = "last-updated-ms";
-  static final String LAST_COLUMN_ID = "last-column-id";
-  static final String SCHEMA = "schema";
-  static final String SCHEMAS = "schemas";
-  static final String CURRENT_SCHEMA_ID = "current-schema-id";
-  static final String PARTITION_SPEC = "partition-spec";
-  static final String PARTITION_SPECS = "partition-specs";
-  static final String DEFAULT_SPEC_ID = "default-spec-id";
-  static final String LAST_PARTITION_ID = "last-partition-id";
-  static final String DEFAULT_SORT_ORDER_ID = "default-sort-order-id";
-  static final String SORT_ORDERS = "sort-orders";
-  static final String PROPERTIES = "properties";
-  static final String CURRENT_SNAPSHOT_ID = "current-snapshot-id";
-  static final String REFS = "refs";
-  static final String SNAPSHOTS = "snapshots";
-  static final String SNAPSHOT_ID = "snapshot-id";
-  static final String TIMESTAMP_MS = "timestamp-ms";
-  static final String SNAPSHOT_LOG = "snapshot-log";
-  static final String METADATA_FILE = "metadata-file";
-  static final String METADATA_LOG = "metadata-log";
+  public static final String FORMAT_VERSION = "format-version";
+  public static final String TABLE_UUID = "table-uuid";
+  public static final String LOCATION = "location";
+  public static final String LAST_SEQUENCE_NUMBER = "last-sequence-number";
+  public static final String LAST_UPDATED_MILLIS = "last-updated-ms";
+  public static final String LAST_COLUMN_ID = "last-column-id";
+  public static final String SCHEMA = "schema";
+  public static final String SCHEMAS = "schemas";
+  public static final String CURRENT_SCHEMA_ID = "current-schema-id";
+  public static final String PARTITION_SPEC = "partition-spec";
+  public static final String PARTITION_SPECS = "partition-specs";
+  public static final String DEFAULT_SPEC_ID = "default-spec-id";
+  public static final String LAST_PARTITION_ID = "last-partition-id";
+  public static final String DEFAULT_SORT_ORDER_ID = "default-sort-order-id";
+  public static final String SORT_ORDERS = "sort-orders";
+  public static final String PROPERTIES = "properties";
+  public static final String CURRENT_SNAPSHOT_ID = "current-snapshot-id";
+  public static final String REFS = "refs";
+  public static final String SNAPSHOTS = "snapshots";
+  public static final String SNAPSHOT_ID = "snapshot-id";
+  public static final String TIMESTAMP_MS = "timestamp-ms";
+  public static final String SNAPSHOT_LOG = "snapshot-log";
+  public static final String METADATA_FILE = "metadata-file";
+  public static final String METADATA_LOG = "metadata-log";

Review Comment:
   I think we can revert this. No need to test `TableMetadataParser`, just `LoadTableResponse`.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/TableMetadata.java:
##########
@@ -133,7 +133,8 @@ public static class SnapshotLogEntry implements HistoryEntry {
     private final long timestampMillis;
     private final long snapshotId;
 
-    SnapshotLogEntry(long timestampMillis, long snapshotId) {
+    // VisibleForTesting
+    public SnapshotLogEntry(long timestampMillis, long snapshotId) {

Review Comment:
   This is one I could make public via reflection, as the class itself is `public static`.



##########
core/src/main/java/org/apache/iceberg/TableMetadataParser.java:
##########
@@ -83,30 +83,30 @@ private TableMetadataParser() {
   }
 
   // visible for testing
-  static final String FORMAT_VERSION = "format-version";
-  static final String TABLE_UUID = "table-uuid";
-  static final String LOCATION = "location";
-  static final String LAST_SEQUENCE_NUMBER = "last-sequence-number";
-  static final String LAST_UPDATED_MILLIS = "last-updated-ms";
-  static final String LAST_COLUMN_ID = "last-column-id";
-  static final String SCHEMA = "schema";
-  static final String SCHEMAS = "schemas";
-  static final String CURRENT_SCHEMA_ID = "current-schema-id";
-  static final String PARTITION_SPEC = "partition-spec";
-  static final String PARTITION_SPECS = "partition-specs";
-  static final String DEFAULT_SPEC_ID = "default-spec-id";
-  static final String LAST_PARTITION_ID = "last-partition-id";
-  static final String DEFAULT_SORT_ORDER_ID = "default-sort-order-id";
-  static final String SORT_ORDERS = "sort-orders";
-  static final String PROPERTIES = "properties";
-  static final String CURRENT_SNAPSHOT_ID = "current-snapshot-id";
-  static final String REFS = "refs";
-  static final String SNAPSHOTS = "snapshots";
-  static final String SNAPSHOT_ID = "snapshot-id";
-  static final String TIMESTAMP_MS = "timestamp-ms";
-  static final String SNAPSHOT_LOG = "snapshot-log";
-  static final String METADATA_FILE = "metadata-file";
-  static final String METADATA_LOG = "metadata-log";
+  public static final String FORMAT_VERSION = "format-version";
+  public static final String TABLE_UUID = "table-uuid";
+  public static final String LOCATION = "location";
+  public static final String LAST_SEQUENCE_NUMBER = "last-sequence-number";
+  public static final String LAST_UPDATED_MILLIS = "last-updated-ms";
+  public static final String LAST_COLUMN_ID = "last-column-id";
+  public static final String SCHEMA = "schema";
+  public static final String SCHEMAS = "schemas";
+  public static final String CURRENT_SCHEMA_ID = "current-schema-id";
+  public static final String PARTITION_SPEC = "partition-spec";
+  public static final String PARTITION_SPECS = "partition-specs";
+  public static final String DEFAULT_SPEC_ID = "default-spec-id";
+  public static final String LAST_PARTITION_ID = "last-partition-id";
+  public static final String DEFAULT_SORT_ORDER_ID = "default-sort-order-id";
+  public static final String SORT_ORDERS = "sort-orders";
+  public static final String PROPERTIES = "properties";
+  public static final String CURRENT_SNAPSHOT_ID = "current-snapshot-id";
+  public static final String REFS = "refs";
+  public static final String SNAPSHOTS = "snapshots";
+  public static final String SNAPSHOT_ID = "snapshot-id";
+  public static final String TIMESTAMP_MS = "timestamp-ms";
+  public static final String SNAPSHOT_LOG = "snapshot-log";
+  public static final String METADATA_FILE = "metadata-file";
+  public static final String METADATA_LOG = "metadata-log";

Review Comment:
   I don't have concern making these public personally as they are generally speaking usable by engines if needeed, but open to other opinions.



##########
core/src/main/java/org/apache/iceberg/BaseSnapshot.java:
##########
@@ -32,7 +32,8 @@
 import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 
-class BaseSnapshot implements Snapshot {
+// visible for testing
+public class BaseSnapshot implements Snapshot {

Review Comment:
   I don't love these changes, but they are in `core`.



##########
core/src/test/java/org/apache/iceberg/LocalTableOperations.java:
##########
@@ -27,13 +27,13 @@
 import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.junit.rules.TemporaryFolder;
 
-class LocalTableOperations implements TableOperations {
+public class LocalTableOperations implements TableOperations {

Review Comment:
   This is in test so I don't personally have concerns making it public, but others might feel differently.



##########
api/src/main/java/org/apache/iceberg/PartitionSpec.java:
##########
@@ -102,7 +102,8 @@ public boolean isUnpartitioned() {
     return !isPartitioned();
   }
 
-  int lastAssignedFieldId() {
+  // visible for testing
+  public int lastAssignedFieldId() {

Review Comment:
   This is in the api module, so we might choose simply to not test this instead (or access the data by other means).



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    // Add location
+    TableMetadata metadataWithLocation = TableMetadata
+        .buildFrom(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadataWithLocation)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerDe() throws JsonProcessingException {
+
+    // Build table metadata so that its uuid and last updated millis can be reused.
+    TableMetadata metaV1 = TableMetadata.buildFrom(
+        TableMetadata.newTableMetadata(
+            TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+
+    LoadTableResponse respV1 = LoadTableResponse.builder()
+        .withTableMetadata(metaV1)
+        .addAllConfig(CONFIG)
+        .build();
+
+    String fullJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":1," +
+        "    \"table-uuid\":\"" + metaV1.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-updated-ms\":" + metaV1.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "       ]" +
+        "    }," +
+        "  \"current-schema-id\":0," +
+        "  \"schemas\":[" +
+        "    {\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "      {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "      {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "      {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"partition-spec\":[]," +
+        "  \"default-spec-id\":0," +
+        "  \"partition-specs\":[{\"spec-id\":0,\"fields\":[]}]," +
+        "  \"last-partition-id\":999,\"default-sort-order-id\":1," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":1,\"fields\":[" +
+        "        {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "        {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"properties\":{\"owner\":\"hank\"}," +
+        "  \"current-snapshot-id\":-1," +
+        "  \"refs\":{},\"snapshots\":[],\"snapshot-log\":[],\"metadata-log\":[]},\"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullJson.replaceAll("\\s", ""), respV1);
+
+    TableMetadata metaV2 = buildV2TableMetadataWithTagsAndSnapshotHistory();
+    LoadTableResponse respV2 = LoadTableResponse.builder()
+        .withTableMetadata(metaV2)
+        .addAllConfig(CONFIG)
+        .build();
+    Snapshot currentSnapshot = metaV2.snapshots().get(1);
+    Snapshot previousSnapshot = metaV2.snapshots().get(0);
+    String fullExampleInstanceJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +

Review Comment:
   Could you just use `TableMetadataParser` and build a string around it? There are similar tests in `TestCreateTableRequest` and that works great. I don't think it's very valuable to have complicated inline JSON like this.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
api/src/main/java/org/apache/iceberg/PartitionSpec.java:
##########
@@ -102,7 +102,8 @@ public boolean isUnpartitioned() {
     return !isPartitioned();
   }
 
-  int lastAssignedFieldId() {
+  // visible for testing
+  public int lastAssignedFieldId() {

Review Comment:
   Reverted this and am no longer testing it.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1464,19 +1464,23 @@ components:
     LoadTableResult:
       description:
         Result used when a table is successfully loaded.
-        
-        
-        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata must be returned in the `metadata-location` field. Clients can check whether metadata has changed by comparing metadata locations.
-        
-        
+
+
+        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata should be returned in the `metadata-location` field, unless it hasn't been determined yet - such as when a transaction begins to stage a table for creation but has not commit.
+
+
+        Clients can check whether metadata has changed by comparing metadata locations after the table has been created.
+
+
         The `config` map returns table-specific configuration for the table's resources, including its HTTP client and FileIO. For example, config may contain a specific FileIO implementation class for the table depending on its underlying storage.
       type: object
       required:
-        - metadata-location
         - metadata
       properties:
         metadata-location:
           type: string
+          nullable: true

Review Comment:
   I don't think that we use `nullable: true`. I thought we only used `required`.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    // Add location
+    TableMetadata metadataWithLocation = TableMetadata
+        .buildFrom(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadataWithLocation)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerDe() throws JsonProcessingException {
+
+    // Build table metadata so that its uuid and last updated millis can be reused.
+    TableMetadata metaV1 = TableMetadata.buildFrom(
+        TableMetadata.newTableMetadata(
+            TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+
+    LoadTableResponse respV1 = LoadTableResponse.builder()
+        .withTableMetadata(metaV1)
+        .addAllConfig(CONFIG)
+        .build();
+
+    String fullJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":1," +
+        "    \"table-uuid\":\"" + metaV1.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-updated-ms\":" + metaV1.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "       ]" +
+        "    }," +
+        "  \"current-schema-id\":0," +
+        "  \"schemas\":[" +
+        "    {\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "      {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "      {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "      {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"partition-spec\":[]," +
+        "  \"default-spec-id\":0," +
+        "  \"partition-specs\":[{\"spec-id\":0,\"fields\":[]}]," +
+        "  \"last-partition-id\":999,\"default-sort-order-id\":1," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":1,\"fields\":[" +
+        "        {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "        {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"properties\":{\"owner\":\"hank\"}," +
+        "  \"current-snapshot-id\":-1," +
+        "  \"refs\":{},\"snapshots\":[],\"snapshot-log\":[],\"metadata-log\":[]},\"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullJson.replaceAll("\\s", ""), respV1);
+
+    TableMetadata metaV2 = buildV2TableMetadataWithTagsAndSnapshotHistory();
+    LoadTableResponse respV2 = LoadTableResponse.builder()
+        .withTableMetadata(metaV2)
+        .addAllConfig(CONFIG)
+        .build();
+    Snapshot currentSnapshot = metaV2.snapshots().get(1);
+    Snapshot previousSnapshot = metaV2.snapshots().get(0);
+    String fullExampleInstanceJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":2," +
+        "    \"table-uuid\":\"" + metaV2.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-sequence-number\":34," +
+        "    \"last-updated-ms\":" + metaV2.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"current-schema-id\":7," +
+        "    \"schemas\":[{" +
+        "      \"type\":\"struct\",\"schema-id\":7,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]},{" +
+        "      \"type\":\"struct\",\"schema-id\":6,\"fields\":[" +
+        "        {\"id\":10,\"name\":\"x\",\"required\":true,\"type\":\"string\"}]" +
+        "    }]," +
+        "  \"default-spec-id\":5," +
+        "  \"partition-specs\":[{\"spec-id\":5,\"fields\":[]}]," +
+        "  \"last-partition-id\":999," +
+        "  \"default-sort-order-id\":3," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":3,\"fields\":[" +
+        "      {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "      {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}]}]," +
+        "  \"properties\":{\"property\":\"value\"}," +
+        "  \"current-snapshot-id\":" + metaV2.currentSnapshot().snapshotId() + "," +
+        "  \"refs\":{" +
+        "    \"main\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("main").snapshotId() + "," +
+        "      \"type\":\"branch\"}," +
+        "    \"previous\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("previous").snapshotId() + "," +
+        "      \"type\":\"tag\"}," +
+        "    \"test\":{" +
+        "      \"snapshot-id\":" + metaV2.ref("test").snapshotId() + "," +
+        "     \"type\":\"branch\"}" +
+        "  }," +
+        "  \"snapshots\":[{" +
+        "    \"snapshot-id\":" + metaV2.snapshots().get(0).snapshotId() + "," +
+        "    \"timestamp-ms\":" + metaV2.snapshots().get(0).timestampMillis() + "," +
+        "    \"manifests\":[" +
+        "      \"/tmp/manifest.1.avro\"" +
+        "    ]},{" +
+        "    \"snapshot-id\":" + metaV2.snapshots().get(1).snapshotId() + "," +
+        "    \"parent-snapshot-id\":" + metaV2.snapshots().get(1).parentId() + "," +
+        "    \"timestamp-ms\":" + metaV2.snapshots().get(1).timestampMillis() + "," +
+        "    \"manifests\":[" +
+        "      \"/tmp/manifest.2.avro\"" +
+        "    ],\"schema-id\":7" +
+        "  }]," +
+        "  \"snapshot-log\":[{" +
+        "      \"timestamp-ms\":" + previousSnapshot.timestampMillis() + "," +
+        "      \"snapshot-id\":" + previousSnapshot.snapshotId() +
+        "    },{" +
+        "      \"timestamp-ms\":" + currentSnapshot.timestampMillis() + "," +
+        "      \"snapshot-id\":" + currentSnapshot.snapshotId() +
+        "  }]," +
+        "  \"metadata-log\":[]}," +
+        "  \"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullExampleInstanceJson.replaceAll("\\s", ""), respV2);
+
+  }
+
+

Review Comment:
   This has been updated



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java:
##########
@@ -69,7 +69,7 @@ public void testRoundTripSerDe() throws JsonProcessingException {
         "\"fields\":[{\"name\":\"id_bucket\",\"transform\":\"bucket[16]\",\"source-id\":1,\"field-id\":1000}]}," +
         "\"order\":{\"order-id\":1,\"fields\":" +
         "[{\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-last\"}]}," +
-        "\"properties\":{\"owner\":\"Hank\"},\"stageCreate\":false}";
+        "\"properties\":{\"owner\":\"Hank\"},\"stage-create\":false}";

Review Comment:
   There's currently one failure `TestCreateTableRequest#testReoundTripSerDe`, that will be fixed once either https://github.com/apache/iceberg/pull/5135 or https://github.com/apache/iceberg/pull/5133 is merged.
   
   ```
   TestCreateTableRequest > testRoundTripSerDe FAILED
       org.junit.ComparisonFailure: The serialized version of the Java record should match the original JSON expected:<...wner":"Hank"},"stage[-c]reate":false}> but was:<...wner":"Hank"},"stage[C]reate":false}>
           at org.junit.Assert.assertEquals(Assert.java:117)
           at org.apache.iceberg.rest.RequestResponseTestBase.assertRoundTripSerializesEquallyFrom(RequestResponseTestBase.java:90)
           at org.apache.iceberg.rest.requests.TestCreateTableRequest.testRoundTripSerDe(TestCreateTableRequest.java:83)
   ```
   
   I've updated this for now so that this test failure won't cause issues and then will rebase when one of those PRs are 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] rdblue commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -28,6 +28,7 @@
 import org.apache.iceberg.BaseMetadataTable;
 import org.apache.iceberg.BaseTable;
 import org.apache.iceberg.BaseTransaction;
+import org.apache.iceberg.HasTableOperations;

Review Comment:
   Why has this class changed? Is it still needed?



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -189,8 +194,12 @@ public static LoadTableResponse stageTableCreate(Catalog catalog, Namespace name
         location,
         properties);
 
+    TableMetadata metadataWithLocation = TableMetadata.buildFrom(metadata)
+        .withMetadataLocation(metadataLocation)
+        .build();
+
     return LoadTableResponse.builder()
-        .withTableMetadata(metadata)
+        .withTableMetadata(metadataWithLocation)

Review Comment:
   This change causes the number of tests in `TestRESTCatalog` that fail if we require `metadataLocation` be non-null via the builder to go from 16 to 13.
   
   If we remove that requirement from the builder, then all tests pass. Given that we use `LoadTableResponse` in various places, I think we should drop the requirement that `metadataLocation` be present at all times and allow for it to 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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -106,6 +113,8 @@ public Builder addAllConfig(Map<String, String> properties) {
 
     public LoadTableResponse build() {
       Preconditions.checkNotNull(metadata, "Invalid metadata: null");
+      // This is causing errors in TestRESTCatalog via CatalogHandlers
+      // Preconditions.checkNotNull(metadataLocation, "Invalid metadata location: null");

Review Comment:
   There's a call to the builder somewhere that's used during the `CatalogTests` that is creating a `LoadTableResponse` without a metadata location.
   
   It seems to be related to the `LoadTableResponse` that's used at the beginning of a transaction, when a table has been staged for creation but the commit has not yet happened.
   
   Will investigate further.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/LocalTableOperations.java:
##########
@@ -27,13 +27,13 @@
 import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.junit.rules.TemporaryFolder;
 
-class LocalTableOperations implements TableOperations {
+public class LocalTableOperations implements TableOperations {

Review Comment:
   No longer public.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/RequestResponseTestBase.java:
##########
@@ -104,7 +103,7 @@ protected void assertRoundTripSerializesEquallyFrom(String json, T expected) thr
     T actual = deserialize(json);
     assertEquals(actual, expected);
 
-    // Check that the deserialized value serializes back into the original JSON
-    Assertions.assertThat(serialize(expected)).isEqualTo(json);
+    Assert.assertEquals("The serialized version of the Java record should match the original JSON",
+        serialize(expected), json);

Review Comment:
   I'll open this change in another PR (or just leave it be). But it's been reverted.



-- 
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 #5118: CORE - Load Table Response Test

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

   On reflecting on this a bit, I think writing the JSON out into test files and then parsing it is probably the ideal way to go, given we’re not testing `TableMetadata` here specifically, just `LoadTableResponse`.


-- 
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 #5118: CORE - Load Table Response Test

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

   Not sure why the Flink CI suite was cancelled after many hours, so I’m going to close and reopen to have CI run again.


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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -53,6 +53,8 @@ private LoadTableResponse(String metadataLocation, TableMetadata metadata, Map<S
 
   @Override
   public void validate() {
+    Preconditions.checkNotNull(metadata, "Invalid metadata: null");
+    Preconditions.checkNotNull(metadataLocation, "Invalid metadata location: null");

Review Comment:
   Nevermind. We've determined that `metadata-location` is not a required field. See the PR description for explanation.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -170,16 +171,20 @@ public static LoadTableResponse stageTableCreate(Catalog catalog, Namespace name
     properties.putAll(request.properties());
 
     String location;
+    String metadataLocation;
     if (request.location() != null) {
       location = request.location();
+      metadataLocation = location + "/metadata";

Review Comment:
   I'll just skip location in this case as it doesn't particularly matter as we're not making it a required field.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -106,6 +113,8 @@ public Builder addAllConfig(Map<String, String> properties) {
 
     public LoadTableResponse build() {
       Preconditions.checkNotNull(metadata, "Invalid metadata: null");
+      // This is causing errors in TestRESTCatalog via CatalogHandlers
+      // Preconditions.checkNotNull(metadataLocation, "Invalid metadata location: null");

Review Comment:
   There's a call to the builder somewhere that's used during the `CatalogTests` that is creating a `LoadTableResponse` without a metadata location.
   
   I think it's somewhere related to here https://github.com/apache/iceberg/blob/e0ff3c6e9876ce04c890908e375c876d14240c3e/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java#L160-L195, or some other create transaction that doesn't commit right away. Will investigate further.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1464,19 +1464,23 @@ components:
     LoadTableResult:
       description:
         Result used when a table is successfully loaded.
-        
-        
-        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata must be returned in the `metadata-location` field. Clients can check whether metadata has changed by comparing metadata locations.
-        
-        
+
+
+        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata should be returned in the `metadata-location` field, unless it hasn't been determined yet - such as when a transaction begins to stage a table for creation but has not commit.
+
+
+        Clients can check whether metadata has changed by comparing metadata locations after the table has been created.
+
+
         The `config` map returns table-specific configuration for the table's resources, including its HTTP client and FileIO. For example, config may contain a specific FileIO implementation class for the table depending on its underlying storage.
       type: object
       required:
-        - metadata-location
         - metadata
       properties:
         metadata-location:
           type: string
+          nullable: true
+          description: Current location of the table's metadata file. Should only be null if the table has been staged for creation as part of a transaction but does not yet exist.

Review Comment:
   ![image](https://user-images.githubusercontent.com/9833362/176279881-8218e3c1-891f-4aa5-9942-dd70be3d4068.png)
   
   Looks better with the shorter description given that the whole result has a thorough description.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1464,19 +1464,23 @@ components:
     LoadTableResult:
       description:
         Result used when a table is successfully loaded.
-        
-        
-        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata must be returned in the `metadata-location` field. Clients can check whether metadata has changed by comparing metadata locations.
-        
-        
+
+
+        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata should be returned in the `metadata-location` field, unless it hasn't been determined yet - such as when a transaction begins to stage a table for creation but has not commit.
+
+
+        Clients can check whether metadata has changed by comparing metadata locations after the table has been created.

Review Comment:
   ![image](https://user-images.githubusercontent.com/9833362/176279176-b2b9075a-ab70-4c22-b3f8-1c02430db7f0.png)
   
   Updated.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -170,16 +171,20 @@ public static LoadTableResponse stageTableCreate(Catalog catalog, Namespace name
     properties.putAll(request.properties());
 
     String location;
+    String metadataLocation;
     if (request.location() != null) {
       location = request.location();
+      metadataLocation = location + "/metadata";
     } else {
-      location = catalog.buildTable(ident, request.schema())
+      Table tbl = catalog.buildTable(ident, request.schema())

Review Comment:
   Changed to table.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/LocalTableOperations.java:
##########
@@ -27,13 +27,13 @@
 import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.junit.rules.TemporaryFolder;
 
-class LocalTableOperations implements TableOperations {
+public class LocalTableOperations implements TableOperations {
   private final TemporaryFolder temp;
   private final FileIO io;
 
   private final Map<String, String> createdMetadataFilePaths = Maps.newHashMap();
 
-  LocalTableOperations(TemporaryFolder temp) {
+  public LocalTableOperations(TemporaryFolder temp) {

Review Comment:
   Nope. Reverting now.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema SCHEMA_7 = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(SCHEMA_7).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(SCHEMA_7)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadata-location", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata =
+        TableMetadata
+            .buildFrom(
+                TableMetadata.newTableMetadata(SCHEMA_7, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+            .withMetadataLocation(TEST_METADATA_LOCATION)
+            .build();
+
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadata)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerde() throws Exception {
+    // Default fields are missing in this JSON
+    String tableMetadataV1Json = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(null, TEST_METADATA_LOCATION, tableMetadataV1Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV1));
+    LoadTableResponse resp1 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV1)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json1, resp1);
+
+    String tableMetadataV2Json = readTableMetadataInputFile("TableMetadataV2Valid.json");
+    TableMetadata metadataV2 = TableMetadataParser.fromJson(null, TEST_METADATA_LOCATION, tableMetadataV2Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json2 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV2));
+    LoadTableResponse resp2 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV2)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json2, resp2);
+  }
+
+  @Test
+  public void testCanDeserializeWithoutDefaultValues() throws Exception {
+    String metadataJson = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, metadataJson);
+
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(null, TEST_METADATA_LOCATION, metadataJson);
+    LoadTableResponse resp1 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV1)
+        .addAllConfig(CONFIG)
+        .build();
+    assertEquals(deserialize(json1), resp1);

Review Comment:
   This is the first half of what `assertRoundTripSerializesEquallyFrom` will check and the case is identical to the one above in `testRoundTripSerde`.
   
   From the test name, I think what you intended to test is that all values that can be omitted will be deserialized to defaults. Should this omit "config" from the JSON string and validate that the response is equal to the one created without calling `addAllConfig`?
   
   Also, there's no need for `json1` and `resp1` names since this only has one JSON string and one response object.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema SCHEMA_7 = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(SCHEMA_7).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(SCHEMA_7)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadata-location", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata =
+        TableMetadata
+            .buildFrom(
+                TableMetadata.newTableMetadata(SCHEMA_7, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+            .withMetadataLocation(TEST_METADATA_LOCATION)
+            .build();
+
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadata)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerde() throws Exception {
+    // Default fields are missing in this JSON
+    String tableMetadataV1Json = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(null, TEST_METADATA_LOCATION, tableMetadataV1Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV1));
+    LoadTableResponse resp1 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV1)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json1, resp1);
+
+    String tableMetadataV2Json = readTableMetadataInputFile("TableMetadataV2Valid.json");
+    TableMetadata metadataV2 = TableMetadataParser.fromJson(null, TEST_METADATA_LOCATION, tableMetadataV2Json);
+    // Convert the TableMetadata JSON from the file to an object and then back to JSON so that missing fields
+    // are filled in with their default values.
+    String json2 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, TableMetadataParser.toJson(metadataV2));
+    LoadTableResponse resp2 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV2)
+        .addAllConfig(CONFIG)
+        .build();
+    assertRoundTripSerializesEquallyFrom(json2, resp2);
+  }
+
+  @Test
+  public void testCanDeserializeWithoutDefaultValues() throws Exception {
+    String metadataJson = readTableMetadataInputFile("TableMetadataV1Valid.json");
+    String json1 = String.format(
+        "{\"metadata-location\":\"%s\",\"metadata\":%s,\"config\":{\"foo\":\"bar\"}}",
+        TEST_METADATA_LOCATION, metadataJson);
+
+    TableMetadata metadataV1 = TableMetadataParser.fromJson(null, TEST_METADATA_LOCATION, metadataJson);
+    LoadTableResponse resp1 = LoadTableResponse.builder()
+        .withTableMetadata(metadataV1)
+        .addAllConfig(CONFIG)
+        .build();
+    assertEquals(deserialize(json1), resp1);

Review Comment:
   Do let me know if you think it would be better some other way (for this test suite at least). Though I'm anxious to get it in.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/BaseSnapshot.java:
##########
@@ -32,7 +32,8 @@
 import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 
-class BaseSnapshot implements Snapshot {
+// visible for testing
+public class BaseSnapshot implements Snapshot {

Review Comment:
   Do you mean that these are in `org.apache.iceberg` and the test is in `org.apache.iceberg.rest.responses`? All of the classes are in the core module.
   
   I don't think there is a need for this change. Because we delegate to the `TableMetadataParser`, we just need to make sure that metadata makes it through correctly. We don't need to make a complicated `TableMetadata` instance. Instead, rely on `TableMetadataParser` to be correct. The metadata in should match the metadata coming out, but we don't want to do a ton of work to simply exercise `TableMetadataParser`.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
api/src/main/java/org/apache/iceberg/PartitionSpec.java:
##########
@@ -102,7 +102,8 @@ public boolean isUnpartitioned() {
     return !isPartitioned();
   }
 
-  int lastAssignedFieldId() {
+  // visible for testing
+  public int lastAssignedFieldId() {

Review Comment:
   I don't think that we need this change. This isn't something that we need to validate because it is derived from the fields for each `PartitionSpec`.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/TableMetadata.java:
##########
@@ -133,7 +133,8 @@ public static class SnapshotLogEntry implements HistoryEntry {
     private final long timestampMillis;
     private final long snapshotId;
 
-    SnapshotLogEntry(long timestampMillis, long snapshotId) {
+    // VisibleForTesting
+    public SnapshotLogEntry(long timestampMillis, long snapshotId) {

Review Comment:
   Let's just omit this from the validation.



-- 
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 closed pull request #5118: CORE - Load Table Response Test

Posted by GitBox <gi...@apache.org>.
kbendick closed pull request #5118: CORE - Load Table Response Test
URL: https://github.com/apache/iceberg/pull/5118


-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    // Add location
+    TableMetadata metadataWithLocation = TableMetadata
+        .buildFrom(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadataWithLocation)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerDe() throws JsonProcessingException {
+
+    // Build table metadata so that its uuid and last updated millis can be reused.
+    TableMetadata metaV1 = TableMetadata.buildFrom(
+        TableMetadata.newTableMetadata(
+            TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+
+    LoadTableResponse respV1 = LoadTableResponse.builder()
+        .withTableMetadata(metaV1)
+        .addAllConfig(CONFIG)
+        .build();
+
+    String fullJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +
+        "    \"format-version\":1," +
+        "    \"table-uuid\":\"" + metaV1.uuid() + "\"," +
+        "    \"location\":\"s3://bucket/test/location\"," +
+        "    \"last-updated-ms\":" + metaV1.lastUpdatedMillis() + "," +
+        "    \"last-column-id\":3," +
+        "    \"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "        {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "        {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "        {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "       ]" +
+        "    }," +
+        "  \"current-schema-id\":0," +
+        "  \"schemas\":[" +
+        "    {\"type\":\"struct\",\"schema-id\":0,\"fields\":[" +
+        "      {\"id\":1,\"name\":\"x\",\"required\":true,\"type\":\"long\"}," +
+        "      {\"id\":2,\"name\":\"y\",\"required\":true,\"type\":\"long\",\"doc\":\"comment\"}," +
+        "      {\"id\":3,\"name\":\"z\",\"required\":true,\"type\":\"long\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"partition-spec\":[]," +
+        "  \"default-spec-id\":0," +
+        "  \"partition-specs\":[{\"spec-id\":0,\"fields\":[]}]," +
+        "  \"last-partition-id\":999,\"default-sort-order-id\":1," +
+        "  \"sort-orders\":[" +
+        "    {\"order-id\":1,\"fields\":[" +
+        "        {\"transform\":\"identity\",\"source-id\":2,\"direction\":\"asc\",\"null-order\":\"nulls-first\"}," +
+        "        {\"transform\":\"bucket[4]\",\"source-id\":3,\"direction\":\"desc\",\"null-order\":\"nulls-last\"}" +
+        "      ]" +
+        "    }" +
+        "  ]," +
+        "  \"properties\":{\"owner\":\"hank\"}," +
+        "  \"current-snapshot-id\":-1," +
+        "  \"refs\":{},\"snapshots\":[],\"snapshot-log\":[],\"metadata-log\":[]},\"config\":{\"foo\":\"bar\"}}";
+
+    assertRoundTripSerializesEquallyFrom(fullJson.replaceAll("\\s", ""), respV1);
+
+    TableMetadata metaV2 = buildV2TableMetadataWithTagsAndSnapshotHistory();
+    LoadTableResponse respV2 = LoadTableResponse.builder()
+        .withTableMetadata(metaV2)
+        .addAllConfig(CONFIG)
+        .build();
+    Snapshot currentSnapshot = metaV2.snapshots().get(1);
+    Snapshot previousSnapshot = metaV2.snapshots().get(0);
+    String fullExampleInstanceJson = "{" +
+        "  \"metadataLocation\":\"s3://bucket/test/location/metadata/v1.metadata.json\"," +
+        "  \"metadata\":{" +

Review Comment:
   Updated to use `TableMetadataParser` as well as read in from existing external JSON files.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.responses;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseSnapshot;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.LocalTableOperations;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.RequestResponseTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.TestHelpers.assertSameSchemaList;
+
+public class TestLoadTableResponse extends RequestResponseTestBase<LoadTableResponse> {
+
+  private static final String TEST_METADATA_LOCATION = "s3://bucket/test/location/metadata/v1.metadata.json";
+
+  private static final String TEST_TABLE_LOCATION = "s3://bucket/test/location";
+
+  private static final Schema TEST_SCHEMA = new Schema(7,
+      Types.NestedField.required(1, "x", Types.LongType.get()),
+      Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+      Types.NestedField.required(3, "z", Types.LongType.get())
+  );
+
+  private static final PartitionSpec SPEC_5 = PartitionSpec.builderFor(TEST_SCHEMA).withSpecId(5).build();
+
+  private static final SortOrder SORT_ORDER_3 = SortOrder.builderFor(TEST_SCHEMA)
+      .withOrderId(3)
+      .asc("y", NullOrder.NULLS_FIRST)
+      .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST)
+      .build();
+
+  private static final long SEQ_NO = 34;
+  private static final int LAST_ASSIGNED_COLUMN_ID = 3;
+
+  private static final long PREVIOUS_SNAPSHOT_ID = System.currentTimeMillis() - new Random(1234).nextInt(3600);
+
+  private static final String TEST_UUIID = "01073077-d2fd-4132-b86c-09d5107e4747";
+
+  private static final Map<String, String> TABLE_PROPS = ImmutableMap.of(
+      "format-version", "1",
+      "owner", "hank");
+
+  private static final Map<String, String> CONFIG = ImmutableMap.of("foo", "bar");
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public TableOperations ops = new LocalTableOperations(temp);
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] { "metadataLocation", "metadata", "config" };
+  }
+
+  @Override
+  public LoadTableResponse createExampleInstance() {
+    TableMetadata metadata = TableMetadata.newTableMetadata(
+        TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS);
+    // Add location
+    TableMetadata metadataWithLocation = TableMetadata
+        .buildFrom(metadata)
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();
+    return LoadTableResponse.builder()
+        .withTableMetadata(metadataWithLocation)
+        .addAllConfig(CONFIG)
+        .build();
+  }
+
+  @Override
+  public LoadTableResponse deserialize(String json) throws JsonProcessingException {
+    LoadTableResponse resp = mapper().readValue(json, LoadTableResponse.class);
+    resp.validate();
+    return resp;
+  }
+
+  @Test
+  public void testFailures() {
+    AssertHelpers.assertThrows(
+        "Table metadata should be required",
+        NullPointerException.class,
+        "Invalid metadata: null",
+        () -> LoadTableResponse.builder().build());
+  }
+
+  @Test
+  public void testRoundTripSerDe() throws JsonProcessingException {
+
+    // Build table metadata so that its uuid and last updated millis can be reused.
+    TableMetadata metaV1 = TableMetadata.buildFrom(
+        TableMetadata.newTableMetadata(
+            TEST_SCHEMA, SPEC_5, SORT_ORDER_3, TEST_TABLE_LOCATION, TABLE_PROPS))
+        .withMetadataLocation(TEST_METADATA_LOCATION)
+        .build();

Review Comment:
   Given I could pull from two existing external TableMetadata.json files, I did.
   
   I could split them up between two different tests though (one for checking that round trip gives the same JSON string - i.e. missing values are not tolerated) and one for checking that missing values still round trip to the same java object.
   
   Or just keep it at one TableMetadata.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/TableMetadataParser.java:
##########
@@ -83,30 +83,30 @@ private TableMetadataParser() {
   }
 
   // visible for testing
-  static final String FORMAT_VERSION = "format-version";
-  static final String TABLE_UUID = "table-uuid";
-  static final String LOCATION = "location";
-  static final String LAST_SEQUENCE_NUMBER = "last-sequence-number";
-  static final String LAST_UPDATED_MILLIS = "last-updated-ms";
-  static final String LAST_COLUMN_ID = "last-column-id";
-  static final String SCHEMA = "schema";
-  static final String SCHEMAS = "schemas";
-  static final String CURRENT_SCHEMA_ID = "current-schema-id";
-  static final String PARTITION_SPEC = "partition-spec";
-  static final String PARTITION_SPECS = "partition-specs";
-  static final String DEFAULT_SPEC_ID = "default-spec-id";
-  static final String LAST_PARTITION_ID = "last-partition-id";
-  static final String DEFAULT_SORT_ORDER_ID = "default-sort-order-id";
-  static final String SORT_ORDERS = "sort-orders";
-  static final String PROPERTIES = "properties";
-  static final String CURRENT_SNAPSHOT_ID = "current-snapshot-id";
-  static final String REFS = "refs";
-  static final String SNAPSHOTS = "snapshots";
-  static final String SNAPSHOT_ID = "snapshot-id";
-  static final String TIMESTAMP_MS = "timestamp-ms";
-  static final String SNAPSHOT_LOG = "snapshot-log";
-  static final String METADATA_FILE = "metadata-file";
-  static final String METADATA_LOG = "metadata-log";
+  public static final String FORMAT_VERSION = "format-version";
+  public static final String TABLE_UUID = "table-uuid";
+  public static final String LOCATION = "location";
+  public static final String LAST_SEQUENCE_NUMBER = "last-sequence-number";
+  public static final String LAST_UPDATED_MILLIS = "last-updated-ms";
+  public static final String LAST_COLUMN_ID = "last-column-id";
+  public static final String SCHEMA = "schema";
+  public static final String SCHEMAS = "schemas";
+  public static final String CURRENT_SCHEMA_ID = "current-schema-id";
+  public static final String PARTITION_SPEC = "partition-spec";
+  public static final String PARTITION_SPECS = "partition-specs";
+  public static final String DEFAULT_SPEC_ID = "default-spec-id";
+  public static final String LAST_PARTITION_ID = "last-partition-id";
+  public static final String DEFAULT_SORT_ORDER_ID = "default-sort-order-id";
+  public static final String SORT_ORDERS = "sort-orders";
+  public static final String PROPERTIES = "properties";
+  public static final String CURRENT_SNAPSHOT_ID = "current-snapshot-id";
+  public static final String REFS = "refs";
+  public static final String SNAPSHOTS = "snapshots";
+  public static final String SNAPSHOT_ID = "snapshot-id";
+  public static final String TIMESTAMP_MS = "timestamp-ms";
+  public static final String SNAPSHOT_LOG = "snapshot-log";
+  public static final String METADATA_FILE = "metadata-file";
+  public static final String METADATA_LOG = "metadata-log";

Review Comment:
   Reverted all of 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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -28,6 +28,7 @@
 import org.apache.iceberg.BaseMetadataTable;
 import org.apache.iceberg.BaseTable;
 import org.apache.iceberg.BaseTransaction;
+import org.apache.iceberg.HasTableOperations;

Review Comment:
   Nope. Sorry. I had asked on here but I think it got lost. I just reverted it though.
   
   I can open those changes in another PR. They're not strictly needed for these tests (all tests still pass - but we should consider making those changes later as they are more correct that way).



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -89,8 +90,16 @@ private Builder() {
     }
 
     public Builder withTableMetadata(TableMetadata tableMetadata) {
-      this.metadataLocation = tableMetadata.metadataFileLocation();
       this.metadata = tableMetadata;
+      if (tableMetadata.metadataFileLocation() != null) {
+        this.metadataLocation = tableMetadata.metadataFileLocation();
+      }
+
+      return this;
+    }
+
+    public Builder withMetadataLocation(String metaLocation) {

Review Comment:
   Removed.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -170,16 +171,20 @@ public static LoadTableResponse stageTableCreate(Catalog catalog, Namespace name
     properties.putAll(request.properties());
 
     String location;
+    String metadataLocation;
     if (request.location() != null) {
       location = request.location();
+      metadataLocation = location + "/metadata";

Review Comment:
   Is there a particular constant you’d like me to use or to skip adding metadata location for this section where only `location` is known?



-- 
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 #5118: CORE - Load Table Response Test

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1464,19 +1464,23 @@ components:
     LoadTableResult:
       description:
         Result used when a table is successfully loaded.
-        
-        
-        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata must be returned in the `metadata-location` field. Clients can check whether metadata has changed by comparing metadata locations.
-        
-        
+
+
+        The table metadata JSON is returned in the `metadata` field. The corresponding file location of table metadata should be returned in the `metadata-location` field, unless it hasn't been determined yet - such as when a transaction begins to stage a table for creation but has not commit.
+
+
+        Clients can check whether metadata has changed by comparing metadata locations after the table has been created.

Review Comment:
   ![image](https://user-images.githubusercontent.com/9833362/176279176-b2b9075a-ab70-4c22-b3f8-1c02430db7f0.png)
   
   Updated what was requested to be part of the previous paragraph.



-- 
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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java:
##########
@@ -106,6 +113,8 @@ public Builder addAllConfig(Map<String, String> properties) {
 
     public LoadTableResponse build() {
       Preconditions.checkNotNull(metadata, "Invalid metadata: null");
+      // This is causing errors in TestRESTCatalog via CatalogHandlers
+      // Preconditions.checkNotNull(metadataLocation, "Invalid metadata location: null");

Review Comment:
   We've determined that `metadata-location` is not required (can be null), due to this very case.
   
   I've updated the spec 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 #5118: CORE - Load Table Response Test

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -189,8 +194,12 @@ public static LoadTableResponse stageTableCreate(Catalog catalog, Namespace name
         location,
         properties);
 
+    TableMetadata metadataWithLocation = TableMetadata.buildFrom(metadata)
+        .withMetadataLocation(metadataLocation)
+        .build();
+
     return LoadTableResponse.builder()
-        .withTableMetadata(metadata)
+        .withTableMetadata(metadataWithLocation)

Review Comment:
   We've determined that `metadata-location` should not be required, specifically as a `LoadTableResponse` might be returned in response to a request to stage the creation of a table (i.e. during a transaction).



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