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/11 02:44:23 UTC

[GitHub] [iceberg] amogh-jahagirdar opened a new pull request, #6559: Core: View core parser implementations

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

   Co-authored-by: John Zhuge <jz...@apache.org>
   
   Discussed offline with @jzhuge,  builds on the core parser changes already done on https://github.com/apache/iceberg/pull/4657 but based off the latest API changes. 


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

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

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


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


[GitHub] [iceberg] nastra commented on a diff in pull request #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {
+    return ImmutableViewMetadata.builder()
+        .formatVersion(DEFAULT_VIEW_FORMAT_VERSION)
+        .location(metadata.location())
+        .properties(metadata.properties())
+        .currentVersionId(metadata.currentVersionId())
+        .versions(metadata.versions())
+        .schemas(metadata.schemas())
+        .currentSchemaId(metadata.currentSchemaId())
+        .history(metadata.history());
+  }
+
+  @Value.Check
+  void check() {
+    Preconditions.checkState(versions().size() > 0, "Expecting non-empty version log");
+    checkVersionLog(history());
+  }
+
+  public abstract int formatVersion();
+
+  public abstract String location();
+
+  public abstract Map<String, String> properties();
+
+  public abstract int currentVersionId();
+
+  public abstract List<ViewVersion> versions();
+
+  public abstract List<ViewHistoryEntry> history();
+
+  @Nullable
+  public abstract List<Schema> schemas();
+
+  @Nullable
+  public abstract Integer currentSchemaId();
+
+  public ViewVersion version(int versionId) {
+    return versionsById().get(versionId);
+  }
+
+  @Value.Derived
+  public ViewVersion currentVersion() {
+    return versionsById().get(currentVersionId());
+  }
+
+  @Value.Derived
+  public Map<Integer, ViewVersion> versionsById() {
+    return indexVersions(versions());
+  }
+
+  @Value.Derived
+  public Map<Integer, Schema> schemasById() {
+    return indexSchemas(schemas());
+  }
+
+  @Value.Derived
+  public Schema schema() {
+    return schemasById().get(currentSchemaId());
+  }
+
+  private static void checkVersionLog(List<ViewHistoryEntry> versionLog) {
+    Preconditions.checkState(versionLog.size() > 0, "Expecting non-empty version log");

Review Comment:
   otherwise the error msg might get confused for the one from L51



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(String.format("Failed to read json file: %s", file), e);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);
+
+    String location = JsonUtil.getString(LOCATION, node);
+    int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, node);
+    Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, node);
+
+    JsonNode versionsListNode = JsonUtil.get(VERSIONS, node);
+    Preconditions.checkArgument(
+        versionsListNode.isArray(), "Cannot parse versions from non-array: %s", versionsListNode);
+
+    List<ViewVersion> versions = Lists.newArrayListWithExpectedSize(versionsListNode.size());
+    ViewVersion currentVersion = null;
+    for (JsonNode versionNode : versionsListNode) {
+      ViewVersion version = ViewVersionParser.fromJson(versionNode);
+      if (version.versionId() == currentVersionId) {
+        currentVersion = version;
+      }
+
+      versions.add(version);
+    }
+
+    Preconditions.checkArgument(
+        currentVersion != null,
+        "Cannot find version with %s=%s from %s",
+        CURRENT_VERSION_ID,
+        currentVersionId,
+        VERSIONS);
+
+    JsonNode versionLogNode = JsonUtil.get(VERSION_LOG, node);
+    Preconditions.checkArgument(
+        versionLogNode.isArray(), "Cannot parse version-log from non-array: %s", versionLogNode);
+    List<ViewHistoryEntry> historyEntries =
+        Lists.newArrayListWithExpectedSize(versionLogNode.size());
+    Iterator<JsonNode> versionLogIterator = versionLogNode.elements();
+    while (versionLogIterator.hasNext()) {
+      historyEntries.add(ViewHistoryEntryParser.fromJson(versionLogIterator.next()));
+    }
+
+    List<Schema> schemas;
+    Integer currentSchemaId;
+    Schema currentSchema = null;
+    JsonNode schemaArray = node.get(SCHEMAS);
+
+    Preconditions.checkArgument(
+        schemaArray.isArray(), "Cannot parse schemas from non-array: %s", schemaArray);
+    currentSchemaId = JsonUtil.getInt(CURRENT_SCHEMA_ID, node);
+
+    ImmutableList.Builder<Schema> builder = ImmutableList.builder();
+    for (JsonNode schemaNode : schemaArray) {
+      Schema schema = SchemaParser.fromJson(schemaNode);
+      if (schema.schemaId() == currentSchemaId) {
+        currentSchema = schema;
+      }
+
+      builder.add(schema);
+    }
+
+    Preconditions.checkArgument(
+        currentSchema != null,
+        "Cannot find schema with %s=%s from %s",
+        CURRENT_SCHEMA_ID,
+        currentSchemaId,
+        SCHEMAS);
+
+    schemas = builder.build();
+
+    int numVersionsToKeep =
+        PropertyUtil.propertyAsInt(
+            properties,
+            ViewProperties.VERSION_HISTORY_SIZE,
+            ViewProperties.VERSION_HISTORY_SIZE_DEFAULT);
+
+    Preconditions.checkArgument(
+        numVersionsToKeep >= 1, "%s must be positive", ViewProperties.VERSION_HISTORY_SIZE);
+
+    if (versions.size() > numVersionsToKeep) {
+      versions = versions.subList(versions.size() - numVersionsToKeep, versions.size());
+      historyEntries =
+          historyEntries.subList(historyEntries.size() - numVersionsToKeep, historyEntries.size());
+    }
+
+    return ImmutableViewMetadata.builder()
+        .location(location)
+        .currentVersionId(currentVersionId)
+        .properties(properties)
+        .versions(versions)
+        .schemas(schemas)
+        .currentSchemaId(currentSchemaId)
+        .history(historyEntries)
+        .formatVersion(formatVersion)
+        .build();
+  }
+
+  static ViewMetadata fromJson(String json) {

Review Comment:
   why this is not public?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {
+    return ImmutableViewMetadata.builder()
+        .formatVersion(DEFAULT_VIEW_FORMAT_VERSION)
+        .location(metadata.location())
+        .properties(metadata.properties())
+        .currentVersionId(metadata.currentVersionId())
+        .versions(metadata.versions())
+        .schemas(metadata.schemas())
+        .currentSchemaId(metadata.currentSchemaId())
+        .history(metadata.history());
+  }
+
+  @Value.Check
+  void check() {
+    Preconditions.checkState(versions().size() > 0, "Expecting non-empty version log");

Review Comment:
   I'm not sure this is a good idea. If we don't have a log, should it stop us from reading the view?



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);

Review Comment:
   +1



-- 
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 #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/BaseSQLViewRepresentation.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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 java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class BaseSQLViewRepresentation implements SQLViewRepresentation {
+
+  private final String query;
+  private final String dialect;
+  private final String defaultCatalog;
+  private final int schemaId;
+  private final Namespace defaultNamespace;
+  private final List<String> fieldAliases;
+  private final List<String> fieldComments;
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static Builder buildFrom(SQLViewRepresentation representation) {
+    return builder()
+        .query(representation.query())
+        .schemaId(representation.schemaId())
+        .dialect(representation.dialect())
+        .defaultCatalog(representation.defaultCatalog())
+        .defaultNamespace(representation.defaultNamespace())
+        .fieldAliases(representation.fieldAliases())
+        .fieldComments(representation.fieldComments());
+  }
+
+  private BaseSQLViewRepresentation(
+      String query,
+      String dialect,
+      String defaultCatalog,
+      Namespace defaultNamespace,
+      int schemaId,
+      List<String> fieldAliases,
+      List<String> fieldComments) {
+    this.query = Preconditions.checkNotNull(query, "sql should not be null");
+    this.dialect = Preconditions.checkNotNull(dialect, "dialect should not be null");
+    this.defaultCatalog =
+        Preconditions.checkNotNull(defaultCatalog, "default catalog should not null");
+    this.defaultNamespace =
+        Preconditions.checkNotNull(defaultNamespace, "default namespace should not be null");
+    this.schemaId = Preconditions.checkNotNull(schemaId, "schema should not be null");
+    this.fieldAliases =
+        Preconditions.checkNotNull(fieldAliases, "field aliases should not be null");
+    this.fieldComments =
+        Preconditions.checkNotNull(fieldComments, "field comments should not be null");
+  }
+
+  @Override
+  public String query() {
+    return query;
+  }
+
+  @Override
+  public String dialect() {
+    return dialect;
+  }
+
+  @Override
+  public String defaultCatalog() {
+    return defaultCatalog;
+  }
+
+  @Override
+  public Namespace defaultNamespace() {
+    return defaultNamespace;
+  }
+
+  @Override
+  public int schemaId() {
+    return schemaId;
+  }
+
+  @Override
+  public List<String> fieldComments() {
+    return fieldComments;
+  }
+
+  @Override
+  public List<String> fieldAliases() {
+    return fieldAliases;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    BaseSQLViewRepresentation that = (BaseSQLViewRepresentation) o;
+    return Objects.equals(query, that.query)
+        && Objects.equals(dialect, that.dialect)
+        && Objects.equals(schemaId, that.schemaId)
+        && Objects.equals(defaultCatalog, that.defaultCatalog)
+        && Objects.equals(defaultNamespace, that.defaultNamespace)
+        && Objects.equals(fieldAliases, that.fieldAliases)
+        && Objects.equals(fieldComments, that.fieldComments);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(
+        query, dialect, schemaId, defaultCatalog, defaultNamespace, fieldAliases, fieldComments);
+  }
+
+  @Override
+  public String toString() {
+    return "BaseViewDefinition{"

Review Comment:
   Can you use a helper rather than building this by hand? It's hard to read this way. We usually base these on `MoreObjects`.



-- 
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 #6559: Core: View core parser implementations

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


##########
core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+
+public class TestViewMetadata {
+
+  private static ViewVersion version1 =
+      ImmutableViewVersion.builder()
+          .versionId(1)
+          .timestampMillis(4353L)
+          .operation("create")
+          .addRepresentations(
+              ImmutableSQLViewRepresentation.builder()
+                  .sql("select 'foo' foo")
+                  .dialect("spark-sql")
+                  .schemaId(1)
+                  .defaultCatalog("some-catalog")
+                  .build())
+          .build();
+  private static ViewHistoryEntry historyEntry1 =
+      ImmutableViewHistoryEntry.builder().timestampMillis(4353L).versionId(1).build();
+
+  private static ViewVersion version2 =
+      ImmutableViewVersion.builder()
+          .versionId(2)
+          .timestampMillis(5555L)
+          .operation("replace")
+          .addRepresentations(
+              ImmutableSQLViewRepresentation.builder()
+                  .sql("select 1 id, 'abc' data")
+                  .defaultCatalog("some-catalog")
+                  .dialect("spark-sql")
+                  .schemaId(1)
+                  .build())
+          .build();
+  private static ViewHistoryEntry historyEntry2 =
+      ImmutableViewHistoryEntry.builder().timestampMillis(5555L).versionId(2).build();
+  private static final Schema TEST_SCHEMA =
+      new Schema(
+          1,
+          Types.NestedField.required(1, "x", Types.LongType.get()),
+          Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+          Types.NestedField.required(3, "z", Types.LongType.get()));
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> parameters() {
+    return Arrays.asList(new Object[][] {});
+  }
+
+  @Test
+  public void testReadValidViewMetadata() throws Exception {
+    String json = readTableMetadataInputFile("ValidViewMetadata.json");
+    ViewMetadata expectedViewMetadata =
+        ImmutableViewMetadata.builder()
+            .currentSchemaId(1)
+            .schemas(ImmutableList.of(TEST_SCHEMA))
+            .versions(ImmutableList.of(version1, version2))
+            .history(ImmutableList.of(historyEntry1, historyEntry2))
+            .location("s3://bucket/test/location")
+            .properties(ImmutableMap.of("some-key", "some-value"))
+            .currentVersionId(2)
+            .formatVersion(1)
+            .build();
+    assertSameViewMetadata(expectedViewMetadata, ViewMetadataParser.fromJson(json));
+  }
+
+  private void assertSameViewMetadata(ViewMetadata expected, ViewMetadata actual) {
+    Assert.assertEquals(expected.currentSchemaId(), actual.currentSchemaId());

Review Comment:
   Prefer to use Assertions to deprecate usage of Junit4



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {

Review Comment:
   @stevenzwu `ImmutableViewMetadata` is a generated class from the `@Value.Immutable` annotation. See also https://immutables.github.io/ for some additional details



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/test/resources/ValidViewMetadata.json:
##########
@@ -0,0 +1,38 @@
+{
+  "format-version": 1,
+  "location": "s3://bucket/test/location",
+  "properties": {"some-key": "some-value"},
+  "current-schema-id": 1,
+  "schemas": [
+    {
+      "type": "struct",
+      "schema-id": 1,
+      "fields": [
+        {
+          "id": 1,
+          "name": "x",
+          "required": true,
+          "type": "long"
+        },
+        {
+          "id": 2,
+          "name": "y",
+          "required": true,
+          "type": "long",
+          "doc": "comment"
+        },
+        {
+          "id": 3,
+          "name": "z",
+          "required": true,
+          "type": "long"
+        }
+      ]
+    }
+  ],
+  "current-version-id": 2,
+  "versions": [
+    {"version-id":1,"timestamp-ms":4353,"summary":{"operation":"create"},"representations":[{"type":"sql","sql":"select 'foo' foo","dialect":"spark-sql","default-catalog":"some-catalog","default-namespace":[],"field-aliases":[],"field-comments":[], "schema-id":1}]},

Review Comment:
   Can you format this?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {
+    return ImmutableViewMetadata.builder()
+        .formatVersion(DEFAULT_VIEW_FORMAT_VERSION)
+        .location(metadata.location())
+        .properties(metadata.properties())
+        .currentVersionId(metadata.currentVersionId())
+        .versions(metadata.versions())
+        .schemas(metadata.schemas())
+        .currentSchemaId(metadata.currentSchemaId())
+        .history(metadata.history());
+  }
+
+  @Value.Check
+  void check() {
+    Preconditions.checkState(versions().size() > 0, "Expecting non-empty version log");
+    checkVersionLog(history());
+  }
+
+  public abstract int formatVersion();
+
+  public abstract String location();
+
+  public abstract Map<String, String> properties();
+
+  public abstract int currentVersionId();
+
+  public abstract List<ViewVersion> versions();
+
+  public abstract List<ViewHistoryEntry> history();
+
+  @Nullable
+  public abstract List<Schema> schemas();
+
+  @Nullable
+  public abstract Integer currentSchemaId();
+
+  public ViewVersion version(int versionId) {
+    return versionsById().get(versionId);
+  }
+
+  @Value.Derived
+  public ViewVersion currentVersion() {
+    return versionsById().get(currentVersionId());
+  }
+
+  @Value.Derived
+  public Map<Integer, ViewVersion> versionsById() {
+    return indexVersions(versions());
+  }
+
+  @Value.Derived
+  public Map<Integer, Schema> schemasById() {
+    return indexSchemas(schemas());
+  }
+
+  @Value.Derived
+  public Schema schema() {
+    return schemasById().get(currentSchemaId());
+  }
+
+  private static void checkVersionLog(List<ViewHistoryEntry> versionLog) {
+    Preconditions.checkState(versionLog.size() > 0, "Expecting non-empty version log");
+    Preconditions.checkState(
+        Ordering.from(Comparator.comparing(ViewHistoryEntry::timestampMillis))
+            .isOrdered(versionLog),
+        "Expected sorted version log entries.");

Review Comment:
   I probably wouldn't bother with this, considering we can just sort the entries.



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {
+    return ImmutableViewMetadata.builder()
+        .formatVersion(DEFAULT_VIEW_FORMAT_VERSION)
+        .location(metadata.location())
+        .properties(metadata.properties())
+        .currentVersionId(metadata.currentVersionId())
+        .versions(metadata.versions())
+        .schemas(metadata.schemas())
+        .currentSchemaId(metadata.currentSchemaId())
+        .history(metadata.history());
+  }
+
+  @Value.Check
+  void check() {
+    Preconditions.checkState(versions().size() > 0, "Expecting non-empty version log");
+    checkVersionLog(history());
+  }
+
+  public abstract int formatVersion();
+
+  public abstract String location();
+
+  public abstract Map<String, String> properties();
+
+  public abstract int currentVersionId();
+
+  public abstract List<ViewVersion> versions();
+
+  public abstract List<ViewHistoryEntry> history();
+
+  @Nullable
+  public abstract List<Schema> schemas();
+
+  @Nullable
+  public abstract Integer currentSchemaId();
+
+  public ViewVersion version(int versionId) {
+    return versionsById().get(versionId);
+  }
+
+  @Value.Derived
+  public ViewVersion currentVersion() {
+    return versionsById().get(currentVersionId());
+  }
+
+  @Value.Derived
+  public Map<Integer, ViewVersion> versionsById() {
+    return indexVersions(versions());
+  }
+
+  @Value.Derived
+  public Map<Integer, Schema> schemasById() {
+    return indexSchemas(schemas());
+  }
+
+  @Value.Derived
+  public Schema schema() {
+    return schemasById().get(currentSchemaId());
+  }
+
+  private static void checkVersionLog(List<ViewHistoryEntry> versionLog) {
+    Preconditions.checkState(versionLog.size() > 0, "Expecting non-empty version log");

Review Comment:
   If the log is missing this implies bad or corrupted view metadata. At minimum there must be at least 1 view version for the view to be considered as well defined. The original rationale was that if the metadata file corrupted, one shouldn't be able to read the view because there may be a correctness issue.
   
   Alternatively, we could remove the validation since at the end of the day as long as current-version is well defined then the read can progress. With this approach I'm just concerned that if version-log is missing, and reads succeed then "corruption" just propagates silently as the view evolves and in the case of a genuine issue it can confuse someone trying to diagnose the issue.
   
    It seems better just to have a guarantee that version-log is well defined a the time of reading the view.



-- 
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 #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewProperties.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/** View properties that can be set during CREATE/REPLACE view or using updateProperties API. */
+public class ViewProperties {
+  public static final String COMMIT_NUM_RETRIES = "commit.retry.num-retries";
+  public static final int COMMIT_NUM_RETRIES_DEFAULT = 4;
+
+  public static final String COMMIT_MIN_RETRY_WAIT_MS = "commit.retry.min-wait-ms";
+  public static final int COMMIT_MIN_RETRY_WAIT_MS_DEFAULT = 100;
+
+  public static final String COMMIT_MAX_RETRY_WAIT_MS = "commit.retry.max-wait-ms";
+  public static final int COMMIT_MAX_RETRY_WAIT_MS_DEFAULT = 60000; // 1 minute
+
+  public static final String COMMIT_TOTAL_RETRY_TIME_MS = "commit.retry.total-timeout-ms";
+  public static final int COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT = 1800000; // 30 minutes
+
+  public static final String VERSION_HISTORY_SIZE = "version.history.num-entries";
+  public static final int VERSION_HISTORY_SIZE_DEFAULT = 10;
+
+  public static final String TABLE_COMMENT = "comment";

Review Comment:
   seems to be unused



##########
core/src/test/java/org/apache/iceberg/view/ParserTestBase.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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 java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.TestJsonUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+public abstract class ParserTestBase<T> {

Review Comment:
   I'm not sure whether it's worth adding this base class. Are we planning to introduce this to all the other test classes that test parsers?



##########
core/src/main/java/org/apache/iceberg/view/ViewProperties.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/** View properties that can be set during CREATE/REPLACE view or using updateProperties API. */
+public class ViewProperties {
+  public static final String COMMIT_NUM_RETRIES = "commit.retry.num-retries";
+  public static final int COMMIT_NUM_RETRIES_DEFAULT = 4;
+
+  public static final String COMMIT_MIN_RETRY_WAIT_MS = "commit.retry.min-wait-ms";
+  public static final int COMMIT_MIN_RETRY_WAIT_MS_DEFAULT = 100;
+
+  public static final String COMMIT_MAX_RETRY_WAIT_MS = "commit.retry.max-wait-ms";
+  public static final int COMMIT_MAX_RETRY_WAIT_MS_DEFAULT = 60000; // 1 minute
+
+  public static final String COMMIT_TOTAL_RETRY_TIME_MS = "commit.retry.total-timeout-ms";
+  public static final int COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT = 1800000; // 30 minutes

Review Comment:
   do we need to have separate min/max/total retry settings for views or could we just use whatever we use for tables?



##########
core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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 java.util.Arrays;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.types.Types;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestViewMetadataParser extends ParserTestBase<ViewMetadata> {

Review Comment:
   I believe this is missing a few tests around nullability/empty jsons and such, similar to what I mentioned in https://github.com/apache/iceberg/pull/6565



##########
core/src/main/java/org/apache/iceberg/view/BaseSQLViewRepresentation.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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 java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class BaseSQLViewRepresentation implements SQLViewRepresentation {
+
+  private final String query;

Review Comment:
   all of these new base classes are adding quite a lot of overhead, since they require manual builder classes (and technically even tests for those builders). It would be better to switch all of those newly added classes to using [`@Immutable`](https://immutables.github.io/). The end result will be:
   * much shorter code
   * we'll get the builders for free
   * no need to add null checks and such as the Immutable builder will derive what needs to be checked for nulls based on the API definition.
   
   Just to give you an example of how much shorter the code can look, I've moved those classes to using `@Immutable` in https://github.com/apache/iceberg/commit/97bcd5030e2a37ef8691d4589c40427e2061a2b8. The diff is a bit difficult to read because lots of code has been removed, but the resulting [files(s)](https://github.com/apache/iceberg/tree/97bcd5030e2a37ef8691d4589c40427e2061a2b8/core/src/main/java/org/apache/iceberg/view) are much shorter, such as https://github.com/apache/iceberg/blob/97bcd5030e2a37ef8691d4589c40427e2061a2b8/core/src/main/java/org/apache/iceberg/view/BaseSQLViewRepresentation.java
   
   



##########
core/src/main/java/org/apache/iceberg/util/JsonUtil.java:
##########
@@ -283,6 +284,63 @@ public static void writeLongFieldIf(
     }
   }
 
+  @FunctionalInterface
+  public interface JsonWriter<T> {

Review Comment:
   it's probably better to have a separate PR for `JsonUtil` + `TestJsonUtil` changes



##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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;
+
+public class ViewRepresentationParser {
+
+  enum Field {
+    TYPE;
+
+    public String fieldName() {
+      return name().toLowerCase(Locale.ENGLISH);
+    }
+  }
+
+  public static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    switch (representation.type()) {
+      case SQL:
+        SQLViewRepresentationParser.toJson((SQLViewRepresentation) representation, generator);
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Unknown view representation type '%s' to serialize", representation.type()));
+    }
+  }
+
+  public static ViewRepresentation fromJson(JsonNode node) {
+    String typeName = node.get(Field.TYPE.fieldName()).asText();

Review Comment:
   this needs a null check for `node` + tests



##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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;
+
+public class ViewRepresentationParser {
+
+  enum Field {
+    TYPE;
+
+    public String fieldName() {
+      return name().toLowerCase(Locale.ENGLISH);
+    }
+  }
+
+  public static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    switch (representation.type()) {

Review Comment:
   I believe this needs a null check for `representation`



##########
core/src/main/java/org/apache/iceberg/view/ViewHistoryEntryParser.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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 org.apache.iceberg.util.JsonUtil;
+
+class ViewHistoryEntryParser {
+
+  // visible for testing
+  static final String VERSION_ID = "version-id";
+  static final String TIMESTAMP_MS = "timestamp-ms";
+
+  static void toJson(ViewHistoryEntry entry, JsonGenerator generator) throws IOException {

Review Comment:
   needs a null check (similar to how we do it in other parsers)



##########
core/src/main/java/org/apache/iceberg/view/ViewHistoryEntryParser.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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 org.apache.iceberg.util.JsonUtil;
+
+class ViewHistoryEntryParser {
+
+  // visible for testing
+  static final String VERSION_ID = "version-id";
+  static final String TIMESTAMP_MS = "timestamp-ms";
+
+  static void toJson(ViewHistoryEntry entry, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+    generator.writeNumberField(TIMESTAMP_MS, entry.timestampMillis());
+    generator.writeNumberField(VERSION_ID, entry.versionId());
+    generator.writeEndObject();
+  }
+
+  static ViewHistoryEntry fromJson(JsonNode node) {
+    return BaseViewHistoryEntry.of(

Review Comment:
   needs a null check (similar to how we do it in other parsers)



##########
core/src/main/java/org/apache/iceberg/view/ViewVersionParser.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.List;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewVersionParser {
+
+  private static final String VERSION_ID = "version-id";
+  private static final String TIMESTAMP_MS = "timestamp-ms";
+  private static final String SUMMARY = "summary";
+  private static final String OPERATION = "operation";
+  private static final String REPRESENTATIONS = "representations";
+
+  static void toJson(ViewVersion version, JsonGenerator generator) throws IOException {

Review Comment:
   fromJson + toJson needs some null checks



##########
core/src/main/java/org/apache/iceberg/view/ViewVersionParser.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.List;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewVersionParser {
+
+  private static final String VERSION_ID = "version-id";
+  private static final String TIMESTAMP_MS = "timestamp-ms";
+  private static final String SUMMARY = "summary";
+  private static final String OPERATION = "operation";

Review Comment:
   seems to be unused



##########
core/src/main/java/org/apache/iceberg/view/ViewVersionParser.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.List;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class ViewVersionParser {
+
+  private static final String VERSION_ID = "version-id";
+  private static final String TIMESTAMP_MS = "timestamp-ms";
+  private static final String SUMMARY = "summary";
+  private static final String OPERATION = "operation";
+  private static final String REPRESENTATIONS = "representations";
+
+  static void toJson(ViewVersion version, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(VERSION_ID, version.versionId());
+    generator.writeNumberField(TIMESTAMP_MS, version.timestampMillis());
+    JsonUtil.writeStringMap(SUMMARY, version.summary(), generator);
+    JsonUtil.writeObjectList(
+        REPRESENTATIONS, version.representations(), ViewRepresentationParser::toJson, generator);
+
+    generator.writeEndObject();
+  }
+
+  static ViewVersion fromJson(JsonNode node) {
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse table version from a non-object: %s", node);
+
+    int versionId = JsonUtil.getInt(VERSION_ID, node);
+    long timestamp = JsonUtil.getLong(TIMESTAMP_MS, node);
+    Map<String, String> summary = JsonUtil.getStringMap(SUMMARY, node);
+    List<ViewRepresentation> representations =
+        JsonUtil.getObjectList(REPRESENTATIONS, node, ViewRepresentationParser::fromJson);
+
+    return BaseViewVersion.builder()
+        .versionId(versionId)
+        .timestampMillis(timestamp)
+        .summary(summary)
+        .representations(representations)
+        .build();
+  }
+
+  private ViewVersionParser() {}

Review Comment:
   nit: same as above, would be good to move it to the top



##########
core/src/main/java/org/apache/iceberg/view/ViewRepresentationParser.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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;
+
+public class ViewRepresentationParser {
+
+  enum Field {
+    TYPE;
+
+    public String fieldName() {
+      return name().toLowerCase(Locale.ENGLISH);
+    }
+  }
+
+  public static void toJson(ViewRepresentation representation, JsonGenerator generator)
+      throws IOException {
+    switch (representation.type()) {
+      case SQL:
+        SQLViewRepresentationParser.toJson((SQLViewRepresentation) representation, generator);
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Unknown view representation type '%s' to serialize", representation.type()));
+    }
+  }
+
+  public static ViewRepresentation fromJson(JsonNode node) {
+    String typeName = node.get(Field.TYPE.fieldName()).asText();
+    ViewRepresentation.Type type = ViewRepresentation.Type.fromString(typeName);
+
+    switch (type) {
+      case SQL:
+        return SQLViewRepresentationParser.fromJson(node);
+
+      default:
+        throw new IllegalStateException(
+            String.format("Unknown view representation type '%s' to deserialize", type));
+    }
+  }
+
+  private ViewRepresentationParser() {}

Review Comment:
   nit: I think it would be better to move this to the top



-- 
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 #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+
+public class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  // stored metadata
+  private final int formatVersion;
+  private final String location;
+  private final Map<String, String> properties;
+  private final int currentVersionId;
+  private final Integer currentSchemaId;
+  private final List<ViewVersion> versions;
+  private final Map<Integer, ViewVersion> versionsById;
+  private final List<ViewHistoryEntry> versionLog;
+  private final List<Schema> schemas;
+  private final Map<Integer, Schema> schemasById;
+  private final String metadataFileLocation;

Review Comment:
   Shouldn't store metadataFileLocation in the metadata file, 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 #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    if (metadata.currentSchemaId() != null) {
+      generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    }
+
+    if (metadata.schemas() != null) {
+      JsonUtil.writeObjectList(SCHEMAS, metadata.schemas(), SchemaParser::toJson, generator);
+    }
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    JsonUtil.writeObjectList(VERSIONS, metadata.versions(), ViewVersionParser::toJson, generator);
+    JsonUtil.writeObjectList(
+        VERSION_LOG, metadata.history(), ViewHistoryEntryParser::toJson, generator);
+
+    generator.writeEndObject();
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to read file: %s", file);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    return fromJson(node, null);
+  }
+
+  public static ViewMetadata fromJson(JsonNode node, String metadataFileLocation) {
+
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);
+
+    String location = JsonUtil.getString(LOCATION, node);
+
+    int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, node);
+
+    Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, node);
+
+    List<ViewVersion> versions =
+        JsonUtil.getObjectList(VERSIONS, node, ViewVersionParser::fromJson);
+
+    List<ViewHistoryEntry> history =
+        JsonUtil.getObjectList(VERSION_LOG, node, ViewHistoryEntryParser::fromJson);
+
+    int numVersionsToKeep =
+        PropertyUtil.propertyAsInt(
+            properties,
+            ViewProperties.VERSION_HISTORY_SIZE,
+            ViewProperties.VERSION_HISTORY_SIZE_DEFAULT);
+
+    versions = versions.subList(0, Math.min(numVersionsToKeep, versions.size()));
+
+    List<Schema> schemas = null;
+    Integer currentSchemaId = null;
+    if (node.has(SCHEMAS)) {
+      JsonNode schemaArray = node.get(SCHEMAS);
+      Schema currentSchema = null;
+      Preconditions.checkArgument(
+          schemaArray.isArray(), "Cannot parse schemas from non-array: %s", schemaArray);
+      // current schema ID is required when the schema array is present
+      currentSchemaId = JsonUtil.getInt(CURRENT_SCHEMA_ID, node);
+      // parse the schema array
+      ImmutableList.Builder<Schema> builder = ImmutableList.builder();
+      for (JsonNode schemaNode : schemaArray) {
+        Schema schema = SchemaParser.fromJson(schemaNode);
+        if (schema.schemaId() == currentSchemaId) {
+          currentSchema = schema;
+        }
+        builder.add(schema);
+      }

Review Comment:
   The doubt I still have is I think schemas and current schema should always be set, and thus the spec should be reflected so that it's required. Maybe there's a case that we don't want schema for the entire view and it's up to individual representations. In that case in my mind for any SQL representation in the view, the schema-id should be required (which then means there must be a schema list considering we only support SQL representations). Right now it's marked as optional 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] nastra commented on a diff in pull request #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {

Review Comment:
   My guess why this is an abstract class rather than an Interface is because of `indexVersions()` and `indexSchemas()` can't be made private if we use an Interface here, meaning that they would become part of the API, which I don't think we want
   



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {

Review Comment:
   why this is not public?



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

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

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


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


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {
+    return ImmutableViewMetadata.builder()
+        .formatVersion(DEFAULT_VIEW_FORMAT_VERSION)
+        .location(metadata.location())
+        .properties(metadata.properties())
+        .currentVersionId(metadata.currentVersionId())
+        .versions(metadata.versions())
+        .schemas(metadata.schemas())
+        .currentSchemaId(metadata.currentSchemaId())
+        .history(metadata.history());
+  }
+
+  @Value.Check
+  void check() {
+    Preconditions.checkState(versions().size() > 0, "Expecting non-empty version log");
+    checkVersionLog(history());
+  }
+
+  public abstract int formatVersion();
+
+  public abstract String location();
+
+  public abstract Map<String, String> properties();
+
+  public abstract int currentVersionId();
+
+  public abstract List<ViewVersion> versions();
+
+  public abstract List<ViewHistoryEntry> history();
+
+  @Nullable
+  public abstract List<Schema> schemas();
+
+  @Nullable
+  public abstract Integer currentSchemaId();
+
+  public ViewVersion version(int versionId) {
+    return versionsById().get(versionId);
+  }
+
+  @Value.Derived
+  public ViewVersion currentVersion() {
+    return versionsById().get(currentVersionId());
+  }
+
+  @Value.Derived
+  public Map<Integer, ViewVersion> versionsById() {
+    return indexVersions(versions());
+  }
+
+  @Value.Derived
+  public Map<Integer, Schema> schemasById() {
+    return indexSchemas(schemas());
+  }
+
+  @Value.Derived
+  public Schema schema() {
+    return schemasById().get(currentSchemaId());
+  }
+
+  private static void checkVersionLog(List<ViewHistoryEntry> versionLog) {
+    Preconditions.checkState(versionLog.size() > 0, "Expecting non-empty version log");

Review Comment:
   After more thought, I think you're right @rdblue . For the purpose of validating view metadata, we should decouple the history of a view, which indicates when a view was considered "current" and the actual versions. We already verify that the current view version is valid at the time of reading the metadata which should be sufficient .



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

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

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


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


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.util.JsonUtil;
+
+public class SQLViewRepresentationParser {
+  private enum Field {
+    SQL("sql"),

Review Comment:
   why do we need enums for these, instead of just strings as static variables?



-- 
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 #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/BaseSQLViewRepresentation.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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 java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class BaseSQLViewRepresentation implements SQLViewRepresentation {
+
+  private final String query;
+  private final String dialect;
+  private final String defaultCatalog;
+  private final int schemaId;
+  private final Namespace defaultNamespace;
+  private final List<String> fieldAliases;
+  private final List<String> fieldComments;
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static Builder buildFrom(SQLViewRepresentation representation) {
+    return builder()
+        .query(representation.query())
+        .schemaId(representation.schemaId())
+        .dialect(representation.dialect())
+        .defaultCatalog(representation.defaultCatalog())
+        .defaultNamespace(representation.defaultNamespace())
+        .fieldAliases(representation.fieldAliases())
+        .fieldComments(representation.fieldComments());
+  }
+
+  private BaseSQLViewRepresentation(
+      String query,
+      String dialect,
+      String defaultCatalog,
+      Namespace defaultNamespace,
+      int schemaId,
+      List<String> fieldAliases,
+      List<String> fieldComments) {
+    this.query = Preconditions.checkNotNull(query, "sql should not be null");
+    this.dialect = Preconditions.checkNotNull(dialect, "dialect should not be null");
+    this.defaultCatalog =
+        Preconditions.checkNotNull(defaultCatalog, "default catalog should not null");
+    this.defaultNamespace =
+        Preconditions.checkNotNull(defaultNamespace, "default namespace should not be null");
+    this.schemaId = Preconditions.checkNotNull(schemaId, "schema should not be null");
+    this.fieldAliases =
+        Preconditions.checkNotNull(fieldAliases, "field aliases should not be null");
+    this.fieldComments =
+        Preconditions.checkNotNull(fieldComments, "field comments should not be null");
+  }
+
+  @Override
+  public String query() {
+    return query;
+  }
+
+  @Override
+  public String dialect() {
+    return dialect;
+  }
+
+  @Override
+  public String defaultCatalog() {
+    return defaultCatalog;
+  }
+
+  @Override
+  public Namespace defaultNamespace() {
+    return defaultNamespace;
+  }
+
+  @Override
+  public int schemaId() {
+    return schemaId;
+  }
+
+  @Override
+  public List<String> fieldComments() {
+    return fieldComments;
+  }
+
+  @Override
+  public List<String> fieldAliases() {
+    return fieldAliases;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    BaseSQLViewRepresentation that = (BaseSQLViewRepresentation) o;

Review Comment:
   Style: this method needs to have line spacing applied.



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    if (metadata.schemas() != null && !metadata.schemas().isEmpty()) {
+      generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+      generator.writeArrayFieldStart(SCHEMAS);
+      for (Schema schema : metadata.schemas()) {
+        SchemaParser.toJson(schema, generator);
+      }
+
+      generator.writeEndArray();
+    }

Review Comment:
   Totally agreed @rdblue on all points. I'll work on a PR to move the schema to ViewVersion. 



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {
+    return ImmutableViewMetadata.builder()
+        .formatVersion(DEFAULT_VIEW_FORMAT_VERSION)
+        .location(metadata.location())
+        .properties(metadata.properties())
+        .currentVersionId(metadata.currentVersionId())
+        .versions(metadata.versions())
+        .schemas(metadata.schemas())
+        .currentSchemaId(metadata.currentSchemaId())
+        .history(metadata.history());
+  }
+
+  @Value.Check
+  void check() {
+    Preconditions.checkState(versions().size() > 0, "Expecting non-empty version log");
+    checkVersionLog(history());
+  }
+
+  public abstract int formatVersion();
+
+  public abstract String location();
+
+  public abstract Map<String, String> properties();
+
+  public abstract int currentVersionId();
+
+  public abstract List<ViewVersion> versions();
+
+  public abstract List<ViewHistoryEntry> history();
+
+  @Nullable
+  public abstract List<Schema> schemas();
+
+  @Nullable
+  public abstract Integer currentSchemaId();
+
+  public ViewVersion version(int versionId) {
+    return versionsById().get(versionId);
+  }
+
+  @Value.Derived
+  public ViewVersion currentVersion() {
+    return versionsById().get(currentVersionId());
+  }
+
+  @Value.Derived
+  public Map<Integer, ViewVersion> versionsById() {
+    return indexVersions(versions());
+  }
+
+  @Value.Derived
+  public Map<Integer, Schema> schemasById() {
+    return indexSchemas(schemas());
+  }
+
+  @Value.Derived
+  public Schema schema() {
+    return schemasById().get(currentSchemaId());
+  }
+
+  private static void checkVersionLog(List<ViewHistoryEntry> versionLog) {
+    Preconditions.checkState(versionLog.size() > 0, "Expecting non-empty version log");
+    Preconditions.checkState(
+        Ordering.from(Comparator.comparing(ViewHistoryEntry::timestampMillis))
+            .isOrdered(versionLog),
+        "Expected sorted version log entries.");

Review Comment:
   Good point, there's not really any value to this check



-- 
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 #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewProperties.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/** View properties that can be set during CREATE/REPLACE view or using updateProperties API. */
+public class ViewProperties {
+  public static final String COMMIT_NUM_RETRIES = "commit.retry.num-retries";

Review Comment:
   These properties are not used at this moment, I think we can add it when we add base view catalog impl 



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    if (metadata.schemas() != null && !metadata.schemas().isEmpty()) {
+      generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+      generator.writeArrayFieldStart(SCHEMAS);
+      for (Schema schema : metadata.schemas()) {
+        SchemaParser.toJson(schema, generator);
+      }
+
+      generator.writeEndArray();
+    }

Review Comment:
   I found https://github.com/apache/iceberg/pull/3188#discussion_r748303084 this thread on the original spec proposal, and then for view representation case there was https://github.com/apache/iceberg/pull/4925#discussion_r1015649887. 
   
   So based on these two I don't think it's required that it's optional? I think it's just a convenience to avoid an engine having to pass in a schema at the time of view creation since some engines will derive from the SQL itself. Based on the second link it seems there is some complexity related to aliasing but if we store the pre-aliased schema that should be sufficient, the post alias columns can be subsequently derived.
   
   Schema handling will probably be a focal point around the view spec so it's important we get this right as we try and conclude this implementation. 
   
   Right now schemaID in `SQLViewRepresentation` is optional. 
   I think we should revisit this. I like @rdblue suggestion in the second link to make schemaID required and at the view version level, since I think all representations should have the same Iceberg schema. Also it is easier to go from schemas being required to schemas being optional if a valid use case comes up, but going the other way is not possible.
   
   Would love to get others 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] rdblue commented on a diff in pull request #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(String.format("Failed to read json file: %s", file), e);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);

Review Comment:
   Minor: what is the value of adding this here rather than in `check()`?



-- 
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 closed pull request #6559: Core: View core parser implementations

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar closed pull request #6559: Core: View core parser implementations
URL: https://github.com/apache/iceberg/pull/6559


-- 
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 #6559: Core: View core parser implementations

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

   I am trying see if we can break this further so that it's easier to review for a broader audience. There are some implementations of the basic interface, such as `BaseViewVersion`, `BaseSQLViewRepresentation`, `BaseViewHistoryEntry`, can we have a PR to first add these objects and their corresponding parsers in 3 small PRs? This would help the reviewers, also help to add tests for each feature we implement. Otherwise there is currently just 1 test for the entire view metadata.


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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.util.JsonUtil;
+
+public class SQLViewRepresentationParser {
+  private enum Field {
+    SQL("sql"),

Review Comment:
   This is a bit different. I'd probably leave these as static strings like we do in other 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] rdblue commented on a diff in pull request #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {

Review Comment:
   Jack makes a good point. A few of these should be reviewed. In addition to this, which reasonably should be public, I think that `toJson(ViewMetadata, JsonGenerator)` should not be public until we need it in another package.



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {

Review Comment:
   The other view objects are interfaces. Why does this use an abstract class? Moving between interfaces and classes is a binary incompatible change, so I think that this should be an interface instead. Is that not possible for some reason?



-- 
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 #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    if (metadata.schemas() != null && !metadata.schemas().isEmpty()) {
+      generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+      generator.writeArrayFieldStart(SCHEMAS);
+      for (Schema schema : metadata.schemas()) {
+        SchemaParser.toJson(schema, generator);
+      }
+
+      generator.writeEndArray();
+    }

Review Comment:
   I found https://github.com/apache/iceberg/pull/3188#discussion_r748303084 this thread on the original spec proposal, and then for view representation case there was https://github.com/apache/iceberg/pull/4925#discussion_r1015649887. 
   
   So based on these two I don't think it's required that it's optional? I think it's just a convenience to avoid an engine having to pass in a schema at the time of view creation since some engines will derive from the SQL itself. Based on the second link it seems there is some complexity related to aliasing but if we store the pre-aliased schema that should be sufficient, the post alias columns can be subsequently derived.
   
   Schema handling will probably be a focal point around the view spec so it's important we get this right as we try and conclude this implementation. 
   
   Right now schemaID in `SQLViewRepresentation` is optional. 
   I think we should revisit this. I like @rdblue suggestion in the second link to make schemaID required and at the view version level, since I think all representations should have the same Iceberg schema. 
   
   Would love to get others 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 #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    if (metadata.schemas() != null && !metadata.schemas().isEmpty()) {
+      generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+      generator.writeArrayFieldStart(SCHEMAS);
+      for (Schema schema : metadata.schemas()) {
+        SchemaParser.toJson(schema, generator);
+      }
+
+      generator.writeEndArray();
+    }

Review Comment:
   I found https://github.com/apache/iceberg/pull/3188#discussion_r748303084 this thread on the original spec proposal, and then for view representation case there was https://github.com/apache/iceberg/pull/4925#discussion_r1015649887. 
   
   So based on these two I don't think it's required that it's optional? I think it's just a convenience to avoid an engine having to pass in a schema at the time of view creation. Based on the second link it seems there is some complexity related to aliasing but if we store the pre-aliased schema that should be sufficient, the post alias columns can be subsequently derived.
   
   Schema handling will probably be a focal point around the view spec so it's important we get this right as we try and conclude this implementation. 
   
   Right now schemaID in `SQLViewRepresentation` is optional. 
   I think we should revisit this. I like @rdblue suggestion in the second link to make schemaID required and at the view version level, since I think all representations should have the same Iceberg schema. 
   
   Would love to get others 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 #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    if (metadata.schemas() != null && !metadata.schemas().isEmpty()) {
+      generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+      generator.writeArrayFieldStart(SCHEMAS);
+      for (Schema schema : metadata.schemas()) {
+        SchemaParser.toJson(schema, generator);
+      }
+
+      generator.writeEndArray();
+    }

Review Comment:
   I need to think about this more. I don't like the idea of schemas being optional in the spec. If an engine chooses to dynamically construct at read time and ignore the schema, that's their choice. I think at minimum when updating the view there must be a schema that is set; the metadata should take some stance on what the output Iceberg schema for the view is, otherwise it seems "under"-spec'd and will open the gates to more interpretation than we would probably like. @rdblue @jackye1995 thoughts?



-- 
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 #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.util.JsonUtil;
+
+public class SQLViewRepresentationParser {
+  private enum Field {
+    SQL("sql"),

Review Comment:
   Good point, this was just how it was in the original PR so I left it as is, we can just use static fields (same as what we do in TableMetadataParser)



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

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

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


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


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(String.format("Failed to read json file: %s", file), e);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);
+
+    String location = JsonUtil.getString(LOCATION, node);
+    int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, node);
+    Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, node);
+
+    JsonNode versionsListNode = JsonUtil.get(VERSIONS, node);
+    Preconditions.checkArgument(
+        versionsListNode.isArray(), "Cannot parse versions from non-array: %s", versionsListNode);
+
+    List<ViewVersion> versions = Lists.newArrayListWithExpectedSize(versionsListNode.size());
+    ViewVersion currentVersion = null;
+    for (JsonNode versionNode : versionsListNode) {
+      ViewVersion version = ViewVersionParser.fromJson(versionNode);
+      if (version.versionId() == currentVersionId) {
+        currentVersion = version;
+      }
+
+      versions.add(version);
+    }
+
+    Preconditions.checkArgument(
+        currentVersion != null,
+        "Cannot find version with %s=%s from %s",
+        CURRENT_VERSION_ID,
+        currentVersionId,
+        VERSIONS);
+
+    JsonNode versionLogNode = JsonUtil.get(VERSION_LOG, node);
+    Preconditions.checkArgument(
+        versionLogNode.isArray(), "Cannot parse version-log from non-array: %s", versionLogNode);
+    List<ViewHistoryEntry> historyEntries =
+        Lists.newArrayListWithExpectedSize(versionLogNode.size());
+    Iterator<JsonNode> versionLogIterator = versionLogNode.elements();
+    while (versionLogIterator.hasNext()) {
+      historyEntries.add(ViewHistoryEntryParser.fromJson(versionLogIterator.next()));
+    }
+
+    List<Schema> schemas;
+    Integer currentSchemaId;
+    Schema currentSchema = null;
+    JsonNode schemaArray = node.get(SCHEMAS);
+
+    Preconditions.checkArgument(
+        schemaArray.isArray(), "Cannot parse schemas from non-array: %s", schemaArray);
+    currentSchemaId = JsonUtil.getInt(CURRENT_SCHEMA_ID, node);
+
+    ImmutableList.Builder<Schema> builder = ImmutableList.builder();
+    for (JsonNode schemaNode : schemaArray) {
+      Schema schema = SchemaParser.fromJson(schemaNode);
+      if (schema.schemaId() == currentSchemaId) {
+        currentSchema = schema;
+      }
+
+      builder.add(schema);
+    }
+
+    Preconditions.checkArgument(
+        currentSchema != null,
+        "Cannot find schema with %s=%s from %s",
+        CURRENT_SCHEMA_ID,
+        currentSchemaId,
+        SCHEMAS);
+
+    schemas = builder.build();
+
+    int numVersionsToKeep =
+        PropertyUtil.propertyAsInt(
+            properties,
+            ViewProperties.VERSION_HISTORY_SIZE,
+            ViewProperties.VERSION_HISTORY_SIZE_DEFAULT);
+
+    Preconditions.checkArgument(
+        numVersionsToKeep >= 1, "%s must be positive", ViewProperties.VERSION_HISTORY_SIZE);
+
+    if (versions.size() > numVersionsToKeep) {

Review Comment:
   Agreed, @nastra fixed this in https://github.com/apache/iceberg/pull/7759/files#diff-037c118271eb576698f0829d876d5c49919eea4bd20e6b2f98b6072cfd0d4b08R133-R135 where all the validation is done in the special check method he referred to.



##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(String.format("Failed to read json file: %s", file), e);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);
+
+    String location = JsonUtil.getString(LOCATION, node);
+    int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, node);
+    Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, node);
+
+    JsonNode versionsListNode = JsonUtil.get(VERSIONS, node);
+    Preconditions.checkArgument(
+        versionsListNode.isArray(), "Cannot parse versions from non-array: %s", versionsListNode);
+
+    List<ViewVersion> versions = Lists.newArrayListWithExpectedSize(versionsListNode.size());
+    ViewVersion currentVersion = null;
+    for (JsonNode versionNode : versionsListNode) {
+      ViewVersion version = ViewVersionParser.fromJson(versionNode);
+      if (version.versionId() == currentVersionId) {
+        currentVersion = version;
+      }
+
+      versions.add(version);
+    }
+
+    Preconditions.checkArgument(

Review Comment:
   Agreed, @nastra fixed this in https://github.com/apache/iceberg/pull/7759/files#diff-037c118271eb576698f0829d876d5c49919eea4bd20e6b2f98b6072cfd0d4b08R133-R135 where all the validation is done in the special check method he referred to.



##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(String.format("Failed to read json file: %s", file), e);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);

Review Comment:
   Agreed, @nastra fixed this in https://github.com/apache/iceberg/pull/7759/files#diff-037c118271eb576698f0829d876d5c49919eea4bd20e6b2f98b6072cfd0d4b08R133-R135 where all the validation is done in the special check method he referred to.



-- 
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 closed pull request #6559: Core: View metadata implementation

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar closed pull request #6559: Core: View metadata implementation
URL: https://github.com/apache/iceberg/pull/6559


-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    if (metadata.schemas() != null && !metadata.schemas().isEmpty()) {
+      generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+      generator.writeArrayFieldStart(SCHEMAS);
+      for (Schema schema : metadata.schemas()) {
+        SchemaParser.toJson(schema, generator);
+      }
+
+      generator.writeEndArray();
+    }

Review Comment:
   I think it is optional for convenience. But I don't think that's a very strong reason.
   
   It would be strange for a view to not have a schema when you inspect it. And it would similarly be unexpected for a view to be created directly by a human rather than an engine. I think it makes sense to require the `schemas`.
   
   However, I would also make a couple of other changes. First, `current-schema-id` doesn't make sense because it would need to be maintained whenever the view version changes. Why keep it in sync when we could just drop the field and use the current version's schema?
   
   Also, I don't think that the view version tracks a schema right now, but since we expect all representations to produce the same schema, I think that it should. I'd probably move `schema-id` from the SQL representation to the view version.



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    if (metadata.schemas() != null && !metadata.schemas().isEmpty()) {
+      generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+      generator.writeArrayFieldStart(SCHEMAS);
+      for (Schema schema : metadata.schemas()) {
+        SchemaParser.toJson(schema, generator);
+      }
+
+      generator.writeEndArray();
+    }

Review Comment:
   Totally agreed @rdblue, I'll work on a PR to move the schema to ViewVersion. 



-- 
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 #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    if (metadata.schemas() != null && !metadata.schemas().isEmpty()) {
+      generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+      generator.writeArrayFieldStart(SCHEMAS);
+      for (Schema schema : metadata.schemas()) {
+        SchemaParser.toJson(schema, generator);
+      }
+
+      generator.writeEndArray();
+    }

Review Comment:
   I found https://github.com/apache/iceberg/pull/3188#discussion_r748303084 this thread on the original spec proposal, and then for view representation case there was https://github.com/apache/iceberg/pull/4925#discussion_r1015649887. 
   
   So based on these two I don't think it's required that it's optional? I think it's just a convenience to avoid an engine having to pass in a schema at the time of view creation. Based on the second link it seems there is some complexity related to aliasing but if we store the pre-aliased schema that should be sufficient, the post alias columns can be subsequently derived.
   
   Schema handling will probably be a focal point around the view spec so it's important we get this right as we try and conclude this implementation. Right now schemaID in `SQLViewRepresentation` is optional. Maybe we revisit this, and make schemaID at the version level, since I think all representations should have the same Iceberg schema. 
   
   Would love to get others 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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {
+    return ImmutableViewMetadata.builder()
+        .formatVersion(DEFAULT_VIEW_FORMAT_VERSION)
+        .location(metadata.location())
+        .properties(metadata.properties())
+        .currentVersionId(metadata.currentVersionId())
+        .versions(metadata.versions())
+        .schemas(metadata.schemas())
+        .currentSchemaId(metadata.currentSchemaId())
+        .history(metadata.history());
+  }
+
+  @Value.Check
+  void check() {
+    Preconditions.checkState(versions().size() > 0, "Expecting non-empty version log");
+    checkVersionLog(history());
+  }
+
+  public abstract int formatVersion();
+
+  public abstract String location();
+
+  public abstract Map<String, String> properties();
+
+  public abstract int currentVersionId();
+
+  public abstract List<ViewVersion> versions();
+
+  public abstract List<ViewHistoryEntry> history();
+
+  @Nullable
+  public abstract List<Schema> schemas();
+
+  @Nullable
+  public abstract Integer currentSchemaId();
+
+  public ViewVersion version(int versionId) {
+    return versionsById().get(versionId);
+  }
+
+  @Value.Derived
+  public ViewVersion currentVersion() {
+    return versionsById().get(currentVersionId());
+  }
+
+  @Value.Derived
+  public Map<Integer, ViewVersion> versionsById() {
+    return indexVersions(versions());
+  }
+
+  @Value.Derived
+  public Map<Integer, Schema> schemasById() {
+    return indexSchemas(schemas());
+  }
+
+  @Value.Derived
+  public Schema schema() {
+    return schemasById().get(currentSchemaId());
+  }
+
+  private static void checkVersionLog(List<ViewHistoryEntry> versionLog) {
+    Preconditions.checkState(versionLog.size() > 0, "Expecting non-empty version log");

Review Comment:
   After more thought, I think you're right @rdblue . For the purpose of validating view metadata, decouple the history of a view, which indicates when a view was considered "current" and the actual versions. We already verify that the current view version is valid at the time of reading the metadata which should be sufficient.



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

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

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


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


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {

Review Comment:
   I think they already are? let me know if i'm missing something



-- 
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] stevenzwu commented on a diff in pull request #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);

Review Comment:
   nit: use `UncheckedIOException`



##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    if (metadata.schemas() != null && !metadata.schemas().isEmpty()) {
+      generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+      generator.writeArrayFieldStart(SCHEMAS);
+      for (Schema schema : metadata.schemas()) {
+        SchemaParser.toJson(schema, generator);
+      }
+
+      generator.writeEndArray();
+    }
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+    generator.writeEndArray();
+
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to read file: %s", file);

Review Comment:
   nit: use `UncheckedIOException`



##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);

Review Comment:
   this blocks can be simplified as. got this good feedback from Eduard in PR #6934  
   ```
       return JsonUtil.generate(
           generator -> ViewMetadataParser.toJson(metadata, generator), false);
   ```



##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    if (metadata.schemas() != null && !metadata.schemas().isEmpty()) {
+      generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+      generator.writeArrayFieldStart(SCHEMAS);
+      for (Schema schema : metadata.schemas()) {
+        SchemaParser.toJson(schema, generator);
+      }
+
+      generator.writeEndArray();
+    }
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+    generator.writeEndArray();
+
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to read file: %s", file);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);
+
+    String location = JsonUtil.getString(LOCATION, node);
+
+    int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, node);
+
+    Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, node);
+    JsonNode versionsListNode = JsonUtil.get(VERSIONS, node);
+    List<ViewVersion> versions = Lists.newArrayListWithExpectedSize(versionsListNode.size());
+    Iterator<JsonNode> iterator = versionsListNode.elements();

Review Comment:
   should we validate the list type for the JSON node? this applies to other non-primitive types (like Map).
   ```
   Preconditions.checkArgument(
             versionsListNode.isArray(), "Cannot parse ... from a non-array: %s", versionsListNode);
   ````
   
   If we define the schema for `ViewMetadata`, we might be able to leverage `SingleValueParser` to handle those. E.g.
   ```
       Map<Integer, ByteBuffer> upperBounds = null;
       if (jsonNode.has(UPPER_BOUNDS)) {
         upperBounds =
             (Map<Integer, ByteBuffer>)
                 SingleValueParser.fromJson(DataFile.UPPER_BOUNDS.type(), jsonNode.get(UPPER_BOUNDS));
       }
   ```



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(String.format("Failed to read json file: %s", file), e);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);
+
+    String location = JsonUtil.getString(LOCATION, node);
+    int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, node);
+    Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, node);
+
+    JsonNode versionsListNode = JsonUtil.get(VERSIONS, node);
+    Preconditions.checkArgument(
+        versionsListNode.isArray(), "Cannot parse versions from non-array: %s", versionsListNode);
+
+    List<ViewVersion> versions = Lists.newArrayListWithExpectedSize(versionsListNode.size());
+    ViewVersion currentVersion = null;
+    for (JsonNode versionNode : versionsListNode) {
+      ViewVersion version = ViewVersionParser.fromJson(versionNode);
+      if (version.versionId() == currentVersionId) {
+        currentVersion = version;
+      }
+
+      versions.add(version);
+    }
+
+    Preconditions.checkArgument(
+        currentVersion != null,
+        "Cannot find version with %s=%s from %s",
+        CURRENT_VERSION_ID,
+        currentVersionId,
+        VERSIONS);
+
+    JsonNode versionLogNode = JsonUtil.get(VERSION_LOG, node);
+    Preconditions.checkArgument(
+        versionLogNode.isArray(), "Cannot parse version-log from non-array: %s", versionLogNode);
+    List<ViewHistoryEntry> historyEntries =
+        Lists.newArrayListWithExpectedSize(versionLogNode.size());
+    Iterator<JsonNode> versionLogIterator = versionLogNode.elements();
+    while (versionLogIterator.hasNext()) {
+      historyEntries.add(ViewHistoryEntryParser.fromJson(versionLogIterator.next()));
+    }
+
+    List<Schema> schemas;
+    Integer currentSchemaId;
+    Schema currentSchema = null;
+    JsonNode schemaArray = node.get(SCHEMAS);
+
+    Preconditions.checkArgument(
+        schemaArray.isArray(), "Cannot parse schemas from non-array: %s", schemaArray);
+    currentSchemaId = JsonUtil.getInt(CURRENT_SCHEMA_ID, node);
+
+    ImmutableList.Builder<Schema> builder = ImmutableList.builder();
+    for (JsonNode schemaNode : schemaArray) {
+      Schema schema = SchemaParser.fromJson(schemaNode);
+      if (schema.schemaId() == currentSchemaId) {
+        currentSchema = schema;
+      }
+
+      builder.add(schema);
+    }
+
+    Preconditions.checkArgument(
+        currentSchema != null,
+        "Cannot find schema with %s=%s from %s",
+        CURRENT_SCHEMA_ID,
+        currentSchemaId,
+        SCHEMAS);
+
+    schemas = builder.build();
+
+    int numVersionsToKeep =
+        PropertyUtil.propertyAsInt(
+            properties,
+            ViewProperties.VERSION_HISTORY_SIZE,
+            ViewProperties.VERSION_HISTORY_SIZE_DEFAULT);
+
+    Preconditions.checkArgument(
+        numVersionsToKeep >= 1, "%s must be positive", ViewProperties.VERSION_HISTORY_SIZE);
+
+    if (versions.size() > numVersionsToKeep) {

Review Comment:
   Why do this here? I don't think it belongs in the parser.



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(String.format("Failed to read json file: %s", file), e);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);
+
+    String location = JsonUtil.getString(LOCATION, node);
+    int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, node);
+    Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, node);
+
+    JsonNode versionsListNode = JsonUtil.get(VERSIONS, node);
+    Preconditions.checkArgument(
+        versionsListNode.isArray(), "Cannot parse versions from non-array: %s", versionsListNode);
+
+    List<ViewVersion> versions = Lists.newArrayListWithExpectedSize(versionsListNode.size());
+    ViewVersion currentVersion = null;
+    for (JsonNode versionNode : versionsListNode) {
+      ViewVersion version = ViewVersionParser.fromJson(versionNode);
+      if (version.versionId() == currentVersionId) {
+        currentVersion = version;
+      }
+
+      versions.add(version);
+    }
+
+    Preconditions.checkArgument(

Review Comment:
   Seems like a lot of these checks should be in the `ViewMetadata` class rather than here. Otherwise, you can construct invalid `ViewMetadata` instances.



-- 
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] stevenzwu commented on a diff in pull request #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {
+    return ImmutableViewMetadata.builder()
+        .formatVersion(DEFAULT_VIEW_FORMAT_VERSION)
+        .location(metadata.location())
+        .properties(metadata.properties())
+        .currentVersionId(metadata.currentVersionId())
+        .versions(metadata.versions())
+        .schemas(metadata.schemas())
+        .currentSchemaId(metadata.currentSchemaId())
+        .history(metadata.history());
+  }
+
+  @Value.Check
+  void check() {
+    Preconditions.checkState(versions().size() > 0, "Expecting non-empty version log");
+  }
+
+  public abstract int formatVersion();
+
+  public abstract String location();
+
+  public abstract Map<String, String> properties();
+
+  public abstract int currentVersionId();
+
+  public abstract List<ViewVersion> versions();
+
+  public abstract List<ViewHistoryEntry> history();
+
+  public abstract List<Schema> schemas();
+
+  public abstract Integer currentSchemaId();
+
+  public ViewVersion version(int versionId) {
+    return versionsById().get(versionId);
+  }
+
+  @Value.Derived
+  public ViewVersion currentVersion() {
+    return versionsById().get(currentVersionId());
+  }
+
+  @Value.Derived
+  public Map<Integer, ViewVersion> versionsById() {
+    return indexVersions(versions());
+  }
+
+  @Value.Derived
+  public Map<Integer, Schema> schemasById() {
+    return indexSchemas(schemas());
+  }
+
+  @Value.Derived
+  public Schema schema() {
+    return schemasById().get(currentSchemaId());
+  }
+
+  private static Map<Integer, ViewVersion> indexVersions(List<ViewVersion> versions) {
+    ImmutableMap.Builder<Integer, ViewVersion> builder = ImmutableMap.builder();
+    for (ViewVersion version : versions) {
+      builder.put(version.versionId(), version);
+    }
+
+    return builder.build();
+  }
+
+  private static Map<Integer, Schema> indexSchemas(List<Schema> schemas) {
+    if (schemas == null) {

Review Comment:
   could schemas be null? the spec says it is `optional`. but it is also says `A list of schemas, the same as the ‘schemas’ field from Iceberg table spec.`
   
   It is optional to support both v1 and v2 tables?



##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {

Review Comment:
   where is `ImmutableViewMetadata` defined? I couldn't find it in this PR or existing 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] nastra commented on a diff in pull request #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(String.format("Failed to read json file: %s", file), e);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);
+
+    String location = JsonUtil.getString(LOCATION, node);
+    int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, node);
+    Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, node);
+
+    JsonNode versionsListNode = JsonUtil.get(VERSIONS, node);
+    Preconditions.checkArgument(
+        versionsListNode.isArray(), "Cannot parse versions from non-array: %s", versionsListNode);
+
+    List<ViewVersion> versions = Lists.newArrayListWithExpectedSize(versionsListNode.size());
+    ViewVersion currentVersion = null;
+    for (JsonNode versionNode : versionsListNode) {
+      ViewVersion version = ViewVersionParser.fromJson(versionNode);
+      if (version.versionId() == currentVersionId) {
+        currentVersion = version;
+      }
+
+      versions.add(version);
+    }
+
+    Preconditions.checkArgument(
+        currentVersion != null,
+        "Cannot find version with %s=%s from %s",
+        CURRENT_VERSION_ID,
+        currentVersionId,
+        VERSIONS);
+
+    JsonNode versionLogNode = JsonUtil.get(VERSION_LOG, node);
+    Preconditions.checkArgument(
+        versionLogNode.isArray(), "Cannot parse version-log from non-array: %s", versionLogNode);
+    List<ViewHistoryEntry> historyEntries =
+        Lists.newArrayListWithExpectedSize(versionLogNode.size());
+    Iterator<JsonNode> versionLogIterator = versionLogNode.elements();
+    while (versionLogIterator.hasNext()) {
+      historyEntries.add(ViewHistoryEntryParser.fromJson(versionLogIterator.next()));
+    }
+
+    List<Schema> schemas;
+    Integer currentSchemaId;
+    Schema currentSchema = null;
+    JsonNode schemaArray = node.get(SCHEMAS);
+
+    Preconditions.checkArgument(
+        schemaArray.isArray(), "Cannot parse schemas from non-array: %s", schemaArray);
+    currentSchemaId = JsonUtil.getInt(CURRENT_SCHEMA_ID, node);
+
+    ImmutableList.Builder<Schema> builder = ImmutableList.builder();
+    for (JsonNode schemaNode : schemaArray) {
+      Schema schema = SchemaParser.fromJson(schemaNode);
+      if (schema.schemaId() == currentSchemaId) {
+        currentSchema = schema;
+      }
+
+      builder.add(schema);
+    }
+
+    Preconditions.checkArgument(
+        currentSchema != null,
+        "Cannot find schema with %s=%s from %s",
+        CURRENT_SCHEMA_ID,
+        currentSchemaId,
+        SCHEMAS);
+
+    schemas = builder.build();
+
+    int numVersionsToKeep =
+        PropertyUtil.propertyAsInt(
+            properties,
+            ViewProperties.VERSION_HISTORY_SIZE,
+            ViewProperties.VERSION_HISTORY_SIZE_DEFAULT);
+
+    Preconditions.checkArgument(
+        numVersionsToKeep >= 1, "%s must be positive", ViewProperties.VERSION_HISTORY_SIZE);
+
+    if (versions.size() > numVersionsToKeep) {

Review Comment:
   I agree that this shouldn't be here. The better option would be to put this into a special version of a `@Check` method that allows [normalization](https://immutables.github.io/immutable.html#normalization). This would allow to retain the number of versions that are desired in `ViewMetadata` directly



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/test/resources/ValidViewMetadata.json:
##########
@@ -0,0 +1,38 @@
+{
+  "format-version": 1,
+  "location": "s3://bucket/test/location",
+  "properties": {"some-key": "some-value"},
+  "current-schema-id": 1,
+  "schemas": [
+    {
+      "type": "struct",
+      "schema-id": 1,
+      "fields": [
+        {
+          "id": 1,
+          "name": "x",
+          "required": true,
+          "type": "long"
+        },
+        {
+          "id": 2,
+          "name": "y",
+          "required": true,
+          "type": "long",
+          "doc": "comment"
+        },
+        {
+          "id": 3,
+          "name": "z",
+          "required": true,
+          "type": "long"
+        }
+      ]
+    }
+  ],
+  "current-version-id": 2,
+  "versions": [
+    {"version-id":1,"timestamp-ms":4353,"summary":{"operation":"create"},"representations":[{"type":"sql","sql":"select 'foo' foo","dialect":"spark-sql","default-catalog":"some-catalog","default-namespace":[],"field-aliases":[],"field-comments":[], "schema-id":1}]},

Review Comment:
   Done! 



-- 
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 #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+
+public class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  // stored metadata
+  private final int formatVersion;
+  private final String location;
+  private final Map<String, String> properties;
+  private final int currentVersionId;
+  private final Integer currentSchemaId;
+  private final List<ViewVersion> versions;
+  private final Map<Integer, ViewVersion> versionsById;
+  private final List<ViewHistoryEntry> versionLog;
+  private final List<Schema> schemas;
+  private final Map<Integer, Schema> schemasById;
+  private final String metadataFileLocation;

Review Comment:
   Shouldn't store metadataFileLocation, I'll fix 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] nastra commented on a diff in pull request #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {

Review Comment:
   I think we should check for null on `metadata` + add a test



##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(String.format("Failed to read json file: %s", file), e);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");

Review Comment:
   would be good to add a test for this



##########
core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * 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 java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+
+public class TestViewMetadata {

Review Comment:
   should this be called `TestViewMetadataParser` instead?



##########
core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * 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 java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+
+public class TestViewMetadata {
+
+  private static ViewVersion version1 =
+      ImmutableViewVersion.builder()
+          .versionId(1)
+          .timestampMillis(4353L)
+          .summary(ImmutableMap.of("operation", "create"))
+          .schemaId(1)
+          .addRepresentations(
+              ImmutableSQLViewRepresentation.builder()
+                  .sql("select 'foo' foo")
+                  .dialect("spark-sql")
+                  .defaultCatalog("some-catalog")
+                  .build())
+          .build();
+
+  private static ViewHistoryEntry historyEntry1 =
+      ImmutableViewHistoryEntry.builder().timestampMillis(4353L).versionId(1).build();
+
+  private static ViewVersion version2 =
+      ImmutableViewVersion.builder()
+          .versionId(2)
+          .schemaId(1)
+          .timestampMillis(5555L)
+          .summary(ImmutableMap.of("operation", "replace"))
+          .addRepresentations(
+              ImmutableSQLViewRepresentation.builder()
+                  .sql("select 1 id, 'abc' data")
+                  .defaultCatalog("some-catalog")
+                  .dialect("spark-sql")
+                  .build())
+          .build();
+
+  private static ViewHistoryEntry historyEntry2 =
+      ImmutableViewHistoryEntry.builder().timestampMillis(5555L).versionId(2).build();
+
+  private static final Schema TEST_SCHEMA =
+      new Schema(
+          1,
+          Types.NestedField.required(1, "x", Types.LongType.get()),
+          Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+          Types.NestedField.required(3, "z", Types.LongType.get()));
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> parameters() {
+    return Arrays.asList(new Object[][] {});
+  }
+
+  @Test
+  public void testReadAndWriteValidViewMetadata() throws Exception {
+    String json = readViewMetadataInputFile("view/ValidViewMetadata.json");
+    ViewMetadata expectedViewMetadata =
+        ImmutableViewMetadata.builder()
+            .currentSchemaId(1)
+            .schemas(ImmutableList.of(TEST_SCHEMA))
+            .versions(ImmutableList.of(version1, version2))
+            .history(ImmutableList.of(historyEntry1, historyEntry2))
+            .location("s3://bucket/test/location")
+            .properties(ImmutableMap.of("some-key", "some-value"))
+            .currentVersionId(2)
+            .formatVersion(1)
+            .build();
+    assertSameViewMetadata(expectedViewMetadata, ViewMetadataParser.fromJson(json));

Review Comment:
   nit: `assertSame` usually suggests that there's some sort of identity comparison between two objects. 



##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {

Review Comment:
   I think what @jackye1995 means here is to move this method to the bottom after all the API definitions (`formatVersion()` / `location()` / ...)



##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(String.format("Failed to read json file: %s", file), e);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);
+
+    String location = JsonUtil.getString(LOCATION, node);
+    int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, node);
+    Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, node);
+
+    JsonNode versionsListNode = JsonUtil.get(VERSIONS, node);
+    Preconditions.checkArgument(
+        versionsListNode.isArray(), "Cannot parse versions from non-array: %s", versionsListNode);
+
+    List<ViewVersion> versions = Lists.newArrayListWithExpectedSize(versionsListNode.size());
+    ViewVersion currentVersion = null;
+    for (JsonNode versionNode : versionsListNode) {
+      ViewVersion version = ViewVersionParser.fromJson(versionNode);
+      if (version.versionId() == currentVersionId) {
+        currentVersion = version;
+      }
+
+      versions.add(version);
+    }
+
+    Preconditions.checkArgument(
+        currentVersion != null,
+        "Cannot find version with %s=%s from %s",
+        CURRENT_VERSION_ID,
+        currentVersionId,
+        VERSIONS);
+
+    JsonNode versionLogNode = JsonUtil.get(VERSION_LOG, node);
+    Preconditions.checkArgument(
+        versionLogNode.isArray(), "Cannot parse version-log from non-array: %s", versionLogNode);
+    List<ViewHistoryEntry> historyEntries =
+        Lists.newArrayListWithExpectedSize(versionLogNode.size());
+    Iterator<JsonNode> versionLogIterator = versionLogNode.elements();
+    while (versionLogIterator.hasNext()) {
+      historyEntries.add(ViewHistoryEntryParser.fromJson(versionLogIterator.next()));
+    }
+
+    List<Schema> schemas;
+    Integer currentSchemaId;
+    Schema currentSchema = null;
+    JsonNode schemaArray = node.get(SCHEMAS);
+
+    Preconditions.checkArgument(
+        schemaArray.isArray(), "Cannot parse schemas from non-array: %s", schemaArray);
+    currentSchemaId = JsonUtil.getInt(CURRENT_SCHEMA_ID, node);
+
+    ImmutableList.Builder<Schema> builder = ImmutableList.builder();
+    for (JsonNode schemaNode : schemaArray) {
+      Schema schema = SchemaParser.fromJson(schemaNode);
+      if (schema.schemaId() == currentSchemaId) {
+        currentSchema = schema;
+      }
+
+      builder.add(schema);
+    }
+
+    Preconditions.checkArgument(
+        currentSchema != null,
+        "Cannot find schema with %s=%s from %s",
+        CURRENT_SCHEMA_ID,
+        currentSchemaId,
+        SCHEMAS);
+
+    schemas = builder.build();
+
+    int numVersionsToKeep =
+        PropertyUtil.propertyAsInt(
+            properties,
+            ViewProperties.VERSION_HISTORY_SIZE,
+            ViewProperties.VERSION_HISTORY_SIZE_DEFAULT);
+
+    Preconditions.checkArgument(
+        numVersionsToKeep >= 1, "%s must be positive", ViewProperties.VERSION_HISTORY_SIZE);
+
+    if (versions.size() > numVersionsToKeep) {
+      versions = versions.subList(versions.size() - numVersionsToKeep, versions.size());
+      historyEntries =
+          historyEntries.subList(historyEntries.size() - numVersionsToKeep, historyEntries.size());
+    }
+
+    return ImmutableViewMetadata.builder()
+        .location(location)
+        .currentVersionId(currentVersionId)
+        .properties(properties)
+        .versions(versions)
+        .schemas(schemas)
+        .currentSchemaId(currentSchemaId)
+        .history(historyEntries)
+        .formatVersion(formatVersion)
+        .build();
+  }
+
+  static ViewMetadata fromJson(String json) {
+    Preconditions.checkArgument(json != null, "Cannot parse view metadata from null string");
+    return JsonUtil.parse(json, ViewMetadataParser::fromJson);
+  }
+
+  private static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new UncheckedIOException(
+          String.format("Failed to write json to file: %s", outputFile), e);
+    }
+  }
+
+  private ViewMetadataParser() {}

Review Comment:
   nit: I think this should be at the top



##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {
+    return ImmutableViewMetadata.builder()
+        .formatVersion(DEFAULT_VIEW_FORMAT_VERSION)
+        .location(metadata.location())
+        .properties(metadata.properties())
+        .currentVersionId(metadata.currentVersionId())
+        .versions(metadata.versions())
+        .schemas(metadata.schemas())
+        .currentSchemaId(metadata.currentSchemaId())
+        .history(metadata.history());
+  }
+
+  @Value.Check
+  void check() {
+    Preconditions.checkState(versions().size() > 0, "Expecting non-empty version log");
+  }
+
+  public abstract int formatVersion();
+
+  public abstract String location();
+
+  public abstract Map<String, String> properties();
+
+  public abstract int currentVersionId();
+
+  public abstract List<ViewVersion> versions();
+
+  public abstract List<ViewHistoryEntry> history();
+
+  public abstract List<Schema> schemas();
+
+  public abstract Integer currentSchemaId();
+
+  public ViewVersion version(int versionId) {
+    return versionsById().get(versionId);
+  }
+
+  @Value.Derived
+  public ViewVersion currentVersion() {
+    return versionsById().get(currentVersionId());
+  }
+
+  @Value.Derived
+  public Map<Integer, ViewVersion> versionsById() {
+    return indexVersions(versions());
+  }
+
+  @Value.Derived
+  public Map<Integer, Schema> schemasById() {
+    return indexSchemas(schemas());
+  }
+
+  @Value.Derived
+  public Schema schema() {
+    return schemasById().get(currentSchemaId());
+  }
+
+  private static Map<Integer, ViewVersion> indexVersions(List<ViewVersion> versions) {

Review Comment:
   nit: I think `indexVersions` and `indexSchemas` don't need to be `static`



##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {

Review Comment:
   is this used anywhere? Also we can probably just use `ImmutableViewMetadata.builder().from(metadata).xyz().build()` here



##########
core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * 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 java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;

Review Comment:
   I think it's worth changing this new test to using JUnit5



##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {

Review Comment:
   nit: maybe move this before the other `toJson(..)` method



##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {

Review Comment:
   or is the goal here to just create a copy? In that case we can also just do `ImmutableViewMetadata.copyOf(metadata)`



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

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

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


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


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewProperties.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/** View properties that can be set during CREATE/REPLACE view or using updateProperties API. */
+public class ViewProperties {
+  public static final String COMMIT_NUM_RETRIES = "commit.retry.num-retries";

Review Comment:
   These properties are not used at this moment, I think we can add them when we add base view catalog impl 



-- 
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 #6559: Core: View core parser implementations

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

   > Thanks for continuing this effort! Added some nit comments. Looks like it's still missing tests?
   
   Sure thing! yes there's still aspects missing, like schema handling and tests. sorry for the confusion I was just rebasing and pushing, the PR's not quite ready yet.


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

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

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


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


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {
+    return ImmutableViewMetadata.builder()
+        .formatVersion(DEFAULT_VIEW_FORMAT_VERSION)
+        .location(metadata.location())
+        .properties(metadata.properties())
+        .currentVersionId(metadata.currentVersionId())
+        .versions(metadata.versions())
+        .schemas(metadata.schemas())
+        .currentSchemaId(metadata.currentSchemaId())
+        .history(metadata.history());
+  }
+
+  @Value.Check
+  void check() {
+    Preconditions.checkState(versions().size() > 0, "Expecting non-empty version log");
+  }
+
+  public abstract int formatVersion();

Review Comment:
   nit: can we have variables above methods?



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

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

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


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


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {

Review Comment:
   I guess this is trying to follow `TableMetadata.buildFrom()`?



-- 
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 #6559: Core: View core parser implementations

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


##########
core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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 java.util.Arrays;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.types.Types;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestViewMetadataParser extends ParserTestBase<ViewMetadata> {
+  private static ViewVersion version1 =
+      BaseViewVersion.builder()
+          .versionId(1)
+          .timestampMillis(4353L)
+          .addRepresentation(
+              BaseSQLViewRepresentation.builder().query("select 'foo' foo").schemaId(1).build())
+          .build();
+
+  private static ViewHistoryEntry historyEntry1 = BaseViewHistoryEntry.of(4353L, 1);
+
+  private static final Schema TEST_SCHEMA =
+      new Schema(
+          1,
+          Types.NestedField.required(1, "x", Types.LongType.get()),
+          Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+          Types.NestedField.required(3, "z", Types.LongType.get()));
+
+  private static ViewVersion version2 =
+      BaseViewVersion.builder()
+          .versionId(2)
+          .timestampMillis(5555L)

Review Comment:
   I'll add more tests for view metadata parsing. One thing I did just a bit differently than the original PR is move the JSON to a separate file in resources



-- 
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 #6559: Core: View core parser implementations

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


##########
api/src/main/java/org/apache/iceberg/view/SQLViewRepresentation.java:
##########
@@ -41,8 +40,8 @@ default Type type() {
   /** The default namespace when the view is created. */
   Namespace defaultNamespace();
 
-  /** The query output schema at version create time, without aliases. */
-  Schema schema();

Review Comment:
   why change it to schema ID? If we know the schema ID, we should also be able to retrieve the Schema object



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {
+    return ImmutableViewMetadata.builder()
+        .formatVersion(DEFAULT_VIEW_FORMAT_VERSION)
+        .location(metadata.location())
+        .properties(metadata.properties())
+        .currentVersionId(metadata.currentVersionId())
+        .versions(metadata.versions())
+        .schemas(metadata.schemas())
+        .currentSchemaId(metadata.currentSchemaId())
+        .history(metadata.history());
+  }
+
+  @Value.Check
+  void check() {
+    Preconditions.checkState(versions().size() > 0, "Expecting non-empty version log");
+  }
+
+  public abstract int formatVersion();
+
+  public abstract String location();
+
+  public abstract Map<String, String> properties();
+
+  public abstract int currentVersionId();
+
+  public abstract List<ViewVersion> versions();
+
+  public abstract List<ViewHistoryEntry> history();
+
+  public abstract List<Schema> schemas();
+
+  public abstract Integer currentSchemaId();
+
+  public ViewVersion version(int versionId) {
+    return versionsById().get(versionId);
+  }
+
+  @Value.Derived
+  public ViewVersion currentVersion() {
+    return versionsById().get(currentVersionId());
+  }
+
+  @Value.Derived
+  public Map<Integer, ViewVersion> versionsById() {
+    return indexVersions(versions());
+  }
+
+  @Value.Derived
+  public Map<Integer, Schema> schemasById() {
+    return indexSchemas(schemas());
+  }
+
+  @Value.Derived
+  public Schema schema() {
+    return schemasById().get(currentSchemaId());
+  }
+
+  private static Map<Integer, ViewVersion> indexVersions(List<ViewVersion> versions) {

Review Comment:
   Isn't static preferred? If someone makes a change in the future, I'd rather they be static since that is more strict. I could see changing from `private` to a more open visibility, in which case we would prefer static.



-- 
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 #6559: Core: View core parser implementations

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

   Agreed @jackye1995 we can break this down further for easier review. I'll raise the version, representation and history entry PRs separately.


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

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

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


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


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,301 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+
+public class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  // stored metadata
+  private final int formatVersion;
+  private final String location;
+  private final Map<String, String> properties;
+  private final int currentVersionId;
+  private final Integer currentSchemaId;
+  private final List<ViewVersion> versions;
+  private final Map<Integer, ViewVersion> versionsById;
+  private final List<ViewHistoryEntry> versionLog;
+  private final List<Schema> schemas;
+  private final Map<Integer, Schema> schemasById;
+  private final String metadataFileLocation;

Review Comment:
   Shouldn't store metadataFileLocation in the metadata file, I'll fix 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] jackye1995 commented on a diff in pull request #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {
+    return ImmutableViewMetadata.builder()
+        .formatVersion(DEFAULT_VIEW_FORMAT_VERSION)
+        .location(metadata.location())
+        .properties(metadata.properties())
+        .currentVersionId(metadata.currentVersionId())
+        .versions(metadata.versions())
+        .schemas(metadata.schemas())
+        .currentSchemaId(metadata.currentSchemaId())
+        .history(metadata.history());
+  }
+
+  @Value.Check
+  void check() {
+    Preconditions.checkState(versions().size() > 0);
+    checkVersionLog(history());
+  }
+
+  public abstract int formatVersion();
+
+  public abstract String location();
+
+  public abstract Map<String, String> properties();
+
+  public abstract int currentVersionId();
+
+  public abstract List<ViewVersion> versions();
+
+  public abstract List<ViewHistoryEntry> history();
+
+  @Nullable
+  public abstract List<Schema> schemas();
+
+  @Nullable
+  public abstract Integer currentSchemaId();
+
+  public ViewVersion version(int versionId) {
+    return versionsById().get(versionId);
+  }
+
+  @Value.Derived
+  public ViewVersion currentVersion() {
+    return versionsById().get(currentVersionId());
+  }
+
+  @Value.Derived
+  public Map<Integer, ViewVersion> versionsById() {
+    return indexVersions(versions());
+  }
+
+  @Value.Derived
+  public Map<Integer, Schema> schemasById() {
+    return indexSchemas(schemas());
+  }
+
+  @Value.Derived
+  public Schema schema() {
+    return schemasById().get(currentSchemaId());
+  }
+
+  private static void checkVersionLog(List<ViewHistoryEntry> versionLog) {
+    Preconditions.checkState(versionLog.size() > 0);

Review Comment:
   Nit: missing message 



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {
+    return ImmutableViewMetadata.builder()
+        .formatVersion(DEFAULT_VIEW_FORMAT_VERSION)
+        .location(metadata.location())
+        .properties(metadata.properties())
+        .currentVersionId(metadata.currentVersionId())
+        .versions(metadata.versions())
+        .schemas(metadata.schemas())
+        .currentSchemaId(metadata.currentSchemaId())
+        .history(metadata.history());
+  }
+
+  @Value.Check
+  void check() {
+    Preconditions.checkState(versions().size() > 0, "Expecting non-empty version log");
+    checkVersionLog(history());
+  }
+
+  public abstract int formatVersion();
+
+  public abstract String location();
+
+  public abstract Map<String, String> properties();
+
+  public abstract int currentVersionId();
+
+  public abstract List<ViewVersion> versions();
+
+  public abstract List<ViewHistoryEntry> history();
+
+  @Nullable
+  public abstract List<Schema> schemas();
+
+  @Nullable
+  public abstract Integer currentSchemaId();
+
+  public ViewVersion version(int versionId) {
+    return versionsById().get(versionId);
+  }
+
+  @Value.Derived
+  public ViewVersion currentVersion() {
+    return versionsById().get(currentVersionId());
+  }
+
+  @Value.Derived
+  public Map<Integer, ViewVersion> versionsById() {
+    return indexVersions(versions());
+  }
+
+  @Value.Derived
+  public Map<Integer, Schema> schemasById() {
+    return indexSchemas(schemas());
+  }
+
+  @Value.Derived
+  public Schema schema() {
+    return schemasById().get(currentSchemaId());
+  }
+
+  private static void checkVersionLog(List<ViewHistoryEntry> versionLog) {
+    Preconditions.checkState(versionLog.size() > 0, "Expecting non-empty version log");

Review Comment:
   ```suggestion
       Preconditions.checkState(versionLog.size() > 0, "Expecting non-empty history log");
   ```



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {
+    return ImmutableViewMetadata.builder()
+        .formatVersion(DEFAULT_VIEW_FORMAT_VERSION)
+        .location(metadata.location())
+        .properties(metadata.properties())
+        .currentVersionId(metadata.currentVersionId())
+        .versions(metadata.versions())
+        .schemas(metadata.schemas())
+        .currentSchemaId(metadata.currentSchemaId())
+        .history(metadata.history());
+  }
+
+  @Value.Check
+  void check() {
+    Preconditions.checkState(versions().size() > 0, "Expecting non-empty version log");
+    checkVersionLog(history());
+  }
+
+  public abstract int formatVersion();
+
+  public abstract String location();
+
+  public abstract Map<String, String> properties();
+
+  public abstract int currentVersionId();
+
+  public abstract List<ViewVersion> versions();
+
+  public abstract List<ViewHistoryEntry> history();
+
+  @Nullable
+  public abstract List<Schema> schemas();
+
+  @Nullable
+  public abstract Integer currentSchemaId();
+
+  public ViewVersion version(int versionId) {
+    return versionsById().get(versionId);
+  }
+
+  @Value.Derived
+  public ViewVersion currentVersion() {
+    return versionsById().get(currentVersionId());
+  }
+
+  @Value.Derived
+  public Map<Integer, ViewVersion> versionsById() {
+    return indexVersions(versions());
+  }
+
+  @Value.Derived
+  public Map<Integer, Schema> schemasById() {
+    return indexSchemas(schemas());
+  }
+
+  @Value.Derived
+  public Schema schema() {
+    return schemasById().get(currentSchemaId());
+  }
+
+  private static void checkVersionLog(List<ViewHistoryEntry> versionLog) {
+    Preconditions.checkState(versionLog.size() > 0, "Expecting non-empty version log");

Review Comment:
   I don't think this is a good idea. Why would we want to fail to read a view if the log is missing? I don't think that's a good reason to fail.



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    if (metadata.schemas() != null && !metadata.schemas().isEmpty()) {
+      generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+      generator.writeArrayFieldStart(SCHEMAS);
+      for (Schema schema : metadata.schemas()) {
+        SchemaParser.toJson(schema, generator);
+      }
+
+      generator.writeEndArray();
+    }
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+    generator.writeEndArray();
+
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to read file: %s", file);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);
+
+    String location = JsonUtil.getString(LOCATION, node);
+
+    int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, node);
+
+    Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, node);
+    JsonNode versionsListNode = JsonUtil.get(VERSIONS, node);
+    List<ViewVersion> versions = Lists.newArrayListWithExpectedSize(versionsListNode.size());
+    Iterator<JsonNode> iterator = versionsListNode.elements();

Review Comment:
   +1. We always check the types to give clear error messages.



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(String.format("Failed to read json file: %s", file), e);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);
+
+    String location = JsonUtil.getString(LOCATION, node);
+    int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, node);
+    Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, node);
+
+    JsonNode versionsListNode = JsonUtil.get(VERSIONS, node);
+    Preconditions.checkArgument(
+        versionsListNode.isArray(), "Cannot parse versions from non-array: %s", versionsListNode);
+
+    List<ViewVersion> versions = Lists.newArrayListWithExpectedSize(versionsListNode.size());
+    ViewVersion currentVersion = null;
+    for (JsonNode versionNode : versionsListNode) {
+      ViewVersion version = ViewVersionParser.fromJson(versionNode);
+      if (version.versionId() == currentVersionId) {
+        currentVersion = version;
+      }
+
+      versions.add(version);
+    }
+
+    Preconditions.checkArgument(
+        currentVersion != null,
+        "Cannot find version with %s=%s from %s",
+        CURRENT_VERSION_ID,
+        currentVersionId,
+        VERSIONS);
+
+    JsonNode versionLogNode = JsonUtil.get(VERSION_LOG, node);
+    Preconditions.checkArgument(
+        versionLogNode.isArray(), "Cannot parse version-log from non-array: %s", versionLogNode);
+    List<ViewHistoryEntry> historyEntries =
+        Lists.newArrayListWithExpectedSize(versionLogNode.size());
+    Iterator<JsonNode> versionLogIterator = versionLogNode.elements();
+    while (versionLogIterator.hasNext()) {
+      historyEntries.add(ViewHistoryEntryParser.fromJson(versionLogIterator.next()));
+    }
+
+    List<Schema> schemas;
+    Integer currentSchemaId;
+    Schema currentSchema = null;
+    JsonNode schemaArray = node.get(SCHEMAS);
+
+    Preconditions.checkArgument(
+        schemaArray.isArray(), "Cannot parse schemas from non-array: %s", schemaArray);
+    currentSchemaId = JsonUtil.getInt(CURRENT_SCHEMA_ID, node);
+
+    ImmutableList.Builder<Schema> builder = ImmutableList.builder();
+    for (JsonNode schemaNode : schemaArray) {
+      Schema schema = SchemaParser.fromJson(schemaNode);
+      if (schema.schemaId() == currentSchemaId) {
+        currentSchema = schema;
+      }
+
+      builder.add(schema);
+    }
+
+    Preconditions.checkArgument(
+        currentSchema != null,
+        "Cannot find schema with %s=%s from %s",
+        CURRENT_SCHEMA_ID,
+        currentSchemaId,
+        SCHEMAS);
+
+    schemas = builder.build();
+
+    int numVersionsToKeep =
+        PropertyUtil.propertyAsInt(
+            properties,
+            ViewProperties.VERSION_HISTORY_SIZE,
+            ViewProperties.VERSION_HISTORY_SIZE_DEFAULT);
+
+    Preconditions.checkArgument(
+        numVersionsToKeep >= 1, "%s must be positive", ViewProperties.VERSION_HISTORY_SIZE);

Review Comment:
   nit: add details of the received value of `numVersionsToKeep` in error message



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(String.format("Failed to read json file: %s", file), e);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);

Review Comment:
   I agree that this should go into `check()`, because otherwise you could create an invalid `ViewMetadata` object



##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(String.format("Failed to read json file: %s", file), e);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);
+
+    String location = JsonUtil.getString(LOCATION, node);
+    int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, node);
+    Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, node);
+
+    JsonNode versionsListNode = JsonUtil.get(VERSIONS, node);
+    Preconditions.checkArgument(
+        versionsListNode.isArray(), "Cannot parse versions from non-array: %s", versionsListNode);
+
+    List<ViewVersion> versions = Lists.newArrayListWithExpectedSize(versionsListNode.size());
+    ViewVersion currentVersion = null;
+    for (JsonNode versionNode : versionsListNode) {
+      ViewVersion version = ViewVersionParser.fromJson(versionNode);
+      if (version.versionId() == currentVersionId) {
+        currentVersion = version;
+      }
+
+      versions.add(version);
+    }
+
+    Preconditions.checkArgument(

Review Comment:
   +1



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+    generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    generator.writeArrayFieldStart(SCHEMAS);
+    for (Schema schema : metadata.schemas()) {
+      SchemaParser.toJson(schema, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(String.format("Failed to read json file: %s", file), e);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);
+
+    String location = JsonUtil.getString(LOCATION, node);
+    int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, node);
+    Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, node);
+
+    JsonNode versionsListNode = JsonUtil.get(VERSIONS, node);
+    Preconditions.checkArgument(
+        versionsListNode.isArray(), "Cannot parse versions from non-array: %s", versionsListNode);
+
+    List<ViewVersion> versions = Lists.newArrayListWithExpectedSize(versionsListNode.size());
+    ViewVersion currentVersion = null;
+    for (JsonNode versionNode : versionsListNode) {
+      ViewVersion version = ViewVersionParser.fromJson(versionNode);
+      if (version.versionId() == currentVersionId) {
+        currentVersion = version;
+      }
+
+      versions.add(version);
+    }
+
+    Preconditions.checkArgument(
+        currentVersion != null,
+        "Cannot find version with %s=%s from %s",
+        CURRENT_VERSION_ID,
+        currentVersionId,
+        VERSIONS);
+
+    JsonNode versionLogNode = JsonUtil.get(VERSION_LOG, node);
+    Preconditions.checkArgument(
+        versionLogNode.isArray(), "Cannot parse version-log from non-array: %s", versionLogNode);
+    List<ViewHistoryEntry> historyEntries =
+        Lists.newArrayListWithExpectedSize(versionLogNode.size());
+    Iterator<JsonNode> versionLogIterator = versionLogNode.elements();
+    while (versionLogIterator.hasNext()) {
+      historyEntries.add(ViewHistoryEntryParser.fromJson(versionLogIterator.next()));
+    }
+
+    List<Schema> schemas;
+    Integer currentSchemaId;
+    Schema currentSchema = null;
+    JsonNode schemaArray = node.get(SCHEMAS);
+
+    Preconditions.checkArgument(
+        schemaArray.isArray(), "Cannot parse schemas from non-array: %s", schemaArray);
+    currentSchemaId = JsonUtil.getInt(CURRENT_SCHEMA_ID, node);
+
+    ImmutableList.Builder<Schema> builder = ImmutableList.builder();
+    for (JsonNode schemaNode : schemaArray) {
+      Schema schema = SchemaParser.fromJson(schemaNode);
+      if (schema.schemaId() == currentSchemaId) {
+        currentSchema = schema;
+      }
+
+      builder.add(schema);
+    }
+
+    Preconditions.checkArgument(
+        currentSchema != null,
+        "Cannot find schema with %s=%s from %s",
+        CURRENT_SCHEMA_ID,
+        currentSchemaId,
+        SCHEMAS);
+
+    schemas = builder.build();
+
+    int numVersionsToKeep =
+        PropertyUtil.propertyAsInt(
+            properties,
+            ViewProperties.VERSION_HISTORY_SIZE,
+            ViewProperties.VERSION_HISTORY_SIZE_DEFAULT);
+
+    Preconditions.checkArgument(
+        numVersionsToKeep >= 1, "%s must be positive", ViewProperties.VERSION_HISTORY_SIZE);
+
+    if (versions.size() > numVersionsToKeep) {

Review Comment:
   I agree that this shouldn't be here. The better option would be to put this into a special version of a `@Check` method that allows normalization. This would allow to retain the number of versions that are desired in `ViewMetadata` directly



-- 
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 #6559: Core: View core parser implementations

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

   Thanks @nastra I'll be taking these suggestions in all the split PRs I'm raising. Agreed, more tests on nullability/missing fields are needed, and now that we use Immutable dependency in the metrics implementation, we have a good precedent to use it here as well which will simplify a lot of the boilerplate 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] rdblue commented on a diff in pull request #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.util.JsonUtil;
+
+public class SQLViewRepresentationParser {
+  private enum Field {
+    SQL("sql"),
+    DIALECT("dialect"),
+    SCHEMA_ID("schema-id"),
+    DEFAULT_CATALOG("default-catalog"),
+    DEFAULT_NAMESPACE("default-namespace"),
+    FIELD_ALIASES("field-aliases"),
+    FIELD_COMMENTS("field-comments");
+
+    private final String name;
+
+    Field(String name) {
+      this.name = name;
+    }
+  }
+
+  static void toJson(SQLViewRepresentation view, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+    generator.writeStringField(
+        ViewRepresentationParser.Field.TYPE.fieldName(), view.type().typeName());
+    generator.writeStringField(SQLViewRepresentationParser.Field.SQL.name, view.query());

Review Comment:
   I think this can just be `Field.SQL.name` right?



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

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

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


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


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    if (metadata.currentSchemaId() != null) {
+      generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+    }
+
+    if (metadata.schemas() != null) {
+      JsonUtil.writeObjectList(SCHEMAS, metadata.schemas(), SchemaParser::toJson, generator);
+    }
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    JsonUtil.writeObjectList(VERSIONS, metadata.versions(), ViewVersionParser::toJson, generator);
+    JsonUtil.writeObjectList(
+        VERSION_LOG, metadata.history(), ViewHistoryEntryParser::toJson, generator);
+
+    generator.writeEndObject();
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to read file: %s", file);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    return fromJson(node, null);
+  }
+
+  public static ViewMetadata fromJson(JsonNode node, String metadataFileLocation) {
+
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);
+
+    String location = JsonUtil.getString(LOCATION, node);
+
+    int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, node);
+
+    Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, node);
+
+    List<ViewVersion> versions =
+        JsonUtil.getObjectList(VERSIONS, node, ViewVersionParser::fromJson);
+
+    List<ViewHistoryEntry> history =
+        JsonUtil.getObjectList(VERSION_LOG, node, ViewHistoryEntryParser::fromJson);
+
+    int numVersionsToKeep =
+        PropertyUtil.propertyAsInt(
+            properties,
+            ViewProperties.VERSION_HISTORY_SIZE,
+            ViewProperties.VERSION_HISTORY_SIZE_DEFAULT);
+
+    versions = versions.subList(0, Math.min(numVersionsToKeep, versions.size()));
+
+    List<Schema> schemas = null;
+    Integer currentSchemaId = null;
+    if (node.has(SCHEMAS)) {
+      JsonNode schemaArray = node.get(SCHEMAS);
+      Schema currentSchema = null;
+      Preconditions.checkArgument(
+          schemaArray.isArray(), "Cannot parse schemas from non-array: %s", schemaArray);
+      // current schema ID is required when the schema array is present
+      currentSchemaId = JsonUtil.getInt(CURRENT_SCHEMA_ID, node);
+      // parse the schema array
+      ImmutableList.Builder<Schema> builder = ImmutableList.builder();
+      for (JsonNode schemaNode : schemaArray) {
+        Schema schema = SchemaParser.fromJson(schemaNode);
+        if (schema.schemaId() == currentSchemaId) {
+          currentSchema = schema;
+        }
+        builder.add(schema);
+      }

Review Comment:
   The doubt I still have is I think schemas and current schema should always be set, and thus the spec should be reflected so that it's required. Maybe there's a case that we don't want schema for the entire view. But at least in my mind for any SQL representation in the view, the schema-id should be required (which then means there must be a schema list considering we only support SQL representations)



-- 
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 #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+
+    generator.writeEndArray();
+    generator.writeEndObject();
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to read file: %s", file);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(

Review Comment:
   Should we also check null 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 a diff in pull request #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Ordering;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public abstract class ViewMetadata implements Serializable {
+  static final int DEFAULT_VIEW_FORMAT_VERSION = 1;
+  static final int SUPPORTED_VIEW_FORMAT_VERSION = 1;
+
+  public static ImmutableViewMetadata.Builder buildFrom(ViewMetadata metadata) {

Review Comment:
   Nit: Can we move methods below variables?



-- 
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 #6559: Core: View core parser implementations

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    if (metadata.schemas() != null && !metadata.schemas().isEmpty()) {
+      generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+      generator.writeArrayFieldStart(SCHEMAS);
+      for (Schema schema : metadata.schemas()) {
+        SchemaParser.toJson(schema, generator);
+      }
+
+      generator.writeEndArray();
+    }

Review Comment:
   I remember there is a use case in Netflix that requires it to be optional that we discussed in a previous PR, do you remember 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 #6559: Core: View core parser implementations

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


##########
core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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 java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.runners.Parameterized;
+
+public class TestViewMetadata {
+
+  private static ViewVersion version1 =
+      ImmutableViewVersion.builder()
+          .versionId(1)
+          .timestampMillis(4353L)
+          .operation("create")
+          .addRepresentations(
+              ImmutableSQLViewRepresentation.builder()
+                  .sql("select 'foo' foo")
+                  .dialect("spark-sql")
+                  .schemaId(1)
+                  .defaultCatalog("some-catalog")
+                  .build())
+          .build();
+  private static ViewHistoryEntry historyEntry1 =
+      ImmutableViewHistoryEntry.builder().timestampMillis(4353L).versionId(1).build();
+
+  private static ViewVersion version2 =
+      ImmutableViewVersion.builder()
+          .versionId(2)
+          .timestampMillis(5555L)
+          .operation("replace")
+          .addRepresentations(
+              ImmutableSQLViewRepresentation.builder()
+                  .sql("select 1 id, 'abc' data")
+                  .defaultCatalog("some-catalog")
+                  .dialect("spark-sql")
+                  .schemaId(1)
+                  .build())
+          .build();
+  private static ViewHistoryEntry historyEntry2 =
+      ImmutableViewHistoryEntry.builder().timestampMillis(5555L).versionId(2).build();
+  private static final Schema TEST_SCHEMA =
+      new Schema(
+          1,
+          Types.NestedField.required(1, "x", Types.LongType.get()),
+          Types.NestedField.required(2, "y", Types.LongType.get(), "comment"),
+          Types.NestedField.required(3, "z", Types.LongType.get()));
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> parameters() {
+    return Arrays.asList(new Object[][] {});
+  }
+
+  @Test

Review Comment:
   More tests needed for null etc.



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);

Review Comment:
   Fixed! 



-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);

Review Comment:
   I think I'm missing something, `JsonUtil.generate` is used for generating the Json String. The simplification mentioned is used below in `toJson`. The internalWrite is used for writing the actual Json file so the generator is initialized with the output stream writer and then we use toJson and flush. Let me know if further simplification can be done 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 #6559: Core: View metadata implementation

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+class ViewMetadataParser {
+
+  // visible for testing
+  static final String FORMAT_VERSION = "format-version";
+  static final String LOCATION = "location";
+  static final String CURRENT_VERSION_ID = "current-version-id";
+  static final String VERSIONS = "versions";
+  static final String VERSION_LOG = "version-log";
+  static final String PROPERTIES = "properties";
+  static final String SCHEMAS = "schemas";
+  static final String CURRENT_SCHEMA_ID = "current-schema-id";
+
+  public static void overwrite(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, true);
+  }
+
+  public static void write(ViewMetadata metadata, OutputFile outputFile) {
+    internalWrite(metadata, outputFile, false);
+  }
+
+  public static void internalWrite(
+      ViewMetadata metadata, OutputFile outputFile, boolean overwrite) {
+    OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create();
+    try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) {
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(metadata, generator);
+      generator.flush();
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to write json to file: %s", outputFile);
+    }
+  }
+
+  public static void toJson(ViewMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeNumberField(FORMAT_VERSION, metadata.formatVersion());
+    generator.writeStringField(LOCATION, metadata.location());
+
+    JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator);
+
+    if (metadata.schemas() != null && !metadata.schemas().isEmpty()) {
+      generator.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId());
+      generator.writeArrayFieldStart(SCHEMAS);
+      for (Schema schema : metadata.schemas()) {
+        SchemaParser.toJson(schema, generator);
+      }
+
+      generator.writeEndArray();
+    }
+
+    generator.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId());
+    generator.writeArrayFieldStart(VERSIONS);
+    for (ViewVersion version : metadata.versions()) {
+      ViewVersionParser.toJson(version, generator);
+    }
+
+    generator.writeEndArray();
+
+    generator.writeArrayFieldStart(VERSION_LOG);
+    for (ViewHistoryEntry viewHistoryEntry : metadata.history()) {
+      ViewHistoryEntryParser.toJson(viewHistoryEntry, generator);
+    }
+    generator.writeEndArray();
+
+    generator.writeEndObject();
+  }
+
+  static String toJson(ViewMetadata viewMetadata) {
+    return JsonUtil.generate(gen -> toJson(viewMetadata, gen), false);
+  }
+
+  public static ViewMetadata read(InputFile file) {
+    try (InputStream is = file.newStream()) {
+      return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class));
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to read file: %s", file);
+    }
+  }
+
+  public static ViewMetadata fromJson(JsonNode node) {
+    Preconditions.checkArgument(node != null, "Cannot parse view metadata from null json");
+    Preconditions.checkArgument(
+        node.isObject(), "Cannot parse metadata from a non-object: %s", node);
+
+    int formatVersion = JsonUtil.getInt(FORMAT_VERSION, node);
+    Preconditions.checkArgument(
+        formatVersion <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION,
+        "Cannot read unsupported version %s",
+        formatVersion);
+
+    String location = JsonUtil.getString(LOCATION, node);
+
+    int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, node);
+
+    Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, node);
+    JsonNode versionsListNode = JsonUtil.get(VERSIONS, node);
+    List<ViewVersion> versions = Lists.newArrayListWithExpectedSize(versionsListNode.size());
+    Iterator<JsonNode> iterator = versionsListNode.elements();

Review Comment:
   Good catch, I updated to verify the data types so the error message is obvious



-- 
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 #6559: Core: View metadata implementation

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

   @nastra is carrying this forward in https://github.com/apache/iceberg/pull/7759, which addresses the pending comments I'll be closing this one just to avoid confusion
   Thanks everyone! 
   


-- 
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 #6559: Core: View metadata implementation

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


##########
core/src/test/resources/view/ValidViewMetadata.json:
##########
@@ -0,0 +1,80 @@
+{

Review Comment:
   Looks like puffin files are in path `org/apache/iceberg/puffin/v1`, we should follow the same pattern



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