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 2023/01/16 07:46:03 UTC

[GitHub] [iceberg] nastra commented on a diff in pull request #6598: Core: View representation core implementation

nastra commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070917805


##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.view;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.util.JsonUtil;
+
+class SQLViewRepresentationParser {
+  private static final String SQL = "sql";
+  private static final String DIALECT = "dialect";
+  private static final String SCHEMA_ID = "schema-id";
+  private static final String DEFAULT_CATALOG = "default-catalog";
+  private static final String DEFAULT_NAMESPACE = "default-namespace";
+  private static final String FIELD_ALIASES = "field-aliases";
+  private static final String FIELD_COMMENTS = "field-comments";
+
+  private SQLViewRepresentationParser() {}
+
+  static void toJson(SQLViewRepresentation view, JsonGenerator generator) throws IOException {
+    Preconditions.checkArgument(view != null, "Invalid view representation: null");
+    generator.writeStartObject();
+    generator.writeStringField(ViewRepresentationParser.TYPE, view.type());
+    generator.writeStringField(SQL, view.query());
+    generator.writeStringField(DIALECT, view.dialect());
+    generator.writeNumberField(SCHEMA_ID, view.schema().schemaId());
+
+    if (view.defaultCatalog() != null) {
+      generator.writeStringField(DEFAULT_CATALOG, view.defaultCatalog());
+    }
+
+    if (view.defaultNamespace() != null) {
+      JsonUtil.writeStringArray(
+          DEFAULT_NAMESPACE, Arrays.asList(view.defaultNamespace().levels()), generator);
+    }
+
+    if (view.fieldAliases() != null && !view.fieldAliases().isEmpty()) {
+      JsonUtil.writeStringArray(
+          SQLViewRepresentationParser.FIELD_ALIASES, view.fieldAliases(), generator);
+    }
+
+    if (view.fieldComments() != null && !view.fieldComments().isEmpty()) {
+      JsonUtil.writeStringArray(
+          SQLViewRepresentationParser.FIELD_COMMENTS, view.fieldComments(), generator);
+    }
+
+    generator.writeEndObject();
+  }
+
+  static SQLViewRepresentation fromJson(JsonNode node, List<Schema> schemas) {
+    int schemaId = JsonUtil.getInt(SCHEMA_ID, node);

Review Comment:
   does this need a null check for `node`?



##########
core/src/test/java/org/apache/iceberg/view/TestViewRepresentationParser.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.view;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import java.util.List;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestViewRepresentationParser {
+
+  @Test
+  public void testCantParseNonSqlViewRepresentation() {
+    String json = "{\"type\":\"non-sql\"}";
+    Assertions.assertThatThrownBy(() -> ViewRepresentationParser.fromJson(json, ImmutableList.of()))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot deserialize unknown view representation type non-sql");
+  }
+
+  @Test
+  public void testParseSqlViewRepresentation() {
+    String requiredFields =
+        "{\"type\":\"sql\", \"sql\": \"select * from foo\", \"schema-id\": 1, \"dialect\": \"spark-sql\"}";
+    List<Schema> schemas = ImmutableList.of(new Schema(1));
+    SQLViewRepresentation viewRepresentation =
+        ImmutableSQLViewRepresentation.builder()
+            .query("select * from foo")
+            .schema(new Schema(1))
+            .dialect("spark-sql")
+            .build();
+    validateSqlRepresentation(
+        viewRepresentation, ViewRepresentationParser.fromJson(requiredFields, schemas));
+
+    String requiredAndOptionalFields =
+        "{\"type\":\"sql\", \"sql\": \"select * from foo\", \"schema-id\": 1, \"dialect\": \"spark-sql\", "
+            + "\"default-catalog\":\"cat\", "
+            + "\"default-namespace\":[\"part1\",\"part2\"], "
+            + "\"field-aliases\":[\"col1\", \"col2\"], "
+            + "\"field-comments\":[\"Comment col1\", \"Comment col2\"]}";
+
+    SQLViewRepresentation viewWithOptionalFields =
+        ImmutableSQLViewRepresentation.builder()
+            .query("select * from foo")
+            .schema(new Schema(1))
+            .dialect("spark-sql")
+            .defaultCatalog("cat")
+            .fieldAliases(ImmutableList.of("col1", "col2"))
+            .fieldComments(ImmutableList.of("Comment col1", "Comment col2"))
+            .defaultNamespace(Namespace.of("part1", "part2"))
+            .build();
+
+    validateSqlRepresentation(
+        viewWithOptionalFields,
+        ViewRepresentationParser.fromJson(requiredAndOptionalFields, schemas));
+  }
+
+  private void validateSqlRepresentation(
+      SQLViewRepresentation expected, ViewRepresentation actual) {
+    Assert.assertEquals("Expected type sql", expected.type(), actual.type());
+
+    SQLViewRepresentation sqlViewRepresentation = (SQLViewRepresentation) actual;
+    Assert.assertEquals(
+        "Expected same query string", expected.query(), sqlViewRepresentation.query());
+    Assert.assertEquals(
+        "Expected same dialect", expected.dialect(), sqlViewRepresentation.dialect());
+    Assert.assertTrue(expected.schema().sameSchema(sqlViewRepresentation.schema()));
+    Assert.assertEquals(
+        "Expected same default namespace",
+        expected.defaultNamespace(),
+        sqlViewRepresentation.defaultNamespace());
+    Assert.assertEquals(
+        "Expected same default catalog",
+        expected.defaultCatalog(),
+        sqlViewRepresentation.defaultCatalog());
+    Assert.assertEquals(
+        "Expected same field aliases",
+        expected.fieldAliases(),
+        sqlViewRepresentation.fieldAliases());
+    Assert.assertEquals(
+        "Expected same field comments",

Review Comment:
   nit: I think mentioning that we're expecting the same XYZ is probably not needed in all of these `assertEquals()` calls



##########
api/src/main/java/org/apache/iceberg/view/SQLViewRepresentation.java:
##########
@@ -36,17 +38,21 @@ default Type type() {
   String dialect();
 
   /** The default catalog when the view is created. */
+  @Nullable
   String defaultCatalog();
 
   /** The default namespace when the view is created. */
+  @Nullable
   Namespace defaultNamespace();
 
-  /** The query output schema at version create time, without aliases. */
-  Schema schema();
+  /** The query output schema id at version create time */
+  int schemaId();
 
   /** The view field comments. */
+  @Nullable

Review Comment:
   I just wanted to mention that when making this non-nullable, then you'd have to explicitly set those to an empty list when constructing the object, as otherwise the Immutable builder will complain (since it is a required field that needs to be set). 
   Making this either nullable or non-nullable both make sense. And as @jackye1995 mentioned, there's a difference on how we treat this at the serialization layer vs the data layer



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

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

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


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