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 23:18:25 UTC

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

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