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 2021/09/13 03:06:08 UTC

[GitHub] [iceberg] jackye1995 opened a new pull request #3104: Core: add snapshot annotation to table metadata (WIP)

jackye1995 opened a new pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104


   continuation of #2961 after some design discussion, still WIP but I'd like to first publish it to receive some feedback and continue with the effort.
   
   https://docs.google.com/document/d/1PvxK_0ebEoX3s7nS6-LOJJZdBYr_olTWH9oepNUfJ-A/edit#
   
   @rdblue @RussellSpitzer @szehon-ho @stevenzwu 


-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r784271405



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -502,7 +519,10 @@ public TableMetadata replaceCurrentSnapshot(Snapshot snapshot) {
   }
 
   public TableMetadata removeSnapshotsIf(Predicate<Snapshot> removeIf) {
-    List<Snapshot> toRemove = snapshots.stream().filter(removeIf).collect(Collectors.toList());
+    List<Snapshot> toRemove = snapshots.stream()
+        .filter(removeIf)
+        .filter(snapshot -> !refsById.containsKey(snapshot.snapshotId()))
+        .collect(Collectors.toList());

Review comment:
       added https://github.com/apache/iceberg/issues/3900 to track 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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
nastra commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r742564069



##########
File path: api/src/main/java/org/apache/iceberg/SnapshotReference.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;
+
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.exceptions.ValidationException;
+
+/**
+ * User-defined information of a named snapshot
+ */
+public class SnapshotReference {
+
+  private final long snapshotId;
+  private final String name;
+  private final SnapshotReferenceType type;
+  private final Integer minSnapshotsToKeep;
+  private final long maxSnapshotAgeMs;
+
+  private SnapshotReference(
+      long snapshotId,
+      String name,
+      SnapshotReferenceType type,
+      Integer minSnapshotsToKeep,
+      long maxSnapshotAgeMs) {
+    this.snapshotId = snapshotId;
+    this.name = name;
+    this.type = type;
+    this.minSnapshotsToKeep = minSnapshotsToKeep;
+    this.maxSnapshotAgeMs = maxSnapshotAgeMs;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public String snapshotName() {
+    return name;
+  }
+
+  public SnapshotReferenceType type() {
+    return type;
+  }
+
+  /**
+   * Returns the minimum number of snapshots to keep for a BRANCH, or null for a TAG

Review comment:
       maybe `minSnapshotsToKeep` can be 0 instead of `null` for a TAG? Then you could use the primitive `int` type rather than `Integer`




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r780846482



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -529,6 +549,25 @@ public TableMetadata replaceProperties(Map<String, String> rawProperties) {
         .build();
   }
 
+  public TableMetadata replaceRefs(Map<String, SnapshotRef> newRefs) {
+

Review comment:
       Nit: unnecessary whitespace starting the function.




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r780845655



##########
File path: api/src/main/java/org/apache/iceberg/SnapshotRef.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;
+
+import java.io.Serializable;
+import java.util.Objects;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+public class SnapshotRef implements Serializable {
+
+  public static final String MAIN_BRANCH = "main";
+
+  private final long snapshotId;
+  private final SnapshotRefType type;
+  private final Integer minSnapshotsToKeep;
+  private final Long maxSnapshotAgeMs;
+  private final Long maxRefAgeMs;
+
+  private SnapshotRef(
+      long snapshotId,
+      SnapshotRefType type,
+      Integer minSnapshotsToKeep,
+      Long maxSnapshotAgeMs,
+      Long maxRefAgeMs) {
+    this.snapshotId = snapshotId;
+    this.type = type;
+    this.minSnapshotsToKeep = minSnapshotsToKeep;
+    this.maxSnapshotAgeMs = maxSnapshotAgeMs;
+    this.maxRefAgeMs = maxRefAgeMs;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public SnapshotRefType type() {
+    return type;
+  }
+
+  public Integer minSnapshotsToKeep() {
+    return minSnapshotsToKeep;
+  }
+
+  public Long maxSnapshotAgeMs() {
+    return maxSnapshotAgeMs;
+  }
+
+  public Long maxRefAgeMs() {
+    return maxRefAgeMs;
+  }
+
+  public static Builder builderForTag(long snapshotId) {
+    return builderFor(snapshotId, SnapshotRefType.TAG);
+  }
+
+  public static Builder builderForBranch(long snapshotId) {
+    return builderFor(snapshotId, SnapshotRefType.BRANCH);
+  }
+
+  public static Builder builderFrom(SnapshotRef ref) {
+    return new Builder(ref.type())
+        .snapshotId(ref.snapshotId())

Review comment:
       Why use a builder method for snapshot ID when it is passed in through the factory method? What happens when you try to change the snapshot ID or set it to null? I think it may be simpler just to make this a `private final` field and remove the builder method for it.




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r764545321



##########
File path: core/src/main/java/org/apache/iceberg/util/JsonUtil.java
##########
@@ -73,6 +73,18 @@ public static long getLong(String property, JsonNode node) {
     return pNode.asLong();
   }
 
+  public static Long getLongOrNull(String property, JsonNode node) {
+    if (!node.has(property)) {
+      return null;
+    }
+
+    Preconditions.checkArgument(node.has(property), "Cannot parse missing long %s", property);

Review comment:
       correct, updated




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

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

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



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


[GitHub] [iceberg] jackye1995 commented on a change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r737135972



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -243,6 +243,9 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final List<SnapshotReference> refs;
+  private final Map<String, SnapshotReference> refsByName;
+  private final String currentBranch;

Review comment:
       This is currently set to `null` by default. In the design doc I said default to `main`, but that has to be added after we add the update API in the next PR. Otherwise the main branch will stale after new snapshot is committed.




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r737135972



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -243,6 +243,9 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final List<SnapshotReference> refs;
+  private final Map<String, SnapshotReference> refsByName;
+  private final String currentBranch;

Review comment:
       This is currently set to `null` by default. In the design doc I said default to `main`, but that has to be added after we add the update API. Otherwise the main branch will stale after new snapshot is committed.




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r780845510



##########
File path: api/src/main/java/org/apache/iceberg/SnapshotRef.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;
+
+import java.io.Serializable;
+import java.util.Objects;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+public class SnapshotRef implements Serializable {
+
+  public static final String MAIN_BRANCH = "main";
+
+  private final long snapshotId;
+  private final SnapshotRefType type;
+  private final Integer minSnapshotsToKeep;
+  private final Long maxSnapshotAgeMs;
+  private final Long maxRefAgeMs;
+
+  private SnapshotRef(
+      long snapshotId,
+      SnapshotRefType type,
+      Integer minSnapshotsToKeep,
+      Long maxSnapshotAgeMs,
+      Long maxRefAgeMs) {
+    this.snapshotId = snapshotId;
+    this.type = type;
+    this.minSnapshotsToKeep = minSnapshotsToKeep;
+    this.maxSnapshotAgeMs = maxSnapshotAgeMs;
+    this.maxRefAgeMs = maxRefAgeMs;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public SnapshotRefType type() {
+    return type;
+  }
+
+  public Integer minSnapshotsToKeep() {
+    return minSnapshotsToKeep;
+  }
+
+  public Long maxSnapshotAgeMs() {
+    return maxSnapshotAgeMs;
+  }
+
+  public Long maxRefAgeMs() {
+    return maxRefAgeMs;
+  }
+
+  public static Builder builderForTag(long snapshotId) {

Review comment:
       What about a slightly more direct name scheme, `tagBuilder` and `branchBuilder`?




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r780846057



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -342,6 +349,8 @@ public String toString() {
     Preconditions.checkArgument(
         currentSnapshotId < 0 || snapshotsById.containsKey(currentSnapshotId),
         "Invalid table metadata: Cannot find current version");
+
+

Review comment:
       Nit: unnecessary whitespace change.




-- 
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] szehon-ho commented on a change in pull request #3104: Core: add snapshot annotation to table metadata (WIP)

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r710753216



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotAnnotationParser.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+public class SnapshotAnnotationParser {
+
+  private SnapshotAnnotationParser() {
+  }
+
+  private static final String SNAPSHOT_ID = "snapshot-id";
+  private static final String SNAPSHOT_NAME = "snapshot-name";
+  private static final String EXPIRE_STRATEGY = "expire-strategy";
+  private static final String RETAINED_UNTIL = "retained-until";
+
+  public static String toJson(SnapshotAnnotation annotation) {
+    return toJson(annotation, false);
+  }
+
+  public static String toJson(SnapshotAnnotation annotation, boolean pretty) {
+    try {
+      StringWriter writer = new StringWriter();
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      if (pretty) {
+        generator.useDefaultPrettyPrinter();
+      }
+      toJson(annotation, generator);
+      generator.flush();
+      return writer.toString();
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  public static void toJson(SnapshotAnnotation annotation, JsonGenerator generator) throws IOException {

Review comment:
       Can we have a better name/signature for this, toJson infers it returns a string (was confused when reading the caller code). Maybe writeSnapshotAnnotation(JsonGenerator generator, SnapshotAnnotation annotation) if not too redundant

##########
File path: api/src/main/java/org/apache/iceberg/SnapshotAnnotation.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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;
+
+import java.util.Objects;
+import org.apache.iceberg.exceptions.ValidationException;
+
+/**
+ * User-defined information of a named snapshot
+ */
+public class SnapshotAnnotation {
+
+  static final SnapshotExpireStrategy EXPIRE_STRATEGY_DEFAULT = SnapshotExpireStrategy.DEFAULT;
+
+  private final long snapshotId;
+  private final String snapshotName;
+  private final SnapshotExpireStrategy expireStrategy;
+  private final Long retainedUntilMillis;
+
+  private SnapshotAnnotation(
+      long snapshotId,
+      String snapshotName,
+      SnapshotExpireStrategy expireStrategy,
+      Long retainedUntilMillis) {
+    this.snapshotId = snapshotId;
+    this.snapshotName = snapshotName;
+    this.expireStrategy = expireStrategy;
+    this.retainedUntilMillis = retainedUntilMillis;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public String snapshotName() {
+    return snapshotName;
+  }
+
+  public SnapshotExpireStrategy expireStrategy() {
+    return expireStrategy;
+  }
+
+  public Long retainedUntilMillis() {
+    return retainedUntilMillis;
+  }
+
+  public static Builder builderFor(long snapshotId, String snapshotName) {
+    return new Builder(snapshotId, snapshotName);
+  }
+
+  public static class Builder {
+
+    private final Long snapshotId;
+    private final String snapshotName;
+
+    private SnapshotExpireStrategy expireStrategy = EXPIRE_STRATEGY_DEFAULT;
+    private Long retainedUntilMillis;
+
+    Builder(long snapshotId, String snapshotName) {
+      ValidationException.check(snapshotId > 0, "Invalid snapshot ID: must be greater than 0");
+      ValidationException.check(snapshotName != null, "Snapshot name must not be null for snapshot annotation");
+      this.snapshotId = snapshotId;
+      this.snapshotName = snapshotName;
+    }
+
+    public Builder expireStrategy(SnapshotExpireStrategy strategy) {
+      this.expireStrategy = strategy;
+      return this;
+    }
+
+    public Builder retainedUntilMillis(Long millis) {
+      this.retainedUntilMillis = millis;
+      return this;
+    }
+
+    public SnapshotAnnotation build() {
+      if (expireStrategy.equals(SnapshotExpireStrategy.RETAINED_UNTIL)) {
+        ValidationException.check(retainedUntilMillis != null,
+            "Retention expiration time must be set if snapshot lifecycle is RETENTION");
+      }
+
+      return new SnapshotAnnotation(snapshotId, snapshotName, expireStrategy, retainedUntilMillis);
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {

Review comment:
       Maybe I missed something, but what is different than default equals/hashcode?

##########
File path: core/src/main/java/org/apache/iceberg/util/JsonUtil.java
##########
@@ -73,6 +73,18 @@ public static long getLong(String property, JsonNode node) {
     return pNode.asLong();
   }
 
+  public static Long getLongOrNull(String property, JsonNode node) {
+    if (!node.has(property)) {
+      return null;
+    }
+
+    Preconditions.checkArgument(node.has(property), "Cannot parse missing long %s", property);

Review comment:
       No point anymore to this check anymore , right ?

##########
File path: core/src/main/java/org/apache/iceberg/SnapshotAnnotationParser.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+public class SnapshotAnnotationParser {
+
+  private SnapshotAnnotationParser() {
+  }
+
+  private static final String SNAPSHOT_ID = "snapshot-id";
+  private static final String SNAPSHOT_NAME = "snapshot-name";
+  private static final String EXPIRE_STRATEGY = "expire-strategy";
+  private static final String RETAINED_UNTIL = "retained-until";
+
+  public static String toJson(SnapshotAnnotation annotation) {
+    return toJson(annotation, false);
+  }
+
+  public static String toJson(SnapshotAnnotation annotation, boolean pretty) {
+    try {
+      StringWriter writer = new StringWriter();
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      if (pretty) {
+        generator.useDefaultPrettyPrinter();
+      }
+      toJson(annotation, generator);
+      generator.flush();
+      return writer.toString();
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  public static void toJson(SnapshotAnnotation annotation, JsonGenerator generator) throws IOException {

Review comment:
       And also do we need make it 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] ajantha-bhat commented on a change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r750886910



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -243,6 +243,9 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final List<SnapshotReference> refs;

Review comment:
       oh, I think we should discuss this in Spec PR, (https://github.com/apache/iceberg/pull/3425)




-- 
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] ajantha-bhat commented on a change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r750886536



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -243,6 +243,9 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final List<SnapshotReference> refs;

Review comment:
       If we do so, how are we gonna support snapshot history ? metadata tables per reference should show and use its own snapshots 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] rdblue commented on a change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r780844471



##########
File path: api/src/main/java/org/apache/iceberg/SnapshotRef.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;
+
+import java.io.Serializable;
+import java.util.Objects;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+public class SnapshotRef implements Serializable {
+
+  public static final String MAIN_BRANCH = "main";
+
+  private final long snapshotId;
+  private final SnapshotRefType type;
+  private final Integer minSnapshotsToKeep;
+  private final Long maxSnapshotAgeMs;
+  private final Long maxRefAgeMs;
+
+  private SnapshotRef(
+      long snapshotId,
+      SnapshotRefType type,
+      Integer minSnapshotsToKeep,
+      Long maxSnapshotAgeMs,
+      Long maxRefAgeMs) {
+    this.snapshotId = snapshotId;
+    this.type = type;
+    this.minSnapshotsToKeep = minSnapshotsToKeep;
+    this.maxSnapshotAgeMs = maxSnapshotAgeMs;
+    this.maxRefAgeMs = maxRefAgeMs;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public SnapshotRefType type() {
+    return type;
+  }
+
+  public Integer minSnapshotsToKeep() {
+    return minSnapshotsToKeep;
+  }
+
+  public Long maxSnapshotAgeMs() {
+    return maxSnapshotAgeMs;
+  }
+
+  public Long maxRefAgeMs() {
+    return maxRefAgeMs;
+  }
+
+  public static Builder builderForTag(long snapshotId) {
+    return builderFor(snapshotId, SnapshotRefType.TAG);
+  }
+
+  public static Builder builderForBranch(long snapshotId) {
+    return builderFor(snapshotId, SnapshotRefType.BRANCH);
+  }
+
+  public static Builder builderFrom(SnapshotRef ref) {
+    return new Builder(ref.type())
+        .snapshotId(ref.snapshotId())
+        .minSnapshotsToKeep(ref.minSnapshotsToKeep())
+        .maxSnapshotAgeMs(ref.maxSnapshotAgeMs())
+        .maxRefAgeMs(ref.maxRefAgeMs());
+  }
+
+  public static Builder builderFor(long snapshotId, SnapshotRefType type) {
+    return new Builder(type).snapshotId(snapshotId);
+  }
+
+  public static class Builder {
+
+    private final SnapshotRefType type;
+
+    private Long snapshotId;
+    private Integer minSnapshotsToKeep;
+    private Long maxSnapshotAgeMs;
+    private Long maxRefAgeMs;
+
+    Builder(SnapshotRefType type) {
+      ValidationException.check(type != null, "Snapshot reference type must not be null");
+      this.type = type;
+    }
+
+    public Builder snapshotId(long id) {
+      this.snapshotId = id;
+      return this;
+    }
+
+    public Builder minSnapshotsToKeep(Integer value) {
+      this.minSnapshotsToKeep = value;
+      return this;
+    }
+
+    public Builder maxSnapshotAgeMs(Long value) {
+      this.maxSnapshotAgeMs = value;
+      return this;
+    }
+
+    public Builder maxRefAgeMs(Long value) {
+      this.maxRefAgeMs = value;
+      return this;
+    }
+
+    public SnapshotRef build() {
+      if (type.equals(SnapshotRefType.TAG)) {
+        ValidationException.check(minSnapshotsToKeep == null,
+            "TAG type snapshot reference does not support setting minSnapshotsToKeep");
+        ValidationException.check(maxSnapshotAgeMs == null,
+            "TAG type snapshot reference does not support setting maxSnapshotAgeMs");
+      } else {
+        if (minSnapshotsToKeep != null) {
+          ValidationException.check(minSnapshotsToKeep > 0,
+              "Min snapshots to keep must be greater than 0");

Review comment:
       This could be `ValidationException.check(minSnapshotsToKeep == null || minSnapshotsToKeep > 0, ...)` rather than using an if block.




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r781460786



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +764,49 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateAndCompleteRefs(Map<String, SnapshotRef> inputRefs) {
+    for (SnapshotRef ref : inputRefs.values()) {
+      Preconditions.checkArgument(snapshotsById.containsKey(ref.snapshotId()) || ref.snapshotId() == -1,
+          "Snapshot reference %s does not exist in the existing snapshots list", ref);

Review comment:
       Do we also want to validate that only `main` can be `-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] jackye1995 commented on a change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r782437052



##########
File path: api/src/main/java/org/apache/iceberg/Table.java
##########
@@ -138,6 +139,22 @@ default String name() {
    */
   List<HistoryEntry> history();
 
+  /**
+   * Get the snapshot references of this table.
+   *
+   * @return a map with ref name as key, {@link SnapshotRef} as value
+   */
+  Map<String, SnapshotRef> refs();
+
+  /**
+   * Get the snapshot references of a snapshot.
+   *
+   * @param snapshotId snapshot ID
+   * @return a set of {@link SnapshotRef snapshot references} of a snapshot.
+   *         Note that when there is no ref, it returns an empty set but not null.
+   */
+  Set<SnapshotRef> refs(long snapshotId);

Review comment:
       I was thinking it's useful in subsequent PRs for updating the snapshot expiration logic, but we can remove this for now and just have a method for querying the snapshot of a specific ref name.




-- 
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] hameizi commented on a change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
hameizi commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r765411972



##########
File path: core/src/main/java/org/apache/iceberg/BaseUpdateSnapshotRefs.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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;
+
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class BaseUpdateSnapshotRefs implements UpdateSnapshotRefs {
+
+  private final TableOperations ops;
+  private final TableMetadata base;
+  private final Map<String, SnapshotRef> refs;
+
+  BaseUpdateSnapshotRefs(TableOperations ops) {
+    this.ops = ops;
+    this.base = ops.current();
+    this.refs = Maps.newHashMap(base.refs());
+  }
+
+  @Override
+  public UpdateSnapshotRefs tag(String name, long snapshotId) {
+    Preconditions.checkArgument(name != null, "Tag name must not be null");
+    Preconditions.checkArgument(base.snapshot(snapshotId) != null, "Cannot find snapshot with ID: %s", snapshotId);
+    Preconditions.checkArgument(!refs.containsKey(name), "Cannot tag snapshot, ref already exists: %s", name);
+
+    refs.put(name, SnapshotRef.builderForTag(snapshotId).build());
+    return this;
+  }
+
+  @Override
+  public UpdateSnapshotRefs branch(String name, long snapshotId) {
+    Preconditions.checkArgument(name != null, "Branch name must not be null");
+    Preconditions.checkArgument(base.snapshot(snapshotId) != null, "Cannot find snapshot with ID: %s", snapshotId);
+    Preconditions.checkArgument(!refs.containsKey(name), "Cannot create branch, ref already exists: %s", name);
+
+    refs.put(name, SnapshotRef.builderForBranch(snapshotId).build());
+    return this;
+  }
+
+  @Override
+  public UpdateSnapshotRefs remove(String name) {
+    Preconditions.checkArgument(name != null, "Ref name must not be null");
+    Preconditions.checkArgument(refs.containsKey(name), "Cannot find ref to remove: %s", name);
+    Preconditions.checkArgument(!SnapshotRef.MAIN_BRANCH.equals(name), "Main branch must not be removed");
+
+    refs.remove(name);
+    return this;
+  }
+
+  @Override
+  public UpdateSnapshotRefs rename(String from, String to) {
+    Preconditions.checkArgument(from != null && to != null, "Names must not be null");
+    Preconditions.checkArgument(refs.containsKey(from), "Cannot find ref to rename from: %s", from);
+    Preconditions.checkArgument(!refs.containsKey(to), "Cannot rename to an existing ref: %s", to);
+
+    refs.put(to, refs.remove(from));
+    return this;
+  }
+
+  @Override
+  public UpdateSnapshotRefs setLifetime(String name, long ageMs) {
+    Preconditions.checkArgument(name != null, "Branch name must not be null");
+    Preconditions.checkArgument(ageMs > 0, "Lifetime must be positive");
+    Preconditions.checkArgument(refs.containsKey(name), "Cannot find ref to set lifetime: %s", name);
+    Preconditions.checkArgument(!SnapshotRef.MAIN_BRANCH.equals(name), "Main branch is retained forever");
+
+    SnapshotRef oldRef = refs.get(name);
+    refs.put(name, SnapshotRef.builderFrom(oldRef).maxRefAgeMs(ageMs).build());
+    return this;
+  }

Review comment:
       I think maybe we should throw error when user update several properties in one operation. Because there is just one property will be update.




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r765322389



##########
File path: core/src/main/java/org/apache/iceberg/BaseUpdateSnapshotRefs.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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;
+
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class BaseUpdateSnapshotRefs implements UpdateSnapshotRefs {
+
+  private final TableOperations ops;
+  private final TableMetadata base;
+  private final Map<String, SnapshotRef> refs;

Review comment:
       Unlike properties update, I just track the final result of `refs` map here, and we will not retry during commit phase to avoid concurrent updates of refs.




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r782436294



##########
File path: api/src/main/java/org/apache/iceberg/Table.java
##########
@@ -173,6 +190,13 @@ default String name() {
    */
   UpdateLocation updateLocation();
 
+  /**
+   * Create a new {@link UpdateSnapshotRefs} to update snapshot references and commit the changes.
+   *
+   * @return a new {@link UpdateSnapshotRefs}
+   */
+  UpdateSnapshotRefs updateRefs();

Review comment:
       Yes I do have exactly the same thought, where append will go to the `main` branch by default and an override is allowed. I think a follow-up PR is needed for this feature.




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r782441368



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +764,49 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateAndCompleteRefs(Map<String, SnapshotRef> inputRefs) {
+    for (SnapshotRef ref : inputRefs.values()) {
+      Preconditions.checkArgument(snapshotsById.containsKey(ref.snapshotId()) || ref.snapshotId() == -1,
+          "Snapshot reference %s does not exist in the existing snapshots list", ref);
+    }
+
+    if (!inputRefs.containsKey(SnapshotRef.MAIN_BRANCH)) {
+      return refsWithMainBranch(currentSnapshotId, inputRefs);
+    }
+
+    return inputRefs;
+  }
+
+  private SetMultimap<Long, SnapshotRef> indexRefs() {
+    ImmutableSetMultimap.Builder<Long, SnapshotRef> builder = ImmutableSetMultimap.builder();
+    for (SnapshotRef ref : refs.values()) {
+      builder.put(ref.snapshotId(), ref);
+    }
+
+    return builder.build();
+  }
+
+  /**
+   * Get a new list of refs with the main branch head pointing to the given snapshot ID
+   *
+   * @param snapshotId snapshot ID of the main branch head
+   * @param currentRefs current table refs list
+   * @return updated refs list
+   */
+  private static Map<String, SnapshotRef> refsWithMainBranch(long snapshotId, Map<String, SnapshotRef> currentRefs) {

Review comment:
       Yes +1 for creating an empty snapshot when creating a table




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

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

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



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


[GitHub] [iceberg] rdblue commented on pull request #3104: Core: add snapshot annotation to table metadata (WIP)

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


   I think that a feature like this should be more understandable and straightforward. If our overall goal is to enable branching and tagging, then I think it makes the most sense to do that directly rather than introducing a similar concept like annotations.


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

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

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



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


[GitHub] [iceberg] jackye1995 commented on a change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r743119758



##########
File path: api/src/main/java/org/apache/iceberg/SnapshotReference.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;
+
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.exceptions.ValidationException;
+
+/**
+ * User-defined information of a named snapshot
+ */
+public class SnapshotReference {
+
+  private final long snapshotId;
+  private final String name;
+  private final SnapshotReferenceType type;
+  private final Integer minSnapshotsToKeep;
+  private final long maxSnapshotAgeMs;
+
+  private SnapshotReference(
+      long snapshotId,
+      String name,
+      SnapshotReferenceType type,
+      Integer minSnapshotsToKeep,
+      long maxSnapshotAgeMs) {
+    this.snapshotId = snapshotId;
+    this.name = name;
+    this.type = type;
+    this.minSnapshotsToKeep = minSnapshotsToKeep;
+    this.maxSnapshotAgeMs = maxSnapshotAgeMs;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public String snapshotName() {
+    return name;
+  }
+
+  public SnapshotReferenceType type() {
+    return type;
+  }
+
+  /**
+   * Returns the minimum number of snapshots to keep for a BRANCH, or null for a TAG
+   */
+  public Integer minSnapshotsToKeep() {
+    return minSnapshotsToKeep;
+  }
+
+  public long maxSnapshotAgeMs() {
+    return maxSnapshotAgeMs;
+  }
+
+  public static Builder builderFor(long snapshotId, String name, SnapshotReferenceType type) {
+    return new Builder(snapshotId, name, type);
+  }
+
+  public static class Builder {
+
+    private final Long snapshotId;
+    private final String name;
+    private final SnapshotReferenceType type;
+    private Integer minSnapshotsToKeep;
+    private Long maxSnapshotAgeMs;
+
+    Builder(long snapshotId, String name, SnapshotReferenceType type) {
+      ValidationException.check(snapshotId > 0, "Snapshot ID must be greater than 0");
+      ValidationException.check(name != null, "Snapshot reference name must not be null");
+      this.snapshotId = snapshotId;
+      this.name = name;
+      this.type = type;
+    }
+
+    public Builder withMinSnapshotsToKeep(Integer value) {

Review comment:
       will be used once we added it to spec

##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -243,6 +243,9 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final List<SnapshotReference> refs;
+  private final Map<String, SnapshotReference> refsByName;
+  private final String currentBranch;

Review comment:
       This is currently set to `null` by default. In the design doc I said default to `main`, but that has to be added after we add the update API in the next PR. Otherwise the main branch will stale after new snapshot is committed.

##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -243,6 +243,9 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final List<SnapshotReference> refs;

Review comment:
       Yes that's the current plan, the metadata.json always contains all the snapshots and all references.




-- 
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] RussellSpitzer commented on a change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r743763276



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -243,6 +243,9 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final List<SnapshotReference> refs;

Review comment:
       This is where I am a little confused. Does every Metadata.json contain the full branch and snapshot references for all other branches and snapshots?




-- 
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] RussellSpitzer commented on a change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r743763276



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -243,6 +243,9 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final List<SnapshotReference> refs;

Review comment:
       This is where I am a little confused. Does every Metadata.json contain the full branch and snapshot references for all other branches and snapshots?




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r780844369



##########
File path: api/src/main/java/org/apache/iceberg/SnapshotRef.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;
+
+import java.io.Serializable;
+import java.util.Objects;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+public class SnapshotRef implements Serializable {
+
+  public static final String MAIN_BRANCH = "main";
+
+  private final long snapshotId;
+  private final SnapshotRefType type;
+  private final Integer minSnapshotsToKeep;
+  private final Long maxSnapshotAgeMs;
+  private final Long maxRefAgeMs;
+
+  private SnapshotRef(
+      long snapshotId,
+      SnapshotRefType type,
+      Integer minSnapshotsToKeep,
+      Long maxSnapshotAgeMs,
+      Long maxRefAgeMs) {
+    this.snapshotId = snapshotId;
+    this.type = type;
+    this.minSnapshotsToKeep = minSnapshotsToKeep;
+    this.maxSnapshotAgeMs = maxSnapshotAgeMs;
+    this.maxRefAgeMs = maxRefAgeMs;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public SnapshotRefType type() {
+    return type;
+  }
+
+  public Integer minSnapshotsToKeep() {
+    return minSnapshotsToKeep;
+  }
+
+  public Long maxSnapshotAgeMs() {
+    return maxSnapshotAgeMs;
+  }
+
+  public Long maxRefAgeMs() {
+    return maxRefAgeMs;
+  }
+
+  public static Builder builderForTag(long snapshotId) {
+    return builderFor(snapshotId, SnapshotRefType.TAG);
+  }
+
+  public static Builder builderForBranch(long snapshotId) {
+    return builderFor(snapshotId, SnapshotRefType.BRANCH);
+  }
+
+  public static Builder builderFrom(SnapshotRef ref) {
+    return new Builder(ref.type())
+        .snapshotId(ref.snapshotId())
+        .minSnapshotsToKeep(ref.minSnapshotsToKeep())
+        .maxSnapshotAgeMs(ref.maxSnapshotAgeMs())
+        .maxRefAgeMs(ref.maxRefAgeMs());
+  }
+
+  public static Builder builderFor(long snapshotId, SnapshotRefType type) {
+    return new Builder(type).snapshotId(snapshotId);
+  }
+
+  public static class Builder {
+
+    private final SnapshotRefType type;
+
+    private Long snapshotId;
+    private Integer minSnapshotsToKeep;
+    private Long maxSnapshotAgeMs;
+    private Long maxRefAgeMs;
+
+    Builder(SnapshotRefType type) {
+      ValidationException.check(type != null, "Snapshot reference type must not be null");
+      this.type = type;
+    }
+
+    public Builder snapshotId(long id) {
+      this.snapshotId = id;
+      return this;
+    }
+
+    public Builder minSnapshotsToKeep(Integer value) {
+      this.minSnapshotsToKeep = value;
+      return this;
+    }
+
+    public Builder maxSnapshotAgeMs(Long value) {
+      this.maxSnapshotAgeMs = value;
+      return this;
+    }
+
+    public Builder maxRefAgeMs(Long value) {
+      this.maxRefAgeMs = value;
+      return this;
+    }
+
+    public SnapshotRef build() {
+      if (type.equals(SnapshotRefType.TAG)) {
+        ValidationException.check(minSnapshotsToKeep == null,
+            "TAG type snapshot reference does not support setting minSnapshotsToKeep");

Review comment:
       Can we use "Tag" instead of "TAG type snapshot reference"? I think people are going to be more familiar with the friendly names, "tag" and "branch", so we should use those rather than focusing on the generalized "reference".




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r780846129



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -238,6 +240,8 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final Map<String, SnapshotRef> refs;
+  private final SetMultimap<Long, SnapshotRef> refsById;

Review comment:
       I don't understand why these are indexed by snapshot rather than name. Is that a common 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


[GitHub] [iceberg] jackye1995 commented on pull request #3104: Core: add snapshot reference to table metadata

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


   @RussellSpitzer yes, let me close this one in favor of #3883 


-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r743119758



##########
File path: api/src/main/java/org/apache/iceberg/SnapshotReference.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;
+
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.exceptions.ValidationException;
+
+/**
+ * User-defined information of a named snapshot
+ */
+public class SnapshotReference {
+
+  private final long snapshotId;
+  private final String name;
+  private final SnapshotReferenceType type;
+  private final Integer minSnapshotsToKeep;
+  private final long maxSnapshotAgeMs;
+
+  private SnapshotReference(
+      long snapshotId,
+      String name,
+      SnapshotReferenceType type,
+      Integer minSnapshotsToKeep,
+      long maxSnapshotAgeMs) {
+    this.snapshotId = snapshotId;
+    this.name = name;
+    this.type = type;
+    this.minSnapshotsToKeep = minSnapshotsToKeep;
+    this.maxSnapshotAgeMs = maxSnapshotAgeMs;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public String snapshotName() {
+    return name;
+  }
+
+  public SnapshotReferenceType type() {
+    return type;
+  }
+
+  /**
+   * Returns the minimum number of snapshots to keep for a BRANCH, or null for a TAG
+   */
+  public Integer minSnapshotsToKeep() {
+    return minSnapshotsToKeep;
+  }
+
+  public long maxSnapshotAgeMs() {
+    return maxSnapshotAgeMs;
+  }
+
+  public static Builder builderFor(long snapshotId, String name, SnapshotReferenceType type) {
+    return new Builder(snapshotId, name, type);
+  }
+
+  public static class Builder {
+
+    private final Long snapshotId;
+    private final String name;
+    private final SnapshotReferenceType type;
+    private Integer minSnapshotsToKeep;
+    private Long maxSnapshotAgeMs;
+
+    Builder(long snapshotId, String name, SnapshotReferenceType type) {
+      ValidationException.check(snapshotId > 0, "Snapshot ID must be greater than 0");
+      ValidationException.check(name != null, "Snapshot reference name must not be null");
+      this.snapshotId = snapshotId;
+      this.name = name;
+      this.type = type;
+    }
+
+    public Builder withMinSnapshotsToKeep(Integer value) {

Review comment:
       will be used once we added it to spec

##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -243,6 +243,9 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final List<SnapshotReference> refs;
+  private final Map<String, SnapshotReference> refsByName;
+  private final String currentBranch;

Review comment:
       This is currently set to `null` by default. In the design doc I said default to `main`, but that has to be added after we add the update API in the next PR. Otherwise the main branch will stale after new snapshot is committed.

##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -243,6 +243,9 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final List<SnapshotReference> refs;

Review comment:
       Yes that's the current plan, the metadata.json always contains all the snapshots and all references.




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r780846388



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -502,7 +519,10 @@ public TableMetadata replaceCurrentSnapshot(Snapshot snapshot) {
   }
 
   public TableMetadata removeSnapshotsIf(Predicate<Snapshot> removeIf) {
-    List<Snapshot> toRemove = snapshots.stream().filter(removeIf).collect(Collectors.toList());
+    List<Snapshot> toRemove = snapshots.stream()
+        .filter(removeIf)
+        .filter(snapshot -> !refsById.containsKey(snapshot.snapshotId()))
+        .collect(Collectors.toList());

Review comment:
       I think it is probably time to remove this method, or at least to stop using it. Instead, I'd probably just produce a list of snapshot IDs to remove. That fits better with the new builder style.




-- 
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] RussellSpitzer commented on pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#issuecomment-1012292532


   @jackye1995 So we should focus on @amogh-jahagirdar s PR now? Just want to close out this one if that's the case.


-- 
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 #3104: Core: add snapshot reference to table metadata

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


   Hi all, I have been discussing snapshot lifecycle management with @jackye1995 and if possible would like to take over addressing any remaining comments for these changes as well as any other potential, relevant implementation improvements. @jackye1995 would this be possible? Would raise another PR and mark @jackye1995 and @hameizi 1249369293@qq.com as co-authors.


-- 
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] rymurr commented on pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#issuecomment-971544477


   Hey @jackye1995 I added a bunch of comments to the design doc. Do you consider this more up to date now? Should I commnet/review here instead?


-- 
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 #3104: Core: add snapshot reference to table metadata

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


   Sure, I am a bit low in capacity for implementation recently, please feel free to open a new PR to push the progress of this PR forward and I will then close this one review it, thank you very much @amogh-jahagirdar !


-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r781462522



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +764,49 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateAndCompleteRefs(Map<String, SnapshotRef> inputRefs) {
+    for (SnapshotRef ref : inputRefs.values()) {
+      Preconditions.checkArgument(snapshotsById.containsKey(ref.snapshotId()) || ref.snapshotId() == -1,
+          "Snapshot reference %s does not exist in the existing snapshots list", ref);
+    }
+
+    if (!inputRefs.containsKey(SnapshotRef.MAIN_BRANCH)) {
+      return refsWithMainBranch(currentSnapshotId, inputRefs);
+    }
+
+    return inputRefs;
+  }
+
+  private SetMultimap<Long, SnapshotRef> indexRefs() {
+    ImmutableSetMultimap.Builder<Long, SnapshotRef> builder = ImmutableSetMultimap.builder();
+    for (SnapshotRef ref : refs.values()) {
+      builder.put(ref.snapshotId(), ref);
+    }
+
+    return builder.build();
+  }
+
+  /**
+   * Get a new list of refs with the main branch head pointing to the given snapshot ID
+   *
+   * @param snapshotId snapshot ID of the main branch head
+   * @param currentRefs current table refs list
+   * @return updated refs list
+   */
+  private static Map<String, SnapshotRef> refsWithMainBranch(long snapshotId, Map<String, SnapshotRef> currentRefs) {

Review comment:
       I've been thinking about how awkward this is for a while. We may want to just create an empty snapshot when creating a table. That would clean up using -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] jackye1995 commented on a change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r765324630



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +764,49 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateAndCompleteRefs(Map<String, SnapshotRef> inputRefs) {
+    for (SnapshotRef ref : inputRefs.values()) {
+      Preconditions.checkArgument(snapshotsById.containsKey(ref.snapshotId()) || ref.snapshotId() == -1,
+          "Snapshot reference %s does not exist in the existing snapshots list", ref);
+    }
+
+    if (!inputRefs.containsKey(SnapshotRef.MAIN_BRANCH)) {
+      return refsWithMainBranch(currentSnapshotId, inputRefs);
+    }
+
+    return inputRefs;
+  }
+
+  private SetMultimap<Long, SnapshotRef> indexRefs() {
+    ImmutableSetMultimap.Builder<Long, SnapshotRef> builder = ImmutableSetMultimap.builder();
+    for (SnapshotRef ref : refs.values()) {
+      builder.put(ref.snapshotId(), ref);
+    }
+
+    return builder.build();
+  }
+
+  /**
+   * Get a new list of refs with the main branch head pointing to the given snapshot ID
+   *
+   * @param snapshotId snapshot ID of the main branch head
+   * @param currentRefs current table refs list
+   * @return updated refs list
+   */
+  private static Map<String, SnapshotRef> refsWithMainBranch(long snapshotId, Map<String, SnapshotRef> currentRefs) {

Review comment:
       The snapshot ID here can be `-1`. I have been thinking if we should keep the main branch ref in case there is no main branch head after a replace operation. But there might be retention policy set at the main branch, and removing the main branch will result in deleting that information.




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r780844208



##########
File path: api/src/main/java/org/apache/iceberg/SnapshotRef.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;
+
+import java.io.Serializable;
+import java.util.Objects;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+public class SnapshotRef implements Serializable {
+
+  public static final String MAIN_BRANCH = "main";
+
+  private final long snapshotId;
+  private final SnapshotRefType type;
+  private final Integer minSnapshotsToKeep;
+  private final Long maxSnapshotAgeMs;
+  private final Long maxRefAgeMs;
+
+  private SnapshotRef(
+      long snapshotId,
+      SnapshotRefType type,
+      Integer minSnapshotsToKeep,
+      Long maxSnapshotAgeMs,
+      Long maxRefAgeMs) {
+    this.snapshotId = snapshotId;
+    this.type = type;
+    this.minSnapshotsToKeep = minSnapshotsToKeep;
+    this.maxSnapshotAgeMs = maxSnapshotAgeMs;
+    this.maxRefAgeMs = maxRefAgeMs;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public SnapshotRefType type() {
+    return type;
+  }
+
+  public Integer minSnapshotsToKeep() {
+    return minSnapshotsToKeep;
+  }
+
+  public Long maxSnapshotAgeMs() {
+    return maxSnapshotAgeMs;
+  }
+
+  public Long maxRefAgeMs() {
+    return maxRefAgeMs;
+  }
+
+  public static Builder builderForTag(long snapshotId) {
+    return builderFor(snapshotId, SnapshotRefType.TAG);
+  }
+
+  public static Builder builderForBranch(long snapshotId) {
+    return builderFor(snapshotId, SnapshotRefType.BRANCH);
+  }
+
+  public static Builder builderFrom(SnapshotRef ref) {
+    return new Builder(ref.type())
+        .snapshotId(ref.snapshotId())
+        .minSnapshotsToKeep(ref.minSnapshotsToKeep())
+        .maxSnapshotAgeMs(ref.maxSnapshotAgeMs())
+        .maxRefAgeMs(ref.maxRefAgeMs());
+  }
+
+  public static Builder builderFor(long snapshotId, SnapshotRefType type) {
+    return new Builder(type).snapshotId(snapshotId);
+  }
+
+  public static class Builder {
+
+    private final SnapshotRefType type;
+
+    private Long snapshotId;
+    private Integer minSnapshotsToKeep;
+    private Long maxSnapshotAgeMs;
+    private Long maxRefAgeMs;
+
+    Builder(SnapshotRefType type) {
+      ValidationException.check(type != null, "Snapshot reference type must not be null");

Review comment:
       I think most builders use `IllegalArgumentException` to reject arguments that aren't allowed and `ValidationException` to catch inconsistencies. The `ValidationException` cases that are thrown in `build` are good examples of when a `ValidationException` is a good idea because, for example, `minSnapshotsToKeep` doesn't make sense with a tag.




-- 
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] RussellSpitzer commented on a change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r743763276



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -243,6 +243,9 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final List<SnapshotReference> refs;

Review comment:
       This is where I am a little confused. Does every Metadata.json contain the full branch and snapshot references for all other branches and snapshots?




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r743119758



##########
File path: api/src/main/java/org/apache/iceberg/SnapshotReference.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;
+
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.exceptions.ValidationException;
+
+/**
+ * User-defined information of a named snapshot
+ */
+public class SnapshotReference {
+
+  private final long snapshotId;
+  private final String name;
+  private final SnapshotReferenceType type;
+  private final Integer minSnapshotsToKeep;
+  private final long maxSnapshotAgeMs;
+
+  private SnapshotReference(
+      long snapshotId,
+      String name,
+      SnapshotReferenceType type,
+      Integer minSnapshotsToKeep,
+      long maxSnapshotAgeMs) {
+    this.snapshotId = snapshotId;
+    this.name = name;
+    this.type = type;
+    this.minSnapshotsToKeep = minSnapshotsToKeep;
+    this.maxSnapshotAgeMs = maxSnapshotAgeMs;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public String snapshotName() {
+    return name;
+  }
+
+  public SnapshotReferenceType type() {
+    return type;
+  }
+
+  /**
+   * Returns the minimum number of snapshots to keep for a BRANCH, or null for a TAG
+   */
+  public Integer minSnapshotsToKeep() {
+    return minSnapshotsToKeep;
+  }
+
+  public long maxSnapshotAgeMs() {
+    return maxSnapshotAgeMs;
+  }
+
+  public static Builder builderFor(long snapshotId, String name, SnapshotReferenceType type) {
+    return new Builder(snapshotId, name, type);
+  }
+
+  public static class Builder {
+
+    private final Long snapshotId;
+    private final String name;
+    private final SnapshotReferenceType type;
+    private Integer minSnapshotsToKeep;
+    private Long maxSnapshotAgeMs;
+
+    Builder(long snapshotId, String name, SnapshotReferenceType type) {
+      ValidationException.check(snapshotId > 0, "Snapshot ID must be greater than 0");
+      ValidationException.check(name != null, "Snapshot reference name must not be null");
+      this.snapshotId = snapshotId;
+      this.name = name;
+      this.type = type;
+    }
+
+    public Builder withMinSnapshotsToKeep(Integer value) {

Review comment:
       will be used once we added it to spec




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

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

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



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


[GitHub] [iceberg] jackye1995 closed pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 closed pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104


   


-- 
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] ajantha-bhat commented on a change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r750893676



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -243,6 +243,9 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final List<SnapshotReference> refs;

Review comment:
       After thinking more on this, I think current snapshot will have parent id. so we can still form a history snapshots per branch just by keeping the current snapshot id. So, no need spec change. 
   
   But we have a change code for handling metadata tables per reference using above information. 




-- 
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] hameizi commented on a change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
hameizi commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r765411972



##########
File path: core/src/main/java/org/apache/iceberg/BaseUpdateSnapshotRefs.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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;
+
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class BaseUpdateSnapshotRefs implements UpdateSnapshotRefs {
+
+  private final TableOperations ops;
+  private final TableMetadata base;
+  private final Map<String, SnapshotRef> refs;
+
+  BaseUpdateSnapshotRefs(TableOperations ops) {
+    this.ops = ops;
+    this.base = ops.current();
+    this.refs = Maps.newHashMap(base.refs());
+  }
+
+  @Override
+  public UpdateSnapshotRefs tag(String name, long snapshotId) {
+    Preconditions.checkArgument(name != null, "Tag name must not be null");
+    Preconditions.checkArgument(base.snapshot(snapshotId) != null, "Cannot find snapshot with ID: %s", snapshotId);
+    Preconditions.checkArgument(!refs.containsKey(name), "Cannot tag snapshot, ref already exists: %s", name);
+
+    refs.put(name, SnapshotRef.builderForTag(snapshotId).build());
+    return this;
+  }
+
+  @Override
+  public UpdateSnapshotRefs branch(String name, long snapshotId) {
+    Preconditions.checkArgument(name != null, "Branch name must not be null");
+    Preconditions.checkArgument(base.snapshot(snapshotId) != null, "Cannot find snapshot with ID: %s", snapshotId);
+    Preconditions.checkArgument(!refs.containsKey(name), "Cannot create branch, ref already exists: %s", name);
+
+    refs.put(name, SnapshotRef.builderForBranch(snapshotId).build());
+    return this;
+  }
+
+  @Override
+  public UpdateSnapshotRefs remove(String name) {
+    Preconditions.checkArgument(name != null, "Ref name must not be null");
+    Preconditions.checkArgument(refs.containsKey(name), "Cannot find ref to remove: %s", name);
+    Preconditions.checkArgument(!SnapshotRef.MAIN_BRANCH.equals(name), "Main branch must not be removed");
+
+    refs.remove(name);
+    return this;
+  }
+
+  @Override
+  public UpdateSnapshotRefs rename(String from, String to) {
+    Preconditions.checkArgument(from != null && to != null, "Names must not be null");
+    Preconditions.checkArgument(refs.containsKey(from), "Cannot find ref to rename from: %s", from);
+    Preconditions.checkArgument(!refs.containsKey(to), "Cannot rename to an existing ref: %s", to);
+
+    refs.put(to, refs.remove(from));
+    return this;
+  }
+
+  @Override
+  public UpdateSnapshotRefs setLifetime(String name, long ageMs) {
+    Preconditions.checkArgument(name != null, "Branch name must not be null");
+    Preconditions.checkArgument(ageMs > 0, "Lifetime must be positive");
+    Preconditions.checkArgument(refs.containsKey(name), "Cannot find ref to set lifetime: %s", name);
+    Preconditions.checkArgument(!SnapshotRef.MAIN_BRANCH.equals(name), "Main branch is retained forever");
+
+    SnapshotRef oldRef = refs.get(name);
+    refs.put(name, SnapshotRef.builderFrom(oldRef).maxRefAgeMs(ageMs).build());
+    return this;
+  }

Review comment:
       I think maybe we should throw error when user update several properties in one operation. Because there is just one property will be update.




-- 
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 #3104: Core: add snapshot reference to table metadata

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


   I have updated this PR based on the TableMetadata builder refactoring. it is quite hard to separate the update logic out of this, so I just included the changes in #3612 and marked @hameizi as coauthor.


-- 
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] hameizi commented on pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
hameizi commented on pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#issuecomment-998462433


   @rdblue Is there any more concern about this PR?


-- 
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 #3104: Core: add snapshot reference to table metadata

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


   @rdblue @szehon-ho @nastra updated based on the last discussion, could you take a look?


-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r780845962



##########
File path: core/src/main/java/org/apache/iceberg/SerializableTable.java
##########
@@ -261,6 +262,16 @@ public Snapshot snapshot(long snapshotId) {
     return lazyTable().history();
   }
 
+  @Override
+  public Map<String, SnapshotRef> refs() {
+    return lazyTable().refs();

Review comment:
       Seems reasonable to read the table metadata for this. I doubt serialized tables will be accessing refs.




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r780844208



##########
File path: api/src/main/java/org/apache/iceberg/SnapshotRef.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;
+
+import java.io.Serializable;
+import java.util.Objects;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+public class SnapshotRef implements Serializable {
+
+  public static final String MAIN_BRANCH = "main";
+
+  private final long snapshotId;
+  private final SnapshotRefType type;
+  private final Integer minSnapshotsToKeep;
+  private final Long maxSnapshotAgeMs;
+  private final Long maxRefAgeMs;
+
+  private SnapshotRef(
+      long snapshotId,
+      SnapshotRefType type,
+      Integer minSnapshotsToKeep,
+      Long maxSnapshotAgeMs,
+      Long maxRefAgeMs) {
+    this.snapshotId = snapshotId;
+    this.type = type;
+    this.minSnapshotsToKeep = minSnapshotsToKeep;
+    this.maxSnapshotAgeMs = maxSnapshotAgeMs;
+    this.maxRefAgeMs = maxRefAgeMs;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public SnapshotRefType type() {
+    return type;
+  }
+
+  public Integer minSnapshotsToKeep() {
+    return minSnapshotsToKeep;
+  }
+
+  public Long maxSnapshotAgeMs() {
+    return maxSnapshotAgeMs;
+  }
+
+  public Long maxRefAgeMs() {
+    return maxRefAgeMs;
+  }
+
+  public static Builder builderForTag(long snapshotId) {
+    return builderFor(snapshotId, SnapshotRefType.TAG);
+  }
+
+  public static Builder builderForBranch(long snapshotId) {
+    return builderFor(snapshotId, SnapshotRefType.BRANCH);
+  }
+
+  public static Builder builderFrom(SnapshotRef ref) {
+    return new Builder(ref.type())
+        .snapshotId(ref.snapshotId())
+        .minSnapshotsToKeep(ref.minSnapshotsToKeep())
+        .maxSnapshotAgeMs(ref.maxSnapshotAgeMs())
+        .maxRefAgeMs(ref.maxRefAgeMs());
+  }
+
+  public static Builder builderFor(long snapshotId, SnapshotRefType type) {
+    return new Builder(type).snapshotId(snapshotId);
+  }
+
+  public static class Builder {
+
+    private final SnapshotRefType type;
+
+    private Long snapshotId;
+    private Integer minSnapshotsToKeep;
+    private Long maxSnapshotAgeMs;
+    private Long maxRefAgeMs;
+
+    Builder(SnapshotRefType type) {
+      ValidationException.check(type != null, "Snapshot reference type must not be null");

Review comment:
       I think most builders use `IllegalArgumentException` to reject arguments that aren't allowed and `ValidationException` to catch inconsistencies.




-- 
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] RussellSpitzer commented on a change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r742922212



##########
File path: api/src/main/java/org/apache/iceberg/SnapshotReference.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;
+
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.exceptions.ValidationException;
+
+/**
+ * User-defined information of a named snapshot
+ */
+public class SnapshotReference {
+
+  private final long snapshotId;
+  private final String name;
+  private final SnapshotReferenceType type;
+  private final Integer minSnapshotsToKeep;
+  private final long maxSnapshotAgeMs;
+
+  private SnapshotReference(
+      long snapshotId,
+      String name,
+      SnapshotReferenceType type,
+      Integer minSnapshotsToKeep,
+      long maxSnapshotAgeMs) {
+    this.snapshotId = snapshotId;
+    this.name = name;
+    this.type = type;
+    this.minSnapshotsToKeep = minSnapshotsToKeep;
+    this.maxSnapshotAgeMs = maxSnapshotAgeMs;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public String snapshotName() {
+    return name;
+  }
+
+  public SnapshotReferenceType type() {
+    return type;
+  }
+
+  /**
+   * Returns the minimum number of snapshots to keep for a BRANCH, or null for a TAG
+   */
+  public Integer minSnapshotsToKeep() {
+    return minSnapshotsToKeep;
+  }
+
+  public long maxSnapshotAgeMs() {
+    return maxSnapshotAgeMs;
+  }
+
+  public static Builder builderFor(long snapshotId, String name, SnapshotReferenceType type) {
+    return new Builder(snapshotId, name, type);
+  }
+
+  public static class Builder {
+
+    private final Long snapshotId;
+    private final String name;
+    private final SnapshotReferenceType type;
+    private Integer minSnapshotsToKeep;
+    private Long maxSnapshotAgeMs;
+
+    Builder(long snapshotId, String name, SnapshotReferenceType type) {
+      ValidationException.check(snapshotId > 0, "Snapshot ID must be greater than 0");
+      ValidationException.check(name != null, "Snapshot reference name must not be null");
+      this.snapshotId = snapshotId;
+      this.name = name;
+      this.type = type;
+    }
+
+    public Builder withMinSnapshotsToKeep(Integer value) {

Review comment:
       Are these two properties for use in "expire snapshots?




-- 
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] ajantha-bhat commented on a change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r750893676



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -243,6 +243,9 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final List<SnapshotReference> refs;

Review comment:
       After thinking more on this, I think current snapshot will have parent id. so we can still form a history snapshots per branch just by keeping the current snapshot id. So, no need spec change. 
   
   But we have to change code for handling metadata tables per reference using above information. 




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r737135474



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -635,8 +656,11 @@ public TableMetadata removeSnapshotsIf(Predicate<Snapshot> removeIf) {
       }
     }
 
-    // update the snapshot log
+    // update the snapshot log and refs
     Set<Long> validIds = Sets.newHashSet(Iterables.transform(filtered, Snapshot::snapshotId));
+    List<SnapshotReference> newRefs = refs.stream()

Review comment:
       my thought is that remove snapshot should work as usual without failing for snapshot with ref, and predicate should be formulated based on ref information.




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r737135972



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -243,6 +243,9 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final List<SnapshotReference> refs;
+  private final Map<String, SnapshotReference> refsByName;
+  private final String currentBranch;

Review comment:
       This is currently set to `null` by default. In the design doc I said default to `main`, but that has to be added after we add the update API in the next PR. Otherwise the main branch will stale after new snapshot is committed.




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r743839079



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -243,6 +243,9 @@ public String toString() {
   private final Map<Integer, SortOrder> sortOrdersById;
   private final List<HistoryEntry> snapshotLog;
   private final List<MetadataLogEntry> previousFiles;
+  private final List<SnapshotReference> refs;

Review comment:
       Yes that's the current plan, the metadata.json always contains all the snapshots and all references.




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r780844718



##########
File path: api/src/main/java/org/apache/iceberg/Table.java
##########
@@ -138,6 +139,22 @@ default String name() {
    */
   List<HistoryEntry> history();
 
+  /**
+   * Get the snapshot references of this table.
+   *
+   * @return a map with ref name as key, {@link SnapshotRef} as value
+   */
+  Map<String, SnapshotRef> refs();
+
+  /**
+   * Get the snapshot references of a snapshot.
+   *
+   * @param snapshotId snapshot ID
+   * @return a set of {@link SnapshotRef snapshot references} of a snapshot.
+   *         Note that when there is no ref, it returns an empty set but not null.
+   */
+  Set<SnapshotRef> refs(long snapshotId);

Review comment:
       Why have a method for returning refs by snapshot but not a ref by name?




-- 
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 change in pull request #3104: Core: add snapshot reference to table metadata

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3104:
URL: https://github.com/apache/iceberg/pull/3104#discussion_r780845128



##########
File path: api/src/main/java/org/apache/iceberg/Table.java
##########
@@ -173,6 +190,13 @@ default String name() {
    */
   UpdateLocation updateLocation();
 
+  /**
+   * Create a new {@link UpdateSnapshotRefs} to update snapshot references and commit the changes.
+   *
+   * @return a new {@link UpdateSnapshotRefs}
+   */
+  UpdateSnapshotRefs updateRefs();

Review comment:
       How do you think the other table mutations should handle refs? For example, an append might look like this:
   
   ```java
   table.newAppend()
       .appendFile(someFile)
       .commit();
   ```
   
   Should we add an optional `branch` to all of the `SnapshotProducer` operations?
   
   ```java
   table.newAppend()
       .appendFile(someFile)
       .branch("some_branch")
       .commit();
   ```
   
   Curious what you've been thinking.




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