rdblue commented on code in PR #5669:
URL: https://github.com/apache/iceberg/pull/5669#discussion_r986046757


##########
core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * File cleanup strategy for snapshot expiration which determines, via an 
in-memory reference set,
+ * metadata and data files that are not reachable given the previous and 
current table states.
+ */
+class ReachableFileCleanup extends FileCleanupStrategy {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ReachableFileCleanup.class);
+
+  private final TableMetadata afterExpiration;
+  private final TableMetadata beforeExpiration;
+
+  ReachableFileCleanup(
+      TableOperations ops,
+      TableMetadata beforeExpiration,
+      ExecutorService deleteExecutorService,
+      ExecutorService planExecutorService,
+      Consumer<String> deleteFunc) {
+    super(ops, deleteExecutorService, planExecutorService, deleteFunc);
+    this.beforeExpiration = beforeExpiration;
+    this.afterExpiration = ops.refresh();
+  }
+
+  @Override
+  public void cleanFiles() {
+    // Identify all of the manifest lists to retain
+    Set<String> manifestListsBeforeExpiration =
+        ReachableFileUtil.manifestListLocations(beforeExpiration);
+
+    // Identify all of the manifest lists that currently exist
+    Set<String> manifestListsAfterExpiration =
+        ReachableFileUtil.manifestListLocations(afterExpiration);
+
+    Set<String> manifestListsToDelete =
+        Sets.difference(manifestListsBeforeExpiration, 
manifestListsAfterExpiration);
+    Map<String, ManifestFile> reachableManifestsAfterExpiration =
+        computeManifestsForManifestLists(manifestListsAfterExpiration);
+
+    // Compute the manifests to delete based on the current manifest lists to 
delete and the current
+    // reachable manifests
+    List<ManifestFile> manifestsToDelete =
+        computeManifestsToDelete(manifestListsToDelete, 
reachableManifestsAfterExpiration);
+
+    List<ManifestFile> currentManifests =
+        Lists.newArrayList(reachableManifestsAfterExpiration.values());
+
+    Set<String> filesToDelete = findFilesToDelete(manifestsToDelete, 
currentManifests);
+    Tasks.foreach(filesToDelete)
+        .executeWith(deleteExecutorService)
+        .retry(3)
+        .stopRetryOn(NotFoundException.class)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.warn("Delete failed for data file: {}", 
file, exc))
+        .run(deleteFunc::accept);
+
+    Set<String> manifestPathsToDelete =
+        
manifestsToDelete.stream().map(ManifestFile::path).collect(Collectors.toSet());
+
+    LOG.warn("Manifests to delete: {}", Joiner.on(", 
").join(manifestsToDelete));
+    LOG.warn("Manifests Lists to delete: {}", Joiner.on(", 
").join(manifestListsToDelete));
+    deleteMetadataFiles(
+        manifestPathsToDelete,
+        (path, exc) -> LOG.warn("Delete failed for {}: {}", "manifest", path, 
exc));
+    deleteMetadataFiles(
+        manifestListsToDelete,
+        (path, exc) -> LOG.warn("Delete failed for {}: {}", "manifest list", 
path, exc));
+  }
+
+  private List<ManifestFile> computeManifestsToDelete(
+      Set<String> manifestListsToDelete, Map<String, ManifestFile> 
reachableFilesAfterExpiration) {
+    Map<String, ManifestFile> manifestCandidatesToDelete =
+        computeManifestsForManifestLists(manifestListsToDelete);
+    List<ManifestFile> manifestFilesToDelete = Lists.newArrayList();
+    for (Map.Entry<String, ManifestFile> candidateManifest :
+        manifestCandidatesToDelete.entrySet()) {
+      if 
(!reachableFilesAfterExpiration.containsKey(candidateManifest.getKey())) {
+        manifestFilesToDelete.add(candidateManifest.getValue());
+      }
+    }
+    return manifestFilesToDelete;
+  }
+
+  private Map<String, ManifestFile> 
computeManifestsForManifestLists(Set<String> manifestLists) {
+    Map<String, ManifestFile> pathToManifestFiles = Maps.newHashMap();
+    for (String manifestList : manifestLists) {
+      CloseableIterable<ManifestFile> manifestFiles = 
readManifestFiles(manifestList);
+      for (ManifestFile manifest : manifestFiles) {
+        pathToManifestFiles.put(manifest.path(), manifest.copy());
+      }
+    }
+    return pathToManifestFiles;
+  }
+
+  // Helper to compute files to delete
+  private Set<String> findFilesToDelete(
+      List<ManifestFile> manifestFilesToDelete, List<ManifestFile> 
currentManifestFiles) {
+    Set<String> filesToDelete = ConcurrentHashMap.newKeySet();
+    Tasks.foreach(manifestFilesToDelete)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .executeWith(planExecutorService)
+        .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, ops.io(), 
ops.current().specsById())) {
+                for (ManifestEntry<?> entry : reader.entries()) {
+                  filesToDelete.add(entry.file().path().toString());
+                }
+              } catch (IOException e) {
+                throw new RuntimeIOException(e, "Failed to read manifest file: 
%s", manifest);
+              }
+            });

Review Comment:
   Please add whitespace where needed.



##########
core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * File cleanup strategy for snapshot expiration which determines, via an 
in-memory reference set,
+ * metadata and data files that are not reachable given the previous and 
current table states.
+ */
+class ReachableFileCleanup extends FileCleanupStrategy {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ReachableFileCleanup.class);
+
+  private final TableMetadata afterExpiration;
+  private final TableMetadata beforeExpiration;
+
+  ReachableFileCleanup(
+      TableOperations ops,
+      TableMetadata beforeExpiration,
+      ExecutorService deleteExecutorService,
+      ExecutorService planExecutorService,
+      Consumer<String> deleteFunc) {
+    super(ops, deleteExecutorService, planExecutorService, deleteFunc);
+    this.beforeExpiration = beforeExpiration;
+    this.afterExpiration = ops.refresh();
+  }
+
+  @Override
+  public void cleanFiles() {
+    // Identify all of the manifest lists to retain
+    Set<String> manifestListsBeforeExpiration =
+        ReachableFileUtil.manifestListLocations(beforeExpiration);
+
+    // Identify all of the manifest lists that currently exist
+    Set<String> manifestListsAfterExpiration =
+        ReachableFileUtil.manifestListLocations(afterExpiration);
+
+    Set<String> manifestListsToDelete =
+        Sets.difference(manifestListsBeforeExpiration, 
manifestListsAfterExpiration);
+    Map<String, ManifestFile> reachableManifestsAfterExpiration =
+        computeManifestsForManifestLists(manifestListsAfterExpiration);
+
+    // Compute the manifests to delete based on the current manifest lists to 
delete and the current
+    // reachable manifests
+    List<ManifestFile> manifestsToDelete =
+        computeManifestsToDelete(manifestListsToDelete, 
reachableManifestsAfterExpiration);
+
+    List<ManifestFile> currentManifests =
+        Lists.newArrayList(reachableManifestsAfterExpiration.values());
+
+    Set<String> filesToDelete = findFilesToDelete(manifestsToDelete, 
currentManifests);

Review Comment:
   Both arguments to this method should be sets so that the same manifest isn't 
read multiple times.



##########
core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * File cleanup strategy for snapshot expiration which determines, via an 
in-memory reference set,
+ * metadata and data files that are not reachable given the previous and 
current table states.
+ */
+class ReachableFileCleanup extends FileCleanupStrategy {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ReachableFileCleanup.class);
+
+  private final TableMetadata afterExpiration;
+  private final TableMetadata beforeExpiration;
+
+  ReachableFileCleanup(
+      TableOperations ops,
+      TableMetadata beforeExpiration,
+      ExecutorService deleteExecutorService,
+      ExecutorService planExecutorService,
+      Consumer<String> deleteFunc) {
+    super(ops, deleteExecutorService, planExecutorService, deleteFunc);
+    this.beforeExpiration = beforeExpiration;
+    this.afterExpiration = ops.refresh();
+  }
+
+  @Override
+  public void cleanFiles() {
+    // Identify all of the manifest lists to retain
+    Set<String> manifestListsBeforeExpiration =
+        ReachableFileUtil.manifestListLocations(beforeExpiration);
+
+    // Identify all of the manifest lists that currently exist
+    Set<String> manifestListsAfterExpiration =
+        ReachableFileUtil.manifestListLocations(afterExpiration);
+
+    Set<String> manifestListsToDelete =
+        Sets.difference(manifestListsBeforeExpiration, 
manifestListsAfterExpiration);
+    Map<String, ManifestFile> reachableManifestsAfterExpiration =
+        computeManifestsForManifestLists(manifestListsAfterExpiration);
+
+    // Compute the manifests to delete based on the current manifest lists to 
delete and the current
+    // reachable manifests
+    List<ManifestFile> manifestsToDelete =
+        computeManifestsToDelete(manifestListsToDelete, 
reachableManifestsAfterExpiration);
+
+    List<ManifestFile> currentManifests =
+        Lists.newArrayList(reachableManifestsAfterExpiration.values());
+
+    Set<String> filesToDelete = findFilesToDelete(manifestsToDelete, 
currentManifests);
+    Tasks.foreach(filesToDelete)
+        .executeWith(deleteExecutorService)
+        .retry(3)
+        .stopRetryOn(NotFoundException.class)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.warn("Delete failed for data file: {}", 
file, exc))
+        .run(deleteFunc::accept);
+
+    Set<String> manifestPathsToDelete =
+        
manifestsToDelete.stream().map(ManifestFile::path).collect(Collectors.toSet());
+
+    LOG.warn("Manifests to delete: {}", Joiner.on(", 
").join(manifestsToDelete));
+    LOG.warn("Manifests Lists to delete: {}", Joiner.on(", 
").join(manifestListsToDelete));
+    deleteMetadataFiles(
+        manifestPathsToDelete,
+        (path, exc) -> LOG.warn("Delete failed for {}: {}", "manifest", path, 
exc));
+    deleteMetadataFiles(
+        manifestListsToDelete,
+        (path, exc) -> LOG.warn("Delete failed for {}: {}", "manifest list", 
path, exc));
+  }
+
+  private List<ManifestFile> computeManifestsToDelete(
+      Set<String> manifestListsToDelete, Map<String, ManifestFile> 
reachableFilesAfterExpiration) {
+    Map<String, ManifestFile> manifestCandidatesToDelete =
+        computeManifestsForManifestLists(manifestListsToDelete);
+    List<ManifestFile> manifestFilesToDelete = Lists.newArrayList();
+    for (Map.Entry<String, ManifestFile> candidateManifest :
+        manifestCandidatesToDelete.entrySet()) {
+      if 
(!reachableFilesAfterExpiration.containsKey(candidateManifest.getKey())) {
+        manifestFilesToDelete.add(candidateManifest.getValue());
+      }
+    }
+    return manifestFilesToDelete;
+  }
+
+  private Map<String, ManifestFile> 
computeManifestsForManifestLists(Set<String> manifestLists) {
+    Map<String, ManifestFile> pathToManifestFiles = Maps.newHashMap();
+    for (String manifestList : manifestLists) {
+      CloseableIterable<ManifestFile> manifestFiles = 
readManifestFiles(manifestList);
+      for (ManifestFile manifest : manifestFiles) {
+        pathToManifestFiles.put(manifest.path(), manifest.copy());
+      }
+    }
+    return pathToManifestFiles;
+  }
+
+  // Helper to compute files to delete
+  private Set<String> findFilesToDelete(
+      List<ManifestFile> manifestFilesToDelete, List<ManifestFile> 
currentManifestFiles) {
+    Set<String> filesToDelete = ConcurrentHashMap.newKeySet();
+    Tasks.foreach(manifestFilesToDelete)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .executeWith(planExecutorService)
+        .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, ops.io(), 
ops.current().specsById())) {
+                for (ManifestEntry<?> entry : reader.entries()) {
+                  filesToDelete.add(entry.file().path().toString());

Review Comment:
   I think you should use a `CharSequenceSet` instead of converting to String.



##########
core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * File cleanup strategy for snapshot expiration which determines, via an 
in-memory reference set,
+ * metadata and data files that are not reachable given the previous and 
current table states.
+ */
+class ReachableFileCleanup extends FileCleanupStrategy {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ReachableFileCleanup.class);
+
+  private final TableMetadata afterExpiration;
+  private final TableMetadata beforeExpiration;
+
+  ReachableFileCleanup(
+      TableOperations ops,
+      TableMetadata beforeExpiration,
+      ExecutorService deleteExecutorService,
+      ExecutorService planExecutorService,
+      Consumer<String> deleteFunc) {
+    super(ops, deleteExecutorService, planExecutorService, deleteFunc);
+    this.beforeExpiration = beforeExpiration;
+    this.afterExpiration = ops.refresh();
+  }
+
+  @Override
+  public void cleanFiles() {
+    // Identify all of the manifest lists to retain
+    Set<String> manifestListsBeforeExpiration =
+        ReachableFileUtil.manifestListLocations(beforeExpiration);
+
+    // Identify all of the manifest lists that currently exist
+    Set<String> manifestListsAfterExpiration =
+        ReachableFileUtil.manifestListLocations(afterExpiration);
+
+    Set<String> manifestListsToDelete =
+        Sets.difference(manifestListsBeforeExpiration, 
manifestListsAfterExpiration);
+    Map<String, ManifestFile> reachableManifestsAfterExpiration =
+        computeManifestsForManifestLists(manifestListsAfterExpiration);
+
+    // Compute the manifests to delete based on the current manifest lists to 
delete and the current
+    // reachable manifests
+    List<ManifestFile> manifestsToDelete =
+        computeManifestsToDelete(manifestListsToDelete, 
reachableManifestsAfterExpiration);
+
+    List<ManifestFile> currentManifests =
+        Lists.newArrayList(reachableManifestsAfterExpiration.values());
+
+    Set<String> filesToDelete = findFilesToDelete(manifestsToDelete, 
currentManifests);
+    Tasks.foreach(filesToDelete)
+        .executeWith(deleteExecutorService)
+        .retry(3)
+        .stopRetryOn(NotFoundException.class)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.warn("Delete failed for data file: {}", 
file, exc))
+        .run(deleteFunc::accept);
+
+    Set<String> manifestPathsToDelete =
+        
manifestsToDelete.stream().map(ManifestFile::path).collect(Collectors.toSet());
+
+    LOG.warn("Manifests to delete: {}", Joiner.on(", 
").join(manifestsToDelete));
+    LOG.warn("Manifests Lists to delete: {}", Joiner.on(", 
").join(manifestListsToDelete));
+    deleteMetadataFiles(
+        manifestPathsToDelete,
+        (path, exc) -> LOG.warn("Delete failed for {}: {}", "manifest", path, 
exc));
+    deleteMetadataFiles(
+        manifestListsToDelete,
+        (path, exc) -> LOG.warn("Delete failed for {}: {}", "manifest list", 
path, exc));
+  }
+
+  private List<ManifestFile> computeManifestsToDelete(
+      Set<String> manifestListsToDelete, Map<String, ManifestFile> 
reachableFilesAfterExpiration) {
+    Map<String, ManifestFile> manifestCandidatesToDelete =
+        computeManifestsForManifestLists(manifestListsToDelete);
+    List<ManifestFile> manifestFilesToDelete = Lists.newArrayList();
+    for (Map.Entry<String, ManifestFile> candidateManifest :
+        manifestCandidatesToDelete.entrySet()) {
+      if 
(!reachableFilesAfterExpiration.containsKey(candidateManifest.getKey())) {
+        manifestFilesToDelete.add(candidateManifest.getValue());
+      }
+    }
+    return manifestFilesToDelete;
+  }
+
+  private Map<String, ManifestFile> 
computeManifestsForManifestLists(Set<String> manifestLists) {
+    Map<String, ManifestFile> pathToManifestFiles = Maps.newHashMap();
+    for (String manifestList : manifestLists) {
+      CloseableIterable<ManifestFile> manifestFiles = 
readManifestFiles(manifestList);
+      for (ManifestFile manifest : manifestFiles) {
+        pathToManifestFiles.put(manifest.path(), manifest.copy());
+      }
+    }
+    return pathToManifestFiles;
+  }
+
+  // Helper to compute files to delete
+  private Set<String> findFilesToDelete(
+      List<ManifestFile> manifestFilesToDelete, List<ManifestFile> 
currentManifestFiles) {
+    Set<String> filesToDelete = ConcurrentHashMap.newKeySet();
+    Tasks.foreach(manifestFilesToDelete)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .executeWith(planExecutorService)
+        .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, ops.io(), 
ops.current().specsById())) {
+                for (ManifestEntry<?> entry : reader.entries()) {
+                  filesToDelete.add(entry.file().path().toString());
+                }
+              } catch (IOException e) {
+                throw new RuntimeIOException(e, "Failed to read manifest file: 
%s", manifest);
+              }
+            });
+    Tasks.foreach(currentManifestFiles)

Review Comment:
   Could you add a couple of sanity checks to this method? If `filesToDelete` 
is empty, then there's no need to scan the current manifests.



##########
core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * File cleanup strategy for snapshot expiration which determines, via an 
in-memory reference set,
+ * metadata and data files that are not reachable given the previous and 
current table states.
+ */
+class ReachableFileCleanup extends FileCleanupStrategy {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ReachableFileCleanup.class);
+
+  private final TableMetadata afterExpiration;
+  private final TableMetadata beforeExpiration;
+
+  ReachableFileCleanup(
+      TableOperations ops,
+      TableMetadata beforeExpiration,
+      ExecutorService deleteExecutorService,
+      ExecutorService planExecutorService,
+      Consumer<String> deleteFunc) {
+    super(ops, deleteExecutorService, planExecutorService, deleteFunc);
+    this.beforeExpiration = beforeExpiration;
+    this.afterExpiration = ops.refresh();
+  }
+
+  @Override
+  public void cleanFiles() {
+    // Identify all of the manifest lists to retain
+    Set<String> manifestListsBeforeExpiration =
+        ReachableFileUtil.manifestListLocations(beforeExpiration);
+
+    // Identify all of the manifest lists that currently exist
+    Set<String> manifestListsAfterExpiration =
+        ReachableFileUtil.manifestListLocations(afterExpiration);
+
+    Set<String> manifestListsToDelete =
+        Sets.difference(manifestListsBeforeExpiration, 
manifestListsAfterExpiration);
+    Map<String, ManifestFile> reachableManifestsAfterExpiration =
+        computeManifestsForManifestLists(manifestListsAfterExpiration);
+
+    // Compute the manifests to delete based on the current manifest lists to 
delete and the current
+    // reachable manifests
+    List<ManifestFile> manifestsToDelete =
+        computeManifestsToDelete(manifestListsToDelete, 
reachableManifestsAfterExpiration);
+
+    List<ManifestFile> currentManifests =
+        Lists.newArrayList(reachableManifestsAfterExpiration.values());
+
+    Set<String> filesToDelete = findFilesToDelete(manifestsToDelete, 
currentManifests);
+    Tasks.foreach(filesToDelete)
+        .executeWith(deleteExecutorService)
+        .retry(3)
+        .stopRetryOn(NotFoundException.class)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.warn("Delete failed for data file: {}", 
file, exc))
+        .run(deleteFunc::accept);
+
+    Set<String> manifestPathsToDelete =
+        
manifestsToDelete.stream().map(ManifestFile::path).collect(Collectors.toSet());
+
+    LOG.warn("Manifests to delete: {}", Joiner.on(", 
").join(manifestsToDelete));
+    LOG.warn("Manifests Lists to delete: {}", Joiner.on(", 
").join(manifestListsToDelete));
+    deleteMetadataFiles(
+        manifestPathsToDelete,
+        (path, exc) -> LOG.warn("Delete failed for {}: {}", "manifest", path, 
exc));
+    deleteMetadataFiles(
+        manifestListsToDelete,
+        (path, exc) -> LOG.warn("Delete failed for {}: {}", "manifest list", 
path, exc));
+  }
+
+  private List<ManifestFile> computeManifestsToDelete(
+      Set<String> manifestListsToDelete, Map<String, ManifestFile> 
reachableFilesAfterExpiration) {
+    Map<String, ManifestFile> manifestCandidatesToDelete =
+        computeManifestsForManifestLists(manifestListsToDelete);
+    List<ManifestFile> manifestFilesToDelete = Lists.newArrayList();
+    for (Map.Entry<String, ManifestFile> candidateManifest :
+        manifestCandidatesToDelete.entrySet()) {
+      if 
(!reachableFilesAfterExpiration.containsKey(candidateManifest.getKey())) {
+        manifestFilesToDelete.add(candidateManifest.getValue());
+      }
+    }
+    return manifestFilesToDelete;
+  }
+
+  private Map<String, ManifestFile> 
computeManifestsForManifestLists(Set<String> manifestLists) {
+    Map<String, ManifestFile> pathToManifestFiles = Maps.newHashMap();
+    for (String manifestList : manifestLists) {
+      CloseableIterable<ManifestFile> manifestFiles = 
readManifestFiles(manifestList);
+      for (ManifestFile manifest : manifestFiles) {
+        pathToManifestFiles.put(manifest.path(), manifest.copy());
+      }
+    }
+    return pathToManifestFiles;
+  }
+
+  // Helper to compute files to delete
+  private Set<String> findFilesToDelete(
+      List<ManifestFile> manifestFilesToDelete, List<ManifestFile> 
currentManifestFiles) {
+    Set<String> filesToDelete = ConcurrentHashMap.newKeySet();
+    Tasks.foreach(manifestFilesToDelete)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .executeWith(planExecutorService)
+        .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, ops.io(), 
ops.current().specsById())) {
+                for (ManifestEntry<?> entry : reader.entries()) {
+                  filesToDelete.add(entry.file().path().toString());
+                }
+              } catch (IOException e) {
+                throw new RuntimeIOException(e, "Failed to read manifest file: 
%s", manifest);
+              }
+            });
+    Tasks.foreach(currentManifestFiles)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .executeWith(planExecutorService)
+        .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, ops.io(), 
ops.current().specsById())) {
+                for (ManifestEntry<?> entry : reader.entries()) {

Review Comment:
   Since the file path is the only field that is used, this should set the 
projection and skip loading the other metadata into memory. This should also 
just iterate over the open reader rather than using entries, since none of the 
entry metadata is used. That will return only live data files.



##########
core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * File cleanup strategy for snapshot expiration which determines, via an 
in-memory reference set,
+ * metadata and data files that are not reachable given the previous and 
current table states.
+ */
+class ReachableFileCleanup extends FileCleanupStrategy {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ReachableFileCleanup.class);
+
+  private final TableMetadata afterExpiration;
+  private final TableMetadata beforeExpiration;
+
+  ReachableFileCleanup(
+      TableOperations ops,
+      TableMetadata beforeExpiration,
+      ExecutorService deleteExecutorService,
+      ExecutorService planExecutorService,
+      Consumer<String> deleteFunc) {
+    super(ops, deleteExecutorService, planExecutorService, deleteFunc);
+    this.beforeExpiration = beforeExpiration;
+    this.afterExpiration = ops.refresh();
+  }
+
+  @Override
+  public void cleanFiles() {
+    // Identify all of the manifest lists to retain
+    Set<String> manifestListsBeforeExpiration =
+        ReachableFileUtil.manifestListLocations(beforeExpiration);
+
+    // Identify all of the manifest lists that currently exist
+    Set<String> manifestListsAfterExpiration =
+        ReachableFileUtil.manifestListLocations(afterExpiration);
+
+    Set<String> manifestListsToDelete =
+        Sets.difference(manifestListsBeforeExpiration, 
manifestListsAfterExpiration);
+    Map<String, ManifestFile> reachableManifestsAfterExpiration =
+        computeManifestsForManifestLists(manifestListsAfterExpiration);
+
+    // Compute the manifests to delete based on the current manifest lists to 
delete and the current
+    // reachable manifests
+    List<ManifestFile> manifestsToDelete =
+        computeManifestsToDelete(manifestListsToDelete, 
reachableManifestsAfterExpiration);
+
+    List<ManifestFile> currentManifests =
+        Lists.newArrayList(reachableManifestsAfterExpiration.values());
+
+    Set<String> filesToDelete = findFilesToDelete(manifestsToDelete, 
currentManifests);
+    Tasks.foreach(filesToDelete)
+        .executeWith(deleteExecutorService)
+        .retry(3)
+        .stopRetryOn(NotFoundException.class)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.warn("Delete failed for data file: {}", 
file, exc))
+        .run(deleteFunc::accept);
+
+    Set<String> manifestPathsToDelete =
+        
manifestsToDelete.stream().map(ManifestFile::path).collect(Collectors.toSet());
+
+    LOG.warn("Manifests to delete: {}", Joiner.on(", 
").join(manifestsToDelete));
+    LOG.warn("Manifests Lists to delete: {}", Joiner.on(", 
").join(manifestListsToDelete));
+    deleteMetadataFiles(
+        manifestPathsToDelete,
+        (path, exc) -> LOG.warn("Delete failed for {}: {}", "manifest", path, 
exc));
+    deleteMetadataFiles(
+        manifestListsToDelete,
+        (path, exc) -> LOG.warn("Delete failed for {}: {}", "manifest list", 
path, exc));
+  }
+
+  private List<ManifestFile> computeManifestsToDelete(
+      Set<String> manifestListsToDelete, Map<String, ManifestFile> 
reachableFilesAfterExpiration) {
+    Map<String, ManifestFile> manifestCandidatesToDelete =
+        computeManifestsForManifestLists(manifestListsToDelete);
+    List<ManifestFile> manifestFilesToDelete = Lists.newArrayList();
+    for (Map.Entry<String, ManifestFile> candidateManifest :
+        manifestCandidatesToDelete.entrySet()) {
+      if 
(!reachableFilesAfterExpiration.containsKey(candidateManifest.getKey())) {
+        manifestFilesToDelete.add(candidateManifest.getValue());
+      }
+    }
+    return manifestFilesToDelete;
+  }
+
+  private Map<String, ManifestFile> 
computeManifestsForManifestLists(Set<String> manifestLists) {
+    Map<String, ManifestFile> pathToManifestFiles = Maps.newHashMap();
+    for (String manifestList : manifestLists) {
+      CloseableIterable<ManifestFile> manifestFiles = 
readManifestFiles(manifestList);
+      for (ManifestFile manifest : manifestFiles) {
+        pathToManifestFiles.put(manifest.path(), manifest.copy());
+      }
+    }
+    return pathToManifestFiles;
+  }
+
+  // Helper to compute files to delete
+  private Set<String> findFilesToDelete(
+      List<ManifestFile> manifestFilesToDelete, List<ManifestFile> 
currentManifestFiles) {
+    Set<String> filesToDelete = ConcurrentHashMap.newKeySet();
+    Tasks.foreach(manifestFilesToDelete)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .executeWith(planExecutorService)
+        .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, ops.io(), 
ops.current().specsById())) {

Review Comment:
   See my comment below about making this read more efficient.



##########
core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * File cleanup strategy for snapshot expiration which determines, via an 
in-memory reference set,
+ * metadata and data files that are not reachable given the previous and 
current table states.
+ */
+class ReachableFileCleanup extends FileCleanupStrategy {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ReachableFileCleanup.class);
+
+  private final TableMetadata afterExpiration;
+  private final TableMetadata beforeExpiration;
+
+  ReachableFileCleanup(
+      TableOperations ops,
+      TableMetadata beforeExpiration,
+      ExecutorService deleteExecutorService,
+      ExecutorService planExecutorService,
+      Consumer<String> deleteFunc) {
+    super(ops, deleteExecutorService, planExecutorService, deleteFunc);
+    this.beforeExpiration = beforeExpiration;
+    this.afterExpiration = ops.refresh();
+  }
+
+  @Override
+  public void cleanFiles() {
+    // Identify all of the manifest lists to retain
+    Set<String> manifestListsBeforeExpiration =
+        ReachableFileUtil.manifestListLocations(beforeExpiration);
+
+    // Identify all of the manifest lists that currently exist
+    Set<String> manifestListsAfterExpiration =
+        ReachableFileUtil.manifestListLocations(afterExpiration);
+
+    Set<String> manifestListsToDelete =
+        Sets.difference(manifestListsBeforeExpiration, 
manifestListsAfterExpiration);
+    Map<String, ManifestFile> reachableManifestsAfterExpiration =
+        computeManifestsForManifestLists(manifestListsAfterExpiration);
+
+    // Compute the manifests to delete based on the current manifest lists to 
delete and the current
+    // reachable manifests
+    List<ManifestFile> manifestsToDelete =
+        computeManifestsToDelete(manifestListsToDelete, 
reachableManifestsAfterExpiration);
+
+    List<ManifestFile> currentManifests =
+        Lists.newArrayList(reachableManifestsAfterExpiration.values());
+
+    Set<String> filesToDelete = findFilesToDelete(manifestsToDelete, 
currentManifests);
+    Tasks.foreach(filesToDelete)
+        .executeWith(deleteExecutorService)
+        .retry(3)
+        .stopRetryOn(NotFoundException.class)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.warn("Delete failed for data file: {}", 
file, exc))
+        .run(deleteFunc::accept);
+
+    Set<String> manifestPathsToDelete =
+        
manifestsToDelete.stream().map(ManifestFile::path).collect(Collectors.toSet());
+
+    LOG.warn("Manifests to delete: {}", Joiner.on(", 
").join(manifestsToDelete));
+    LOG.warn("Manifests Lists to delete: {}", Joiner.on(", 
").join(manifestListsToDelete));
+    deleteMetadataFiles(
+        manifestPathsToDelete,
+        (path, exc) -> LOG.warn("Delete failed for {}: {}", "manifest", path, 
exc));
+    deleteMetadataFiles(
+        manifestListsToDelete,
+        (path, exc) -> LOG.warn("Delete failed for {}: {}", "manifest list", 
path, exc));
+  }
+
+  private List<ManifestFile> computeManifestsToDelete(
+      Set<String> manifestListsToDelete, Map<String, ManifestFile> 
reachableFilesAfterExpiration) {
+    Map<String, ManifestFile> manifestCandidatesToDelete =
+        computeManifestsForManifestLists(manifestListsToDelete);
+    List<ManifestFile> manifestFilesToDelete = Lists.newArrayList();
+    for (Map.Entry<String, ManifestFile> candidateManifest :
+        manifestCandidatesToDelete.entrySet()) {
+      if 
(!reachableFilesAfterExpiration.containsKey(candidateManifest.getKey())) {
+        manifestFilesToDelete.add(candidateManifest.getValue());
+      }
+    }
+    return manifestFilesToDelete;
+  }
+
+  private Map<String, ManifestFile> 
computeManifestsForManifestLists(Set<String> manifestLists) {
+    Map<String, ManifestFile> pathToManifestFiles = Maps.newHashMap();
+    for (String manifestList : manifestLists) {
+      CloseableIterable<ManifestFile> manifestFiles = 
readManifestFiles(manifestList);
+      for (ManifestFile manifest : manifestFiles) {
+        pathToManifestFiles.put(manifest.path(), manifest.copy());
+      }
+    }
+    return pathToManifestFiles;
+  }
+
+  // Helper to compute files to delete
+  private Set<String> findFilesToDelete(
+      List<ManifestFile> manifestFilesToDelete, List<ManifestFile> 
currentManifestFiles) {
+    Set<String> filesToDelete = ConcurrentHashMap.newKeySet();
+    Tasks.foreach(manifestFilesToDelete)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .executeWith(planExecutorService)
+        .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, ops.io(), 
ops.current().specsById())) {
+                for (ManifestEntry<?> entry : reader.entries()) {
+                  filesToDelete.add(entry.file().path().toString());
+                }
+              } catch (IOException e) {
+                throw new RuntimeIOException(e, "Failed to read manifest file: 
%s", manifest);
+              }
+            });
+    Tasks.foreach(currentManifestFiles)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .executeWith(planExecutorService)
+        .onFailure(
+            (item, exc) ->
+                LOG.warn("Failed to get deleted files: this may cause orphaned 
data files", exc))
+        .run(
+            manifest -> {

Review Comment:
   I think each run should check to see if there are still files to delete and 
skip reading the manifest if there are none.



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to