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 2020/07/15 21:57:29 UTC

[GitHub] [iceberg] RussellSpitzer opened a new pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

RussellSpitzer opened a new pull request #1211:
URL: https://github.com/apache/iceberg/pull/1211


   Hi everyone, This is a work in progress on an idea to speed up and scale Manifest scanning during
   snapshot expiration. I would be glad to hear any thoughts anyone has on the concept or implementation.
   
   Thanks for your feedback!
   
   
   Adds a Spark Action with the aim of parallelizing the manifest scanning
   portion of Snapshot expiration.
   
   Previously there was only a single method for expiring old data files associated
   with expired Snapshots which required scanning all effected manifests locally. In
   order to take advantage of systems which can handle more simultaneous requests and IO
   we move the Manifest Scanning portiion of the expiration to Spark. The new functionality
   is accesible in a new Spark Action ExpireSnapshotsAction which has a similiar api to
   the local task but is instead exeuted on Spark.
   
   The new action is implemented by utilizing the local code to determine which manifest
   fils are effected by Snapshot Expiration. Then parallelizing the file names and performing
   the scanning of the manifest files remotely. The actual deletion of unneeded data files is still
   performed locally.
   
   To get the information required for performing the deletes, the Remove Snapshot class is refactored
   so that the methods relating to discovering effected manifests can be called by other modules.


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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private SparkSession session;
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession session, Table table) {
+    this.session = session;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().deleteExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Execute is a synonym for commit in this implementation. Calling either commit or execute will
+   * launch the Spark equivalent of RemoveSnapshots.
+   *
+   * @return nothing
+   */
+  @Override
+  public ExpireSnapshotResults execute() {
+    localExpireSnapshots.commit();
+
+    TableMetadata currentMetadata = ops.refresh();
+
+    //Locally determine which snapshots have been expired
+    SnapshotExpirationChanges expiredSnapshotChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(currentMetadata, base);
+
+    //Locally determine which manifests will need to be scanned, reverted, deleted
+    ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            expiredSnapshotChanges.validSnapshotIds(), expiredSnapshotChanges.expiredSnapshotIds(),
+            currentMetadata, base, ops.io());
+
+    FileIO io = SparkUtil.serializableFileIO(table);
+
+    //Going the RDD Route because our reader functions all work with full Manifest Files
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(session.sparkContext());
+
+    JavaRDD<ManifestFile> manifestsToScan =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToScan()));
+
+    JavaRDD<ManifestFile> manifestsToRevert =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToRevert()));
+
+    FileIO serializableIO = SparkUtil.serializableFileIO(table);

Review comment:
       Will do! Thanks for the heads up




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

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] aokolnychyi commented on a change in pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -152,168 +164,33 @@ private void cleanExpiredSnapshots() {
     // 2. Delete any data files that were deleted by those snapshots and are not in the table
     // 3. Delete any manifests that are no longer used by current snapshots
     // 4. Delete the manifest lists
+    SnapshotExpirationChanges snapshotExpirationChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(ops, base);
 
-    TableMetadata current = ops.refresh();
-
-    Set<Long> validIds = Sets.newHashSet();
-    for (Snapshot snapshot : current.snapshots()) {
-      validIds.add(snapshot.snapshotId());
-    }
-
-    Set<Long> expiredIds = Sets.newHashSet();
-    for (Snapshot snapshot : base.snapshots()) {
-      long snapshotId = snapshot.snapshotId();
-      if (!validIds.contains(snapshotId)) {
-        // the snapshot was expired
-        LOG.info("Expired snapshot: {}", snapshot);
-        expiredIds.add(snapshotId);
-      }
-    }
-
-    if (expiredIds.isEmpty()) {
+    if (snapshotExpirationChanges.getExpiredSnapshotIds().isEmpty()) {
       // if no snapshots were expired, skip cleanup
       return;
     }
 
-    LOG.info("Committed snapshot changes; cleaning up expired manifests and data files.");
-
-    cleanExpiredFiles(current.snapshots(), validIds, expiredIds);
+    LOG.info("Cleaning up expired manifests and data files locally.");
+    cleanExpiredFiles(
+        snapshotExpirationChanges.getCurrentSnapshots(),
+        snapshotExpirationChanges.getValidSnapshotIds(),
+        snapshotExpirationChanges.getExpiredSnapshotIds());
   }
 
-  @SuppressWarnings("checkstyle:CyclomaticComplexity")
-  private void cleanExpiredFiles(List<Snapshot> snapshots, Set<Long> validIds, Set<Long> expiredIds) {
+  private void cleanExpiredFiles(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds) {
     // Reads and deletes are done using Tasks.foreach(...).suppressFailureWhenFinished to complete
     // as much of the delete work as possible and avoid orphaned data or manifest files.
-
-    // this is the set of ancestors of the current table state. when removing snapshots, this must
-    // only remove files that were deleted in an ancestor of the current table state to avoid
-    // physically deleting files that were logically deleted in a commit that was rolled back.
-    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil.ancestorIds(base.currentSnapshot(), base::snapshot));
-
-    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
-    for (long snapshotId : ancestorIds) {
-      String sourceSnapshotId = base.snapshot(snapshotId).summary().get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
-      if (sourceSnapshotId != null) {
-        // protect any snapshot that was cherry-picked into the current table state
-        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
-      }
-    }
-
-    // find manifests to clean up that are still referenced by a valid snapshot, but written by an expired snapshot
-    Set<String> validManifests = Sets.newHashSet();
-    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
-    Tasks.foreach(snapshots).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                for (ManifestFile manifest : manifests) {
-                  validManifests.add(manifest.path());
-
-                  long snapshotId = manifest.snapshotId();
-                  // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
-                  boolean fromValidSnapshots = validIds.contains(snapshotId);
-                  // whether the snapshot that created the manifest was an ancestor of the table state
-                  boolean isFromAncestor = ancestorIds.contains(snapshotId);
-                  // whether the changes in this snapshot have been picked into the current table state
-                  boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
-                  // if the snapshot that wrote this manifest is no longer valid (has expired),
-                  // then delete its deleted files. note that this is only for expired snapshots that are in the
-                  // current table state
-                  if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
-                    manifestsToScan.add(manifest.copy());
-                  }
-                }
-
-              } catch (IOException e) {
-                throw new RuntimeIOException(e,
-                    "Failed to close manifest list: %s", snapshot.manifestListLocation());
-              }
-            });
-
-    // find manifests to clean up that were only referenced by snapshots that have expired
-    Set<String> manifestListsToDelete = Sets.newHashSet();
-    Set<String> manifestsToDelete = Sets.newHashSet();
-    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
-    Tasks.foreach(base.snapshots()).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              long snapshotId = snapshot.snapshotId();
-              if (!validIds.contains(snapshotId)) {
-                // determine whether the changes in this snapshot are in the current table state
-                if (pickedAncestorSnapshotIds.contains(snapshotId)) {
-                  // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
-                  // its changes will expire when the picked snapshot expires.
-                  // A -- C -- D (source=B)
-                  //  `- B <-- this commit
-                  return;
-                }
-
-                long sourceSnapshotId = PropertyUtil.propertyAsLong(
-                    snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
-                if (ancestorIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- B -- C
-                  //  `- D (source=B) <-- this commit
-                  return;
-                }
-
-                if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- C -- E (source=B)
-                  //  `- B `- D (source=B) <-- this commit
-                  return;
-                }
-
-                // find any manifests that are no longer needed
-                try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                  for (ManifestFile manifest : manifests) {
-                    if (!validManifests.contains(manifest.path())) {
-                      manifestsToDelete.add(manifest.path());
-
-                      boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
-                      boolean isFromExpiringSnapshot = expiredIds.contains(manifest.snapshotId());
-
-                      if (isFromAncestor && manifest.hasDeletedFiles()) {
-                        // Only delete data files that were deleted in by an expired snapshot if that
-                        // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
-                        // deleted files and was rolled back will delete files that could be in the current
-                        // table state.
-                        manifestsToScan.add(manifest.copy());
-                      }
-
-                      if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
-                        // Because the manifest was written by a snapshot that is not an ancestor of the
-                        // current table state, the files added in this manifest can be removed. The extra
-                        // check whether the manifest was written by a known snapshot that was expired in
-                        // this commit ensures that the full ancestor list between when the snapshot was
-                        // written and this expiration is known and there is no missing history. If history
-                        // were missing, then the snapshot could be an ancestor of the table state but the
-                        // ancestor ID set would not contain it and this would be unsafe.
-                        manifestsToRevert.add(manifest.copy());
-                      }
-                    }
-                  }
-                } catch (IOException e) {
-                  throw new RuntimeIOException(e,
-                      "Failed to close manifest list: %s", snapshot.manifestListLocation());
-                }
-
-                // add the manifest list to the delete set, if present
-                if (snapshot.manifestListLocation() != null) {
-                  manifestListsToDelete.add(snapshot.manifestListLocation());
-                }
-              }
-            });
-    deleteDataFiles(manifestsToScan, manifestsToRevert, validIds);
-    deleteMetadataFiles(manifestsToDelete, manifestListsToDelete);
+    ExpireSnapshotUtil.ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(

Review comment:
       After refactoring, `cleanExpiredFiles` became really small. I'd consider merging it with `cleanExpiredSnapshots`. Then the utility method can accept `current`, `base`, `validIds`, `expiredIds`, `io` to determine manifests to scan. This should make the argument naming straightforward.




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,
+      TableMetadata currentTableMetadata, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = getValidManifests(currentTableSnapshots, ops);

Review comment:
       This was always done in two passes, It was just more confusing before :) See my comment above on the old code. Although maybe I misunderstand your meaning here?




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -152,168 +164,33 @@ private void cleanExpiredSnapshots() {
     // 2. Delete any data files that were deleted by those snapshots and are not in the table
     // 3. Delete any manifests that are no longer used by current snapshots
     // 4. Delete the manifest lists
+    SnapshotExpirationChanges snapshotExpirationChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(ops, base);

Review comment:
       Good call, I think that makes more sense




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

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] prodeezy commented on pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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


   cc @mehtaashish23 


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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {

Review comment:
       I fixed the code here, I'll try to get another PR up to make sure this loop is tested




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {

Review comment:
       sure!




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

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] aokolnychyi commented on a change in pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,
+      TableMetadata currentTableMetadata, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = getValidManifests(currentTableSnapshots, ops);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validIds,

Review comment:
       As far as I remember, `validManifests` contains manifests referenced by any of current snapshot and `manifestsToScan` contains a subset of valid manifests that were written by expired snapshots but still are still referenced by some current snapshots. We have to scan such manifests if they contain deleted files so that we can remove files that were deleted by expired snapshots while keeping all files that were added by those snapshots.
   
   This part is tricky so I think some comments would help here.




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -152,168 +164,33 @@ private void cleanExpiredSnapshots() {
     // 2. Delete any data files that were deleted by those snapshots and are not in the table
     // 3. Delete any manifests that are no longer used by current snapshots
     // 4. Delete the manifest lists
+    SnapshotExpirationChanges snapshotExpirationChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(ops, base);
 
-    TableMetadata current = ops.refresh();
-
-    Set<Long> validIds = Sets.newHashSet();
-    for (Snapshot snapshot : current.snapshots()) {
-      validIds.add(snapshot.snapshotId());
-    }
-
-    Set<Long> expiredIds = Sets.newHashSet();
-    for (Snapshot snapshot : base.snapshots()) {
-      long snapshotId = snapshot.snapshotId();
-      if (!validIds.contains(snapshotId)) {
-        // the snapshot was expired
-        LOG.info("Expired snapshot: {}", snapshot);
-        expiredIds.add(snapshotId);
-      }
-    }
-
-    if (expiredIds.isEmpty()) {
+    if (snapshotExpirationChanges.getExpiredSnapshotIds().isEmpty()) {
       // if no snapshots were expired, skip cleanup
       return;
     }
 
-    LOG.info("Committed snapshot changes; cleaning up expired manifests and data files.");
-
-    cleanExpiredFiles(current.snapshots(), validIds, expiredIds);
+    LOG.info("Cleaning up expired manifests and data files locally.");
+    cleanExpiredFiles(
+        snapshotExpirationChanges.getCurrentSnapshots(),
+        snapshotExpirationChanges.getValidSnapshotIds(),
+        snapshotExpirationChanges.getExpiredSnapshotIds());
   }
 
-  @SuppressWarnings("checkstyle:CyclomaticComplexity")
-  private void cleanExpiredFiles(List<Snapshot> snapshots, Set<Long> validIds, Set<Long> expiredIds) {
+  private void cleanExpiredFiles(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds) {
     // Reads and deletes are done using Tasks.foreach(...).suppressFailureWhenFinished to complete
     // as much of the delete work as possible and avoid orphaned data or manifest files.
-
-    // this is the set of ancestors of the current table state. when removing snapshots, this must
-    // only remove files that were deleted in an ancestor of the current table state to avoid
-    // physically deleting files that were logically deleted in a commit that was rolled back.
-    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil.ancestorIds(base.currentSnapshot(), base::snapshot));
-
-    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
-    for (long snapshotId : ancestorIds) {
-      String sourceSnapshotId = base.snapshot(snapshotId).summary().get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
-      if (sourceSnapshotId != null) {
-        // protect any snapshot that was cherry-picked into the current table state
-        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
-      }
-    }
-
-    // find manifests to clean up that are still referenced by a valid snapshot, but written by an expired snapshot
-    Set<String> validManifests = Sets.newHashSet();
-    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
-    Tasks.foreach(snapshots).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                for (ManifestFile manifest : manifests) {
-                  validManifests.add(manifest.path());
-
-                  long snapshotId = manifest.snapshotId();
-                  // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
-                  boolean fromValidSnapshots = validIds.contains(snapshotId);
-                  // whether the snapshot that created the manifest was an ancestor of the table state
-                  boolean isFromAncestor = ancestorIds.contains(snapshotId);
-                  // whether the changes in this snapshot have been picked into the current table state
-                  boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
-                  // if the snapshot that wrote this manifest is no longer valid (has expired),
-                  // then delete its deleted files. note that this is only for expired snapshots that are in the
-                  // current table state
-                  if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
-                    manifestsToScan.add(manifest.copy());
-                  }
-                }
-
-              } catch (IOException e) {
-                throw new RuntimeIOException(e,
-                    "Failed to close manifest list: %s", snapshot.manifestListLocation());
-              }
-            });
-
-    // find manifests to clean up that were only referenced by snapshots that have expired
-    Set<String> manifestListsToDelete = Sets.newHashSet();
-    Set<String> manifestsToDelete = Sets.newHashSet();
-    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
-    Tasks.foreach(base.snapshots()).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              long snapshotId = snapshot.snapshotId();
-              if (!validIds.contains(snapshotId)) {
-                // determine whether the changes in this snapshot are in the current table state
-                if (pickedAncestorSnapshotIds.contains(snapshotId)) {
-                  // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
-                  // its changes will expire when the picked snapshot expires.
-                  // A -- C -- D (source=B)
-                  //  `- B <-- this commit
-                  return;
-                }
-
-                long sourceSnapshotId = PropertyUtil.propertyAsLong(
-                    snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
-                if (ancestorIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- B -- C
-                  //  `- D (source=B) <-- this commit
-                  return;
-                }
-
-                if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- C -- E (source=B)
-                  //  `- B `- D (source=B) <-- this commit
-                  return;
-                }
-
-                // find any manifests that are no longer needed
-                try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                  for (ManifestFile manifest : manifests) {
-                    if (!validManifests.contains(manifest.path())) {
-                      manifestsToDelete.add(manifest.path());
-
-                      boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
-                      boolean isFromExpiringSnapshot = expiredIds.contains(manifest.snapshotId());
-
-                      if (isFromAncestor && manifest.hasDeletedFiles()) {
-                        // Only delete data files that were deleted in by an expired snapshot if that
-                        // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
-                        // deleted files and was rolled back will delete files that could be in the current
-                        // table state.
-                        manifestsToScan.add(manifest.copy());
-                      }
-
-                      if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
-                        // Because the manifest was written by a snapshot that is not an ancestor of the
-                        // current table state, the files added in this manifest can be removed. The extra
-                        // check whether the manifest was written by a known snapshot that was expired in
-                        // this commit ensures that the full ancestor list between when the snapshot was
-                        // written and this expiration is known and there is no missing history. If history
-                        // were missing, then the snapshot could be an ancestor of the table state but the
-                        // ancestor ID set would not contain it and this would be unsafe.
-                        manifestsToRevert.add(manifest.copy());
-                      }
-                    }
-                  }
-                } catch (IOException e) {
-                  throw new RuntimeIOException(e,
-                      "Failed to close manifest list: %s", snapshot.manifestListLocation());
-                }
-
-                // add the manifest list to the delete set, if present
-                if (snapshot.manifestListLocation() != null) {
-                  manifestListsToDelete.add(snapshot.manifestListLocation());
-                }
-              }
-            });
-    deleteDataFiles(manifestsToScan, manifestsToRevert, validIds);
-    deleteMetadataFiles(manifestsToDelete, manifestListsToDelete);
+    ExpireSnapshotUtil.ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            currentTableSnapshots, validIds, expiredIds, base, ops);
+
+    deleteDataFiles(manifestExpirationChanges.getManifestsToScan(),

Review comment:
       sure, I think even just "changes" is fine since we are only a few lines away from the ClassName




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -89,7 +96,7 @@ public ExpireSnapshots expireOlderThan(long timestampMillis) {
   @Override
   public ExpireSnapshots retainLast(int numSnapshots) {
     Preconditions.checkArgument(1 <= numSnapshots,
-            "Number of snapshots to retain must be at least 1, cannot be: %s", numSnapshots);

Review comment:
       Sorry my auto formatter was going a little overboard!




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

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] aokolnychyi commented on pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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


   Thanks for updating the PR quickly, @RussellSpitzer! I'll take a look today.


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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: api/src/main/java/org/apache/iceberg/ExpireSnapshots.java
##########
@@ -82,4 +82,16 @@
    * @return this for method chaining
    */
   ExpireSnapshots deleteWith(Consumer<String> deleteFunc);
+
+
+  /**
+   * Allows expiration of snapshots without any cleanup of underlying manifest or data files.
+   * <p>
+   * Allows control in removing data and manifest files which may be more efficiently removed non-locally using
+   * an action in a distributed framework such as RemoveOrphanFilesAction.
+   * </p>
+   * @param clean setting this to false will skip deleting expired manifests and files
+   * @return this for method chaining
+   */
+  ExpireSnapshots cleanUpFiles(boolean clean);

Review comment:
       That makes more sense, I was just copying the old method names but they are much less clear to someone using the api than `deleteExpiredFiles`




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,

Review comment:
       Sounds good to me, I'll remove 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.

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] aokolnychyi commented on a change in pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,
+      TableMetadata currentTableMetadata, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = getValidManifests(currentTableSnapshots, ops);

Review comment:
       Previously, it was one run to determine `validManifests` and `manifestsToScan`. Now, it is done in two passes. It seems fine to me as we iterate through the snapshots after expiry and I don't think it will affect the performance. Just a note for other reviewers. 




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(

Review comment:
       I decided on original




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -152,168 +164,33 @@ private void cleanExpiredSnapshots() {
     // 2. Delete any data files that were deleted by those snapshots and are not in the table
     // 3. Delete any manifests that are no longer used by current snapshots
     // 4. Delete the manifest lists
+    SnapshotExpirationChanges snapshotExpirationChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(ops, base);
 
-    TableMetadata current = ops.refresh();
-
-    Set<Long> validIds = Sets.newHashSet();
-    for (Snapshot snapshot : current.snapshots()) {
-      validIds.add(snapshot.snapshotId());
-    }
-
-    Set<Long> expiredIds = Sets.newHashSet();
-    for (Snapshot snapshot : base.snapshots()) {
-      long snapshotId = snapshot.snapshotId();
-      if (!validIds.contains(snapshotId)) {
-        // the snapshot was expired
-        LOG.info("Expired snapshot: {}", snapshot);
-        expiredIds.add(snapshotId);
-      }
-    }
-
-    if (expiredIds.isEmpty()) {
+    if (snapshotExpirationChanges.getExpiredSnapshotIds().isEmpty()) {
       // if no snapshots were expired, skip cleanup
       return;
     }
 
-    LOG.info("Committed snapshot changes; cleaning up expired manifests and data files.");
-
-    cleanExpiredFiles(current.snapshots(), validIds, expiredIds);
+    LOG.info("Cleaning up expired manifests and data files locally.");
+    cleanExpiredFiles(
+        snapshotExpirationChanges.getCurrentSnapshots(),
+        snapshotExpirationChanges.getValidSnapshotIds(),
+        snapshotExpirationChanges.getExpiredSnapshotIds());
   }
 
-  @SuppressWarnings("checkstyle:CyclomaticComplexity")
-  private void cleanExpiredFiles(List<Snapshot> snapshots, Set<Long> validIds, Set<Long> expiredIds) {
+  private void cleanExpiredFiles(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds) {
     // Reads and deletes are done using Tasks.foreach(...).suppressFailureWhenFinished to complete
     // as much of the delete work as possible and avoid orphaned data or manifest files.
-
-    // this is the set of ancestors of the current table state. when removing snapshots, this must
-    // only remove files that were deleted in an ancestor of the current table state to avoid
-    // physically deleting files that were logically deleted in a commit that was rolled back.
-    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil.ancestorIds(base.currentSnapshot(), base::snapshot));
-
-    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
-    for (long snapshotId : ancestorIds) {
-      String sourceSnapshotId = base.snapshot(snapshotId).summary().get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
-      if (sourceSnapshotId != null) {
-        // protect any snapshot that was cherry-picked into the current table state
-        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
-      }
-    }
-
-    // find manifests to clean up that are still referenced by a valid snapshot, but written by an expired snapshot
-    Set<String> validManifests = Sets.newHashSet();
-    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
-    Tasks.foreach(snapshots).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                for (ManifestFile manifest : manifests) {
-                  validManifests.add(manifest.path());
-
-                  long snapshotId = manifest.snapshotId();
-                  // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
-                  boolean fromValidSnapshots = validIds.contains(snapshotId);
-                  // whether the snapshot that created the manifest was an ancestor of the table state
-                  boolean isFromAncestor = ancestorIds.contains(snapshotId);
-                  // whether the changes in this snapshot have been picked into the current table state
-                  boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
-                  // if the snapshot that wrote this manifest is no longer valid (has expired),
-                  // then delete its deleted files. note that this is only for expired snapshots that are in the
-                  // current table state
-                  if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
-                    manifestsToScan.add(manifest.copy());
-                  }
-                }
-
-              } catch (IOException e) {
-                throw new RuntimeIOException(e,
-                    "Failed to close manifest list: %s", snapshot.manifestListLocation());
-              }
-            });
-
-    // find manifests to clean up that were only referenced by snapshots that have expired
-    Set<String> manifestListsToDelete = Sets.newHashSet();
-    Set<String> manifestsToDelete = Sets.newHashSet();
-    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
-    Tasks.foreach(base.snapshots()).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              long snapshotId = snapshot.snapshotId();
-              if (!validIds.contains(snapshotId)) {
-                // determine whether the changes in this snapshot are in the current table state
-                if (pickedAncestorSnapshotIds.contains(snapshotId)) {
-                  // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
-                  // its changes will expire when the picked snapshot expires.
-                  // A -- C -- D (source=B)
-                  //  `- B <-- this commit
-                  return;
-                }
-
-                long sourceSnapshotId = PropertyUtil.propertyAsLong(
-                    snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
-                if (ancestorIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- B -- C
-                  //  `- D (source=B) <-- this commit
-                  return;
-                }
-
-                if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- C -- E (source=B)
-                  //  `- B `- D (source=B) <-- this commit
-                  return;
-                }
-
-                // find any manifests that are no longer needed
-                try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                  for (ManifestFile manifest : manifests) {
-                    if (!validManifests.contains(manifest.path())) {
-                      manifestsToDelete.add(manifest.path());
-
-                      boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
-                      boolean isFromExpiringSnapshot = expiredIds.contains(manifest.snapshotId());
-
-                      if (isFromAncestor && manifest.hasDeletedFiles()) {
-                        // Only delete data files that were deleted in by an expired snapshot if that
-                        // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
-                        // deleted files and was rolled back will delete files that could be in the current
-                        // table state.
-                        manifestsToScan.add(manifest.copy());
-                      }
-
-                      if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
-                        // Because the manifest was written by a snapshot that is not an ancestor of the
-                        // current table state, the files added in this manifest can be removed. The extra
-                        // check whether the manifest was written by a known snapshot that was expired in
-                        // this commit ensures that the full ancestor list between when the snapshot was
-                        // written and this expiration is known and there is no missing history. If history
-                        // were missing, then the snapshot could be an ancestor of the table state but the
-                        // ancestor ID set would not contain it and this would be unsafe.
-                        manifestsToRevert.add(manifest.copy());
-                      }
-                    }
-                  }
-                } catch (IOException e) {
-                  throw new RuntimeIOException(e,
-                      "Failed to close manifest list: %s", snapshot.manifestListLocation());
-                }
-
-                // add the manifest list to the delete set, if present
-                if (snapshot.manifestListLocation() != null) {
-                  manifestListsToDelete.add(snapshot.manifestListLocation());
-                }
-              }
-            });
-    deleteDataFiles(manifestsToScan, manifestsToRevert, validIds);
-    deleteMetadataFiles(manifestsToDelete, manifestListsToDelete);
+    ExpireSnapshotUtil.ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(

Review comment:
       This is where my refactoring left me a little confused, this is how the previous call was made, Let me see if I can clean this up and make arg names more explanitory.




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -152,168 +164,33 @@ private void cleanExpiredSnapshots() {
     // 2. Delete any data files that were deleted by those snapshots and are not in the table
     // 3. Delete any manifests that are no longer used by current snapshots
     // 4. Delete the manifest lists
+    SnapshotExpirationChanges snapshotExpirationChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(ops, base);
 
-    TableMetadata current = ops.refresh();
-
-    Set<Long> validIds = Sets.newHashSet();
-    for (Snapshot snapshot : current.snapshots()) {
-      validIds.add(snapshot.snapshotId());
-    }
-
-    Set<Long> expiredIds = Sets.newHashSet();
-    for (Snapshot snapshot : base.snapshots()) {
-      long snapshotId = snapshot.snapshotId();
-      if (!validIds.contains(snapshotId)) {
-        // the snapshot was expired
-        LOG.info("Expired snapshot: {}", snapshot);
-        expiredIds.add(snapshotId);
-      }
-    }
-
-    if (expiredIds.isEmpty()) {
+    if (snapshotExpirationChanges.getExpiredSnapshotIds().isEmpty()) {
       // if no snapshots were expired, skip cleanup
       return;
     }
 
-    LOG.info("Committed snapshot changes; cleaning up expired manifests and data files.");
-
-    cleanExpiredFiles(current.snapshots(), validIds, expiredIds);
+    LOG.info("Cleaning up expired manifests and data files locally.");
+    cleanExpiredFiles(
+        snapshotExpirationChanges.getCurrentSnapshots(),
+        snapshotExpirationChanges.getValidSnapshotIds(),
+        snapshotExpirationChanges.getExpiredSnapshotIds());
   }
 
-  @SuppressWarnings("checkstyle:CyclomaticComplexity")
-  private void cleanExpiredFiles(List<Snapshot> snapshots, Set<Long> validIds, Set<Long> expiredIds) {
+  private void cleanExpiredFiles(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds) {
     // Reads and deletes are done using Tasks.foreach(...).suppressFailureWhenFinished to complete
     // as much of the delete work as possible and avoid orphaned data or manifest files.
-
-    // this is the set of ancestors of the current table state. when removing snapshots, this must
-    // only remove files that were deleted in an ancestor of the current table state to avoid
-    // physically deleting files that were logically deleted in a commit that was rolled back.
-    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil.ancestorIds(base.currentSnapshot(), base::snapshot));
-
-    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
-    for (long snapshotId : ancestorIds) {
-      String sourceSnapshotId = base.snapshot(snapshotId).summary().get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
-      if (sourceSnapshotId != null) {
-        // protect any snapshot that was cherry-picked into the current table state
-        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
-      }
-    }
-
-    // find manifests to clean up that are still referenced by a valid snapshot, but written by an expired snapshot
-    Set<String> validManifests = Sets.newHashSet();
-    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
-    Tasks.foreach(snapshots).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                for (ManifestFile manifest : manifests) {
-                  validManifests.add(manifest.path());
-
-                  long snapshotId = manifest.snapshotId();
-                  // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
-                  boolean fromValidSnapshots = validIds.contains(snapshotId);
-                  // whether the snapshot that created the manifest was an ancestor of the table state
-                  boolean isFromAncestor = ancestorIds.contains(snapshotId);
-                  // whether the changes in this snapshot have been picked into the current table state
-                  boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
-                  // if the snapshot that wrote this manifest is no longer valid (has expired),
-                  // then delete its deleted files. note that this is only for expired snapshots that are in the
-                  // current table state
-                  if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
-                    manifestsToScan.add(manifest.copy());
-                  }
-                }
-
-              } catch (IOException e) {
-                throw new RuntimeIOException(e,
-                    "Failed to close manifest list: %s", snapshot.manifestListLocation());
-              }
-            });
-
-    // find manifests to clean up that were only referenced by snapshots that have expired
-    Set<String> manifestListsToDelete = Sets.newHashSet();
-    Set<String> manifestsToDelete = Sets.newHashSet();
-    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
-    Tasks.foreach(base.snapshots()).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              long snapshotId = snapshot.snapshotId();
-              if (!validIds.contains(snapshotId)) {
-                // determine whether the changes in this snapshot are in the current table state
-                if (pickedAncestorSnapshotIds.contains(snapshotId)) {
-                  // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
-                  // its changes will expire when the picked snapshot expires.
-                  // A -- C -- D (source=B)
-                  //  `- B <-- this commit
-                  return;
-                }
-
-                long sourceSnapshotId = PropertyUtil.propertyAsLong(
-                    snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
-                if (ancestorIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- B -- C
-                  //  `- D (source=B) <-- this commit
-                  return;
-                }
-
-                if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- C -- E (source=B)
-                  //  `- B `- D (source=B) <-- this commit
-                  return;
-                }
-
-                // find any manifests that are no longer needed
-                try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                  for (ManifestFile manifest : manifests) {
-                    if (!validManifests.contains(manifest.path())) {
-                      manifestsToDelete.add(manifest.path());
-
-                      boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
-                      boolean isFromExpiringSnapshot = expiredIds.contains(manifest.snapshotId());
-
-                      if (isFromAncestor && manifest.hasDeletedFiles()) {
-                        // Only delete data files that were deleted in by an expired snapshot if that
-                        // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
-                        // deleted files and was rolled back will delete files that could be in the current
-                        // table state.
-                        manifestsToScan.add(manifest.copy());
-                      }
-
-                      if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
-                        // Because the manifest was written by a snapshot that is not an ancestor of the
-                        // current table state, the files added in this manifest can be removed. The extra
-                        // check whether the manifest was written by a known snapshot that was expired in
-                        // this commit ensures that the full ancestor list between when the snapshot was
-                        // written and this expiration is known and there is no missing history. If history
-                        // were missing, then the snapshot could be an ancestor of the table state but the
-                        // ancestor ID set would not contain it and this would be unsafe.
-                        manifestsToRevert.add(manifest.copy());
-                      }
-                    }
-                  }
-                } catch (IOException e) {
-                  throw new RuntimeIOException(e,
-                      "Failed to close manifest list: %s", snapshot.manifestListLocation());
-                }
-
-                // add the manifest list to the delete set, if present
-                if (snapshot.manifestListLocation() != null) {
-                  manifestListsToDelete.add(snapshot.manifestListLocation());
-                }
-              }
-            });
-    deleteDataFiles(manifestsToScan, manifestsToRevert, validIds);
-    deleteMetadataFiles(manifestsToDelete, manifestListsToDelete);
+    ExpireSnapshotUtil.ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(

Review comment:
       combined and modified




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;

Review comment:
       I don't think this will be different than in the local TestRemove code but I'm not sure ... That code path should have similar concurrency issues if this branch does I think.




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

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 closed pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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


   


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

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] aokolnychyi commented on a change in pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);

Review comment:
       Ideally, we want to compute ancestor ids and picked ancestor ids only once. I see we do this multiple times now.




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/ManifestFiles.java
##########
@@ -138,8 +138,8 @@ private ManifestFiles() {
     return open(manifest, io, null);
   }
 
-  static ManifestReader<?> open(ManifestFile manifest, FileIO io,
-                                Map<Integer, PartitionSpec> specsById) {
+  public static ManifestReader<?> open(

Review comment:
       I'm switching this back to package protected but leaving the two lines because it would be a cosmetic 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.

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private SparkSession session;
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession session, Table table) {
+    this.session = session;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().deleteExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Execute is a synonym for commit in this implementation. Calling either commit or execute will
+   * launch the Spark equivalent of RemoveSnapshots.
+   *
+   * @return nothing
+   */
+  @Override
+  public ExpireSnapshotResults execute() {
+    localExpireSnapshots.commit();
+
+    TableMetadata currentMetadata = ops.refresh();
+
+    //Locally determine which snapshots have been expired
+    SnapshotExpirationChanges expiredSnapshotChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(currentMetadata, base);
+
+    //Locally determine which manifests will need to be scanned, reverted, deleted
+    ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            expiredSnapshotChanges.validSnapshotIds(), expiredSnapshotChanges.expiredSnapshotIds(),
+            currentMetadata, base, ops.io());
+
+    FileIO io = SparkUtil.serializableFileIO(table);
+
+    //Going the RDD Route because our reader functions all work with full Manifest Files
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(session.sparkContext());
+
+    JavaRDD<ManifestFile> manifestsToScan =

Review comment:
       This would fall back to default.parallelism yes. We can add an api for specifying the number of slices though if we want a finer control




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,
+      TableMetadata currentTableMetadata, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = getValidManifests(currentTableSnapshots, ops);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validIds,
+        currentTableMetadata);
+
+    List<Snapshot> snapshotsNotChangingTableState = filterOutSnapshotsInTableState(validIds,
+        currentTableMetadata);
+
+    // find manifests to clean up that were only referenced by snapshots that have expired
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            currentTableMetadata, expiredIds, ops);
+
+    manifestExpirationChanges.getManifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableOperations ops, TableMetadata originalMetadata) {
+
+    TableMetadata current = ops.refresh();
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : current.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // the snapshot was expired
+        LOG.info("Expired snapshot: {}", snapshot);

Review comment:
       sgtm




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

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] aokolnychyi edited a comment on pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

Posted by GitBox <gi...@apache.org>.
aokolnychyi edited a comment on pull request #1211:
URL: https://github.com/apache/iceberg/pull/1211#issuecomment-659490881


   Thanks, @RussellSpitzer! Let me take a look at this today.
   
   To give a bit of background, we had a case where people compacted their table that reduced the number of files from 5+ million to under 1 million in a short period of time. That processed generated a huge number of new snapshots as different parts were optimized separately (one leaf partition at a time). In addition, there was a huge number of manifests to scan and a lot of expired data files. As a result, the expiry of snapshots was really slow.
   
   This is an attempt to fix such edge cases.


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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,
+      TableMetadata currentTableMetadata, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = getValidManifests(currentTableSnapshots, ops);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validIds,
+        currentTableMetadata);
+
+    List<Snapshot> snapshotsNotChangingTableState = filterOutSnapshotsInTableState(validIds,
+        currentTableMetadata);
+
+    // find manifests to clean up that were only referenced by snapshots that have expired
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            currentTableMetadata, expiredIds, ops);
+
+    manifestExpirationChanges.getManifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableOperations ops, TableMetadata originalMetadata) {
+
+    TableMetadata current = ops.refresh();
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : current.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // the snapshot was expired
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(current.snapshots(), validIds, expiredIds);
+  }
+
+  public static class SnapshotExpirationChanges {
+
+    private final List<Snapshot> currentSnapshots;
+    private final Set<Long> validSnapshotIds;
+    private final Set<Long> expiredSnapshotIds;
+
+    public SnapshotExpirationChanges(
+        List<Snapshot> currentSnapshots, Set<Long> validSnapshotIds, Set<Long> expiredSnapshotIds) {
+
+      this.currentSnapshots = currentSnapshots;
+      this.validSnapshotIds = validSnapshotIds;
+      this.expiredSnapshotIds = expiredSnapshotIds;
+    }
+
+    public List<Snapshot> getCurrentSnapshots() {
+      return currentSnapshots;
+    }
+
+    public Set<Long> getValidSnapshotIds() {
+      return validSnapshotIds;
+    }
+
+    public Set<Long> getExpiredSnapshotIds() {
+      return expiredSnapshotIds;
+    }
+  }
+
+  public static class ManifestExpirationChanges {
+
+    private final Set<ManifestFile> manifestsToScan;
+    private final Set<ManifestFile> manifestsToRevert;
+    private final Set<String> manifestsToDelete;
+    private final Set<String> manifestListsToDelete;
+
+    private ManifestExpirationChanges(
+        Set<ManifestFile> manifestsToScan, Set<ManifestFile> manifestsToRevert,
+        Set<String> manifestsToDelete, Set<String> manifestListsToDelete) {
+
+      this.manifestsToScan = manifestsToScan;
+      this.manifestsToRevert = manifestsToRevert;
+      this.manifestsToDelete = manifestsToDelete;
+      this.manifestListsToDelete = manifestListsToDelete;
+    }
+
+
+    public Set<ManifestFile> getManifestsToScan() {
+      return manifestsToScan;
+    }
+
+    public Set<ManifestFile> getManifestsToRevert() {
+      return manifestsToRevert;
+    }
+
+    public Set<String> getManifestsToDelete() {
+      return manifestsToDelete;
+    }
+
+    public Set<String> getManifestListsToDelete() {
+      return manifestListsToDelete;
+    }
+  }
+
+  private static class AncestorIds {

Review comment:
       ActiveSnapshots? AncestorInfo ? 




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java
##########
@@ -404,6 +404,55 @@ public void dataFilesCleanup() throws IOException {
     Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
   }
 
+  @Test
+  public void noDataFileCleanup() throws IOException {
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    List<ManifestFile> manifests = table.currentSnapshot().dataManifests();
+
+    ManifestFile newManifest = writeManifest(
+        "manifest-file-1.avro",
+        manifestEntry(Status.EXISTING, thirdSnapshotId, FILE_C),
+        manifestEntry(Status.EXISTING, fourthSnapshotId, FILE_D));
+
+    RewriteManifests rewriteManifests = table.rewriteManifests();
+    manifests.forEach(rewriteManifests::deleteManifest);
+    rewriteManifests.addManifest(newManifest);
+    rewriteManifests.commit();
+
+    Set<String> deletedFiles = Sets.newHashSet();
+
+    table.expireSnapshots()
+        .cleanUpFiles(false)
+        .expireOlderThan(t4)
+        .deleteWith(deletedFiles::add)
+        .commit();
+
+    Assert.assertFalse("FILE_A should not be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertFalse("FILE_B should not be deleted", deletedFiles.contains(FILE_B.path().toString()));

Review comment:
       All of the tests here just check whether or not the set contains or doesn't contain a path. The deleteFunction here just adds paths to a set instead of deleting things so if they don't appear in the set the file was not "deleted". So in all the tests in this file the function will never actually delete. 

##########
File path: core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java
##########
@@ -404,6 +404,55 @@ public void dataFilesCleanup() throws IOException {
     Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
   }
 
+  @Test
+  public void noDataFileCleanup() throws IOException {
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    List<ManifestFile> manifests = table.currentSnapshot().dataManifests();
+
+    ManifestFile newManifest = writeManifest(
+        "manifest-file-1.avro",
+        manifestEntry(Status.EXISTING, thirdSnapshotId, FILE_C),
+        manifestEntry(Status.EXISTING, fourthSnapshotId, FILE_D));
+
+    RewriteManifests rewriteManifests = table.rewriteManifests();
+    manifests.forEach(rewriteManifests::deleteManifest);
+    rewriteManifests.addManifest(newManifest);
+    rewriteManifests.commit();
+
+    Set<String> deletedFiles = Sets.newHashSet();
+
+    table.expireSnapshots()
+        .cleanUpFiles(false)
+        .expireOlderThan(t4)
+        .deleteWith(deletedFiles::add)
+        .commit();
+
+    Assert.assertFalse("FILE_A should not be deleted", deletedFiles.contains(FILE_A.path().toString()));

Review comment:
       Sure we can do that




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

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] aokolnychyi commented on a change in pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/ManifestEntry.java
##########
@@ -25,7 +25,7 @@
 import static org.apache.iceberg.types.Types.NestedField.optional;
 import static org.apache.iceberg.types.Types.NestedField.required;
 
-interface ManifestEntry<F extends ContentFile<F>> {
+public interface ManifestEntry<F extends ContentFile<F>> {

Review comment:
       @rdblue had an idea of building `ManifestExpirationManager` similarly to `ManifestMergeManager` and `ManfiestFilterManager` to encapsulate the logic of finding files to delete. The utility class will accept manifests to scan. Then we can either parallelize the scanning using Spark or run that locally. That should be a clean and unified way of handling 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.

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] aokolnychyi commented on a change in pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {

Review comment:
       Definitely +1. Can you submit a separate PR for that?




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableMetadata currentMetadata, TableMetadata originalMetadata) {
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : currentMetadata.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // This snapshot is no longer in the updated metadata
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(validIds, expiredIds);
+  }
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static Set<Long> getPickedAncestorIds(TableMetadata currentMetadata, Set<Long> ancestorIds) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return pickedAncestorSnapshotIds;
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots. If
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(List<Snapshot> currentSnapshots, FileIO io) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, io)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param originalMeta A reference to the table before expiration
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(

Review comment:
       sgtm




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/ManifestEntry.java
##########
@@ -25,7 +25,7 @@
 import static org.apache.iceberg.types.Types.NestedField.optional;
 import static org.apache.iceberg.types.Types.NestedField.required;
 
-interface ManifestEntry<F extends ContentFile<F>> {
+public interface ManifestEntry<F extends ContentFile<F>> {

Review comment:
       Sounds good to me, This is what I was thinking about with that higher level api exposing. So the end user doesn't see anything with the inner enums. I'll get on this asap!




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

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] aokolnychyi commented on a change in pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,
+      TableMetadata currentTableMetadata, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = getValidManifests(currentTableSnapshots, ops);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validIds,
+        currentTableMetadata);
+
+    List<Snapshot> snapshotsNotChangingTableState = filterOutSnapshotsInTableState(validIds,
+        currentTableMetadata);
+
+    // find manifests to clean up that were only referenced by snapshots that have expired
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            currentTableMetadata, expiredIds, ops);
+
+    manifestExpirationChanges.getManifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableOperations ops, TableMetadata originalMetadata) {
+
+    TableMetadata current = ops.refresh();
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : current.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // the snapshot was expired
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(current.snapshots(), validIds, expiredIds);
+  }
+
+  public static class SnapshotExpirationChanges {
+
+    private final List<Snapshot> currentSnapshots;
+    private final Set<Long> validSnapshotIds;
+    private final Set<Long> expiredSnapshotIds;
+
+    public SnapshotExpirationChanges(
+        List<Snapshot> currentSnapshots, Set<Long> validSnapshotIds, Set<Long> expiredSnapshotIds) {
+
+      this.currentSnapshots = currentSnapshots;
+      this.validSnapshotIds = validSnapshotIds;
+      this.expiredSnapshotIds = expiredSnapshotIds;
+    }
+
+    public List<Snapshot> getCurrentSnapshots() {
+      return currentSnapshots;
+    }
+
+    public Set<Long> getValidSnapshotIds() {
+      return validSnapshotIds;
+    }
+
+    public Set<Long> getExpiredSnapshotIds() {
+      return expiredSnapshotIds;
+    }
+  }
+
+  public static class ManifestExpirationChanges {
+
+    private final Set<ManifestFile> manifestsToScan;
+    private final Set<ManifestFile> manifestsToRevert;
+    private final Set<String> manifestsToDelete;
+    private final Set<String> manifestListsToDelete;
+
+    private ManifestExpirationChanges(
+        Set<ManifestFile> manifestsToScan, Set<ManifestFile> manifestsToRevert,
+        Set<String> manifestsToDelete, Set<String> manifestListsToDelete) {
+
+      this.manifestsToScan = manifestsToScan;
+      this.manifestsToRevert = manifestsToRevert;
+      this.manifestsToDelete = manifestsToDelete;
+      this.manifestListsToDelete = manifestListsToDelete;
+    }
+
+
+    public Set<ManifestFile> getManifestsToScan() {
+      return manifestsToScan;
+    }
+
+    public Set<ManifestFile> getManifestsToRevert() {
+      return manifestsToRevert;
+    }
+
+    public Set<String> getManifestsToDelete() {
+      return manifestsToDelete;
+    }
+
+    public Set<String> getManifestListsToDelete() {
+      return manifestListsToDelete;
+    }
+  }
+
+  private static class AncestorIds {

Review comment:
       Actually, we may just have a method that computes `pickedAncestorSnapshotIds` that could accept `ancestorIds` as a param. Computing `ancestorIds` is one line. Then we won't need the wrapper and can return `Set<Long>`.




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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


   Dividing this PR into multiple Sub PRS
   https://github.com/apache/iceberg/pull/1244 - Flag to ignore File Deletes
   https://github.com/apache/iceberg/pull/1245 - Refactor RemoveSnapshots
    - Spark Action //Will be dependent on the above two PRs


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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableMetadata currentMetadata, TableMetadata originalMetadata) {
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : currentMetadata.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // This snapshot is no longer in the updated metadata
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(validIds, expiredIds);
+  }
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static Set<Long> getPickedAncestorIds(TableMetadata currentMetadata, Set<Long> ancestorIds) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return pickedAncestorSnapshotIds;
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots. If
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(List<Snapshot> currentSnapshots, FileIO io) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, io)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param originalMeta A reference to the table before expiration
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<ManifestFile> validManfiests, TableMetadata originalMeta, Set<Long> validSnapshotIds) {
+
+    Set<Long> ancestorIds = SnapshotUtil.ancestorIds(originalMeta.currentSnapshot(), originalMeta::snapshot)

Review comment:
       I have this allergic reaction to guava :P I'm trying to get over 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.

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: api/src/main/java/org/apache/iceberg/ExpireSnapshots.java
##########
@@ -82,4 +82,16 @@
    * @return this for method chaining
    */
   ExpireSnapshots deleteWith(Consumer<String> deleteFunc);
+
+
+  /**
+   * Allows expiration of snapshots without any cleanup of underlying manifest or data files.
+   * <p>
+   * Allows control in removing data and manifest files which may be more efficiently removed non-locally using
+   * an action in a distributed framework such as RemoveOrphanFilesAction.

Review comment:
       Sgtm




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

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] aokolnychyi commented on a change in pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private SparkSession session;
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession session, Table table) {
+    this.session = session;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().deleteExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Execute is a synonym for commit in this implementation. Calling either commit or execute will
+   * launch the Spark equivalent of RemoveSnapshots.
+   *
+   * @return nothing
+   */
+  @Override
+  public ExpireSnapshotResults execute() {
+    localExpireSnapshots.commit();
+
+    TableMetadata currentMetadata = ops.refresh();
+
+    //Locally determine which snapshots have been expired
+    SnapshotExpirationChanges expiredSnapshotChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(currentMetadata, base);
+
+    //Locally determine which manifests will need to be scanned, reverted, deleted
+    ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            expiredSnapshotChanges.validSnapshotIds(), expiredSnapshotChanges.expiredSnapshotIds(),
+            currentMetadata, base, ops.io());
+
+    FileIO io = SparkUtil.serializableFileIO(table);
+
+    //Going the RDD Route because our reader functions all work with full Manifest Files
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(session.sparkContext());
+
+    JavaRDD<ManifestFile> manifestsToScan =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToScan()));
+
+    JavaRDD<ManifestFile> manifestsToRevert =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToRevert()));
+
+    FileIO serializableIO = SparkUtil.serializableFileIO(table);
+
+    Broadcast<Map<Integer, PartitionSpec>> broadcastedSpecLookup =
+        javaSparkContext.broadcast(ops.current().specsById());
+
+    Broadcast<Set<Long>> broadcastValidIDs =
+        javaSparkContext.broadcast(expiredSnapshotChanges.validSnapshotIds());
+
+    JavaRDD<String> filesToDeleteFromScan = manifestsToScan.mapPartitions(manifests -> {
+      Map<Integer, PartitionSpec> specLookup = broadcastedSpecLookup.getValue();
+      Set<Long> validIds = broadcastValidIDs.getValue();
+      Set<String> filesToDelete = new HashSet<>();
+      Tasks.foreach(ImmutableList.copyOf(manifests))
+          .retry(3).suppressFailureWhenFinished()
+          .executeWith(ThreadPools.getWorkerPool())
+          .onFailure((item, exc) -> LOG
+              .warn("Failed to get deleted files: this may cause orphaned data files", exc))
+          .run(manifest -> {
+            // the manifest has deletes, scan it to find files to delete
+            try (ManifestReader<?> reader = ManifestFiles
+                .open(manifest, serializableIO, specLookup)) {
+              for (ManifestEntry<?> entry : reader.entries()) {
+                // if the snapshot ID of the DELETE entry is no longer valid, the data can be deleted
+                if (entry.status() == ManifestEntry.Status.DELETED &&
+                    !validIds.contains(entry.snapshotId())) {
+                  // use toString to ensure the path will not change (Utf8 is reused)
+                  filesToDelete.add(entry.file().path().toString());
+                }
+              }
+            } catch (IOException e) {
+              throw new UncheckedIOException(
+                  String.format("Failed to read manifest file: %s", manifest), e);
+            }
+          });
+      return filesToDelete.iterator();
+    });
+
+    JavaRDD<String> filesToDeleteFromRevert = manifestsToRevert.mapPartitions(manifests -> {
+      Map<Integer, PartitionSpec> specLookup = broadcastedSpecLookup.getValue();
+      Set<String> filesToDelete = new HashSet<>();
+      Tasks.foreach(ImmutableList.copyOf(manifests))
+          .retry(3).suppressFailureWhenFinished()
+          .executeWith(ThreadPools.getWorkerPool())
+          .onFailure((item, exc) -> LOG.warn("Failed to get deleted files: this may cause orphaned data files", exc))
+          .run(manifest -> {
+            // the manifest has deletes, scan it to find files to delete
+            try (ManifestReader<?> reader = ManifestFiles
+                .open(manifest, serializableIO, specLookup)) {
+              for (ManifestEntry<?> entry : reader.entries()) {
+                // delete any ADDED file from manifests that were reverted
+                if (entry.status() == ManifestEntry.Status.ADDED) {
+                  // use toString to ensure the path will not change (Utf8 is reused)
+                  filesToDelete.add(entry.file().path().toString());
+                }
+              }
+            } catch (IOException e) {
+              throw new UncheckedIOException(
+                  String.format("Failed to read manifest file: %s", manifest), e);
+            }
+          });
+      return filesToDelete.iterator();
+    });
+
+    Set<String> dataFilesToDelete = new HashSet<>(
+        filesToDeleteFromRevert.union(filesToDeleteFromScan).collect());
+
+    LOG.warn("Deleting {} data files", dataFilesToDelete.size());
+
+    return new ExpireSnapshotResults(
+        deleteManifestFiles(manifestExpirationChanges.manifestsToDelete()),
+        deleteManifestLists(manifestExpirationChanges.manifestListsToDelete()),
+        deleteDataFiles(dataFilesToDelete));
+  }
+
+  private Long deleteManifestFiles(Set<String> manifestsToDelete) {
+    LOG.warn("Manifests to delete: {}", Joiner.on(", ").join(manifestsToDelete));
+    AtomicReference<Long> deleteCount = new AtomicReference<>(0L);
+
+    Tasks.foreach(manifestsToDelete)
+        .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished()
+        .onFailure((manifest, exc) -> LOG.warn("Delete failed for manifest: {}", manifest, exc))
+        .run(file -> {
+          deleteFunc.accept(file);
+          deleteCount.updateAndGet(v -> v + 1);
+        });
+    return deleteCount.get();
+  }
+
+  private Long deleteManifestLists(Set<String> manifestListsToDelete) {
+    LOG.warn("Manifests Lists to delete: {}", Joiner.on(", ").join(manifestListsToDelete));
+    AtomicReference<Long> deleteCount = new AtomicReference<>(0L);
+
+    Tasks.foreach(manifestListsToDelete)
+        .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished()
+        .onFailure((list, exc) -> LOG.warn("Delete failed for manifest list: {}", list, exc))
+        .run(file -> {
+          deleteFunc.accept(file);
+          deleteCount.updateAndGet(v -> v + 1);
+        });
+    return deleteCount.get();
+  }
+
+  private Long deleteDataFiles(Set<String> dataFilesToDelete) {

Review comment:
       Can we generalize these methods? 




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableMetadata currentMetadata, TableMetadata originalMetadata) {
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : currentMetadata.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // This snapshot is no longer in the updated metadata
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(validIds, expiredIds);
+  }
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static Set<Long> getPickedAncestorIds(TableMetadata currentMetadata, Set<Long> ancestorIds) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return pickedAncestorSnapshotIds;
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots. If
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(List<Snapshot> currentSnapshots, FileIO io) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, io)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param originalMeta A reference to the table before expiration
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<ManifestFile> validManfiests, TableMetadata originalMeta, Set<Long> validSnapshotIds) {
+
+    Set<Long> ancestorIds = SnapshotUtil.ancestorIds(originalMeta.currentSnapshot(), originalMeta::snapshot)
+        .stream().collect(Collectors.toSet());
+    Set<Long> pickedAncestorSnapshotIds = getPickedAncestorIds(originalMeta, ancestorIds);
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    validManfiests.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param originalMeta TableMetadata for the table we are expiring from
+   * @param validSnapshotIds Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> snapshotsNotInTableState(Set<Long> validSnapshotIds, TableMetadata originalMeta) {
+
+    Set<Long> ancestorIds = SnapshotUtil.ancestorIds(originalMeta.currentSnapshot(), originalMeta::snapshot)

Review comment:
       I think only by passing it as an arg ... I"ll try that out




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

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] aokolnychyi commented on a change in pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -152,168 +164,33 @@ private void cleanExpiredSnapshots() {
     // 2. Delete any data files that were deleted by those snapshots and are not in the table
     // 3. Delete any manifests that are no longer used by current snapshots
     // 4. Delete the manifest lists
+    SnapshotExpirationChanges snapshotExpirationChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(ops, base);
 
-    TableMetadata current = ops.refresh();
-
-    Set<Long> validIds = Sets.newHashSet();
-    for (Snapshot snapshot : current.snapshots()) {
-      validIds.add(snapshot.snapshotId());
-    }
-
-    Set<Long> expiredIds = Sets.newHashSet();
-    for (Snapshot snapshot : base.snapshots()) {
-      long snapshotId = snapshot.snapshotId();
-      if (!validIds.contains(snapshotId)) {
-        // the snapshot was expired
-        LOG.info("Expired snapshot: {}", snapshot);
-        expiredIds.add(snapshotId);
-      }
-    }
-
-    if (expiredIds.isEmpty()) {
+    if (snapshotExpirationChanges.getExpiredSnapshotIds().isEmpty()) {
       // if no snapshots were expired, skip cleanup
       return;
     }
 
-    LOG.info("Committed snapshot changes; cleaning up expired manifests and data files.");
-
-    cleanExpiredFiles(current.snapshots(), validIds, expiredIds);
+    LOG.info("Cleaning up expired manifests and data files locally.");
+    cleanExpiredFiles(
+        snapshotExpirationChanges.getCurrentSnapshots(),
+        snapshotExpirationChanges.getValidSnapshotIds(),
+        snapshotExpirationChanges.getExpiredSnapshotIds());
   }
 
-  @SuppressWarnings("checkstyle:CyclomaticComplexity")
-  private void cleanExpiredFiles(List<Snapshot> snapshots, Set<Long> validIds, Set<Long> expiredIds) {
+  private void cleanExpiredFiles(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds) {
     // Reads and deletes are done using Tasks.foreach(...).suppressFailureWhenFinished to complete
     // as much of the delete work as possible and avoid orphaned data or manifest files.
-
-    // this is the set of ancestors of the current table state. when removing snapshots, this must
-    // only remove files that were deleted in an ancestor of the current table state to avoid
-    // physically deleting files that were logically deleted in a commit that was rolled back.
-    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil.ancestorIds(base.currentSnapshot(), base::snapshot));
-
-    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
-    for (long snapshotId : ancestorIds) {
-      String sourceSnapshotId = base.snapshot(snapshotId).summary().get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
-      if (sourceSnapshotId != null) {
-        // protect any snapshot that was cherry-picked into the current table state
-        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
-      }
-    }
-
-    // find manifests to clean up that are still referenced by a valid snapshot, but written by an expired snapshot
-    Set<String> validManifests = Sets.newHashSet();
-    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
-    Tasks.foreach(snapshots).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                for (ManifestFile manifest : manifests) {
-                  validManifests.add(manifest.path());
-
-                  long snapshotId = manifest.snapshotId();
-                  // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
-                  boolean fromValidSnapshots = validIds.contains(snapshotId);
-                  // whether the snapshot that created the manifest was an ancestor of the table state
-                  boolean isFromAncestor = ancestorIds.contains(snapshotId);
-                  // whether the changes in this snapshot have been picked into the current table state
-                  boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
-                  // if the snapshot that wrote this manifest is no longer valid (has expired),
-                  // then delete its deleted files. note that this is only for expired snapshots that are in the
-                  // current table state
-                  if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
-                    manifestsToScan.add(manifest.copy());
-                  }
-                }
-
-              } catch (IOException e) {
-                throw new RuntimeIOException(e,
-                    "Failed to close manifest list: %s", snapshot.manifestListLocation());
-              }
-            });
-
-    // find manifests to clean up that were only referenced by snapshots that have expired
-    Set<String> manifestListsToDelete = Sets.newHashSet();
-    Set<String> manifestsToDelete = Sets.newHashSet();
-    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
-    Tasks.foreach(base.snapshots()).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              long snapshotId = snapshot.snapshotId();
-              if (!validIds.contains(snapshotId)) {
-                // determine whether the changes in this snapshot are in the current table state
-                if (pickedAncestorSnapshotIds.contains(snapshotId)) {
-                  // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
-                  // its changes will expire when the picked snapshot expires.
-                  // A -- C -- D (source=B)
-                  //  `- B <-- this commit
-                  return;
-                }
-
-                long sourceSnapshotId = PropertyUtil.propertyAsLong(
-                    snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
-                if (ancestorIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- B -- C
-                  //  `- D (source=B) <-- this commit
-                  return;
-                }
-
-                if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- C -- E (source=B)
-                  //  `- B `- D (source=B) <-- this commit
-                  return;
-                }
-
-                // find any manifests that are no longer needed
-                try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                  for (ManifestFile manifest : manifests) {
-                    if (!validManifests.contains(manifest.path())) {
-                      manifestsToDelete.add(manifest.path());
-
-                      boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
-                      boolean isFromExpiringSnapshot = expiredIds.contains(manifest.snapshotId());
-
-                      if (isFromAncestor && manifest.hasDeletedFiles()) {
-                        // Only delete data files that were deleted in by an expired snapshot if that
-                        // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
-                        // deleted files and was rolled back will delete files that could be in the current
-                        // table state.
-                        manifestsToScan.add(manifest.copy());
-                      }
-
-                      if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
-                        // Because the manifest was written by a snapshot that is not an ancestor of the
-                        // current table state, the files added in this manifest can be removed. The extra
-                        // check whether the manifest was written by a known snapshot that was expired in
-                        // this commit ensures that the full ancestor list between when the snapshot was
-                        // written and this expiration is known and there is no missing history. If history
-                        // were missing, then the snapshot could be an ancestor of the table state but the
-                        // ancestor ID set would not contain it and this would be unsafe.
-                        manifestsToRevert.add(manifest.copy());
-                      }
-                    }
-                  }
-                } catch (IOException e) {
-                  throw new RuntimeIOException(e,
-                      "Failed to close manifest list: %s", snapshot.manifestListLocation());
-                }
-
-                // add the manifest list to the delete set, if present
-                if (snapshot.manifestListLocation() != null) {
-                  manifestListsToDelete.add(snapshot.manifestListLocation());
-                }
-              }
-            });
-    deleteDataFiles(manifestsToScan, manifestsToRevert, validIds);
-    deleteMetadataFiles(manifestsToDelete, manifestListsToDelete);
+    ExpireSnapshotUtil.ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(

Review comment:
       I think `cleanExpiredFiles` used to accept `currentSnapshots` but use `base` (i.e. the previous version of table state) to perform further analysis. In this case, the arg name seems to be wrong.




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

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] aokolnychyi commented on a change in pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java
##########
@@ -404,6 +404,55 @@ public void dataFilesCleanup() throws IOException {
     Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
   }
 
+  @Test
+  public void noDataFileCleanup() throws IOException {
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    List<ManifestFile> manifests = table.currentSnapshot().dataManifests();
+
+    ManifestFile newManifest = writeManifest(
+        "manifest-file-1.avro",
+        manifestEntry(Status.EXISTING, thirdSnapshotId, FILE_C),
+        manifestEntry(Status.EXISTING, fourthSnapshotId, FILE_D));
+
+    RewriteManifests rewriteManifests = table.rewriteManifests();
+    manifests.forEach(rewriteManifests::deleteManifest);
+    rewriteManifests.addManifest(newManifest);
+    rewriteManifests.commit();
+
+    Set<String> deletedFiles = Sets.newHashSet();
+
+    table.expireSnapshots()
+        .cleanUpFiles(false)
+        .expireOlderThan(t4)
+        .deleteWith(deletedFiles::add)
+        .commit();
+
+    Assert.assertFalse("FILE_A should not be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertFalse("FILE_B should not be deleted", deletedFiles.contains(FILE_B.path().toString()));

Review comment:
       I mean `cleanUpFiles(false)` ensures that `deleteFunc` won't be called. That's exactly what we test here. My bad.




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

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] aokolnychyi commented on a change in pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -152,168 +164,33 @@ private void cleanExpiredSnapshots() {
     // 2. Delete any data files that were deleted by those snapshots and are not in the table
     // 3. Delete any manifests that are no longer used by current snapshots
     // 4. Delete the manifest lists
+    SnapshotExpirationChanges snapshotExpirationChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(ops, base);

Review comment:
       I think it would be more natural to call refresh before and pass two `TableMetadata` instances (base, current) to the utility method. 

##########
File path: api/src/main/java/org/apache/iceberg/ExpireSnapshots.java
##########
@@ -82,4 +82,16 @@
    * @return this for method chaining
    */
   ExpireSnapshots deleteWith(Consumer<String> deleteFunc);
+
+

Review comment:
       nit: extra empty line

##########
File path: api/src/main/java/org/apache/iceberg/ExpireSnapshots.java
##########
@@ -82,4 +82,16 @@
    * @return this for method chaining
    */
   ExpireSnapshots deleteWith(Consumer<String> deleteFunc);
+
+
+  /**
+   * Allows expiration of snapshots without any cleanup of underlying manifest or data files.
+   * <p>
+   * Allows control in removing data and manifest files which may be more efficiently removed non-locally using
+   * an action in a distributed framework such as RemoveOrphanFilesAction.

Review comment:
       Do we want to mention a specific class as a string in the comment? We cannot link it as it is in an inaccessible module. Also, `RemoveOrphanFilesAction` is a different action.
   
   What about a more generic comment? For example, `may be more efficiently removed using a distributed framework through the actions API`? Or something of that sort?

##########
File path: core/src/main/java/org/apache/iceberg/ManifestEntry.java
##########
@@ -25,7 +25,7 @@
 import static org.apache.iceberg.types.Types.NestedField.optional;
 import static org.apache.iceberg.types.Types.NestedField.required;
 
-interface ManifestEntry<F extends ContentFile<F>> {
+public interface ManifestEntry<F extends ContentFile<F>> {

Review comment:
       There were multiple points in time when we thought about opening this API but we always got around it. I think it makes sense now as the entry API may be used by various maintenance jobs unless we can switch to metadata tables for this action too.

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,
+      TableMetadata currentTableMetadata, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = getValidManifests(currentTableSnapshots, ops);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validIds,
+        currentTableMetadata);
+
+    List<Snapshot> snapshotsNotChangingTableState = filterOutSnapshotsInTableState(validIds,
+        currentTableMetadata);
+
+    // find manifests to clean up that were only referenced by snapshots that have expired
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            currentTableMetadata, expiredIds, ops);
+
+    manifestExpirationChanges.getManifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableOperations ops, TableMetadata originalMetadata) {
+
+    TableMetadata current = ops.refresh();
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : current.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // the snapshot was expired
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(current.snapshots(), validIds, expiredIds);
+  }
+
+  public static class SnapshotExpirationChanges {
+
+    private final List<Snapshot> currentSnapshots;
+    private final Set<Long> validSnapshotIds;
+    private final Set<Long> expiredSnapshotIds;
+
+    public SnapshotExpirationChanges(
+        List<Snapshot> currentSnapshots, Set<Long> validSnapshotIds, Set<Long> expiredSnapshotIds) {
+
+      this.currentSnapshots = currentSnapshots;
+      this.validSnapshotIds = validSnapshotIds;
+      this.expiredSnapshotIds = expiredSnapshotIds;
+    }
+
+    public List<Snapshot> getCurrentSnapshots() {
+      return currentSnapshots;
+    }
+
+    public Set<Long> getValidSnapshotIds() {
+      return validSnapshotIds;
+    }
+
+    public Set<Long> getExpiredSnapshotIds() {
+      return expiredSnapshotIds;
+    }
+  }
+
+  public static class ManifestExpirationChanges {
+
+    private final Set<ManifestFile> manifestsToScan;
+    private final Set<ManifestFile> manifestsToRevert;
+    private final Set<String> manifestsToDelete;
+    private final Set<String> manifestListsToDelete;
+
+    private ManifestExpirationChanges(
+        Set<ManifestFile> manifestsToScan, Set<ManifestFile> manifestsToRevert,
+        Set<String> manifestsToDelete, Set<String> manifestListsToDelete) {
+
+      this.manifestsToScan = manifestsToScan;
+      this.manifestsToRevert = manifestsToRevert;
+      this.manifestsToDelete = manifestsToDelete;
+      this.manifestListsToDelete = manifestListsToDelete;
+    }
+
+
+    public Set<ManifestFile> getManifestsToScan() {

Review comment:
       nit: same here

##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -152,168 +164,33 @@ private void cleanExpiredSnapshots() {
     // 2. Delete any data files that were deleted by those snapshots and are not in the table
     // 3. Delete any manifests that are no longer used by current snapshots
     // 4. Delete the manifest lists
+    SnapshotExpirationChanges snapshotExpirationChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(ops, base);
 
-    TableMetadata current = ops.refresh();
-
-    Set<Long> validIds = Sets.newHashSet();
-    for (Snapshot snapshot : current.snapshots()) {
-      validIds.add(snapshot.snapshotId());
-    }
-
-    Set<Long> expiredIds = Sets.newHashSet();
-    for (Snapshot snapshot : base.snapshots()) {
-      long snapshotId = snapshot.snapshotId();
-      if (!validIds.contains(snapshotId)) {
-        // the snapshot was expired
-        LOG.info("Expired snapshot: {}", snapshot);
-        expiredIds.add(snapshotId);
-      }
-    }
-
-    if (expiredIds.isEmpty()) {
+    if (snapshotExpirationChanges.getExpiredSnapshotIds().isEmpty()) {
       // if no snapshots were expired, skip cleanup
       return;
     }
 
-    LOG.info("Committed snapshot changes; cleaning up expired manifests and data files.");
-
-    cleanExpiredFiles(current.snapshots(), validIds, expiredIds);
+    LOG.info("Cleaning up expired manifests and data files locally.");
+    cleanExpiredFiles(
+        snapshotExpirationChanges.getCurrentSnapshots(),
+        snapshotExpirationChanges.getValidSnapshotIds(),
+        snapshotExpirationChanges.getExpiredSnapshotIds());
   }
 
-  @SuppressWarnings("checkstyle:CyclomaticComplexity")
-  private void cleanExpiredFiles(List<Snapshot> snapshots, Set<Long> validIds, Set<Long> expiredIds) {
+  private void cleanExpiredFiles(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds) {
     // Reads and deletes are done using Tasks.foreach(...).suppressFailureWhenFinished to complete
     // as much of the delete work as possible and avoid orphaned data or manifest files.
-
-    // this is the set of ancestors of the current table state. when removing snapshots, this must
-    // only remove files that were deleted in an ancestor of the current table state to avoid
-    // physically deleting files that were logically deleted in a commit that was rolled back.
-    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil.ancestorIds(base.currentSnapshot(), base::snapshot));
-
-    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
-    for (long snapshotId : ancestorIds) {
-      String sourceSnapshotId = base.snapshot(snapshotId).summary().get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
-      if (sourceSnapshotId != null) {
-        // protect any snapshot that was cherry-picked into the current table state
-        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
-      }
-    }
-
-    // find manifests to clean up that are still referenced by a valid snapshot, but written by an expired snapshot
-    Set<String> validManifests = Sets.newHashSet();
-    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
-    Tasks.foreach(snapshots).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                for (ManifestFile manifest : manifests) {
-                  validManifests.add(manifest.path());
-
-                  long snapshotId = manifest.snapshotId();
-                  // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
-                  boolean fromValidSnapshots = validIds.contains(snapshotId);
-                  // whether the snapshot that created the manifest was an ancestor of the table state
-                  boolean isFromAncestor = ancestorIds.contains(snapshotId);
-                  // whether the changes in this snapshot have been picked into the current table state
-                  boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
-                  // if the snapshot that wrote this manifest is no longer valid (has expired),
-                  // then delete its deleted files. note that this is only for expired snapshots that are in the
-                  // current table state
-                  if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
-                    manifestsToScan.add(manifest.copy());
-                  }
-                }
-
-              } catch (IOException e) {
-                throw new RuntimeIOException(e,
-                    "Failed to close manifest list: %s", snapshot.manifestListLocation());
-              }
-            });
-
-    // find manifests to clean up that were only referenced by snapshots that have expired
-    Set<String> manifestListsToDelete = Sets.newHashSet();
-    Set<String> manifestsToDelete = Sets.newHashSet();
-    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
-    Tasks.foreach(base.snapshots()).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              long snapshotId = snapshot.snapshotId();
-              if (!validIds.contains(snapshotId)) {
-                // determine whether the changes in this snapshot are in the current table state
-                if (pickedAncestorSnapshotIds.contains(snapshotId)) {
-                  // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
-                  // its changes will expire when the picked snapshot expires.
-                  // A -- C -- D (source=B)
-                  //  `- B <-- this commit
-                  return;
-                }
-
-                long sourceSnapshotId = PropertyUtil.propertyAsLong(
-                    snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
-                if (ancestorIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- B -- C
-                  //  `- D (source=B) <-- this commit
-                  return;
-                }
-
-                if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- C -- E (source=B)
-                  //  `- B `- D (source=B) <-- this commit
-                  return;
-                }
-
-                // find any manifests that are no longer needed
-                try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                  for (ManifestFile manifest : manifests) {
-                    if (!validManifests.contains(manifest.path())) {
-                      manifestsToDelete.add(manifest.path());
-
-                      boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
-                      boolean isFromExpiringSnapshot = expiredIds.contains(manifest.snapshotId());
-
-                      if (isFromAncestor && manifest.hasDeletedFiles()) {
-                        // Only delete data files that were deleted in by an expired snapshot if that
-                        // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
-                        // deleted files and was rolled back will delete files that could be in the current
-                        // table state.
-                        manifestsToScan.add(manifest.copy());
-                      }
-
-                      if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
-                        // Because the manifest was written by a snapshot that is not an ancestor of the
-                        // current table state, the files added in this manifest can be removed. The extra
-                        // check whether the manifest was written by a known snapshot that was expired in
-                        // this commit ensures that the full ancestor list between when the snapshot was
-                        // written and this expiration is known and there is no missing history. If history
-                        // were missing, then the snapshot could be an ancestor of the table state but the
-                        // ancestor ID set would not contain it and this would be unsafe.
-                        manifestsToRevert.add(manifest.copy());
-                      }
-                    }
-                  }
-                } catch (IOException e) {
-                  throw new RuntimeIOException(e,
-                      "Failed to close manifest list: %s", snapshot.manifestListLocation());
-                }
-
-                // add the manifest list to the delete set, if present
-                if (snapshot.manifestListLocation() != null) {
-                  manifestListsToDelete.add(snapshot.manifestListLocation());
-                }
-              }
-            });
-    deleteDataFiles(manifestsToScan, manifestsToRevert, validIds);
-    deleteMetadataFiles(manifestsToDelete, manifestListsToDelete);
+    ExpireSnapshotUtil.ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            currentTableSnapshots, validIds, expiredIds, base, ops);
+
+    deleteDataFiles(manifestExpirationChanges.getManifestsToScan(),

Review comment:
       For example, something like this:
   
   ```
   deleteDataFiles(expirationChanges.manifestsToScan(), expirationChanges.manifestsToRevert(), validIds);
   deleteMetadataFiles(expirationChanges.manifestsToDelete(), expirationChanges.manifestListsToDelete());
   ```

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,
+      TableMetadata currentTableMetadata, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = getValidManifests(currentTableSnapshots, ops);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validIds,
+        currentTableMetadata);
+
+    List<Snapshot> snapshotsNotChangingTableState = filterOutSnapshotsInTableState(validIds,
+        currentTableMetadata);
+
+    // find manifests to clean up that were only referenced by snapshots that have expired
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            currentTableMetadata, expiredIds, ops);
+
+    manifestExpirationChanges.getManifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableOperations ops, TableMetadata originalMetadata) {
+
+    TableMetadata current = ops.refresh();
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : current.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // the snapshot was expired
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(current.snapshots(), validIds, expiredIds);
+  }
+
+  public static class SnapshotExpirationChanges {
+
+    private final List<Snapshot> currentSnapshots;
+    private final Set<Long> validSnapshotIds;
+    private final Set<Long> expiredSnapshotIds;
+
+    public SnapshotExpirationChanges(
+        List<Snapshot> currentSnapshots, Set<Long> validSnapshotIds, Set<Long> expiredSnapshotIds) {
+
+      this.currentSnapshots = currentSnapshots;
+      this.validSnapshotIds = validSnapshotIds;
+      this.expiredSnapshotIds = expiredSnapshotIds;
+    }
+
+    public List<Snapshot> getCurrentSnapshots() {
+      return currentSnapshots;
+    }
+
+    public Set<Long> getValidSnapshotIds() {
+      return validSnapshotIds;
+    }
+
+    public Set<Long> getExpiredSnapshotIds() {
+      return expiredSnapshotIds;
+    }
+  }
+
+  public static class ManifestExpirationChanges {
+
+    private final Set<ManifestFile> manifestsToScan;
+    private final Set<ManifestFile> manifestsToRevert;
+    private final Set<String> manifestsToDelete;
+    private final Set<String> manifestListsToDelete;
+
+    private ManifestExpirationChanges(
+        Set<ManifestFile> manifestsToScan, Set<ManifestFile> manifestsToRevert,
+        Set<String> manifestsToDelete, Set<String> manifestListsToDelete) {
+
+      this.manifestsToScan = manifestsToScan;
+      this.manifestsToRevert = manifestsToRevert;
+      this.manifestsToDelete = manifestsToDelete;
+      this.manifestListsToDelete = manifestListsToDelete;
+    }
+
+
+    public Set<ManifestFile> getManifestsToScan() {
+      return manifestsToScan;
+    }
+
+    public Set<ManifestFile> getManifestsToRevert() {
+      return manifestsToRevert;
+    }
+
+    public Set<String> getManifestsToDelete() {
+      return manifestsToDelete;
+    }
+
+    public Set<String> getManifestListsToDelete() {
+      return manifestListsToDelete;
+    }
+  }
+
+  private static class AncestorIds {

Review comment:
       We may need to play around with the name as it is not simply ancestor ids, it is used for cherry-picked changes.

##########
File path: api/src/main/java/org/apache/iceberg/ExpireSnapshots.java
##########
@@ -82,4 +82,16 @@
    * @return this for method chaining
    */
   ExpireSnapshots deleteWith(Consumer<String> deleteFunc);
+
+
+  /**
+   * Allows expiration of snapshots without any cleanup of underlying manifest or data files.
+   * <p>
+   * Allows control in removing data and manifest files which may be more efficiently removed non-locally using
+   * an action in a distributed framework such as RemoveOrphanFilesAction.
+   * </p>
+   * @param clean setting this to false will skip deleting expired manifests and files
+   * @return this for method chaining
+   */
+  ExpireSnapshots cleanUpFiles(boolean clean);

Review comment:
       What about `deleteExpiredFiles` as a name?

##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -121,8 +128,8 @@ private TableMetadata internalApply() {
 
     return base.removeSnapshotsIf(snapshot ->
         idsToRemove.contains(snapshot.snapshotId()) ||
-        (expireOlderThan != null && snapshot.timestampMillis() < expireOlderThan &&
-            !idsToRetain.contains(snapshot.snapshotId())));
+            (expireOlderThan != null && snapshot.timestampMillis() < expireOlderThan &&

Review comment:
       nit: same here

##########
File path: core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java
##########
@@ -404,6 +404,55 @@ public void dataFilesCleanup() throws IOException {
     Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
   }
 
+  @Test
+  public void noDataFileCleanup() throws IOException {
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    List<ManifestFile> manifests = table.currentSnapshot().dataManifests();
+
+    ManifestFile newManifest = writeManifest(
+        "manifest-file-1.avro",
+        manifestEntry(Status.EXISTING, thirdSnapshotId, FILE_C),
+        manifestEntry(Status.EXISTING, fourthSnapshotId, FILE_D));
+
+    RewriteManifests rewriteManifests = table.rewriteManifests();
+    manifests.forEach(rewriteManifests::deleteManifest);
+    rewriteManifests.addManifest(newManifest);
+    rewriteManifests.commit();
+
+    Set<String> deletedFiles = Sets.newHashSet();
+
+    table.expireSnapshots()
+        .cleanUpFiles(false)
+        .expireOlderThan(t4)
+        .deleteWith(deletedFiles::add)
+        .commit();
+
+    Assert.assertFalse("FILE_A should not be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertFalse("FILE_B should not be deleted", deletedFiles.contains(FILE_B.path().toString()));

Review comment:
       I am not sure it is actually a valid assertion as `deleteFunc` won't be called at all, no?

##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -152,168 +164,33 @@ private void cleanExpiredSnapshots() {
     // 2. Delete any data files that were deleted by those snapshots and are not in the table
     // 3. Delete any manifests that are no longer used by current snapshots
     // 4. Delete the manifest lists
+    SnapshotExpirationChanges snapshotExpirationChanges =

Review comment:
       The line limit is 120, so this would fit on one line. 

##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -89,7 +96,7 @@ public ExpireSnapshots expireOlderThan(long timestampMillis) {
   @Override
   public ExpireSnapshots retainLast(int numSnapshots) {
     Preconditions.checkArgument(1 <= numSnapshots,
-            "Number of snapshots to retain must be at least 1, cannot be: %s", numSnapshots);

Review comment:
       nit: let's try to avoid cosmetic changes in such large PRs. We can submit them separately if we want to.

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,
+      TableMetadata currentTableMetadata, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = getValidManifests(currentTableSnapshots, ops);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validIds,
+        currentTableMetadata);
+
+    List<Snapshot> snapshotsNotChangingTableState = filterOutSnapshotsInTableState(validIds,
+        currentTableMetadata);
+
+    // find manifests to clean up that were only referenced by snapshots that have expired
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            currentTableMetadata, expiredIds, ops);
+
+    manifestExpirationChanges.getManifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableOperations ops, TableMetadata originalMetadata) {
+
+    TableMetadata current = ops.refresh();
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : current.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // the snapshot was expired
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(current.snapshots(), validIds, expiredIds);
+  }
+
+  public static class SnapshotExpirationChanges {
+
+    private final List<Snapshot> currentSnapshots;

Review comment:
       Actually, how valuable is it to have `currentTableSnapshots` as part of  `SnapshotExpirationChanges`? We can always compute that as `current.snapshots()`.

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,
+      TableMetadata currentTableMetadata, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = getValidManifests(currentTableSnapshots, ops);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validIds,
+        currentTableMetadata);
+
+    List<Snapshot> snapshotsNotChangingTableState = filterOutSnapshotsInTableState(validIds,
+        currentTableMetadata);
+
+    // find manifests to clean up that were only referenced by snapshots that have expired
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            currentTableMetadata, expiredIds, ops);
+
+    manifestExpirationChanges.getManifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableOperations ops, TableMetadata originalMetadata) {
+
+    TableMetadata current = ops.refresh();
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : current.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // the snapshot was expired
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(current.snapshots(), validIds, expiredIds);
+  }
+
+  public static class SnapshotExpirationChanges {
+
+    private final List<Snapshot> currentSnapshots;
+    private final Set<Long> validSnapshotIds;
+    private final Set<Long> expiredSnapshotIds;
+
+    public SnapshotExpirationChanges(
+        List<Snapshot> currentSnapshots, Set<Long> validSnapshotIds, Set<Long> expiredSnapshotIds) {
+
+      this.currentSnapshots = currentSnapshots;
+      this.validSnapshotIds = validSnapshotIds;
+      this.expiredSnapshotIds = expiredSnapshotIds;
+    }
+
+    public List<Snapshot> getCurrentSnapshots() {

Review comment:
       nit: Iceberg does not usually add `get` prefix for getters.

##########
File path: core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java
##########
@@ -404,6 +404,55 @@ public void dataFilesCleanup() throws IOException {
     Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
   }
 
+  @Test
+  public void noDataFileCleanup() throws IOException {
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    List<ManifestFile> manifests = table.currentSnapshot().dataManifests();
+
+    ManifestFile newManifest = writeManifest(
+        "manifest-file-1.avro",
+        manifestEntry(Status.EXISTING, thirdSnapshotId, FILE_C),
+        manifestEntry(Status.EXISTING, fourthSnapshotId, FILE_D));
+
+    RewriteManifests rewriteManifests = table.rewriteManifests();
+    manifests.forEach(rewriteManifests::deleteManifest);
+    rewriteManifests.addManifest(newManifest);
+    rewriteManifests.commit();
+
+    Set<String> deletedFiles = Sets.newHashSet();
+
+    table.expireSnapshots()
+        .cleanUpFiles(false)
+        .expireOlderThan(t4)
+        .deleteWith(deletedFiles::add)
+        .commit();
+
+    Assert.assertFalse("FILE_A should not be deleted", deletedFiles.contains(FILE_A.path().toString()));

Review comment:
       Shall we also have a test case when we check that a manifest and a manifest list is left in place?

##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -152,168 +164,33 @@ private void cleanExpiredSnapshots() {
     // 2. Delete any data files that were deleted by those snapshots and are not in the table
     // 3. Delete any manifests that are no longer used by current snapshots
     // 4. Delete the manifest lists
+    SnapshotExpirationChanges snapshotExpirationChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(ops, base);
 
-    TableMetadata current = ops.refresh();
-
-    Set<Long> validIds = Sets.newHashSet();
-    for (Snapshot snapshot : current.snapshots()) {
-      validIds.add(snapshot.snapshotId());
-    }
-
-    Set<Long> expiredIds = Sets.newHashSet();
-    for (Snapshot snapshot : base.snapshots()) {
-      long snapshotId = snapshot.snapshotId();
-      if (!validIds.contains(snapshotId)) {
-        // the snapshot was expired
-        LOG.info("Expired snapshot: {}", snapshot);
-        expiredIds.add(snapshotId);
-      }
-    }
-
-    if (expiredIds.isEmpty()) {
+    if (snapshotExpirationChanges.getExpiredSnapshotIds().isEmpty()) {
       // if no snapshots were expired, skip cleanup
       return;
     }
 
-    LOG.info("Committed snapshot changes; cleaning up expired manifests and data files.");
-
-    cleanExpiredFiles(current.snapshots(), validIds, expiredIds);
+    LOG.info("Cleaning up expired manifests and data files locally.");
+    cleanExpiredFiles(
+        snapshotExpirationChanges.getCurrentSnapshots(),
+        snapshotExpirationChanges.getValidSnapshotIds(),
+        snapshotExpirationChanges.getExpiredSnapshotIds());
   }
 
-  @SuppressWarnings("checkstyle:CyclomaticComplexity")
-  private void cleanExpiredFiles(List<Snapshot> snapshots, Set<Long> validIds, Set<Long> expiredIds) {
+  private void cleanExpiredFiles(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds) {

Review comment:
       I think it makes sense to prepend `current` to the name to make it more explicit, but I'd remove `Table` from the name as it is obvious and makes the line longer.  If we rename it to `currentSnapshots`, args will fit on one line too.

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {

Review comment:
       Shall we have public methods first?

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,

Review comment:
       We don't need `currentTableSnapshots`, as we can compute it as `current.snapshots()`, also we should simply pass `FileIO` instead of `TableOperations`.

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(

Review comment:
       The utility should not accept or depend on `TableOperations`. We should determine `current` and `base` `TableMetadata` and pass it here. Also, we can pass `FileIO` directly. We don't need to pass complete `TableOperations`.

##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -152,168 +164,33 @@ private void cleanExpiredSnapshots() {
     // 2. Delete any data files that were deleted by those snapshots and are not in the table
     // 3. Delete any manifests that are no longer used by current snapshots
     // 4. Delete the manifest lists
+    SnapshotExpirationChanges snapshotExpirationChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(ops, base);
 
-    TableMetadata current = ops.refresh();
-
-    Set<Long> validIds = Sets.newHashSet();
-    for (Snapshot snapshot : current.snapshots()) {
-      validIds.add(snapshot.snapshotId());
-    }
-
-    Set<Long> expiredIds = Sets.newHashSet();
-    for (Snapshot snapshot : base.snapshots()) {
-      long snapshotId = snapshot.snapshotId();
-      if (!validIds.contains(snapshotId)) {
-        // the snapshot was expired
-        LOG.info("Expired snapshot: {}", snapshot);
-        expiredIds.add(snapshotId);
-      }
-    }
-
-    if (expiredIds.isEmpty()) {
+    if (snapshotExpirationChanges.getExpiredSnapshotIds().isEmpty()) {
       // if no snapshots were expired, skip cleanup
       return;
     }
 
-    LOG.info("Committed snapshot changes; cleaning up expired manifests and data files.");
-
-    cleanExpiredFiles(current.snapshots(), validIds, expiredIds);
+    LOG.info("Cleaning up expired manifests and data files locally.");
+    cleanExpiredFiles(
+        snapshotExpirationChanges.getCurrentSnapshots(),
+        snapshotExpirationChanges.getValidSnapshotIds(),
+        snapshotExpirationChanges.getExpiredSnapshotIds());
   }
 
-  @SuppressWarnings("checkstyle:CyclomaticComplexity")
-  private void cleanExpiredFiles(List<Snapshot> snapshots, Set<Long> validIds, Set<Long> expiredIds) {
+  private void cleanExpiredFiles(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds) {
     // Reads and deletes are done using Tasks.foreach(...).suppressFailureWhenFinished to complete
     // as much of the delete work as possible and avoid orphaned data or manifest files.
-
-    // this is the set of ancestors of the current table state. when removing snapshots, this must
-    // only remove files that were deleted in an ancestor of the current table state to avoid
-    // physically deleting files that were logically deleted in a commit that was rolled back.
-    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil.ancestorIds(base.currentSnapshot(), base::snapshot));
-
-    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
-    for (long snapshotId : ancestorIds) {
-      String sourceSnapshotId = base.snapshot(snapshotId).summary().get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
-      if (sourceSnapshotId != null) {
-        // protect any snapshot that was cherry-picked into the current table state
-        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
-      }
-    }
-
-    // find manifests to clean up that are still referenced by a valid snapshot, but written by an expired snapshot
-    Set<String> validManifests = Sets.newHashSet();
-    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
-    Tasks.foreach(snapshots).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                for (ManifestFile manifest : manifests) {
-                  validManifests.add(manifest.path());
-
-                  long snapshotId = manifest.snapshotId();
-                  // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
-                  boolean fromValidSnapshots = validIds.contains(snapshotId);
-                  // whether the snapshot that created the manifest was an ancestor of the table state
-                  boolean isFromAncestor = ancestorIds.contains(snapshotId);
-                  // whether the changes in this snapshot have been picked into the current table state
-                  boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
-                  // if the snapshot that wrote this manifest is no longer valid (has expired),
-                  // then delete its deleted files. note that this is only for expired snapshots that are in the
-                  // current table state
-                  if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
-                    manifestsToScan.add(manifest.copy());
-                  }
-                }
-
-              } catch (IOException e) {
-                throw new RuntimeIOException(e,
-                    "Failed to close manifest list: %s", snapshot.manifestListLocation());
-              }
-            });
-
-    // find manifests to clean up that were only referenced by snapshots that have expired
-    Set<String> manifestListsToDelete = Sets.newHashSet();
-    Set<String> manifestsToDelete = Sets.newHashSet();
-    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
-    Tasks.foreach(base.snapshots()).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              long snapshotId = snapshot.snapshotId();
-              if (!validIds.contains(snapshotId)) {
-                // determine whether the changes in this snapshot are in the current table state
-                if (pickedAncestorSnapshotIds.contains(snapshotId)) {
-                  // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
-                  // its changes will expire when the picked snapshot expires.
-                  // A -- C -- D (source=B)
-                  //  `- B <-- this commit
-                  return;
-                }
-
-                long sourceSnapshotId = PropertyUtil.propertyAsLong(
-                    snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
-                if (ancestorIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- B -- C
-                  //  `- D (source=B) <-- this commit
-                  return;
-                }
-
-                if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- C -- E (source=B)
-                  //  `- B `- D (source=B) <-- this commit
-                  return;
-                }
-
-                // find any manifests that are no longer needed
-                try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                  for (ManifestFile manifest : manifests) {
-                    if (!validManifests.contains(manifest.path())) {
-                      manifestsToDelete.add(manifest.path());
-
-                      boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
-                      boolean isFromExpiringSnapshot = expiredIds.contains(manifest.snapshotId());
-
-                      if (isFromAncestor && manifest.hasDeletedFiles()) {
-                        // Only delete data files that were deleted in by an expired snapshot if that
-                        // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
-                        // deleted files and was rolled back will delete files that could be in the current
-                        // table state.
-                        manifestsToScan.add(manifest.copy());
-                      }
-
-                      if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
-                        // Because the manifest was written by a snapshot that is not an ancestor of the
-                        // current table state, the files added in this manifest can be removed. The extra
-                        // check whether the manifest was written by a known snapshot that was expired in
-                        // this commit ensures that the full ancestor list between when the snapshot was
-                        // written and this expiration is known and there is no missing history. If history
-                        // were missing, then the snapshot could be an ancestor of the table state but the
-                        // ancestor ID set would not contain it and this would be unsafe.
-                        manifestsToRevert.add(manifest.copy());
-                      }
-                    }
-                  }
-                } catch (IOException e) {
-                  throw new RuntimeIOException(e,
-                      "Failed to close manifest list: %s", snapshot.manifestListLocation());
-                }
-
-                // add the manifest list to the delete set, if present
-                if (snapshot.manifestListLocation() != null) {
-                  manifestListsToDelete.add(snapshot.manifestListLocation());
-                }
-              }
-            });
-    deleteDataFiles(manifestsToScan, manifestsToRevert, validIds);
-    deleteMetadataFiles(manifestsToDelete, manifestListsToDelete);
+    ExpireSnapshotUtil.ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            currentTableSnapshots, validIds, expiredIds, base, ops);
+
+    deleteDataFiles(manifestExpirationChanges.getManifestsToScan(),

Review comment:
       Can we shorten the var name a bit? E.g., `manifestExpirationChanges` -> `expirationChanges` or something. I think it will be obvious from the context and the class name but would allow us to keep two statements below on one line.

##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -152,168 +164,33 @@ private void cleanExpiredSnapshots() {
     // 2. Delete any data files that were deleted by those snapshots and are not in the table
     // 3. Delete any manifests that are no longer used by current snapshots
     // 4. Delete the manifest lists
+    SnapshotExpirationChanges snapshotExpirationChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(ops, base);
 
-    TableMetadata current = ops.refresh();
-
-    Set<Long> validIds = Sets.newHashSet();
-    for (Snapshot snapshot : current.snapshots()) {
-      validIds.add(snapshot.snapshotId());
-    }
-
-    Set<Long> expiredIds = Sets.newHashSet();
-    for (Snapshot snapshot : base.snapshots()) {
-      long snapshotId = snapshot.snapshotId();
-      if (!validIds.contains(snapshotId)) {
-        // the snapshot was expired
-        LOG.info("Expired snapshot: {}", snapshot);
-        expiredIds.add(snapshotId);
-      }
-    }
-
-    if (expiredIds.isEmpty()) {
+    if (snapshotExpirationChanges.getExpiredSnapshotIds().isEmpty()) {
       // if no snapshots were expired, skip cleanup
       return;
     }
 
-    LOG.info("Committed snapshot changes; cleaning up expired manifests and data files.");
-
-    cleanExpiredFiles(current.snapshots(), validIds, expiredIds);
+    LOG.info("Cleaning up expired manifests and data files locally.");
+    cleanExpiredFiles(
+        snapshotExpirationChanges.getCurrentSnapshots(),
+        snapshotExpirationChanges.getValidSnapshotIds(),
+        snapshotExpirationChanges.getExpiredSnapshotIds());
   }
 
-  @SuppressWarnings("checkstyle:CyclomaticComplexity")
-  private void cleanExpiredFiles(List<Snapshot> snapshots, Set<Long> validIds, Set<Long> expiredIds) {
+  private void cleanExpiredFiles(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds) {
     // Reads and deletes are done using Tasks.foreach(...).suppressFailureWhenFinished to complete
     // as much of the delete work as possible and avoid orphaned data or manifest files.
-
-    // this is the set of ancestors of the current table state. when removing snapshots, this must
-    // only remove files that were deleted in an ancestor of the current table state to avoid
-    // physically deleting files that were logically deleted in a commit that was rolled back.
-    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil.ancestorIds(base.currentSnapshot(), base::snapshot));
-
-    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
-    for (long snapshotId : ancestorIds) {
-      String sourceSnapshotId = base.snapshot(snapshotId).summary().get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
-      if (sourceSnapshotId != null) {
-        // protect any snapshot that was cherry-picked into the current table state
-        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
-      }
-    }
-
-    // find manifests to clean up that are still referenced by a valid snapshot, but written by an expired snapshot
-    Set<String> validManifests = Sets.newHashSet();
-    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
-    Tasks.foreach(snapshots).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                for (ManifestFile manifest : manifests) {
-                  validManifests.add(manifest.path());
-
-                  long snapshotId = manifest.snapshotId();
-                  // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
-                  boolean fromValidSnapshots = validIds.contains(snapshotId);
-                  // whether the snapshot that created the manifest was an ancestor of the table state
-                  boolean isFromAncestor = ancestorIds.contains(snapshotId);
-                  // whether the changes in this snapshot have been picked into the current table state
-                  boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
-                  // if the snapshot that wrote this manifest is no longer valid (has expired),
-                  // then delete its deleted files. note that this is only for expired snapshots that are in the
-                  // current table state
-                  if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
-                    manifestsToScan.add(manifest.copy());
-                  }
-                }
-
-              } catch (IOException e) {
-                throw new RuntimeIOException(e,
-                    "Failed to close manifest list: %s", snapshot.manifestListLocation());
-              }
-            });
-
-    // find manifests to clean up that were only referenced by snapshots that have expired
-    Set<String> manifestListsToDelete = Sets.newHashSet();
-    Set<String> manifestsToDelete = Sets.newHashSet();
-    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
-    Tasks.foreach(base.snapshots()).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              long snapshotId = snapshot.snapshotId();
-              if (!validIds.contains(snapshotId)) {
-                // determine whether the changes in this snapshot are in the current table state
-                if (pickedAncestorSnapshotIds.contains(snapshotId)) {
-                  // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
-                  // its changes will expire when the picked snapshot expires.
-                  // A -- C -- D (source=B)
-                  //  `- B <-- this commit
-                  return;
-                }
-
-                long sourceSnapshotId = PropertyUtil.propertyAsLong(
-                    snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
-                if (ancestorIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- B -- C
-                  //  `- D (source=B) <-- this commit
-                  return;
-                }
-
-                if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- C -- E (source=B)
-                  //  `- B `- D (source=B) <-- this commit
-                  return;
-                }
-
-                // find any manifests that are no longer needed
-                try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                  for (ManifestFile manifest : manifests) {
-                    if (!validManifests.contains(manifest.path())) {
-                      manifestsToDelete.add(manifest.path());
-
-                      boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
-                      boolean isFromExpiringSnapshot = expiredIds.contains(manifest.snapshotId());
-
-                      if (isFromAncestor && manifest.hasDeletedFiles()) {
-                        // Only delete data files that were deleted in by an expired snapshot if that
-                        // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
-                        // deleted files and was rolled back will delete files that could be in the current
-                        // table state.
-                        manifestsToScan.add(manifest.copy());
-                      }
-
-                      if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
-                        // Because the manifest was written by a snapshot that is not an ancestor of the
-                        // current table state, the files added in this manifest can be removed. The extra
-                        // check whether the manifest was written by a known snapshot that was expired in
-                        // this commit ensures that the full ancestor list between when the snapshot was
-                        // written and this expiration is known and there is no missing history. If history
-                        // were missing, then the snapshot could be an ancestor of the table state but the
-                        // ancestor ID set would not contain it and this would be unsafe.
-                        manifestsToRevert.add(manifest.copy());
-                      }
-                    }
-                  }
-                } catch (IOException e) {
-                  throw new RuntimeIOException(e,
-                      "Failed to close manifest list: %s", snapshot.manifestListLocation());
-                }
-
-                // add the manifest list to the delete set, if present
-                if (snapshot.manifestListLocation() != null) {
-                  manifestListsToDelete.add(snapshot.manifestListLocation());
-                }
-              }
-            });
-    deleteDataFiles(manifestsToScan, manifestsToRevert, validIds);
-    deleteMetadataFiles(manifestsToDelete, manifestListsToDelete);
+    ExpireSnapshotUtil.ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(

Review comment:
       Here we pass `base` as `currentTableMetadata`. Is it on purpose?

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,
+      TableMetadata currentTableMetadata, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = getValidManifests(currentTableSnapshots, ops);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validIds,
+        currentTableMetadata);
+
+    List<Snapshot> snapshotsNotChangingTableState = filterOutSnapshotsInTableState(validIds,
+        currentTableMetadata);
+
+    // find manifests to clean up that were only referenced by snapshots that have expired
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            currentTableMetadata, expiredIds, ops);
+
+    manifestExpirationChanges.getManifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableOperations ops, TableMetadata originalMetadata) {
+
+    TableMetadata current = ops.refresh();
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : current.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // the snapshot was expired
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(current.snapshots(), validIds, expiredIds);
+  }
+
+  public static class SnapshotExpirationChanges {
+
+    private final List<Snapshot> currentSnapshots;
+    private final Set<Long> validSnapshotIds;
+    private final Set<Long> expiredSnapshotIds;
+
+    public SnapshotExpirationChanges(
+        List<Snapshot> currentSnapshots, Set<Long> validSnapshotIds, Set<Long> expiredSnapshotIds) {
+
+      this.currentSnapshots = currentSnapshots;
+      this.validSnapshotIds = validSnapshotIds;
+      this.expiredSnapshotIds = expiredSnapshotIds;
+    }
+
+    public List<Snapshot> getCurrentSnapshots() {
+      return currentSnapshots;
+    }
+
+    public Set<Long> getValidSnapshotIds() {
+      return validSnapshotIds;
+    }
+
+    public Set<Long> getExpiredSnapshotIds() {
+      return expiredSnapshotIds;
+    }
+  }
+
+  public static class ManifestExpirationChanges {
+
+    private final Set<ManifestFile> manifestsToScan;
+    private final Set<ManifestFile> manifestsToRevert;
+    private final Set<String> manifestsToDelete;
+    private final Set<String> manifestListsToDelete;
+
+    private ManifestExpirationChanges(
+        Set<ManifestFile> manifestsToScan, Set<ManifestFile> manifestsToRevert,
+        Set<String> manifestsToDelete, Set<String> manifestListsToDelete) {
+
+      this.manifestsToScan = manifestsToScan;
+      this.manifestsToRevert = manifestsToRevert;
+      this.manifestsToDelete = manifestsToDelete;
+      this.manifestListsToDelete = manifestListsToDelete;
+    }
+
+
+    public Set<ManifestFile> getManifestsToScan() {
+      return manifestsToScan;
+    }
+
+    public Set<ManifestFile> getManifestsToRevert() {
+      return manifestsToRevert;
+    }
+
+    public Set<String> getManifestsToDelete() {
+      return manifestsToDelete;
+    }
+
+    public Set<String> getManifestListsToDelete() {
+      return manifestListsToDelete;
+    }
+  }
+
+  private static class AncestorIds {
+
+    private final Set<Long> ancestorIds;
+    private final Set<Long> pickedAncestorIds;
+
+    private AncestorIds(Set<Long> ancestorIds, Set<Long> pickedAncestorIds) {
+      this.ancestorIds = ancestorIds;
+      this.pickedAncestorIds = pickedAncestorIds;
+    }
+
+    public Set<Long> getPickedAncestorIds() {

Review comment:
       nit: same here

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,
+      TableMetadata currentTableMetadata, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = getValidManifests(currentTableSnapshots, ops);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validIds,
+        currentTableMetadata);
+
+    List<Snapshot> snapshotsNotChangingTableState = filterOutSnapshotsInTableState(validIds,
+        currentTableMetadata);
+
+    // find manifests to clean up that were only referenced by snapshots that have expired
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            currentTableMetadata, expiredIds, ops);
+
+    manifestExpirationChanges.getManifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableOperations ops, TableMetadata originalMetadata) {
+
+    TableMetadata current = ops.refresh();
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : current.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // the snapshot was expired
+        LOG.info("Expired snapshot: {}", snapshot);

Review comment:
       I think it would make sense to log this entry in callers such as `RemoveSnapshots` rather than in the utility. For example, we could log a list of expired snapshots after this method is called.

##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -152,168 +164,33 @@ private void cleanExpiredSnapshots() {
     // 2. Delete any data files that were deleted by those snapshots and are not in the table
     // 3. Delete any manifests that are no longer used by current snapshots
     // 4. Delete the manifest lists
+    SnapshotExpirationChanges snapshotExpirationChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(ops, base);
 
-    TableMetadata current = ops.refresh();
-
-    Set<Long> validIds = Sets.newHashSet();
-    for (Snapshot snapshot : current.snapshots()) {
-      validIds.add(snapshot.snapshotId());
-    }
-
-    Set<Long> expiredIds = Sets.newHashSet();
-    for (Snapshot snapshot : base.snapshots()) {
-      long snapshotId = snapshot.snapshotId();
-      if (!validIds.contains(snapshotId)) {
-        // the snapshot was expired
-        LOG.info("Expired snapshot: {}", snapshot);
-        expiredIds.add(snapshotId);
-      }
-    }
-
-    if (expiredIds.isEmpty()) {
+    if (snapshotExpirationChanges.getExpiredSnapshotIds().isEmpty()) {
       // if no snapshots were expired, skip cleanup
       return;
     }
 
-    LOG.info("Committed snapshot changes; cleaning up expired manifests and data files.");
-
-    cleanExpiredFiles(current.snapshots(), validIds, expiredIds);
+    LOG.info("Cleaning up expired manifests and data files locally.");
+    cleanExpiredFiles(
+        snapshotExpirationChanges.getCurrentSnapshots(),
+        snapshotExpirationChanges.getValidSnapshotIds(),
+        snapshotExpirationChanges.getExpiredSnapshotIds());
   }
 
-  @SuppressWarnings("checkstyle:CyclomaticComplexity")
-  private void cleanExpiredFiles(List<Snapshot> snapshots, Set<Long> validIds, Set<Long> expiredIds) {
+  private void cleanExpiredFiles(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds) {
     // Reads and deletes are done using Tasks.foreach(...).suppressFailureWhenFinished to complete
     // as much of the delete work as possible and avoid orphaned data or manifest files.
-
-    // this is the set of ancestors of the current table state. when removing snapshots, this must
-    // only remove files that were deleted in an ancestor of the current table state to avoid
-    // physically deleting files that were logically deleted in a commit that was rolled back.
-    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil.ancestorIds(base.currentSnapshot(), base::snapshot));
-
-    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
-    for (long snapshotId : ancestorIds) {
-      String sourceSnapshotId = base.snapshot(snapshotId).summary().get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
-      if (sourceSnapshotId != null) {
-        // protect any snapshot that was cherry-picked into the current table state
-        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
-      }
-    }
-
-    // find manifests to clean up that are still referenced by a valid snapshot, but written by an expired snapshot
-    Set<String> validManifests = Sets.newHashSet();
-    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
-    Tasks.foreach(snapshots).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                for (ManifestFile manifest : manifests) {
-                  validManifests.add(manifest.path());
-
-                  long snapshotId = manifest.snapshotId();
-                  // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
-                  boolean fromValidSnapshots = validIds.contains(snapshotId);
-                  // whether the snapshot that created the manifest was an ancestor of the table state
-                  boolean isFromAncestor = ancestorIds.contains(snapshotId);
-                  // whether the changes in this snapshot have been picked into the current table state
-                  boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
-                  // if the snapshot that wrote this manifest is no longer valid (has expired),
-                  // then delete its deleted files. note that this is only for expired snapshots that are in the
-                  // current table state
-                  if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
-                    manifestsToScan.add(manifest.copy());
-                  }
-                }
-
-              } catch (IOException e) {
-                throw new RuntimeIOException(e,
-                    "Failed to close manifest list: %s", snapshot.manifestListLocation());
-              }
-            });
-
-    // find manifests to clean up that were only referenced by snapshots that have expired
-    Set<String> manifestListsToDelete = Sets.newHashSet();
-    Set<String> manifestsToDelete = Sets.newHashSet();
-    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
-    Tasks.foreach(base.snapshots()).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              long snapshotId = snapshot.snapshotId();
-              if (!validIds.contains(snapshotId)) {
-                // determine whether the changes in this snapshot are in the current table state
-                if (pickedAncestorSnapshotIds.contains(snapshotId)) {
-                  // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
-                  // its changes will expire when the picked snapshot expires.
-                  // A -- C -- D (source=B)
-                  //  `- B <-- this commit
-                  return;
-                }
-
-                long sourceSnapshotId = PropertyUtil.propertyAsLong(
-                    snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
-                if (ancestorIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- B -- C
-                  //  `- D (source=B) <-- this commit
-                  return;
-                }
-
-                if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- C -- E (source=B)
-                  //  `- B `- D (source=B) <-- this commit
-                  return;
-                }
-
-                // find any manifests that are no longer needed
-                try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                  for (ManifestFile manifest : manifests) {
-                    if (!validManifests.contains(manifest.path())) {
-                      manifestsToDelete.add(manifest.path());
-
-                      boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
-                      boolean isFromExpiringSnapshot = expiredIds.contains(manifest.snapshotId());
-
-                      if (isFromAncestor && manifest.hasDeletedFiles()) {
-                        // Only delete data files that were deleted in by an expired snapshot if that
-                        // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
-                        // deleted files and was rolled back will delete files that could be in the current
-                        // table state.
-                        manifestsToScan.add(manifest.copy());
-                      }
-
-                      if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
-                        // Because the manifest was written by a snapshot that is not an ancestor of the
-                        // current table state, the files added in this manifest can be removed. The extra
-                        // check whether the manifest was written by a known snapshot that was expired in
-                        // this commit ensures that the full ancestor list between when the snapshot was
-                        // written and this expiration is known and there is no missing history. If history
-                        // were missing, then the snapshot could be an ancestor of the table state but the
-                        // ancestor ID set would not contain it and this would be unsafe.
-                        manifestsToRevert.add(manifest.copy());
-                      }
-                    }
-                  }
-                } catch (IOException e) {
-                  throw new RuntimeIOException(e,
-                      "Failed to close manifest list: %s", snapshot.manifestListLocation());
-                }
-
-                // add the manifest list to the delete set, if present
-                if (snapshot.manifestListLocation() != null) {
-                  manifestListsToDelete.add(snapshot.manifestListLocation());
-                }
-              }
-            });
-    deleteDataFiles(manifestsToScan, manifestsToRevert, validIds);
-    deleteMetadataFiles(manifestsToDelete, manifestListsToDelete);
+    ExpireSnapshotUtil.ManifestExpirationChanges manifestExpirationChanges =

Review comment:
       Shall we import `ManifestExpirationChanges` directly so that we can refer to it without the enclosing class?




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

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] aokolnychyi edited a comment on pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

Posted by GitBox <gi...@apache.org>.
aokolnychyi edited a comment on pull request #1211:
URL: https://github.com/apache/iceberg/pull/1211#issuecomment-659490881


   Thanks, @RussellSpitzer! Let me take a look at this today.
   
   To give a bit of background, we had a case where people compacted their table that reduced the number of files from 5+ million to under 1 million in a short period of time. That process generated a huge number of new snapshots as different parts were optimized separately (one leaf partition at a time). In addition, there was a huge number of manifests to scan and a lot of expired data files. As a result, the expiry of snapshots was really slow.
   
   This is an attempt to fix such edge cases.


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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,
+      TableMetadata currentTableMetadata, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = getValidManifests(currentTableSnapshots, ops);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validIds,
+        currentTableMetadata);
+
+    List<Snapshot> snapshotsNotChangingTableState = filterOutSnapshotsInTableState(validIds,
+        currentTableMetadata);
+
+    // find manifests to clean up that were only referenced by snapshots that have expired
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            currentTableMetadata, expiredIds, ops);
+
+    manifestExpirationChanges.getManifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableOperations ops, TableMetadata originalMetadata) {
+
+    TableMetadata current = ops.refresh();
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : current.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // the snapshot was expired
+        LOG.info("Expired snapshot: {}", snapshot);

Review comment:
       The only difficulty here is that we don't save the actual snapshots in this method, I can change the wrapper to save actual Snapshot objects instead of just their Ids? This would require a bit more refactoring
   




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(

Review comment:
       I feel like "base" is not very clear, but I'll take old?




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,
+      TableMetadata currentTableMetadata, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = getValidManifests(currentTableSnapshots, ops);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validIds,

Review comment:
       ManifestsToScan contains entries from two different loops (This is the same as the old code)
   
   Snapshots that survived expiration loop
   1. List all Snapshots which haven't expired
   2. Get every manifest from each of those snapshots
   3. If that manifest is originally from an expired snapshot AND was cherrypicked or is an ancestor AND there are deletes in the manifest
      Then it's possible that those deletes can be realized and we can remove those files so we need to scan that manifest
   
   Snapshots which did not survive Expiration and don't effect the current state
   1. List all expired snapshots from the old state which have no effect on the current table state (SnapshotsNotInTableState)
   2. Get all the manifests in these snapshots that we didn't scan in the above loop 
   3. For these expired manifests check if they need to be scanned
     a. If they have deletes we need to Scan
     b. If they have files added we need to Scan (for Revert)
     




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/ManifestEntry.java
##########
@@ -25,7 +25,7 @@
 import static org.apache.iceberg.types.Types.NestedField.optional;
 import static org.apache.iceberg.types.Types.NestedField.required;
 
-interface ManifestEntry<F extends ContentFile<F>> {
+public interface ManifestEntry<F extends ContentFile<F>> {

Review comment:
       We could, if it was more palatable, expose a higher level api such as getAddedFiles, getExistingFiles, .... and so on so that callers don't need to know this lower level detail. I'm not sure if that is better since you still basically need to expose the enum values anyway ...




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableMetadata currentMetadata, TableMetadata originalMetadata) {
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : currentMetadata.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // This snapshot is no longer in the updated metadata
+        LOG.info("Expired snapshot: {}", snapshot);

Review comment:
       The hard part about that is that the caller does not get the "Snapshot", it only gets the "id" of the snapshots back so I can't recreate this logging message. I could change it to "Expired snapshot: #x" or change the return type to return the whole snapshot object?




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(

Review comment:
       That makes sense, I"ll switch all calls to using TableMetdata where it makes sense




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

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] aokolnychyi commented on a change in pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/ManifestFiles.java
##########
@@ -138,8 +138,8 @@ private ManifestFiles() {
     return open(manifest, io, null);
   }
 
-  static ManifestReader<?> open(ManifestFile manifest, FileIO io,
-                                Map<Integer, PartitionSpec> specsById) {
+  public static ManifestReader<?> open(

Review comment:
       nit: this can actually fit on one line

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotResults.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.actions;
+
+public class ExpireSnapshotResults {

Review comment:
       Let's call it `ExpireSnapshotsActionResult` to be consistent with other actions.

##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -51,6 +51,7 @@
 import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT;
 
 class RemoveSnapshots implements ExpireSnapshots {
+

Review comment:
       nit: cosmetic change

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableMetadata currentMetadata, TableMetadata originalMetadata) {
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : currentMetadata.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // This snapshot is no longer in the updated metadata
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(validIds, expiredIds);
+  }
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static Set<Long> getPickedAncestorIds(TableMetadata currentMetadata, Set<Long> ancestorIds) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return pickedAncestorSnapshotIds;
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots. If
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(List<Snapshot> currentSnapshots, FileIO io) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, io)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",

Review comment:
       nit: the args can be on one line

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableMetadata currentMetadata, TableMetadata originalMetadata) {
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : currentMetadata.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // This snapshot is no longer in the updated metadata
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(validIds, expiredIds);
+  }
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static Set<Long> getPickedAncestorIds(TableMetadata currentMetadata, Set<Long> ancestorIds) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return pickedAncestorSnapshotIds;
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots. If
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(List<Snapshot> currentSnapshots, FileIO io) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, io)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param originalMeta A reference to the table before expiration
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<ManifestFile> validManfiests, TableMetadata originalMeta, Set<Long> validSnapshotIds) {
+
+    Set<Long> ancestorIds = SnapshotUtil.ancestorIds(originalMeta.currentSnapshot(), originalMeta::snapshot)
+        .stream().collect(Collectors.toSet());
+    Set<Long> pickedAncestorSnapshotIds = getPickedAncestorIds(originalMeta, ancestorIds);
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    validManfiests.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param originalMeta TableMetadata for the table we are expiring from
+   * @param validSnapshotIds Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> snapshotsNotInTableState(Set<Long> validSnapshotIds, TableMetadata originalMeta) {
+
+    Set<Long> ancestorIds = SnapshotUtil.ancestorIds(originalMeta.currentSnapshot(), originalMeta::snapshot)

Review comment:
       Can we avoid computing ancestor ids twice?

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private SparkSession session;

Review comment:
       We tend to put Spark-related vars at the beginning of the list, make them final and call session as `spark`. For example, we have the following in `RewriteManifestsAction`:
   
   ```
     private final SparkSession spark;
     private final JavaSparkContext sparkContext;
   ```
   
   And init that in the constructor:
   
   ```
       this.spark = spark;
       this.sparkContext = new JavaSparkContext(spark.sparkContext());
   ```

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private SparkSession session;
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession session, Table table) {
+    this.session = session;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().deleteExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Execute is a synonym for commit in this implementation. Calling either commit or execute will
+   * launch the Spark equivalent of RemoveSnapshots.
+   *
+   * @return nothing
+   */
+  @Override
+  public ExpireSnapshotResults execute() {
+    localExpireSnapshots.commit();
+
+    TableMetadata currentMetadata = ops.refresh();
+
+    //Locally determine which snapshots have been expired
+    SnapshotExpirationChanges expiredSnapshotChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(currentMetadata, base);
+
+    //Locally determine which manifests will need to be scanned, reverted, deleted
+    ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            expiredSnapshotChanges.validSnapshotIds(), expiredSnapshotChanges.expiredSnapshotIds(),
+            currentMetadata, base, ops.io());
+
+    FileIO io = SparkUtil.serializableFileIO(table);
+
+    //Going the RDD Route because our reader functions all work with full Manifest Files
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(session.sparkContext());
+
+    JavaRDD<ManifestFile> manifestsToScan =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToScan()));
+
+    JavaRDD<ManifestFile> manifestsToRevert =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToRevert()));
+
+    FileIO serializableIO = SparkUtil.serializableFileIO(table);
+
+    Broadcast<Map<Integer, PartitionSpec>> broadcastedSpecLookup =

Review comment:
       I think this broadcast optional as there may be a couple of specs at most.

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return

Review comment:
       nit: empty return statement

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return

Review comment:
       nit: empty return statement

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableMetadata currentMetadata, TableMetadata originalMetadata) {
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : currentMetadata.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // This snapshot is no longer in the updated metadata
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(validIds, expiredIds);
+  }
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}

Review comment:
       I'd put the static variable and the constructor at the beginning of the class and then public static methods.

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableMetadata currentMetadata, TableMetadata originalMetadata) {
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : currentMetadata.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // This snapshot is no longer in the updated metadata
+        LOG.info("Expired snapshot: {}", snapshot);

Review comment:
       I cannot comment on the [old](https://github.com/apache/iceberg/pull/1211#discussion_r456635324) thread so I'll do it here.
   
   What I meant is simply taking this line out and logging a set of expired snapshots in the caller. 

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots

Review comment:
       nit: `Metada`

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableMetadata currentMetadata, TableMetadata originalMetadata) {
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : currentMetadata.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // This snapshot is no longer in the updated metadata
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(validIds, expiredIds);
+  }
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static Set<Long> getPickedAncestorIds(TableMetadata currentMetadata, Set<Long> ancestorIds) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return pickedAncestorSnapshotIds;
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots. If
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(List<Snapshot> currentSnapshots, FileIO io) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, io)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param originalMeta A reference to the table before expiration
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<ManifestFile> validManfiests, TableMetadata originalMeta, Set<Long> validSnapshotIds) {
+
+    Set<Long> ancestorIds = SnapshotUtil.ancestorIds(originalMeta.currentSnapshot(), originalMeta::snapshot)

Review comment:
       Naming `originalMetadata` as `base` and `currentMetadata` as `current` may potentially reduce the length of lines and make code shorter in the utility class. We use it in a couple of places so it makes sense exploring, but not required.
   
   Then these lines can be replaced with one:
   
   ```
   Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil.ancestorIds(base.currentSnapshot(), base::snapshot));
   ```

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableMetadata currentMetadata, TableMetadata originalMetadata) {
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : currentMetadata.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // This snapshot is no longer in the updated metadata
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(validIds, expiredIds);
+  }
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static Set<Long> getPickedAncestorIds(TableMetadata currentMetadata, Set<Long> ancestorIds) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return pickedAncestorSnapshotIds;
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots. If
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(List<Snapshot> currentSnapshots, FileIO io) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, io)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param originalMeta A reference to the table before expiration
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<ManifestFile> validManfiests, TableMetadata originalMeta, Set<Long> validSnapshotIds) {

Review comment:
       nit: typo in `validManfiests`

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private SparkSession session;
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession session, Table table) {
+    this.session = session;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().deleteExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Execute is a synonym for commit in this implementation. Calling either commit or execute will
+   * launch the Spark equivalent of RemoveSnapshots.
+   *
+   * @return nothing
+   */
+  @Override
+  public ExpireSnapshotResults execute() {
+    localExpireSnapshots.commit();
+
+    TableMetadata currentMetadata = ops.refresh();
+
+    //Locally determine which snapshots have been expired
+    SnapshotExpirationChanges expiredSnapshotChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(currentMetadata, base);
+
+    //Locally determine which manifests will need to be scanned, reverted, deleted
+    ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            expiredSnapshotChanges.validSnapshotIds(), expiredSnapshotChanges.expiredSnapshotIds(),
+            currentMetadata, base, ops.io());
+
+    FileIO io = SparkUtil.serializableFileIO(table);
+
+    //Going the RDD Route because our reader functions all work with full Manifest Files
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(session.sparkContext());
+
+    JavaRDD<ManifestFile> manifestsToScan =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToScan()));
+
+    JavaRDD<ManifestFile> manifestsToRevert =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToRevert()));
+
+    FileIO serializableIO = SparkUtil.serializableFileIO(table);

Review comment:
       It is important to broadcast `FileIO` as it is the most expensive part.

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(

Review comment:
       nit: If we renamed args to `current` and `base`, this would also fit on one line.
   
   ```
   ... (TableMetadata current, TableMetadata base) {
   
   }
   ```

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;

Review comment:
       We need to verify that saving the base version in the constructor is safe and doesn't cause any issues if there is a concurrent operation after we checked the base and before we committed.

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableMetadata currentMetadata, TableMetadata originalMetadata) {
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : currentMetadata.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // This snapshot is no longer in the updated metadata
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(validIds, expiredIds);
+  }
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static Set<Long> getPickedAncestorIds(TableMetadata currentMetadata, Set<Long> ancestorIds) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return pickedAncestorSnapshotIds;
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots. If
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(List<Snapshot> currentSnapshots, FileIO io) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, io)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param originalMeta A reference to the table before expiration
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<ManifestFile> validManfiests, TableMetadata originalMeta, Set<Long> validSnapshotIds) {
+
+    Set<Long> ancestorIds = SnapshotUtil.ancestorIds(originalMeta.currentSnapshot(), originalMeta::snapshot)
+        .stream().collect(Collectors.toSet());
+    Set<Long> pickedAncestorSnapshotIds = getPickedAncestorIds(originalMeta, ancestorIds);
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    validManfiests.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param originalMeta TableMetadata for the table we are expiring from
+   * @param validSnapshotIds Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> snapshotsNotInTableState(Set<Long> validSnapshotIds, TableMetadata originalMeta) {

Review comment:
       What about adding a verb to the name here too?

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableMetadata currentMetadata, TableMetadata originalMetadata) {
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : currentMetadata.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // This snapshot is no longer in the updated metadata
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(validIds, expiredIds);
+  }
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static Set<Long> getPickedAncestorIds(TableMetadata currentMetadata, Set<Long> ancestorIds) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return pickedAncestorSnapshotIds;
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots. If
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(List<Snapshot> currentSnapshots, FileIO io) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, io)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param originalMeta A reference to the table before expiration
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(

Review comment:
       Will it make sense to add a verb to the method name? Like `findValidManifests...`?

##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * Our goal is to determine which manifest files we actually need to read through because they
+   * may refer to files which are no longer accessible from any valid snapshot and do not effect
+   * the current table.
+   *
+   * For this we need to look through
+   *   1. Snapshots which have not expired but contain manifests from expired snapshots
+   *   2. Snapshots which have expired and contain manifests referring to now orphaned files
+   *
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentMetadata       The table metadata from after the snapshot expiration
+   * @param originalMetadata      The table metadata from before the snapshot expiration
+   * @param io                    FileIO for reading manifest info
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(Set<Long> validIds,
+      Set<Long> expiredIds, TableMetadata currentMetadata, TableMetadata originalMetadata, FileIO io) {
+
+    List<Snapshot> currentSnapshots = currentMetadata.snapshots();
+
+    //Snapshots which are not expired but refer to manifests from expired snapshots
+    Set<ManifestFile> validManifests = getValidManifests(currentSnapshots, io);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validManifests,
+        originalMetadata, validIds);
+
+    //Snapshots which are expired and do not effect the current table
+    List<Snapshot> snapshotsNotChangingTableState = snapshotsNotInTableState(validIds, originalMetadata);
+    ManifestExpirationChanges manifestExpirationChanges =
+        findExpiredManifestsInUnusedSnapshots(snapshotsNotChangingTableState, validManifests,
+            originalMetadata, expiredIds, io);
+
+    manifestExpirationChanges.manifestsToScan().addAll(manifestsToScan);
+    return manifestExpirationChanges;
+  }
+
+  /**
+   * Compares the Snapshots from the two TableMetadata objects and identifies the snapshots
+   * still in use and those no longer in use
+   * @param currentMetadata Metadata from a table after an expiration of snapshots
+   * @param originalMetadata Metada from the table before expiration of snapshots
+   * @return
+   */
+  public static SnapshotExpirationChanges getExpiredSnapshots(
+      TableMetadata currentMetadata, TableMetadata originalMetadata) {
+
+    Set<Long> validIds = Sets.newHashSet();
+    for (Snapshot snapshot : currentMetadata.snapshots()) {
+      validIds.add(snapshot.snapshotId());
+    }
+
+    Set<Long> expiredIds = Sets.newHashSet();
+    for (Snapshot snapshot : originalMetadata.snapshots()) {
+      long snapshotId = snapshot.snapshotId();
+      if (!validIds.contains(snapshotId)) {
+        // This snapshot is no longer in the updated metadata
+        LOG.info("Expired snapshot: {}", snapshot);
+        expiredIds.add(snapshotId);
+      }
+    }
+
+    return new SnapshotExpirationChanges(validIds, expiredIds);
+  }
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static Set<Long> getPickedAncestorIds(TableMetadata currentMetadata, Set<Long> ancestorIds) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentMetadata.snapshot(snapshotId).summary()

Review comment:
       I think this should be `base`, not `current`. Also, if we rename `currentMetadata` to `base`, this would fit on one line like before.

##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -152,168 +165,23 @@ private void cleanExpiredSnapshots() {
     // 2. Delete any data files that were deleted by those snapshots and are not in the table
     // 3. Delete any manifests that are no longer used by current snapshots
     // 4. Delete the manifest lists
+    TableMetadata currentMetadata = ops.refresh();
+    SnapshotExpirationChanges snapshotChanges = ExpireSnapshotUtil.getExpiredSnapshots(currentMetadata, base);
 
-    TableMetadata current = ops.refresh();
-
-    Set<Long> validIds = Sets.newHashSet();
-    for (Snapshot snapshot : current.snapshots()) {
-      validIds.add(snapshot.snapshotId());
-    }
-
-    Set<Long> expiredIds = Sets.newHashSet();
-    for (Snapshot snapshot : base.snapshots()) {
-      long snapshotId = snapshot.snapshotId();
-      if (!validIds.contains(snapshotId)) {
-        // the snapshot was expired
-        LOG.info("Expired snapshot: {}", snapshot);
-        expiredIds.add(snapshotId);
-      }
-    }
-
-    if (expiredIds.isEmpty()) {
+    if (snapshotChanges.expiredSnapshotIds().isEmpty()) {
       // if no snapshots were expired, skip cleanup
       return;
     }
 
-    LOG.info("Committed snapshot changes; cleaning up expired manifests and data files.");
+    LOG.info("Cleaning up expired manifests and data files locally.");
 
-    cleanExpiredFiles(current.snapshots(), validIds, expiredIds);
-  }
-
-  @SuppressWarnings("checkstyle:CyclomaticComplexity")
-  private void cleanExpiredFiles(List<Snapshot> snapshots, Set<Long> validIds, Set<Long> expiredIds) {
     // Reads and deletes are done using Tasks.foreach(...).suppressFailureWhenFinished to complete

Review comment:
       Is this comment more appropriate in a different place now?

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private SparkSession session;
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession session, Table table) {
+    this.session = session;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().deleteExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Execute is a synonym for commit in this implementation. Calling either commit or execute will
+   * launch the Spark equivalent of RemoveSnapshots.
+   *
+   * @return nothing
+   */
+  @Override
+  public ExpireSnapshotResults execute() {
+    localExpireSnapshots.commit();
+
+    TableMetadata currentMetadata = ops.refresh();
+
+    //Locally determine which snapshots have been expired
+    SnapshotExpirationChanges expiredSnapshotChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(currentMetadata, base);
+
+    //Locally determine which manifests will need to be scanned, reverted, deleted
+    ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            expiredSnapshotChanges.validSnapshotIds(), expiredSnapshotChanges.expiredSnapshotIds(),
+            currentMetadata, base, ops.io());
+
+    FileIO io = SparkUtil.serializableFileIO(table);
+
+    //Going the RDD Route because our reader functions all work with full Manifest Files
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(session.sparkContext());
+
+    JavaRDD<ManifestFile> manifestsToScan =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToScan()));
+
+    JavaRDD<ManifestFile> manifestsToRevert =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToRevert()));
+
+    FileIO serializableIO = SparkUtil.serializableFileIO(table);
+
+    Broadcast<Map<Integer, PartitionSpec>> broadcastedSpecLookup =
+        javaSparkContext.broadcast(ops.current().specsById());
+
+    Broadcast<Set<Long>> broadcastValidIDs =
+        javaSparkContext.broadcast(expiredSnapshotChanges.validSnapshotIds());
+
+    JavaRDD<String> filesToDeleteFromScan = manifestsToScan.mapPartitions(manifests -> {
+      Map<Integer, PartitionSpec> specLookup = broadcastedSpecLookup.getValue();
+      Set<Long> validIds = broadcastValidIDs.getValue();
+      Set<String> filesToDelete = new HashSet<>();
+      Tasks.foreach(ImmutableList.copyOf(manifests))
+          .retry(3).suppressFailureWhenFinished()
+          .executeWith(ThreadPools.getWorkerPool())
+          .onFailure((item, exc) -> LOG
+              .warn("Failed to get deleted files: this may cause orphaned data files", exc))
+          .run(manifest -> {
+            // the manifest has deletes, scan it to find files to delete
+            try (ManifestReader<?> reader = ManifestFiles
+                .open(manifest, serializableIO, specLookup)) {
+              for (ManifestEntry<?> entry : reader.entries()) {
+                // if the snapshot ID of the DELETE entry is no longer valid, the data can be deleted
+                if (entry.status() == ManifestEntry.Status.DELETED &&
+                    !validIds.contains(entry.snapshotId())) {
+                  // use toString to ensure the path will not change (Utf8 is reused)
+                  filesToDelete.add(entry.file().path().toString());
+                }
+              }
+            } catch (IOException e) {
+              throw new UncheckedIOException(
+                  String.format("Failed to read manifest file: %s", manifest), e);
+            }
+          });
+      return filesToDelete.iterator();
+    });
+
+    JavaRDD<String> filesToDeleteFromRevert = manifestsToRevert.mapPartitions(manifests -> {
+      Map<Integer, PartitionSpec> specLookup = broadcastedSpecLookup.getValue();
+      Set<String> filesToDelete = new HashSet<>();
+      Tasks.foreach(ImmutableList.copyOf(manifests))
+          .retry(3).suppressFailureWhenFinished()
+          .executeWith(ThreadPools.getWorkerPool())

Review comment:
       same here

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private SparkSession session;
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession session, Table table) {
+    this.session = session;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().deleteExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Execute is a synonym for commit in this implementation. Calling either commit or execute will
+   * launch the Spark equivalent of RemoveSnapshots.
+   *
+   * @return nothing
+   */
+  @Override
+  public ExpireSnapshotResults execute() {
+    localExpireSnapshots.commit();
+
+    TableMetadata currentMetadata = ops.refresh();
+
+    //Locally determine which snapshots have been expired
+    SnapshotExpirationChanges expiredSnapshotChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(currentMetadata, base);
+
+    //Locally determine which manifests will need to be scanned, reverted, deleted
+    ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            expiredSnapshotChanges.validSnapshotIds(), expiredSnapshotChanges.expiredSnapshotIds(),
+            currentMetadata, base, ops.io());
+
+    FileIO io = SparkUtil.serializableFileIO(table);
+
+    //Going the RDD Route because our reader functions all work with full Manifest Files
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(session.sparkContext());
+
+    JavaRDD<ManifestFile> manifestsToScan =

Review comment:
       Will we always rely on `spark.default.parallelism` here?

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private SparkSession session;
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession session, Table table) {
+    this.session = session;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().deleteExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Execute is a synonym for commit in this implementation. Calling either commit or execute will
+   * launch the Spark equivalent of RemoveSnapshots.
+   *
+   * @return nothing
+   */
+  @Override
+  public ExpireSnapshotResults execute() {
+    localExpireSnapshots.commit();
+
+    TableMetadata currentMetadata = ops.refresh();
+
+    //Locally determine which snapshots have been expired
+    SnapshotExpirationChanges expiredSnapshotChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(currentMetadata, base);
+
+    //Locally determine which manifests will need to be scanned, reverted, deleted
+    ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            expiredSnapshotChanges.validSnapshotIds(), expiredSnapshotChanges.expiredSnapshotIds(),
+            currentMetadata, base, ops.io());
+
+    FileIO io = SparkUtil.serializableFileIO(table);
+
+    //Going the RDD Route because our reader functions all work with full Manifest Files
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(session.sparkContext());
+
+    JavaRDD<ManifestFile> manifestsToScan =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToScan()));
+
+    JavaRDD<ManifestFile> manifestsToRevert =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToRevert()));
+
+    FileIO serializableIO = SparkUtil.serializableFileIO(table);
+
+    Broadcast<Map<Integer, PartitionSpec>> broadcastedSpecLookup =
+        javaSparkContext.broadcast(ops.current().specsById());
+
+    Broadcast<Set<Long>> broadcastValidIDs =
+        javaSparkContext.broadcast(expiredSnapshotChanges.validSnapshotIds());
+
+    JavaRDD<String> filesToDeleteFromScan = manifestsToScan.mapPartitions(manifests -> {
+      Map<Integer, PartitionSpec> specLookup = broadcastedSpecLookup.getValue();
+      Set<Long> validIds = broadcastValidIDs.getValue();
+      Set<String> filesToDelete = new HashSet<>();
+      Tasks.foreach(ImmutableList.copyOf(manifests))
+          .retry(3).suppressFailureWhenFinished()
+          .executeWith(ThreadPools.getWorkerPool())

Review comment:
       I am not sure we need to use `ThreadPools` on executors. First, `ThreadPools` in Iceberg was meant to parallelize some work on the driver. Second, we will have only one core executing a task in most cases. Since we are already parallelizing the work using Spark, I don't think we need the thread pool here.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private SparkSession session;
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession session, Table table) {
+    this.session = session;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().deleteExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Execute is a synonym for commit in this implementation. Calling either commit or execute will
+   * launch the Spark equivalent of RemoveSnapshots.
+   *
+   * @return nothing
+   */
+  @Override
+  public ExpireSnapshotResults execute() {
+    localExpireSnapshots.commit();
+
+    TableMetadata currentMetadata = ops.refresh();
+
+    //Locally determine which snapshots have been expired
+    SnapshotExpirationChanges expiredSnapshotChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(currentMetadata, base);
+
+    //Locally determine which manifests will need to be scanned, reverted, deleted
+    ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            expiredSnapshotChanges.validSnapshotIds(), expiredSnapshotChanges.expiredSnapshotIds(),
+            currentMetadata, base, ops.io());
+
+    FileIO io = SparkUtil.serializableFileIO(table);

Review comment:
       This seems not used?




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private SparkSession session;
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession session, Table table) {
+    this.session = session;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().deleteExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Execute is a synonym for commit in this implementation. Calling either commit or execute will
+   * launch the Spark equivalent of RemoveSnapshots.
+   *
+   * @return nothing
+   */
+  @Override
+  public ExpireSnapshotResults execute() {
+    localExpireSnapshots.commit();
+
+    TableMetadata currentMetadata = ops.refresh();
+
+    //Locally determine which snapshots have been expired
+    SnapshotExpirationChanges expiredSnapshotChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(currentMetadata, base);
+
+    //Locally determine which manifests will need to be scanned, reverted, deleted
+    ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            expiredSnapshotChanges.validSnapshotIds(), expiredSnapshotChanges.expiredSnapshotIds(),
+            currentMetadata, base, ops.io());
+
+    FileIO io = SparkUtil.serializableFileIO(table);
+
+    //Going the RDD Route because our reader functions all work with full Manifest Files
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(session.sparkContext());
+
+    JavaRDD<ManifestFile> manifestsToScan =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToScan()));
+
+    JavaRDD<ManifestFile> manifestsToRevert =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToRevert()));
+
+    FileIO serializableIO = SparkUtil.serializableFileIO(table);
+
+    Broadcast<Map<Integer, PartitionSpec>> broadcastedSpecLookup =
+        javaSparkContext.broadcast(ops.current().specsById());
+
+    Broadcast<Set<Long>> broadcastValidIDs =
+        javaSparkContext.broadcast(expiredSnapshotChanges.validSnapshotIds());
+
+    JavaRDD<String> filesToDeleteFromScan = manifestsToScan.mapPartitions(manifests -> {
+      Map<Integer, PartitionSpec> specLookup = broadcastedSpecLookup.getValue();
+      Set<Long> validIds = broadcastValidIDs.getValue();
+      Set<String> filesToDelete = new HashSet<>();
+      Tasks.foreach(ImmutableList.copyOf(manifests))
+          .retry(3).suppressFailureWhenFinished()
+          .executeWith(ThreadPools.getWorkerPool())
+          .onFailure((item, exc) -> LOG
+              .warn("Failed to get deleted files: this may cause orphaned data files", exc))
+          .run(manifest -> {
+            // the manifest has deletes, scan it to find files to delete
+            try (ManifestReader<?> reader = ManifestFiles
+                .open(manifest, serializableIO, specLookup)) {
+              for (ManifestEntry<?> entry : reader.entries()) {
+                // if the snapshot ID of the DELETE entry is no longer valid, the data can be deleted
+                if (entry.status() == ManifestEntry.Status.DELETED &&
+                    !validIds.contains(entry.snapshotId())) {
+                  // use toString to ensure the path will not change (Utf8 is reused)
+                  filesToDelete.add(entry.file().path().toString());
+                }
+              }
+            } catch (IOException e) {
+              throw new UncheckedIOException(
+                  String.format("Failed to read manifest file: %s", manifest), e);
+            }
+          });
+      return filesToDelete.iterator();
+    });
+
+    JavaRDD<String> filesToDeleteFromRevert = manifestsToRevert.mapPartitions(manifests -> {
+      Map<Integer, PartitionSpec> specLookup = broadcastedSpecLookup.getValue();
+      Set<String> filesToDelete = new HashSet<>();
+      Tasks.foreach(ImmutableList.copyOf(manifests))
+          .retry(3).suppressFailureWhenFinished()
+          .executeWith(ThreadPools.getWorkerPool())
+          .onFailure((item, exc) -> LOG.warn("Failed to get deleted files: this may cause orphaned data files", exc))
+          .run(manifest -> {
+            // the manifest has deletes, scan it to find files to delete
+            try (ManifestReader<?> reader = ManifestFiles
+                .open(manifest, serializableIO, specLookup)) {
+              for (ManifestEntry<?> entry : reader.entries()) {
+                // delete any ADDED file from manifests that were reverted
+                if (entry.status() == ManifestEntry.Status.ADDED) {
+                  // use toString to ensure the path will not change (Utf8 is reused)
+                  filesToDelete.add(entry.file().path().toString());
+                }
+              }
+            } catch (IOException e) {
+              throw new UncheckedIOException(
+                  String.format("Failed to read manifest file: %s", manifest), e);
+            }
+          });
+      return filesToDelete.iterator();
+    });
+
+    Set<String> dataFilesToDelete = new HashSet<>(
+        filesToDeleteFromRevert.union(filesToDeleteFromScan).collect());
+
+    LOG.warn("Deleting {} data files", dataFilesToDelete.size());
+
+    return new ExpireSnapshotResults(
+        deleteManifestFiles(manifestExpirationChanges.manifestsToDelete()),
+        deleteManifestLists(manifestExpirationChanges.manifestListsToDelete()),
+        deleteDataFiles(dataFilesToDelete));
+  }
+
+  private Long deleteManifestFiles(Set<String> manifestsToDelete) {
+    LOG.warn("Manifests to delete: {}", Joiner.on(", ").join(manifestsToDelete));
+    AtomicReference<Long> deleteCount = new AtomicReference<>(0L);
+
+    Tasks.foreach(manifestsToDelete)
+        .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished()
+        .onFailure((manifest, exc) -> LOG.warn("Delete failed for manifest: {}", manifest, exc))
+        .run(file -> {
+          deleteFunc.accept(file);
+          deleteCount.updateAndGet(v -> v + 1);
+        });
+    return deleteCount.get();
+  }
+
+  private Long deleteManifestLists(Set<String> manifestListsToDelete) {
+    LOG.warn("Manifests Lists to delete: {}", Joiner.on(", ").join(manifestListsToDelete));
+    AtomicReference<Long> deleteCount = new AtomicReference<>(0L);
+
+    Tasks.foreach(manifestListsToDelete)
+        .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished()
+        .onFailure((list, exc) -> LOG.warn("Delete failed for manifest list: {}", list, exc))
+        .run(file -> {
+          deleteFunc.accept(file);
+          deleteCount.updateAndGet(v -> v + 1);
+        });
+    return deleteCount.get();
+  }
+
+  private Long deleteDataFiles(Set<String> dataFilesToDelete) {

Review comment:
       Yep!




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

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] aokolnychyi commented on a change in pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {

Review comment:
       Do we iterate over an empty set?




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private SparkSession session;
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession session, Table table) {
+    this.session = session;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().deleteExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Execute is a synonym for commit in this implementation. Calling either commit or execute will
+   * launch the Spark equivalent of RemoveSnapshots.
+   *
+   * @return nothing
+   */
+  @Override
+  public ExpireSnapshotResults execute() {
+    localExpireSnapshots.commit();
+
+    TableMetadata currentMetadata = ops.refresh();
+
+    //Locally determine which snapshots have been expired
+    SnapshotExpirationChanges expiredSnapshotChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(currentMetadata, base);
+
+    //Locally determine which manifests will need to be scanned, reverted, deleted
+    ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            expiredSnapshotChanges.validSnapshotIds(), expiredSnapshotChanges.expiredSnapshotIds(),
+            currentMetadata, base, ops.io());
+
+    FileIO io = SparkUtil.serializableFileIO(table);
+
+    //Going the RDD Route because our reader functions all work with full Manifest Files
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(session.sparkContext());
+
+    JavaRDD<ManifestFile> manifestsToScan =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToScan()));
+
+    JavaRDD<ManifestFile> manifestsToRevert =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToRevert()));
+
+    FileIO serializableIO = SparkUtil.serializableFileIO(table);
+
+    Broadcast<Map<Integer, PartitionSpec>> broadcastedSpecLookup =
+        javaSparkContext.broadcast(ops.current().specsById());
+
+    Broadcast<Set<Long>> broadcastValidIDs =
+        javaSparkContext.broadcast(expiredSnapshotChanges.validSnapshotIds());
+
+    JavaRDD<String> filesToDeleteFromScan = manifestsToScan.mapPartitions(manifests -> {
+      Map<Integer, PartitionSpec> specLookup = broadcastedSpecLookup.getValue();
+      Set<Long> validIds = broadcastValidIDs.getValue();
+      Set<String> filesToDelete = new HashSet<>();
+      Tasks.foreach(ImmutableList.copyOf(manifests))
+          .retry(3).suppressFailureWhenFinished()
+          .executeWith(ThreadPools.getWorkerPool())

Review comment:
       I like to parallelize within executors as well just because it is more flexible that doing so with number of tasks but I can remove it if you want. In general I don't like doing any blocking IO if I can help 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.

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {

Review comment:
       ... that's ... not right :/  This probably indicates we also need another test since this doesn't break any of the current tests ...




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private SparkSession session;
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession session, Table table) {
+    this.session = session;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().deleteExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Execute is a synonym for commit in this implementation. Calling either commit or execute will
+   * launch the Spark equivalent of RemoveSnapshots.
+   *
+   * @return nothing
+   */
+  @Override
+  public ExpireSnapshotResults execute() {
+    localExpireSnapshots.commit();
+
+    TableMetadata currentMetadata = ops.refresh();
+
+    //Locally determine which snapshots have been expired
+    SnapshotExpirationChanges expiredSnapshotChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(currentMetadata, base);
+
+    //Locally determine which manifests will need to be scanned, reverted, deleted
+    ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            expiredSnapshotChanges.validSnapshotIds(), expiredSnapshotChanges.expiredSnapshotIds(),
+            currentMetadata, base, ops.io());
+
+    FileIO io = SparkUtil.serializableFileIO(table);
+
+    //Going the RDD Route because our reader functions all work with full Manifest Files
+    JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(session.sparkContext());
+
+    JavaRDD<ManifestFile> manifestsToScan =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToScan()));
+
+    JavaRDD<ManifestFile> manifestsToRevert =
+        javaSparkContext
+            .parallelize(new LinkedList<>(manifestExpirationChanges.manifestsToRevert()));
+
+    FileIO serializableIO = SparkUtil.serializableFileIO(table);
+
+    Broadcast<Map<Integer, PartitionSpec>> broadcastedSpecLookup =

Review comment:
       I can switch to just letting Java figure out the closure if it's not huge?




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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


   cc @aokolnychyi 


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

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] aokolnychyi commented on a change in pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
##########
@@ -152,168 +164,33 @@ private void cleanExpiredSnapshots() {
     // 2. Delete any data files that were deleted by those snapshots and are not in the table
     // 3. Delete any manifests that are no longer used by current snapshots
     // 4. Delete the manifest lists
+    SnapshotExpirationChanges snapshotExpirationChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(ops, base);
 
-    TableMetadata current = ops.refresh();
-
-    Set<Long> validIds = Sets.newHashSet();
-    for (Snapshot snapshot : current.snapshots()) {
-      validIds.add(snapshot.snapshotId());
-    }
-
-    Set<Long> expiredIds = Sets.newHashSet();
-    for (Snapshot snapshot : base.snapshots()) {
-      long snapshotId = snapshot.snapshotId();
-      if (!validIds.contains(snapshotId)) {
-        // the snapshot was expired
-        LOG.info("Expired snapshot: {}", snapshot);
-        expiredIds.add(snapshotId);
-      }
-    }
-
-    if (expiredIds.isEmpty()) {
+    if (snapshotExpirationChanges.getExpiredSnapshotIds().isEmpty()) {
       // if no snapshots were expired, skip cleanup
       return;
     }
 
-    LOG.info("Committed snapshot changes; cleaning up expired manifests and data files.");
-
-    cleanExpiredFiles(current.snapshots(), validIds, expiredIds);
+    LOG.info("Cleaning up expired manifests and data files locally.");
+    cleanExpiredFiles(
+        snapshotExpirationChanges.getCurrentSnapshots(),
+        snapshotExpirationChanges.getValidSnapshotIds(),
+        snapshotExpirationChanges.getExpiredSnapshotIds());
   }
 
-  @SuppressWarnings("checkstyle:CyclomaticComplexity")
-  private void cleanExpiredFiles(List<Snapshot> snapshots, Set<Long> validIds, Set<Long> expiredIds) {
+  private void cleanExpiredFiles(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds) {
     // Reads and deletes are done using Tasks.foreach(...).suppressFailureWhenFinished to complete
     // as much of the delete work as possible and avoid orphaned data or manifest files.
-
-    // this is the set of ancestors of the current table state. when removing snapshots, this must
-    // only remove files that were deleted in an ancestor of the current table state to avoid
-    // physically deleting files that were logically deleted in a commit that was rolled back.
-    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil.ancestorIds(base.currentSnapshot(), base::snapshot));
-
-    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
-    for (long snapshotId : ancestorIds) {
-      String sourceSnapshotId = base.snapshot(snapshotId).summary().get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
-      if (sourceSnapshotId != null) {
-        // protect any snapshot that was cherry-picked into the current table state
-        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
-      }
-    }
-
-    // find manifests to clean up that are still referenced by a valid snapshot, but written by an expired snapshot
-    Set<String> validManifests = Sets.newHashSet();
-    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
-    Tasks.foreach(snapshots).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                for (ManifestFile manifest : manifests) {
-                  validManifests.add(manifest.path());
-
-                  long snapshotId = manifest.snapshotId();
-                  // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
-                  boolean fromValidSnapshots = validIds.contains(snapshotId);
-                  // whether the snapshot that created the manifest was an ancestor of the table state
-                  boolean isFromAncestor = ancestorIds.contains(snapshotId);
-                  // whether the changes in this snapshot have been picked into the current table state
-                  boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
-                  // if the snapshot that wrote this manifest is no longer valid (has expired),
-                  // then delete its deleted files. note that this is only for expired snapshots that are in the
-                  // current table state
-                  if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
-                    manifestsToScan.add(manifest.copy());
-                  }
-                }
-
-              } catch (IOException e) {
-                throw new RuntimeIOException(e,
-                    "Failed to close manifest list: %s", snapshot.manifestListLocation());
-              }
-            });
-
-    // find manifests to clean up that were only referenced by snapshots that have expired
-    Set<String> manifestListsToDelete = Sets.newHashSet();
-    Set<String> manifestsToDelete = Sets.newHashSet();
-    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
-    Tasks.foreach(base.snapshots()).retry(3).suppressFailureWhenFinished()
-        .onFailure((snapshot, exc) ->
-            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
-                snapshot.manifestListLocation(), exc))
-        .run(
-            snapshot -> {
-              long snapshotId = snapshot.snapshotId();
-              if (!validIds.contains(snapshotId)) {
-                // determine whether the changes in this snapshot are in the current table state
-                if (pickedAncestorSnapshotIds.contains(snapshotId)) {
-                  // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
-                  // its changes will expire when the picked snapshot expires.
-                  // A -- C -- D (source=B)
-                  //  `- B <-- this commit
-                  return;
-                }
-
-                long sourceSnapshotId = PropertyUtil.propertyAsLong(
-                    snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
-                if (ancestorIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- B -- C
-                  //  `- D (source=B) <-- this commit
-                  return;
-                }
-
-                if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
-                  // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
-                  // changes because it would revert data file additions that are in the current table.
-                  // A -- C -- E (source=B)
-                  //  `- B `- D (source=B) <-- this commit
-                  return;
-                }
-
-                // find any manifests that are no longer needed
-                try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot)) {
-                  for (ManifestFile manifest : manifests) {
-                    if (!validManifests.contains(manifest.path())) {
-                      manifestsToDelete.add(manifest.path());
-
-                      boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
-                      boolean isFromExpiringSnapshot = expiredIds.contains(manifest.snapshotId());
-
-                      if (isFromAncestor && manifest.hasDeletedFiles()) {
-                        // Only delete data files that were deleted in by an expired snapshot if that
-                        // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
-                        // deleted files and was rolled back will delete files that could be in the current
-                        // table state.
-                        manifestsToScan.add(manifest.copy());
-                      }
-
-                      if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
-                        // Because the manifest was written by a snapshot that is not an ancestor of the
-                        // current table state, the files added in this manifest can be removed. The extra
-                        // check whether the manifest was written by a known snapshot that was expired in
-                        // this commit ensures that the full ancestor list between when the snapshot was
-                        // written and this expiration is known and there is no missing history. If history
-                        // were missing, then the snapshot could be an ancestor of the table state but the
-                        // ancestor ID set would not contain it and this would be unsafe.
-                        manifestsToRevert.add(manifest.copy());
-                      }
-                    }
-                  }
-                } catch (IOException e) {
-                  throw new RuntimeIOException(e,
-                      "Failed to close manifest list: %s", snapshot.manifestListLocation());
-                }
-
-                // add the manifest list to the delete set, if present
-                if (snapshot.manifestListLocation() != null) {
-                  manifestListsToDelete.add(snapshot.manifestListLocation());
-                }
-              }
-            });
-    deleteDataFiles(manifestsToScan, manifestsToRevert, validIds);
-    deleteMetadataFiles(manifestsToDelete, manifestListsToDelete);
+    ExpireSnapshotUtil.ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(

Review comment:
       After refactoring, `cleanExpiredFiles` became really small. I'd consider merging it with `cleanExpiredSnapshots`. Then the utility method can accept `current`, `base`, `validIds`, `expiredIds`, `io` to determine manifests to scan.




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestEntry;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil;
+import org.apache.iceberg.util.ExpireSnapshotUtil.ManifestExpirationChanges;
+import org.apache.iceberg.util.ExpireSnapshotUtil.SnapshotExpirationChanges;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotResults> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private SparkSession session;
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession session, Table table) {
+    this.session = session;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().deleteExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Execute is a synonym for commit in this implementation. Calling either commit or execute will
+   * launch the Spark equivalent of RemoveSnapshots.
+   *
+   * @return nothing
+   */
+  @Override
+  public ExpireSnapshotResults execute() {
+    localExpireSnapshots.commit();
+
+    TableMetadata currentMetadata = ops.refresh();
+
+    //Locally determine which snapshots have been expired
+    SnapshotExpirationChanges expiredSnapshotChanges =
+        ExpireSnapshotUtil.getExpiredSnapshots(currentMetadata, base);
+
+    //Locally determine which manifests will need to be scanned, reverted, deleted
+    ManifestExpirationChanges manifestExpirationChanges =
+        ExpireSnapshotUtil.determineManifestChangesFromSnapshotExpiration(
+            expiredSnapshotChanges.validSnapshotIds(), expiredSnapshotChanges.expiredSnapshotIds(),
+            currentMetadata, base, ops.io());
+
+    FileIO io = SparkUtil.serializableFileIO(table);

Review comment:
       ah yeah, leftover will remove




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

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] aokolnychyi commented on a change in pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,

Review comment:
       Please disregard my previous comment as `currentTableMetadata` should be `base`. We could accept `current`, `base`, `validIds`, `expiredIds`, `io` here.




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

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 #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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



##########
File path: core/src/main/java/org/apache/iceberg/util/ExpireSnapshotUtil.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.GenericManifestFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExpireSnapshotUtil {
+
+  //Utility Class No Instantiation Allowed
+  private ExpireSnapshotUtil() {}
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotUtil.class);
+
+  private static AncestorIds getAncestorIds(TableMetadata currentTableMetadata) {
+    // this is the set of ancestors of the current table state. when removing snapshots, this must
+    // only remove files that were deleted in an ancestor of the current table state to avoid
+    // physically deleting files that were logically deleted in a commit that was rolled back.
+    Set<Long> ancestorIds = Sets.newHashSet(SnapshotUtil
+        .ancestorIds(currentTableMetadata.currentSnapshot(), currentTableMetadata::snapshot));
+
+    Set<Long> pickedAncestorSnapshotIds = Sets.newHashSet();
+    for (long snapshotId : ancestorIds) {
+      String sourceSnapshotId = currentTableMetadata.snapshot(snapshotId).summary()
+          .get(SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP);
+      if (sourceSnapshotId != null) {
+        // protect any snapshot that was cherry-picked into the current table state
+        pickedAncestorSnapshotIds.add(Long.parseLong(sourceSnapshotId));
+      }
+    }
+
+    return new AncestorIds(ancestorIds, pickedAncestorSnapshotIds);
+  }
+
+  /**
+   * Given a list of currently valid snapshots, extract all the manifests from those snapshots if
+   * there is an error while reading manifest lists an incomplete list of manifests will be
+   * produced.
+   *
+   * @param currentTableSnapshots a list of currently valid non-expired snapshots
+   * @return all of the manifests of those snapshots
+   */
+  private static Set<ManifestFile> getValidManifests(
+      List<Snapshot> currentTableSnapshots, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = Sets.newHashSet();
+    Tasks.foreach(currentTableSnapshots).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}", snapshot.snapshotId(),
+                snapshot.manifestListLocation(), exc))
+        .run(
+            snapshot -> {
+              try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+                for (ManifestFile manifest : manifests) {
+                  validManifests.add(manifest);
+                }
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Failed to close manifest list: %s",
+                        snapshot.manifestListLocation()),
+                    e);
+              }
+            });
+    return validManifests;
+  }
+
+  /**
+   * Find manifests to clean up that are still referenced by a valid snapshot, but written by an
+   * expired snapshot.
+   *
+   * @param validSnapshotIds     A list of the snapshots which are not expired
+   * @param currentTableMetadata A reference to the table containing the snapshots
+   * @return MetadataFiles which must be scanned to look for files to delete
+   */
+  private static Set<ManifestFile> validManifestsInExpiredSnapshots(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    manifestsToScan.forEach(manifest -> {
+      long snapshotId = manifest.snapshotId();
+      // whether the manifest was created by a valid snapshot (true) or an expired snapshot (false)
+      boolean fromValidSnapshots = validSnapshotIds.contains(snapshotId);
+      // whether the snapshot that created the manifest was an ancestor of the table state
+      boolean isFromAncestor = ancestorIds.contains(snapshotId);
+      // whether the changes in this snapshot have been picked into the current table state
+      boolean isPicked = pickedAncestorSnapshotIds.contains(snapshotId);
+      // if the snapshot that wrote this manifest is no longer valid (has expired),
+      // then delete its deleted files. note that this is only for expired snapshots that are in the
+      // current table state
+      if (!fromValidSnapshots && (isFromAncestor || isPicked) && manifest.hasDeletedFiles()) {
+        manifestsToScan.add(manifest.copy());
+      }
+    });
+    return manifestsToScan;
+  }
+
+  /**
+   * Removes snapshots whose changes impact the current table state leaving only those which may
+   * have files that could potentially need to be deleted.
+   *
+   * @param currentTableMetadata TableMetadata for the table we are expiring from
+   * @param validSnapshotIds     Snapshots which are not expired
+   * @return A list of those snapshots which may have files that need to be deleted
+   */
+  private static List<Snapshot> filterOutSnapshotsInTableState(
+      Set<Long> validSnapshotIds, TableMetadata currentTableMetadata) {
+
+    AncestorIds ids = getAncestorIds(currentTableMetadata);
+    Set<Long> ancestorIds = ids.getAncestorIds();
+    Set<Long> pickedAncestorSnapshotIds = ids.getPickedAncestorIds();
+
+    List<Snapshot> currentSnapshots = currentTableMetadata.snapshots();
+    return currentSnapshots.stream().filter(snapshot -> {
+      long snapshotId = snapshot.snapshotId();
+      if (!validSnapshotIds.contains(snapshotId)) {
+        // determine whether the changes in this snapshot are in the current table state
+        if (pickedAncestorSnapshotIds.contains(snapshotId)) {
+          // this snapshot was cherry-picked into the current table state, so skip cleaning it up.
+          // its changes will expire when the picked snapshot expires.
+          // A -- C -- D (source=B)
+          //  `- B <-- this commit
+          return false;
+        }
+        long sourceSnapshotId = PropertyUtil.propertyAsLong(
+            snapshot.summary(), SnapshotSummary.SOURCE_SNAPSHOT_ID_PROP, -1);
+        if (ancestorIds.contains(sourceSnapshotId)) {
+          // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+          // changes because it would revert data file additions that are in the current table.
+          // A -- B -- C
+              //  `- D (source=B) <-- this commit
+              return false;
+            }
+
+            if (pickedAncestorSnapshotIds.contains(sourceSnapshotId)) {
+              // this commit was cherry-picked from a commit that is in the current table state. do not clean up its
+              // changes because it would revert data file additions that are in the current table.
+              // A -- C -- E (source=B)
+              //  `- B `- D (source=B) <-- this commit
+              return false;
+            }
+            return true;
+          }
+          return false;
+        }
+    ).collect(Collectors.toList());
+  }
+
+  private static ManifestExpirationChanges findExpiredManifestsInUnusedSnapshots(
+      List<Snapshot> snapshotsNotInTableState, Set<ManifestFile> validManifests,
+      TableMetadata oldMetadata, Set<Long> expiredSnapshotIds, TableOperations ops) {
+
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+    Set<String> manifestsToDelete = Sets.newHashSet();
+    Set<ManifestFile> manifestsToRevert = Sets.newHashSet();
+    Set<ManifestFile> manifestsToScan = Sets.newHashSet();
+    Set<Long> ancestorIds = getAncestorIds(oldMetadata).getAncestorIds();
+
+    Tasks.foreach(snapshotsNotInTableState).retry(3).suppressFailureWhenFinished()
+        .onFailure((snapshot, exc) ->
+            LOG.warn("Failed on snapshot {} while reading manifest list: {}",
+                snapshot.snapshotId(), snapshot.manifestListLocation(), exc))
+        .run(snapshot -> {
+          // find any manifests that are no longer needed
+          try (CloseableIterable<ManifestFile> manifests = readManifestFiles(snapshot, ops)) {
+            for (ManifestFile manifest : manifests) {
+              if (!validManifests.contains(manifest)) {
+                manifestsToDelete.add(manifest.path());
+
+                boolean isFromAncestor = ancestorIds.contains(manifest.snapshotId());
+                boolean isFromExpiringSnapshot = expiredSnapshotIds.contains(manifest.snapshotId());
+
+                if (isFromAncestor && manifest.hasDeletedFiles()) {
+                  // Only delete data files that were deleted in by an expired snapshot if that
+                  // snapshot is an ancestor of the current table state. Otherwise, a snapshot that
+                  // deleted files and was rolled back will delete files that could be in the current
+                  // table state.
+                  manifestsToScan.add(manifest.copy());
+                }
+
+                if (!isFromAncestor && isFromExpiringSnapshot && manifest.hasAddedFiles()) {
+                  // Because the manifest was written by a snapshot that is not an ancestor of the
+                  // current table state, the files added in this manifest can be removed. The extra
+                  // check whether the manifest was written by a known snapshot that was expired in
+                  // this commit ensures that the full ancestor list between when the snapshot was
+                  // written and this expiration is known and there is no missing history. If history
+                  // were missing, then the snapshot could be an ancestor of the table state but the
+                  // ancestor ID set would not contain it and this would be unsafe.
+                  manifestsToRevert.add(manifest.copy());
+                }
+              }
+            }
+            // add the manifest list to the delete set, if present
+            if (snapshot.manifestListLocation() != null) {
+              manifestListsToDelete.add(snapshot.manifestListLocation());
+            }
+          } catch (IOException e) {
+            throw new UncheckedIOException(
+                String.format("Failed to close manifest list: %s", snapshot.manifestListLocation()),
+                e);
+          }
+        });
+    return new ManifestExpirationChanges(manifestsToScan, manifestsToRevert, manifestsToDelete,
+        manifestListsToDelete);
+  }
+
+  private static final Schema MANIFEST_PROJECTION = ManifestFile.schema()
+      .select("manifest_path", "added_snapshot_id", "deleted_data_files_count");
+
+  private static CloseableIterable<ManifestFile> readManifestFiles(
+      Snapshot snapshot, TableOperations ops) {
+
+    if (snapshot.manifestListLocation() != null) {
+      return Avro.read(ops.io().newInputFile(snapshot.manifestListLocation()))
+          .rename("manifest_file", GenericManifestFile.class.getName())
+          .classLoader(GenericManifestFile.class.getClassLoader())
+          .project(MANIFEST_PROJECTION)
+          .reuseContainers(true)
+          .build();
+
+    } else {
+      return CloseableIterable.withNoopClose(snapshot.allManifests());
+    }
+  }
+
+  /**
+   * Determines the manifest files which need to be inspected because they refer to data files which
+   * can be removed after a Snapshot Expiration.
+   *
+   * @param currentTableSnapshots A list of Snapshots Currently used by the Table
+   * @param validIds              The Ids of the Snapshots which have not been expired
+   * @param expiredIds            The Ids of the Snapshots which have been expired
+   * @param currentTableMetadata  The metadata of the table being expired
+   * @param ops                   The Table Operations module for the table in question, required
+   *                              for several IO operations
+   * @return
+   */
+  public static ManifestExpirationChanges determineManifestChangesFromSnapshotExpiration(
+      List<Snapshot> currentTableSnapshots, Set<Long> validIds, Set<Long> expiredIds,
+      TableMetadata currentTableMetadata, TableOperations ops) {
+
+    Set<ManifestFile> validManifests = getValidManifests(currentTableSnapshots, ops);
+    Set<ManifestFile> manifestsToScan = validManifestsInExpiredSnapshots(validIds,

Review comment:
       This may be a good time to rename ManifestsToScan and Revert to, manifestsWithDeletes, manifestsWithAdds?




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

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] aokolnychyi commented on pull request #1211: Use Spark for Scanning of Manifest Files during Snapshot Expiration

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


   Thanks, @RussellSpitzer! Let me take a look at this today.
   
   To give a bit of background, we had a case where people compacted their table that reduced the number of files from 5+ million to under 1 million in a short period of time. That processed generated a huge number of new snapshots as different parts were optimized separately (one leaf partition at a time). As a result, the expiry of snapshots was really slow.
   
   This is an attempt to fix those edge cases.


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

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