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


##########
core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.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.RuntimeIOException;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+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);
+
+  ReachableFileCleanup(
+      FileIO fileIO,
+      ExecutorService deleteExecutorService,
+      ExecutorService planExecutorService,
+      Consumer<String> deleteFunc) {
+    super(fileIO, deleteExecutorService, planExecutorService, deleteFunc);
+  }
+
+  @Override
+  public void cleanFiles(TableMetadata beforeExpiration, TableMetadata 
afterExpiration) {
+    Set<String> manifestListsToDelete = Sets.newHashSet();
+
+    Set<Snapshot> snapshotsBeforeExpiration = 
Sets.newHashSet(beforeExpiration.snapshots());
+    Set<Snapshot> snapshotsAfterExpiration = 
Sets.newHashSet(afterExpiration.snapshots());
+    Set<Snapshot> expiredSnapshots = Sets.newHashSet();
+    for (Snapshot snapshot : snapshotsBeforeExpiration) {
+      if (!snapshotsAfterExpiration.contains(snapshot)) {
+        expiredSnapshots.add(snapshot);
+        if (snapshot.manifestListLocation() != null) {
+          manifestListsToDelete.add(snapshot.manifestListLocation());
+        }
+      }
+    }
+
+    Set<ManifestFile> candidateManifestFilesForDeletion = 
readManifests(expiredSnapshots);
+    Set<ManifestFile> manifestFilesAfterExpiration = 
readManifests(snapshotsAfterExpiration);
+
+    Set<ManifestFile> manifestsToDelete = Sets.newHashSet();
+    for (ManifestFile candidateManifestFile : 
candidateManifestFilesForDeletion) {
+      if (!manifestFilesAfterExpiration.contains(candidateManifestFile)) {
+        manifestsToDelete.add(candidateManifestFile);
+      }
+    }
+
+    Set<String> dataFilesToDelete =
+        findFilesToDelete(manifestsToDelete, manifestFilesAfterExpiration);
+    deleteFiles(dataFilesToDelete, "data");
+    Set<String> manifestPathsToDelete =
+        
manifestsToDelete.stream().map(ManifestFile::path).collect(Collectors.toSet());
+
+    deleteFiles(manifestPathsToDelete, "manifest");
+    deleteFiles(manifestListsToDelete, "manifest list");
+  }
+
+  private Set<ManifestFile> readManifests(Set<Snapshot> snapshots) {
+    Set<ManifestFile> manifestFiles = Sets.newHashSet();
+    for (Snapshot snapshot : snapshots) {
+      try (CloseableIterable<ManifestFile> manifestFilesForSnapshot = 
readManifestFiles(snapshot)) {
+        for (ManifestFile manifestFile : manifestFilesForSnapshot) {
+          manifestFiles.add(manifestFile.copy());
+        }
+      } catch (IOException e) {
+        throw new RuntimeIOException(
+            e, "Failed to close manifest list: %s", 
snapshot.manifestListLocation());
+      }
+    }
+
+    return manifestFiles;
+  }
+
+  // Helper to determine data files to delete
+  private Set<String> findFilesToDelete(
+      Set<ManifestFile> manifestFilesToDelete, Set<ManifestFile> 
currentManifestFiles) {
+    Set<String> filesToDelete = ConcurrentHashMap.newKeySet();
+
+    Tasks.foreach(manifestFilesToDelete)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .executeWith(planExecutorService)
+        .onFailure(
+            (item, exc) ->
+                LOG.warn(
+                    "Failed to determine live files in manifest {}: this may 
cause orphaned data files",
+                    item.path(),
+                    exc))
+        .run(
+            manifest -> {
+              try (CloseableIterable<String> paths = 
ManifestFiles.readPaths(manifest, fileIO)) {
+                paths.forEach(filesToDelete::add);
+              } catch (IOException e) {
+                throw new RuntimeIOException(e, "Failed to read manifest file: 
%s", manifest);
+              }
+            });
+
+    if (filesToDelete.isEmpty()) {
+      return filesToDelete;
+    }
+
+    try {
+      Tasks.foreach(currentManifestFiles)
+          .retry(3)
+          .stopOnFailure()
+          .throwFailureWhenFinished()
+          .executeWith(planExecutorService)
+          .onFailure(
+              (item, exc) ->
+                  LOG.warn(
+                      "Failed to determine live files in manifest {}: this may 
cause orphaned data files",
+                      item.path(),
+                      exc))
+          .run(
+              manifest -> {
+                if (filesToDelete.isEmpty()) {
+                  return;
+                }
+
+                // Remove all the live files from the candidate deletion set
+                try (CloseableIterable<String> paths = 
ManifestFiles.readPaths(manifest, fileIO)) {
+                  paths.forEach(filesToDelete::remove);
+                } catch (IOException e) {
+                  throw new RuntimeIOException(e, "Failed to read manifest 
file: %s", manifest);
+                }
+              });
+
+    } catch (Throwable e) {
+      LOG.warn("Failed to determine the data files to be removed", e);

Review Comment:
   As a follow up, I think this message could be a little better. This makes 
sense if you know what the code is currently doing, but if you don't then I 
think it will be hard to understand that the error was in reading current 
manifests, rather than dropped manifests. I'd use something like "Failed to 
list all reachable files".



-- 
This is an automated message from the 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