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/15 18:51:51 UTC

[GitHub] [iceberg] kbendick opened a new pull request, #5052: [CORE] Add SerDe tests for CreateTableRequest

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

   This adds tests for the serialization / deserialization of `CreateTableRequest`.
   
   This class is already being used quite a bit in other tests, specifically `CatalogTests` and `TestRESTCatalog`. It also relied pretty heavily on existing serializers inside of `RESTSerializers` for things like `Schema` etc, which simply call out to our existing parsers.
   
   Thus, I kept the tests light in terms of combinations of schemas etc as this is a combination of several tested components.


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

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

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


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


[GitHub] [iceberg] singhpk234 commented on a diff in pull request #5052: [CORE] Add SerDe tests for CreateTableRequest

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


##########
core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java:
##########
@@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.rest.requests;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PartitionSpecParser;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.SortOrderParser;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+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.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestCreateTableRequest extends RequestResponseTestBase<CreateTableRequest> {
+
+  /* Values used to fill in request fields */
+  private static final Namespace NAMESPACE = Namespace.of("accounting", "tax");
+  private static final Map<String, String> SAMPLE_PROPERTIES = ImmutableMap.of("owner", "Hank");
+  private static final Map<String, String> EMPTY_PROPERTIES = ImmutableMap.of();
+  private static final String SAMPLE_NAME = "test_tbl";
+  private static final String SAMPLE_LOCATION = "file://tmp/location/";
+  private static final Schema SAMPLE_SCHEMA = new Schema(
+      required(1, "id", Types.IntegerType.get()),
+      optional(2, "data", Types.StringType.get()));
+  private static final String SAMPLE_SCHEMA_JSON = SchemaParser.toJson(SAMPLE_SCHEMA);
+  private static final PartitionSpec SAMPLE_SPEC = PartitionSpec.builderFor(SAMPLE_SCHEMA)
+      .bucket("id", 16)
+      .build();
+  private static final SortOrder SAMPLE_WRITE_ORDER = SortOrder.builderFor(SAMPLE_SCHEMA)
+      .asc("data", NullOrder.NULLS_LAST)
+      .build();
+
+  @Test
+  // Test cases that are JSON that can be created via the Builder
+  public void testRoundTripSerDe() throws JsonProcessingException {
+    String fullJsonRaw =
+        "{\"name\":\"test_tbl\",\"location\":\"file://tmp/location/\",\"schema\":{\"type\":\"struct\"," +
+        "\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"id\",\"required\":true,\"type\":\"int\"}," +
+        "{\"id\":2,\"name\":\"data\",\"required\":false,\"type\":\"string\"}]},\"spec\":{\"spec-id\":0," +
+        "\"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}";
+
+    CreateTableRequest req = CreateTableRequest.builder()
+        .withName(SAMPLE_NAME)
+        .withLocation(SAMPLE_LOCATION)
+        .withSchema(SAMPLE_SCHEMA)
+        .setProperties(SAMPLE_PROPERTIES)
+        .withPartitionSpec(SAMPLE_SPEC)
+        .withWriteOrder(SAMPLE_WRITE_ORDER)
+        .build();
+
+    assertRoundTripSerializesEquallyFrom(fullJsonRaw, req);
+
+    // The same JSON but using existing parsers for clarity and staging the request instead of committing
+    String jsonStagedReq = String.format(
+        "{\"name\":\"%s\",\"location\":\"%s\",\"schema\":%s,\"spec\":%s," +
+            "\"order\":%s,\"properties\":%s,\"stageCreate\":%b}",
+            SAMPLE_NAME, SAMPLE_LOCATION, SchemaParser.toJson(SAMPLE_SCHEMA),
+            PartitionSpecParser.toJson(SAMPLE_SPEC.toUnbound()), SortOrderParser.toJson(SAMPLE_WRITE_ORDER.toUnbound()),
+            mapper().writeValueAsString(SAMPLE_PROPERTIES), true);
+
+    CreateTableRequest stagedReq  = CreateTableRequest.builder()

Review Comment:
   nit : extra space
   ```suggestion
       CreateTableRequest stagedReq = CreateTableRequest.builder()
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5052: [CORE] Add SerDe tests for CreateTableRequest

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/CreateTableRequest.java:
##########
@@ -43,7 +43,7 @@ public class CreateTableRequest implements RESTRequest {
   private UnboundPartitionSpec spec;
   private UnboundSortOrder order;
   private Map<String, String> properties;
-  private Boolean stageCreate;
+  private Boolean stageCreate = false;

Review Comment:
   I defaulted this to `false` so that users didn't need to explicitly set it if writing JSON. Because it's a boxed `Boolean`, if it's not present in the JSON, it gets set as `null`.
   
   I did this because `CreateTableRequest` does not have `stageCreate` in its list of required fields in the OpenAPI spec.
   
   As there is an assertion that `stageCreate` is not `null` in the `validate` method of this class, I would also be comfortable with requiring that `stageCreate` be set in the OpenAPI spec, but I prefer this as it matches the builders behavior as well (if `.stageCreate()` isn't called on the builder, `false` is used).



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5052: [CORE] Add SerDe tests for CreateTableRequest

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


##########
core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java:
##########
@@ -0,0 +1,290 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.rest.requests;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PartitionSpecParser;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.SortOrderParser;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+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.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestCreateTableRequest extends RequestResponseTestBase<CreateTableRequest> {
+
+  /* Values used to fill in request fields */
+  private static final Namespace NAMESPACE = Namespace.of("accounting", "tax");
+  private static final Map<String, String> SAMPLE_PROPERTIES = ImmutableMap.of("owner", "Hank");
+  private static final Map<String, String> EMPTY_PROPERTIES = ImmutableMap.of();
+  private static final String SAMPLE_NAME = "test_tbl";
+  private static final String SAMPLE_LOCATION = "file://tmp/location/";
+  private static final Schema SAMPLE_SCHEMA = new Schema(
+      required(1, "id", Types.IntegerType.get()),
+      optional(2, "data", Types.StringType.get()));
+  private static final String SAMPLE_SCHEMA_JSON = SchemaParser.toJson(SAMPLE_SCHEMA);
+  private static final PartitionSpec SAMPLE_SPEC = PartitionSpec.builderFor(SAMPLE_SCHEMA)
+      .bucket("id", 16)
+      .build();
+  private static final SortOrder SAMPLE_WRITE_ORDER = SortOrder.builderFor(SAMPLE_SCHEMA)
+      .asc("data", NullOrder.NULLS_LAST)
+      .build();
+
+  @Test
+  // Test cases that are JSON that can be created via the Builder
+  public void testRoundTripSerDe() throws JsonProcessingException {
+    String fullJsonRaw =
+        "{\"name\":\"test_tbl\",\"location\":\"file://tmp/location/\",\"schema\":{\"type\":\"struct\"," +
+        "\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"id\",\"required\":true,\"type\":\"int\"}," +
+        "{\"id\":2,\"name\":\"data\",\"required\":false,\"type\":\"string\"}]},\"spec\":{\"spec-id\":0," +
+        "\"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}";
+
+    CreateTableRequest req = CreateTableRequest.builder()
+        .withName(SAMPLE_NAME)
+        .withLocation(SAMPLE_LOCATION)
+        .withSchema(SAMPLE_SCHEMA)
+        .setProperties(SAMPLE_PROPERTIES)
+        .withPartitionSpec(SAMPLE_SPEC)
+        .withWriteOrder(SAMPLE_WRITE_ORDER)
+        .build();
+
+    assertRoundTripSerializesEquallyFrom(fullJsonRaw, req);
+
+    // The same JSON but using existing parsers for clarity
+    String fullJson = String.format(
+        "{\"name\":\"%s\",\"location\":\"%s\",\"schema\":%s,\"spec\":%s," +
+            "\"order\":%s,\"properties\":%s,\"stageCreate\":%b}",

Review Comment:
   For strings that continue onto the next line, my editor (with our configuration applied) wanted the next string line to be indented. I can remove that 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 #5052: [CORE] Add SerDe tests for CreateTableRequest

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


##########
core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java:
##########
@@ -0,0 +1,290 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.rest.requests;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PartitionSpecParser;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.SortOrderParser;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+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.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestCreateTableRequest extends RequestResponseTestBase<CreateTableRequest> {
+
+  /* Values used to fill in request fields */
+  private static final Namespace NAMESPACE = Namespace.of("accounting", "tax");
+  private static final Map<String, String> SAMPLE_PROPERTIES = ImmutableMap.of("owner", "Hank");
+  private static final Map<String, String> EMPTY_PROPERTIES = ImmutableMap.of();
+  private static final String SAMPLE_NAME = "test_tbl";
+  private static final String SAMPLE_LOCATION = "file://tmp/location/";
+  private static final Schema SAMPLE_SCHEMA = new Schema(
+      required(1, "id", Types.IntegerType.get()),
+      optional(2, "data", Types.StringType.get()));
+  private static final String SAMPLE_SCHEMA_JSON = SchemaParser.toJson(SAMPLE_SCHEMA);
+  private static final PartitionSpec SAMPLE_SPEC = PartitionSpec.builderFor(SAMPLE_SCHEMA)
+      .bucket("id", 16)
+      .build();
+  private static final SortOrder SAMPLE_WRITE_ORDER = SortOrder.builderFor(SAMPLE_SCHEMA)
+      .asc("data", NullOrder.NULLS_LAST)
+      .build();
+
+  @Test
+  // Test cases that are JSON that can be created via the Builder
+  public void testRoundTripSerDe() throws JsonProcessingException {
+    String fullJsonRaw =

Review Comment:
   As most of the complex fields (all of the classes we define, such as `Schema` etc) are serialized and deserialized via the existing `*Parser`, I only put one test where the whole JSON is defined.
   
   Formatting it well across lines was somewhat difficult, as the `assertRoundTripSerializesEquallyFrom` method _requires_ that the JSON matches letter for letter.
   
   I can split this out with newlines and then replace them if we'd like 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 #5052: [CORE] Add SerDe tests for CreateTableRequest

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


##########
core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java:
##########
@@ -0,0 +1,290 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.rest.requests;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PartitionSpecParser;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.SortOrderParser;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+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.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestCreateTableRequest extends RequestResponseTestBase<CreateTableRequest> {
+
+  /* Values used to fill in request fields */
+  private static final Namespace NAMESPACE = Namespace.of("accounting", "tax");
+  private static final Map<String, String> SAMPLE_PROPERTIES = ImmutableMap.of("owner", "Hank");
+  private static final Map<String, String> EMPTY_PROPERTIES = ImmutableMap.of();
+  private static final String SAMPLE_NAME = "test_tbl";
+  private static final String SAMPLE_LOCATION = "file://tmp/location/";
+  private static final Schema SAMPLE_SCHEMA = new Schema(
+      required(1, "id", Types.IntegerType.get()),
+      optional(2, "data", Types.StringType.get()));
+  private static final String SAMPLE_SCHEMA_JSON = SchemaParser.toJson(SAMPLE_SCHEMA);
+  private static final PartitionSpec SAMPLE_SPEC = PartitionSpec.builderFor(SAMPLE_SCHEMA)
+      .bucket("id", 16)
+      .build();
+  private static final SortOrder SAMPLE_WRITE_ORDER = SortOrder.builderFor(SAMPLE_SCHEMA)
+      .asc("data", NullOrder.NULLS_LAST)
+      .build();
+
+  @Test
+  // Test cases that are JSON that can be created via the Builder
+  public void testRoundTripSerDe() throws JsonProcessingException {
+    String fullJsonRaw =
+        "{\"name\":\"test_tbl\",\"location\":\"file://tmp/location/\",\"schema\":{\"type\":\"struct\"," +
+        "\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"id\",\"required\":true,\"type\":\"int\"}," +
+        "{\"id\":2,\"name\":\"data\",\"required\":false,\"type\":\"string\"}]},\"spec\":{\"spec-id\":0," +
+        "\"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}";
+
+    CreateTableRequest req = CreateTableRequest.builder()
+        .withName(SAMPLE_NAME)
+        .withLocation(SAMPLE_LOCATION)
+        .withSchema(SAMPLE_SCHEMA)
+        .setProperties(SAMPLE_PROPERTIES)
+        .withPartitionSpec(SAMPLE_SPEC)
+        .withWriteOrder(SAMPLE_WRITE_ORDER)
+        .build();
+
+    assertRoundTripSerializesEquallyFrom(fullJsonRaw, req);
+
+    // The same JSON but using existing parsers for clarity
+    String fullJson = String.format(
+        "{\"name\":\"%s\",\"location\":\"%s\",\"schema\":%s,\"spec\":%s," +
+            "\"order\":%s,\"properties\":%s,\"stageCreate\":%b}",

Review Comment:
   For strings that continue onto the next line, my editor (with our configuration applied) wanted the next string line to be indented. I can remove that though I started questioning what the style standard is 😅 



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

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

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


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


[GitHub] [iceberg] kbendick commented on pull request #5052: [CORE] Add SerDe tests for CreateTableRequest

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

   cc @danielcweeks @rdblue @Fokko @nastra and @singhpk234 who has helped contribute to some areas of the REST spec.
   
   Kept this one relatively simple given that most of the heavy lifting is done by existing Parsers.


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

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

For queries about this service, please contact Infrastructure at:
users@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 #5052: [CORE] Add SerDe tests for CreateTableRequest

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


##########
core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java:
##########
@@ -0,0 +1,290 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.rest.requests;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PartitionSpecParser;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.SortOrderParser;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+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.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestCreateTableRequest extends RequestResponseTestBase<CreateTableRequest> {
+
+  /* Values used to fill in request fields */
+  private static final Namespace NAMESPACE = Namespace.of("accounting", "tax");
+  private static final Map<String, String> SAMPLE_PROPERTIES = ImmutableMap.of("owner", "Hank");
+  private static final Map<String, String> EMPTY_PROPERTIES = ImmutableMap.of();
+  private static final String SAMPLE_NAME = "test_tbl";
+  private static final String SAMPLE_LOCATION = "file://tmp/location/";
+  private static final Schema SAMPLE_SCHEMA = new Schema(
+      required(1, "id", Types.IntegerType.get()),
+      optional(2, "data", Types.StringType.get()));
+  private static final String SAMPLE_SCHEMA_JSON = SchemaParser.toJson(SAMPLE_SCHEMA);
+  private static final PartitionSpec SAMPLE_SPEC = PartitionSpec.builderFor(SAMPLE_SCHEMA)
+      .bucket("id", 16)
+      .build();
+  private static final SortOrder SAMPLE_WRITE_ORDER = SortOrder.builderFor(SAMPLE_SCHEMA)
+      .asc("data", NullOrder.NULLS_LAST)
+      .build();
+
+  @Test
+  // Test cases that are JSON that can be created via the Builder
+  public void testRoundTripSerDe() throws JsonProcessingException {
+    String fullJsonRaw =

Review Comment:
   As most of the complex fields (all of the classes we define, such as `Schema` etc) are serialized and deserialized via the existing `*Parser`, I only put one test where the whole JSON is defined as a raw string).
   
   Formatting it well across lines was somewhat difficult, as the `assertRoundTripSerializesEquallyFrom` method _requires_ that the JSON matches letter for letter, so it can't have newlines.
   
   I can split this out with newlines and then replace them if we'd like though, or even read from an external file if need be.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5052: [CORE] Add SerDe tests for CreateTableRequest

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


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

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

For queries about this service, please contact Infrastructure at:
users@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 #5052: [CORE] Add SerDe tests for CreateTableRequest

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

   Thanks, @kbendick!


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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #5052: [CORE] Add SerDe tests for CreateTableRequest

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


##########
core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.rest.requests;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PartitionSpecParser;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.SortOrderParser;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+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.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestCreateTableRequest extends RequestResponseTestBase<CreateTableRequest> {
+
+  /* Values used to fill in request fields */
+  private static final Namespace NAMESPACE = Namespace.of("accounting", "tax");
+  private static final Map<String, String> SAMPLE_PROPERTIES = ImmutableMap.of("owner", "Hank");
+  private static final Map<String, String> EMPTY_PROPERTIES = ImmutableMap.of();
+  private static final String SAMPLE_NAME = "test_tbl";
+  private static final String SAMPLE_LOCATION = "file://tmp/location/";
+  private static final Schema SAMPLE_SCHEMA = new Schema(
+      required(1, "id", Types.IntegerType.get()),
+      optional(2, "data", Types.StringType.get()));
+  private static final String SAMPLE_SCHEMA_JSON = SchemaParser.toJson(SAMPLE_SCHEMA);
+  private static final PartitionSpec SAMPLE_SPEC = PartitionSpec.builderFor(SAMPLE_SCHEMA)
+      .bucket("id", 16)
+      .build();
+  private static final SortOrder SAMPLE_WRITE_ORDER = SortOrder.builderFor(SAMPLE_SCHEMA)
+      .asc("data", NullOrder.NULLS_LAST)
+      .build();
+
+  @Test
+  // Test cases that are JSON that can be created via the Builder
+  public void testRoundTripSerDe() throws JsonProcessingException {
+    String fullJsonRaw =
+        "{\"name\":\"test_tbl\",\"location\":\"file://tmp/location/\",\"schema\":{\"type\":\"struct\"," +
+        "\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"id\",\"required\":true,\"type\":\"int\"}," +
+        "{\"id\":2,\"name\":\"data\",\"required\":false,\"type\":\"string\"}]},\"spec\":{\"spec-id\":0," +
+        "\"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}";
+
+    CreateTableRequest req = CreateTableRequest.builder()
+        .withName(SAMPLE_NAME)
+        .withLocation(SAMPLE_LOCATION)
+        .withSchema(SAMPLE_SCHEMA)
+        .setProperties(SAMPLE_PROPERTIES)
+        .withPartitionSpec(SAMPLE_SPEC)
+        .withWriteOrder(SAMPLE_WRITE_ORDER)
+        .build();
+
+    assertRoundTripSerializesEquallyFrom(fullJsonRaw, req);
+
+    // The same JSON but using existing parsers for clarity and staging the request instead of committing
+    String jsonStagedReq = String.format(
+        "{\"name\":\"%s\",\"location\":\"%s\",\"schema\":%s,\"spec\":%s," +
+            "\"order\":%s,\"properties\":%s,\"stageCreate\":%b}",
+            SAMPLE_NAME, SAMPLE_LOCATION, SchemaParser.toJson(SAMPLE_SCHEMA),
+            PartitionSpecParser.toJson(SAMPLE_SPEC.toUnbound()), SortOrderParser.toJson(SAMPLE_WRITE_ORDER.toUnbound()),
+            mapper().writeValueAsString(SAMPLE_PROPERTIES), true);
+
+    CreateTableRequest stagedReq  = CreateTableRequest.builder()
+        .withName(SAMPLE_NAME)
+        .withLocation(SAMPLE_LOCATION)
+        .withSchema(SAMPLE_SCHEMA)
+        .setProperties(SAMPLE_PROPERTIES)
+        .withPartitionSpec(SAMPLE_SPEC)
+        .withWriteOrder(SAMPLE_WRITE_ORDER)
+        .build();
+
+    assertRoundTripSerializesEquallyFrom(jsonStagedReq, req);
+
+    // Partition spec and write order can be null or use PartitionSpec.unpartitioned() and SortOrder.unsorted()
+    String jsonWithExplicitUnsortedUnordered = String.format(
+        "{\"name\":\"%s\",\"location\":null,\"schema\":%s,\"spec\":%s," +
+            "\"order\":%s,\"properties\":{},\"stageCreate\":%b}",
+        SAMPLE_NAME, SchemaParser.toJson(SAMPLE_SCHEMA),
+        PartitionSpecParser.toJson(PartitionSpec.unpartitioned()),
+        SortOrderParser.toJson(SortOrder.unsorted().toUnbound()),
+        /* stageCreate */ false);
+
+    CreateTableRequest reqOnlyRequiredFieldsExplicitDefaults = CreateTableRequest.builder()
+        .withName(SAMPLE_NAME)
+        .withLocation(null)
+        .withSchema(SAMPLE_SCHEMA)
+        .setProperties(EMPTY_PROPERTIES)
+        .withPartitionSpec(PartitionSpec.unpartitioned())
+        .withWriteOrder(SortOrder.unsorted())
+        .build();
+
+    assertRoundTripSerializesEquallyFrom(
+        jsonWithExplicitUnsortedUnordered, reqOnlyRequiredFieldsExplicitDefaults);
+
+    String jsonOnlyRequiredFieldsNullAsDefault = String.format(
+        "{\"name\":\"%s\",\"location\":null,\"schema\":%s,\"spec\":null,\"order\":null,\"properties\":{}," +
+            "\"stageCreate\":false}",
+        SAMPLE_NAME, SchemaParser.toJson(SAMPLE_SCHEMA));
+
+    CreateTableRequest reqOnlyRequiredFieldsMissingDefaults = CreateTableRequest.builder()
+        .withName(SAMPLE_NAME)
+        .withSchema(SAMPLE_SCHEMA)
+        .withPartitionSpec(null)
+        .withWriteOrder(null)
+        .build();
+
+    assertRoundTripSerializesEquallyFrom(
+        jsonOnlyRequiredFieldsNullAsDefault, reqOnlyRequiredFieldsMissingDefaults);
+  }
+
+  @Test
+  // Test cases that can't be constructed with our Builder class but that will parse correctly
+  public void testCanDeserializeWithoutDefaultValues() throws JsonProcessingException {
+    // Name and schema are only two required fields
+    String jsonOnlyRequiredFieldsMissingDefaults = String.format(
+        "{\"name\":\"%s\",\"schema\":%s}", SAMPLE_NAME, SchemaParser.toJson(SAMPLE_SCHEMA));
+
+    CreateTableRequest reqOnlyRequiredFieldsMissingDefaults = CreateTableRequest.builder()
+        .withName(SAMPLE_NAME)
+        .withSchema(SAMPLE_SCHEMA)
+        .build();
+
+    assertEquals(deserialize(jsonOnlyRequiredFieldsMissingDefaults), reqOnlyRequiredFieldsMissingDefaults);
+  }
+
+  @Test
+  public void testDeserializeInvalidRequest() {
+    String jsonMissingSchema =
+        "{\"name\":\"foo\",\"location\":null,\"spec\":null,\"order\":null,\"properties\":{},\"stageCreate\":false}";
+    AssertHelpers.assertThrows(
+        "A JSON request with the keys spelled incorrectly should fail to deserialize and validate",
+        IllegalArgumentException.class,
+        "Invalid schema: null",
+        () -> deserialize(jsonMissingSchema)
+    );
+
+    String jsonMissingName = String.format(
+        "{\"location\":null,\"schema\":%s,\"spec\":null,\"order\":null,\"properties\":{}," +
+            "\"stageCreate\":false}", SAMPLE_SCHEMA_JSON);
+    AssertHelpers.assertThrows(
+        "A JSON request with the keys spelled incorrectly should fail to deserialize and validate",
+        IllegalArgumentException.class,
+        "Invalid table name: null",
+        () -> deserialize(jsonMissingName)
+    );
+
+    String jsonIncorrectTypeForProperties = String.format(
+        "{\"name\":\"foo\",\"location\":null,\"schema\":%s,\"spec\":null,\"order\":null,\"properties\":[]," +
+            "\"stageCreate\":false}", SAMPLE_SCHEMA_JSON);
+    AssertHelpers.assertThrows(
+        "A JSON request with incorrect types for fields should fail to parse and validate",
+        JsonProcessingException.class,
+        () -> deserialize(jsonIncorrectTypeForProperties)
+    );
+
+    AssertHelpers.assertThrows(
+        "An empty JSON object should not parse into a CreateNamespaceRequest instance that passes validation",
+        IllegalArgumentException.class,
+        "Invalid table name: null",
+        () -> deserialize("{}")
+    );
+
+    AssertHelpers.assertThrows(
+        "An empty JSON request should fail to deserialize",
+        IllegalArgumentException.class,
+        () -> deserialize(null)
+    );
+  }
+
+  @Test
+  public void testBuilderDoesNotBuildInvalidRequests() {
+    AssertHelpers.assertThrows(
+        "The builder should not allow using null for the namespace",
+        NullPointerException.class,
+        "Invalid name: null",
+        () -> CreateTableRequest.builder().withName(null)
+    );
+
+    AssertHelpers.assertThrows(
+        "The builder should not allow using null for the schema",
+        NullPointerException.class,
+        "Invalid schema: null",
+        () -> CreateTableRequest.builder().withSchema(null)
+    );
+
+    AssertHelpers.assertThrows(
+        "The builder should not allow passing a null collection of properties",
+        NullPointerException.class,
+        "Invalid collection of properties: null",
+        () -> CreateTableRequest.builder().setProperties(null)
+    );
+
+    Map<String, String> mapWithNullKey = Maps.newHashMap();
+    mapWithNullKey.put(null, "hello");
+    AssertHelpers.assertThrows(
+        "The builder should not allow using null as a key in the properties to set",
+        IllegalArgumentException.class,
+        "Invalid property: null",
+        () -> CreateTableRequest.builder().setProperties(mapWithNullKey)
+    );
+
+    Map<String, String> mapWithNullValue = Maps.newHashMap();
+    mapWithNullValue.put("a", null);
+    mapWithNullValue.put("b", "b");
+    AssertHelpers.assertThrows(
+        "The builder should not allow using null as a value in the properties to set",
+        IllegalArgumentException.class,
+        "Invalid value for properties [a]: null",
+        () -> CreateTableRequest.builder().setProperties(mapWithNullValue).build()
+    );
+
+    AssertHelpers.assertThrows(
+        "The builder should not allow using null as a value when setting a single property",
+        IllegalArgumentException.class,
+        "Invalid value for property foo: null",
+        () -> CreateTableRequest.builder().setProperty("foo", null)
+    );
+
+    AssertHelpers.assertThrows(
+        "The builder should not allow using null as a key when setting a single property",
+        IllegalArgumentException.class,
+        "Invalid property: null",
+        () -> CreateTableRequest.builder().setProperty(null, "foo")
+    );
+  }
+
+  @Override
+  public String[] allFieldsFromSpec() {
+    return new String[] {"name", "location", "schema", "spec", "order", "properties", "stageCreate"};
+  }
+
+  @Override
+  public CreateTableRequest createExampleInstance() {
+    return CreateTableRequest.builder()
+        .withName(SAMPLE_NAME)
+        .withLocation(SAMPLE_LOCATION)
+        .withSchema(SAMPLE_SCHEMA)
+        .withPartitionSpec(SAMPLE_SPEC)
+        .withWriteOrder(SAMPLE_WRITE_ORDER)
+        .setProperties(SAMPLE_PROPERTIES)
+        .stageCreate()
+        .build();
+  }
+
+  @Override
+  public void assertEquals(CreateTableRequest actual, CreateTableRequest expected) {
+    Assert.assertEquals("Name should be the same", expected.name(), actual.name());
+    Assert.assertEquals("Location should be the same if provided", expected.location(), actual.location());
+    Assert.assertTrue("Schemas should be equivalent and have same schema id",
+        expected.schema().sameSchema(actual.schema()) && expected.schema().schemaId() == actual.schema().schemaId());

Review Comment:
   I originally had this as two assertions (one for `sameSchema` and one for equality checking on `schemaId`).
   
   If we prefer, I can revert to 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