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/15 21:52:21 UTC

[GitHub] [iceberg] amogh-jahagirdar opened a new pull request, #6598: Core: View representation core implementation

amogh-jahagirdar opened a new pull request, #6598:
URL: https://github.com/apache/iceberg/pull/6598

    View representation core implementation
    Co-authored-by: John Zhuge <jz...@apache.org>


-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1081669003


##########
core/src/test/java/org/apache/iceberg/view/TestViewRepresentationParser.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 {

Review Comment:
   Sure 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] jackye1995 commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1088457431


##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewRepresentationParser {
+  static final String TYPE = "type";
+
+  private ViewRepresentationParser() {}
+
+  static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    Preconditions.checkArgument(representation != null, "Invalid view representation: null");
+    switch (representation.type()) {
+      case ViewRepresentation.Type.SQL:
+        SQLViewRepresentationParser.toJson((SQLViewRepresentation) representation, generator);
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot serialize view representation type: %s", representation.type()));
+    }
+  }
+
+  static String toJson(ViewRepresentation entry) {
+    return JsonUtil.generate(gen -> toJson(entry, gen), false);
+  }
+
+  static ViewRepresentation fromJson(String json) {
+    return JsonUtil.parse(json, ViewRepresentationParser::fromJson);
+  }
+
+  static ViewRepresentation fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view representation from null object");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse view representation from non-object: %s", node);
+    String type = JsonUtil.getString(TYPE, node).toLowerCase(Locale.ENGLISH);
+    switch (type) {
+      case "sql":
+        return SQLViewRepresentationParser.fromJson(node);
+
+      default:
+        return ImmutableUnknownViewRepresentation.builder().type(type).build();

Review Comment:
   can we log an warning message 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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1072871952


##########
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();

Review Comment:
   @jackye1995 exactly my thoughts as well, a schema is known at the time of SQL representation creation so late binding like we do with PartitionSpec doesn't apply or would needlessly complicate the logic of constructing the representation. Will hold for @nastra and @rdblue thought as well



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070698774


##########
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();

Review Comment:
   Since we have the flexibility to change this API until core implementation is complete, I changed the API to schemaID instead of schema. This is mostly because it simplifies the parsing logic and then it's still easy for a caller to obtain the schema from the top level view schema mapping.
   
   Currently in the view spec, the schema ID is stored per SQL view representation. So in the current implementation building the representation when parsing is straightforward. 
   
   There are a few routes to go here:
   
   1.) Maintain schema ID in metadata and the API just returns schema ID as well (keeping parsing logic simple). That's what's done in this PR and I think is preferable since looking up schema via view.schemas().get(schemaID) should be straightforward.
   
   2.) Preserve the schema at the API level, maintain schema ID in metadata. This complicates parsing logic (although not too much) because during parsing we need to pass the top level schemas list to SQLViewRepresentationParser https://iceberg.apache.org/view-spec/#view-metadata and then obtain the schema based on the parsed schema ID.
   
   3.) Update the spec so that the entire schema object is stored in metadata and then serialize/deserialize.
   
   Another topic (regardless of option 1 or 3) is the spec currently marks schemaID as optional for sql view representation. I think it must be required (I can't think of a case where for a SQL representation we don't want to maintain a well defined Iceberg schema, engines can still choose to ignore it if they want to although I can't really think of such a case). I can raise a PR to update that if we think it's the right approach. 
   
   cc: @jzhuge @rdblue @jackye1995 @nastra 
   
   Let me know your thoughts on which approach above you find preferable and if we agree schema should be updated to be required for SQL view representation in 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 #6598: Core: View representation core implementation

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1092354135


##########
api/src/main/java/org/apache/iceberg/view/ViewRepresentation.java:
##########
@@ -18,21 +18,16 @@
  */
 package org.apache.iceberg.view;
 
-import java.util.Locale;
+import org.immutables.value.Value;
 
+@Value.Immutable
 public interface ViewRepresentation {
 
-  enum Type {
-    SQL;
+  class Type {
+    private Type() {}
 
-    public static Type fromString(String typeName) {
-      return valueOf(typeName.toUpperCase(Locale.ENGLISH));
-    }
-
-    public String typeName() {
-      return name().toLowerCase(Locale.ENGLISH);
-    }
+    public static final String SQL = "sql";

Review Comment:
   I think it depends on whether we want to use this enum in switch statements in our code or if we want to extend it. For example, we could have a reference to the parser in the enum so we look up the symbol and then call something like `ViewRepresentation.SQL.parse(jsonNode)`.
   
   Since we only have the parser selection right now, it doesn't seem like it matters much.



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1081672308


##########
api/src/main/java/org/apache/iceberg/view/ViewRepresentation.java:
##########
@@ -18,21 +18,16 @@
  */
 package org.apache.iceberg.view;
 
-import java.util.Locale;
+import org.immutables.value.Value;
 
+@Value.Immutable
 public interface ViewRepresentation {
 
-  enum Type {
-    SQL;
+  class Type {
+    private Type() {}
 
-    public static Type fromString(String typeName) {
-      return valueOf(typeName.toUpperCase(Locale.ENGLISH));
-    }
-
-    public String typeName() {
-      return name().toLowerCase(Locale.ENGLISH);
-    }
+    public static final String SQL = "sql";

Review Comment:
   Yeah it seems like an established pattern elsewhere just to use a constant string and it simplifies the parsing logic a bit but I'm happy to revert back to enum if there's other advantages. Let me know your thoughts @rdblue  @jzhuge 



-- 
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] yyanyy commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "yyanyy (via GitHub)" <gi...@apache.org>.
yyanyy commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1082942731


##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.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 String toJson(SQLViewRepresentation sqlViewRepresentation) {
+    return JsonUtil.generate(gen -> toJson(sqlViewRepresentation, gen), false);
+  }
+
+  static void toJson(SQLViewRepresentation view, JsonGenerator generator) throws IOException {
+    Preconditions.checkArgument(view != null, "Invalid SQL view representation: null");
+    generator.writeStartObject();
+    generator.writeStringField(ViewRepresentationParser.TYPE, view.type());
+    generator.writeStringField(SQL, view.sql());
+    generator.writeStringField(DIALECT, view.dialect());
+
+    if (view.schemaId() != null) {
+      generator.writeNumberField(SCHEMA_ID, view.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()) {

Review Comment:
   nit: if we decide to use nullable annotation, sounds like `fieldAliases` and `fieldComments` can be null since we do null check here; should we mark them as nullable in `SQLViewRepresentation`?



-- 
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] jackye1995 commented on a diff in pull request #6598: Core: View representation core implementation

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


##########
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();

Review Comment:
   I see. After taking a deeper look, there are multiple places that we have `schemaId()` as a part of the API model, such as in `Snapshot`. 
   
   I was using the case of `PartitionSpec` to argue that we should stick with using `schema()`, but it was because there is a process of binding to a schema, and the serialized version of partition spec is unbounded.
   
   So the question here is not which one is more convenient to implement parser, but does a view representation needs to bind to a schema at runtime, or only have a static schema. I think the answer is that it is static, as described as "ID of the view’s schema when the version was created". So from that perspective, I agree `schemaId()` seems like a better choice so we don't need to cross reference existing schemas to make the parser work. 
   
   Any thoughts? @amogh-jahagirdar @nastra 



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1085985352


##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewRepresentationParser {
+  static final String TYPE = "type";
+
+  private ViewRepresentationParser() {}
+
+  static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    Preconditions.checkArgument(representation != null, "Invalid view representation: null");
+    switch (representation.type()) {
+      case ViewRepresentation.Type.SQL:
+        SQLViewRepresentationParser.toJson((SQLViewRepresentation) representation, generator);
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Cannot serialize unknown view representation type %s", representation.type()));
+    }
+  }
+
+  static String toJson(ViewRepresentation entry) {
+    return JsonUtil.generate(gen -> toJson(entry, gen), false);
+  }
+
+  static ViewRepresentation fromJson(String json) {
+    return JsonUtil.parse(json, ViewRepresentationParser::fromJson);
+  }
+
+  static ViewRepresentation fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view representation from null object");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse view representation from non-object: %s", node);
+    String type = JsonUtil.getString(TYPE, node).toLowerCase(Locale.ENGLISH);
+    switch (type) {
+      case "sql":
+        return SQLViewRepresentationParser.fromJson(node);
+
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot deserialize unknown view representation type %s", type));

Review Comment:
   This makes sense after reading how UnknownTransform works. I've updated the PR so that when reading, any unknown representation is deserialized into an UnknownViewRepresentation with a String type. When serializing the metadata we will still fail. This will effectively make the View read only if there's a single unknown representation and we can update later to serialize if desired



-- 
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] jackye1995 commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1088423077


##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.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 String toJson(SQLViewRepresentation sqlViewRepresentation) {
+    return JsonUtil.generate(gen -> toJson(sqlViewRepresentation, gen), false);
+  }
+
+  static void toJson(SQLViewRepresentation view, JsonGenerator generator) throws IOException {
+    Preconditions.checkArgument(view != null, "Invalid SQL view representation: null");
+    generator.writeStartObject();
+    generator.writeStringField(ViewRepresentationParser.TYPE, view.type());
+    generator.writeStringField(SQL, view.sql());
+    generator.writeStringField(DIALECT, view.dialect());
+
+    if (view.schemaId() != null) {
+      generator.writeNumberField(SCHEMA_ID, view.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().isEmpty()) {
+      JsonUtil.writeStringArray(
+          SQLViewRepresentationParser.FIELD_ALIASES, view.fieldAliases(), generator);
+    }
+
+    if (!view.fieldComments().isEmpty()) {
+      JsonUtil.writeStringArray(
+          SQLViewRepresentationParser.FIELD_COMMENTS, view.fieldComments(), generator);
+    }
+
+    generator.writeEndObject();
+  }
+
+  static SQLViewRepresentation fromJson(String json) {
+    Preconditions.checkArgument(

Review Comment:
   Do we check for null JSON string for other parsers? If not then we don't really need to worry about this case. Because it's parsing string, which is mostly for testing purpose, I don't think we need to worry about the error message too much. 



-- 
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] jackye1995 commented on a diff in pull request #6598: Core: View representation core implementation

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


##########
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();

Review Comment:
   I see. After taking a deeper look, there are multiple places that we have `schemaId()` as a part of the API model, such as in `Snapshot`. 
   
   I was using the case of `PartitionSpec` to argue that we should stick with using `schema()`, but it was because there is a process of binding to a schema, and the serialized version of partition spec is unbounded.
   
   So the question here is not which one is more convenient to implement parser, but if a view representation needs to bind to a schema at runtime, or have a static schema. I think the answer is that it is static, as described as "ID of the view’s schema when the version was created". So from that perspective, I agree `schemaId()` seems like a better choice so we don't need to cross reference existing schemas to make the parser work. 
   
   Any thoughts? @amogh-jahagirdar @nastra 



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1096290654


##########
api/src/main/java/org/apache/iceberg/view/SQLViewRepresentation.java:
##########
@@ -18,14 +18,17 @@
  */
 package org.apache.iceberg.view;
 
+import edu.umd.cs.findbugs.annotations.Nullable;

Review Comment:
   Makes sense. There's still a few other places in the Iceberg code base which are using the umd.cs instead of javax. It make sense to use the javax one, it looks like the umd.cs one is deprecated? I'll create a tracking issue so that we can update to use `javax` in the remaining parts of the code



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

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

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


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


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

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1088481012


##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewRepresentationParser {
+  static final String TYPE = "type";
+
+  private ViewRepresentationParser() {}
+
+  static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    Preconditions.checkArgument(representation != null, "Invalid view representation: null");
+    switch (representation.type()) {
+      case ViewRepresentation.Type.SQL:
+        SQLViewRepresentationParser.toJson((SQLViewRepresentation) representation, generator);
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot serialize view representation type: %s", representation.type()));
+    }
+  }
+
+  static String toJson(ViewRepresentation entry) {
+    return JsonUtil.generate(gen -> toJson(entry, gen), false);
+  }
+
+  static ViewRepresentation fromJson(String json) {
+    return JsonUtil.parse(json, ViewRepresentationParser::fromJson);
+  }
+
+  static ViewRepresentation fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view representation from null object");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse view representation from non-object: %s", node);
+    String type = JsonUtil.getString(TYPE, node).toLowerCase(Locale.ENGLISH);
+    switch (type) {
+      case "sql":
+        return SQLViewRepresentationParser.fromJson(node);
+
+      default:
+        return ImmutableUnknownViewRepresentation.builder().type(type).build();

Review Comment:
   I was checking this, if we consider this similar to the [UnknownTransform](https://github.com/apache/iceberg/blob/master/api/src/main/java/org/apache/iceberg/transforms/UnknownTransform.java#L28) then it's never really treated as a warning case, and also it seems like Iceberg doesn't do any logging in parsers (although we don't necessarily need to follow that here). 
   
   I was thinking if we do want to consider this as a warning case we may want to do it in the higher level metadata parser. After reading the representation list field, we would go record by record and if the read result is an instance of UnknownViewrepresentation than we can log there.
   
   But overall, I'm not sure if we even want to treat this as a warning case in the parser level? I think engines integrating with the view spec would do hard validations like uhttps://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/Partitioning.java#L261 which will ultimately fail if there's unknown representations and get surfaced to a user . let me know your thoughts 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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1088481596


##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewRepresentationParser {
+  static final String TYPE = "type";
+
+  private ViewRepresentationParser() {}
+
+  static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    Preconditions.checkArgument(representation != null, "Invalid view representation: null");
+    switch (representation.type()) {
+      case ViewRepresentation.Type.SQL:
+        SQLViewRepresentationParser.toJson((SQLViewRepresentation) representation, generator);
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot serialize view representation type: %s", representation.type()));
+    }
+  }
+
+  static String toJson(ViewRepresentation entry) {
+    return JsonUtil.generate(gen -> toJson(entry, gen), false);
+  }
+
+  static ViewRepresentation fromJson(String json) {
+    return JsonUtil.parse(json, ViewRepresentationParser::fromJson);
+  }
+
+  static ViewRepresentation fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view representation from null object");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse view representation from non-object: %s", node);
+    String type = JsonUtil.getString(TYPE, node).toLowerCase(Locale.ENGLISH);
+    switch (type) {
+      case "sql":

Review Comment:
   good catch, i missed 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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070722634


##########
api/src/main/java/org/apache/iceberg/view/ViewRepresentation.java:
##########
@@ -19,14 +19,21 @@
 package org.apache.iceberg.view;
 
 import java.util.Locale;
+import org.immutables.value.Value;
 
+@Value.Immutable
 public interface ViewRepresentation {
 
   enum Type {

Review Comment:
   Agreed, see the discussion here on this topic of enums in the original API PR https://github.com/apache/iceberg/pull/4925/files#r1015167642
   
   I think as long as we have well defined and documented type strings that should be good (we get well defined handling of different representation types and a simpler code base). I'll update to use strings and we can get feedback from the community



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070762329


##########
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();

Review Comment:
   After implementing it, while it's doable it does seem to complicate more. I'll see about simplifying it, but we may just want to go back to maintaining schema IDs or even serializing the entire schema to keep parsing really simple. For just using schema ID, then engines can select their representation and then do view.schema(repr.schemaId()) 



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070738898


##########
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()));

Review Comment:
   This is one awkward part if we maintain just schema, since schema doesn't have a notion of .equals() we compare using sameSchema which just ignores schemaId and compared the columns/types. I don't think this should be the case. We should always compare equality on the basis of schema ID. 
   
   To do this I could extend SQLViewRepresentation with a custom equalsTo but that would add more complexity.



-- 
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] nastra commented on a diff in pull request #6598: Core: View representation core implementation

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


##########
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:
   yes exactly, I mainly had a use case in mind where some downstream consumer would eventually use this parser independently from `ViewRepresentationParser`. 



-- 
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] jackye1995 commented on a diff in pull request #6598: Core: View representation core implementation

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


##########
api/src/main/java/org/apache/iceberg/view/ViewRepresentation.java:
##########
@@ -18,21 +18,16 @@
  */
 package org.apache.iceberg.view;
 
-import java.util.Locale;
+import org.immutables.value.Value;
 
+@Value.Immutable
 public interface ViewRepresentation {
 
-  enum Type {
-    SQL;
+  class Type {
+    private Type() {}
 
-    public static Type fromString(String typeName) {
-      return valueOf(typeName.toUpperCase(Locale.ENGLISH));
-    }
-
-    public String typeName() {
-      return name().toLowerCase(Locale.ENGLISH);
-    }
+    public static final String SQL = "sql";

Review Comment:
   I made the suggestion since I see some other enum like classes are also implemented directly as strings, such as `DataOperations`, and it seems to simplify the code a bit. But if there is a specific reason for using enum we can stick 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


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

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


##########
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:
   @amogh-jahagirdar ah that's true, I forgot that Immutable inits collections to be empty by default. Therefore, no need to have `@Nullable` or any other annotation on those collection fields. Thanks for double-checking 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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070738205


##########
core/src/main/java/org/apache/iceberg/util/JsonUtil.java:
##########
@@ -98,6 +103,14 @@ public static <T> T parse(String json, FromJson<T> parser) {
     }
   }
 
+  public static <T, A> T parseWithArg(String json, FromJsonWithArgs<T, A> parser, A args) {
+    try {
+      return parser.parse(JsonUtil.mapper().readValue(json, JsonNode.class), args);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+

Review Comment:
   Trying to think of a better abstraction so we don't need this and the FromJsonWithArgs



-- 
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] nastra commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1086786399


##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.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 String toJson(SQLViewRepresentation sqlViewRepresentation) {
+    return JsonUtil.generate(gen -> toJson(sqlViewRepresentation, gen), false);
+  }
+
+  static void toJson(SQLViewRepresentation view, JsonGenerator generator) throws IOException {
+    Preconditions.checkArgument(view != null, "Invalid SQL view representation: null");
+    generator.writeStartObject();
+    generator.writeStringField(ViewRepresentationParser.TYPE, view.type());
+    generator.writeStringField(SQL, view.sql());
+    generator.writeStringField(DIALECT, view.dialect());
+
+    if (view.schemaId() != null) {
+      generator.writeNumberField(SCHEMA_ID, view.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().isEmpty()) {
+      JsonUtil.writeStringArray(
+          SQLViewRepresentationParser.FIELD_ALIASES, view.fieldAliases(), generator);
+    }
+
+    if (!view.fieldComments().isEmpty()) {
+      JsonUtil.writeStringArray(
+          SQLViewRepresentationParser.FIELD_COMMENTS, view.fieldComments(), generator);
+    }
+
+    generator.writeEndObject();
+  }
+
+  static SQLViewRepresentation fromJson(String json) {
+    Preconditions.checkArgument(

Review Comment:
   nit: I think this Precondition can be removed, since it will always hit the one `static SQLViewRepresentation fromJson(JsonNode node)`, so no need to repeat here



##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewRepresentationParser {
+  static final String TYPE = "type";
+
+  private ViewRepresentationParser() {}
+
+  static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    Preconditions.checkArgument(representation != null, "Invalid view representation: null");
+    switch (representation.type()) {
+      case ViewRepresentation.Type.SQL:
+        SQLViewRepresentationParser.toJson((SQLViewRepresentation) representation, generator);
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Cannot serialize unknown view representation type %s", representation.type()));
+    }
+  }
+
+  static String toJson(ViewRepresentation entry) {
+    return JsonUtil.generate(gen -> toJson(entry, gen), false);
+  }
+
+  static ViewRepresentation fromJson(String json) {
+    Preconditions.checkArgument(json != null, "Cannot parse view representation from null object");

Review Comment:
   the check in this method can be removed then (it seems it's still there)



##########
core/src/main/java/org/apache/iceberg/view/UnknownViewRepresentation.java:
##########
@@ -0,0 +1,27 @@
+/*
+ * 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 org.immutables.value.Value;
+
+@Value.Immutable
+public interface UnknownViewRepresentation extends ViewRepresentation {
+  @Override

Review Comment:
   no need to override  `type()` here to make this work. This will automatically require anything that's also required from `ViewRepresentation` during instantiation, which is `type()`



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1088432686


##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.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 String toJson(SQLViewRepresentation sqlViewRepresentation) {
+    return JsonUtil.generate(gen -> toJson(sqlViewRepresentation, gen), false);
+  }
+
+  static void toJson(SQLViewRepresentation view, JsonGenerator generator) throws IOException {
+    Preconditions.checkArgument(view != null, "Invalid SQL view representation: null");
+    generator.writeStartObject();
+    generator.writeStringField(ViewRepresentationParser.TYPE, view.type());
+    generator.writeStringField(SQL, view.sql());
+    generator.writeStringField(DIALECT, view.dialect());
+
+    if (view.schemaId() != null) {
+      generator.writeNumberField(SCHEMA_ID, view.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().isEmpty()) {
+      JsonUtil.writeStringArray(
+          SQLViewRepresentationParser.FIELD_ALIASES, view.fieldAliases(), generator);
+    }
+
+    if (!view.fieldComments().isEmpty()) {
+      JsonUtil.writeStringArray(
+          SQLViewRepresentationParser.FIELD_COMMENTS, view.fieldComments(), generator);
+    }
+
+    generator.writeEndObject();
+  }
+
+  static SQLViewRepresentation fromJson(String json) {
+    Preconditions.checkArgument(

Review Comment:
   It's mixed across the parsers for example for [SnapshotRefParser](https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/SnapshotRefParser.java#L63)  we do. For [ViewHistoryEntryParser we don't](https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/view/ViewHistoryEntryParser.java#L47)
   
   If I look at all though, the majority don't do this check. Also considering that in practice in the ViewMetadataParser we will use JsonUtil.get as we loop over the representation entries, and that already has a clear error message for null cases so in practice we won't need this. So, I'll remove 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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070698774


##########
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();

Review Comment:
   Since we have the flexibility to change this API until core implementation is complete, I changed the API to schemaID instead of schema. This is mostly because it simplifies the parsing logic and then it's still easy for a caller to obtain the schema from the top level view schema mapping.
   
   Currently in the view spec, the schema ID is stored per SQL view representation. So in the current implementation building the representation when parsing is straightforward. 
   
   There are a few routes to go here:
   
   1.) Maintain schema ID in metadata and the API just returns schema ID as well (keeping parsing logic simple). That's what's done in this PR and I think is preferable since looking up schema via view.schemas().get(schemaID) should be straightforward.
   
   2.) Preserve the schema at the API level, maintain schema ID in metadata. This complicates parsing logic (although not too much) because during parsing we need to pass the top level schemas list to SQLViewRepresentationParser https://iceberg.apache.org/view-spec/#view-metadata and then obtain the schema based on the parsed schema ID.
   
   3.) Update the spec so that the entire schema object is stored in metadata and then serialize/deserialize.
   
   Another topic (independent of which option we choose) is the spec currently marks schemaID as optional for sql view representation. I think it must be required (I can't think of a case where for a SQL representation we don't want to maintain a well defined Iceberg schema, engines can still choose to ignore it if they want to although I can't really think of such a case). I can raise a PR to update that if we think it's the right approach. 
   
   cc: @jzhuge @rdblue @jackye1995 @nastra 
   
   Let me know your thoughts on which approach above you find preferable and if we agree schema should be updated to be required for SQL view representation in 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] jackye1995 commented on a diff in pull request #6598: Core: View representation core implementation

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


##########
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 think there is a difference in the data model vs the serialization strategy, yes it does not need to be serialized, but in that case we can still return an empty list instead of null for the data model



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1071424411


##########
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:
   Sure, I've updated so that we just won't serialize if null or empty for field aliases and field comments but at the API level we return an empty list. I think default catalog and namespace should still be nullable. 
   
   One benefit is that we just avoid callers having to do null checks (and they would have to do empty list checks anyways) so it simplifies for callers. 
   
   @nastra I thought that this was the case, but it seems like for collections, Immutables will initialize an empty collection in the builder state. So it seems not required to explicitly set those (if it's never set, it ends up being an empty collection). let me know if you have any concern on not setting these explicitly! 



-- 
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] nastra commented on a diff in pull request #6598: Core: View representation core implementation

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


##########
core/src/test/java/org/apache/iceberg/view/TestViewRepresentationParser.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 {

Review Comment:
   @amogh-jahagirdar I think main testing should be in `TestSQLViewRepresentationParser` (since technically the `SQLViewRepresentationParser` could be used independently). Then the `TestViewRepresentationParser` would only focus on nullability/empty json and the `type` field being set/unset



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1081672308


##########
api/src/main/java/org/apache/iceberg/view/ViewRepresentation.java:
##########
@@ -18,21 +18,16 @@
  */
 package org.apache.iceberg.view;
 
-import java.util.Locale;
+import org.immutables.value.Value;
 
+@Value.Immutable
 public interface ViewRepresentation {
 
-  enum Type {
-    SQL;
+  class Type {
+    private Type() {}
 
-    public static Type fromString(String typeName) {
-      return valueOf(typeName.toUpperCase(Locale.ENGLISH));
-    }
-
-    public String typeName() {
-      return name().toLowerCase(Locale.ENGLISH);
-    }
+    public static final String SQL = "sql";

Review Comment:
   Yeah it seems like an established pattern elsewhere just to use a constant string and it simplifies the parsing logic a bit but I'm happy to revert back to enum if there's other advantages. Let me know your thoughts @rdblue  



-- 
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 #6598: Core: View representation core implementation

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1084415330


##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewRepresentationParser {
+  static final String TYPE = "type";
+
+  private ViewRepresentationParser() {}
+
+  static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    Preconditions.checkArgument(representation != null, "Invalid view representation: null");
+    switch (representation.type()) {
+      case ViewRepresentation.Type.SQL:
+        SQLViewRepresentationParser.toJson((SQLViewRepresentation) representation, generator);
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Cannot serialize unknown view representation type %s", representation.type()));
+    }
+  }
+
+  static String toJson(ViewRepresentation entry) {
+    return JsonUtil.generate(gen -> toJson(entry, gen), false);
+  }
+
+  static ViewRepresentation fromJson(String json) {
+    return JsonUtil.parse(json, ViewRepresentationParser::fromJson);
+  }
+
+  static ViewRepresentation fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view representation from null object");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse view representation from non-object: %s", node);
+    String type = JsonUtil.getString(TYPE, node).toLowerCase(Locale.ENGLISH);
+    switch (type) {
+      case "sql":
+        return SQLViewRepresentationParser.fromJson(node);
+
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot deserialize unknown view representation type %s", type));

Review Comment:
   Unknown representations should not cause a failure. Instead, I think that we want to have an `UnknownRepresentation` (similar to `UnknownTransform`). That way, we can still read a view that has a SQL representation and another representation and this is forward-compatible.
   
   If there is an `UnknownRepresentation` then the view should be read-only because we don't want to drop or corrupt representations. (We could also keep the `JsonNode` around and reserialize it, but it's easier for now to make the view read-only.)



-- 
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] nastra commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1086786399


##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.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 String toJson(SQLViewRepresentation sqlViewRepresentation) {
+    return JsonUtil.generate(gen -> toJson(sqlViewRepresentation, gen), false);
+  }
+
+  static void toJson(SQLViewRepresentation view, JsonGenerator generator) throws IOException {
+    Preconditions.checkArgument(view != null, "Invalid SQL view representation: null");
+    generator.writeStartObject();
+    generator.writeStringField(ViewRepresentationParser.TYPE, view.type());
+    generator.writeStringField(SQL, view.sql());
+    generator.writeStringField(DIALECT, view.dialect());
+
+    if (view.schemaId() != null) {
+      generator.writeNumberField(SCHEMA_ID, view.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().isEmpty()) {
+      JsonUtil.writeStringArray(
+          SQLViewRepresentationParser.FIELD_ALIASES, view.fieldAliases(), generator);
+    }
+
+    if (!view.fieldComments().isEmpty()) {
+      JsonUtil.writeStringArray(
+          SQLViewRepresentationParser.FIELD_COMMENTS, view.fieldComments(), generator);
+    }
+
+    generator.writeEndObject();
+  }
+
+  static SQLViewRepresentation fromJson(String json) {
+    Preconditions.checkArgument(

Review Comment:
   nit: I think this Precondition can be removed, since it will always hit the one in `static SQLViewRepresentation fromJson(JsonNode node)`, so no need to repeat 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] jackye1995 merged pull request #6598: Core: View representation core implementation

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 merged PR #6598:
URL: https://github.com/apache/iceberg/pull/6598


-- 
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] amogh-jahagirdar commented on pull request #6598: Core: View representation core implementation

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#issuecomment-1427158205

   No problem! Let me raise a PR to address the banned dependency, sorry about 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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070754570


##########
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:
   Yeah field comments/aliases/default catalog are optional fields. According to the table spec https://iceberg.apache.org/spec/#writer-requirements optional fields may be written but it's not required to be serialized, so just opted for maintaining that optional fields can just be null at the API level. 
   
   Do you see any benefits to serializing empty lists/empty strings for the optional fields? 



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1072871952


##########
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();

Review Comment:
   @jackye1995 exactly my thoughts as well, a schema is known at the time of view creation so late binding like we do with PartitionSpec doesn't apply or would needlessly complicate the logic of constructing the representation. Will hold for @nastra and @rdblue thought as well



-- 
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] nastra commented on a diff in pull request #6598: Core: View representation core implementation

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


##########
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();

Review Comment:
   After reading through the discussion I think just having `schemaId` makes a lot of sense to me



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1071414863


##########
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:
   If it was null it would have already been caught in ViewRepresentationParser, but we can add the check here as well so SQLViewRepresentationParser can stand alone properly.



##########
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:
   If it was null it would have already been caught in ViewRepresentationParser, but we should add the check here as well so SQLViewRepresentationParser can stand alone properly.



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070698774


##########
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();

Review Comment:
   Since we have the flexibility to change this API until core implementation is complete, I changed the API to schemaID instead of schema. This is mostly because it simplifies the parsing logic and then it's still easy for a caller to obtain the schema from the top level view schema mapping.
   
   Currently in the view spec, the schema ID is stored per SQL view representation. So in the current implementation building the representation when parsing is straightforward. 
   
   There are a few options here:
   
   1.) Maintain schema ID in metadata and the API just returns schema ID as well (keeping parsing logic simple). That's what's done in this PR and I think is preferable since looking up schema via view.schemas().get(schemaID) should be straightforward.
   
   2.) Preserve the schema at the API level, maintain schema ID in metadata. This complicates parsing logic (although not too much) because during parsing we need to pass the top level schemas list to SQLViewRepresentationParser https://iceberg.apache.org/view-spec/#view-metadata and then obtain the schema based on the parsed schema ID.
   
   3.) Update the spec so that the entire schema object is stored in metadata and then serialize/deserialize.
   
   Another topic (independent of which option we choose) is the spec currently marks schemaID as optional for sql view representation. I think it must be required (I can't think of a case where for a SQL representation we don't want to maintain a well defined Iceberg schema, engines can still choose to ignore it if they want to although I can't really think of such a case). I can raise a PR to update that if we think it's the right approach. 
   
   cc: @jzhuge @rdblue @jackye1995 @nastra 
   
   Let me know your thoughts on which approach above you find preferable and if we agree schema should be updated to be required for SQL view representation in 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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070738898


##########
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()));

Review Comment:
   This is one awkward part if we maintain just schema, since schema doesn't override .equals() we compare using sameSchema which just ignores schemaId and compared the columns/types. I don't think this should be the case. We should always compare equality on the basis of schema ID. 
   
   To do this I could extend SQLViewRepresentation with a custom equalsTo but that would add more complexity.



-- 
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] jackye1995 commented on pull request #6598: Core: View representation core implementation

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#issuecomment-1423068671

   @rdblue @danielcweeks the PR looks good to be merged, do you have any additional comment?


-- 
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] nastra commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1086812953


##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewRepresentationParser {
+  static final String TYPE = "type";
+
+  private ViewRepresentationParser() {}
+
+  static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    Preconditions.checkArgument(representation != null, "Invalid view representation: null");
+    switch (representation.type()) {
+      case ViewRepresentation.Type.SQL:
+        SQLViewRepresentationParser.toJson((SQLViewRepresentation) representation, generator);
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Cannot serialize view representation type: %s", representation.type()));
+    }
+  }
+
+  static String toJson(ViewRepresentation entry) {
+    return JsonUtil.generate(gen -> toJson(entry, gen), false);
+  }
+
+  static ViewRepresentation fromJson(String json) {
+    Preconditions.checkArgument(json != null, "Cannot parse view representation from null object");

Review Comment:
   can be removed, since it will always hit the Precondition from `static ViewRepresentation fromJson(JsonNode node)`



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1088481012


##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewRepresentationParser {
+  static final String TYPE = "type";
+
+  private ViewRepresentationParser() {}
+
+  static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    Preconditions.checkArgument(representation != null, "Invalid view representation: null");
+    switch (representation.type()) {
+      case ViewRepresentation.Type.SQL:
+        SQLViewRepresentationParser.toJson((SQLViewRepresentation) representation, generator);
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot serialize view representation type: %s", representation.type()));
+    }
+  }
+
+  static String toJson(ViewRepresentation entry) {
+    return JsonUtil.generate(gen -> toJson(entry, gen), false);
+  }
+
+  static ViewRepresentation fromJson(String json) {
+    return JsonUtil.parse(json, ViewRepresentationParser::fromJson);
+  }
+
+  static ViewRepresentation fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view representation from null object");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse view representation from non-object: %s", node);
+    String type = JsonUtil.getString(TYPE, node).toLowerCase(Locale.ENGLISH);
+    switch (type) {
+      case "sql":
+        return SQLViewRepresentationParser.fromJson(node);
+
+      default:
+        return ImmutableUnknownViewRepresentation.builder().type(type).build();

Review Comment:
   I was checking this, if we consider this similar to the [UnknownTransform](https://github.com/apache/iceberg/blob/master/api/src/main/java/org/apache/iceberg/transforms/UnknownTransform.java#L28) then it's never really treated as a warning case, and also it seems like Iceberg doesn't do any logging in parsers (although we don't necessarily need to follow that here). 
   
   I was thinking if we do want to consider this as a warning case we may want to do it in the higher level metadata parser. After reading the representation list field, we would go record by record and if the read result is an instance of UnknownViewrepresentation than we can log there.
   
   But overall, I'm not sure if we even want to treat this as a warning case in the parser level? I think engines integrating with the view spec would do hard validations like https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/Partitioning.java#L261 which will ultimately fail if there's unknown representations and get surfaced to a user . let me know your thoughts 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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1088320961


##########
core/src/main/java/org/apache/iceberg/view/UnknownViewRepresentation.java:
##########
@@ -0,0 +1,27 @@
+/*
+ * 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 org.immutables.value.Value;
+
+@Value.Immutable
+public interface UnknownViewRepresentation extends ViewRepresentation {
+  @Override

Review Comment:
   Oh nice, didn't know this, 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] danielcweeks commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "danielcweeks (via GitHub)" <gi...@apache.org>.
danielcweeks commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1096201824


##########
api/src/main/java/org/apache/iceberg/view/SQLViewRepresentation.java:
##########
@@ -18,14 +18,16 @@
  */
 package org.apache.iceberg.view;
 
+import edu.umd.cs.findbugs.annotations.Nullable;

Review Comment:
   Can we use `javax.annotation.Nullable` instead?



##########
api/src/main/java/org/apache/iceberg/view/ViewRepresentation.java:
##########
@@ -18,21 +18,16 @@
  */
 package org.apache.iceberg.view;
 
-import java.util.Locale;
+import org.immutables.value.Value;

Review Comment:
   Same here: `javax.annotation.Nullable`



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070754570


##########
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:
   Yeah field comments/aliases/default catalog are optional fields. According to the table spec https://iceberg.apache.org/spec/#writer-requirements optional fields may be written but it's not required to be serialized, so just opted for maintaining that optional fields can just be null at the API level and we don't serialize them if they are null or empty. 
   
   Do you see any benefits to serializing empty lists/empty strings for the optional fields? 



-- 
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] jackye1995 commented on a diff in pull request #6598: Core: View representation core implementation

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


##########
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:
   is this really nullable? Or if null it should be an empty list



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070698774


##########
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();

Review Comment:
   Since we have the flexibility to change this API until core implementation is complete, I changed the API to schemaID instead of schema. This is mostly because it simplifies the parsing logic and then it's still easy for a caller to obtain the schema from the top level view schema mapping.
   
   Currently in the view spec, the schema ID is stored per SQL view representation. So in the current implementation building the representation when parsing is straightforward. 
   
   There are a few routes to go here:
   
   1.) Maintain schema ID in metadata and the API just returns schema ID as well (keeping parsing logic simple). That's what's done in this PR and I think is preferable since looking up schema via view.schemas().get(schemaID) should be straightforward.
   
   2.) Preserve the schema at the API level, maintain schema ID in metadata. This complicates parsing logic (although not too much) because during parsing we need to pass the top level schemas list to SQLViewRepresentationParser https://iceberg.apache.org/view-spec/#view-metadata and then obtain the schema based on the parsed schema ID.
   
   3.) Update the spec so that the entire schema object is stored in metadata and then serialize/deserialize.
   
   Another topic (regardless of option 1 or 3) is the spec currently marks schemaID as optional for sql view representation. I think it must be required (I can't think of a case where for a SQL representation we don't want to maintain a well defined Iceberg schema, engines can still choose to ignore it if they want to although I can't really think of such a case). I can raise a PR to update that if we think it's the right approach. 
   
   cc: @jzhuge @rdblue @jackye1995 @nastra 
   
   
   
   



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070762329


##########
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();

Review Comment:
   After implementing it, while it's doable it does seem to complicate more. I'll see about simplifying it, but we may just want to go back to surfacing schema IDs at the data model level or even serializing the entire schema to keep parsing really simple. For just using schema ID, then engines can select their representation and then do view.schema(repr.schemaId()) 



-- 
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] nastra commented on a diff in pull request #6598: Core: View representation core implementation

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


##########
api/src/main/java/org/apache/iceberg/view/SQLViewRepresentation.java:
##########
@@ -18,14 +18,17 @@
  */
 package org.apache.iceberg.view;
 
+import edu.umd.cs.findbugs.annotations.Nullable;

Review Comment:
   since `SQLViewRepresentation` is an Immutable, this indicates to the builder that certain fields are not required and can indeed be null when an instance is constructed



-- 
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] amogh-jahagirdar commented on pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#issuecomment-1397753619

   We probably want to establish a standard in the community at this point on Immutable/Nullable or not. Right now we're in this partial state, where it's used in some cases but defining a standard can help. I do think Immutables are really nice at keeping boiler plate code to a minimum but I don't have a strong opinion other than just setting a standard practice in Iceberg :) 


-- 
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 #6598: Core: View representation core implementation

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


##########
api/src/main/java/org/apache/iceberg/view/ViewRepresentation.java:
##########
@@ -18,21 +18,16 @@
  */
 package org.apache.iceberg.view;
 
-import java.util.Locale;
+import org.immutables.value.Value;
 
+@Value.Immutable
 public interface ViewRepresentation {
 
-  enum Type {
-    SQL;
+  class Type {
+    private Type() {}
 
-    public static Type fromString(String typeName) {
-      return valueOf(typeName.toUpperCase(Locale.ENGLISH));
-    }
-
-    public String typeName() {
-      return name().toLowerCase(Locale.ENGLISH);
-    }
+    public static final String SQL = "sql";

Review Comment:
   Why change this from an enum to a String?



-- 
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 #6598: Core: View representation core implementation

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


##########
api/src/main/java/org/apache/iceberg/view/SQLViewRepresentation.java:
##########
@@ -18,14 +18,17 @@
  */
 package org.apache.iceberg.view;
 
+import edu.umd.cs.findbugs.annotations.Nullable;

Review Comment:
   Can you remove this? We don't use nullable annotations.



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070719355


##########
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();

Review Comment:
   Thanks @jackye1995 yeah that's achievable, that's approach 2 above. It does complicate parsing more than I'd like but it's very doable. I'll update the PR and the community can take a look and we can see which one we prefer more.



##########
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();

Review Comment:
   Thanks @jackye1995 yeah that's achievable, that's what I meant in approach 2 above. It does complicate parsing more than I'd like but it's very doable. I'll update the PR and the community can take a look and we can see which one we prefer more.



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070758730


##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.List;
+import java.util.Locale;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewRepresentationParser {
+  static final String TYPE = "type";
+
+  private ViewRepresentationParser() {}
+
+  static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    Preconditions.checkArgument(representation != null, "Invalid view representation: null");
+    switch (representation.type()) {
+      case ViewRepresentation.Type.SQL:
+        SQLViewRepresentationParser.toJson((SQLViewRepresentation) representation, generator);
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Cannot serialize unknown view representation type %s", representation.type()));
+    }
+  }
+
+  static String toJson(ViewRepresentation entry) {
+    return JsonUtil.generate(gen -> toJson(entry, gen), false);
+  }
+
+  static ViewRepresentation fromJson(String json, List<Schema> schemas) {
+    return JsonUtil.parseWithArg(json, ViewRepresentationParser::fromJson, schemas);
+  }

Review Comment:
   What would happen in this model is that ViewMetadataParser would parse the list of schemas first and then pass that here. But trying to think of ways to simplify this. We may just want to go back to only having schema IDs for the representation API. Or even just serialize the entire schema alongside the representation. That should simplify a lot.



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070762329


##########
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();

Review Comment:
   After implementing it, while it's doable it does seem to complicate more. I'll see about simplifying it, but we may just want to go back to maintaining schema IDs or even serializing the entire schema to keep parsing really simple. and then engines can just do view.schema(repr.schemaId()) 



-- 
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] jackye1995 commented on a diff in pull request #6598: Core: View representation core implementation

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


##########
api/src/main/java/org/apache/iceberg/view/ViewRepresentation.java:
##########
@@ -19,14 +19,21 @@
 package org.apache.iceberg.view;
 
 import java.util.Locale;
+import org.immutables.value.Value;
 
+@Value.Immutable
 public interface ViewRepresentation {
 
   enum Type {

Review Comment:
   can we potentially avoid the enum type and directly use string? That seems to be able to simplify things a bit. (this should probably be asked in the API PR, but I guess it's still not too late).



-- 
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] nastra commented on a diff in pull request #6598: Core: View representation core implementation

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


##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.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 String toJson(SQLViewRepresentation sqlViewRepresentation) {
+    return JsonUtil.generate(gen -> toJson(sqlViewRepresentation, gen), false);
+  }
+
+  static void toJson(SQLViewRepresentation view, JsonGenerator generator) throws IOException {
+    Preconditions.checkArgument(view != null, "Invalid SQL view representation: null");
+    generator.writeStartObject();
+    generator.writeStringField(ViewRepresentationParser.TYPE, view.type());
+    generator.writeStringField(SQL, view.sql());
+    generator.writeStringField(DIALECT, view.dialect());
+
+    if (view.schemaId() != null) {
+      generator.writeNumberField(SCHEMA_ID, view.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(String json) {
+    Preconditions.checkArgument(
+        json != null, "Cannot parse SQL view representation from null JSON");
+    return JsonUtil.parse(json, SQLViewRepresentationParser::fromJson);
+  }
+
+  static SQLViewRepresentation fromJson(JsonNode node) {
+    Preconditions.checkArgument(

Review Comment:
   nit: it would probably be good to also add Preconditions.checkArgument(node.isObject(), "Cannot parse SQL view representation from non-object: %s", node);`, similary to how you have it in the other parser



##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewRepresentationParser {
+  static final String TYPE = "type";
+
+  private ViewRepresentationParser() {}
+
+  static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    Preconditions.checkArgument(representation != null, "Invalid view representation: null");
+    switch (representation.type()) {
+      case ViewRepresentation.Type.SQL:
+        SQLViewRepresentationParser.toJson((SQLViewRepresentation) representation, generator);
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Cannot serialize unknown view representation type %s", representation.type()));
+    }
+  }
+
+  static String toJson(ViewRepresentation entry) {
+    return JsonUtil.generate(gen -> toJson(entry, gen), false);
+  }
+
+  static ViewRepresentation fromJson(String json) {
+    Preconditions.checkArgument(json != null, "Cannot parse view representation from null object");

Review Comment:
   I think this check here isn't necessary (same in the other parser) because via `JsonUtil.parse(..)` you'll end up in `fromJson(JsonNode node)` anyway and those checks will be performed, so no need to perform the check at two different places



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1084397362


##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.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 String toJson(SQLViewRepresentation sqlViewRepresentation) {
+    return JsonUtil.generate(gen -> toJson(sqlViewRepresentation, gen), false);
+  }
+
+  static void toJson(SQLViewRepresentation view, JsonGenerator generator) throws IOException {
+    Preconditions.checkArgument(view != null, "Invalid SQL view representation: null");
+    generator.writeStartObject();
+    generator.writeStringField(ViewRepresentationParser.TYPE, view.type());
+    generator.writeStringField(SQL, view.sql());
+    generator.writeStringField(DIALECT, view.dialect());
+
+    if (view.schemaId() != null) {
+      generator.writeNumberField(SCHEMA_ID, view.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()) {

Review Comment:
   See https://github.com/apache/iceberg/pull/6598#discussion_r1070708186 , there's pro's and cons for either leaving it be null at the data model level or just in serialization. In the end we just opted for having an empty list at the data model layer (and a client doesn't have to do a null check). So here one possible cleanup is not do the null check anyways! 



##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.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 String toJson(SQLViewRepresentation sqlViewRepresentation) {
+    return JsonUtil.generate(gen -> toJson(sqlViewRepresentation, gen), false);
+  }
+
+  static void toJson(SQLViewRepresentation view, JsonGenerator generator) throws IOException {
+    Preconditions.checkArgument(view != null, "Invalid SQL view representation: null");
+    generator.writeStartObject();
+    generator.writeStringField(ViewRepresentationParser.TYPE, view.type());
+    generator.writeStringField(SQL, view.sql());
+    generator.writeStringField(DIALECT, view.dialect());
+
+    if (view.schemaId() != null) {
+      generator.writeNumberField(SCHEMA_ID, view.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()) {

Review Comment:
   See https://github.com/apache/iceberg/pull/6598#discussion_r1070708186 , there's pro's and cons for either leaving it be null at the data model level or just in serialization. In the end we just opted for having an empty list at the data model layer (and a client doesn't have to do a null check). So here one possible cleanup is not do the null check anyways since we have the guarantee at the API level! 



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1088432686


##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.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 String toJson(SQLViewRepresentation sqlViewRepresentation) {
+    return JsonUtil.generate(gen -> toJson(sqlViewRepresentation, gen), false);
+  }
+
+  static void toJson(SQLViewRepresentation view, JsonGenerator generator) throws IOException {
+    Preconditions.checkArgument(view != null, "Invalid SQL view representation: null");
+    generator.writeStartObject();
+    generator.writeStringField(ViewRepresentationParser.TYPE, view.type());
+    generator.writeStringField(SQL, view.sql());
+    generator.writeStringField(DIALECT, view.dialect());
+
+    if (view.schemaId() != null) {
+      generator.writeNumberField(SCHEMA_ID, view.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().isEmpty()) {
+      JsonUtil.writeStringArray(
+          SQLViewRepresentationParser.FIELD_ALIASES, view.fieldAliases(), generator);
+    }
+
+    if (!view.fieldComments().isEmpty()) {
+      JsonUtil.writeStringArray(
+          SQLViewRepresentationParser.FIELD_COMMENTS, view.fieldComments(), generator);
+    }
+
+    generator.writeEndObject();
+  }
+
+  static SQLViewRepresentation fromJson(String json) {
+    Preconditions.checkArgument(

Review Comment:
   It's mixed across the parsers for example for [SnapshotRefParser](https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/SnapshotRefParser.java#L63)  we do. For [ViewHistoryEntryParser we don't](https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/view/ViewHistoryEntryParser.java#L47)
   
   If I look at all though, the majority don't do this check. Also considering that in practice in the ViewMetadataParser we will use JsonUtil.get as we loop over the representation entries, and that already has a clear error message for null so in practice we won't need this. So, I'll remove 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] danielcweeks commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "danielcweeks (via GitHub)" <gi...@apache.org>.
danielcweeks commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1096213570


##########
api/src/main/java/org/apache/iceberg/view/SQLViewRepresentation.java:
##########
@@ -18,14 +18,17 @@
  */
 package org.apache.iceberg.view;
 
+import edu.umd.cs.findbugs.annotations.Nullable;

Review Comment:
   Nullable annotations have been used in puffin and metrics, but I do think we should standardize on `javax.annotation.Nullable` as opposed to this findbugs version if we are going to be explicit about nullability.



-- 
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] jackye1995 commented on a diff in pull request #6598: Core: View representation core implementation

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


##########
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();

Review Comment:
   I think there are still ways to use Schema, and ideally that is preferred from API perspective. For parser, similar to PartitionSpec parser, I would imagine the parser to take the schemas and use that. Is the same strategy achieveable 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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1070722634


##########
api/src/main/java/org/apache/iceberg/view/ViewRepresentation.java:
##########
@@ -19,14 +19,21 @@
 package org.apache.iceberg.view;
 
 import java.util.Locale;
+import org.immutables.value.Value;
 
+@Value.Immutable
 public interface ViewRepresentation {
 
   enum Type {

Review Comment:
   Agreed, see the discussion here on this topic of enums in the original API PR https://github.com/apache/iceberg/pull/4925/files#r1015167642
   
   I think as long as we have well defined and documented type strings that should be acceptable. 



##########
api/src/main/java/org/apache/iceberg/view/ViewRepresentation.java:
##########
@@ -19,14 +19,21 @@
 package org.apache.iceberg.view;
 
 import java.util.Locale;
+import org.immutables.value.Value;
 
+@Value.Immutable
 public interface ViewRepresentation {
 
   enum Type {

Review Comment:
   Agreed, see the discussion here on this topic of enums in the original API PR https://github.com/apache/iceberg/pull/4925/files#r1015167642
   
   I think as long as we have well defined and documented type strings that should be acceptable. I'll update to use strings and we can get feedback from the community



-- 
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] nastra commented on a diff in pull request #6598: Core: View representation core implementation

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1081676028


##########
api/src/main/java/org/apache/iceberg/view/SQLViewRepresentation.java:
##########
@@ -18,14 +18,17 @@
  */
 package org.apache.iceberg.view;
 
+import edu.umd.cs.findbugs.annotations.Nullable;

Review Comment:
   It seems like we are using Nullable in a few places already, for example https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/metrics/CommitMetricsResult.java#L54 . I'm good to revert this but that would imply not using Immutables as well here (unless there's another acceptable way to indicate to Immutable values that a field can be null).



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

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

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


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


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

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1081678527


##########
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();

Review Comment:
   Updated to use Integer schemaId(). Since schemaId may not be defined at creation time as discussed in https://github.com/apache/iceberg/pull/6611/files schemaId() can return 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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1096287950


##########
api/src/main/java/org/apache/iceberg/view/ViewRepresentation.java:
##########
@@ -18,21 +18,16 @@
  */
 package org.apache.iceberg.view;
 
-import java.util.Locale;
+import org.immutables.value.Value;

Review Comment:
   ViewRepresentations currently has a single required "type" field so it wasn't marked as Nullable to begin with. Let me know if I'm missing something! 



##########
api/src/main/java/org/apache/iceberg/view/SQLViewRepresentation.java:
##########
@@ -18,14 +18,16 @@
  */
 package org.apache.iceberg.view;
 
+import edu.umd.cs.findbugs.annotations.Nullable;

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] jackye1995 commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1088457279


##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewRepresentationParser {
+  static final String TYPE = "type";
+
+  private ViewRepresentationParser() {}
+
+  static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    Preconditions.checkArgument(representation != null, "Invalid view representation: null");
+    switch (representation.type()) {
+      case ViewRepresentation.Type.SQL:
+        SQLViewRepresentationParser.toJson((SQLViewRepresentation) representation, generator);
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot serialize view representation type: %s", representation.type()));
+    }
+  }
+
+  static String toJson(ViewRepresentation entry) {
+    return JsonUtil.generate(gen -> toJson(entry, gen), false);
+  }
+
+  static ViewRepresentation fromJson(String json) {
+    return JsonUtil.parse(json, ViewRepresentationParser::fromJson);
+  }
+
+  static ViewRepresentation fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view representation from null object");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse view representation from non-object: %s", node);
+    String type = JsonUtil.getString(TYPE, node).toLowerCase(Locale.ENGLISH);
+    switch (type) {
+      case "sql":

Review Comment:
   should use the `Type.SQL` variable



-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1088290116


##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.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 String toJson(SQLViewRepresentation sqlViewRepresentation) {
+    return JsonUtil.generate(gen -> toJson(sqlViewRepresentation, gen), false);
+  }
+
+  static void toJson(SQLViewRepresentation view, JsonGenerator generator) throws IOException {
+    Preconditions.checkArgument(view != null, "Invalid SQL view representation: null");
+    generator.writeStartObject();
+    generator.writeStringField(ViewRepresentationParser.TYPE, view.type());
+    generator.writeStringField(SQL, view.sql());
+    generator.writeStringField(DIALECT, view.dialect());
+
+    if (view.schemaId() != null) {
+      generator.writeNumberField(SCHEMA_ID, view.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().isEmpty()) {
+      JsonUtil.writeStringArray(
+          SQLViewRepresentationParser.FIELD_ALIASES, view.fieldAliases(), generator);
+    }
+
+    if (!view.fieldComments().isEmpty()) {
+      JsonUtil.writeStringArray(
+          SQLViewRepresentationParser.FIELD_COMMENTS, view.fieldComments(), generator);
+    }
+
+    generator.writeEndObject();
+  }
+
+  static SQLViewRepresentation fromJson(String json) {
+    Preconditions.checkArgument(

Review Comment:
   It ends up failing as an invalid argument to JsonUtil.parse, in specifically `JsonUtil.mapper().readValue(json, JsonNode.class)` and the error message becomes less clear so I added this back. Let me know if you have any concerns 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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1088290778


##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewRepresentationParser {
+  static final String TYPE = "type";
+
+  private ViewRepresentationParser() {}
+
+  static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    Preconditions.checkArgument(representation != null, "Invalid view representation: null");
+    switch (representation.type()) {
+      case ViewRepresentation.Type.SQL:
+        SQLViewRepresentationParser.toJson((SQLViewRepresentation) representation, generator);
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Cannot serialize view representation type: %s", representation.type()));
+    }
+  }
+
+  static String toJson(ViewRepresentation entry) {
+    return JsonUtil.generate(gen -> toJson(entry, gen), false);
+  }
+
+  static ViewRepresentation fromJson(String json) {
+    Preconditions.checkArgument(json != null, "Cannot parse view representation from null object");

Review Comment:
   Same as above It ends up failing as an invalid argument to JsonUtil.parse, in specifically JsonUtil.mapper().readValue(json, JsonNode.class) and the error message becomes less clear so I added this back. Let me know if you have any concerns 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] jackye1995 commented on pull request #6598: Core: View representation core implementation

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#issuecomment-1425003848

   Seems like we don't have much movement on the review at this point, given the fact that all current comments are addressed and this is a part of many PRs for view catalog integration, I will go ahead to merge this. We can address any remaining comments in #6559 if any. Thanks @amogh-jahagirdar and @jzhuge for the work and thanks everyone for review!


-- 
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] amogh-jahagirdar commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1084395003


##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewRepresentationParser {
+  static final String TYPE = "type";
+
+  private ViewRepresentationParser() {}
+
+  static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    Preconditions.checkArgument(representation != null, "Invalid view representation: null");
+    switch (representation.type()) {
+      case ViewRepresentation.Type.SQL:
+        SQLViewRepresentationParser.toJson((SQLViewRepresentation) representation, generator);
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Cannot serialize unknown view representation type %s", representation.type()));
+    }
+  }
+
+  static String toJson(ViewRepresentation entry) {
+    return JsonUtil.generate(gen -> toJson(entry, gen), false);
+  }
+
+  static ViewRepresentation fromJson(String json) {
+    Preconditions.checkArgument(json != null, "Cannot parse view representation from null object");

Review Comment:
   Updated!



##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.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 String toJson(SQLViewRepresentation sqlViewRepresentation) {
+    return JsonUtil.generate(gen -> toJson(sqlViewRepresentation, gen), false);
+  }
+
+  static void toJson(SQLViewRepresentation view, JsonGenerator generator) throws IOException {
+    Preconditions.checkArgument(view != null, "Invalid SQL view representation: null");
+    generator.writeStartObject();
+    generator.writeStringField(ViewRepresentationParser.TYPE, view.type());
+    generator.writeStringField(SQL, view.sql());
+    generator.writeStringField(DIALECT, view.dialect());
+
+    if (view.schemaId() != null) {
+      generator.writeNumberField(SCHEMA_ID, view.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(String json) {
+    Preconditions.checkArgument(
+        json != null, "Cannot parse SQL view representation from null JSON");
+    return JsonUtil.parse(json, SQLViewRepresentationParser::fromJson);
+  }
+
+  static SQLViewRepresentation fromJson(JsonNode node) {
+    Preconditions.checkArgument(

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] rdblue commented on a diff in pull request #6598: Core: View representation core implementation

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6598:
URL: https://github.com/apache/iceberg/pull/6598#discussion_r1103888557


##########
versions.props:
##########
@@ -8,6 +8,7 @@ org.apache.orc:* = 1.8.2
 org.apache.parquet:* = 1.12.3
 org.apache.pig:pig = 0.14.0
 com.fasterxml.jackson.*:* = 2.14.1
+com.google.code.findbugs:jsr305 = 3.0.2

Review Comment:
   @jackye1995 and @amogh-jahagirdar, this should be a banned dependency that is replaced by stephenc's reimplementation. This is a 1.2.0 release blocker.



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